From issues-return-167717-archive-asf-public=cust-asf.ponee.io@hive.apache.org Fri Sep 27 09:18:07 2019 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 91AEC180660 for ; Fri, 27 Sep 2019 11:18:07 +0200 (CEST) Received: (qmail 13369 invoked by uid 500); 27 Sep 2019 09:18:07 -0000 Mailing-List: contact issues-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hive.apache.org Delivered-To: mailing list issues@hive.apache.org Received: (qmail 13360 invoked by uid 99); 27 Sep 2019 09:18:07 -0000 Received: from mailrelay1-us-west.apache.org (HELO mailrelay1-us-west.apache.org) (209.188.14.139) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 27 Sep 2019 09:18:07 +0000 Received: from jira-he-de.apache.org (static.172.67.40.188.clients.your-server.de [188.40.67.172]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id D03C2E312D for ; Fri, 27 Sep 2019 09:18:05 +0000 (UTC) Received: from jira-he-de.apache.org (localhost.localdomain [127.0.0.1]) by jira-he-de.apache.org (ASF Mail Server at jira-he-de.apache.org) with ESMTP id B5DA37822D2 for ; Fri, 27 Sep 2019 09:18:02 +0000 (UTC) Date: Fri, 27 Sep 2019 09:18:02 +0000 (UTC) From: "ASF GitHub Bot (Jira)" To: issues@hive.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Work logged] (HIVE-21924) Split text files even if header/footer exists MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HIVE-21924?focusedWorklogId=319435&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-319435 ] 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_r328914761 ########## 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 startIndexMap = new ConcurrentHashMap(); + private final Map endIndexMap = new ConcurrentHashMap(); + 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; Review comment: Can we move this outside the loop? Also, as per current logic, if data have only footer rows and headerCount=0, then startIndexForFile will be null. Having this statement outside the loop would handle 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. For queries about this service, please contact Infrastructure at: users@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 319435) Time Spent: 40m (was: 0.5h) > 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: 40m > 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)