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


##########
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:
   high-cardinality is out of the scope of initial impl. But I see @pvary 's 
point regarding compatibility for future support. For that, we may want to 
introduce a `TrafficStatistics` interface. Currently, we can provide a 
`SimpleCountStatistics` impl.  We can implement a 
`TrafficStatisticsSerializer`. The serializer can be expanded in the future to 
handle high-cardinality statistics like `TDigestStatistics` e.g.. 



-- 
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