You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2020/04/28 13:26:02 UTC

[GitHub] [hive] pgaref opened a new pull request #998: HIVE-22769 Fix SkippingTextInputFormat

pgaref opened a new pull request #998:
URL: https://github.com/apache/hive/pull/998


   * Compressed format check for SkippingTextInputFormat
   * LLAP record reader does header/footer skipping for compressed files
   * More tests
   
   Change-Id: Ifb68bd9e3b103424aed2b9d6871b536a5437a095


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pgaref commented on a change in pull request #998: HIVE-22769 Fix SkippingTextInputFormat

Posted by GitBox <gi...@apache.org>.
pgaref commented on a change in pull request #998:
URL: https://github.com/apache/hive/pull/998#discussion_r417299400



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/PassThruOffsetReader.java
##########
@@ -20,23 +20,81 @@
 import java.io.IOException;
 
 import org.apache.hadoop.hive.llap.io.encoded.SerDeEncodedDataReader.ReaderWithOffsets;
+import org.apache.hadoop.hive.ql.exec.FooterBuffer;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 
 @SuppressWarnings("rawtypes") class PassThruOffsetReader implements ReaderWithOffsets {
   protected final RecordReader sourceReader;
   protected final Object key;
   protected final Writable value;
+  protected final JobConf jobConf;
+  protected final int skipHeaderCnt;
+  protected final int skipFooterCnt;
+  private transient FooterBuffer footerBuffer;
+  private transient boolean initialized = false;
 
-  PassThruOffsetReader(RecordReader sourceReader) {
+  PassThruOffsetReader(RecordReader sourceReader, JobConf jobConf, int headerCnt, int footerCnt) {
     this.sourceReader = sourceReader;
-    key = sourceReader.createKey();
-    value = (Writable)sourceReader.createValue();
+    this.key = sourceReader.createKey();
+    this.value = (Writable)sourceReader.createValue();
+    this.jobConf = jobConf;
+    this.skipHeaderCnt = headerCnt;
+    this.skipFooterCnt = footerCnt;
   }
 
   @Override
   public boolean next() throws IOException {
-    return sourceReader.next(key, value);
+    try {
+      boolean opNotEOF = true;
+      /**
+       * Start reading a new file.
+       * If file contains header, skip header lines before reading the records.
+       * If file contains footer, used FooterBuffer to cache and remove footer
+       * records at the end of the file.
+       */
+      if (!initialized) {
+        // Skip header lines.
+        opNotEOF = Utilities.skipHeader(sourceReader, skipFooterCnt, key, value);
+
+        // Initialize footer buffer.
+        if (opNotEOF && skipFooterCnt > 0) {
+          footerBuffer = new FooterBuffer();
+          opNotEOF = footerBuffer.initializeBuffer(jobConf, sourceReader, skipFooterCnt, (WritableComparable) key, value);
+        }
+        this.initialized = true;
+      }
+
+      if (opNotEOF && footerBuffer == null) {
+        /**
+         * When file doesn't end after skipping header line
+         * and there is NO footer lines, read normally.
+         */
+        opNotEOF = sourceReader.next(key, value);
+      }
+
+      if (opNotEOF && footerBuffer != null) {
+        /**
+         * When file doesn't end after skipping header line
+         * and there IS footer lines, update footerBuffer
+         */
+        opNotEOF = footerBuffer.updateBuffer(jobConf, sourceReader, (WritableComparable) key, value);
+      }
+
+      if (opNotEOF) {
+        // File reached the end
+        return true;
+      } else {
+        // Done reading
+        close();

Review comment:
       You are right, EncodedDataReader does take care of closing the reader -- so removing it from here.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] mustafaiman commented on a change in pull request #998: HIVE-22769 Fix SkippingTextInputFormat

Posted by GitBox <gi...@apache.org>.
mustafaiman commented on a change in pull request #998:
URL: https://github.com/apache/hive/pull/998#discussion_r416947329



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/PassThruOffsetReader.java
##########
@@ -20,23 +20,81 @@
 import java.io.IOException;
 
 import org.apache.hadoop.hive.llap.io.encoded.SerDeEncodedDataReader.ReaderWithOffsets;
+import org.apache.hadoop.hive.ql.exec.FooterBuffer;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 
 @SuppressWarnings("rawtypes") class PassThruOffsetReader implements ReaderWithOffsets {
   protected final RecordReader sourceReader;
   protected final Object key;
   protected final Writable value;
+  protected final JobConf jobConf;
+  protected final int skipHeaderCnt;
+  protected final int skipFooterCnt;
+  private transient FooterBuffer footerBuffer;
+  private transient boolean initialized = false;
 
-  PassThruOffsetReader(RecordReader sourceReader) {
+  PassThruOffsetReader(RecordReader sourceReader, JobConf jobConf, int headerCnt, int footerCnt) {
     this.sourceReader = sourceReader;
-    key = sourceReader.createKey();
-    value = (Writable)sourceReader.createValue();
+    this.key = sourceReader.createKey();
+    this.value = (Writable)sourceReader.createValue();
+    this.jobConf = jobConf;
+    this.skipHeaderCnt = headerCnt;
+    this.skipFooterCnt = footerCnt;
   }
 
   @Override
   public boolean next() throws IOException {
-    return sourceReader.next(key, value);
+    try {
+      boolean opNotEOF = true;
+      /**
+       * Start reading a new file.
+       * If file contains header, skip header lines before reading the records.
+       * If file contains footer, used FooterBuffer to cache and remove footer
+       * records at the end of the file.
+       */
+      if (!initialized) {
+        // Skip header lines.
+        opNotEOF = Utilities.skipHeader(sourceReader, skipFooterCnt, key, value);
+
+        // Initialize footer buffer.
+        if (opNotEOF && skipFooterCnt > 0) {
+          footerBuffer = new FooterBuffer();
+          opNotEOF = footerBuffer.initializeBuffer(jobConf, sourceReader, skipFooterCnt, (WritableComparable) key, value);
+        }
+        this.initialized = true;
+      }
+
+      if (opNotEOF && footerBuffer == null) {
+        /**
+         * When file doesn't end after skipping header line
+         * and there is NO footer lines, read normally.
+         */
+        opNotEOF = sourceReader.next(key, value);
+      }
+
+      if (opNotEOF && footerBuffer != null) {
+        /**
+         * When file doesn't end after skipping header line
+         * and there IS footer lines, update footerBuffer
+         */
+        opNotEOF = footerBuffer.updateBuffer(jobConf, sourceReader, (WritableComparable) key, value);
+      }
+
+      if (opNotEOF) {
+        // File reached the end
+        return true;
+      } else {
+        // Done reading
+        close();

Review comment:
       there were no close calls in this method before. How did closing the file become the responsibility of this class now?

##########
File path: ql/src/test/org/apache/hadoop/hive/ql/io/TestSkippingTextInputFormat.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.hadoop.hive.ql.io;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.plan.MapredWork;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapred.FileInputFormat;
+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 org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unittest for SkippingTextInputFormat with Skip Header/Footer.
+ */
+public class TestSkippingTextInputFormat {
+
+  private Configuration conf;
+  private JobConf job;
+  private FileSystem fileSystem;
+  private Path testDir;
+  Reporter reporter;
+
+  private Path dataDir;
+
+  private CompressionCodecFactory compressionCodecs = null;
+  private CompressionCodec codec;
+
+  @Before
+  public void setUp() throws IOException {
+    conf = new Configuration();
+    job = new JobConf(conf);
+
+    TableDesc tblDesc = Utilities.defaultTd;
+    PartitionDesc partDesc = new PartitionDesc(tblDesc, null);
+    LinkedHashMap<Path, PartitionDesc> pt = new LinkedHashMap<>();
+    pt.put(new Path("/tmp/testfolder"), partDesc);
+    MapredWork mrwork = new MapredWork();
+    mrwork.getMapWork().setPathToPartitionInfo(pt);
+    Utilities.setMapRedWork(job, mrwork,new Path("/tmp/" + System.getProperty("user.name"), "hive"));
+
+    fileSystem = FileSystem.getLocal(conf);
+    testDir = new Path(System.getProperty("test.tmp.dir", System.getProperty(
+        "user.dir", new File(".").getAbsolutePath()))
+        + "/TestSkippingTextInputFormat");
+    reporter = Reporter.NULL;
+    fileSystem.delete(testDir, true);
+
+    dataDir =  new Path(testDir, "datadir");
+    fileSystem.mkdirs(dataDir);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    fileSystem.delete(testDir, true);
+  }
+
+  /**
+   * Test CSV input file with header/footer skip.
+   */
+  @Test
+  public void testSkipFileSplits() throws Exception {
+    FileSystem fs = dataDir.getFileSystem(job);
+    FileInputFormat.setInputPaths(job, dataDir);
+
+    // First Dir with 1 File
+    Path dir1_file1 = new Path(dataDir, "skipfile1.csv");
+    writeTextFile(dir1_file1,

Review comment:
       can we put the test files in resources directory in advance rather than creating on the spot? 

##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java
##########
@@ -1633,16 +1638,20 @@ private ObjectInspector getOiFromSerDe() throws IOException {
     }
   }
 
-  private ReaderWithOffsets createOffsetReader(RecordReader<?, ?> sourceReader) {
+  private ReaderWithOffsets createOffsetReader(RecordReader<?, ?> sourceReader, TableDesc tableDesc)
+      throws IOException {
+    int headerCount = Utilities.getHeaderCount(tableDesc);
+    int footerCount = Utilities.getFooterCount(tableDesc, jobConf);
     if (LlapIoImpl.LOG.isDebugEnabled()) {
-      LlapIoImpl.LOG.debug("Using " + sourceReader.getClass().getSimpleName() + " to read data");
+      LlapIoImpl.LOG.debug("Using {} to read data with HeaderSkip {} and FooterSkip {}",

Review comment:
       Log lines such as "HeaderSkip 2 and FooterSkip 1" may be a bit hard to understand for someone who did not know this part of the code. Can you be more clear? Like "skipping {} lines for header, skipping {} lines for footer"




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pgaref commented on a change in pull request #998: HIVE-22769 Fix SkippingTextInputFormat

Posted by GitBox <gi...@apache.org>.
pgaref commented on a change in pull request #998:
URL: https://github.com/apache/hive/pull/998#discussion_r417300530



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java
##########
@@ -1633,16 +1638,20 @@ private ObjectInspector getOiFromSerDe() throws IOException {
     }
   }
 
-  private ReaderWithOffsets createOffsetReader(RecordReader<?, ?> sourceReader) {
+  private ReaderWithOffsets createOffsetReader(RecordReader<?, ?> sourceReader, TableDesc tableDesc)
+      throws IOException {
+    int headerCount = Utilities.getHeaderCount(tableDesc);
+    int footerCount = Utilities.getFooterCount(tableDesc, jobConf);
     if (LlapIoImpl.LOG.isDebugEnabled()) {
-      LlapIoImpl.LOG.debug("Using " + sourceReader.getClass().getSimpleName() + " to read data");
+      LlapIoImpl.LOG.debug("Using {} to read data with HeaderSkip {} and FooterSkip {}",

Review comment:
       sure, will use table settings names instead




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pgaref closed pull request #998: HIVE-22769 Fix SkippingTextInputFormat

Posted by GitBox <gi...@apache.org>.
pgaref closed pull request #998:
URL: https://github.com/apache/hive/pull/998


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pgaref commented on a change in pull request #998: HIVE-22769 Fix SkippingTextInputFormat

Posted by GitBox <gi...@apache.org>.
pgaref commented on a change in pull request #998:
URL: https://github.com/apache/hive/pull/998#discussion_r417303568



##########
File path: ql/src/test/org/apache/hadoop/hive/ql/io/TestSkippingTextInputFormat.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.hadoop.hive.ql.io;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.plan.MapredWork;
+import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.mapred.FileInputFormat;
+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 org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unittest for SkippingTextInputFormat with Skip Header/Footer.
+ */
+public class TestSkippingTextInputFormat {
+
+  private Configuration conf;
+  private JobConf job;
+  private FileSystem fileSystem;
+  private Path testDir;
+  Reporter reporter;
+
+  private Path dataDir;
+
+  private CompressionCodecFactory compressionCodecs = null;
+  private CompressionCodec codec;
+
+  @Before
+  public void setUp() throws IOException {
+    conf = new Configuration();
+    job = new JobConf(conf);
+
+    TableDesc tblDesc = Utilities.defaultTd;
+    PartitionDesc partDesc = new PartitionDesc(tblDesc, null);
+    LinkedHashMap<Path, PartitionDesc> pt = new LinkedHashMap<>();
+    pt.put(new Path("/tmp/testfolder"), partDesc);
+    MapredWork mrwork = new MapredWork();
+    mrwork.getMapWork().setPathToPartitionInfo(pt);
+    Utilities.setMapRedWork(job, mrwork,new Path("/tmp/" + System.getProperty("user.name"), "hive"));
+
+    fileSystem = FileSystem.getLocal(conf);
+    testDir = new Path(System.getProperty("test.tmp.dir", System.getProperty(
+        "user.dir", new File(".").getAbsolutePath()))
+        + "/TestSkippingTextInputFormat");
+    reporter = Reporter.NULL;
+    fileSystem.delete(testDir, true);
+
+    dataDir =  new Path(testDir, "datadir");
+    fileSystem.mkdirs(dataDir);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    fileSystem.delete(testDir, true);
+  }
+
+  /**
+   * Test CSV input file with header/footer skip.
+   */
+  @Test
+  public void testSkipFileSplits() throws Exception {
+    FileSystem fs = dataDir.getFileSystem(job);
+    FileInputFormat.setInputPaths(job, dataDir);
+
+    // First Dir with 1 File
+    Path dir1_file1 = new Path(dataDir, "skipfile1.csv");
+    writeTextFile(dir1_file1,

Review comment:
       Just wanted to avoid creating even more files for this simple test.
   Moreover, I find it easier to have both data and test in one place where possible
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] mustafaiman commented on a change in pull request #998: HIVE-22769 Fix SkippingTextInputFormat

Posted by GitBox <gi...@apache.org>.
mustafaiman commented on a change in pull request #998:
URL: https://github.com/apache/hive/pull/998#discussion_r417460328



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/PassThruOffsetReader.java
##########
@@ -20,23 +20,80 @@
 import java.io.IOException;
 
 import org.apache.hadoop.hive.llap.io.encoded.SerDeEncodedDataReader.ReaderWithOffsets;
+import org.apache.hadoop.hive.ql.exec.FooterBuffer;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 
 @SuppressWarnings("rawtypes") class PassThruOffsetReader implements ReaderWithOffsets {
   protected final RecordReader sourceReader;
   protected final Object key;
   protected final Writable value;
+  protected final JobConf jobConf;
+  protected final int skipHeaderCnt;
+  protected final int skipFooterCnt;
+  private transient FooterBuffer footerBuffer;
+  private transient boolean initialized = false;
 
-  PassThruOffsetReader(RecordReader sourceReader) {
+  PassThruOffsetReader(RecordReader sourceReader, JobConf jobConf, int headerCnt, int footerCnt) {
     this.sourceReader = sourceReader;
-    key = sourceReader.createKey();
-    value = (Writable)sourceReader.createValue();
+    this.key = sourceReader.createKey();
+    this.value = (Writable)sourceReader.createValue();
+    this.jobConf = jobConf;
+    this.skipHeaderCnt = headerCnt;
+    this.skipFooterCnt = footerCnt;
   }
 
   @Override
   public boolean next() throws IOException {
-    return sourceReader.next(key, value);
+    try {
+      boolean opNotEOF = true;
+      /**
+       * Start reading a new file.
+       * If file contains header, skip header lines before reading the records.
+       * If file contains footer, used FooterBuffer to cache and remove footer
+       * records at the end of the file.
+       */
+      if (!initialized) {
+        // Skip header lines.
+        opNotEOF = Utilities.skipHeader(sourceReader, skipFooterCnt, key, value);
+
+        // Initialize footer buffer.
+        if (opNotEOF && skipFooterCnt > 0) {
+          footerBuffer = new FooterBuffer();
+          opNotEOF = footerBuffer.initializeBuffer(jobConf, sourceReader, skipFooterCnt, (WritableComparable) key, value);
+        }
+        this.initialized = true;
+      }
+
+      if (opNotEOF && footerBuffer == null) {
+        /**
+         * When file doesn't end after skipping header line
+         * and there is NO footer lines, read normally.
+         */
+        opNotEOF = sourceReader.next(key, value);
+      }
+
+      if (opNotEOF && footerBuffer != null) {
+        /**
+         * When file doesn't end after skipping header line
+         * and there IS footer lines, update footerBuffer
+         */
+        opNotEOF = footerBuffer.updateBuffer(jobConf, sourceReader, (WritableComparable) key, value);
+      }
+
+      if (opNotEOF) {

Review comment:
       This if-else block can be simplified now




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org