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/02 01:57:37 UTC

[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

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