You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/12/12 08:09:29 UTC

[GitHub] [iceberg] yegangy0718 commented on a diff in pull request #6382: Implement ShuffleOperator to collect data statistics

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


##########
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;
+  private transient Map<K, Long> globalDataDistributionWeightMap;
+  private transient ListState<Map<K, Long>> globalDataDistributionWeightState;
+
+  public ShuffleOperator(
+      KeySelector<T, K> keySelector,
+      TypeInformation<K> keyType,

Review Comment:
   I see what you're saying. Even though `K` is not serializable itself, user can still use `TypeInformation` to define `TypeSerializer`. 
   I will remove the `Serializable` limitation for K in latest commit. 



##########
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;
+  private transient Map<K, Long> globalDataDistributionWeightMap;
+  private transient ListState<Map<K, Long>> globalDataDistributionWeightState;
+
+  public ShuffleOperator(
+      KeySelector<T, K> keySelector,
+      TypeInformation<K> keyType,
+      OperatorEventGateway operatorEventGateway) {
+    this.keySelector = keySelector;
+    this.keyType = keyType;
+    this.operatorEventGateway = operatorEventGateway;
+  }
+
+  @VisibleForTesting
+  ListStateDescriptor<Map<K, Long>> generateGlobalDataDistributionWeightDescriptor() {
+    return new ListStateDescriptor<>(
+        "globalDataDistributionWeight", new MapTypeInfo<>(keyType, TypeInformation.of(Long.class)));
+  }
+
+  @Override
+  public void initializeState(StateInitializationContext context) throws Exception {
+    localDataStatisticsMap = Maps.newHashMap();
+    globalDataDistributionWeightState =

Review Comment:
   OK. Let me take some time to investigate if `BroadcastState` can be applied to this case. 



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/ShuffleRecordWrapper.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * The wrapper class for record and data distribution weight
+ *
+ * <p>ShuffleRecordWrapper is sent from ShuffleOperator to partitioner. It may contain a record or
+ * data distribution weight. Once partitioner receives the weight, it will use that to decide the
+ * coming record should send to which writer subtask. After shuffling, a filter and mapper are
+ * required to filter out the data distribution weight, unwrap the object and extract the original
+ * record type T.
+ */
+@Internal
+public class ShuffleRecordWrapper<T, K extends Serializable> implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  private final Map<K, Long> globalDataDistributionWeight;
+  private final T record;
+
+  private ShuffleRecordWrapper(T record, Map<K, Long> globalDataDistributionWeight) {
+    Preconditions.checkArgument(
+        record != null ^ globalDataDistributionWeight != null,
+        "A ShuffleRecordWrapper has to contain exactly one of record and stats, not neither or both");

Review Comment:
   will update in next version. 



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/ShuffleRecordWrapper.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * The wrapper class for record and data distribution weight
+ *
+ * <p>ShuffleRecordWrapper is sent from ShuffleOperator to partitioner. It may contain a record or
+ * data distribution weight. Once partitioner receives the weight, it will use that to decide the
+ * coming record should send to which writer subtask. After shuffling, a filter and mapper are
+ * required to filter out the data distribution weight, unwrap the object and extract the original
+ * record type T.
+ */
+@Internal
+public class ShuffleRecordWrapper<T, K extends Serializable> implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  private final Map<K, Long> globalDataDistributionWeight;
+  private final T record;
+
+  private ShuffleRecordWrapper(T record, Map<K, Long> globalDataDistributionWeight) {
+    Preconditions.checkArgument(
+        record != null ^ globalDataDistributionWeight != null,
+        "A ShuffleRecordWrapper has to contain exactly one of record and stats, not neither or both");
+    this.globalDataDistributionWeight = globalDataDistributionWeight;
+    this.record = record;
+  }
+
+  static <T, K extends Serializable> ShuffleRecordWrapper<T, K> fromRecord(T record) {
+    return new ShuffleRecordWrapper<>(record, null);
+  }
+
+  static <T, K extends Serializable> ShuffleRecordWrapper<T, K> fromDistribution(
+      Map<K, Long> globalDataDistributionWeight) {
+    return new ShuffleRecordWrapper<>(null, globalDataDistributionWeight);
+  }
+
+  boolean hasGlobalDataDistributionWeight() {
+    return globalDataDistributionWeight != null;
+  }
+
+  boolean hasRecord() {
+    return record != null;
+  }
+
+  Map<K, Long> globalDataDistributionWeight() {
+    return globalDataDistributionWeight;
+  }
+
+  T record() {
+    return record;
+  }
+
+  @Override
+  public String toString() {
+    return String.format(

Review Comment:
   Oh, yeah, other Iceberg class use guava. to generate String. I will update in next version



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/ShuffleRecordWrapper.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * The wrapper class for record and data distribution weight
+ *
+ * <p>ShuffleRecordWrapper is sent from ShuffleOperator to partitioner. It may contain a record or
+ * data distribution weight. Once partitioner receives the weight, it will use that to decide the
+ * coming record should send to which writer subtask. After shuffling, a filter and mapper are
+ * required to filter out the data distribution weight, unwrap the object and extract the original
+ * record type T.
+ */
+@Internal
+public class ShuffleRecordWrapper<T, K extends Serializable> implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  private final Map<K, Long> globalDataDistributionWeight;
+  private final T record;
+
+  private ShuffleRecordWrapper(T record, Map<K, Long> globalDataDistributionWeight) {
+    Preconditions.checkArgument(
+        record != null ^ globalDataDistributionWeight != null,
+        "A ShuffleRecordWrapper has to contain exactly one of record and stats, not neither or both");
+    this.globalDataDistributionWeight = globalDataDistributionWeight;
+    this.record = record;
+  }
+
+  static <T, K extends Serializable> ShuffleRecordWrapper<T, K> fromRecord(T record) {
+    return new ShuffleRecordWrapper<>(record, null);
+  }
+
+  static <T, K extends Serializable> ShuffleRecordWrapper<T, K> fromDistribution(

Review Comment:
   will update in next version.



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

Review Comment:
   if we don't implement `OneInputStreamOperator` and only implement `Input<IN>`, then we need to implement function `setKeyContextElement`, but `OneInputStreamOperator` provides the default implementation.



-- 
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-unsubscribe@iceberg.apache.org

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


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