codope commented on code in PR #10422:
URL: https://github.com/apache/hudi/pull/10422#discussion_r1622387730


##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderRecordReader.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.hudi.hadoop;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.config.HoodieCommonConfig;
+import org.apache.hudi.common.config.HoodieReaderConfig;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.BaseFile;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.TableSchemaResolver;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.TablePathUtils;
+import org.apache.hudi.common.util.collection.ExternalSpillableMap;
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader;
+import org.apache.hudi.hadoop.realtime.RealtimeSplit;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.UnaryOperator;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.hudi.common.config.HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED;
+import static 
org.apache.hudi.common.config.HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE;
+import static 
org.apache.hudi.common.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE;
+import static 
org.apache.hudi.common.config.HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH;
+
+public class HoodieFileGroupReaderRecordReader implements 
RecordReader<NullWritable, ArrayWritable>  {
+
+  public interface HiveReaderCreator {
+    org.apache.hadoop.mapred.RecordReader<NullWritable, ArrayWritable> 
getRecordReader(
+        final org.apache.hadoop.mapred.InputSplit split,
+        final org.apache.hadoop.mapred.JobConf job,
+        final org.apache.hadoop.mapred.Reporter reporter
+    ) throws IOException;
+  }
+
+  private final HiveHoodieReaderContext readerContext;
+  private final HoodieFileGroupReader<ArrayWritable> fileGroupReader;
+  private final ArrayWritable arrayWritable;
+  private final NullWritable nullWritable = NullWritable.get();
+  private final InputSplit inputSplit;
+  private final JobConf jobConfCopy;
+  private final UnaryOperator<ArrayWritable> reverseProjection;
+
+  public HoodieFileGroupReaderRecordReader(HiveReaderCreator readerCreator,
+                                           final InputSplit split,
+                                           final JobConf jobConf,
+                                           final Reporter reporter) throws 
IOException {
+    this.jobConfCopy = new JobConf(jobConf);
+    HoodieRealtimeInputFormatUtils.cleanProjectionColumnIds(jobConfCopy);
+    Set<String> partitionColumns = new 
HashSet<>(getPartitionFieldNames(jobConfCopy));
+    this.inputSplit = split;
+
+    FileSplit fileSplit = (FileSplit) split;
+    String tableBasePath = getTableBasePath(split, jobConfCopy);
+    HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+        .setConf(jobConfCopy)
+        .setBasePath(tableBasePath)
+        .build();
+    String latestCommitTime = getLatestCommitTime(split, metaClient);
+    Schema tableSchema = getLatestTableSchema(metaClient, jobConfCopy, 
latestCommitTime);
+    Schema requestedSchema = createRequestedSchema(tableSchema, jobConfCopy);
+    Map<String, String[]> hosts = new HashMap<>();
+    this.readerContext = new HiveHoodieReaderContext(readerCreator, split, 
jobConfCopy, reporter, tableSchema, hosts, metaClient);

Review Comment:
   looked at the code and we need it per split as we measure progress based on 
split



-- 
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: [email protected]

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

Reply via email to