You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2021/11/02 21:31:59 UTC

[GitHub] [drill] cgivre opened a new pull request #2359: DRILL-8028: Add PDF Format Plugin

cgivre opened a new pull request #2359:
URL: https://github.com/apache/drill/pull/2359


   # [DRILL-8028](https://issues.apache.org/jira/browse/DRILL-8028): Add PDF Format Plugin
   
   ## Description
   This PR adds the capability to query data contained in PDF tables. 
   
   ## Documentation
   One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
    PDF tables using Drill's SQL interface.  
   
   ## Data Model
   Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
   provided schema. 
   
   ### Merging Pages
   The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
   will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.
   
   ## Configuration
   To configure the PDF reader, simply add the information below to the `formats` section of a file base storage plugin. 
   
   ```json
   "pdf": {
     "type": "pdf",
     "extensions": [
       "pdf"
     ],
     "extractHeaders": true,
     "combinePages": false
   }
   ```
   The avaialable options are:
   * `extractHeaders`: Extracts the first row of any tables as the header row.  If set to false, Drill will assign column names of `field_0`, `field_1` to each column.
   * `combinePages`: Merges multipage tables together.
   * `defaultTableIndex`:  Allows you to query different tables within the PDF file. Index starts at `0`. 
   
   
   ## Accessing Document Metadata Fields
   PDF files have a considerable amount of metadata which can be useful for analysis.  Drill will extract the following fields from every PDF file.  Note that these fields are not
    projected in star queries and must be selected explicitly.  The document's creator populates these fields and some or all may be empty. With the exception of `_page_count
    ` which is an `INT` and the two date fields, all the other fields are `VARCHAR` fields.
    
    The fields are:
    * `_page_count`
    * `_author`
    * `_title`
    * `_keywords`
    * `_creator`
    * `_producer`
    * `_creation_date`
    * `_modification_date`
    * `_trapped`
    * `_table_count`
    
    The query below will access a document's metadata:
    
    ```sql
   SELECT _page_count, _title, _author, _subject, 
   _keywords, _creator, _producer, _creation_date, 
   _modification_date, _trapped 
   FROM dfs.`pdf/20.pdf`
   ```
   The query below demonstrates how to define a schema at query time:
   
   ```sql
   SELECT * FROM table(cp.`pdf/schools.pdf` (type => 'pdf', combinePages => true, 
   schema => 'inline=(`Last Name` VARCHAR, `First Name Address` VARCHAR, 
   `field_0` VARCHAR, `City` VARCHAR, `State` VARCHAR, `Zip` VARCHAR, 
   `field_1` VARCHAR, `Occupation Employer` VARCHAR, 
   `Date` VARCHAR, `field_2` DATE properties {`drill.format` = `M/d/yyyy`}, 
   `Amount` DOUBLE)')) 
   LIMIT 5
   ```
   
   ## Testing
   Added unit tests


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-961327918


   @cgivre, @dzamo raise good points. So, what is Drill today? Is it still primarily used for distributed queries at scale? Or, as a handy desktop tool for data scientists? Probably both. The problem is, a tool that tries to be both a desert topping and a floor wax (let's see how old the readers are with this one), ends up being good at neither.
   
   One approach, if we had resources, would be to create a Drill Desktop that is optimized for that case and encourages all kinds of specialized data connectors. Create an easy way to define those connectors (YAML files created by specialized web apps?) Ensure Drill has good integration with Jupyter and the other usual suspects.
   
   Another approach, if we had resources, is the oft-discussed idea of separating the less-common plugins from the Drill core. Work started on this: to create an extension mechanism that made this possible. (Today, most plugins need quite a bit of Drill internal code.)
   
   So, no harm in adding the PDF reader, but I expect usage will be pretty limited just because, for the folks that need it, configuration will be too hard. Better would be a Python or Spark job that extracts the data into a CSV file, then query the CSV file with Drill. Each step could be debugged easily. I can't imagine anyone will want to debug their PDF extraction using Drill's overly generous Java stack traces... 


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-962673076


   @dzamo, the reference was to an early Saturday Night Live skit.
   
   Just to refocus the discussion, my question is really around configuration. When querying CSV, Parquet and the like, it is very clear where the data lies. When querying Excel, PDF, HTML, Word, "the web", it is less clear: there is some amount of data mining needed to say that it is THIS table and not THAT one, that the table spans three pages, etc.
   
   The question is, how does the user specify this? If it were me, I would not want to be tinkering with a JSON storage plugin config, watching my query fail with a Drill stack trace, or wondering why I got no data. Instead, I'd want a tool better suited for the task. Once I had that, if I then wanted to run at scale, I'd want to say, "Drill, just use X and consume the data."
   
   So, the question here is: is the JSON storage plugin config an effective way for a data scientist to mine PDF, Excel, HTML and other messy sources? I don't have an answer, I'm just asking the question.
   
   Again, if we had the ability to have true external plugins, then this could easily be an add-on project. Those who know PDF could go hog wild creating a good solution. But, we don't, so every specialized plugin has to be part of core Drill. Is that good? That's also a question for debate. 


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r757144135



##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  

Review comment:
       Nit: extra leading space.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);

Review comment:
       Better:
   
   ```
         throw UserException
           .dataReadError(e)
           .addContext("Failed to open open input file: %s", split.getPath().toString())
           .addContext(errorContext)
           .build(logger);
   ```

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));
+  }
+
+  private void writeMetadata() {
+    int startingIndex = columnHeaders.size();
+    writers.get(startingIndex).getWriter().setInt(pageCount);
+    writeStringMetadataField(title, startingIndex+1);
+    writeStringMetadataField(author, startingIndex+2);
+    writeStringMetadataField(subject, startingIndex+3);
+    writeStringMetadataField(keywords, startingIndex+4);
+    writeStringMetadataField(creator, startingIndex+5);
+    writeStringMetadataField(producer, startingIndex+6);
+    writeTimestampMetadataField(creationDate, startingIndex+7);
+    writeTimestampMetadataField(modificationDate, startingIndex+8);
+    writeStringMetadataField(trapped, startingIndex+9);
+    writers.get(startingIndex+10).getWriter().setInt(tableCount);
+  }
+
+  private void writeStringMetadataField(String value, int index) {
+    if (value == null) {
+      return;
+    }
+    writers.get(index).getWriter().setString(value);
+  }
+
+  private void writeTimestampMetadataField(Calendar dateValue, int index) {
+    if (dateValue == null) {
+      return;
+    }
+
+    writers.get(index).getWriter().setTimestamp(Instant.ofEpochMilli(dateValue.getTimeInMillis()));
+  }
+
+  private void addUnknownColumnToSchemaAndCreateWriter (TupleWriter rowWriter, String name) {
+    int index = rowWriter.tupleSchema().index(name);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(name, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
+      index = rowWriter.addColumn(colSchema);
+    }
+    ScalarWriter colWriter = rowWriter.scalar(index);
+
+    // Create a new column name which will be field_n
+    String newColumnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+    unregisteredColumnCount++;
+
+    // Add a new writer.  Since we want the metadata always to be at the end of the schema, we must track the metadata
+    // index and add this before the metadata, so that the column index tracks with the writer index.
+    writers.add(metadataIndex, new StringPdfColumnWriter(1, newColumnName, (RowSetLoader) rowWriter));
+
+    // Increment the metadata index
+    metadataIndex++;
+  }
+
+  private TupleMetadata buildSchema() {
+    Table table = tables.get(startingTableIndex);
+    columns = table.getColCount();
+    rows = table.getRowCount();
+
+    // Get column header names
+    columnHeaders = Utils.extractRowValues(table);
+
+    // Add columns to table
+    int index = 0;
+    for (String columnName : columnHeaders) {
+      if (Strings.isNullOrEmpty(columnName)) {
+        columnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+        columnHeaders.set(index, columnName);
+        unregisteredColumnCount++;
+      }
+      builder.addNullable(columnName, MinorType.VARCHAR);
+      index++;
+    }
+
+    return builder.buildSchema();
+  }
+
+  private void buildWriterList() {
+    for (String header : columnHeaders) {
+      writers.add(new StringPdfColumnWriter(columnHeaders.indexOf(header), header, rowWriter));
+    }
+  }
+
+  private void buildWriterListFromProvidedSchema(TupleMetadata schema) {
+    if (schema == null) {
+      buildWriterList();
+      return;
+    }
+    int counter = 0;
+    for (MaterializedField field: schema.toFieldList()) {
+      String fieldName = field.getName();
+      MinorType type = field.getType().getMinorType();
+      columnHeaders.add(fieldName);
+
+      switch (type) {
+        case VARCHAR:
+          writers.add(new StringPdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case SMALLINT:
+        case TINYINT:
+        case INT:
+          writers.add(new IntPdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case BIGINT:
+          writers.add(new BigIntPdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case FLOAT4:
+        case FLOAT8:
+          writers.add(new DoublePdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case DATE:
+          writers.add(new DatePdfColumnWriter(counter, fieldName, rowWriter, negotiator));
+          break;
+        case TIME:
+          writers.add(new TimePdfColumnWriter(counter, fieldName, rowWriter, negotiator));
+          break;
+        case TIMESTAMP:
+          writers.add(new TimestampPdfColumnWriter(counter, fieldName, rowWriter, negotiator));
+          break;
+        default:
+          throw UserException.unsupportedError()
+            .message("PDF Reader with provided schema does not support " + type.name() + " data type.")
+            .addContext(errorContext)
+            .build(logger);
+      }
+    }
+  }
+
+  public abstract static class PdfColumnWriter {
+    final String columnName;
+    final ScalarWriter writer;
+    final int columnIndex;
+
+    public PdfColumnWriter(int columnIndex, String columnName, ScalarWriter writer) {
+      this.columnIndex = columnIndex;
+      this.columnName = columnName;
+      this.writer = writer;
+    }
+
+    public abstract void load (RectangularTextContainer<?> cell);
+
+    public ScalarWriter getWriter() {
+      return writer;
+    }
+  }
+
+  public static class IntPdfColumnWriter extends PdfColumnWriter {
+    IntPdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setInt(Integer.parseInt(cell.getText()));
+    }
+  }
+
+  public static class BigIntPdfColumnWriter extends PdfColumnWriter {
+    BigIntPdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setLong(Long.parseLong(cell.getText()));

Review comment:
       First, bravo for using this trick to optimize the innermost per-column load path.
   
   What happens if the parse fails? Need to catch the number format exception and convert it to a `UserException`. Or, you could let the EVF do it for you with its column converters which do exactly this work.
   
   Aside: if this were a quick & dirty add-on, I'd not worry about error handling or performance. But, because it is part of core Drill, and Drill must scale, we have to worry about all these details, even if maybe no one will ever read a zillion huge PDFs. Because, if I don't, how do I then turn around and ask the next reader to provide a scalable solution?

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");

Review comment:
       Oh, my. Drill can run embedded. Suppose I'm running Drill embedded in my AWT application. Will this break things?
   
   Drill is multi-threaded: we an have a dozen readers all setting this same system property concurrently with other code reading it. Is this a safe thing to do?
   
   More fundamentally, should we be mucking about with shared system properties in a reader? Probably not.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());

Review comment:
       Think concurrency. Found {} tables in which file?

##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 
+
+### Merging Pages
+The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
+will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.
+
+## Configuration
+To configure the PDF reader, simply add the information below to the `formats` section of a file base storage plugin. 

Review comment:
       What is a "file base storage plugin"? Maybe add "such as dfs".

##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 

Review comment:
       Probably want to link to the reference for provided schema. I suspect it's not a well-known feature. Also, explain what the provided schema provides (fields within the table?)

##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 
+
+### Merging Pages
+The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
+will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.
+
+## Configuration
+To configure the PDF reader, simply add the information below to the `formats` section of a file base storage plugin. 
+
+```json
+"pdf": {
+  "type": "pdf",
+  "extensions": [
+    "pdf"
+  ],
+  "extractHeaders": true,
+  "combinePages": false
+}
+```
+The avaialable options are:
+* `extractHeaders`: Extracts the first row of any tables as the header row.  If set to false, Drill will assign column names of `field_0`, `field_1` to each column.
+* `combinePages`: Merges multipage tables together.
+* `defaultTableIndex`:  Allows you to query different tables within the PDF file. Index starts at `0`. 
+
+
+## Accessing Document Metadata Fields
+PDF files have a considerable amount of metadata which can be useful for analysis.  Drill will extract the following fields from every PDF file.  Note that these fields are not
+ projected in star queries and must be selected explicitly.  The document's creator populates these fields and some or all may be empty. With the exception of `_page_count

Review comment:
       Nit: leading space.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();

Review comment:
       Nit:
   
   ```java
       Something row = row.get(i);
       if (!Strings.isNullOrEmpty(row.getText()) {
         writers.get(i).load(row);
       }
   ```

##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 
+
+### Merging Pages
+The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
+will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.
+
+## Configuration
+To configure the PDF reader, simply add the information below to the `formats` section of a file base storage plugin. 
+
+```json
+"pdf": {
+  "type": "pdf",
+  "extensions": [
+    "pdf"
+  ],
+  "extractHeaders": true,
+  "combinePages": false
+}
+```
+The avaialable options are:
+* `extractHeaders`: Extracts the first row of any tables as the header row.  If set to false, Drill will assign column names of `field_0`, `field_1` to each column.
+* `combinePages`: Merges multipage tables together.
+* `defaultTableIndex`:  Allows you to query different tables within the PDF file. Index starts at `0`. 

Review comment:
       It should start with 1: makes it easier for humans to count.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();

Review comment:
       I'd think you'd want to do this before creating the writer. Add columns to the schema before creating the vectors so that we get a simple picture before we start grinding the vector details.

##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 
+
+### Merging Pages
+The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
+will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.
+
+## Configuration
+To configure the PDF reader, simply add the information below to the `formats` section of a file base storage plugin. 
+
+```json
+"pdf": {
+  "type": "pdf",
+  "extensions": [
+    "pdf"
+  ],
+  "extractHeaders": true,
+  "combinePages": false
+}
+```
+The avaialable options are:
+* `extractHeaders`: Extracts the first row of any tables as the header row.  If set to false, Drill will assign column names of `field_0`, `field_1` to each column.

Review comment:
       Explain a bit more. I have two tables with titles "a | b", and "c | d | e". Do I get a union row with "a, b, c, d, e" as fields? Or, just "a, b" as fields? When I read the second table, do I then get "a, b, e"?

##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 
+
+### Merging Pages
+The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
+will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.
+
+## Configuration
+To configure the PDF reader, simply add the information below to the `formats` section of a file base storage plugin. 
+
+```json
+"pdf": {
+  "type": "pdf",
+  "extensions": [
+    "pdf"
+  ],
+  "extractHeaders": true,
+  "combinePages": false
+}
+```
+The avaialable options are:
+* `extractHeaders`: Extracts the first row of any tables as the header row.  If set to false, Drill will assign column names of `field_0`, `field_1` to each column.
+* `combinePages`: Merges multipage tables together.
+* `defaultTableIndex`:  Allows you to query different tables within the PDF file. Index starts at `0`. 
+
+
+## Accessing Document Metadata Fields
+PDF files have a considerable amount of metadata which can be useful for analysis.  Drill will extract the following fields from every PDF file.  Note that these fields are not
+ projected in star queries and must be selected explicitly.  The document's creator populates these fields and some or all may be empty. With the exception of `_page_count
+ ` which is an `INT` and the two date fields, all the other fields are `VARCHAR` fields.
+ 
+ The fields are:
+ * `_page_count`
+ * `_author`
+ * `_title`
+ * `_keywords`
+ * `_creator`
+ * `_producer`
+ * `_creation_date`
+ * `_modification_date`
+ * `_trapped`
+ * `_table_count`

Review comment:
       Here, I assume `_author` will repeat for every table row? That `_page_count` will increase and gives the page number? Or, is constant and will always be, say, 25 if that's the number of pages?
   
   Does `_table_count` stay fixed at the number of tables (5, say), or does it count tables (1, 2, 3, 4, 5)?
   
   What is `_trapped`? Is there some PDF standard we could point to where this stuff is defined?
   
   Actually, there is a larger question. If I'm exploring a pile of PDFs, I may want to extract metadata, such as the above. If I'm extracting tables, I don't need the metadata repeated per table row.
   
   Should the plugin allow two query types? Either metadata or tables? Can this be done with some kind of Calcite trickery? `FROM "foo.pdf"` means the tables, `FROM "foo.pdf.metadata"` means the metadata?
   
   Or, we check and if we see only metadata columns, we return one row per file rather than one row per table row?
   
   Wouldn't something like this be useful to extract metadata from a PDF with no tables?

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {

Review comment:
       Is this for `LIMIT` pushdown? If so, I'd suggest we add it to the RSL itself as a new parameter. That way, the RSL knows to declare EOF and stop reading from the input once the limit is reached. It can also ignore any rows past the limit.

##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 
+
+### Merging Pages
+The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
+will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.

Review comment:
       Suppose my file has three, mutually incompatible tables, and I only want to read the first. Can I? If so, how?
   
   Or, do I read all of them (sales of apples by state, shelf life of various apple kinds, list of largest apple growers) into a big messy, combined table, then use a `WHERE` clause to try to keep just the shelf life info?

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");

Review comment:
       See comment above. We shouldn't be doing this!

##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 
+
+### Merging Pages
+The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
+will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.
+
+## Configuration
+To configure the PDF reader, simply add the information below to the `formats` section of a file base storage plugin. 
+
+```json
+"pdf": {
+  "type": "pdf",
+  "extensions": [
+    "pdf"
+  ],
+  "extractHeaders": true,
+  "combinePages": false
+}
+```
+The avaialable options are:
+* `extractHeaders`: Extracts the first row of any tables as the header row.  If set to false, Drill will assign column names of `field_0`, `field_1` to each column.
+* `combinePages`: Merges multipage tables together.
+* `defaultTableIndex`:  Allows you to query different tables within the PDF file. Index starts at `0`. 
+
+
+## Accessing Document Metadata Fields
+PDF files have a considerable amount of metadata which can be useful for analysis.  Drill will extract the following fields from every PDF file.  Note that these fields are not
+ projected in star queries and must be selected explicitly.  The document's creator populates these fields and some or all may be empty. With the exception of `_page_count
+ ` which is an `INT` and the two date fields, all the other fields are `VARCHAR` fields.

Review comment:
       Nit: I think the newline gets included in the quoted bit.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();

Review comment:
       Suggestion: create a `Metadata` class to hold this stuff rather than adding it to the base reader. More modular.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);

Review comment:
       Does this scale? Get all data on open, cached in memory? In each of a dozen or more PDF readers in the same process? (This is why having this kind of plugin in the core is problematic: won't scale for PDFs with large numbers of tables.)

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));

Review comment:
       Again, want to do this while building the schema, no on the live writer.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.
+   * @param document The input PDF document to search for tables
+   * @return A list of tables found in the document.
+   */
+  public static List<Table> extractTablesFromPDF(PDDocument document) {
+    return extractTablesFromPDF(document, DEFAULT_ALGORITHM);
+  }
+
+  public static List<Table> extractTablesFromPDF(PDDocument document, ExtractionAlgorithm algorithm) {
+    System.setProperty("java.awt.headless", "true");
+
+    NurminenDetectionAlgorithm detectionAlgorithm = new NurminenDetectionAlgorithm();
+
+    ExtractionAlgorithm algExtractor;
+
+    SpreadsheetExtractionAlgorithm extractor = new SpreadsheetExtractionAlgorithm();
+
+    ObjectExtractor objectExtractor = new ObjectExtractor(document);
+    PageIterator pages = objectExtractor.extract();
+    List<Table> tables= new ArrayList<>();

Review comment:
       Nit: space before `=`.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {

Review comment:
       Given that the file is read only in `openFile()`, no need to keep the file stream around: just close it in `openFile()`.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));
+  }
+
+  private void writeMetadata() {
+    int startingIndex = columnHeaders.size();
+    writers.get(startingIndex).getWriter().setInt(pageCount);
+    writeStringMetadataField(title, startingIndex+1);
+    writeStringMetadataField(author, startingIndex+2);
+    writeStringMetadataField(subject, startingIndex+3);
+    writeStringMetadataField(keywords, startingIndex+4);
+    writeStringMetadataField(creator, startingIndex+5);
+    writeStringMetadataField(producer, startingIndex+6);
+    writeTimestampMetadataField(creationDate, startingIndex+7);
+    writeTimestampMetadataField(modificationDate, startingIndex+8);
+    writeStringMetadataField(trapped, startingIndex+9);
+    writers.get(startingIndex+10).getWriter().setInt(tableCount);
+  }
+
+  private void writeStringMetadataField(String value, int index) {
+    if (value == null) {
+      return;
+    }
+    writers.get(index).getWriter().setString(value);

Review comment:
       Cache the writer in the proposed Metadata class. Cache the values in an array, then, for speed:
   
   ```java
     for (int i = 0; i < columns.length; i++) {
       writers[i].setObject(columns[i]);
     }
   ```

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfFormatConfig.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+@JsonTypeName(PdfFormatPlugin.DEFAULT_NAME)
+public class PdfFormatConfig implements FormatPluginConfig {
+
+  private final List<String> extensions;
+  private final boolean extractHeaders;
+  private final String extractionAlgorithm;
+  private final boolean combinePages;
+  private final int defaultTableIndex;
+
+  @JsonCreator
+  public PdfFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                         @JsonProperty("extractHeaders") boolean extractHeaders,
+                         @JsonProperty("extractionAlgorithm") String extractionAlgorithm,
+                         @JsonProperty("combinePages") boolean combinePages,
+                         @JsonProperty("defaultTableIndex") int defaultTableIndex) {
+    this.extensions = extensions == null
+      ? Collections.singletonList("pdf")
+      : ImmutableList.copyOf(extensions);
+    this.extractHeaders = extractHeaders;
+    this.extractionAlgorithm = extractionAlgorithm;
+    this.combinePages = combinePages;
+    this.defaultTableIndex = defaultTableIndex;
+  }
+
+  public PdfBatchReader.PdfReaderConfig getReaderConfig(PdfFormatPlugin plugin) {
+    return new PdfBatchReader.PdfReaderConfig(plugin);
+  }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  @JsonProperty("extensions")
+  public List<String> getExtensions() {
+    return extensions;
+  }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  @JsonProperty("combinePages")
+  public boolean getCombinePages() { return combinePages; }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  @JsonProperty("extractHeaders")
+  public boolean getExtractHeaders() {
+    return extractHeaders;
+  }
+
+  @JsonProperty("extractionAlgorithm")

Review comment:
       `@JsonInclude(Include.NON_NULL)`?

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));
+  }
+
+  private void writeMetadata() {
+    int startingIndex = columnHeaders.size();
+    writers.get(startingIndex).getWriter().setInt(pageCount);
+    writeStringMetadataField(title, startingIndex+1);
+    writeStringMetadataField(author, startingIndex+2);
+    writeStringMetadataField(subject, startingIndex+3);
+    writeStringMetadataField(keywords, startingIndex+4);
+    writeStringMetadataField(creator, startingIndex+5);
+    writeStringMetadataField(producer, startingIndex+6);
+    writeTimestampMetadataField(creationDate, startingIndex+7);
+    writeTimestampMetadataField(modificationDate, startingIndex+8);
+    writeStringMetadataField(trapped, startingIndex+9);
+    writers.get(startingIndex+10).getWriter().setInt(tableCount);
+  }
+
+  private void writeStringMetadataField(String value, int index) {
+    if (value == null) {
+      return;
+    }
+    writers.get(index).getWriter().setString(value);
+  }
+
+  private void writeTimestampMetadataField(Calendar dateValue, int index) {
+    if (dateValue == null) {
+      return;
+    }
+
+    writers.get(index).getWriter().setTimestamp(Instant.ofEpochMilli(dateValue.getTimeInMillis()));
+  }
+
+  private void addUnknownColumnToSchemaAndCreateWriter (TupleWriter rowWriter, String name) {
+    int index = rowWriter.tupleSchema().index(name);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(name, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
+      index = rowWriter.addColumn(colSchema);
+    }
+    ScalarWriter colWriter = rowWriter.scalar(index);
+
+    // Create a new column name which will be field_n
+    String newColumnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+    unregisteredColumnCount++;
+
+    // Add a new writer.  Since we want the metadata always to be at the end of the schema, we must track the metadata
+    // index and add this before the metadata, so that the column index tracks with the writer index.
+    writers.add(metadataIndex, new StringPdfColumnWriter(1, newColumnName, (RowSetLoader) rowWriter));
+
+    // Increment the metadata index
+    metadataIndex++;
+  }
+
+  private TupleMetadata buildSchema() {
+    Table table = tables.get(startingTableIndex);
+    columns = table.getColCount();
+    rows = table.getRowCount();
+
+    // Get column header names
+    columnHeaders = Utils.extractRowValues(table);
+
+    // Add columns to table
+    int index = 0;
+    for (String columnName : columnHeaders) {
+      if (Strings.isNullOrEmpty(columnName)) {
+        columnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+        columnHeaders.set(index, columnName);
+        unregisteredColumnCount++;
+      }
+      builder.addNullable(columnName, MinorType.VARCHAR);
+      index++;
+    }
+
+    return builder.buildSchema();
+  }
+
+  private void buildWriterList() {
+    for (String header : columnHeaders) {
+      writers.add(new StringPdfColumnWriter(columnHeaders.indexOf(header), header, rowWriter));
+    }
+  }
+
+  private void buildWriterListFromProvidedSchema(TupleMetadata schema) {
+    if (schema == null) {
+      buildWriterList();
+      return;
+    }
+    int counter = 0;
+    for (MaterializedField field: schema.toFieldList()) {
+      String fieldName = field.getName();
+      MinorType type = field.getType().getMinorType();
+      columnHeaders.add(fieldName);
+
+      switch (type) {
+        case VARCHAR:
+          writers.add(new StringPdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case SMALLINT:
+        case TINYINT:
+        case INT:
+          writers.add(new IntPdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case BIGINT:
+          writers.add(new BigIntPdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case FLOAT4:
+        case FLOAT8:
+          writers.add(new DoublePdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case DATE:
+          writers.add(new DatePdfColumnWriter(counter, fieldName, rowWriter, negotiator));
+          break;
+        case TIME:
+          writers.add(new TimePdfColumnWriter(counter, fieldName, rowWriter, negotiator));
+          break;
+        case TIMESTAMP:
+          writers.add(new TimestampPdfColumnWriter(counter, fieldName, rowWriter, negotiator));
+          break;
+        default:
+          throw UserException.unsupportedError()
+            .message("PDF Reader with provided schema does not support " + type.name() + " data type.")
+            .addContext(errorContext)
+            .build(logger);
+      }
+    }
+  }
+
+  public abstract static class PdfColumnWriter {
+    final String columnName;
+    final ScalarWriter writer;
+    final int columnIndex;
+
+    public PdfColumnWriter(int columnIndex, String columnName, ScalarWriter writer) {
+      this.columnIndex = columnIndex;
+      this.columnName = columnName;
+      this.writer = writer;
+    }
+
+    public abstract void load (RectangularTextContainer<?> cell);
+
+    public ScalarWriter getWriter() {
+      return writer;
+    }
+  }
+
+  public static class IntPdfColumnWriter extends PdfColumnWriter {
+    IntPdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setInt(Integer.parseInt(cell.getText()));
+    }
+  }
+
+  public static class BigIntPdfColumnWriter extends PdfColumnWriter {
+    BigIntPdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setLong(Long.parseLong(cell.getText()));
+    }
+  }
+
+  public static class DoublePdfColumnWriter extends PdfColumnWriter {
+    DoublePdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setDouble(Double.parseDouble(cell.getText()));
+    }
+  }
+
+  public static class StringPdfColumnWriter extends PdfColumnWriter {
+    StringPdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setString(cell.getText());
+    }
+  }
+
+  public static class DatePdfColumnWriter extends PdfColumnWriter {
+    private String dateFormat;
+
+    DatePdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter, FileScanFramework.FileSchemaNegotiator negotiator) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+
+      ColumnMetadata metadata = negotiator.providedSchema().metadata(columnName);
+      if (metadata != null) {
+        this.dateFormat = metadata.property("drill.format");
+      }
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      LocalDate localDate;
+      if (Strings.isNullOrEmpty(this.dateFormat)) {
+       localDate = LocalDate.parse(cell.getText());

Review comment:
       Nit: missing leading space.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);

Review comment:
       Do this in the proposed metadata class.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();

Review comment:
       Here's were we've got an issue: can't see metadata without there being tables and having to fetch them.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));
+  }
+
+  private void writeMetadata() {
+    int startingIndex = columnHeaders.size();
+    writers.get(startingIndex).getWriter().setInt(pageCount);
+    writeStringMetadataField(title, startingIndex+1);
+    writeStringMetadataField(author, startingIndex+2);
+    writeStringMetadataField(subject, startingIndex+3);
+    writeStringMetadataField(keywords, startingIndex+4);
+    writeStringMetadataField(creator, startingIndex+5);
+    writeStringMetadataField(producer, startingIndex+6);
+    writeTimestampMetadataField(creationDate, startingIndex+7);
+    writeTimestampMetadataField(modificationDate, startingIndex+8);
+    writeStringMetadataField(trapped, startingIndex+9);
+    writers.get(startingIndex+10).getWriter().setInt(tableCount);
+  }
+
+  private void writeStringMetadataField(String value, int index) {
+    if (value == null) {
+      return;
+    }
+    writers.get(index).getWriter().setString(value);
+  }
+
+  private void writeTimestampMetadataField(Calendar dateValue, int index) {
+    if (dateValue == null) {
+      return;
+    }
+
+    writers.get(index).getWriter().setTimestamp(Instant.ofEpochMilli(dateValue.getTimeInMillis()));
+  }
+
+  private void addUnknownColumnToSchemaAndCreateWriter (TupleWriter rowWriter, String name) {
+    int index = rowWriter.tupleSchema().index(name);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(name, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
+      index = rowWriter.addColumn(colSchema);
+    }
+    ScalarWriter colWriter = rowWriter.scalar(index);
+
+    // Create a new column name which will be field_n
+    String newColumnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+    unregisteredColumnCount++;
+
+    // Add a new writer.  Since we want the metadata always to be at the end of the schema, we must track the metadata
+    // index and add this before the metadata, so that the column index tracks with the writer index.
+    writers.add(metadataIndex, new StringPdfColumnWriter(1, newColumnName, (RowSetLoader) rowWriter));
+
+    // Increment the metadata index
+    metadataIndex++;
+  }
+
+  private TupleMetadata buildSchema() {
+    Table table = tables.get(startingTableIndex);
+    columns = table.getColCount();
+    rows = table.getRowCount();
+
+    // Get column header names
+    columnHeaders = Utils.extractRowValues(table);
+
+    // Add columns to table
+    int index = 0;
+    for (String columnName : columnHeaders) {
+      if (Strings.isNullOrEmpty(columnName)) {
+        columnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+        columnHeaders.set(index, columnName);
+        unregisteredColumnCount++;
+      }
+      builder.addNullable(columnName, MinorType.VARCHAR);
+      index++;
+    }
+
+    return builder.buildSchema();
+  }
+
+  private void buildWriterList() {

Review comment:
       We build the writers two ways depending on provided schema or not. With V2, the idea is to build the schema once, give it to the `SchemaNegotiator`, and get back a writer with all the needed columns. Adding columns to a writer is a bit old-school, needed only for formats, such as JSON, where we really don't know the schema until we read a record.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));
+  }
+
+  private void writeMetadata() {
+    int startingIndex = columnHeaders.size();
+    writers.get(startingIndex).getWriter().setInt(pageCount);
+    writeStringMetadataField(title, startingIndex+1);
+    writeStringMetadataField(author, startingIndex+2);
+    writeStringMetadataField(subject, startingIndex+3);
+    writeStringMetadataField(keywords, startingIndex+4);
+    writeStringMetadataField(creator, startingIndex+5);
+    writeStringMetadataField(producer, startingIndex+6);
+    writeTimestampMetadataField(creationDate, startingIndex+7);
+    writeTimestampMetadataField(modificationDate, startingIndex+8);
+    writeStringMetadataField(trapped, startingIndex+9);
+    writers.get(startingIndex+10).getWriter().setInt(tableCount);
+  }
+
+  private void writeStringMetadataField(String value, int index) {
+    if (value == null) {
+      return;
+    }
+    writers.get(index).getWriter().setString(value);
+  }
+
+  private void writeTimestampMetadataField(Calendar dateValue, int index) {
+    if (dateValue == null) {
+      return;
+    }
+
+    writers.get(index).getWriter().setTimestamp(Instant.ofEpochMilli(dateValue.getTimeInMillis()));
+  }
+
+  private void addUnknownColumnToSchemaAndCreateWriter (TupleWriter rowWriter, String name) {
+    int index = rowWriter.tupleSchema().index(name);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(name, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
+      index = rowWriter.addColumn(colSchema);
+    }
+    ScalarWriter colWriter = rowWriter.scalar(index);
+
+    // Create a new column name which will be field_n
+    String newColumnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+    unregisteredColumnCount++;
+
+    // Add a new writer.  Since we want the metadata always to be at the end of the schema, we must track the metadata
+    // index and add this before the metadata, so that the column index tracks with the writer index.
+    writers.add(metadataIndex, new StringPdfColumnWriter(1, newColumnName, (RowSetLoader) rowWriter));
+
+    // Increment the metadata index
+    metadataIndex++;
+  }
+
+  private TupleMetadata buildSchema() {
+    Table table = tables.get(startingTableIndex);
+    columns = table.getColCount();
+    rows = table.getRowCount();
+
+    // Get column header names
+    columnHeaders = Utils.extractRowValues(table);
+
+    // Add columns to table
+    int index = 0;
+    for (String columnName : columnHeaders) {
+      if (Strings.isNullOrEmpty(columnName)) {
+        columnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+        columnHeaders.set(index, columnName);
+        unregisteredColumnCount++;
+      }
+      builder.addNullable(columnName, MinorType.VARCHAR);
+      index++;
+    }
+
+    return builder.buildSchema();
+  }
+
+  private void buildWriterList() {
+    for (String header : columnHeaders) {
+      writers.add(new StringPdfColumnWriter(columnHeaders.indexOf(header), header, rowWriter));
+    }
+  }
+
+  private void buildWriterListFromProvidedSchema(TupleMetadata schema) {
+    if (schema == null) {
+      buildWriterList();
+      return;
+    }
+    int counter = 0;
+    for (MaterializedField field: schema.toFieldList()) {
+      String fieldName = field.getName();
+      MinorType type = field.getType().getMinorType();
+      columnHeaders.add(fieldName);
+
+      switch (type) {
+        case VARCHAR:
+          writers.add(new StringPdfColumnWriter(counter, fieldName, rowWriter));

Review comment:
       V2 does this for you: builds writers based on the declared field type.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {

Review comment:
       As far as I can tell, the only utility here is the extractor. Should it be `TableExtractor` instead?

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.
+   * @param document The input PDF document to search for tables
+   * @return A list of tables found in the document.
+   */
+  public static List<Table> extractTablesFromPDF(PDDocument document) {
+    return extractTablesFromPDF(document, DEFAULT_ALGORITHM);
+  }
+
+  public static List<Table> extractTablesFromPDF(PDDocument document, ExtractionAlgorithm algorithm) {
+    System.setProperty("java.awt.headless", "true");
+
+    NurminenDetectionAlgorithm detectionAlgorithm = new NurminenDetectionAlgorithm();
+
+    ExtractionAlgorithm algExtractor;
+
+    SpreadsheetExtractionAlgorithm extractor = new SpreadsheetExtractionAlgorithm();
+
+    ObjectExtractor objectExtractor = new ObjectExtractor(document);
+    PageIterator pages = objectExtractor.extract();
+    List<Table> tables= new ArrayList<>();
+    while (pages.hasNext()) {
+      Page page = pages.next();
+
+      algExtractor = algorithm;
+      List<Rectangle> tablesOnPage = detectionAlgorithm.detect(page);
+
+      for (Rectangle guessRect : tablesOnPage) {
+        Page guess = page.getArea(guessRect);
+        tables.addAll(algExtractor.extract(guess));
+      }
+    }
+
+    try {
+      objectExtractor.close();
+    } catch (Exception e) {
+      logger.debug("Error closing Object extractor.");

Review comment:
       Should we be doing something useful with any exceptions, such as telling the user that the extract failed?
   
   This line should say something about which file failed. Else, in a Drill that processed 100 PDF files, how will we know where to look for problems?

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.
+   * @param document The input PDF document to search for tables
+   * @return A list of tables found in the document.
+   */
+  public static List<Table> extractTablesFromPDF(PDDocument document) {
+    return extractTablesFromPDF(document, DEFAULT_ALGORITHM);
+  }
+
+  public static List<Table> extractTablesFromPDF(PDDocument document, ExtractionAlgorithm algorithm) {
+    System.setProperty("java.awt.headless", "true");

Review comment:
       Again, we probably shouldn't be doing this.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.

Review comment:
       Maybe reference where this algorithm came from? What are the other algorithms?

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.
+   * @param document The input PDF document to search for tables
+   * @return A list of tables found in the document.
+   */
+  public static List<Table> extractTablesFromPDF(PDDocument document) {
+    return extractTablesFromPDF(document, DEFAULT_ALGORITHM);
+  }
+
+  public static List<Table> extractTablesFromPDF(PDDocument document, ExtractionAlgorithm algorithm) {
+    System.setProperty("java.awt.headless", "true");
+
+    NurminenDetectionAlgorithm detectionAlgorithm = new NurminenDetectionAlgorithm();
+
+    ExtractionAlgorithm algExtractor;
+
+    SpreadsheetExtractionAlgorithm extractor = new SpreadsheetExtractionAlgorithm();
+
+    ObjectExtractor objectExtractor = new ObjectExtractor(document);
+    PageIterator pages = objectExtractor.extract();
+    List<Table> tables= new ArrayList<>();
+    while (pages.hasNext()) {
+      Page page = pages.next();
+
+      algExtractor = algorithm;
+      List<Rectangle> tablesOnPage = detectionAlgorithm.detect(page);
+
+      for (Rectangle guessRect : tablesOnPage) {
+        Page guess = page.getArea(guessRect);
+        tables.addAll(algExtractor.extract(guess));
+      }
+    }
+
+    try {
+      objectExtractor.close();

Review comment:
       How critical is this `close()`? Should it be in a `finally` block in case any of the above lines throws an exception?
   

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.
+   * @param document The input PDF document to search for tables
+   * @return A list of tables found in the document.
+   */
+  public static List<Table> extractTablesFromPDF(PDDocument document) {
+    return extractTablesFromPDF(document, DEFAULT_ALGORITHM);
+  }
+
+  public static List<Table> extractTablesFromPDF(PDDocument document, ExtractionAlgorithm algorithm) {
+    System.setProperty("java.awt.headless", "true");
+
+    NurminenDetectionAlgorithm detectionAlgorithm = new NurminenDetectionAlgorithm();
+
+    ExtractionAlgorithm algExtractor;
+
+    SpreadsheetExtractionAlgorithm extractor = new SpreadsheetExtractionAlgorithm();
+
+    ObjectExtractor objectExtractor = new ObjectExtractor(document);
+    PageIterator pages = objectExtractor.extract();
+    List<Table> tables= new ArrayList<>();
+    while (pages.hasNext()) {
+      Page page = pages.next();
+
+      algExtractor = algorithm;
+      List<Rectangle> tablesOnPage = detectionAlgorithm.detect(page);
+
+      for (Rectangle guessRect : tablesOnPage) {
+        Page guess = page.getArea(guessRect);
+        tables.addAll(algExtractor.extract(guess));
+      }
+    }
+
+    try {
+      objectExtractor.close();
+    } catch (Exception e) {
+      logger.debug("Error closing Object extractor.");
+    }
+
+    return tables;
+  }
+
+  /**
+   * Returns the values contained in a PDF Table row
+   * @param table The source table
+   * @return A list of the header rows
+   */
+  public static List<String> extractRowValues(Table table) {
+    List<RectangularTextContainer> firstRow = table.getRows().get(0);
+    List<String> values = new ArrayList<>();
+
+    if (firstRow != null) {
+      for (int i =0; i < firstRow.size(); i++) {

Review comment:
       Nit: space after `=`.
   
   What happens if the rows have different sizes? Is that possible if we guessed wrong about the table? If the table merges cells? How should we handle such issues? Do we have tests for them?

##########
File path: contrib/format-pdf/src/test/resources/logback-test.xml
##########
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8" ?>

Review comment:
       Not clear if we want to do this. IIRC, Logback gets very confused if there are multiple config files on the class path: it prints a warning an arbitrarily picks one.
   
   Why do we need a separate config file for this module?

##########
File path: contrib/format-pdf/src/main/resources/bootstrap-format-plugins.json
##########
@@ -0,0 +1,43 @@
+{
+  "storage":{
+    "dfs": {
+      "type": "file",
+      "formats": {
+        "pdf": {
+          "type": "pdf",
+          "extensions": [
+            "pdf"
+          ],
+          "extractHeaders": true,
+          "combinePages": false
+        }
+      }
+    },
+    "cp": {

Review comment:
       Will we provide any PDFs built into Drill?

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfFormatConfig.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+@JsonTypeName(PdfFormatPlugin.DEFAULT_NAME)
+public class PdfFormatConfig implements FormatPluginConfig {
+
+  private final List<String> extensions;
+  private final boolean extractHeaders;
+  private final String extractionAlgorithm;
+  private final boolean combinePages;
+  private final int defaultTableIndex;
+
+  @JsonCreator
+  public PdfFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                         @JsonProperty("extractHeaders") boolean extractHeaders,
+                         @JsonProperty("extractionAlgorithm") String extractionAlgorithm,
+                         @JsonProperty("combinePages") boolean combinePages,
+                         @JsonProperty("defaultTableIndex") int defaultTableIndex) {
+    this.extensions = extensions == null
+      ? Collections.singletonList("pdf")

Review comment:
       `extractionAlgorithm` is not described in the README.

##########
File path: contrib/format-pdf/src/test/resources/logback-test.xml
##########
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+
+    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.
+
+-->
+<configuration>
+  <if condition='property("drill.lilith.enable").equalsIgnoreCase("true")'>
+    <then>
+      <appender name="SOCKET" class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
+        <Compressing>true</Compressing>
+        <ReconnectionDelay>10000</ReconnectionDelay>
+        <IncludeCallerData>true</IncludeCallerData>
+        <RemoteHosts>${LILITH_HOSTNAME:-localhost}</RemoteHosts>
+      </appender>
+      <logger name="org.apache.drill" additivity="false">
+        <level value="DEBUG"/>
+        <appender-ref ref="SOCKET"/>
+      </logger>
+      <logger name="query.logger" additivity="false">
+        <level value="ERROR"/>
+        <appender-ref ref="SOCKET"/>
+      </logger>
+      <logger name="org.apache.drill.exec.store.pdf">
+        <level value="DEBUG"/>
+        <appender-ref ref="SOCKET"/>
+      </logger>
+    </then>
+  </if>
+
+  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <!-- encoders are assigned the type
+         ch.qos.logback.classic.encoder.PatternLayoutEncoder by default -->
+    <encoder>
+      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
+    </encoder>
+  </appender>
+
+  <logger name="org.apache.drill.exec.store.pdf" additivity="false">
+    <level value="DEBUG" />
+    <appender-ref ref="STDOUT" />
+  </logger>
+</configuration>

Review comment:
       Missing newline.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&

Review comment:
       If so, I'm a bit confused by the comparison of row count and table count. Imagine each table has 100K rows. It would have to split across Drill batches. And, if each table has 3 rows, all tables should be combined in a single Drill batch.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfFormatConfig.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+@JsonTypeName(PdfFormatPlugin.DEFAULT_NAME)
+public class PdfFormatConfig implements FormatPluginConfig {
+
+  private final List<String> extensions;
+  private final boolean extractHeaders;
+  private final String extractionAlgorithm;
+  private final boolean combinePages;
+  private final int defaultTableIndex;
+
+  @JsonCreator
+  public PdfFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                         @JsonProperty("extractHeaders") boolean extractHeaders,
+                         @JsonProperty("extractionAlgorithm") String extractionAlgorithm,
+                         @JsonProperty("combinePages") boolean combinePages,
+                         @JsonProperty("defaultTableIndex") int defaultTableIndex) {
+    this.extensions = extensions == null
+      ? Collections.singletonList("pdf")
+      : ImmutableList.copyOf(extensions);
+    this.extractHeaders = extractHeaders;
+    this.extractionAlgorithm = extractionAlgorithm;
+    this.combinePages = combinePages;
+    this.defaultTableIndex = defaultTableIndex;
+  }
+
+  public PdfBatchReader.PdfReaderConfig getReaderConfig(PdfFormatPlugin plugin) {
+    return new PdfBatchReader.PdfReaderConfig(plugin);
+  }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  @JsonProperty("extensions")
+  public List<String> getExtensions() {
+    return extensions;
+  }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  @JsonProperty("combinePages")
+  public boolean getCombinePages() { return combinePages; }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  @JsonProperty("extractHeaders")
+  public boolean getExtractHeaders() {
+    return extractHeaders;
+  }
+
+  @JsonProperty("extractionAlgorithm")
+  public String getExtractionAlgorithm() {
+    return extractionAlgorithm;
+  }
+
+  @JsonProperty("defaultTableIndex")
+  public int getDefaultTableIndex() { return defaultTableIndex; }

Review comment:
       Not sure I like this name. What is a `defaultTableIndex`? I'm specifying it, right? So, it's not actually a "default." It is a `tableIndex`. The default is what we get if I don't specify anything.
   
   If I don't specify anything I get... all tables? So, I can either get one table or all tables? What if I want tables 1, 5 and 20-32? Should we allow the user to specify a range, such as in the print dialog of any Windows app?
   
   And, if we did, none of these would be "default", would they?




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] luocooong commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
luocooong commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-959477711


   @cgivre Thanks for the new format plugin. Is it ready for review?


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-983645762


   Dear PR author and reviewers.
   
   This is a generic message to say that we would like to merge this PR in time for the 1.20 release.  Currently we're targeting a master branch freeze date of 2021-12-10 (10 Dec).  Please strive to complete development and review by this time, or indicate that the PR will need more time (and how much).
   
   Thank you.


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-997322720


   @paul-rogers 
   Thanks for all your review.  I addressed all your comments (I think) and did the following:
   * Added additional unit tests
   * Refactored table list so that all tables are not read into memory if not requested
   * Added iterator classes to avoid counters in the batch reader
   * Moved metadata collection to separate class
   * Refactored to allow a pdf with no tables to return metadata if requested
   * Added config option for different extraction algorithms.
   * General code cleanup
   
   I removed all but one of the `System.env` calls and I'm a little stuck on this.  The reason I added this line is that when querying a PDF with Drill in embedded mode, it opens an additional java window.  This does not occur when running unit tests which makes for difficult debugging.   I'm going to keep digging into this, but I was wondering if you could take a look at the rest of the revisions in the mean time?   The issue seems to be in either Tabula or PdfBox, which are the underlying libraries that read the PDF file. Thanks!


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r774243742



##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 
+
+### Merging Pages
+The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
+will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.
+
+## Configuration
+To configure the PDF reader, simply add the information below to the `formats` section of a file base storage plugin. 
+
+```json
+"pdf": {
+  "type": "pdf",
+  "extensions": [
+    "pdf"
+  ],
+  "extractHeaders": true,
+  "combinePages": false
+}
+```
+The avaialable options are:
+* `extractHeaders`: Extracts the first row of any tables as the header row.  If set to false, Drill will assign column names of `field_0`, `field_1` to each column.
+* `combinePages`: Merges multipage tables together.
+* `defaultTableIndex`:  Allows you to query different tables within the PDF file. Index starts at `0`. 
+
+
+## Accessing Document Metadata Fields
+PDF files have a considerable amount of metadata which can be useful for analysis.  Drill will extract the following fields from every PDF file.  Note that these fields are not
+ projected in star queries and must be selected explicitly.  The document's creator populates these fields and some or all may be empty. With the exception of `_page_count
+ ` which is an `INT` and the two date fields, all the other fields are `VARCHAR` fields.
+ 
+ The fields are:
+ * `_page_count`
+ * `_author`
+ * `_title`
+ * `_keywords`
+ * `_creator`
+ * `_producer`
+ * `_creation_date`
+ * `_modification_date`
+ * `_trapped`
+ * `_table_count`

Review comment:
       I fixed this such that if a PDF has no tables, you can still get the metadata.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-999686126


   @paul-rogers 
   This PR is now ready for review.  I did some research into the `headless` option and why specifically I was not getting this issue when I ran unit tests, and only when I ran manual queries.  So, the issue was caused when PDFBox opens a document, it calls some AWT libraries that it needs for deciphering PDF files.  (Also probably for writing PDF files)   In any event, this is what was causing the window to open. 
   
   When Drill runs the unit tests, we actually have the `headless` option specified as a Java option.  So, I moved this setting to `drill-config.sh` and included a comment about why it is there.  If someone wants to remove it for some reason, they can and they can disable the PDF functionality simply by not including it in the configs for the storage plugin.  This seems like the right place for this to live. 
   
   I believe that I've addressed all your other comments as well.  Thank you as always for the thorough review!  It's ready for another look.
   
   


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769004049



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));
+  }
+
+  private void writeMetadata() {
+    int startingIndex = columnHeaders.size();
+    writers.get(startingIndex).getWriter().setInt(pageCount);
+    writeStringMetadataField(title, startingIndex+1);
+    writeStringMetadataField(author, startingIndex+2);
+    writeStringMetadataField(subject, startingIndex+3);
+    writeStringMetadataField(keywords, startingIndex+4);
+    writeStringMetadataField(creator, startingIndex+5);
+    writeStringMetadataField(producer, startingIndex+6);
+    writeTimestampMetadataField(creationDate, startingIndex+7);
+    writeTimestampMetadataField(modificationDate, startingIndex+8);
+    writeStringMetadataField(trapped, startingIndex+9);
+    writers.get(startingIndex+10).getWriter().setInt(tableCount);
+  }
+
+  private void writeStringMetadataField(String value, int index) {
+    if (value == null) {
+      return;
+    }
+    writers.get(index).getWriter().setString(value);
+  }
+
+  private void writeTimestampMetadataField(Calendar dateValue, int index) {
+    if (dateValue == null) {
+      return;
+    }
+
+    writers.get(index).getWriter().setTimestamp(Instant.ofEpochMilli(dateValue.getTimeInMillis()));
+  }
+
+  private void addUnknownColumnToSchemaAndCreateWriter (TupleWriter rowWriter, String name) {
+    int index = rowWriter.tupleSchema().index(name);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(name, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.OPTIONAL);
+      index = rowWriter.addColumn(colSchema);
+    }
+    ScalarWriter colWriter = rowWriter.scalar(index);
+
+    // Create a new column name which will be field_n
+    String newColumnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+    unregisteredColumnCount++;
+
+    // Add a new writer.  Since we want the metadata always to be at the end of the schema, we must track the metadata
+    // index and add this before the metadata, so that the column index tracks with the writer index.
+    writers.add(metadataIndex, new StringPdfColumnWriter(1, newColumnName, (RowSetLoader) rowWriter));
+
+    // Increment the metadata index
+    metadataIndex++;
+  }
+
+  private TupleMetadata buildSchema() {
+    Table table = tables.get(startingTableIndex);
+    columns = table.getColCount();
+    rows = table.getRowCount();
+
+    // Get column header names
+    columnHeaders = Utils.extractRowValues(table);
+
+    // Add columns to table
+    int index = 0;
+    for (String columnName : columnHeaders) {
+      if (Strings.isNullOrEmpty(columnName)) {
+        columnName = NEW_FIELD_PREFIX + unregisteredColumnCount;
+        columnHeaders.set(index, columnName);
+        unregisteredColumnCount++;
+      }
+      builder.addNullable(columnName, MinorType.VARCHAR);
+      index++;
+    }
+
+    return builder.buildSchema();
+  }
+
+  private void buildWriterList() {
+    for (String header : columnHeaders) {
+      writers.add(new StringPdfColumnWriter(columnHeaders.indexOf(header), header, rowWriter));
+    }
+  }
+
+  private void buildWriterListFromProvidedSchema(TupleMetadata schema) {
+    if (schema == null) {
+      buildWriterList();
+      return;
+    }
+    int counter = 0;
+    for (MaterializedField field: schema.toFieldList()) {
+      String fieldName = field.getName();
+      MinorType type = field.getType().getMinorType();
+      columnHeaders.add(fieldName);
+
+      switch (type) {
+        case VARCHAR:
+          writers.add(new StringPdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case SMALLINT:
+        case TINYINT:
+        case INT:
+          writers.add(new IntPdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case BIGINT:
+          writers.add(new BigIntPdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case FLOAT4:
+        case FLOAT8:
+          writers.add(new DoublePdfColumnWriter(counter, fieldName, rowWriter));
+          break;
+        case DATE:
+          writers.add(new DatePdfColumnWriter(counter, fieldName, rowWriter, negotiator));
+          break;
+        case TIME:
+          writers.add(new TimePdfColumnWriter(counter, fieldName, rowWriter, negotiator));
+          break;
+        case TIMESTAMP:
+          writers.add(new TimestampPdfColumnWriter(counter, fieldName, rowWriter, negotiator));
+          break;
+        default:
+          throw UserException.unsupportedError()
+            .message("PDF Reader with provided schema does not support " + type.name() + " data type.")
+            .addContext(errorContext)
+            .build(logger);
+      }
+    }
+  }
+
+  public abstract static class PdfColumnWriter {
+    final String columnName;
+    final ScalarWriter writer;
+    final int columnIndex;
+
+    public PdfColumnWriter(int columnIndex, String columnName, ScalarWriter writer) {
+      this.columnIndex = columnIndex;
+      this.columnName = columnName;
+      this.writer = writer;
+    }
+
+    public abstract void load (RectangularTextContainer<?> cell);
+
+    public ScalarWriter getWriter() {
+      return writer;
+    }
+  }
+
+  public static class IntPdfColumnWriter extends PdfColumnWriter {
+    IntPdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setInt(Integer.parseInt(cell.getText()));
+    }
+  }
+
+  public static class BigIntPdfColumnWriter extends PdfColumnWriter {
+    BigIntPdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setLong(Long.parseLong(cell.getText()));
+    }
+  }
+
+  public static class DoublePdfColumnWriter extends PdfColumnWriter {
+    DoublePdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setDouble(Double.parseDouble(cell.getText()));
+    }
+  }
+
+  public static class StringPdfColumnWriter extends PdfColumnWriter {
+    StringPdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      writer.setString(cell.getText());
+    }
+  }
+
+  public static class DatePdfColumnWriter extends PdfColumnWriter {
+    private String dateFormat;
+
+    DatePdfColumnWriter (int columnIndex, String columnName, RowSetLoader rowWriter, FileScanFramework.FileSchemaNegotiator negotiator) {
+      super(columnIndex, columnName, rowWriter.scalar(columnName));
+
+      ColumnMetadata metadata = negotiator.providedSchema().metadata(columnName);
+      if (metadata != null) {
+        this.dateFormat = metadata.property("drill.format");
+      }
+    }
+
+    @Override
+    public void load(RectangularTextContainer<?> cell) {
+      LocalDate localDate;
+      if (Strings.isNullOrEmpty(this.dateFormat)) {
+       localDate = LocalDate.parse(cell.getText());

Review comment:
       Fixed




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] luocooong commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
luocooong commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r755766696



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfFormatConfig.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+@JsonTypeName(PdfFormatPlugin.DEFAULT_NAME)
+public class PdfFormatConfig implements FormatPluginConfig {
+
+  private final List<String> extensions;
+  private final boolean extractHeaders;
+  private final String extractionAlgorithm;
+  private final boolean combinePages;
+  private final int defaultTableIndex;
+
+  @JsonCreator
+  public PdfFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                         @JsonProperty("extractHeaders") boolean extractHeaders,
+                         @JsonProperty("extractionAlgorithm") String extractionAlgorithm,
+                         @JsonProperty("combinePages") boolean combinePages,
+                         @JsonProperty("defaultTableIndex") int defaultTableIndex) {
+    this.extensions = extensions == null
+      ? Collections.singletonList("pdf")

Review comment:
       Can also use the `PdfFormatPlugin.DEFAULT_NAME` instead of `"pdf"`.

##########
File path: contrib/format-pdf/src/main/resources/drill-module.conf
##########
@@ -0,0 +1,23 @@
+#
+# 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.
+#
+
+#  This file tells Drill to consider this module when class path scanning.
+#  This file can also include any supplementary configuration information.
+#  This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
+
+drill.classpath.scanning.packages += "org.apache.drill.exec.store.pdf"

Review comment:
       No newfile at end of file.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfFormatConfig.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+import org.apache.drill.common.PlanStringBuilder;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+@JsonTypeName(PdfFormatPlugin.DEFAULT_NAME)
+public class PdfFormatConfig implements FormatPluginConfig {
+
+  private final List<String> extensions;
+  private final boolean extractHeaders;
+  private final String extractionAlgorithm;
+  private final boolean combinePages;
+  private final int defaultTableIndex;
+
+  @JsonCreator
+  public PdfFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                         @JsonProperty("extractHeaders") boolean extractHeaders,
+                         @JsonProperty("extractionAlgorithm") String extractionAlgorithm,
+                         @JsonProperty("combinePages") boolean combinePages,
+                         @JsonProperty("defaultTableIndex") int defaultTableIndex) {
+    this.extensions = extensions == null
+      ? Collections.singletonList("pdf")
+      : ImmutableList.copyOf(extensions);
+    this.extractHeaders = extractHeaders;
+    this.extractionAlgorithm = extractionAlgorithm;
+    this.combinePages = combinePages;
+    this.defaultTableIndex = defaultTableIndex;
+  }
+
+  public PdfBatchReader.PdfReaderConfig getReaderConfig(PdfFormatPlugin plugin) {
+    return new PdfBatchReader.PdfReaderConfig(plugin);
+  }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  @JsonProperty("extensions")
+  public List<String> getExtensions() {
+    return extensions;
+  }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  @JsonProperty("combinePages")
+  public boolean getCombinePages() { return combinePages; }

Review comment:
       Keep the code format of getter as consistent as possible.
   ```java
   public boolean getX() {
     return x;
   }
   ```
   or
   ```java
   public boolean getX() { return x; }
   ```

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");

Review comment:
       This property has been set many times, maybe it can be done like this :
   ```java
   if (System.getProperty("java.awt.headless") == null) {
     System.setProperty("java.awt.headless", "true");
   }
   ```
   And remove the `Utils.java`, line 60. `PdfBatchReader.java`, line 164?

##########
File path: contrib/format-pdf/src/test/java/org/apache/drill/exec/store/pdf/testPDFUtils.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.junit.Test;
+import technology.tabula.Table;
+import java.io.File;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class testPDFUtils {
+
+  private static final String DATA_PATH = "src/test/resources/pdf/";
+
+  @Test
+  public void testTableExtractor() throws Exception {
+    PDDocument document = getDocument("argentina_diputados_voting_record.pdf");
+    List<Table> tableList = Utils.extractTablesFromPDF(document);
+    document.close();
+    assertEquals(tableList.size(), 1);
+
+    PDDocument document2 = getDocument("twotables.pdf");
+    List<Table> tableList2 = Utils.extractTablesFromPDF(document2);
+    document2.close();
+    assertEquals(tableList2.size(), 2);
+  }
+
+  @Test
+  public void testTableExtractorWithNoBoundingFrame() throws Exception {
+    PDDocument document = getDocument("spreadsheet_no_bounding_frame.pdf");
+    List<Table> tableList = Utils.extractTablesFromPDF(document);
+    document.close();
+    assertEquals(tableList.size(), 1);
+  }
+
+  @Test
+  public void testTableExtractorWitMultipage() throws Exception {
+    PDDocument document = getDocument("us-020.pdf");
+    List<Table> tableList = Utils.extractTablesFromPDF(document);
+    document.close();
+    assertEquals(tableList.size(), 4);
+  }
+
+  @Test
+  public void testFirstRowExtractor() throws Exception {
+    PDDocument document = getDocument("schools.pdf");
+    List<Table> tableList = Utils.extractTablesFromPDF(document);
+    document.close();
+
+    List<String> values = Utils.extractRowValues(tableList.get(0));
+    assertEquals(values.size(), 11);
+  }
+
+

Review comment:
       Too many blank lines.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&

Review comment:
       Is this the logic to handle a table that spans multiple pages?

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));
+  }
+
+  private void writeMetadata() {
+    int startingIndex = columnHeaders.size();
+    writers.get(startingIndex).getWriter().setInt(pageCount);
+    writeStringMetadataField(title, startingIndex+1);
+    writeStringMetadataField(author, startingIndex+2);
+    writeStringMetadataField(subject, startingIndex+3);
+    writeStringMetadataField(keywords, startingIndex+4);
+    writeStringMetadataField(creator, startingIndex+5);
+    writeStringMetadataField(producer, startingIndex+6);
+    writeTimestampMetadataField(creationDate, startingIndex+7);
+    writeTimestampMetadataField(modificationDate, startingIndex+8);
+    writeStringMetadataField(trapped, startingIndex+9);
+    writers.get(startingIndex+10).getWriter().setInt(tableCount);
+  }
+
+  private void writeStringMetadataField(String value, int index) {
+    if (value == null) {
+      return;
+    }
+    writers.get(index).getWriter().setString(value);
+  }
+
+  private void writeTimestampMetadataField(Calendar dateValue, int index) {
+    if (dateValue == null) {
+      return;
+    }
+
+    writers.get(index).getWriter().setTimestamp(Instant.ofEpochMilli(dateValue.getTimeInMillis()));
+  }
+
+  private void addUnknownColumnToSchemaAndCreateWriter (TupleWriter rowWriter, String name) {

Review comment:
       This method is never used locally, the `metadataIndex` is only used in this method.
   In this class, at line 95 and 322, the value of field is not used.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.
+   * @param document The input PDF document to search for tables
+   * @return A list of tables found in the document.
+   */
+  public static List<Table> extractTablesFromPDF(PDDocument document) {
+    return extractTablesFromPDF(document, DEFAULT_ALGORITHM);
+  }
+
+  public static List<Table> extractTablesFromPDF(PDDocument document, ExtractionAlgorithm algorithm) {
+    System.setProperty("java.awt.headless", "true");
+
+    NurminenDetectionAlgorithm detectionAlgorithm = new NurminenDetectionAlgorithm();
+
+    ExtractionAlgorithm algExtractor;
+
+    SpreadsheetExtractionAlgorithm extractor = new SpreadsheetExtractionAlgorithm();
+
+    ObjectExtractor objectExtractor = new ObjectExtractor(document);
+    PageIterator pages = objectExtractor.extract();
+    List<Table> tables= new ArrayList<>();
+    while (pages.hasNext()) {
+      Page page = pages.next();
+
+      algExtractor = algorithm;
+      List<Rectangle> tablesOnPage = detectionAlgorithm.detect(page);
+
+      for (Rectangle guessRect : tablesOnPage) {
+        Page guess = page.getArea(guessRect);
+        tables.addAll(algExtractor.extract(guess));
+      }
+    }
+
+    try {
+      objectExtractor.close();
+    } catch (Exception e) {
+      logger.debug("Error closing Object extractor.");
+    }
+
+    return tables;
+  }
+
+  /**
+   * Returns the values contained in a PDF Table row
+   * @param table The source table
+   * @return A list of the header rows
+   */
+  public static List<String> extractRowValues(Table table) {
+    List<RectangularTextContainer> firstRow = table.getRows().get(0);

Review comment:
       Is it possible to `table.getRows()` is null?

##########
File path: contrib/format-pdf/src/test/java/org/apache/drill/exec/store/pdf/TestPdfFormat.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.QueryBuilder.QuerySummary;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.nio.file.Paths;
+import java.time.LocalDate;
+
+import static org.apache.drill.test.QueryTestUtil.generateCompressedFile;
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTests.class)
+public class TestPdfFormat extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+
+    // Needed for compressed file unit test
+    dirTestWatcher.copyResourceToRoot(Paths.get("pdf/"));
+  }
+
+  @Test
+  public void testStarQuery() throws RpcException {
+    String sql = "SELECT * FROM cp.`pdf/argentina_diputados_voting_record.pdf` WHERE `Provincia` = 'Rio Negro'";
+
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+      .addNullable("Apellido y Nombre", MinorType.VARCHAR)
+      .addNullable("Bloque político", MinorType.VARCHAR)
+      .addNullable("Provincia", MinorType.VARCHAR)
+      .addNullable("field_0", MinorType.VARCHAR)
+      .buildSchema();
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+      .addRow("ALBRIEU, Oscar Edmundo Nicolas", "Frente para la Victoria - PJ", "Rio Negro", "AFIRMATIVO")
+      .addRow("AVOSCAN, Herman Horacio", "Frente para la Victoria - PJ", "Rio Negro", "AFIRMATIVO")
+      .addRow("CEJAS, Jorge Alberto", "Frente para la Victoria - PJ", "Rio Negro", "AFIRMATIVO")
+      .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testExplicitQuery() throws RpcException {
+    String sql = "SELECT `Apellido y Nombre`, `Bloque político`, `Provincia`, `field_0` " +
+      "FROM cp.`pdf/argentina_diputados_voting_record.pdf` WHERE `Provincia` = 'Rio Negro'";
+
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+      .addNullable("Apellido y Nombre", MinorType.VARCHAR)
+      .addNullable("Bloque político", MinorType.VARCHAR)
+      .addNullable("Provincia", MinorType.VARCHAR)
+      .addNullable("field_0", MinorType.VARCHAR)

Review comment:
       A good solution to the blank header.

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);

Review comment:
       What happens if `tables.size() < startingTableIndex`?

##########
File path: contrib/format-pdf/src/test/resources/logback-test.xml
##########
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8" ?>

Review comment:
       Please remove this file or move to `/dev-support` folder.

##########
File path: contrib/format-pdf/src/test/java/org/apache/drill/exec/store/pdf/testPDFUtils.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.junit.Test;
+import technology.tabula.Table;
+import java.io.File;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class testPDFUtils {

Review comment:
       Use the `TestPDFUtils`.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r757676131



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.
+   * @param document The input PDF document to search for tables
+   * @return A list of tables found in the document.
+   */
+  public static List<Table> extractTablesFromPDF(PDDocument document) {
+    return extractTablesFromPDF(document, DEFAULT_ALGORITHM);
+  }
+
+  public static List<Table> extractTablesFromPDF(PDDocument document, ExtractionAlgorithm algorithm) {
+    System.setProperty("java.awt.headless", "true");
+
+    NurminenDetectionAlgorithm detectionAlgorithm = new NurminenDetectionAlgorithm();
+
+    ExtractionAlgorithm algExtractor;
+
+    SpreadsheetExtractionAlgorithm extractor = new SpreadsheetExtractionAlgorithm();
+
+    ObjectExtractor objectExtractor = new ObjectExtractor(document);
+    PageIterator pages = objectExtractor.extract();
+    List<Table> tables= new ArrayList<>();
+    while (pages.hasNext()) {
+      Page page = pages.next();
+
+      algExtractor = algorithm;
+      List<Rectangle> tablesOnPage = detectionAlgorithm.detect(page);
+
+      for (Rectangle guessRect : tablesOnPage) {
+        Page guess = page.getArea(guessRect);
+        tables.addAll(algExtractor.extract(guess));
+      }
+    }
+
+    try {
+      objectExtractor.close();
+    } catch (Exception e) {
+      logger.debug("Error closing Object extractor.");

Review comment:
       Changed to throw `UserException` if the table is unparsable.  This may actually be an interesting point.  One of the ways Pandas operates is that the user can choose the behavior when it can't parse data.   We may want to think about modifying Drill such that a user could decide "I want an error if Drill can't read the data" or "I want Drill to ignore bad data"
   
   @dzamo Any thoughts 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.

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r749714747



##########
File path: distribution/src/assemble/component.xml
##########
@@ -49,7 +49,8 @@
         <include>org.apache.drill.contrib:drill-format-pcapng:jar</include>
         <include>org.apache.drill.contrib:drill-format-hdf5:jar</include>
         <include>org.apache.drill.contrib:drill-format-ltsv:jar</include>
-        <include>org.apache.drill.contrib:drill-format-httpd:jar</include>
+        <include>org.apache.drill.contrib:drill-format-https:jar</include>

Review comment:
       Good catch!




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-959821275






-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] luocooong commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
luocooong commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-959477711






-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769237516



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));
+  }
+
+  private void writeMetadata() {
+    int startingIndex = columnHeaders.size();
+    writers.get(startingIndex).getWriter().setInt(pageCount);
+    writeStringMetadataField(title, startingIndex+1);
+    writeStringMetadataField(author, startingIndex+2);
+    writeStringMetadataField(subject, startingIndex+3);
+    writeStringMetadataField(keywords, startingIndex+4);
+    writeStringMetadataField(creator, startingIndex+5);
+    writeStringMetadataField(producer, startingIndex+6);
+    writeTimestampMetadataField(creationDate, startingIndex+7);
+    writeTimestampMetadataField(modificationDate, startingIndex+8);
+    writeStringMetadataField(trapped, startingIndex+9);
+    writers.get(startingIndex+10).getWriter().setInt(tableCount);
+  }
+
+  private void writeStringMetadataField(String value, int index) {
+    if (value == null) {
+      return;
+    }
+    writers.get(index).getWriter().setString(value);

Review comment:
       Fixed (I think)

##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);
+    addMetadataColumnToSchema("_title", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_author", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_subject", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_keywords", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creator", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_producer", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_creation_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_modification_date", MinorType.TIMESTAMP);
+    addMetadataColumnToSchema("_trapped", MinorType.VARCHAR);
+    addMetadataColumnToSchema("_table_count", MinorType.INT);
+  }
+
+  private void addMetadataColumnToSchema(String columnName, MinorType dataType) {
+    int index = rowWriter.tupleSchema().index(columnName);
+    if (index == -1) {
+      ColumnMetadata colSchema = MetadataUtils.newScalar(columnName, dataType, DataMode.OPTIONAL);
+
+      // Exclude from wildcard queries
+      colSchema.setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
+      metadataIndex++;
+
+      index = rowWriter.addColumn(colSchema);
+    }
+
+    writers.add(new StringPdfColumnWriter(index, columnName, rowWriter));
+  }
+
+  private void writeMetadata() {
+    int startingIndex = columnHeaders.size();
+    writers.get(startingIndex).getWriter().setInt(pageCount);
+    writeStringMetadataField(title, startingIndex+1);
+    writeStringMetadataField(author, startingIndex+2);
+    writeStringMetadataField(subject, startingIndex+3);
+    writeStringMetadataField(keywords, startingIndex+4);
+    writeStringMetadataField(creator, startingIndex+5);
+    writeStringMetadataField(producer, startingIndex+6);
+    writeTimestampMetadataField(creationDate, startingIndex+7);
+    writeTimestampMetadataField(modificationDate, startingIndex+8);
+    writeStringMetadataField(trapped, startingIndex+9);
+    writers.get(startingIndex+10).getWriter().setInt(tableCount);
+  }
+
+  private void writeStringMetadataField(String value, int index) {
+    if (value == null) {
+      return;
+    }
+    writers.get(index).getWriter().setString(value);

Review comment:
       I refactored all this with a metadata class.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769238112



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();

Review comment:
       I was thinking about this... if you try to query an empty file of another type in Drill, would you get the implicit fields?  While I see your point, I'm not sure what the "correct" behavior should be.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-979755114


   Copy that @paul-rogers, I [have incorporated your advice above into the Drill 2.0 wiki page](https://github.com/apache/drill/wiki/Drill-2.0-Proposal#project-structure-packaging-and-distribution).  I am hopeful that after 1.20 is out the door we can split out the new branches, and repos, for 2.0 (but it is of course the project elders that decide such things).  In my youthful exuberance I imagine it starting early next year!
   
   I would think that getting the source code reorganised in a way that will carry us into the future should probably be the very first Drill 2.0 activity to take place.  Surely we would want our most experienced contributors to define such a task.


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-960472017


   @cgivre @paul-rogers, my 2c.  I guess some partial precedents for a format plugin like this are ones like format-image and format-esri (as noted), though those do only go after the explicitly structured content.  It would not surprise me if there are quite often cases of unfortunate people needing to scrape data out of 10^0 - 10^5 PDFs.  The purist in me agrees with Paul's thought: a Groovy script over whatever Java lib is used here could be employed instead.  That would not be automatically be parallelised like a Drill query is so a user with many PDFs _might_ be pushed all the way through GNU parallel to Spark.  All that is followed by this recurring thought: if Drill is disciplined and focussed only on SQL against standard big data formats then it finds itself trying to compete in an uncomfortably crowded space.  Probably fatally crowded.
   
   I do also note that the "big" and "small" data worlds are not disjoint.  I have in practice joined big data in Parquet with small reference data in Excel (actually EtherCalc).  Even in the big data regime reference data remains small and is maintained by humans in human forrmats rather than pumped out by machines in machine formats.
   
   This is getting long again.  Last thoughts.
   
   - I feel this really should be good at finding and parsing tables (work often, rather than work seldom) for inclusion.
   - We should consider a subproject to contain our long tail of "non-standard" data formats.  This would separate away plugins that should not be expected to run with speed or realiability of the core data formats and keep the core distributable size down as we add format after format.  We could then start to distribute tarballs for `drill-core` and `drill-extra`.
   - This looks like a plugin that will benefit from Drill's optional schema.  That means we might have some unique ability to compete 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.

To unsubscribe, e-mail: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-963148567


   > * Sometimes the PDF reader does not read tables perfectly and you get a mix of found headers and not found headers, so that's one reason I took that approach.
   
   This consideration will only apply for `extractHeaders = true`, right?  In this case all of the readers do split out columns so I think we're good.
    
   > * I actually dislike the `columns` approach from the CSV readers because it increases the level of complexity of queries.  In theory, if someone is querying a table (doesn't matter from where) they will want that broken into columns and rows.  The columns array approach (IMHO) makes this a lot harder that it needs to be.
   
   The columns array does allow text files to contain jagged arrays, which is perhaps valuable?  I don't see a huge saving in typing `select field_0, field_1` over `select columns[0], columns[1]`.
   
   > * This actually follows the model used in the Excel reader.
   
   My fear, and I don't know if this is real or not, is that if we profilerate plugin-specific quirks in how the schema is represented to the user then the promise of a standard SQL interface to the data gets tainted and developers and users will be put off.  
   
   "It's standard except every plugin presents its data the way that author prefers"
   "That fragment of code I sent you for the columns array won't work here because this plugin does it differently"
   "This SQL script makes for confusing reading because the plugins involved name their generated columns differently, sometimes 'column', sometimes 'field', sometimes 'var'"
   etc.
   
   I don't feel all that strongly about `field_0` vs `columns[0]` (I mean, maybe we deprecate the columns array?) but I am finding myself thinking more and more about consistency.


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r757673667



##########
File path: contrib/format-pdf/src/main/resources/bootstrap-format-plugins.json
##########
@@ -0,0 +1,43 @@
+{
+  "storage":{
+    "dfs": {
+      "type": "file",
+      "formats": {
+        "pdf": {
+          "type": "pdf",
+          "extensions": [
+            "pdf"
+          ],
+          "extractHeaders": true,
+          "combinePages": false
+        }
+      }
+    },
+    "cp": {

Review comment:
       I didn't plan on it.  We can include one in the docs that people can d/l and try out. 




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769007276



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&

Review comment:
       I cleaned this up by wrapping this in iterator classes.  This logic was here to support the page merge feature.  Basically... it works like this... a `Page` can contain 0 or more `Tables`.   If you take a look at the included file `schools.pdf` you'll see it is a 5 page spreadsheet.  The problem is that if you wanted to actually query this data, you'd want the whole thing, not an individual page.  So, what Drill can do is merge all the Pages (each containing 1 table) together. 
   
   You would only want to use this feature in the case of multi-page spreadsheet PDFs.  The logic therefore is:
   1. Read the current table until there are no more rows.
   2. Get the next table (if there is one) 
   3. Read it until there are no more rows
   4. Go back to step 2. 
   
   




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769237668



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();
+  }
+
+  private void addImplicitColumnsToSchema() {
+    metadataIndex = columns;
+    // Add to schema
+    addMetadataColumnToSchema("_page_count", MinorType.INT);

Review comment:
       Done




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-962569638


   >  The problem is, a tool that tries to be both a desert topping and a floor wax (let's see how old the readers are with this one), ends up being good at neither.
   
   @paul-rogers you got me with this idiom, but I like it!  The broader topic is super interesting.  If SQLite started adding the features needed to compete with Oracle Database 21c it would quickly fail at being SQLite.  If Linux tried to be an OS kernel for both TVs and supercomputers it would... continue to dominate both extremes!  There are some twists here!
   
   Pigeonholing formats into small scale and large scales is also a tricky business.  For example, we naturally want to declare PDF a desktop format, but I can easily imagine a conversation like the following.
   
   "Hey Bob, remember that we sent decades of paper archives from the basement out to that big scanning centre for digitisation?  They've come back as millions of pages of PDFs.  Someone just asked me if we can help them find all invoices containing a particular SKU, and pull out the price on that line.  The ERP system only has the last 10 years loaded into it and they want to go back further".
   
   "Chuck 'em in HDFS, we'll run a Drill query"
   
   "But PDF is a desktop publishing format, not a big data format!  Surely our big data cluster will want nothing to do with it!?"
   
   "Drill's got a plugin architecture which led to people adding support for all sorts of weird and wonderful formats.  Querying PDFs is a dubious business but we'll know after ~10 lines of SQL if we can do this with Drill or not.  If not, miserable days or weeks of programming with a PDF library await one of our interns."
   
   
   
   


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-962903289


   @cgivre Instead of generating `field_0`, `field_1` when no column names can be determined, is it possible to generate a `columns` array to be accessed using `columns[0], columns[1], ...` to remain consistent with what we do for CSV?


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-963117787


   > @dzamo, the reference was to an early Saturday Night Live skit.
   > 
   > Just to refocus the discussion, my question is really around configuration. When querying CSV, Parquet and the like, it is very clear where the data lies. When querying Excel, PDF, HTML, Word, "the web", it is less clear: there is some amount of data mining needed to say that it is THIS table and not THAT one, that the table spans three pages, etc.
   > 
   > The question is, how does the user specify this? If it were me, I would not want to be tinkering with a JSON storage plugin config, watching my query fail with a Drill stack trace, or wondering why I got no data. Instead, I'd want a tool better suited for the task. Once I had that, if I then wanted to run at scale, I'd want to say, "Drill, just use X and consume the data."
   > 
   > So, the question here is: is the JSON storage plugin config an effective way for a data scientist to mine PDF, Excel, HTML and other messy sources? I don't have an answer, I'm just asking the question.
   > 
   > Again, if we had the ability to have true external plugins, then this could easily be an add-on project. Those who know PDF could go hog wild creating a good solution. But, we don't, so every specialized plugin has to be part of core Drill. Is that good? That's also a question for debate.
   
   @dzamo @paul-rogers , These are all interesting points.  I'd like to focus on the config variables for a moment.  The PDF reader has 2 config variables:  the `mergeAllTables` and the `tableNumber`.   While a user could theoretically set these globally, they really are intended to be set via the `table()` function at query time.   For some reference the Excel reader has similar configs which allow a user to select different sheets within an Excel file, or define regions in a file where their data lives etc. 
   
   IMHO, this flexibility is actually very good for the user, because it allows an administrator to configure global default values that make sense but also allow a user to make query-time changes so they can access their data.
   
   With respect to the fixed-width plugin (https://github.com/apache/drill/pull/2282) I actually have a different vision of how this can be used, and will post comments there.
   
   


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre edited a comment on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre edited a comment on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-997322720


   @paul-rogers 
   Thanks for all your review.  I addressed all your comments (I think) and did the following:
   * Added additional unit tests
   * Refactored table list so that all tables are not read into memory if not requested
   * Added iterator classes to avoid counters in the batch reader
   * Moved metadata collection to separate class
   * Refactored to allow a pdf with no tables to return metadata if requested (And unit test)
   * Added config option for different extraction algorithms.
   * Removed extraneous test PDF files
   * General code cleanup
   
   I removed all but one of the `System.env` calls and I'm a little stuck on this.  The reason I added this line is that when querying a PDF with Drill in embedded mode, it opens an additional java window.  This does not occur when running unit tests which makes for difficult debugging.   I'm going to keep digging into this, but I was wondering if you could take a look at the rest of the revisions in the mean time?   The issue seems to be in either Tabula or PdfBox, which are the underlying libraries that read the PDF file. Thanks!


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r757676406



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.
+   * @param document The input PDF document to search for tables
+   * @return A list of tables found in the document.
+   */
+  public static List<Table> extractTablesFromPDF(PDDocument document) {
+    return extractTablesFromPDF(document, DEFAULT_ALGORITHM);
+  }
+
+  public static List<Table> extractTablesFromPDF(PDDocument document, ExtractionAlgorithm algorithm) {
+    System.setProperty("java.awt.headless", "true");
+
+    NurminenDetectionAlgorithm detectionAlgorithm = new NurminenDetectionAlgorithm();
+
+    ExtractionAlgorithm algExtractor;
+
+    SpreadsheetExtractionAlgorithm extractor = new SpreadsheetExtractionAlgorithm();
+
+    ObjectExtractor objectExtractor = new ObjectExtractor(document);
+    PageIterator pages = objectExtractor.extract();
+    List<Table> tables= new ArrayList<>();
+    while (pages.hasNext()) {
+      Page page = pages.next();
+
+      algExtractor = algorithm;
+      List<Rectangle> tablesOnPage = detectionAlgorithm.detect(page);
+
+      for (Rectangle guessRect : tablesOnPage) {
+        Page guess = page.getArea(guessRect);
+        tables.addAll(algExtractor.extract(guess));
+      }
+    }
+
+    try {
+      objectExtractor.close();
+    } catch (Exception e) {
+      logger.debug("Error closing Object extractor.");
+    }
+
+    return tables;
+  }
+
+  /**
+   * Returns the values contained in a PDF Table row
+   * @param table The source table
+   * @return A list of the header rows
+   */
+  public static List<String> extractRowValues(Table table) {
+    List<RectangularTextContainer> firstRow = table.getRows().get(0);
+    List<String> values = new ArrayList<>();
+
+    if (firstRow != null) {
+      for (int i =0; i < firstRow.size(); i++) {

Review comment:
       We didn't test for all those use cases, but the underlying library (Tabula) actually has a pretty robust collection of tests that do cover a lot of what you're asking about.[1] 
   
   In my manual testing, what I found was that if you had inconsistent tables, Tabula would basically produce some sort of mapping to columns and rows that Drill could read.   Not ideal, but neither is trying to extract data from PDFs.
   
   [1]: https://github.com/tabulapdf/tabula-java/tree/master/src/test/java/technology/tabula




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-959493139


   > @cgivre Thanks for the new format plugin. Is it ready for review?
   
   This should be ready for review :-)


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r766622138



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/Utils.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.ObjectExtractor;
+import technology.tabula.Page;
+import technology.tabula.PageIterator;
+import technology.tabula.Rectangle;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+import technology.tabula.detectors.NurminenDetectionAlgorithm;
+import technology.tabula.extractors.BasicExtractionAlgorithm;
+import technology.tabula.extractors.ExtractionAlgorithm;
+import technology.tabula.extractors.SpreadsheetExtractionAlgorithm;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Utils {
+
+  public static final ExtractionAlgorithm DEFAULT_ALGORITHM = new BasicExtractionAlgorithm();
+  private static final Logger logger = LoggerFactory.getLogger(Utils.class);
+
+  /**
+   * Returns a list of tables found in a given PDF document.  There are several extraction algorithms
+   * available and this function uses the default Basic Extraction Algorithm.
+   * @param document The input PDF document to search for tables
+   * @return A list of tables found in the document.
+   */
+  public static List<Table> extractTablesFromPDF(PDDocument document) {
+    return extractTablesFromPDF(document, DEFAULT_ALGORITHM);
+  }
+
+  public static List<Table> extractTablesFromPDF(PDDocument document, ExtractionAlgorithm algorithm) {
+    System.setProperty("java.awt.headless", "true");
+
+    NurminenDetectionAlgorithm detectionAlgorithm = new NurminenDetectionAlgorithm();
+
+    ExtractionAlgorithm algExtractor;
+
+    SpreadsheetExtractionAlgorithm extractor = new SpreadsheetExtractionAlgorithm();
+
+    ObjectExtractor objectExtractor = new ObjectExtractor(document);
+    PageIterator pages = objectExtractor.extract();
+    List<Table> tables= new ArrayList<>();
+    while (pages.hasNext()) {
+      Page page = pages.next();
+
+      algExtractor = algorithm;
+      List<Rectangle> tablesOnPage = detectionAlgorithm.detect(page);
+
+      for (Rectangle guessRect : tablesOnPage) {
+        Page guess = page.getArea(guessRect);
+        tables.addAll(algExtractor.extract(guess));
+      }
+    }
+
+    try {
+      objectExtractor.close();
+    } catch (Exception e) {
+      logger.debug("Error closing Object extractor.");

Review comment:
       @cgivre I have just added a description of an option like that to the Drill 2.0 wiki page.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo edited a comment on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo edited a comment on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-963148567


   > * Sometimes the PDF reader does not read tables perfectly and you get a mix of found headers and not found headers, so that's one reason I took that approach.
   
   This consideration will only apply for `extractHeaders = true`, right?  In this case all of the readers do split out columns so I think we're good.
    
   > * I actually dislike the `columns` approach from the CSV readers because it increases the level of complexity of queries.  In theory, if someone is querying a table (doesn't matter from where) they will want that broken into columns and rows.  The columns array approach (IMHO) makes this a lot harder that it needs to be.
   
   The columns array does allow text files to contain jagged arrays, which is perhaps valuable?  I don't see a huge saving in typing `select field_0, field_1` over `select columns[0], columns[1]`, am I missing some other complication?
   
   > * This actually follows the model used in the Excel reader.
   
   My fear, and I don't know if this is real or not, is that if we profilerate plugin-specific quirks in how the schema is represented to the user then the promise of a standard SQL interface to the data gets tainted and developers and users will be put off.  
   
   "It's standard except every plugin presents its data the way that author prefers"
   "That fragment of code I sent you for the columns array won't work here because this plugin does it differently"
   "This SQL script makes for confusing reading because the plugins involved name their generated columns differently, sometimes 'column', sometimes 'field', sometimes 'var'"
   etc.
   
   I don't feel all that strongly about `field_0` vs `columns[0]` (I mean, maybe we deprecate the columns array?) but I am finding myself thinking more and more about consistency.


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] pjfanning commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
pjfanning commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r749714232



##########
File path: distribution/src/assemble/component.xml
##########
@@ -49,7 +49,8 @@
         <include>org.apache.drill.contrib:drill-format-pcapng:jar</include>
         <include>org.apache.drill.contrib:drill-format-hdf5:jar</include>
         <include>org.apache.drill.contrib:drill-format-ltsv:jar</include>
-        <include>org.apache.drill.contrib:drill-format-httpd:jar</include>
+        <include>org.apache.drill.contrib:drill-format-https:jar</include>

Review comment:
       the httpd to https change looks like an unintentional change




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-961327918


   @cgivre, @dzamo raise good points. So, what is Drill today? Is it still primarily used for distributed queries at scale? Or, as a handy desktop tool for data scientists? Probably both. The problem is, a tool that tries to be both a desert topping and a floor wax (let's see how old the readers are with this one), ends up being good at neither.
   
   One approach, if we had resources, would be to create a Drill Desktop that is optimized for that case and encourages all kinds of specialized data connectors. Create an easy way to define those connectors (YAML files created by specialized web apps?) Ensure Drill has good integration with Jupyter and the other usual suspects.
   
   Another approach, if we had resources, is the oft-discussed idea of separating the less-common plugins from the Drill core. Work started on this: to create an extension mechanism that made this possible. (Today, most plugins need quite a bit of Drill internal code.)
   
   So, no harm in adding the PDF reader, but I expect usage will be pretty limited just because, for the folks that need it, configuration will be too hard. Better would be a Python or Spark job that extracts the data into a CSV file, then query the CSV file with Drill. Each step could be debugged easily. I can't imagine anyone will want to debug their PDF extraction using Drill's overly generous Java stack traces... 


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769761345



##########
File path: contrib/format-pdf/README.md
##########
@@ -0,0 +1,67 @@
+# Format Plugin for PDF Table Reader
+One of the most annoying tasks is when you are working on a data science project and you get data that is in a PDF file. This plugin endeavours to enable you to query data in
+ PDF tables using Drill's SQL interface.  
+
+## Data Model
+Since PDF files generally are not intended to be queried or read by machines, mapping the data to tables and rows is not a perfect process.  The PDF reader does support 
+provided schema. 
+
+### Merging Pages
+The PDF reader reads tables from PDF files on each page.  If your PDF file has tables that span multiple pages, you can set the `combinePages` parameter to `true` and Drill 
+will merge all the tables in the PDF file.  You can also do this at query time with the `table()` function.
+
+## Configuration
+To configure the PDF reader, simply add the information below to the `formats` section of a file base storage plugin. 
+
+```json
+"pdf": {
+  "type": "pdf",
+  "extensions": [
+    "pdf"
+  ],
+  "extractHeaders": true,
+  "combinePages": false
+}
+```
+The avaialable options are:
+* `extractHeaders`: Extracts the first row of any tables as the header row.  If set to false, Drill will assign column names of `field_0`, `field_1` to each column.
+* `combinePages`: Merges multipage tables together.
+* `defaultTableIndex`:  Allows you to query different tables within the PDF file. Index starts at `0`. 
+
+
+## Accessing Document Metadata Fields
+PDF files have a considerable amount of metadata which can be useful for analysis.  Drill will extract the following fields from every PDF file.  Note that these fields are not
+ projected in star queries and must be selected explicitly.  The document's creator populates these fields and some or all may be empty. With the exception of `_page_count
+ ` which is an `INT` and the two date fields, all the other fields are `VARCHAR` fields.
+ 
+ The fields are:
+ * `_page_count`
+ * `_author`
+ * `_title`
+ * `_keywords`
+ * `_creator`
+ * `_producer`
+ * `_creation_date`
+ * `_modification_date`
+ * `_trapped`
+ * `_table_count`

Review comment:
       > Here, I assume `_author` will repeat for every table row? That `_page_count` will increase and gives the page number? Or, is constant and will always be, say, 25 if that's the number of pages?
   
   Page count is the total number of pages in the document.  All these metafields will be the same for every row.  Drill does this with the other file based readers and adds implicit fields, where if you query them, you'll get the same info for every row.  
   > 
   > Does `_table_count` stay fixed at the number of tables (5, say), or does it count tables (1, 2, 3, 4, 5)?
   
   It was supposed to be the total number of tables in the document, however, I ended up removing it because in order to compute it, I had to load all the tables into memory. 
   
   > 
   > What is `_trapped`? Is there some PDF standard we could point to where this stuff is defined?
   
   Yes.... I'll add that to the docs.  It has something to do with how a PDF doc is printed.
   
   > 
   > Actually, there is a larger question. If I'm exploring a pile of PDFs, I may want to extract metadata, such as the above. If I'm extracting tables, I don't need the metadata repeated per table row.
   > 
   > Should the plugin allow two query types? Either metadata or tables? Can this be done with some kind of Calcite trickery? `FROM "foo.pdf"` means the tables, `FROM "foo.pdf.metadata"` means the metadata?
   
   It could but this is significantly more complex to do that.
   
   > 
   > Or, we check and if we see only metadata columns, we return one row per file rather than one row per table row?
   > 
   > Wouldn't something like this be useful to extract metadata from a PDF with no tables?
   
   I'm debating this point... stand by.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769000687



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {

Review comment:
       Good idea.  I'd prefer to do that in a separate PR. 




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-979494770


   @dzamo, @cgivre: we should certainly encourage people to add plugins (connectors) to everything under the sun. The question really is: does the rather limited Drill core team want to support all of them at the same level as the main ones that Just Gotta Work. For huge files. On local, S3, HDFS. With splits. With metadata. With excellent error reporting. And on and on.
   
   The PDF thing seems like the prototypical "not in the core, but a great add-on" example. Yes, some one user might want to scan zillions of PDF files on an HDFS system. But, the core Drill team can't spend its resources on that when there are still basics to address.
   
   Let's do this: let's get this into Drill for now. Then:
   
   * Ensure we have a good way to build plugins separate from the Drill code. (Last time I tried, I couldn't get it to work.)
   * Explain how to create a plugin in a users own repo, built against Drill.
   * Explain how to drop the plugin into a running Drill to add that functionality.
   
   This way, even without a marketplace, folks can build their own connectors separate from the Drill repo.
   
   A next step might be to split Drill's monorepo into multiple parts, as do many other projects. The current repo would be the core. Then, the contrib stuff could move to its own repo(s) under the Drill project. And, users can create their own extensions in their own repos.  


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r780725576



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {

Review comment:
       I'd forgotten this comment. Limit was added to the EVF framework in the pending EVF V2 PR.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre merged pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre merged pull request #2359:
URL: https://github.com/apache/drill/pull/2359


   


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-961327918


   @cgivre, @dzamo raise good points. So, what is Drill today? Is it still primarily used for distributed queries at scale? Or, as a handy desktop tool for data scientists? Probably both. The problem is, a tool that tries to be both a desert topping and a floor wax (let's see how old the readers are with this one), ends up being good at neither.
   
   One approach, if we had resources, would be to create a Drill Desktop that is optimized for that case and encourages all kinds of specialized data connectors. Create an easy way to define those connectors (YAML files created by specialized web apps?) Ensure Drill has good integration with Jupyter and the other usual suspects.
   
   Another approach, if we had resources, is the oft-discussed idea of separating the less-common plugins from the Drill core. Work started on this: to create an extension mechanism that made this possible. (Today, most plugins need quite a bit of Drill internal code.)
   
   So, no harm in adding the PDF reader, but I expect usage will be pretty limited just because, for the folks that need it, configuration will be too hard. Better would be a Python or Spark job that extracts the data into a CSV file, then query the CSV file with Drill. Each step could be debugged easily. I can't imagine anyone will want to debug their PDF extraction using Drill's overly generous Java stack traces... 


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] luocooong commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
luocooong commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-959477711


   @cgivre Thanks for the new format plugin. Is it ready for review?


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-959821275


   Cool contribution. I'm not entirely convinced this is something that Drill should handle. There are too many variables for the very limited controls which Drill provides. It is likely that this will work for one or two limited use cases, but not the vast majority of PDF files. Using JSON plugin config files to specify the mapping is awkward. Probably each file will need its own config, which is not scalable.
   
   Drill's fundamental design is to run at scale. It is highly unlikely that someone will use PDF files to store GB of data. If they do, they have problems bigger than Drill can help them solve. Thus, this kind of plugin works only at the small scale: one or two files in, say, an embedded Drillbit with JDBC or SQLine.
   
   A better choice would be to wrap this thing in a script: tinker with the PDF extraction, using whatever tools are available, to get the right mapping. Then, wrap that in a script that produces, say, a CSV format to stdout. Drill can then read that input.
   
   Such an approach enables all manner of ad-hoc, small scale data extraction.
   
   Or, maybe Drill should offer a "desktop edition" that is designed for small, ad-hoc projects based on local files, with some way to handle all the tinkering needed when reading PDF files, images, Word files, spreadsheets, Twitter feeds, Slack posts another formats popular with data scientists. Such features would not normally be part of the massive-scale deployments for which Drill is designed.
   
   Thoughts?


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-999062953


   > @cgivre new pdfbox release: https://downloads.apache.org/pdfbox/2.0.25/RELEASE-NOTES.txt
   
   Thanks @pjfanning !  Unfortunately, that didn't solve the issue with opening up the java window.  I emailed the PDFbox user list and hopefully they can give me some suggestions.


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-964848311


   > The bigger implication of having a `columns` array vs `field_n` is when a user starts with `SELECT *` queries. It makes it harder for BI tools to gather schema metadata and it also is non-standard SQL. Now... querying PDF is also non-standard SQL... so maybe that's less important. But, it makes the discovery a little harder IMHO.
   
   @cgivre Oh right, that makes sense.  So should we put in support for both `columns[n]` and `field_n` as widely as possible, with a standardised option which lets users switch between each mode?  Maybe standardising on naming of `columns[n]` and `column_n` is a small saving on cognitive load for users here?
   
   @paul-rogers with apologies to this PR for saddling it with so much broader design chat, I wanted to share a last set of findings from talking with others with you and finally ask if we might go over a couple of questions with you, away from this PR.
   
   1. I've polled some Drill devs and going after the "long tail" of formats and storage systems is mostly of interest to them.  @vvysotskyi even has an intriguing idea of a marketplace for these plugins, I guess something like the Eclipse plugin marketplace.
   2. I have developed a conviction that to go after the "long tail" and not produce a sprawling mess that neither developers nor users want to touch, we need to try to get strict (to the extent possible) about consistency in how plugins behave and how they are configured.  Today we already are not all that consistent (e.g. see remarks on `columns[n]` vs `field_n` above, on column `name` and `type` in fixed width format).
   3. Those I've spoken with do also like the idea of splitting our distributed packages into "core" and "kitchen sink", or something like that, to put us in a better position to go after the "long tail".  It sounds like we're okay with our existing mono repo containing many plugins but end users should not have to download the kitchen sink to query e.g. just JSON or Parquet.  Drill startup times will probably be slow for the kitchen sink because the Java class loader will have a huge amount to scan.  And developer testing could get onerous if we cannot compile only a subset.
   4. By chance I saw that BigQuery, which for some reason I've designated in my mind of as kind of the Rolls Royce of Dremel-family engines even though I know little about it, can query Google Sheets.  So even they entertain some "small data" formats, although nothing like what we're imagining.  Just an anecdote.
   
   I would love to consult with you on 2 and 3 in a sort of "Very well, if you _must_ do a distribution of Drill with this long tail of formats, storage systems and UDFs in it then at least equip yourselves with the following practices" chat.  Perhaps in the upcoming community meetup, otherwise outside (if it's of any interest on your end of course).
   
   Thanks
   James


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] dzamo commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
dzamo commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-962895343


   @paul-rogers, right, okay it's an expressiveness thing here rather than a scale thing.  The expressiveness of Drill SQL ∪ Drill format config JSON falls well short of that of a general purpose scripting language and for reading fiddly unstructured data that shortfall might rapidly become uncomfortable.  The format config for this particular plugin looks quite succinct, like the plugin will either automagically get your data out, or it won't and then you need to pack up and go and open the interpreter of your favourite scripting language.  Making your resulting script scale to millions of pages, if it that's needed, is left to the student.  I quite like the Ray project for Python myself.
   
   This thread has triggered some thoughts.  If we find ourselves starting to write long essays of JSON in format configs then we should probably be concerned.  If we find ourselves trying to embed a miniature data processing DSL into format config JSON then we need to stop moving immediately and pray to the ancestors that we might be shown a path that will return us from wilderness.  I want to revisit the draft fixed width format plugin with these ideas in mind.  Its config allows setting names and types for columns, but for other formats we must do this in SQL.  I think we should only ever do this in SQL.
   
   I think we can do something on the packaging front.  These format plugins live under contrib/ in the source tree and are compiled to their own jar files.  If we simply change the final tarball-building stage of our Maven build to give us something like the following on our download page, would we not be in reasonable shape? 
   
   Package|Size|Description
   --|--|--
   drill-core|300MB|Drill with core storage layer libs only.  Use this in a focussed big data environment to query standard formats like Parquet, CSV and JSON in HDFS or object storage with predictable results and performance.  Supplement this with indiviudal plugins listed below as needed.
   drill-ktichen-sink|1.5GB|Drill core plus all 100+ storage and format plugins.  Use this for maximum compatibility.  Results and performance may vary across plugins.
   drill-storage-jdbc|130KB|Plugin to query systems that provide a JDBC driver using a generic SQL dialect.
   drill-format-pdf|90KB|Plugin to query tables scraped from PDF files.
   ...
   
   P.S. We'd be persisting with a monolithic Git repo containing multiple "projects" here, but I personally don't mind mono repos.


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769237733



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+
+  /**
+   * Metadata fields are calculated once when the file is opened.  This function populates
+   * the metadata fields so that these are only calculated once.
+   */
+  private void populateMetadata() {
+    PDDocumentInformation info = document.getDocumentInformation();
+    pageCount = document.getNumberOfPages();
+    title = info.getTitle();
+    author = info.getAuthor();
+    subject = info.getSubject();
+    keywords = info.getKeywords();
+    creator = info.getCreator();
+    producer = info.getProducer();
+    creationDate = info.getCreationDate();
+    modificationDate = info.getModificationDate();
+    trapped = info.getTrapped();
+    tableCount = tables.size();

Review comment:
       Done




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769001349



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {
+      AutoCloseables.closeSilently(fsStream);
+      fsStream = null;
+    }
+
+    if (document != null) {
+      AutoCloseables.closeSilently(document.getDocument());
+      AutoCloseables.closeSilently(document);
+      document = null;
+    }
+  }
+
+  /**
+   * This method opens the PDF file, and finds the tables
+   */
+  private void openFile() {
+    try {
+      fsStream = negotiator.fileSystem().openPossiblyCompressedStream(split.getPath());
+      document = PDDocument.load(fsStream);
+    } catch (Exception e) {
+      throw UserException
+        .dataReadError(e)
+        .message("Failed to open open input file: %s", split.getPath().toString())
+        .addContext(e.getMessage())
+        .addContext(errorContext)
+        .build(logger);

Review comment:
       Fixed




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r769008268



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);

Review comment:
       Refactored to only open the desired table.  If the user has the `combineTables` set to true it does read all the tables in a given file at once.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r757668507



##########
File path: contrib/format-pdf/src/test/resources/logback-test.xml
##########
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+
+    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.
+
+-->
+<configuration>
+  <if condition='property("drill.lilith.enable").equalsIgnoreCase("true")'>
+    <then>
+      <appender name="SOCKET" class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
+        <Compressing>true</Compressing>
+        <ReconnectionDelay>10000</ReconnectionDelay>
+        <IncludeCallerData>true</IncludeCallerData>
+        <RemoteHosts>${LILITH_HOSTNAME:-localhost}</RemoteHosts>
+      </appender>
+      <logger name="org.apache.drill" additivity="false">
+        <level value="DEBUG"/>
+        <appender-ref ref="SOCKET"/>
+      </logger>
+      <logger name="query.logger" additivity="false">
+        <level value="ERROR"/>
+        <appender-ref ref="SOCKET"/>
+      </logger>
+      <logger name="org.apache.drill.exec.store.pdf">
+        <level value="DEBUG"/>
+        <appender-ref ref="SOCKET"/>
+      </logger>
+    </then>
+  </if>
+
+  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <!-- encoders are assigned the type
+         ch.qos.logback.classic.encoder.PatternLayoutEncoder by default -->
+    <encoder>
+      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
+    </encoder>
+  </appender>
+
+  <logger name="org.apache.drill.exec.store.pdf" additivity="false">
+    <level value="DEBUG" />
+    <appender-ref ref="STDOUT" />
+  </logger>
+</configuration>

Review comment:
       This was accidentally left in while I was writing this.  Will remove entirely.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-963111943


   > @cgivre Instead of generating `field_0`, `field_1` when no column names can be determined, is it possible to generate a `columns` array to be accessed using `columns[0], columns[1], ...` to remain consistent with what we do for CSV?
   
   It's possible to do that, but a few things:
   1.  Sometimes the PDF reader does not read tables perfectly and you get a mix of found headers and not found headers, so that's one reason I took that approach.
   2.  I actually dislike the `columns` approach from the CSV readers because it increases the level of complexity of queries.  In theory, if someone is querying a table (doesn't matter from where) they will want that broken into columns and rows.  The columns array approach (IMHO) makes this a lot harder that it needs to be.  
   3. This actually follows the model used in the Excel reader.


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] paul-rogers commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-959821275


   Cool contribution. I'm not entirely convinced this is something that Drill should handle. There are too many variables for the very limited controls which Drill provides. It is likely that this will work for one or two limited use cases, but not the vast majority of PDF files. Using JSON plugin config files to specify the mapping is awkward. Probably each file will need its own config, which is not scalable.
   
   Drill's fundamental design is to run at scale. It is highly unlikely that someone will use PDF files to store GB of data. If they do, they have problems bigger than Drill can help them solve. Thus, this kind of plugin works only at the small scale: one or two files in, say, an embedded Drillbit with JDBC or SQLine.
   
   A better choice would be to wrap this thing in a script: tinker with the PDF extraction, using whatever tools are available, to get the right mapping. Then, wrap that in a script that produces, say, a CSV format to stdout. Drill can then read that input.
   
   Such an approach enables all manner of ad-hoc, small scale data extraction.
   
   Or, maybe Drill should offer a "desktop edition" that is designed for small, ad-hoc projects based on local files, with some way to handle all the tinkering needed when reading PDF files, images, Word files, spreadsheets, Twitter feeds, Slack posts another formats popular with data scientists. Such features would not normally be part of the massive-scale deployments for which Drill is designed.
   
   Thoughts?


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-959493139






-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-960348867






-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-964722438


   > > * Sometimes the PDF reader does not read tables perfectly and you get a mix of found headers and not found headers, so that's one reason I took that approach.
   > 
   > This consideration will only apply for `extractHeaders = true`, right? In this case all of the readers do split out columns so I think we're good.
   > 
   > > * I actually dislike the `columns` approach from the CSV readers because it increases the level of complexity of queries.  In theory, if someone is querying a table (doesn't matter from where) they will want that broken into columns and rows.  The columns array approach (IMHO) makes this a lot harder that it needs to be.
   > 
   > The columns array does allow text files to contain jagged arrays, which is perhaps valuable? I don't see a huge saving in typing `select field_0, field_1` over `select columns[0], columns[1]`, am I missing some other complication?
   > 
   > > * This actually follows the model used in the Excel reader.
   > 
   > My fear, and I don't know if this is real or not, is that if we profilerate plugin-specific quirks in how the schema is represented to the user then the promise of a standard SQL interface to the data gets tainted and developers and users will be put off.
   > 
   > "It's standard except every plugin presents its data the way that author prefers" "That fragment of code I sent you for the columns array won't work here because this plugin does it differently" "This SQL script makes for confusing reading because the plugins involved name their generated columns differently, sometimes 'column', sometimes 'field', sometimes 'var'" etc.
   > 
   > I don't feel all that strongly about `field_0` vs `columns[0]` (I mean, maybe we deprecate the columns array?) but I am finding myself thinking more and more about consistency.
   
   The bigger implication of having a `columns` array vs `field_n` is when a user starts with `SELECT *` queries.  It makes it harder for BI tools to gather schema metadata and it also is non-standard SQL.  Now... querying PDF is also non-standard SQL... so maybe that's less important.  But, it makes the discovery a little harder IMHO.  
   


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] pjfanning commented on pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
pjfanning commented on pull request #2359:
URL: https://github.com/apache/drill/pull/2359#issuecomment-999002485


   @cgivre new pdfbox release: https://downloads.apache.org/pdfbox/2.0.25/RELEASE-NOTES.txt


-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [drill] cgivre commented on a change in pull request #2359: DRILL-8028: Add PDF Format Plugin

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2359:
URL: https://github.com/apache/drill/pull/2359#discussion_r768094421



##########
File path: contrib/format-pdf/src/main/java/org/apache/drill/exec/store/pdf/PdfBatchReader.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.drill.exec.store.pdf;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.shaded.guava.com.google.common.base.Strings;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework;
+import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.pdfbox.pdmodel.PDDocument;
+import org.apache.pdfbox.pdmodel.PDDocumentInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import technology.tabula.RectangularTextContainer;
+import technology.tabula.Table;
+
+import java.io.InputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+
+public class PdfBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> {
+
+  private static final Logger logger = LoggerFactory.getLogger(PdfBatchReader.class);
+  private static final String NEW_FIELD_PREFIX = "field_";
+  private final int maxRecords;
+
+  private final List<PdfColumnWriter> writers;
+  private FileSplit split;
+  private CustomErrorContext errorContext;
+  private RowSetLoader rowWriter;
+  private InputStream fsStream;
+  private PDDocument document;
+  private PdfReaderConfig config;
+
+  private SchemaBuilder builder;
+  private List<String> columnHeaders;
+  private int currentRowIndex;
+  private Table currentTable;
+  private int currentTableIndex;
+  private int startingTableIndex;
+  private FileScanFramework.FileSchemaNegotiator negotiator;
+
+  // Document Metadata Fields
+  private int pageCount;
+  private String title;
+  private String author;
+  private String subject;
+  private String keywords;
+  private String creator;
+  private String producer;
+  private Calendar creationDate;
+  private Calendar modificationDate;
+  private String trapped;
+  private int unregisteredColumnCount;
+  private int columns;
+  private int tableCount;
+  private int rows;
+  private int metadataIndex;
+
+
+  // Tables
+  private List<Table> tables;
+
+
+  static class PdfReaderConfig {
+    final PdfFormatPlugin plugin;
+
+    PdfReaderConfig(PdfFormatPlugin plugin) {
+      this.plugin = plugin;
+    }
+  }
+
+  public PdfBatchReader(PdfReaderConfig readerConfig, int maxRecords) {
+    this.maxRecords = maxRecords;
+    this.unregisteredColumnCount = 0;
+    this.writers = new ArrayList<>();
+    this.config = readerConfig;
+    this.startingTableIndex = readerConfig.plugin.getConfig().getDefaultTableIndex() < 0 ? 0 : readerConfig.plugin.getConfig().getDefaultTableIndex();
+    this.currentTableIndex = this.startingTableIndex;
+    this.columnHeaders = new ArrayList<>();
+  }
+
+  @Override
+  public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) {
+    System.setProperty("java.awt.headless", "true");
+    this.negotiator = negotiator;
+
+    split = negotiator.split();
+    errorContext = negotiator.parentErrorContext();
+    builder = new SchemaBuilder();
+
+    openFile();
+
+    // Get the tables
+    tables = Utils.extractTablesFromPDF(document);
+    populateMetadata();
+    logger.debug("Found {} tables", tables.size());
+
+    // Support provided schema
+    TupleMetadata schema = null;
+    if (this.negotiator.hasProvidedSchema()) {
+      schema = this.negotiator.providedSchema();
+      this.negotiator.tableSchema(schema, false);
+    } else {
+      this.negotiator.tableSchema(buildSchema(), false);
+    }
+    ResultSetLoader loader = this.negotiator.build();
+    rowWriter = loader.writer();
+
+    if (negotiator.hasProvidedSchema()) {
+      buildWriterListFromProvidedSchema(schema);
+    } else {
+      buildWriterList();
+    }
+    addImplicitColumnsToSchema();
+
+    // Prepare for reading
+    currentRowIndex = 1;  // Skip the first line if there are headers
+    currentTable = tables.get(startingTableIndex);
+
+    return true;
+  }
+
+  @Override
+  public boolean next() {
+    System.setProperty("java.awt.headless", "true");
+
+    while(!rowWriter.isFull()) {
+      // Check to see if the limit has been reached
+      if (rowWriter.limitReached(maxRecords)) {
+        return false;
+      } else if (currentRowIndex >= currentTable.getRows().size() &&
+                  currentTableIndex < tables.size() &&
+                  config.plugin.getConfig().getCombinePages()) {
+        currentRowIndex = 0;
+        currentTable = tables.get(currentTableIndex++);
+      } else if (currentRowIndex >= currentTable.getRows().size()) {
+        return false;
+      }
+
+      // Process the row
+      processRow(currentTable.getRows().get(currentRowIndex));
+      currentRowIndex++;
+    }
+    return true;
+  }
+
+  private void processRow(List<RectangularTextContainer> row) {
+    if (row == null || row.size() == 0) {
+      return;
+    }
+
+    String value;
+    rowWriter.start();
+    for (int i = 0; i < row.size(); i++) {
+      value = row.get(i).getText();
+
+      if (Strings.isNullOrEmpty(value)) {
+        continue;
+      }
+      writers.get(i).load(row.get(i));
+    }
+    writeMetadata();
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    if (fsStream != null) {

Review comment:
       Fixed.




-- 
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: dev-unsubscribe@drill.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org