mxm commented on code in PR #12996:
URL: https://github.com/apache/iceberg/pull/12996#discussion_r2084813214


##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/RowDataSerializerCache.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.dynamic;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import java.io.Serializable;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+@Internal
+class RowDataSerializerCache implements Serializable {

Review Comment:
   Added.



##########
flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java:
##########
@@ -0,0 +1,292 @@
+/*
+ * 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.dynamic;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.api.java.tuple.Tuple3;
+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.table.runtime.typeutils.RowDataSerializer;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+@Internal
+class DynamicRecordInternalSerializer extends 
TypeSerializer<DynamicRecordInternal> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final RowDataSerializerCache serializerCache;
+  private final boolean writeSchemaAndSpec;
+
+  DynamicRecordInternalSerializer(
+      RowDataSerializerCache serializerCache, boolean writeSchemaAndSpec) {
+    this.serializerCache = serializerCache;
+    this.writeSchemaAndSpec = writeSchemaAndSpec;
+  }
+
+  @Override
+  public TypeSerializer<DynamicRecordInternal> duplicate() {
+    return new DynamicRecordInternalSerializer(
+        new RowDataSerializerCache(serializerCache.catalogLoader(), 
serializerCache.maximumSize()),
+        writeSchemaAndSpec);
+  }
+
+  @Override
+  public DynamicRecordInternal createInstance() {
+    return new DynamicRecordInternal();
+  }
+
+  @Override
+  public void serialize(DynamicRecordInternal toSerialize, DataOutputView 
dataOutputView)
+      throws IOException {
+    dataOutputView.writeUTF(toSerialize.tableName());
+    dataOutputView.writeUTF(toSerialize.branch());
+    if (writeSchemaAndSpec) {
+      dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema()));
+      dataOutputView.writeUTF(PartitionSpecParser.toJson(toSerialize.spec()));
+    } else {
+      dataOutputView.writeInt(toSerialize.schema().schemaId());
+      dataOutputView.writeInt(toSerialize.spec().specId());
+    }
+    dataOutputView.writeInt(toSerialize.writerKey());
+    final RowDataSerializer rowDataSerializer;
+    if (writeSchemaAndSpec) {
+      rowDataSerializer =
+          serializerCache.serializer(
+              toSerialize.tableName(), toSerialize.schema(), 
toSerialize.spec());
+    } else {
+      // Check that the schema id can be resolved. Not strictly necessary for 
serialization.
+      Tuple3<RowDataSerializer, Schema, PartitionSpec> serializer =
+          serializerCache.serializerWithSchemaAndSpec(
+              toSerialize.tableName(),
+              toSerialize.schema().schemaId(),
+              toSerialize.spec().specId());
+      rowDataSerializer = serializer.f0;
+    }
+    rowDataSerializer.serialize(toSerialize.rowData(), dataOutputView);
+    dataOutputView.writeBoolean(toSerialize.upsertMode());
+    dataOutputView.writeInt(toSerialize.equalityFields().size());
+    for (Integer equalityField : toSerialize.equalityFields()) {
+      dataOutputView.writeInt(equalityField);
+    }
+  }
+
+  @Override
+  public DynamicRecordInternal deserialize(DataInputView dataInputView) throws 
IOException {
+    String tableName = dataInputView.readUTF();
+    String branch = dataInputView.readUTF();
+
+    final Schema schema;
+    final PartitionSpec spec;
+    final RowDataSerializer rowDataSerializer;
+    if (writeSchemaAndSpec) {
+      schema = SchemaParser.fromJson(dataInputView.readUTF());
+      spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF());
+      rowDataSerializer = serializerCache.serializer(tableName, schema, spec);
+    } else {
+      Integer schemaId = dataInputView.readInt();
+      Integer specId = dataInputView.readInt();
+      Tuple3<RowDataSerializer, Schema, PartitionSpec> 
serializerWithSchemaAndSpec =
+          serializerCache.serializerWithSchemaAndSpec(tableName, schemaId, 
specId);
+      schema = serializerWithSchemaAndSpec.f1;
+      spec = serializerWithSchemaAndSpec.f2;
+      rowDataSerializer = serializerWithSchemaAndSpec.f0;
+    }
+
+    int writerKey = dataInputView.readInt();
+    RowData rowData = rowDataSerializer.deserialize(dataInputView);
+    boolean upsertMode = dataInputView.readBoolean();
+    int numEqualityFields = dataInputView.readInt();
+    final List<Integer> equalityFieldIds;
+    if (numEqualityFields > 0) {
+      equalityFieldIds = Lists.newArrayList();
+    } else {
+      equalityFieldIds = Collections.emptyList();
+    }

Review Comment:
   Addressed.



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