yegangy0718 commented on code in PR #6382:
URL: https://github.com/apache/iceberg/pull/6382#discussion_r1061820186


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/ShuffleOperator.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink.shuffle;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.MapTypeInfo;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+/**
+ * Shuffle operator can help to improve data clustering based on the key.
+ *
+ * <p>It collects the data statistics information, sends to coordinator and 
gets the global data
+ * distribution weight from coordinator. Then it will ingest the weight into 
data stream(wrap by a
+ * class{@link ShuffleRecordWrapper}) and send to partitioner.
+ */
+@Internal
+public class ShuffleOperator<T, K extends Serializable>
+    extends AbstractStreamOperator<ShuffleRecordWrapper<T, K>>
+    implements OneInputStreamOperator<T, ShuffleRecordWrapper<T, K>>, 
OperatorEventHandler {
+
+  private static final long serialVersionUID = 1L;
+
+  private final KeySelector<T, K> keySelector;
+  // the type of the key to collect data statistics
+  private final TypeInformation<K> keyType;
+  private final OperatorEventGateway operatorEventGateway;
+  // key is generated by applying KeySelector to record
+  // value is the times key occurs
+  private transient Map<K, Long> localDataStatisticsMap;

Review Comment:
   It sounds good to me to add `DataStatistics` interface. 
   For the usage of `DataStatisticsFactory `, I think it will take table/mode 
configuration to decide which `DataStatistics` to generate. For now,  function 
`newDataStatistic/createDataStatistic` will always return `MapDataStatistic` 
since this is the only supported type. 
   I will make the modification in the latest commit. 



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestShuffleOperator.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.flink.sink.shuffle;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.OperatorStateStore;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.runtime.execution.Environment;
+import 
org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway;
+import org.apache.flink.runtime.operators.testutils.MockEnvironment;
+import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
+import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateInitializationContextImpl;
+import org.apache.flink.runtime.state.TestTaskStateManager;
+import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment;
+import org.apache.flink.streaming.util.MockOutput;
+import org.apache.flink.streaming.util.MockStreamConfig;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestShuffleOperator {
+  private ShuffleOperator<String, String> operator;
+
+  private Environment getTestingEnvironment() {
+    return new StreamMockEnvironment(
+        new Configuration(),
+        new Configuration(),
+        new ExecutionConfig(),
+        1L,
+        new MockInputSplitProvider(),
+        1,
+        new TestTaskStateManager());
+  }
+
+  @Before
+  public void before() throws Exception {
+    MockOperatorEventGateway mockGateway = new MockOperatorEventGateway();
+    KeySelector<String, String> keySelector =
+        new KeySelector<String, String>() {
+          private static final long serialVersionUID = 7662520075515707428L;
+
+          @Override
+          public String getKey(String value) {
+            return value;
+          }
+        };
+
+    this.operator =
+        new ShuffleOperator<>(keySelector, TypeInformation.of(String.class), 
mockGateway);
+    Environment env = getTestingEnvironment();
+    this.operator.setup(
+        new OneInputStreamTask<String, String>(env),
+        new MockStreamConfig(new Configuration(), 1),
+        new MockOutput<>(Lists.newArrayList()));
+  }
+
+  @After
+  public void clean() throws Exception {
+    operator.close();
+  }
+
+  @Test
+  public void testInitializeState() throws Exception {

Review Comment:
   The logic to send data statistics from operator to coordinator and data 
statistics aggregation function in coordinator are not defined in the PR. The 
global state is empty. 
   I will add the rebalance case in the next PR once the above parts are there. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to