pvary commented on code in PR #6584:
URL: https://github.com/apache/iceberg/pull/6584#discussion_r1080920039


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.source.reader;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader;
+import org.apache.iceberg.flink.source.DataIterator;
+import org.apache.iceberg.flink.source.RowDataFileScanTaskReader;
+import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/** Read Iceberg rows as {@link GenericRecord}. */
+public class AvroGenericRecordReaderFunction extends 
DataIteratorReaderFunction<GenericRecord> {
+  private final String tableName;
+  private final Schema readSchema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final RowDataFileScanTaskReader rowDataReader;
+
+  private transient RowDataToAvroGenericRecordConverter converter;
+
+  /**
+   * Create a reader function without projection and name mapping. Column name 
is case-insensitive.
+   */
+  public static AvroGenericRecordReaderFunction fromTable(Table table) {
+    return new AvroGenericRecordReaderFunction(
+        table.name(),
+        new Configuration(),
+        table.schema(),
+        null,
+        null,
+        false,

Review Comment:
   > > Can we provide a similar entry where the user simply tells the Builder 
that he wants to use AvroGenericRecordReaderFunction , and then lets the 
Builder create it?
   > 
   > I see your point. I agree that it is cumbersome for users to construct 
`AvroGenericRecordReaderFunction` or `RowDataReaderFunction`. Since we can 
extract type information from generic `T`, one idea would to add a setter for 
type class to the builder. If the type class is `RowData`, builder constructs 
`RowDataReaderFunction`. If it is Avro `GenericRecord, builder constructs 
`AvroGenericRecordReaderFunction`.
   > 
   > ```
   > public outputType(Class<T> outputRecordTypeClass)
   > ```
   > 
   
   I think this is a good idea, but I think this would merit another PR.
   We should do this for the Source and the Sink as well.
   
   > > we should also allow users to query metadata tables via the flink java 
api, right?
   > 
   > I am not sure if that is necessary. Doesn't Iceberg Java API cover that 
already?
   
   For the metadata table, the most important use-case is when we read the 
metadata on the job initialisation and change the job based on the metadata 
read. Do we have use-case when we want to stream the results of the metadata 
query? Like when new files added to the table, or new snapshot created for the 
table?
   
   IMHO, this is something for another PR if we decide to do it.



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