You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by cg...@apache.org on 2023/01/16 05:08:46 UTC

[drill] branch master updated: DRILL-8384: Add Format Plugin for Microsoft Access (#2737)

This is an automated email from the ASF dual-hosted git repository.

cgivre pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new 5db6faf695 DRILL-8384: Add Format Plugin for Microsoft Access (#2737)
5db6faf695 is described below

commit 5db6faf695d602b8031f7a35563062ad3d03c591
Author: Charles S. Givre <cg...@apache.org>
AuthorDate: Mon Jan 16 00:08:39 2023 -0500

    DRILL-8384: Add Format Plugin for Microsoft Access (#2737)
---
 contrib/format-access/README.md                    |  42 +++
 contrib/format-access/pom.xml                      |  73 ++++
 .../exec/store/msaccess/MSAccessBatchReader.java   | 388 +++++++++++++++++++++
 .../exec/store/msaccess/MSAccessFormatConfig.java  |  83 +++++
 .../exec/store/msaccess/MSAccessFormatPlugin.java  | 109 ++++++
 .../main/resources/bootstrap-format-plugins.json   |  37 ++
 .../src/main/resources/drill-module.conf           |  24 ++
 .../exec/store/msaccess/TestMSAccessReader.java    | 129 +++++++
 .../src/test/resources/data/V1997/testV1997.mdb    | Bin 0 -> 118784 bytes
 .../src/test/resources/data/V2000/testV2000.mdb    | Bin 0 -> 3043328 bytes
 .../src/test/resources/data/V2003/testV2003.mdb    | Bin 0 -> 3162112 bytes
 .../src/test/resources/data/V2007/testV2007.accdb  | Bin 0 -> 425984 bytes
 .../src/test/resources/data/V2010/testV2010.accdb  | Bin 0 -> 446464 bytes
 .../resources/data/V2019/extDateTestV2019.accdb    | Bin 0 -> 557056 bytes
 .../src/test/resources/data/adox_jet4.mdb          | Bin 0 -> 65536 bytes
 contrib/pom.xml                                    |   1 +
 distribution/pom.xml                               |   5 +
 distribution/src/assemble/component.xml            |   1 +
 pom.xml                                            |   4 +
 19 files changed, 896 insertions(+)

diff --git a/contrib/format-access/README.md b/contrib/format-access/README.md
new file mode 100644
index 0000000000..0c3035117a
--- /dev/null
+++ b/contrib/format-access/README.md
@@ -0,0 +1,42 @@
+# Drill MS Access Format Plugin
+This plugin enables Drill to read Microsoft Access database files. This plugin can read Access Files from all versions later than Access 1997.
+
+## Configuration
+Simply add the following to any Drill file system configuration.  Typically, MS Access files will use the extension `accdb` or `mdb`.  Drill comes pre-configured to recognize these extensions as MS Access.
+
+```json
+"msaccess": {
+  "type": "msaccess",
+  "extensions": ["mdb", "accdb"]
+}
+```
+
+## Schemas
+Drill will discover the schema automatically from the Access file.  The plugin does support schema provisioning for consistency, but is not recommended.
+
+## Querying a Table
+Access files will contain multiple tables.  To access a specific table, use the `table()` function in the `FROM` clause, and specify the table name using the `tableName` parameter, as shown below.
+
+```sql
+SELECT * 
+FROM table(dfs.`file_name.accdb` (type=> 'msaccess', tableName => 'Table1'))
+```
+
+## Metadata Queries
+Since an Access file may contain multiple tables, there needs to be a way to determine what tables are present in the Access file.  In Drill, simply querying a file, without specifying a `tableName` will result in a metadata query, rather than getting the actual data back.  
+
+For example:
+
+```sql
+SELECT * FROM dfs.test.`access/data/V2019/extDateTestV2019.accdb`;
++--------+-------------------------+-------------------------+-----------+-----------+----------------------------------------------------------------------+
+| table  |      created_date       |      updated_date       | row_count | col_count |                               columns                                |
++--------+-------------------------+-------------------------+-----------+-----------+----------------------------------------------------------------------+
+| Table1 | 2021-06-03 20:09:56.993 | 2021-06-03 20:09:56.993 | 9         | 6         | ["ID","Field1","DateExt","DateNormal","DateExtStr","DateNormalCalc"] |
++--------+-------------------------+-------------------------+-----------+-----------+----------------------------------------------------------------------+
+
+```
+
+
+## Password-Protected Files
+The password protection in Access is just a software level protection and really does not offer any security.  Drill can query password protected files without any password.
\ No newline at end of file
diff --git a/contrib/format-access/pom.xml b/contrib/format-access/pom.xml
new file mode 100644
index 0000000000..45fa4e3b22
--- /dev/null
+++ b/contrib/format-access/pom.xml
@@ -0,0 +1,73 @@
+<?xml version="1.0"?>
+<!--
+
+    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.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>drill-contrib-parent</artifactId>
+        <groupId>org.apache.drill.contrib</groupId>
+        <version>1.21.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>drill-format-msaccess</artifactId>
+
+    <name>Drill : Contrib : Format : MS Access</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.drill.exec</groupId>
+            <artifactId>drill-java-exec</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.healthmarketscience.jackcess</groupId>
+            <artifactId>jackcess</artifactId>
+            <version>4.0.4</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>commons-logging</groupId>
+                    <artifactId>commons-logging-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-logging</groupId>
+                    <artifactId>commons-logging</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <!-- Test dependency -->
+        <dependency>
+            <groupId>org.apache.drill.exec</groupId>
+            <artifactId>drill-java-exec</artifactId>
+            <classifier>tests</classifier>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.drill</groupId>
+            <artifactId>drill-common</artifactId>
+            <classifier>tests</classifier>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+</project>
diff --git a/contrib/format-access/src/main/java/org/apache/drill/exec/store/msaccess/MSAccessBatchReader.java b/contrib/format-access/src/main/java/org/apache/drill/exec/store/msaccess/MSAccessBatchReader.java
new file mode 100644
index 0000000000..370cbd290b
--- /dev/null
+++ b/contrib/format-access/src/main/java/org/apache/drill/exec/store/msaccess/MSAccessBatchReader.java
@@ -0,0 +1,388 @@
+/*
+ * 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.msaccess;
+
+import com.healthmarketscience.jackcess.Column;
+import com.healthmarketscience.jackcess.DataType;
+import com.healthmarketscience.jackcess.Database;
+import com.healthmarketscience.jackcess.DatabaseBuilder;
+import com.healthmarketscience.jackcess.Row;
+import com.healthmarketscience.jackcess.Table;
+import org.apache.commons.lang3.StringUtils;
+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.MinorType;
+import org.apache.drill.exec.physical.impl.scan.v3.FixedReceiver;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileDescrip;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+public class MSAccessBatchReader implements ManagedReader {
+
+  private static final Logger logger = LoggerFactory.getLogger(MSAccessBatchReader.class);
+
+  private final FileDescrip file;
+  private final CustomErrorContext errorContext;
+  private final RowSetLoader rowWriter;
+  private final File tempDir;
+  private final List<MSAccessColumn> columnList;
+  private final MSAccessFormatConfig config;
+  private final boolean metadataOnly;
+  private File tempFile;
+  private Set<String> tableList;
+  private Iterator<Row> rowIterator;
+  private Iterator<String> tableIterator;
+  private InputStream fsStream;
+  private Table table;
+  private Database db;
+
+  public MSAccessBatchReader(FileSchemaNegotiator negotiator, File tempDir, MSAccessFormatConfig config) {
+    this.tempDir = tempDir;
+    this.columnList = new ArrayList<>();
+    this.config = config;
+    this.file = negotiator.file();
+    this.errorContext = negotiator.parentErrorContext();
+    this.metadataOnly = StringUtils.isEmpty(config.getTableName());
+
+    openFile();
+    buildSchema(negotiator);
+
+    if (metadataOnly) {
+      tableIterator = tableList.iterator();
+    } else {
+      rowIterator = table.iterator();
+    }
+
+    ResultSetLoader loader = negotiator.build();
+    rowWriter = loader.writer();
+  }
+
+  /**
+   * Constructs the schema and adds it to the {@link FileSchemaNegotiator}. In the event a table is not
+   * specified, a metadata schema will be returned which will be useful for discovering the tables present in a
+   * given file.
+   * @param negotiator The {@link FileSchemaNegotiator} from the plugin.
+   */
+  private void buildSchema(FileSchemaNegotiator negotiator) {
+    // Now build the schema
+    SchemaBuilder schemaBuilder = new SchemaBuilder();
+
+    if (metadataOnly) {
+      TupleMetadata metadataSchema = buildMetadataSchema(schemaBuilder);
+      negotiator.tableSchema(metadataSchema, true);
+    } else {
+      // Add schema if provided. Users probably shouldn't use this.
+      TupleMetadata derivedSchema = buildSchemaFromTable(schemaBuilder, config.getTableName());
+      if (negotiator.providedSchema() != null) {
+        // Merge the provided schema with the schema from the file.
+        TupleMetadata mergeSchemas = FixedReceiver.Builder.mergeSchemas(negotiator.providedSchema(), derivedSchema);
+        negotiator.tableSchema(mergeSchemas, true);
+      } else {
+        negotiator.tableSchema(derivedSchema, true);
+      }
+    }
+  }
+
+  private TupleMetadata buildMetadataSchema(SchemaBuilder builder) {
+    // Adds the table name
+    builder.add("table", MinorType.VARCHAR);
+    builder.add("created_date", MinorType.TIMESTAMP);
+    builder.add("updated_date", MinorType.TIMESTAMP);
+    builder.add("row_count", MinorType.INT);
+    builder.add("col_count", MinorType.INT);
+    builder.addArray("columns", MinorType.VARCHAR);
+
+    return builder.buildSchema();
+  }
+
+  private TupleMetadata buildSchemaFromTable(SchemaBuilder builder, String tableName) {
+    try {
+      table = db.getTable(tableName);
+    } catch (IOException e) {
+      deleteTempFile();
+      throw UserException.dataReadError(e)
+          .message("Table " + config.getTableName() + " not found. " + e.getMessage())
+          .addContext(errorContext)
+          .build(logger);
+    }
+
+    List<? extends Column> columns = table.getColumns();
+
+    for (Column column : columns) {
+      MinorType drillDataType;
+      String columnName = column.getName();
+      DataType dataType = column.getType();
+
+      switch (dataType) {
+        case BOOLEAN:
+          builder.addNullable(columnName, MinorType.BIT);
+          drillDataType = MinorType.BIT;
+          break;
+        case BYTE:
+        case INT:
+          builder.addNullable(columnName, MinorType.SMALLINT);
+          drillDataType = MinorType.SMALLINT;
+          break;
+        case LONG:
+          builder.addNullable(columnName, MinorType.INT);
+          drillDataType = MinorType.INT;
+          break;
+        case BIG_INT:
+        case COMPLEX_TYPE:
+          builder.addNullable(columnName, MinorType.BIGINT);
+          drillDataType = MinorType.BIGINT;
+          break;
+        case FLOAT:
+          builder.addNullable(columnName, MinorType.FLOAT4);
+          drillDataType = MinorType.FLOAT4;
+          break;
+        case MEMO:
+        case TEXT:
+        case GUID:
+          builder.addNullable(columnName, MinorType.VARCHAR);
+          drillDataType = MinorType.VARCHAR;
+          break;
+        case MONEY:
+        case DOUBLE:
+        case NUMERIC:
+          builder.addNullable(columnName, MinorType.FLOAT8);
+          drillDataType = MinorType.FLOAT8;
+          break;
+        case OLE:
+        case BINARY:
+        case UNSUPPORTED_VARLEN:
+        case UNSUPPORTED_FIXEDLEN:
+        case UNKNOWN_0D:
+        case UNKNOWN_11:
+          builder.addNullable(columnName, MinorType.VARBINARY);
+          drillDataType = MinorType.VARBINARY;
+          break;
+        case EXT_DATE_TIME:
+        case SHORT_DATE_TIME:
+          builder.addNullable(columnName, MinorType.TIMESTAMP);
+          drillDataType = MinorType.TIMESTAMP;
+          break;
+        default:
+          deleteTempFile();
+          throw UserException.dataReadError()
+              .message(dataType.name() + " is not supported.")
+              .build(logger);
+      }
+      columnList.add(new MSAccessColumn(columnName, drillDataType));
+    }
+    return builder.buildSchema();
+  }
+
+  @Override
+  public boolean next() {
+    while (!rowWriter.isFull()) {
+      if (metadataOnly) {
+        if (tableIterator.hasNext()) {
+          try {
+            processMetadataRow(tableIterator.next());
+          } catch (IOException e) {
+            deleteTempFile();
+            throw UserException.dataReadError(e)
+                .message("Error retrieving metadata for table: " + e.getMessage())
+                .addContext(errorContext)
+                .build(logger);
+          }
+        } else {
+          return false;
+        }
+      } else {
+        if (rowIterator.hasNext()) {
+          processRow(rowIterator.next());
+        } else {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  private void openFile() {
+    try {
+      fsStream = file.fileSystem().openPossiblyCompressedStream(file.split().getPath());
+      db = DatabaseBuilder.open(convertInputStreamToFile(fsStream));
+      tableList = db.getTableNames();
+    } catch (IOException e) {
+      deleteTempFile();
+      throw UserException.dataReadError(e)
+          .message("Error reading MS Access file: " + e.getMessage())
+          .addContext(errorContext)
+          .build(logger);
+    }
+  }
+
+  private void processMetadataRow(String tableName) throws IOException {
+    Table table;
+    try {
+      table = db.getTable(tableName);
+    } catch (IOException e) {
+      deleteTempFile();
+      throw UserException.dataReadError(e)
+          .message("Error retrieving metadata for table " + tableName + ": " + e.getMessage())
+          .addContext(errorContext)
+          .build(logger);
+    }
+
+    rowWriter.start();
+    rowWriter.scalar("table").setString(tableName);
+    LocalDateTime createdDate = table.getCreatedDate();
+    rowWriter.scalar("created_date").setTimestamp(createdDate.toInstant(ZoneOffset.UTC));
+    LocalDateTime updatedDate = table.getCreatedDate();
+    rowWriter.scalar("updated_date").setTimestamp(updatedDate.toInstant(ZoneOffset.UTC));
+    rowWriter.scalar("row_count").setInt(table.getRowCount());
+    rowWriter.scalar("col_count").setInt(table.getColumnCount());
+    // Write the columns
+    ArrayWriter arrayWriter = rowWriter.array("columns");
+    for (Column column : table.getColumns()) {
+      arrayWriter.scalar().setString(column.getName());
+    }
+    arrayWriter.save();
+    rowWriter.save();
+  }
+
+  private void processRow(Row next) {
+    rowWriter.start();
+    for (MSAccessColumn col : columnList) {
+      switch (col.dataType) {
+        case BIT:
+          Boolean boolValue = next.getBoolean(col.columnName);
+          rowWriter.scalar(col.columnName).setBoolean(boolValue);
+          break;
+        case SMALLINT:
+          Short shortValue = next.getShort(col.columnName);
+          rowWriter.scalar(col.columnName).setInt(shortValue);
+          break;
+        case BIGINT:
+        case INT:
+          Integer intValue = next.getInt(col.columnName);
+          if (intValue != null) {
+            rowWriter.scalar(col.columnName).setInt(intValue);
+          }
+          break;
+        case FLOAT4:
+          Float floatValue = next.getFloat(col.columnName);
+          if (floatValue != null) {
+            rowWriter.scalar(col.columnName).setFloat(floatValue);
+          }
+          break;
+        case FLOAT8:
+          Double doubleValue = next.getDouble(col.columnName);
+          rowWriter.scalar(col.columnName).setDouble(doubleValue);
+          break;
+        case VARCHAR:
+          String stringValue = next.getString(col.columnName);
+          if (StringUtils.isNotEmpty(stringValue)) {
+            rowWriter.scalar(col.columnName).setString(stringValue);
+          }
+          break;
+        case TIMESTAMP:
+          LocalDateTime tsValue = next.getLocalDateTime(col.columnName);
+          if (tsValue != null) {
+            rowWriter.scalar(col.columnName).setTimestamp(tsValue.toInstant(ZoneOffset.UTC));
+          }
+          break;
+        case VARBINARY:
+          byte[] byteValue = next.getBytes(col.columnName);
+          rowWriter.scalar(col.columnName).setBytes(byteValue, byteValue.length);
+          break;
+      }
+    }
+    rowWriter.save();
+  }
+
+  @Override
+  public void close() {
+    AutoCloseables.closeSilently(db);
+    AutoCloseables.closeSilently(fsStream);
+    deleteTempFile();
+  }
+
+  /**
+   * This function converts the Drill InputStream into a File object for the Jackcess library. This function
+   * exists due to a known limitation in the Jacksess library which cannot parse MS Access directly from an input stream.
+   *
+   * @param stream The {@link InputStream} to be converted to a File
+   * @return {@link File} The file which was converted from an {@link InputStream}
+   */
+  private File convertInputStreamToFile(InputStream stream) {
+    String tempFileName = tempDir.getPath() + "/~" + file.filePath().getName();
+    tempFile = new File(tempFileName);
+
+    try {
+      Files.copy(stream, tempFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+    } catch (Exception e) {
+      if (tempFile.exists()) {
+        if (!tempFile.delete()) {
+          logger.warn("{} not deleted.", tempFile.getName());
+        }
+      }
+      throw UserException
+          .dataWriteError(e)
+          .message("Failed to create temp HDF5 file: %s", file.filePath())
+          .addContext(e.getMessage())
+          .build(logger);
+    }
+
+    AutoCloseables.closeSilently(stream);
+    return tempFile;
+  }
+
+  private void deleteTempFile() {
+    if (tempFile != null) {
+      if (!tempFile.delete()) {
+        logger.warn("{} file not deleted.", tempFile.getName());
+      }
+      tempFile = null;
+    }
+  }
+
+  private static class MSAccessColumn {
+    private final String columnName;
+    private final MinorType dataType;
+
+    public MSAccessColumn(String columnName, MinorType dataType) {
+      this.columnName = columnName;
+      this.dataType = dataType;
+    }
+  }
+}
diff --git a/contrib/format-access/src/main/java/org/apache/drill/exec/store/msaccess/MSAccessFormatConfig.java b/contrib/format-access/src/main/java/org/apache/drill/exec/store/msaccess/MSAccessFormatConfig.java
new file mode 100644
index 0000000000..c291e752ef
--- /dev/null
+++ b/contrib/format-access/src/main/java/org/apache/drill/exec/store/msaccess/MSAccessFormatConfig.java
@@ -0,0 +1,83 @@
+/*
+ * 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.msaccess;
+
+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.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+@JsonTypeName(MSAccessFormatPlugin.DEFAULT_NAME)
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class MSAccessFormatConfig implements FormatPluginConfig {
+  private final List<String> extensions;
+  private final String tableName;
+
+  // Omitted properties take reasonable defaults
+  @JsonCreator
+  public MSAccessFormatConfig(@JsonProperty("extensions") List<String> extensions,
+                              @JsonProperty("tableName") String tableName) {
+    this.extensions = extensions == null ? Arrays.asList("accdb", "mdb") : ImmutableList.copyOf(extensions);
+    this.tableName = tableName;
+  }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  public List<String> getExtensions() {
+    return extensions;
+  }
+
+  @JsonInclude(Include.NON_DEFAULT)
+  public String getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    MSAccessFormatConfig that = (MSAccessFormatConfig) o;
+    return Objects.equals(extensions, that.extensions) &&
+        Objects.equals(tableName, that.tableName);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(extensions, tableName);
+  }
+
+  @Override
+  public String toString() {
+    return new PlanStringBuilder(this)
+        .field("extensions", extensions)
+        .field("tableName", tableName)
+        .toString();
+  }
+}
diff --git a/contrib/format-access/src/main/java/org/apache/drill/exec/store/msaccess/MSAccessFormatPlugin.java b/contrib/format-access/src/main/java/org/apache/drill/exec/store/msaccess/MSAccessFormatPlugin.java
new file mode 100644
index 0000000000..bfad78a7a6
--- /dev/null
+++ b/contrib/format-access/src/main/java/org/apache/drill/exec/store/msaccess/MSAccessFormatPlugin.java
@@ -0,0 +1,109 @@
+/*
+ * 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.msaccess;
+
+import org.apache.drill.shaded.guava.com.google.common.io.Files;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileReaderFactory;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileScanLifecycleBuilder;
+import org.apache.drill.exec.physical.impl.scan.v3.file.FileSchemaNegotiator;
+import org.apache.drill.exec.physical.impl.scan.v3.ManagedReader;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
+import org.apache.drill.exec.store.dfs.easy.EasySubScan;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.File;
+
+
+public class MSAccessFormatPlugin extends EasyFormatPlugin<MSAccessFormatConfig> {
+
+  protected static final String DEFAULT_NAME = "msaccess";
+  private final DrillbitContext context;
+
+  private static class MSAccessReaderFactory extends FileReaderFactory {
+
+    private final File tempDir;
+    private final MSAccessFormatConfig config;
+    public MSAccessReaderFactory(File tempDir, MSAccessFormatConfig config) {
+      this.tempDir = tempDir;
+      this.config = config;
+    }
+
+    @Override
+    public ManagedReader newReader(FileSchemaNegotiator negotiator) {
+      return new MSAccessBatchReader(negotiator, tempDir, config);
+    }
+  }
+
+  public MSAccessFormatPlugin(String name, DrillbitContext context,
+                         Configuration fsConf, StoragePluginConfig storageConfig,
+                         MSAccessFormatConfig formatConfig) {
+    super(name, easyConfig(fsConf, formatConfig), context, storageConfig, formatConfig);
+    this.context = context;
+  }
+
+  private static EasyFormatConfig easyConfig(Configuration fsConf, MSAccessFormatConfig pluginConfig) {
+    return EasyFormatConfig.builder()
+        .readable(true)
+        .writable(false)
+        .blockSplittable(false)
+        .blockSplittable(false)
+        .compressible(true)
+        .supportsProjectPushdown(true)
+        .extensions(pluginConfig.getExtensions())
+        .fsConf(fsConf)
+        .defaultName(DEFAULT_NAME)
+        .scanVersion(ScanFrameworkVersion.EVF_V2)
+        .supportsLimitPushdown(true)
+        .build();
+  }
+
+  @Override
+  protected void configureScan(FileScanLifecycleBuilder builder, EasySubScan scan) {
+    builder.nullType(Types.optional(TypeProtos.MinorType.VARCHAR));
+    builder.readerFactory(new MSAccessReaderFactory(getTmpDir(), formatConfig));
+  }
+
+  /**
+   * First tries to get drill temporary directory value from config ${drill.tmp-dir},
+   * then checks environmental variable $DRILL_TMP_DIR.
+   * If value is still missing, generates directory using {@link Files#createTempDir()}.
+   *
+   * @return drill temporary directory path
+   */
+  protected File getTmpDir() {
+    DrillConfig config = context.getConfig();
+    String drillTempDir;
+    if (config.hasPath(ExecConstants.DRILL_TMP_DIR)) {
+      drillTempDir = config.getString(ExecConstants.DRILL_TMP_DIR);
+    } else {
+      drillTempDir = System.getenv("DRILL_TMP_DIR");
+    }
+
+    if (drillTempDir == null) {
+      return Files.createTempDir();
+    }
+    return new File(drillTempDir);
+  }
+}
diff --git a/contrib/format-access/src/main/resources/bootstrap-format-plugins.json b/contrib/format-access/src/main/resources/bootstrap-format-plugins.json
new file mode 100644
index 0000000000..73e78d1f54
--- /dev/null
+++ b/contrib/format-access/src/main/resources/bootstrap-format-plugins.json
@@ -0,0 +1,37 @@
+{
+  "storage": {
+    "dfs": {
+      "type": "file",
+      "formats": {
+        "msaccess": {
+          "type": "msaccess",
+          "extensions": [
+            "mdb", "accdb"
+          ]
+        }
+      }
+    },
+    "cp": {
+      "type": "file",
+      "formats": {
+        "msaccess": {
+          "type": "msaccess",
+          "extensions": [
+            "mdb", "accdb"
+          ]
+        }
+      }
+    },
+    "s3": {
+      "type": "file",
+      "formats": {
+        "msaccess": {
+          "type": "msaccess",
+          "extensions": [
+            "mdb", "accdb"
+          ]
+        }
+      }
+    }
+  }
+}
diff --git a/contrib/format-access/src/main/resources/drill-module.conf b/contrib/format-access/src/main/resources/drill-module.conf
new file mode 100644
index 0000000000..7004e22a8b
--- /dev/null
+++ b/contrib/format-access/src/main/resources/drill-module.conf
@@ -0,0 +1,24 @@
+#
+# 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.msaccess"
+}
diff --git a/contrib/format-access/src/test/java/org/apache/drill/exec/store/msaccess/TestMSAccessReader.java b/contrib/format-access/src/test/java/org/apache/drill/exec/store/msaccess/TestMSAccessReader.java
new file mode 100644
index 0000000000..b170df8710
--- /dev/null
+++ b/contrib/format-access/src/test/java/org/apache/drill/exec/store/msaccess/TestMSAccessReader.java
@@ -0,0 +1,129 @@
+/*
+ * 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.msaccess;
+
+import org.apache.drill.categories.RowSetTest;
+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.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryTestUtil;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+
+import static org.apache.drill.test.rowSet.RowSetUtilities.strArray;
+import static org.junit.Assert.assertEquals;
+
+@Category(RowSetTest.class)
+public class TestMSAccessReader extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+  }
+
+  @Test
+  public void testStarQuery() throws Exception {
+    String sql = "SELECT * FROM table(cp.`data/V2019/extDateTestV2019.accdb` (type=> 'msaccess', tableName => 'Table1')) LIMIT 5";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("ID", MinorType.INT)
+        .addNullable("Field1", MinorType.VARCHAR)
+        .addNullable("DateExt", MinorType.TIMESTAMP)
+        .addNullable("DateNormal", MinorType.TIMESTAMP)
+        .addNullable("DateExtStr", MinorType.VARCHAR)
+        .addNullable("DateNormalCalc", MinorType.TIMESTAMP)
+        .buildSchema();
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+        .addRow(1, "row1", QueryTestUtil.ConvertDateToLong("2020-06-17T00:00:00Z"), QueryTestUtil.ConvertDateToLong("2020-06-17T00:00:00Z"), "6/17/2020", QueryTestUtil.ConvertDateToLong("2020-06-17T00:00:00Z"))
+        .addRow(2, "row2", QueryTestUtil.ConvertDateToLong("2021-06-14T00:00:00Z"), QueryTestUtil.ConvertDateToLong("2021-06-14T00:00:00Z"), "6/14/2021", QueryTestUtil.ConvertDateToLong("2021-06-14T00:00:00Z"))
+        .addRow(3, "row3", QueryTestUtil.ConvertDateToLong("2021-06-14T12:45:00Z"), QueryTestUtil.ConvertDateToLong("2021-06-14T12:45:00Z"), "6/14/2021 12:45:00.0000000 PM", QueryTestUtil.ConvertDateToLong("2021-06-14T12:45:00Z"))
+        .addRow(4, "row4", QueryTestUtil.ConvertDateToLong("2021-06-14T01:45:00Z"), QueryTestUtil.ConvertDateToLong("2021-06-14T01:45:00Z"), "6/14/2021 1:45:00.0000000 AM", QueryTestUtil.ConvertDateToLong("2021-06-14T01:45:00Z"))
+        .addRow(5, "row5", null, null, null, null)
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testExplicitQuery() throws Exception {
+    String sql = "SELECT ID, Field1, DateExt, DateNormal, DateExtStr, DateNormalCalc " +
+        "FROM table(cp.`data/V2019/extDateTestV2019.accdb` (type=> 'msaccess', tableName => 'Table1')) LIMIT 5";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("ID", MinorType.INT)
+        .addNullable("Field1", MinorType.VARCHAR)
+        .addNullable("DateExt", MinorType.TIMESTAMP)
+        .addNullable("DateNormal", MinorType.TIMESTAMP)
+        .addNullable("DateExtStr", MinorType.VARCHAR)
+        .addNullable("DateNormalCalc", MinorType.TIMESTAMP)
+        .buildSchema();
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+        .addRow(1, "row1", QueryTestUtil.ConvertDateToLong("2020-06-17T00:00:00Z"), QueryTestUtil.ConvertDateToLong("2020-06-17T00:00:00Z"), "6/17/2020", QueryTestUtil.ConvertDateToLong("2020-06-17T00:00:00Z"))
+        .addRow(2, "row2", QueryTestUtil.ConvertDateToLong("2021-06-14T00:00:00Z"), QueryTestUtil.ConvertDateToLong("2021-06-14T00:00:00Z"), "6/14/2021", QueryTestUtil.ConvertDateToLong("2021-06-14T00:00:00Z"))
+        .addRow(3, "row3", QueryTestUtil.ConvertDateToLong("2021-06-14T12:45:00Z"), QueryTestUtil.ConvertDateToLong("2021-06-14T12:45:00Z"), "6/14/2021 12:45:00.0000000 PM", QueryTestUtil.ConvertDateToLong("2021-06-14T12:45:00Z"))
+        .addRow(4, "row4", QueryTestUtil.ConvertDateToLong("2021-06-14T01:45:00Z"), QueryTestUtil.ConvertDateToLong("2021-06-14T01:45:00Z"), "6/14/2021 1:45:00.0000000 AM", QueryTestUtil.ConvertDateToLong("2021-06-14T01:45:00Z"))
+        .addRow(5, "row5", null, null, null, null)
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testMetadataStarQuery() throws Exception {
+    String sql = "SELECT * FROM cp.`data/V2019/extDateTestV2019.accdb`";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("table", MinorType.VARCHAR)
+        .add("created_date", MinorType.TIMESTAMP)
+        .add("updated_date", MinorType.TIMESTAMP)
+        .add("row_count", MinorType.INT)
+        .add("col_count", MinorType.INT)
+        .addArray("columns", MinorType.VARCHAR)
+        .buildSchema();
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+        .addRow("Table1", QueryTestUtil.ConvertDateToLong("2021-06-03T20:09:56.993Z"),
+            QueryTestUtil.ConvertDateToLong("2021-06-03T20:09:56.993Z"), 9, 6, strArray("ID", "Field1", "DateExt", "DateNormal", "DateExtStr", "DateNormalCalc"))
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+
+  @Test
+  public void testSerDe() throws Exception {
+    String sql = "SELECT COUNT(*) AS cnt FROM " +
+        "table(cp.`data/V2019/extDateTestV2019.accdb` (type=> 'msaccess', tableName => 'Table1'))";
+    String plan = queryBuilder().sql(sql).explainJson();
+    long cnt = queryBuilder().physical(plan).singletonLong();
+    assertEquals("Counts should match",9L, cnt);
+  }
+}
diff --git a/contrib/format-access/src/test/resources/data/V1997/testV1997.mdb b/contrib/format-access/src/test/resources/data/V1997/testV1997.mdb
new file mode 100644
index 0000000000..2aa10601a1
Binary files /dev/null and b/contrib/format-access/src/test/resources/data/V1997/testV1997.mdb differ
diff --git a/contrib/format-access/src/test/resources/data/V2000/testV2000.mdb b/contrib/format-access/src/test/resources/data/V2000/testV2000.mdb
new file mode 100644
index 0000000000..b9d004aa5f
Binary files /dev/null and b/contrib/format-access/src/test/resources/data/V2000/testV2000.mdb differ
diff --git a/contrib/format-access/src/test/resources/data/V2003/testV2003.mdb b/contrib/format-access/src/test/resources/data/V2003/testV2003.mdb
new file mode 100644
index 0000000000..a1a7fd878c
Binary files /dev/null and b/contrib/format-access/src/test/resources/data/V2003/testV2003.mdb differ
diff --git a/contrib/format-access/src/test/resources/data/V2007/testV2007.accdb b/contrib/format-access/src/test/resources/data/V2007/testV2007.accdb
new file mode 100644
index 0000000000..e16c664c03
Binary files /dev/null and b/contrib/format-access/src/test/resources/data/V2007/testV2007.accdb differ
diff --git a/contrib/format-access/src/test/resources/data/V2010/testV2010.accdb b/contrib/format-access/src/test/resources/data/V2010/testV2010.accdb
new file mode 100644
index 0000000000..179486a9b6
Binary files /dev/null and b/contrib/format-access/src/test/resources/data/V2010/testV2010.accdb differ
diff --git a/contrib/format-access/src/test/resources/data/V2019/extDateTestV2019.accdb b/contrib/format-access/src/test/resources/data/V2019/extDateTestV2019.accdb
new file mode 100755
index 0000000000..5a7d1eb0e8
Binary files /dev/null and b/contrib/format-access/src/test/resources/data/V2019/extDateTestV2019.accdb differ
diff --git a/contrib/format-access/src/test/resources/data/adox_jet4.mdb b/contrib/format-access/src/test/resources/data/adox_jet4.mdb
new file mode 100644
index 0000000000..7fbdb1b148
Binary files /dev/null and b/contrib/format-access/src/test/resources/data/adox_jet4.mdb differ
diff --git a/contrib/pom.xml b/contrib/pom.xml
index 9f58aa910e..fa3d1d3df7 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -49,6 +49,7 @@
     <module>format-httpd</module>
     <module>format-esri</module>
     <module>format-log</module>
+    <module>format-access</module>
     <module>format-pdf</module>
     <module>format-hdf5</module>
     <module>format-sas</module>
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 233853f36b..bc4f4907a0 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -469,6 +469,11 @@
           <artifactId>drill-format-excel</artifactId>
           <version>${project.version}</version>
         </dependency>
+        <dependency>
+          <groupId>org.apache.drill.contrib</groupId>
+          <artifactId>drill-format-msaccess</artifactId>
+          <version>${project.version}</version>
+        </dependency>
         <dependency>
           <groupId>org.apache.drill.contrib</groupId>
           <artifactId>drill-format-log</artifactId>
diff --git a/distribution/src/assemble/component.xml b/distribution/src/assemble/component.xml
index 36c7133430..e2b255269c 100644
--- a/distribution/src/assemble/component.xml
+++ b/distribution/src/assemble/component.xml
@@ -54,6 +54,7 @@
         <include>org.apache.drill.contrib:drill-format-httpd:jar</include>
         <include>org.apache.drill.contrib:drill-format-pdf:jar</include>
         <include>org.apache.drill.contrib:drill-format-excel:jar</include>
+        <include>org.apache.drill.contrib:drill-format-msaccess:jar</include>
         <include>org.apache.drill.contrib:drill-format-spss:jar</include>
         <include>org.apache.drill.contrib:drill-format-sas:jar</include>
         <include>org.apache.drill.contrib:drill-jdbc-storage:jar</include>
diff --git a/pom.xml b/pom.xml
index 9c94b922bb..649977af05 100644
--- a/pom.xml
+++ b/pom.xml
@@ -379,6 +379,8 @@
             <exclude>**/*.woff2</exclude>
             <exclude>**/*.ks</exclude>
             <exclude>**/*.pcap</exclude>
+            <exclude>**/*.mdb</exclude>
+            <exclude>**/*.accdb</exclude>
             <exclude>**/*.sas7bdat</exclude>
             <exclude>**/*.log1</exclude>
             <exclude>**/*.log2</exclude>
@@ -713,6 +715,8 @@
               <exclude>**/*.syslog</exclude>
               <exclude>**/*.xls</exclude>
               <exclude>**/*.xlsx</exclude>
+              <exclude>**/*.mdb</exclude>
+              <exclude>**/*.accdb</exclude>
               <exclude>**/*.syslog1</exclude>
               <exclude>**/*.ssdlog</exclude>
               <exclude>**/*.ltsv</exclude>