[ 
https://issues.apache.org/jira/browse/HIVE-21924?focusedWorklogId=319434&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-319434
 ]

ASF GitHub Bot logged work on HIVE-21924:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 27/Sep/19 09:17
            Start Date: 27/Sep/19 09:17
    Worklog Time Spent: 10m 
      Work Description: sankarh commented on pull request #791: HIVE-21924
URL: https://github.com/apache/hive/pull/791#discussion_r328918236
 
 

 ##########
 File path: 
ql/src/java/org/apache/hadoop/hive/ql/io/SkippingTextInputFormat.java
 ##########
 @@ -0,0 +1,136 @@
+package org.apache.hadoop.hive.ql.io;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * SkippingInputFormat is a header/footer aware input format. It truncates
+ * splits identified by TextInputFormat. Header and footers are removed
+ * from the splits.
+ */
+public class SkippingTextInputFormat extends TextInputFormat {
+
+  private final Map<Path, Long> startIndexMap = new ConcurrentHashMap<Path, 
Long>();
+  private final Map<Path, Long> endIndexMap = new ConcurrentHashMap<Path, 
Long>();
+  private JobConf conf;
+  private int headerCount;
+  private int footerCount;
+
+  @Override
+  public void configure(JobConf conf) {
+    this.conf = conf;
+    super.configure(conf);
+  }
+
+  public void configure(JobConf conf, int headerCount, int footerCount) {
+    configure(conf);
+    this.headerCount = headerCount;
+    this.footerCount = footerCount;
+  }
+
+  @Override
+  protected FileSplit makeSplit(Path file, long start, long length, String[] 
hosts) {
+    long cachedStart = getCachedStartIndex(file);
+    long cachedEnd = getCachedEndIndex(file);
+    if (cachedStart > start + length) {
+      return new NullRowsInputFormat.DummyInputSplit(file);
+    } else if (cachedStart > start) {
+      length = length - (cachedStart - start);
+      start = cachedStart;
+    }
+    if (cachedEnd > -1 && cachedEnd < start) {
+      return new NullRowsInputFormat.DummyInputSplit(file);
+    } else if (cachedEnd < start + length) {
+      length = cachedEnd - start;
+    }
+    return super.makeSplit(file, start, length, hosts);
+  }
+
+  @Override
+  protected FileSplit makeSplit(Path file, long start, long length, String[] 
hosts, String[] inMemoryHosts) {
+    long cachedStart = getCachedStartIndex(file);
+    long cachedEnd = getCachedEndIndex(file);
+    if (cachedStart > start + length) {
+      return new NullRowsInputFormat.DummyInputSplit(file);
+    } else if (cachedStart > start) {
+      length = length - (cachedStart - start);
+      start = cachedStart;
+    }
+    if (cachedEnd > - 1 && cachedEnd < start) {
+      return new NullRowsInputFormat.DummyInputSplit(file);
+    } else if (cachedEnd < start + length) {
+      length = cachedEnd - start;
+    }
+    return super.makeSplit(file, start, length, hosts, inMemoryHosts);
+  }
+
+  private long getCachedStartIndex(Path path) {
+    Long startIndexForFile = startIndexMap.get(path);
+    if (startIndexForFile == null) {
+      try {
+        FSDataInputStream fis = path.getFileSystem(conf).open(path);
+        for (int j = 0; j < headerCount; j++) {
+          fis.readLine();
+          // back 1 byte because readers skip the entire first row if split 
start is not 0
+          startIndexForFile = fis.getPos() - 1;
+        }
+      } catch (IOException e) {
+        startIndexForFile = 0L;
+      }
+      startIndexMap.put(path, startIndexForFile);
+    }
+    return startIndexForFile;
+  }
+
+  private long getCachedEndIndex(Path path) {
+    Long endIndexForFile = endIndexMap.get(path);
+    if (endIndexForFile == null) {
+      try {
+        final long bufferSectionSize = 1024;
+        long bufferSectionEnd = 
path.getFileSystem(conf).getFileStatus(path).getLen();
+        long bufferSectionStart = Math.max(0, bufferSectionEnd - 
bufferSectionSize);
+        Queue<Long> lineEndBuffer = new ArrayDeque<Long>(headerCount + 1);
+        FSDataInputStream fis = path.getFileSystem(conf).open(path);
 
 Review comment:
   fis.close() needed in finally block.
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 319434)

> Split text files even if header/footer exists
> ---------------------------------------------
>
>                 Key: HIVE-21924
>                 URL: https://issues.apache.org/jira/browse/HIVE-21924
>             Project: Hive
>          Issue Type: Improvement
>          Components: File Formats
>    Affects Versions: 2.4.0, 4.0.0, 3.2.0
>            Reporter: Prasanth Jayachandran
>            Assignee: Mustafa Iman
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: HIVE-21924.patch
>
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> https://github.com/apache/hive/blob/967a1cc98beede8e6568ce750ebeb6e0d048b8ea/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java#L494-L503
>  
> {code}
>     int headerCount = 0;
>     int footerCount = 0;
>     if (table != null) {
>       headerCount = Utilities.getHeaderCount(table);
>       footerCount = Utilities.getFooterCount(table, conf);
>       if (headerCount != 0 || footerCount != 0) {
>         // Input file has header or footer, cannot be splitted.
>         HiveConf.setLongVar(conf, ConfVars.MAPREDMINSPLITSIZE, 
> Long.MAX_VALUE);
>       }
>     }
> {code}
> this piece of code makes the CSV (or any text files with header/footer) files 
> not splittable if header or footer is present. 
> If only header is present, we can find the offset after first line break and 
> use that to split. Similarly for footer, may be read few KB's of data at the 
> end and find the last line break offset. Use that to determine the data range 
> which can be used for splitting. Few reads during split generation are 
> cheaper than not splitting the file at all.  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to