openinx commented on a change in pull request #1477:
URL: https://github.com/apache/iceberg/pull/1477#discussion_r511805556



##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroEncoderUtil.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.avro;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AvroEncoderUtil {
+
+  private AvroEncoderUtil() {
+  }
+
+  static {
+    LogicalTypes.register(LogicalMap.NAME, schema -> LogicalMap.get());
+  }
+
+  private static final int VERSION = 1;
+  private static final byte[] MAGIC_BYTES = new byte[] {'a', 'V', 'R', 
VERSION};
+
+  private static byte[] encodeInt(int value) {
+    return ByteBuffer.allocate(4).putInt(value).array();
+  }
+
+  private static int decodeInt(byte[] value) {
+    return ByteBuffer.wrap(value).getInt();
+  }
+
+  public static <T> byte[] encode(T datum, Schema avroSchema) throws 
IOException {

Review comment:
       About the native avro's single-message encoding issue, I read the code 
in 
[IcebergEncoder](https://github.com/apache/iceberg/blob/68e417c3591439044b229b657a529cad0fec66e3/core/src/main/java/org/apache/iceberg/data/avro/IcebergEncoder.java)
 and 
[IcebergDecoder](https://github.com/apache/iceberg/blob/68e417c3591439044b229b657a529cad0fec66e3/core/src/main/java/org/apache/iceberg/data/avro/IcebergDecoder.java),
   seems it don't encode the writeSchema into the binary, that seems validate 
the discussion 
[here](https://github.com/apache/iceberg/pull/1477/files#r496317201).  Besides, 
 it need a user-provided `writeSchema` to read the avro binary while in our 
flink states we only have the binary . Unless we decode the binary to get the 
schema, we could not call the `IcebergDecoder`.  Finally,  the code would be 
similar to this PR.

##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroEncoderUtil.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.avro;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AvroEncoderUtil {
+
+  private AvroEncoderUtil() {
+  }
+
+  static {
+    LogicalTypes.register(LogicalMap.NAME, schema -> LogicalMap.get());
+  }
+
+  private static final int VERSION = 1;
+  private static final byte[] MAGIC_BYTES = new byte[] {'a', 'V', 'R', 
VERSION};
+
+  private static byte[] encodeInt(int value) {
+    return ByteBuffer.allocate(4).putInt(value).array();
+  }
+
+  private static int decodeInt(byte[] value) {
+    return ByteBuffer.wrap(value).getInt();
+  }
+
+  public static <T> byte[] encode(T datum, Schema avroSchema) throws 
IOException {

Review comment:
       Yeah, in theory the internal avro encoding / decoding should not be 
exposed to users, but as we've discussed before that the `GenericAvroWriter` 
and `GenerciAvroReader` should not be exposed to users.  So we have to access 
the writer & reader package-only, and then expose another methods to the 
package outside.  Moving this to the package where ManifestFiles located does 
not work.

##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroEncoderUtil.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.avro;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AvroEncoderUtil {
+
+  private AvroEncoderUtil() {
+  }
+
+  static {
+    LogicalTypes.register(LogicalMap.NAME, schema -> LogicalMap.get());
+  }
+
+  private static final int VERSION = 1;
+  private static final byte[] MAGIC_BYTES = new byte[] {'a', 'V', 'R', 
VERSION};
+
+  private static byte[] encodeInt(int value) {
+    return ByteBuffer.allocate(4).putInt(value).array();
+  }
+
+  private static int decodeInt(byte[] value) {
+    return ByteBuffer.wrap(value).getInt();
+  }
+
+  public static <T> byte[] encode(T datum, Schema avroSchema) throws 
IOException {
+    try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+      // Write the magic bytes
+      out.write(MAGIC_BYTES);
+
+      // Write the length of avro schema string.
+      byte[] avroSchemaBytes = 
avroSchema.toString().getBytes(StandardCharsets.UTF_8);
+      out.write(encodeInt(avroSchemaBytes.length));
+
+      // Write the avro schema string.
+      out.write(avroSchemaBytes);
+
+      // Encode the datum with avro schema.
+      BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
+      DatumWriter<T> writer = new GenericAvroWriter<>(avroSchema);
+      writer.write(datum, encoder);
+      encoder.flush();
+      return out.toByteArray();
+    }
+  }
+
+  public static <T> T decode(byte[] data) throws IOException {
+    byte[] buffer4 = new byte[4];
+    try (ByteArrayInputStream in = new ByteArrayInputStream(data, 0, 
data.length)) {

Review comment:
       Sounds like a great idea, it helps a lot.

##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroEncoderUtil.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.avro;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AvroEncoderUtil {
+
+  private AvroEncoderUtil() {
+  }
+
+  static {
+    LogicalTypes.register(LogicalMap.NAME, schema -> LogicalMap.get());
+  }
+
+  private static final int VERSION = 1;
+  private static final byte[] MAGIC_BYTES = new byte[] {'a', 'V', 'R', 
VERSION};
+
+  private static byte[] encodeInt(int value) {
+    return ByteBuffer.allocate(4).putInt(value).array();
+  }
+
+  private static int decodeInt(byte[] value) {
+    return ByteBuffer.wrap(value).getInt();
+  }
+
+  public static <T> byte[] encode(T datum, Schema avroSchema) throws 
IOException {
+    try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+      // Write the magic bytes
+      out.write(MAGIC_BYTES);
+
+      // Write the length of avro schema string.
+      byte[] avroSchemaBytes = 
avroSchema.toString().getBytes(StandardCharsets.UTF_8);
+      out.write(encodeInt(avroSchemaBytes.length));
+
+      // Write the avro schema string.
+      out.write(avroSchemaBytes);
+
+      // Encode the datum with avro schema.
+      BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
+      DatumWriter<T> writer = new GenericAvroWriter<>(avroSchema);
+      writer.write(datum, encoder);
+      encoder.flush();
+      return out.toByteArray();
+    }
+  }
+
+  public static <T> T decode(byte[] data) throws IOException {
+    byte[] buffer4 = new byte[4];
+    try (ByteArrayInputStream in = new ByteArrayInputStream(data, 0, 
data.length)) {
+      // Read the magic bytes
+      Preconditions.checkState(in.read(buffer4) == 4, "Size of magic bytes 
isn't 4.");
+      Preconditions.checkState(Arrays.equals(MAGIC_BYTES, buffer4), "Magic 
bytes mismatched.");
+
+      // Read the length of avro schema string.
+      Preconditions.checkState(in.read(buffer4) == 4, "Could not read an 
integer from input stream.");
+      int avroSchemaLength = decodeInt(buffer4);
+      Preconditions.checkState(avroSchemaLength > 0, "Length of avro schema 
string should be positive");
+
+      // Read the avro schema string.
+      byte[] avroSchemaBytes = new byte[avroSchemaLength];

Review comment:
       After using the `DataInputStream` and `DataOutputStream`,  the schema 
encoding and decoding will be much simpler, so we don't need the extra small 
method any more. 
   
   ```java
         // Write avro schema
         dataOut.writeUTF(avroSchema.toString());
   
         // Read avro schema
         Schema avroSchema = new Schema.Parser().parse(dataInput.readUTF());
   ```

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifest.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestWriter;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+class FlinkManifest {

Review comment:
       Em,  its function is not quite clear now. I will make it more clear.

##########
File path: core/src/main/java/org/apache/iceberg/avro/AvroEncoderUtil.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.avro;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AvroEncoderUtil {
+
+  private AvroEncoderUtil() {
+  }
+
+  static {
+    LogicalTypes.register(LogicalMap.NAME, schema -> LogicalMap.get());
+  }
+
+  private static final int VERSION = 1;

Review comment:
       We introduced a `AvroEncoderUtil` here because the `IndexedRecord` avro 
serialization is depending on the `GenericAvroWriter` and `GenericAvroReader` ( 
which are package-access and should not be exposed to users). 




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

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to