You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "stevenzwu (via GitHub)" <gi...@apache.org> on 2023/05/01 20:46:40 UTC

[GitHub] [iceberg] stevenzwu opened a new pull request, #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

stevenzwu opened a new pull request, #7494:
URL: https://github.com/apache/iceberg/pull/7494

   , because FlinkSink normalize the data type to RowData before coming to writer. Also added custom type serializers for MapDataStatistics and /DataStatisticsOrRecord.


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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1184504748


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java:
##########
@@ -40,50 +40,49 @@
  * shuffle record to improve data clustering while maintaining relative balanced traffic
  * distribution to downstream subtasks.
  */
-class DataStatisticsOperator<T, K> extends AbstractStreamOperator<DataStatisticsOrRecord<T, K>>
-    implements OneInputStreamOperator<T, DataStatisticsOrRecord<T, K>>, OperatorEventHandler {
+class DataStatisticsOperator<D extends DataStatistics<D, S>, S>

Review Comment:
   Yes, this is only for collecting statistics to guide the partitioner decision. we will implement a custom range partitioner (Flink )  that splits the value into ranges (one for each assigner writer subtask) based on the statistics.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1191869158


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.IOException;
+import java.util.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.table.data.RowData;
+
+@Internal
+class DataStatisticsOrRecordSerializer<D extends DataStatistics<D, S>, S>
+    extends TypeSerializer<DataStatisticsOrRecord<D, S>> {
+  private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer;
+  private final TypeSerializer<RowData> recordSerializer;
+
+  DataStatisticsOrRecordSerializer(
+      TypeSerializer<DataStatistics<D, S>> statisticsSerializer,
+      TypeSerializer<RowData> recordSerializer) {
+    this.statisticsSerializer = statisticsSerializer;
+    this.recordSerializer = recordSerializer;
+  }
+
+  @Override
+  public boolean isImmutableType() {
+    return false;
+  }
+
+  @SuppressWarnings("ReferenceEquality")
+  @Override
+  public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() {
+    TypeSerializer<DataStatistics<D, S>> duplicateStatisticsSerializer =
+        statisticsSerializer.duplicate();
+    TypeSerializer<RowData> duplicateRowDataSerializer = recordSerializer.duplicate();
+    if ((statisticsSerializer != duplicateStatisticsSerializer)
+        || (recordSerializer != duplicateRowDataSerializer)) {
+      return new DataStatisticsOrRecordSerializer<>(
+          duplicateStatisticsSerializer, duplicateRowDataSerializer);
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> createInstance() {
+    // arbitrarily always create RowData value instance
+    return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance());
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(DataStatisticsOrRecord<D, S> from) {
+    if (from.hasRecord()) {
+      return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record()));
+    } else {
+      return DataStatisticsOrRecord.fromDataStatistics(
+          statisticsSerializer.copy(from.dataStatistics()));
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(
+      DataStatisticsOrRecord<D, S> from, DataStatisticsOrRecord<D, S> reuse) {
+    DataStatisticsOrRecord<D, S> to;
+    if (from.hasRecord()) {
+      to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer);
+      RowData record = recordSerializer.copy(from.record(), to.record());
+      to.record(record);
+    } else {
+      to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer);
+      DataStatistics<D, S> statistics =
+          statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics());
+      to.dataStatistics(statistics);
+    }
+
+    return to;
+  }
+
+  @Override
+  public int getLength() {
+    return -1;

Review Comment:
   @hililiwei not sure if I understand the question here. can you elaborate?
   
   here is the javadoc for this method from the Flink `TypeSerializer` interface.
   ```
   Returns:
   The length of the data type, or -1 for variable length data types.
   ```



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


[GitHub] [iceberg] stevenzwu commented on pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#issuecomment-1538652434

   > @stevenzwu: I like that with a correct reader we can make sure that the `IcebergSource` could read whatever data structure we want. It would be good to have the possibility for the same thing for the `FlinkSink` (or IcebergSink later). Wouldn't this PR close this possibility by forcing the Sink to use `RowData`?
   
   @pvary `FlinkSink` can still support other input types (like Avro GenericRecord). The behavior (internally) is that all input type is converted to Flink `RowData`, which is the only supported record format for the underline file writer in Iceberg. 
   
   That is the reason why @huyuanfeng2018 was suggesting if `RowData` is[ the only key type we need to support](https://github.com/apache/iceberg/issues/7393). If in the future, Flink sink supports other record format for the underline file writer, maybe we can revisit this decision. for now, `RowData` is the only type Flink file writer supports.


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


[GitHub] [iceberg] stevenzwu merged pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu merged PR #7494:
URL: https://github.com/apache/iceberg/pull/7494


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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1184503212


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java:
##########
@@ -42,12 +43,19 @@
    *
    * @param key generate from data by applying key selector
    */
-  void add(K key);
+  void add(RowData key);

Review Comment:
   we are only counting records per key. To get the bytes, it would require serialization or some other trick of estimation. Agree bytes would be the best. but record count is probably also good enough.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1191663833


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java:
##########
@@ -132,19 +130,19 @@ public void snapshotState(StateSnapshotContext context) throws Exception {
 
     // For now, we make it simple to send globalStatisticsState at checkpoint
     operatorEventGateway.sendEventToCoordinator(
-        new DataStatisticsEvent<>(checkpointId, localStatistics));
+        new DataStatisticsEvent(checkpointId, localStatistics));
 
     // Recreate the local statistics
-    localStatistics = statisticsFactory.createDataStatistics();
+    localStatistics = statisticsSerializer.createInstance();
   }
 
   @VisibleForTesting
-  DataStatistics<K> localDataStatistics() {
+  DataStatistics localDataStatistics() {
     return localStatistics;
   }
 
   @VisibleForTesting
-  DataStatistics<K> globalDataStatistics() {
+  DataStatistics globalDataStatistics() {

Review Comment:
   good batch. will fix.



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


[GitHub] [iceberg] stevenzwu commented on pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#issuecomment-1546444317

   > 
   
   
   
   > One naive question, how do we plan to register the DataStatisticsOrRecordSerializer? Do we plan to define TypeInfoFactory or use env config to register it?
   
   @yegangy0718 good question. we probably also need to implement a TypeInfo class when adding the DataStatisticsOperator. that way, Flink would know which type serializer to use.
   
   Here is the DataStream API.
   ```
       /**
        * Method for passing user defined operators along with the type information that will transform
        * the DataStream.
        *
        * @param operatorName name of the operator, for logging purposes
        * @param outTypeInfo the output type of the operator
        * @param operator the object containing the transformation logic
        * @param <R> type of the return stream
        * @return the data stream constructed
        * @see #transform(String, TypeInformation, OneInputStreamOperatorFactory)
        */
       @PublicEvolving
       public <R> SingleOutputStreamOperator<R> transform(
               String operatorName,
               TypeInformation<R> outTypeInfo,
               OneInputStreamOperator<T, R> operator) {
   
           return doTransform(operatorName, outTypeInfo, SimpleOperatorFactory.of(operator));
       }
   ```


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


[GitHub] [iceberg] pvary commented on pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#issuecomment-1537965740

   @stevenzwu: I like that with a correct reader we can make sure that the `IcebergSource` could read whatever data structure we want. It would be good to have the possibility for the same thing for the `FlinkSink` (or IcebergSink later). Wouldn't this PR close this possibility by forcing the Sink to use `RowData`?


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


[GitHub] [iceberg] hililiwei commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "hililiwei (via GitHub)" <gi...@apache.org>.
hililiwei commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1188185907


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.IOException;
+import java.util.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.table.data.RowData;
+
+@Internal
+class DataStatisticsOrRecordSerializer<D extends DataStatistics<D, S>, S>
+    extends TypeSerializer<DataStatisticsOrRecord<D, S>> {
+  private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer;
+  private final TypeSerializer<RowData> recordSerializer;
+
+  DataStatisticsOrRecordSerializer(
+      TypeSerializer<DataStatistics<D, S>> statisticsSerializer,
+      TypeSerializer<RowData> recordSerializer) {
+    this.statisticsSerializer = statisticsSerializer;
+    this.recordSerializer = recordSerializer;
+  }
+
+  @Override
+  public boolean isImmutableType() {
+    return false;
+  }
+
+  @SuppressWarnings("ReferenceEquality")
+  @Override
+  public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() {
+    TypeSerializer<DataStatistics<D, S>> duplicateStatisticsSerializer =
+        statisticsSerializer.duplicate();
+    TypeSerializer<RowData> duplicateRowDataSerializer = recordSerializer.duplicate();
+    if ((statisticsSerializer != duplicateStatisticsSerializer)
+        || (recordSerializer != duplicateRowDataSerializer)) {
+      return new DataStatisticsOrRecordSerializer<>(
+          duplicateStatisticsSerializer, duplicateRowDataSerializer);
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> createInstance() {
+    // arbitrarily always create RowData value instance
+    return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance());
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(DataStatisticsOrRecord<D, S> from) {
+    if (from.hasRecord()) {
+      return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record()));
+    } else {
+      return DataStatisticsOrRecord.fromDataStatistics(
+          statisticsSerializer.copy(from.dataStatistics()));
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(
+      DataStatisticsOrRecord<D, S> from, DataStatisticsOrRecord<D, S> reuse) {
+    DataStatisticsOrRecord<D, S> to;
+    if (from.hasRecord()) {
+      to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer);
+      RowData record = recordSerializer.copy(from.record(), to.record());
+      to.record(record);
+    } else {
+      to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer);
+      DataStatistics<D, S> statistics =
+          statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics());
+      to.dataStatistics(statistics);
+    }
+
+    return to;
+  }
+
+  @Override
+  public int getLength() {
+    return -1;

Review Comment:
   We only care about the amount of data, not the size of data, right?
   
   



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1181969400


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java:
##########
@@ -28,10 +29,10 @@
  * (sketching) can be used.
  */
 @Internal
-interface DataStatistics<K> {
+interface DataStatistics<D extends DataStatistics, S> {

Review Comment:
   used generic trick for the strong type check. it shouldn't matter to users since all these are internal classes.



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsFactory.java:
##########


Review Comment:
   Factory is replaced by TypeSerializer



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.IOException;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.common.typeutils.base.MapSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+@Internal
+class MapDataStatisticsSerializer
+    extends TypeSerializer<DataStatistics<MapDataStatistics, Map<RowData, Long>>> {

Review Comment:
   use the base interface `DataStatistics` so that it can be used by `DataStatisticsOrRecordSerializer`.
   
   ```
   DataStatisticsOrRecordSerializer(
         TypeSerializer<DataStatistics<D, S>> statisticsSerializer,
         TypeSerializer<RowData> recordSerializer)
   ```



##########
flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java:
##########
@@ -99,99 +113,104 @@ public void clean() throws Exception {
 
   @Test
   public void testProcessElement() throws Exception {
-    StateInitializationContext stateContext = getStateContext();
-    operator.initializeState(stateContext);
-    operator.processElement(new StreamRecord<>("a"));
-    operator.processElement(new StreamRecord<>("a"));
-    operator.processElement(new StreamRecord<>("b"));
-    assertTrue(operator.localDataStatistics() instanceof MapDataStatistics);
-    MapDataStatistics<String> mapDataStatistics =
-        (MapDataStatistics<String>) operator.localDataStatistics();
-    assertTrue(mapDataStatistics.dataStatistics().containsKey("a"));
-    assertTrue(mapDataStatistics.dataStatistics().containsKey("b"));
-    assertEquals(2L, (long) mapDataStatistics.dataStatistics().get("a"));
-    assertEquals(1L, (long) mapDataStatistics.dataStatistics().get("b"));
+    try (OneInputStreamOperatorTestHarness<

Review Comment:
   need to wrap the block in try with test harness. otherwise, Flink don't know how to serialize the output type of`DataStatisticsOrRecord <>`. test harness has the proper `setup(...)` on output type serializer



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java:
##########
@@ -42,12 +43,19 @@
    *
    * @param key generate from data by applying key selector
    */
-  void add(K key);
+  void add(RowData key);

Review Comment:
   we will use `RowDataProjection` to extract the key. PR #7493 is related.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1191665197


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.IOException;
+import java.util.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.table.data.RowData;
+
+@Internal
+class DataStatisticsOrRecordSerializer<D extends DataStatistics<D, S>, S>
+    extends TypeSerializer<DataStatisticsOrRecord<D, S>> {
+  private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer;
+  private final TypeSerializer<RowData> recordSerializer;
+
+  DataStatisticsOrRecordSerializer(
+      TypeSerializer<DataStatistics<D, S>> statisticsSerializer,
+      TypeSerializer<RowData> recordSerializer) {
+    this.statisticsSerializer = statisticsSerializer;
+    this.recordSerializer = recordSerializer;
+  }
+
+  @Override
+  public boolean isImmutableType() {
+    return false;
+  }
+
+  @SuppressWarnings("ReferenceEquality")
+  @Override
+  public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() {
+    TypeSerializer<DataStatistics<D, S>> duplicateStatisticsSerializer =
+        statisticsSerializer.duplicate();
+    TypeSerializer<RowData> duplicateRowDataSerializer = recordSerializer.duplicate();
+    if ((statisticsSerializer != duplicateStatisticsSerializer)
+        || (recordSerializer != duplicateRowDataSerializer)) {
+      return new DataStatisticsOrRecordSerializer<>(
+          duplicateStatisticsSerializer, duplicateRowDataSerializer);
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> createInstance() {
+    // arbitrarily always create RowData value instance
+    return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance());
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(DataStatisticsOrRecord<D, S> from) {
+    if (from.hasRecord()) {
+      return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record()));
+    } else {
+      return DataStatisticsOrRecord.fromDataStatistics(
+          statisticsSerializer.copy(from.dataStatistics()));
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(
+      DataStatisticsOrRecord<D, S> from, DataStatisticsOrRecord<D, S> reuse) {
+    DataStatisticsOrRecord<D, S> to;
+    if (from.hasRecord()) {
+      to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer);
+      RowData record = recordSerializer.copy(from.record(), to.record());
+      to.record(record);
+    } else {
+      to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer);
+      DataStatistics<D, S> statistics =
+          statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics());
+      to.dataStatistics(statistics);
+    }
+
+    return to;
+  }
+
+  @Override
+  public int getLength() {
+    return -1;

Review Comment:
   @hililiwei not sure if I understand the question here. can you elaborate?
   
   here is the javadoc for this method from the Flink `TypeSerializer` interface.
   ```
   Returns:
   The length of the data type, or -1 for variable length data types.
   ```



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


[GitHub] [iceberg] stevenzwu commented on pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#issuecomment-1530355108

   address some of the comments from issue #7393. 
   
   @huyuanfeng2018 @hililiwei @yegangy0718 can you help review?
   
   @yegangy0718 will follow up with a separate PR on jmh benchmark.


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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1191664997


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.IOException;
+import java.util.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.table.data.RowData;
+
+@Internal
+class DataStatisticsOrRecordSerializer<D extends DataStatistics<D, S>, S>
+    extends TypeSerializer<DataStatisticsOrRecord<D, S>> {
+  private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer;
+  private final TypeSerializer<RowData> recordSerializer;
+
+  DataStatisticsOrRecordSerializer(
+      TypeSerializer<DataStatistics<D, S>> statisticsSerializer,
+      TypeSerializer<RowData> recordSerializer) {
+    this.statisticsSerializer = statisticsSerializer;
+    this.recordSerializer = recordSerializer;
+  }
+
+  @Override
+  public boolean isImmutableType() {
+    return false;
+  }
+
+  @SuppressWarnings("ReferenceEquality")
+  @Override
+  public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() {
+    TypeSerializer<DataStatistics<D, S>> duplicateStatisticsSerializer =
+        statisticsSerializer.duplicate();
+    TypeSerializer<RowData> duplicateRowDataSerializer = recordSerializer.duplicate();
+    if ((statisticsSerializer != duplicateStatisticsSerializer)
+        || (recordSerializer != duplicateRowDataSerializer)) {
+      return new DataStatisticsOrRecordSerializer<>(
+          duplicateStatisticsSerializer, duplicateRowDataSerializer);
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> createInstance() {
+    // arbitrarily always create RowData value instance
+    return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance());
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(DataStatisticsOrRecord<D, S> from) {
+    if (from.hasRecord()) {
+      return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record()));
+    } else {
+      return DataStatisticsOrRecord.fromDataStatistics(
+          statisticsSerializer.copy(from.dataStatistics()));
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(
+      DataStatisticsOrRecord<D, S> from, DataStatisticsOrRecord<D, S> reuse) {
+    DataStatisticsOrRecord<D, S> to;
+    if (from.hasRecord()) {
+      to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer);
+      RowData record = recordSerializer.copy(from.record(), to.record());
+      to.record(record);
+    } else {
+      to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer);
+      DataStatistics<D, S> statistics =
+          statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics());
+      to.dataStatistics(statistics);
+    }
+
+    return to;
+  }
+
+  @Override
+  public int getLength() {
+    return -1;

Review Comment:
   @hililiwei not sure if I understand the question here. can you elaborate?
   
   here is the javadoc for this method from the TypeSerializer interface.
   ```
   Returns:
   The length of the data type, or -1 for variable length data types.
   ```



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


[GitHub] [iceberg] pvary commented on pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "pvary (via GitHub)" <gi...@apache.org>.
pvary commented on PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#issuecomment-1540192274

   
   > @pvary `FlinkSink` can still support other input types (like Avro GenericRecord). The behavior (internally) is that all input type is converted to Flink `RowData`, which is the only supported record format for the underline file writer in Iceberg.
   > 
   > That is the reason why @huyuanfeng2018 was suggesting if `RowData` is[ the only key type we need to support](https://github.com/apache/iceberg/issues/7393). If in the future, Flink sink supports other record format for the underline file writer, maybe we can revisit this decision. for now, `RowData` is the only type Flink file writer supports.
   
   Thanks @stevenzwu for the answer!


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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1191869158


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.IOException;
+import java.util.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.table.data.RowData;
+
+@Internal
+class DataStatisticsOrRecordSerializer<D extends DataStatistics<D, S>, S>
+    extends TypeSerializer<DataStatisticsOrRecord<D, S>> {
+  private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer;
+  private final TypeSerializer<RowData> recordSerializer;
+
+  DataStatisticsOrRecordSerializer(
+      TypeSerializer<DataStatistics<D, S>> statisticsSerializer,
+      TypeSerializer<RowData> recordSerializer) {
+    this.statisticsSerializer = statisticsSerializer;
+    this.recordSerializer = recordSerializer;
+  }
+
+  @Override
+  public boolean isImmutableType() {
+    return false;
+  }
+
+  @SuppressWarnings("ReferenceEquality")
+  @Override
+  public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() {
+    TypeSerializer<DataStatistics<D, S>> duplicateStatisticsSerializer =
+        statisticsSerializer.duplicate();
+    TypeSerializer<RowData> duplicateRowDataSerializer = recordSerializer.duplicate();
+    if ((statisticsSerializer != duplicateStatisticsSerializer)
+        || (recordSerializer != duplicateRowDataSerializer)) {
+      return new DataStatisticsOrRecordSerializer<>(
+          duplicateStatisticsSerializer, duplicateRowDataSerializer);
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> createInstance() {
+    // arbitrarily always create RowData value instance
+    return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance());
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(DataStatisticsOrRecord<D, S> from) {
+    if (from.hasRecord()) {
+      return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record()));
+    } else {
+      return DataStatisticsOrRecord.fromDataStatistics(
+          statisticsSerializer.copy(from.dataStatistics()));
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(
+      DataStatisticsOrRecord<D, S> from, DataStatisticsOrRecord<D, S> reuse) {
+    DataStatisticsOrRecord<D, S> to;
+    if (from.hasRecord()) {
+      to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer);
+      RowData record = recordSerializer.copy(from.record(), to.record());
+      to.record(record);
+    } else {
+      to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer);
+      DataStatistics<D, S> statistics =
+          statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics());
+      to.dataStatistics(statistics);
+    }
+
+    return to;
+  }
+
+  @Override
+  public int getLength() {
+    return -1;

Review Comment:
   @hililiwei not sure if I understand the question here. can you elaborate?
   
   here is the javadoc for this method from the Flink `TypeSerializer` interface.
   ```
   Returns:
   The length of the data type, or -1 for variable length data types.
   ```



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.IOException;
+import java.util.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.table.data.RowData;
+
+@Internal
+class DataStatisticsOrRecordSerializer<D extends DataStatistics<D, S>, S>
+    extends TypeSerializer<DataStatisticsOrRecord<D, S>> {
+  private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer;
+  private final TypeSerializer<RowData> recordSerializer;
+
+  DataStatisticsOrRecordSerializer(
+      TypeSerializer<DataStatistics<D, S>> statisticsSerializer,
+      TypeSerializer<RowData> recordSerializer) {
+    this.statisticsSerializer = statisticsSerializer;
+    this.recordSerializer = recordSerializer;
+  }
+
+  @Override
+  public boolean isImmutableType() {
+    return false;
+  }
+
+  @SuppressWarnings("ReferenceEquality")
+  @Override
+  public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() {
+    TypeSerializer<DataStatistics<D, S>> duplicateStatisticsSerializer =
+        statisticsSerializer.duplicate();
+    TypeSerializer<RowData> duplicateRowDataSerializer = recordSerializer.duplicate();
+    if ((statisticsSerializer != duplicateStatisticsSerializer)
+        || (recordSerializer != duplicateRowDataSerializer)) {
+      return new DataStatisticsOrRecordSerializer<>(
+          duplicateStatisticsSerializer, duplicateRowDataSerializer);
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> createInstance() {
+    // arbitrarily always create RowData value instance
+    return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance());
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(DataStatisticsOrRecord<D, S> from) {
+    if (from.hasRecord()) {
+      return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record()));
+    } else {
+      return DataStatisticsOrRecord.fromDataStatistics(
+          statisticsSerializer.copy(from.dataStatistics()));
+    }
+  }
+
+  @Override
+  public DataStatisticsOrRecord<D, S> copy(
+      DataStatisticsOrRecord<D, S> from, DataStatisticsOrRecord<D, S> reuse) {
+    DataStatisticsOrRecord<D, S> to;
+    if (from.hasRecord()) {
+      to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer);
+      RowData record = recordSerializer.copy(from.record(), to.record());
+      to.record(record);
+    } else {
+      to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer);
+      DataStatistics<D, S> statistics =
+          statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics());
+      to.dataStatistics(statistics);
+    }
+
+    return to;
+  }
+
+  @Override
+  public int getLength() {
+    return -1;

Review Comment:
   @hililiwei not sure if I understand the question here. can you elaborate?
   
   here is the javadoc for this method from the Flink `TypeSerializer` interface.
   ```
   Returns:
   The length of the data type, or -1 for variable length data types.
   ```



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


[GitHub] [iceberg] huyuanfeng2018 commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "huyuanfeng2018 (via GitHub)" <gi...@apache.org>.
huyuanfeng2018 commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1184480444


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java:
##########
@@ -42,12 +43,19 @@
    *
    * @param key generate from data by applying key selector
    */
-  void add(K key);
+  void add(RowData key);

Review Comment:
   Is it necessary to add a method here to count statistical information with values? For example, data under different keys may have different sizes. Is it also a consideration when controlling subsequent balance,like:``` add(Rowdata key, V v)``` Among them, v may represent the record bytes of the row corresponding to the current key. What do you think?



##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java:
##########
@@ -40,50 +40,49 @@
  * shuffle record to improve data clustering while maintaining relative balanced traffic
  * distribution to downstream subtasks.
  */
-class DataStatisticsOperator<T, K> extends AbstractStreamOperator<DataStatisticsOrRecord<T, K>>
-    implements OneInputStreamOperator<T, DataStatisticsOrRecord<T, K>>, OperatorEventHandler {
+class DataStatisticsOperator<D extends DataStatistics<D, S>, S>

Review Comment:
   Ok, I think this DataStatisticsOperator is good for collecting some statistical information. It seems that we also need an Operator to determine the partitionID, such as PartitionIdAssignerOperator and then pass ```org.apache.flink.api.java.functions.IdPartitioner``` From custom data distribution, I haven’t seen the design of this piece in the design document, can you briefly introduce the follow-up 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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1191659332


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java:
##########
@@ -40,50 +40,49 @@
  * shuffle record to improve data clustering while maintaining relative balanced traffic
  * distribution to downstream subtasks.
  */
-class DataStatisticsOperator<T, K> extends AbstractStreamOperator<DataStatisticsOrRecord<T, K>>
-    implements OneInputStreamOperator<T, DataStatisticsOrRecord<T, K>>, OperatorEventHandler {
+class DataStatisticsOperator<D extends DataStatistics<D, S>, S>

Review Comment:
   @hililiwei that is correct. custom range partitioner for Flink DataStream will distribute the data to writer subtasks with good clustering based on the data statistics.



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


[GitHub] [iceberg] yegangy0718 commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "yegangy0718 (via GitHub)" <gi...@apache.org>.
yegangy0718 commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1188139904


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java:
##########
@@ -132,19 +130,19 @@ public void snapshotState(StateSnapshotContext context) throws Exception {
 
     // For now, we make it simple to send globalStatisticsState at checkpoint
     operatorEventGateway.sendEventToCoordinator(
-        new DataStatisticsEvent<>(checkpointId, localStatistics));
+        new DataStatisticsEvent(checkpointId, localStatistics));
 
     // Recreate the local statistics
-    localStatistics = statisticsFactory.createDataStatistics();
+    localStatistics = statisticsSerializer.createInstance();
   }
 
   @VisibleForTesting
-  DataStatistics<K> localDataStatistics() {
+  DataStatistics localDataStatistics() {
     return localStatistics;
   }
 
   @VisibleForTesting
-  DataStatistics<K> globalDataStatistics() {
+  DataStatistics globalDataStatistics() {

Review Comment:
   <img width="562" alt="Screenshot 2023-05-08 at 10 13 38 PM" src="https://user-images.githubusercontent.com/8229749/237000080-a0cb6b7e-031d-4c4b-a3ec-b5b7327172b5.png">
   
   IDE shows warning for raw use of parameterized class 'DataStatistics' , shall we make the return type `DataStatistics<D, S>`



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


[GitHub] [iceberg] hililiwei commented on a diff in pull request #7494: Flink: change sink shuffle to use RowData as data type and statistics key type

Posted by "hililiwei (via GitHub)" <gi...@apache.org>.
hililiwei commented on code in PR #7494:
URL: https://github.com/apache/iceberg/pull/7494#discussion_r1188186620


##########
flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java:
##########
@@ -40,50 +40,49 @@
  * shuffle record to improve data clustering while maintaining relative balanced traffic
  * distribution to downstream subtasks.
  */
-class DataStatisticsOperator<T, K> extends AbstractStreamOperator<DataStatisticsOrRecord<T, K>>
-    implements OneInputStreamOperator<T, DataStatisticsOrRecord<T, K>>, OperatorEventHandler {
+class DataStatisticsOperator<D extends DataStatistics<D, S>, S>

Review Comment:
   According to my understanding, each writer will be responsible for one or more partitions, and we will distribute the data arriving from upstream to the corresponding writers, right?
   
   



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