bvaradar commented on a change in pull request #674: Upgrade to Hive 2.x, MOR 
read query fixes and performance improvement
URL: https://github.com/apache/incubator-hudi/pull/674#discussion_r292596526
 
 

 ##########
 File path: 
hoodie-common/src/main/java/com/uber/hoodie/common/util/LogReaderUtils.java
 ##########
 @@ -0,0 +1,80 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. ([email protected])
+ *
+ *  Licensed 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 com.uber.hoodie.common.util;
+
+import com.uber.hoodie.common.model.HoodieLogFile;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.log.HoodieLogFormat;
+import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
+import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
+import 
com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
+import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+
+/**
+ * Utils class for performing various log file reading operations
+ */
+public class LogReaderUtils {
+
+  private static Schema readSchemaFromLogFileInReverse(FileSystem fs, 
HoodieActiveTimeline activeTimeline, Path path)
+      throws IOException {
+    HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, new 
HoodieLogFile(path), null, true, true);
+    Schema writerSchema = null;
+    HoodieTimeline completedTimeline = 
activeTimeline.getCommitsTimeline().filterCompletedInstants();
+    while (reader.hasPrev()) {
+      HoodieLogBlock block = reader.prev();
+      if (block instanceof HoodieAvroDataBlock && block != null) {
+        HoodieAvroDataBlock lastBlock = (HoodieAvroDataBlock) block;
+        if 
(completedTimeline.containsOrBeforeTimelineStarts(lastBlock.getLogBlockHeader().get(HeaderMetadataType
 
 Review comment:
   This will be problematic when we are reading very old un-compacted log files 
whose commitId is no longer in the active timeline. We wont be able to know if 
the block is rolledback or not. right ? We can have similar logic as what we 
have in AbstractHoodieLogRecordScanner ?

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


With regards,
Apache Git Services

Reply via email to