You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2019/07/22 07:14:24 UTC

[drill] branch master updated: DRILL-7272: Drill Metastore Read / Write API and Drill Iceberg Metastore implementation

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

arina 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 efbf4fc  DRILL-7272: Drill Metastore Read / Write API and Drill Iceberg Metastore implementation
efbf4fc is described below

commit efbf4fc4a7ae1acd2641f4e40e0697d987337a42
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Tue Jun 18 21:29:22 2019 +0300

    DRILL-7272: Drill Metastore Read / Write API and Drill Iceberg Metastore implementation
    
    1. Drill Metastore Read / Write API.
    2. Drill Iceberg Metastore implementation in iceberg-metastore module.
    3. Patches Guava Preconditions class for Apache Iceberg.
    4. General refactoring.
    5. Unit tests.
    6. Documentation.
---
 .../apache/drill/common/config/DrillConfig.java    |  31 +-
 .../org/apache/drill/common/util/GuavaPatcher.java |  86 +++
 distribution/pom.xml                               |  21 +
 distribution/src/assemble/component.xml            |   3 +
 distribution/src/deb/control/control               |   2 +-
 .../metastore/SimpleFileTableMetadataProvider.java |   2 +-
 .../org/apache/drill/exec/ops/QueryContext.java    |   5 +
 .../planner/cost/DrillRelMdDistinctRowCount.java   |   2 +-
 .../apache/drill/exec/server/DrillbitContext.java  |  10 +-
 .../store/parquet/BaseParquetMetadataProvider.java |  16 +-
 .../store/parquet/ParquetTableMetadataUtils.java   |   6 +-
 .../org/apache/drill/exec/work/WorkManager.java    |   2 +-
 metastore/iceberg-metastore/README.md              | 178 ++++++
 metastore/iceberg-metastore/pom.xml                | 188 ++++++
 .../drill/metastore/iceberg/IcebergMetastore.java  | 273 +++++++++
 .../drill/metastore/iceberg/MetastoreContext.java  |  53 ++
 .../iceberg/components/tables/IcebergTables.java   |  96 +++
 .../iceberg/components/tables/TableKey.java        | 106 ++++
 .../tables/TablesOperationTransformer.java         |  74 +++
 .../tables/TablesOutputDataTransformer.java        |  57 ++
 .../components/tables/TablesTransformer.java       |  58 ++
 .../iceberg/config/IcebergConfigConstants.java     |  93 +++
 .../exceptions/IcebergMetastoreException.java}     |  22 +-
 .../drill/metastore/iceberg/operate/Delete.java}   |  31 +-
 .../iceberg/operate/IcebergMetadata.java}          |  42 +-
 .../metastore/iceberg/operate/IcebergModify.java   |  83 +++
 .../iceberg/operate/IcebergOperation.java}         |  15 +-
 .../metastore/iceberg/operate/IcebergRead.java     |  77 +++
 .../drill/metastore/iceberg/operate/Overwrite.java |  53 ++
 .../iceberg/schema/IcebergTableSchema.java         | 168 ++++++
 .../iceberg/transform/FilterExpressionVisitor.java | 118 ++++
 .../iceberg/transform/FilterTransformer.java       |  56 ++
 .../iceberg/transform/InputDataTransformer.java    | 129 ++++
 .../iceberg/transform/OperationTransformer.java    |  85 +++
 .../iceberg/transform/OutputDataTransformer.java   |  85 +++
 .../metastore/iceberg/transform/Transformer.java}  |  47 +-
 .../metastore/iceberg/transform/WriteData.java}    |  31 +-
 .../drill/metastore/iceberg/write/File.java}       |  42 +-
 .../drill/metastore/iceberg/write/FileWriter.java  |  60 ++
 .../metastore/iceberg/write/ParquetFileWriter.java |  97 +++
 .../src/main/resources/drill-metastore-module.conf |  60 ++
 .../drill/metastore/iceberg/IcebergBaseTest.java   |  81 +++
 .../components/tables/TestBasicRequests.java       | 484 +++++++++++++++
 .../tables/TestIcebergTablesMetastore.java         | 486 +++++++++++++++
 .../iceberg/components/tables/TestTableKey.java    |  68 +++
 .../tables/TestTablesInputDataTransformer.java     | 209 +++++++
 .../tables/TestTablesOperationTransformer.java     | 116 ++++
 .../tables/TestTablesOutputDataTransformer.java    | 131 ++++
 .../iceberg/schema/TestIcebergTableSchema.java     | 300 ++++++++++
 .../iceberg/transform/TestFilterTransformer.java   | 222 +++++++
 .../iceberg/write/TestParquetFileWriter.java       | 283 +++++++++
 metastore/metastore-api/README.md                  | 334 +++++++++++
 metastore/metastore-api/pom.xml                    |  38 +-
 .../LocationProvider.java => Metastore.java}       |  26 +-
 ...Provider.java => MetastoreFieldDefinition.java} |  24 +-
 .../apache/drill/metastore/MetastoreRegistry.java  | 107 ++++
 .../components/tables/BasicTablesRequests.java     | 657 +++++++++++++++++++++
 .../components/tables/BasicTablesTransformer.java  | 147 +++++
 .../components/tables/MetastoreTableInfo.java      | 118 ++++
 .../components/tables/TableMetadataUnit.java       | 561 ++++++++++++++++++
 .../tables/Tables.java}                            |  18 +-
 .../views/ViewMetadataUnit.java}                   |  18 +-
 .../views/Views.java}                              |  16 +-
 .../metastore/config/MetastoreConfigConstants.java |  57 ++
 .../MetastoreConfigFileInfo.java}                  |  33 +-
 .../MetastoreException.java}                       |  22 +-
 .../expressions/DoubleExpressionPredicate.java     |  90 +++
 .../metastore/expressions/FilterExpression.java    | 146 +++++
 .../drill/metastore/expressions/IsPredicate.java   |  83 +++
 .../drill/metastore/expressions/ListPredicate.java |  97 +++
 .../metastore/expressions/SimplePredicate.java     | 168 ++++++
 .../expressions/SingleExpressionPredicate.java     |  67 +++
 .../drill/metastore/metadata/BaseMetadata.java     |  82 ++-
 .../metastore/metadata/BaseTableMetadata.java      |  44 +-
 .../drill/metastore/metadata/FileMetadata.java     |  34 +-
 .../drill/metastore/metadata/LocationProvider.java |  15 +-
 .../apache/drill/metastore/metadata/Metadata.java  |   9 +
 .../drill/metastore/metadata/MetadataInfo.java     |  94 ++-
 .../drill/metastore/metadata/MetadataType.java     |  22 +-
 .../metadata/NonInterestingColumnsMetadata.java    |   6 +
 .../metastore/metadata/PartitionMetadata.java      |  41 +-
 .../drill/metastore/metadata/RowGroupMetadata.java |  24 +-
 .../drill/metastore/metadata/SegmentMetadata.java  |  65 +-
 .../apache/drill/metastore/metadata/TableInfo.java |  78 ++-
 .../apache/drill/metastore/operate/Metadata.java   |  64 ++
 .../org/apache/drill/metastore/operate/Modify.java |  74 +++
 .../Operations.java}                               |  27 +-
 .../org/apache/drill/metastore/operate/Read.java   |  64 ++
 .../metastore/statistics/ColumnStatistics.java     |  20 +-
 .../metastore/statistics/StatisticsHolder.java     |  19 +-
 .../main/resources/drill-metastore-default.conf    |  19 +
 .../components/tables/TestBasicTablesRequests.java | 133 +++++
 .../tables/TestBasicTablesTransformer.java         | 333 +++++++++++
 .../components/tables/TestMetastoreTableInfo.java  |  95 +++
 .../tables/TestTableMetadataUnitConversion.java    | 437 ++++++++++++++
 .../metastore/metadata/MetadataSerDeTest.java      |  14 +-
 metastore/pom.xml                                  |  31 +-
 pom.xml                                            |   2 +-
 98 files changed, 9383 insertions(+), 333 deletions(-)

diff --git a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
index 6fa09e5..a655943 100644
--- a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
+++ b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
@@ -19,6 +19,7 @@ package org.apache.drill.common.config;
 
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
+import com.typesafe.config.ConfigMergeable;
 import com.typesafe.config.ConfigRenderOptions;
 import io.netty.util.internal.PlatformDependent;
 import org.apache.drill.common.exceptions.DrillConfigurationException;
@@ -153,7 +154,7 @@ public class DrillConfig extends NestedConfig {
    */
   @VisibleForTesting
   public static DrillConfig create(Properties testConfigurations) {
-    return create(null, testConfigurations, true, new DrillExecConfigFileInfo());
+    return create(null, testConfigurations, true, new DrillExecConfigFileInfo(), null);
   }
 
   /**
@@ -161,7 +162,7 @@ public class DrillConfig extends NestedConfig {
    *          see {@link #create(String)}'s {@code overrideFileResourcePathname}
    */
   public static DrillConfig create(String overrideFileResourcePathname, boolean enableServerConfigs) {
-    return create(overrideFileResourcePathname, null, enableServerConfigs, new DrillExecConfigFileInfo());
+    return create(overrideFileResourcePathname, null, enableServerConfigs, new DrillExecConfigFileInfo(), null);
   }
 
   /**
@@ -181,7 +182,7 @@ public class DrillConfig extends NestedConfig {
    *  @return A merged Config object.
    */
   public static DrillConfig createForRM() {
-    return create(null, null, true, new DrillRMConfigFileInfo());
+    return create(null, null, true, new DrillRMConfigFileInfo(), null);
   }
 
   /**
@@ -203,12 +204,15 @@ public class DrillConfig extends NestedConfig {
    *          whether to enable server-specific configuration options
    * @param configInfo
    *          see {@link ConfigFileInfo}
+   * @param fallbackConfig
+   *          existing config which will be used as fallback
    * @return {@link DrillConfig} object with all configs from passed in resource files
    */
-  private static DrillConfig create(String overrideFileResourcePathname,
-                                    final Properties overriderProps,
-                                    final boolean enableServerConfigs,
-                                    ConfigFileInfo configInfo) {
+  public static DrillConfig create(String overrideFileResourcePathname,
+                                   Properties overriderProps,
+                                   boolean enableServerConfigs,
+                                   ConfigFileInfo configInfo,
+                                   ConfigMergeable fallbackConfig) {
     final StringBuilder logString = new StringBuilder();
     final Stopwatch watch = Stopwatch.createStarted();
     overrideFileResourcePathname = overrideFileResourcePathname == null ?
@@ -239,11 +243,16 @@ public class DrillConfig extends NestedConfig {
     }
     logString.append("\n");
 
+    // Add fallback config for default and module configuration
+    if (fallbackConfig != null) {
+      fallback = fallback.withFallback(fallbackConfig);
+    }
+
     final ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
 
     // 3. Load distribution specific configuration file.
     final URL distribConfigFileUrl = classLoader.getResource(configInfo.getDistributionFileName());
-    if (null != distribConfigFileUrl ) {
+    if (distribConfigFileUrl != null) {
       logString.append("Distribution Specific Configuration File: ").append(distribConfigFileUrl).append("\n");
     }
     fallback =
@@ -254,13 +263,13 @@ public class DrillConfig extends NestedConfig {
 
     // (Per ConfigFactory.load(...)'s mention of using Thread.getContextClassLoader():)
     final URL overrideFileUrl = classLoader.getResource(overrideFileResourcePathname);
-    if (null != overrideFileUrl ) {
+    if (overrideFileUrl != null) {
       logString.append("Override File: ").append(overrideFileUrl).append("\n");
     }
     Config effectiveConfig =
         ConfigFactory.load(overrideFileResourcePathname).withFallback(fallback);
 
-    // 4. Apply any overriding properties.
+    // 5. Apply any overriding properties.
     if (overriderProps != null) {
       logString.append("Overridden Properties:\n");
       for(Entry<Object, Object> entry : overriderProps.entrySet()){
@@ -273,7 +282,7 @@ public class DrillConfig extends NestedConfig {
           ConfigFactory.parseProperties(overriderProps).withFallback(effectiveConfig);
     }
 
-    // 5. Create DrillConfig object from Config object.
+    // 6. Create DrillConfig object from Config object.
     logger.info("Configuration and plugin file(s) identified in {}ms.\n{}",
         watch.elapsed(TimeUnit.MILLISECONDS),
         logString);
diff --git a/common/src/main/java/org/apache/drill/common/util/GuavaPatcher.java b/common/src/main/java/org/apache/drill/common/util/GuavaPatcher.java
index 22a4cb5..921d56d 100644
--- a/common/src/main/java/org/apache/drill/common/util/GuavaPatcher.java
+++ b/common/src/main/java/org/apache/drill/common/util/GuavaPatcher.java
@@ -18,12 +18,19 @@
 package org.apache.drill.common.util;
 
 import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 
+import javassist.CannotCompileException;
 import javassist.ClassPool;
 import javassist.CtClass;
 import javassist.CtConstructor;
 import javassist.CtMethod;
 import javassist.CtNewMethod;
+import javassist.NotFoundException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,6 +44,7 @@ public class GuavaPatcher {
       try {
         patchStopwatch();
         patchCloseables();
+        patchPreconditions();
         patched = true;
       } catch (Throwable e) {
         logger.warn("Unable to patch Guava classes.", e);
@@ -88,4 +96,82 @@ public class GuavaPatcher {
     logger.info("Google's Closeables patched for old HBase Guava version.");
   }
 
+  /**
+   * Patches Guava Preconditions with missing methods, added for the Apache Iceberg.
+   */
+  private static void patchPreconditions() throws NotFoundException, CannotCompileException {
+    ClassPool cp = ClassPool.getDefault();
+    CtClass cc = cp.get("com.google.common.base.Preconditions");
+
+    // Javassist does not support varargs, generate methods with varying number of arguments
+    int startIndex = 1;
+    int endIndex = 5;
+
+    List<String> methodsWithVarargsTemplates = Arrays.asList(
+      "public static void checkArgument(boolean expression, String errorMessageTemplate, %s) {\n"
+        + "    if (!expression) {\n"
+        + "      throw new IllegalArgumentException(format(errorMessageTemplate, new Object[] { %s }));\n"
+        + "    }\n"
+        + "  }",
+
+      "public static Object checkNotNull(Object reference, String errorMessageTemplate, %s) {\n"
+        + "    if (reference == null) {\n"
+        + "      throw new NullPointerException(format(errorMessageTemplate, new Object[] { %s }));\n"
+        + "    } else {\n"
+        + "      return reference;\n"
+        + "    }\n"
+        + "  }",
+
+      "public static void checkState(boolean expression, String errorMessageTemplate, %s) {\n"
+        + "    if (!expression) {\n"
+        + "      throw new IllegalStateException(format(errorMessageTemplate, new Object[] { %s }));\n"
+        + "    }\n"
+        + "  }"
+    );
+
+    List<String> methodsWithPrimitives = Arrays.asList(
+      "public static void checkArgument(boolean expression, String errorMessageTemplate, int arg1) {\n"
+        + "    if (!expression) {\n"
+        + "      throw new IllegalArgumentException(format(errorMessageTemplate, new Object[] { new Integer(arg1) }));\n"
+        + "    }\n"
+        + "  }",
+      "public static Object checkNotNull(Object reference, String errorMessageTemplate, int arg1) {\n"
+        + "    if (reference == null) {\n"
+        + "      throw new NullPointerException(format(errorMessageTemplate, new Object[] { new Integer(arg1) }));\n"
+        + "    } else {\n"
+        + "      return reference;\n"
+        + "    }\n"
+        + "  }"
+    );
+
+    List<String> newMethods = IntStream.rangeClosed(startIndex, endIndex)
+      .mapToObj(
+        i -> {
+          List<String> args = IntStream.rangeClosed(startIndex, i)
+            .mapToObj(j -> "arg" + j)
+            .collect(Collectors.toList());
+
+          String methodInput = args.stream()
+            .map(arg -> "Object " + arg)
+            .collect(Collectors.joining(", "));
+
+          String arrayInput = String.join(", ", args);
+
+          return methodsWithVarargsTemplates.stream()
+            .map(method -> String.format(method, methodInput, arrayInput))
+            .collect(Collectors.toList());
+        })
+      .flatMap(Collection::stream)
+      .collect(Collectors.toList());
+
+    newMethods.addAll(methodsWithPrimitives);
+
+    for (String method : newMethods) {
+      CtMethod newMethod = CtNewMethod.make(method, cc);
+      cc.addMethod(newMethod);
+    }
+
+    cc.toClass();
+    logger.info("Google's Preconditions were patched to hold new methods.");
+  }
 }
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 0e3d792..7eca321 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -78,6 +78,16 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.drill.metastore</groupId>
+      <artifactId>drill-metastore-api</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.drill.metastore</groupId>
+      <artifactId>drill-iceberg-metastore</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <exclusions>
@@ -294,6 +304,17 @@
           <artifactId>drill-format-ltsv</artifactId>
           <version>${project.version}</version>
         </dependency>
+        <!--
+         drill-format-mapr is intentionally excluded from distribution for the default profile
+         to avoid adding mapr-specific jars into jars/3rdparty folder and inflation of the target jar.
+         For more details please see DRILL-7213 and DRILL-6929.
+
+         <dependency>
+           <groupId>org.apache.drill.contrib</groupId>
+           <artifactId>drill-format-mapr</artifactId>
+           <version>${project.version}</version>
+         </dependency>
+         -->
       </dependencies>
     </profile>
 
diff --git a/distribution/src/assemble/component.xml b/distribution/src/assemble/component.xml
index 30afc47..4b4fc56 100644
--- a/distribution/src/assemble/component.xml
+++ b/distribution/src/assemble/component.xml
@@ -33,6 +33,8 @@
         <include>org.apache.drill.memory:drill-memory-base:jar</include>
         <include>org.apache.drill.exec:drill-rpc:jar</include>
         <include>org.apache.drill.exec:drill-java-exec:jar</include>
+        <include>org.apache.drill.metastore:drill-metastore-api:jar</include>
+        <include>org.apache.drill.metastore:drill-iceberg-metastore:jar</include>
         <include>org.apache.drill.contrib.storage-hive:drill-storage-hive-core:jar</include>
         <include>org.apache.drill.contrib.storage-hive:drill-hive-exec-shaded:jar</include>
         <include>org.apache.drill.contrib.data:tpch-sample-data:jar</include>
@@ -119,6 +121,7 @@
         <exclude>org.apache.drill.contrib.data</exclude>
         <exclude>org.apache.drill.contrib.storage-hive</exclude>
         <exclude>org.apache.drill.memory</exclude>
+        <exclude>org.apache.drill.metastore</exclude>
         <!-- Below Hive jars are already included in drill-hive-exec-shaded.jar -->
         <exclude>junit:junit:jar</exclude>
         <!-- exclude or sqlline has problems -->
diff --git a/distribution/src/deb/control/control b/distribution/src/deb/control/control
index f15a843..ae0d657 100644
--- a/distribution/src/deb/control/control
+++ b/distribution/src/deb/control/control
@@ -1,6 +1,6 @@
 Homepage: http://drill.apache.org/
 Package: Apache Drill
-Version: 1.16.0-SNAPSHOT
+Version: 1.17.0-SNAPSHOT
 Section: misc
 Priority: optional
 Architecture: all
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java
index b73404f..64ef4e8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java
@@ -167,7 +167,7 @@ public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
           logger.debug("Unable to read schema from schema provider [{}]: {}", (tableName != null ? tableName : location), e.getMessage());
           logger.trace("Error when reading the schema", e);
         }
-        MetadataInfo metadataInfo = new MetadataInfo(MetadataType.TABLE, MetadataInfo.GENERAL_INFO_KEY, null);
+        MetadataInfo metadataInfo = MetadataInfo.builder().type(MetadataType.TABLE).build();
         TableMetadata tableMetadata = BaseTableMetadata.builder()
             .tableInfo(TableInfo.UNKNOWN_TABLE_INFO)
             .metadataInfo(metadataInfo)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index 97db9b2..fb1b4a9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -52,6 +52,7 @@ import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.exec.util.Utilities;
 
+import org.apache.drill.metastore.Metastore;
 import org.apache.drill.shaded.guava.com.google.common.base.Function;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
@@ -389,4 +390,8 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
   public boolean isSkipProfileWrite() {
     return skipProfileWrite;
   }
+
+  public Metastore getMetastore() {
+    return drillbitContext.getMetastore();
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
index 6ddfd48..3053297 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
@@ -186,7 +186,7 @@ public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount{
     if (!allColsHaveNDV) {
       if (logger.isDebugEnabled()) {
         logger.debug(String.format("NDV not available for %s(%s). Using default rowcount for group-by %s",
-            (tableMetadata != null ? tableMetadata.getTableInfo().getName() : ""), colName, groupKey.toString()));
+            (tableMetadata != null ? tableMetadata.getTableInfo().name() : ""), colName, groupKey.toString()));
       }
       // Could not get any NDV estimate from stats - probably stats not present for GBY cols. So Guess!
       return scan.estimateRowCount(mq) * 0.1;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index b2802de..397e7bc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -44,6 +44,8 @@ import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.sys.PersistentStoreProvider;
 import org.apache.drill.exec.work.foreman.rm.ResourceManager;
 import org.apache.drill.exec.work.foreman.rm.ResourceManagerBuilder;
+import org.apache.drill.metastore.Metastore;
+import org.apache.drill.metastore.MetastoreRegistry;
 
 import java.util.Collection;
 import java.util.Map.Entry;
@@ -74,6 +76,7 @@ public class DrillbitContext implements AutoCloseable {
   private final DrillOperatorTable table;
   private final QueryProfileStoreContext profileStoreContext;
   private ResourceManager resourceManager;
+  private final MetastoreRegistry metastoreRegistry;
 
   public DrillbitContext(
       DrillbitEndpoint endpoint,
@@ -120,7 +123,8 @@ public class DrillbitContext implements AutoCloseable {
     table = new DrillOperatorTable(functionRegistry, systemOptions);
 
     //This profile store context is built from the profileStoreProvider
-    profileStoreContext = new QueryProfileStoreContext(context.getConfig(), profileStoreProvider, coord);
+    profileStoreContext = new QueryProfileStoreContext(config, profileStoreProvider, coord);
+    this.metastoreRegistry = new MetastoreRegistry(config);
   }
 
   public QueryProfileStoreContext getProfileStoreContext() {
@@ -301,4 +305,8 @@ public class DrillbitContext implements AutoCloseable {
   public ResourceManager getResourceManager() {
     return resourceManager;
   }
+
+  public Metastore getMetastore() {
+    return metastoreRegistry.get();
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java
index 92d582e..3ccf264 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java
@@ -255,7 +255,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
               new ColumnStatistics(DrillStatsTable.getEstimatedColumnStats(statsTable, column)));
         }
       }
-      MetadataInfo metadataInfo = new MetadataInfo(MetadataType.TABLE, MetadataInfo.GENERAL_INFO_KEY, null);
+      MetadataInfo metadataInfo = MetadataInfo.builder().type(MetadataType.TABLE).build();
       tableMetadata = BaseTableMetadata.builder()
           .tableInfo(TableInfo.UNKNOWN_TABLE_INFO)
           .metadataInfo(metadataInfo)
@@ -337,7 +337,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
             columnsStatistics.put(partitionColumn,
                 new ColumnStatistics<>(statistics,
                     getParquetGroupScanStatistics().getTypeForColumn(partitionColumn).getMinorType()));
-            MetadataInfo metadataInfo = new MetadataInfo(MetadataType.PARTITION, MetadataInfo.GENERAL_INFO_KEY, null);
+            MetadataInfo metadataInfo = MetadataInfo.builder().type(MetadataType.PARTITION).build();
             TableMetadata tableMetadata = getTableMetadata();
             PartitionMetadata partitionMetadata = PartitionMetadata.builder()
                 .tableInfo(tableMetadata.getTableInfo())
@@ -429,16 +429,16 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
   }
 
   private static <T extends BaseMetadata & LocationProvider> SegmentMetadata combineToSegmentMetadata(Collection<T> metadataList, SchemaPath column) {
-    List<Path> metadataLocations = metadataList.stream()
+    Set<Path> metadataLocations = metadataList.stream()
         .map(metadata -> metadata.getPath()) // used lambda instead of method reference due to JDK-8141508
-        .collect(Collectors.toList());
+        .collect(Collectors.toSet());
     return combineToSegmentMetadata(metadataList, column, metadataLocations);
   }
 
   private static SegmentMetadata combineSegmentMetadata(Collection<SegmentMetadata> metadataList, SchemaPath column) {
-    List<Path> metadataLocations = metadataList.stream()
+    Set<Path> metadataLocations = metadataList.stream()
         .flatMap(metadata -> metadata.getLocations().stream())
-        .collect(Collectors.toList());
+        .collect(Collectors.toSet());
 
     return combineToSegmentMetadata(metadataList, column, metadataLocations);
   }
@@ -453,7 +453,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
    * @return {@link SegmentMetadata} from combined metadata
    */
   private static <T extends BaseMetadata & LocationProvider> SegmentMetadata combineToSegmentMetadata(Collection<T> metadataList,
-      SchemaPath column, List<Path> metadataLocations) {
+      SchemaPath column, Set<Path> metadataLocations) {
     List<StatisticsHolder> segmentStatistics =
         Collections.singletonList(
             new StatisticsHolder<>(
@@ -461,7 +461,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
                 TableStatisticsKind.ROW_COUNT));
     // this code is used only to collect segment metadata to be used only during filtering,
     // so metadata identifier is not required here and in other places in this class
-    MetadataInfo metadataInfo = new MetadataInfo(MetadataType.SEGMENT, MetadataInfo.GENERAL_INFO_KEY, null);
+    MetadataInfo metadataInfo = MetadataInfo.builder().type(MetadataType.SEGMENT).build();
     T firstMetadata = metadataList.iterator().next();
 
     return SegmentMetadata.builder()
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataUtils.java
index 9df3f73..68ad0db 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataUtils.java
@@ -151,7 +151,7 @@ public class ParquetTableMetadataUtils {
     TupleSchema schema = new TupleSchema();
     columns.forEach((schemaPath, majorType) -> SchemaPathUtils.addColumnMetadata(schema, schemaPath, majorType));
 
-    MetadataInfo metadataInfo = new MetadataInfo(MetadataType.ROW_GROUP, MetadataInfo.GENERAL_INFO_KEY, null);
+    MetadataInfo metadataInfo = MetadataInfo.builder().type(MetadataType.ROW_GROUP).build();
 
     return RowGroupMetadata.builder()
         .tableInfo(TableInfo.UNKNOWN_TABLE_INFO)
@@ -183,7 +183,7 @@ public class ParquetTableMetadataUtils {
 
     Set<SchemaPath> columns = rowGroupMetadata.getColumnsStatistics().keySet();
 
-    MetadataInfo metadataInfo = new MetadataInfo(MetadataType.FILE, MetadataInfo.GENERAL_INFO_KEY, null);
+    MetadataInfo metadataInfo = MetadataInfo.builder().type(MetadataType.FILE).build();
 
     return FileMetadata.builder()
         .tableInfo(rowGroupMetadata.getTableInfo())
@@ -213,7 +213,7 @@ public class ParquetTableMetadataUtils {
 
     FileMetadata fileMetadata = files.iterator().next();
 
-    MetadataInfo metadataInfo = new MetadataInfo(MetadataType.PARTITION, MetadataInfo.GENERAL_INFO_KEY, null);
+    MetadataInfo metadataInfo = MetadataInfo.builder().type(MetadataType.PARTITION).build();
 
     return PartitionMetadata.builder()
         .tableInfo(fileMetadata.getTableInfo())
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index a739c5c..ac1b4cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -66,7 +66,7 @@ import java.util.concurrent.locks.ReentrantLock;
 public class WorkManager implements AutoCloseable {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkManager.class);
 
-  public static final int EXIT_TIMEOUT_MS = 5000;
+  private static final int EXIT_TIMEOUT_MS = 5000;
 
   /*
    * We use a {@see java.util.concurrent.ConcurrentHashMap} because it promises never to throw a
diff --git a/metastore/iceberg-metastore/README.md b/metastore/iceberg-metastore/README.md
new file mode 100644
index 0000000..7d805c1
--- /dev/null
+++ b/metastore/iceberg-metastore/README.md
@@ -0,0 +1,178 @@
+# Iceberg Metastore
+
+Iceberg Metastore implementation is based on [Iceberg tables](http://iceberg.incubator.apache.org). 
+One Iceberg table corresponds to one metastore component (`tables`, `views`, etc,)
+Iceberg tables schemas (field names and types) are based on component metadata units classes fields 
+annotated with `MetastoreFieldDefinition` annotation.
+
+## Configuration
+
+Iceberg Metastore configuration is indicated in `drill-metastore-module.conf` and 
+can be overwritten in `drill-metastore-distrib.conf` or  `drill-metastore-override.conf` files.
+
+`drill.metastore.config.properties` - allows to specify properties connected with file system.
+
+`drill.metastore.iceberg.location.base_path` and `drill.metastore.iceberg.location.relative_path` -
+indicate Iceberg Metastore base location.
+
+`drill.metastore.iceberg.components` - provides configuration for specific Metastore components:
+`drill.metastore.iceberg.components.tables`, `drill.metastore.iceberg.components.views`.
+
+`drill.metastore.iceberg.common.properties` - allows to specify common for all Iceberg tables,
+refer to `org.apache.iceberg.TableProperties` class for the list of available table properties. 
+Each component can specify specific table properties which override the same common table properties:
+`drill.metastore.iceberg.components.tables.properties`, `drill.metastore.iceberg.components.views.properties`.
+
+Each component must specify Iceberg table location within Iceberg Metastore base location:
+`drill.metastore.iceberg.components.tables.location`, `drill.metastore.iceberg.components.views.location`.
+
+## Iceberg Tables Location
+
+Iceberg tables will reside on the file system in the location based on
+Iceberg Metastore base location and component specific location.
+If Iceberg Metastore base location is `/drill/metastore/iceberg`
+and tables component location is `tables`. Iceberg table for tables component
+will be located in `/drill/metastore/iceberg/tables` folder.
+
+Inside given location, Iceberg table will create `metadata` folder where 
+it will store its snapshots, manifests and table metadata.
+
+Note: Iceberg table supports concurrent writes and transactions 
+but they are only effective on file systems that support atomic rename.
+
+## Component Data Location
+
+Metastore metadata will be stored inside Iceberg table location provided
+in the configuration file. Drill table metadata location will be constructed
+based on specific component storage keys. For example, for `tables` component,
+storage keys are storage plugin, workspace and table name: unique table identifier in Drill.
+
+Assume Iceberg table location is `/drill/metastore/iceberg/tables`, metadata for the table
+`dfs.tmp.nation` will be stored in the `/drill/metastore/iceberg/tables/dfs/tmp/nation` folder.
+
+### Metadata Storage Format
+
+By default, Metadata will be stored in Parquet files. 
+Each parquet file will hold information for one partition.
+Partition keys will depend on Metastore component characteristics.
+For example, for `tables` component, partitions keys are
+storage plugin, workspace, table name and metadata key.
+
+Parquet files name will be based on `UUID` to ensure uniqueness.
+
+Iceberg also supports data storage in Avro and ORC files, writing metadata
+in these formats can be added later.
+
+## Metastore Operations flow
+
+Metastore main goal is to provide ability to read and modify metadata.
+
+### Read
+
+Metastore data is read using `IcebergGenerics#read`. Iceberg will automatically determine
+format in which data is stored (three formats are supported Parquet, Avro, ORC).
+Based on given filter and select columns list, data will be returned in 
+`org.apache.iceberg.data.Record` format which will be transformed 
+into the list of Metastore component units and returned to the caller.
+For example, for `tables` component, metadata unit is `TableMetadataUnit`.
+
+Iceberg supports partition pruning, to avoid scanning all data and improve performance,
+partition keys can be included into filter expression.
+
+### Add
+
+To add metadata to Iceberg table, caller provides list of component units which
+will be written into Parquet files (current default format) and grouped by partition keys.
+Each group will be written into separate Parquet file 
+and stored in the location inside of Iceberg table based on component unit location keys.
+Note: partition keys must not be null.
+
+For example, for `tables` component data is grouped by storage plugin, workspace, table name
+and metadata key, location is determined based on storage plugin, workspace and table name.
+
+For each Parquet file, `IcebergOperation#Overwrite` will be created.
+List of overwrite operations will be executed in one transaction.
+Main goal of overwrite operation to add or replace pointer of the existing file
+in Iceberg table's partition information.
+
+If transaction was successful, Iceberg table generates new snapshot and updates
+its own metadata.
+
+Assume, caller wants to add metadata for Drill table `dfs.tmp.nation` into `tables` component.
+Parquet files with metadata for this table will be stored in 
+`[METASTORE_ROOT_DIRECTORY]/[COMPONENT_LOCATION]/dfs/tmp/nation` folder.
+
+If `dfs.tmp.nation` is un-partitioned, it's metadata will be stored in two
+parquet files: one file with general table information, 
+another file with default segment information. 
+If `dfs.tmp.nation` is partitioned, it will have also one file with general
+information and `N` files with top-level segments information. 
+
+File with general table information will always have one row.
+Number of rows for default or top-level segment file will depend on segments 
+metadata. Each row corresponds to one metadata unit: segment, file,
+row group or partition.
+
+If `dfs.tmp.nation` table has one segment to which belongs one file 
+with two row groups and one partition and no inner segments, 
+this segment's file will have five rows: 
+- one row with top-level segment metadata;
+- one row with file metadata;
+- two rows with row group metadata;
+- one row with partition metadata.
+
+Such storage model allows easily to overwrite or delete existing top-level segments
+without necessity to re-write all table metadata when it was only partially changed.
+
+When `dfs.tmp.nation` metadata is created, table metadata location will store two files.
+File names are generated based on `UUID` to ensure uniqueness.
+
+```
+...\dfs\tmp\nation\282205e0-88f2-4df2-aa3c-90d26ae0ad28.parquet (general info file)
+...\dfs\tmp\nation\e6b20998-d640-4e53-9ece-5a063e498e1a.parquet (default segment file)
+
+```
+
+Once overwrite operation is committed, Iceberg table will have two partitions:
+`\dfs\tmp\nation\GENERAL_INFO` and `\dfs\tmp\nation\DEFAULT_SEGMENT`.
+Each of these partitions will point to dedicated Parquet file.
+If table is partitioned, instead of `DEFAULT_SEGMENT`, top-level
+segment metadata key will be indicated. For example, `\dfs\tmp\nation\dir0`.
+
+### Overwrite
+
+Process of overwriting existing partitions is almost the same as adding new partition.
+Assume, `dfs.tmp.nation` default segment metadata has changed: new file was added.
+Caller passes updated default segment metadata to the Metastore.
+First, new Parquet file for updated default segment metadata is created.
+Once overwrite operation is committed, `\dfs\tmp\nation\DEFAULT_SEGMENT` partition
+will point to Parquet file with updated metadata. 
+Table metadata location will store three files:
+
+```
+...\dfs\tmp\nation\282205e0-88f2-4df2-aa3c-90d26ae0ad28.parquet (general info file)
+...\dfs\tmp\nation\e6b20998-d640-4e53-9ece-5a063e498e1a.parquet (old segment file)
+...\dfs\tmp\nation\4930061e-1c1d-4c8e-a19e-b7b9a5f5f246.parquet (new segment file)
+
+```
+
+### Delete
+
+To delete data from Iceberg table, caller provides filter by which data will be deleted.
+Filter expression must be based on component partition keys.
+
+Delete operation removes partitions from Iceberg table, it does not remove data files to which
+these partitions were pointing.
+
+If delete operation was successful, Iceberg table generates new snapshot and updates
+its own metadata.
+
+### Purge
+
+Allows to delete all data from Iceberg table. During this operation Iceberg table
+is not deleted, history of all operations and data files are preserved.
+
+## Data cleanup
+
+Iceberg table provides ability to remove outdated data files and snapshots 
+when they are no longer needed. Such support in Iceberg Metastore will be added later.
diff --git a/metastore/iceberg-metastore/pom.xml b/metastore/iceberg-metastore/pom.xml
new file mode 100644
index 0000000..1caaa7c
--- /dev/null
+++ b/metastore/iceberg-metastore/pom.xml
@@ -0,0 +1,188 @@
+<?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.
+
+-->
+<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>metastore-parent</artifactId>
+    <groupId>org.apache.drill.metastore</groupId>
+    <version>1.17.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>drill-iceberg-metastore</artifactId>
+  <name>metastore/Drill Iceberg Metastore</name>
+
+  <properties>
+    <iceberg.version>cfd2737</iceberg.version>
+    <caffeine.version>2.7.0</caffeine.version>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.drill.metastore</groupId>
+      <artifactId>drill-metastore-api</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.github.apache.incubator-iceberg</groupId>
+      <artifactId>iceberg-parquet</artifactId>
+      <version>${iceberg.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.github.apache.incubator-iceberg</groupId>
+      <artifactId>iceberg-data</artifactId>
+      <version>${iceberg.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.github.apache.incubator-iceberg</groupId>
+      <artifactId>iceberg-core</artifactId>
+      <version>${iceberg.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.github.apache.incubator-iceberg</groupId>
+      <artifactId>iceberg-common</artifactId>
+      <version>${iceberg.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.github.apache.incubator-iceberg</groupId>
+      <artifactId>iceberg-api</artifactId>
+      <version>${iceberg.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-avro</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.github.ben-manes.caffeine</groupId>
+      <artifactId>caffeine</artifactId>
+      <version>${caffeine.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>log4j</groupId>
+          <artifactId>log4j</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-log4j12</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.protobuf</groupId>
+          <artifactId>protobuf-java</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.inject.extensions</groupId>
+          <artifactId>guice-servlet</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>com.sun.codemodel</groupId>
+      <artifactId>codemodel</artifactId>
+      <version>${codemodel.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>net.openhft</groupId>
+      <artifactId>compiler</artifactId>
+      <version>2.3.4</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>mapr</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-client</artifactId>
+          <scope>test</scope>
+          <exclusions>
+            <exclusion>
+              <groupId>commons-codec</groupId>
+              <artifactId>commons-codec</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs</artifactId>
+          <scope>test</scope>
+          <classifier>tests</classifier>
+          <exclusions>
+            <exclusion>
+              <groupId>commons-codec</groupId>
+              <artifactId>commons-codec</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+        <dependency>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>jetty-servlet</artifactId>
+          <scope>test</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+
+</project>
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/IcebergMetastore.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/IcebergMetastore.java
new file mode 100644
index 0000000..6cd375c
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/IcebergMetastore.java
@@ -0,0 +1,273 @@
+/*
+ * 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.metastore.iceberg;
+
+import com.typesafe.config.Config;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.metastore.Metastore;
+import org.apache.drill.metastore.components.tables.Tables;
+import org.apache.drill.metastore.components.views.Views;
+import org.apache.drill.metastore.iceberg.components.tables.IcebergTables;
+import org.apache.drill.metastore.iceberg.config.IcebergConfigConstants;
+import org.apache.drill.metastore.iceberg.exceptions.IcebergMetastoreException;
+import org.apache.drill.metastore.iceberg.schema.IcebergTableSchema;
+import org.apache.drill.shaded.guava.com.google.common.collect.MapDifference;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.UpdateProperties;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+/**
+ * Iceberg Drill Metastore implementation that inits / loads Iceberg tables
+ * which correspond to Metastore components: tables, views, etc.
+ */
+public class IcebergMetastore implements Metastore {
+
+  private static final Logger logger = LoggerFactory.getLogger(IcebergMetastore.class);
+
+  private final DrillConfig config;
+  private final org.apache.iceberg.Tables tables;
+  private final String baseLocation;
+  private final Map<String, String> commonProperties;
+
+  /**
+   * Table properties for each Iceberg table should be updated only once,
+   * since during Iceberg Metastore instance existence, Drill config is not changed
+   * and there is no need to update table properties each Metastore component call.
+   *
+   * Concurrent map stores component interface class if corresponding Iceberg table
+   * was loaded, which is indication that table properties should be updated.
+   */
+  private final Map<Class<?>, Boolean> loadStatusMap = new ConcurrentHashMap<>();
+
+  public IcebergMetastore(DrillConfig config) {
+    this.config = config;
+    Configuration configuration = configuration(config);
+    this.tables = new HadoopTables(new Configuration(configuration));
+    this.baseLocation = baseLocation(config, new Configuration(configuration));
+    this.commonProperties = properties(config, IcebergConfigConstants.COMPONENTS_COMMON_PROPERTIES);
+  }
+
+  @Override
+  public Tables tables() {
+    Table table = loadTable(IcebergConfigConstants.COMPONENTS_TABLES_LOCATION,
+      IcebergConfigConstants.COMPONENTS_TABLES_PROPERTIES,
+      IcebergTables.SCHEMA, Tables.class);
+    return new IcebergTables(table);
+  }
+
+  @Override
+  public Views views() {
+    throw new UnsupportedOperationException("Views metadata support is not implemented");
+  }
+
+  /**
+   * Initializes {@link Configuration} based on config properties.
+   * if config properties are not indicated, returns default instance.
+   *
+   * @param config Drill config
+   * @return {@link Configuration} instance
+   */
+  private Configuration configuration(DrillConfig config) {
+    Configuration configuration = new Configuration();
+    if (config.hasPath(IcebergConfigConstants.CONFIG_PROPERTIES)) {
+      Config configProperties = config.getConfig(IcebergConfigConstants.CONFIG_PROPERTIES);
+      configProperties.entrySet().forEach(
+        entry -> configuration.set(entry.getKey(), String.valueOf(entry.getValue().unwrapped()))
+      );
+    }
+    return configuration;
+  }
+
+  /**
+   * Constructs Iceberg tables base location based on given base and relative paths.
+   * If {@link IcebergConfigConstants#BASE_PATH} is not set, user home directory is used.
+   * {@link IcebergConfigConstants#RELATIVE_PATH} must be set.
+   *
+   * @param config Drill config
+   * @param configuration Hadoop configuration
+   * @return Iceberg table base location
+   * @throws IcebergMetastoreException if unable to init file system
+   *         or Iceberg Metastore relative path is not indicated
+   */
+  private String baseLocation(DrillConfig config, Configuration configuration) {
+    FileSystem fs;
+    try {
+      fs = FileSystem.get(configuration);
+    } catch (IOException e) {
+      throw new IcebergMetastoreException(
+        String.format("Error during file system [%s] setup", configuration.get(FileSystem.FS_DEFAULT_NAME_KEY)));
+    }
+
+    String root = fs.getHomeDirectory().toUri().getPath();
+    if (config.hasPath(IcebergConfigConstants.BASE_PATH)) {
+      root = config.getString(IcebergConfigConstants.BASE_PATH);
+    }
+
+    String relativeLocation = config.getString(IcebergConfigConstants.RELATIVE_PATH);
+    if (relativeLocation == null) {
+      throw new IcebergMetastoreException(String.format(
+        "Iceberg Metastore relative path [%s] is not provided", IcebergConfigConstants.RELATIVE_PATH));
+    }
+
+    String location = new Path(root, relativeLocation).toUri().getPath();
+    logger.info("Iceberg Metastore is located in [{}] on file system [{}]", location, fs.getUri());
+    return location;
+  }
+
+  /**
+   * Collects properties name and values into map if they are present in the given config,
+   * returns empty map otherwise.
+   *
+   * @param config Drill config
+   * @param propertiesPath path to properties in the config
+   * @return map with properties names and their values
+   */
+  private Map<String, String> properties(DrillConfig config, String propertiesPath) {
+    return config.hasPath(propertiesPath)
+      ? config.getConfig(propertiesPath).entrySet().stream()
+      .collect(Collectors.toMap(
+        Map.Entry::getKey,
+        entry -> String.valueOf(entry.getValue().unwrapped()),
+        (o, n) -> n))
+      : Collections.emptyMap();
+  }
+
+  /**
+   * Creates / loads Iceberg table for specific component based on given location
+   * and config properties and table schema.
+   * Updates table properties for existing Iceberg table only once based on
+   * {@link #loadStatusMap} status.
+   *
+   * @param componentLocationConfig path to component location config
+   * @param componentPropertiesConfig path to component properties config
+   * @param schema Iceberg table schema
+   * @param loadClass Metastore component implementation interface
+   * @return Iceberg table instance
+   */
+  private Table loadTable(String componentLocationConfig,
+                          String componentPropertiesConfig,
+                          IcebergTableSchema schema,
+                          Class<?> loadClass) {
+    String location = tableLocation(componentLocationConfig);
+    Map<String, String> tableProperties = tableProperties(componentPropertiesConfig);
+
+    Table table;
+    try {
+      table = tables.load(location);
+    } catch (NoSuchTableException e) {
+      try {
+
+        // creating new Iceberg table, no need to update table properties
+        return tables.create(schema.tableSchema(), schema.partitionSpec(), tableProperties, location);
+      } catch (AlreadyExistsException ex) {
+        table = tables.load(location);
+      }
+    }
+
+    // updates table properties only during first component table call
+    if (loadStatusMap.putIfAbsent(loadClass, Boolean.TRUE) == null) {
+      updateTableProperties(table, tableProperties);
+    }
+    return table;
+  }
+
+  /**
+   * Constructs Metastore component Iceberg table location based on
+   * Iceberg Metastore base location and component specific location.
+   *
+   * @param componentLocationConfig path to component location config
+   * @return component Iceberg table location
+   * @throws IcebergMetastoreException if component location config is absent
+   */
+  private String tableLocation(String componentLocationConfig) {
+    String componentLocation;
+    if (config.hasPath(componentLocationConfig)) {
+      componentLocation = config.getString(componentLocationConfig);
+    } else {
+      throw new IcebergMetastoreException(
+        String.format("Component location config [%s] is not defined", componentLocationConfig));
+    }
+
+    return new Path(baseLocation, componentLocation).toUri().getPath();
+  }
+
+  /**
+   * Collects common Iceberg metastore properties and component specific properties
+   * into one map. Component properties take precedence.
+   *
+   * @param componentPropertiesConfig path to component properties config
+   * @return map with properties names and values
+   */
+  private Map<String, String> tableProperties(String componentPropertiesConfig) {
+    Map<String, String> properties = new HashMap<>(commonProperties);
+    properties.putAll(properties(config, componentPropertiesConfig));
+    return properties;
+  }
+
+  /**
+   * Checks config table properties against current table properties.
+   * Adds properties that are absent, updates existing and removes absent.
+   * If properties are the same, does nothing.
+   *
+   * @param table Iceberg table instance
+   * @param tableProperties table properties from the config
+   */
+  private void updateTableProperties(Table table, Map<String, String> tableProperties) {
+    Map<String, String> currentProperties = table.properties();
+    MapDifference<String, String> difference = Maps.difference(tableProperties, currentProperties);
+
+    if (difference.areEqual()) {
+      return;
+    }
+
+    UpdateProperties updateProperties = table.updateProperties();
+
+    // collect properties that are different
+    Map<String, String> propertiesToUpdate = difference.entriesDiffering().entrySet().stream()
+      .collect(Collectors.toMap(
+        Map.Entry::getKey,
+        entry -> entry.getValue().leftValue(),
+        (o, n) -> n));
+
+    // add new properties
+    propertiesToUpdate.putAll(difference.entriesOnlyOnLeft());
+
+    logger.debug("Updating Iceberg table [{}] properties: {}", table.location(), updateProperties);
+    propertiesToUpdate.forEach(updateProperties::set);
+
+    logger.debug("Removing Iceberg table [{}] properties: {}", table.location(), difference.entriesOnlyOnRight());
+    difference.entriesOnlyOnRight().keySet().forEach(updateProperties::remove);
+
+    updateProperties.commit();
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/MetastoreContext.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/MetastoreContext.java
new file mode 100644
index 0000000..c4806d8
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/MetastoreContext.java
@@ -0,0 +1,53 @@
+/*
+ * 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.metastore.iceberg;
+
+import org.apache.drill.metastore.iceberg.transform.Transformer;
+import org.apache.drill.metastore.iceberg.write.FileWriter;
+import org.apache.iceberg.Table;
+
+/**
+ * Provides Iceberg Metastore component tools to transform, read or write data from / in Iceberg table.
+ *
+ * @param <T> Metastore component unit type
+ */
+public interface MetastoreContext<T> {
+
+  /**
+   * Returns Iceberg table implementation used as storage for Metastore component data.
+   *
+   * @return Iceberg table instance
+   */
+  Table table();
+
+  /**
+   * Returns file writer which stores Metastore component data in the format
+   * supported by the {@link FileWriter} implementation.
+   *
+   * @return file writer instance
+   */
+  FileWriter fileWriter();
+
+  /**
+   * Returns transformer instance that provides tools to
+   * transform filters, operations and data.
+   *
+   * @return transformer instance
+   */
+  Transformer<T> transformer();
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/IcebergTables.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/IcebergTables.java
new file mode 100644
index 0000000..c28c0ec
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/IcebergTables.java
@@ -0,0 +1,96 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.metastore.components.tables.Tables;
+import org.apache.drill.metastore.operate.Metadata;
+import org.apache.drill.metastore.operate.Modify;
+import org.apache.drill.metastore.operate.Read;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.iceberg.MetastoreContext;
+import org.apache.drill.metastore.iceberg.operate.IcebergMetadata;
+import org.apache.drill.metastore.iceberg.schema.IcebergTableSchema;
+import org.apache.drill.metastore.iceberg.operate.IcebergModify;
+import org.apache.drill.metastore.iceberg.operate.IcebergRead;
+import org.apache.drill.metastore.iceberg.transform.Transformer;
+import org.apache.drill.metastore.iceberg.write.FileWriter;
+import org.apache.drill.metastore.iceberg.write.ParquetFileWriter;
+import org.apache.iceberg.Table;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Metastore Tables component which stores tables metadata in the corresponding Iceberg table.
+ * Provides methods to read and modify tables metadata.
+ */
+public class IcebergTables implements Tables, MetastoreContext<TableMetadataUnit> {
+
+  public static final String STORAGE_PLUGIN = "storagePlugin";
+  public static final String WORKSPACE = "workspace";
+  public static final String TABLE_NAME = "tableName";
+  public static final String METADATA_KEY = "metadataKey";
+
+  /**
+   * Metastore Tables component partition keys, order of partitioning will be determined based
+   * on order in {@link List} holder.
+   */
+  private static final List<String> PARTITION_KEYS = Arrays.asList(STORAGE_PLUGIN, WORKSPACE, TABLE_NAME, METADATA_KEY);
+
+  public static IcebergTableSchema SCHEMA = IcebergTableSchema.of(TableMetadataUnit.class, PARTITION_KEYS);
+
+  private final Table table;
+
+  public IcebergTables(Table table) {
+    this.table = table;
+  }
+
+  public MetastoreContext<TableMetadataUnit> context() {
+    return this;
+  }
+
+  @Override
+  public Metadata metadata() {
+    return new IcebergMetadata(table);
+  }
+
+  @Override
+  public Read<TableMetadataUnit> read() {
+    return new IcebergRead<>(context());
+  }
+
+  @Override
+  public Modify<TableMetadataUnit> modify() {
+    return new IcebergModify<>(context());
+  }
+
+  @Override
+  public Table table() {
+    return table;
+  }
+
+  @Override
+  public FileWriter fileWriter() {
+    return new ParquetFileWriter(table);
+  }
+
+  @Override
+  public Transformer<TableMetadataUnit> transformer() {
+    return new TablesTransformer(context());
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TableKey.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TableKey.java
new file mode 100644
index 0000000..452c0b8
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TableKey.java
@@ -0,0 +1,106 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.hadoop.fs.Path;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Is used to uniquely identify Drill table in Metastore Tables component
+ * based on storage plugin, workspace and table name.
+ */
+public class TableKey {
+
+  private final String storagePlugin;
+  private final String workspace;
+  private final String tableName;
+
+  public TableKey(String storagePlugin, String workspace, String tableName) {
+    this.storagePlugin = storagePlugin;
+    this.workspace = workspace;
+    this.tableName = tableName;
+  }
+
+  public static TableKey of(TableMetadataUnit unit) {
+    return new TableKey(unit.storagePlugin(), unit.workspace(), unit.tableName());
+  }
+
+  public String storagePlugin() {
+    return storagePlugin;
+  }
+
+  public String workspace() {
+    return workspace;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  /**
+   * Constructs table location based on given Iceberg table location.
+   * For example, metadata for the table dfs.tmp.nation will be stored in
+   * [METASTORE_TABLES_ROOT_DIRECTORY]/dfs/tmp/nation folder.
+   *
+   * @param base Iceberg table location
+   * @return table location
+   */
+  public String toLocation(String base) {
+    Path path = new Path(base);
+    path = new Path(path, storagePlugin);
+    path = new Path(path, workspace);
+    path = new Path(path, tableName);
+    return path.toUri().getPath();
+  }
+
+  /**
+   * Convert table key data into filter conditions.
+   *
+   * @return map of with condition references anf values
+   */
+  public Map<String, Object> toFilterConditions() {
+    Map<String, Object> conditions = new HashMap<>();
+    conditions.put(IcebergTables.STORAGE_PLUGIN, storagePlugin);
+    conditions.put(IcebergTables.WORKSPACE, workspace);
+    conditions.put(IcebergTables.TABLE_NAME, tableName);
+    return conditions;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(storagePlugin, workspace, tableName);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    TableKey that = (TableKey) o;
+    return Objects.equals(storagePlugin, that.storagePlugin)
+      && Objects.equals(workspace, that.workspace)
+      && Objects.equals(tableName, that.tableName);
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TablesOperationTransformer.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TablesOperationTransformer.java
new file mode 100644
index 0000000..67d1bc4
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TablesOperationTransformer.java
@@ -0,0 +1,74 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.iceberg.MetastoreContext;
+import org.apache.drill.metastore.iceberg.operate.Overwrite;
+import org.apache.drill.metastore.iceberg.transform.OperationTransformer;
+import org.apache.iceberg.expressions.Expression;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Metastore Tables component operations transformer that provides mechanism
+ * to convert {@link TableMetadataUnit} data to Metastore overwrite / delete operations.
+ */
+public class TablesOperationTransformer extends OperationTransformer<TableMetadataUnit> {
+
+  public TablesOperationTransformer(MetastoreContext<TableMetadataUnit> context) {
+    super(context);
+  }
+
+  /**
+   * Groups given list of {@link TableMetadataUnit} based on table key
+   * (storage plugin, workspace and table name), each table key is grouped by metadata key.
+   * Each group is converted into overwrite operation.
+   *
+   * @param units Metastore component units
+   * @return list of overwrite operations
+   */
+  public List<Overwrite> toOverwrite(List<TableMetadataUnit> units) {
+    Map<TableKey, Map<String, List<TableMetadataUnit>>> data = units.stream()
+      .collect(
+        Collectors.groupingBy(TableKey::of,
+          Collectors.groupingBy(TableMetadataUnit::metadataKey)
+        ));
+
+    return data.entrySet().parallelStream()
+      .map(dataEntry -> dataEntry.getValue().entrySet().parallelStream()
+        .map(operationEntry -> {
+          TableKey tableKey = dataEntry.getKey();
+
+          String location = tableKey.toLocation(context.table().location());
+
+          Map<String, Object> filterConditions = new HashMap<>(tableKey.toFilterConditions());
+          filterConditions.put(IcebergTables.METADATA_KEY, operationEntry.getKey());
+          Expression expression = context.transformer().filter().transform(filterConditions);
+
+          return toOverwrite(location, expression, operationEntry.getValue());
+        })
+        .collect(Collectors.toList()))
+      .flatMap(Collection::stream)
+      .collect(Collectors.toList());
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TablesOutputDataTransformer.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TablesOutputDataTransformer.java
new file mode 100644
index 0000000..c8ca9e1
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TablesOutputDataTransformer.java
@@ -0,0 +1,57 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.iceberg.exceptions.IcebergMetastoreException;
+import org.apache.drill.metastore.iceberg.transform.OutputDataTransformer;
+
+import java.lang.invoke.MethodHandle;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Metastore Tables component output data transformer that transforms
+ * {@link org.apache.iceberg.data.Record} into {@link TableMetadataUnit}.
+ */
+public class TablesOutputDataTransformer extends OutputDataTransformer<TableMetadataUnit> {
+
+  public TablesOutputDataTransformer(Map<String, MethodHandle> unitSetters) {
+    super(unitSetters);
+  }
+
+  @Override
+  public List<TableMetadataUnit> execute() {
+    List<TableMetadataUnit> results = new ArrayList<>();
+    for (Map<MethodHandle, Object> valueToSet : valuesToSet()) {
+      TableMetadataUnit.Builder builder = TableMetadataUnit.builder();
+      for (Map.Entry<MethodHandle, Object> entry : valueToSet.entrySet()) {
+        try {
+          entry.getKey().invokeWithArguments(builder, entry.getValue());
+        } catch (Throwable e) {
+          throw new IcebergMetastoreException(
+            String.format("Unable to invoke setter for [%s] using [%s]",
+              TableMetadataUnit.Builder.class.getSimpleName(), entry.getKey()), e);
+        }
+      }
+      results.add(builder.build());
+    }
+    return results;
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TablesTransformer.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TablesTransformer.java
new file mode 100644
index 0000000..cf37fad
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/components/tables/TablesTransformer.java
@@ -0,0 +1,58 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.iceberg.MetastoreContext;
+import org.apache.drill.metastore.iceberg.transform.InputDataTransformer;
+import org.apache.drill.metastore.iceberg.transform.OperationTransformer;
+import org.apache.drill.metastore.iceberg.transform.OutputDataTransformer;
+import org.apache.drill.metastore.iceberg.transform.Transformer;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+
+/**
+ * Metastore Tables component filter, data and operations transformer.
+ * Provides needed transformations when reading / writing {@link TableMetadataUnit}
+ * from / into Iceberg table.
+ */
+public class TablesTransformer implements Transformer<TableMetadataUnit> {
+
+  private final MetastoreContext<TableMetadataUnit> context;
+
+  public TablesTransformer(MetastoreContext<TableMetadataUnit> context) {
+    this.context = context;
+  }
+
+  @Override
+  public InputDataTransformer<TableMetadataUnit> inputData() {
+    Table table = context.table();
+    return new InputDataTransformer<>(table.schema(), new Schema(table.spec().partitionType().fields()),
+      TableMetadataUnit.SCHEMA.unitGetters());
+  }
+
+  @Override
+  public OutputDataTransformer<TableMetadataUnit> outputData() {
+    return new TablesOutputDataTransformer(TableMetadataUnit.SCHEMA.unitBuilderSetters());
+  }
+
+  @Override
+  public OperationTransformer<TableMetadataUnit> operation() {
+    return new TablesOperationTransformer(context);
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/config/IcebergConfigConstants.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/config/IcebergConfigConstants.java
new file mode 100644
index 0000000..6f326ac
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/config/IcebergConfigConstants.java
@@ -0,0 +1,93 @@
+/*
+ * 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.metastore.iceberg.config;
+
+import org.apache.drill.metastore.config.MetastoreConfigConstants;
+
+/**
+ * Drill Iceberg Metastore configuration which is defined
+ * in {@link MetastoreConfigConstants#MODULE_RESOURCE_FILE_NAME} file.
+ */
+public interface IcebergConfigConstants {
+
+  /**
+   * Drill Iceberg Metastore configuration properties namespace.
+   */
+  String BASE = MetastoreConfigConstants.BASE + "iceberg.";
+
+  /**
+   * Defines file system configuration properties which can set
+   * using {@link org.apache.hadoop.conf.Configuration} class.
+   */
+  String CONFIG_PROPERTIES = BASE + "config.properties";
+
+  /**
+   * Iceberg Metastore config location namespace.
+   */
+  String LOCATION_NAMESPACE = BASE  + "location.";
+
+  /**
+   * Defines Iceberg Metastore base path.
+   */
+  String BASE_PATH = LOCATION_NAMESPACE + "base_path";
+
+  /**
+   * Defines Iceberg Metastore relative path.
+   */
+  String RELATIVE_PATH = LOCATION_NAMESPACE + "relative_path";
+
+  /**
+   * Drill Iceberg Metastore components configuration properties namespace.
+   */
+  String COMPONENTS = BASE + "components.";
+
+  /**
+   * Common components Iceberg tables properties.
+   */
+  String COMPONENTS_COMMON_PROPERTIES = COMPONENTS + "common.properties";
+
+  /**
+   * Drill Iceberg Metastore Tables components configuration properties namespace.
+   */
+  String COMPONENTS_TABLES = COMPONENTS + "tables.";
+
+  /**
+   * Metastore Tables Iceberg table location inside Iceberg Metastore.
+   */
+  String COMPONENTS_TABLES_LOCATION = COMPONENTS_TABLES + "location";
+
+  /**
+   * Metastore Tables Iceberg table properties.
+   */
+  String COMPONENTS_TABLES_PROPERTIES = COMPONENTS_TABLES + "properties";
+
+  /**
+   * Drill Iceberg Metastore Views components configuration properties namespace.
+   */
+  String COMPONENTS_VIEWS = COMPONENTS + "views.";
+
+  /**
+   * Metastore Views Iceberg table location inside Iceberg Metastore.
+   */
+  String COMPONENTS_VIEWS_LOCATION = COMPONENTS + "location";
+
+  /**
+   * Metastore Views Iceberg table properties.
+   */
+  String COMPONENTS_VIEWS_PROPERTIES = COMPONENTS_VIEWS + "properties";
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/exceptions/IcebergMetastoreException.java
similarity index 61%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/exceptions/IcebergMetastoreException.java
index 7db95fb..31ec8a7 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/exceptions/IcebergMetastoreException.java
@@ -15,19 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.iceberg.exceptions;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.drill.metastore.exceptions.MetastoreException;
 
 /**
- * A metadata which has specific location.
+ * Specific Iceberg Drill Metastore runtime exception to indicate exceptions thrown
+ * during Iceberg Drill Metastore code execution.
  */
-public interface LocationProvider {
+public class IcebergMetastoreException extends MetastoreException {
 
-  /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
-   */
-  Path getPath();
+  public IcebergMetastoreException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public IcebergMetastoreException(String message) {
+    super(message);
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/Delete.java
similarity index 59%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/Delete.java
index 7db95fb..3ce3298 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/Delete.java
@@ -15,19 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.iceberg.operate;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Transaction;
+import org.apache.iceberg.expressions.Expression;
 
 /**
- * A metadata which has specific location.
+ * Iceberg delete operation: deletes data based on given row filter.
  */
-public interface LocationProvider {
+public class Delete implements IcebergOperation {
 
-  /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
-   */
-  Path getPath();
+  private final Expression filter;
+
+  public Delete(Expression filter) {
+    this.filter = filter;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  @Override
+  public void add(Transaction transaction) {
+    transaction.newDelete()
+      .deleteFromRowFilter(filter)
+      .commit();
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergMetadata.java
similarity index 50%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergMetadata.java
index 7db95fb..d6cdabf 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergMetadata.java
@@ -15,19 +15,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.iceberg.operate;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.drill.metastore.operate.Metadata;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+
+import java.util.Map;
 
 /**
- * A metadata which has specific location.
+ * Implementation of {@link Metadata} interface.
+ * Provides information about current Metastore component version and its properties.
  */
-public interface LocationProvider {
-
-  /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
-   */
-  Path getPath();
+public class IcebergMetadata implements Metadata {
+
+  private final Table table;
+
+  public IcebergMetadata(Table table) {
+    this.table = table;
+  }
+
+  @Override
+  public boolean supportsVersioning() {
+    return true;
+  }
+
+  @Override
+  public long version() {
+    Snapshot snapshot = table.currentSnapshot();
+    return snapshot == null ? 0 : snapshot.snapshotId();
+  }
+
+  @Override
+  public Map<String, String> properties() {
+    return table.properties();
+  }
 }
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergModify.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergModify.java
new file mode 100644
index 0000000..a05c2dc
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergModify.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.metastore.iceberg.operate;
+
+import org.apache.drill.metastore.operate.Modify;
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.iceberg.MetastoreContext;
+import org.apache.drill.metastore.iceberg.transform.OperationTransformer;
+import org.apache.iceberg.Transaction;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Implementation of {@link Modify} interface.
+ * Modifies information in Iceberg table based on given overwrite or delete operations.
+ * Executes given operations in one transaction.
+ *
+ * @param <T> Metastore component unit type
+ */
+public class IcebergModify<T> implements Modify<T> {
+
+  private final MetastoreContext<T> context;
+  private final List<T> overwriteUnits = new ArrayList<>();
+  private final List<FilterExpression> deleteFilters = new ArrayList<>();
+  private boolean purge = false;
+
+  public IcebergModify(MetastoreContext<T> context) {
+    this.context = context;
+  }
+
+  @Override
+  public Modify<T> overwrite(List<T> units) {
+    overwriteUnits.addAll(units);
+    return this;
+  }
+
+  @Override
+  public Modify<T> delete(FilterExpression filter) {
+    deleteFilters.add(filter);
+    return this;
+  }
+
+  @Override
+  public Modify<T> purge() {
+    purge = true;
+    return this;
+  }
+
+  @Override
+  public void execute() {
+    OperationTransformer<T> transformer = context.transformer().operation();
+    List<IcebergOperation> operations = new ArrayList<>(transformer.toOverwrite(overwriteUnits));
+    operations.addAll(transformer.toDelete(deleteFilters));
+
+    if (purge) {
+      operations.add(transformer.toDelete((FilterExpression) null));
+    }
+
+    if (operations.isEmpty()) {
+      return;
+    }
+
+    Transaction transaction = context.table().newTransaction();
+    operations.forEach(op -> op.add(transaction));
+    transaction.commitTransaction();
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergOperation.java
similarity index 68%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergOperation.java
index 7db95fb..3d45ac9 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergOperation.java
@@ -15,19 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.iceberg.operate;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Transaction;
 
 /**
- * A metadata which has specific location.
+ * Iceberg operation main goal is to add itself to the given transaction.
+ * Process of adding will depend on operation implementation.
  */
-public interface LocationProvider {
+public interface IcebergOperation {
 
   /**
-   * Returns location of this metadata.
+   * Adds operation to the given transaction.
    *
-   * @return metadata location
+   * @param transaction current transaction
    */
-  Path getPath();
+  void add(Transaction transaction);
 }
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergRead.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergRead.java
new file mode 100644
index 0000000..520a993
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/IcebergRead.java
@@ -0,0 +1,77 @@
+/*
+ * 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.metastore.iceberg.operate;
+
+import org.apache.drill.metastore.operate.Read;
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.iceberg.MetastoreContext;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.iceberg.data.IcebergGenerics;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.types.Types;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Implementation of {@link Read} interface.
+ * Reads information from Iceberg table based on given filter expression.
+ * Supports reading information for specific columns.
+ *
+ * @param <T> Metastore component unit type
+ */
+public class IcebergRead<T> implements Read<T> {
+
+  private final MetastoreContext<T> context;
+  private final String[] defaultColumns;
+  private final List<String> columns = new ArrayList<>();
+  private FilterExpression filter;
+
+  public IcebergRead(MetastoreContext<T> context) {
+    this.context = context;
+    this.defaultColumns = context.table().schema().columns().stream()
+      .map(Types.NestedField::name)
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public Read<T> filter(FilterExpression filter) {
+    this.filter = filter;
+    return this;
+  }
+
+  @Override
+  public Read<T> columns(List<String> columns) {
+    this.columns.addAll(columns);
+    return this;
+  }
+
+  @Override
+  public List<T> execute() {
+    String[] selectedColumns = columns.isEmpty() ? defaultColumns : columns.toArray(new String[0]);
+    Iterable<Record> records = IcebergGenerics.read(context.table())
+      .select(selectedColumns)
+      .where(context.transformer().filter().transform(filter))
+      .build();
+
+    return context.transformer().outputData()
+      .columns(selectedColumns)
+      .records(Lists.newArrayList(records))
+      .execute();
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/Overwrite.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/Overwrite.java
new file mode 100644
index 0000000..67f88fd
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/operate/Overwrite.java
@@ -0,0 +1,53 @@
+/*
+ * 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.metastore.iceberg.operate;
+
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Transaction;
+import org.apache.iceberg.expressions.Expression;
+
+/**
+ * Iceberg overwrite operation: overwrites data with given data file based on given row filter.
+ */
+public class Overwrite implements IcebergOperation {
+
+  private final Expression filter;
+  private final DataFile dataFile;
+
+  public Overwrite(DataFile dataFile, Expression filter) {
+    this.dataFile = dataFile;
+    this.filter = filter;
+  }
+
+  public Expression filter() {
+    return filter;
+  }
+
+  public DataFile dataFile() {
+    return dataFile;
+  }
+
+  @Override
+  public void add(Transaction transaction) {
+    transaction.newOverwrite()
+      .overwriteByRowFilter(filter)
+      .addFile(dataFile)
+      .validateAddedFiles()
+      .commit();
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/schema/IcebergTableSchema.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/schema/IcebergTableSchema.java
new file mode 100644
index 0000000..04665a9
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/schema/IcebergTableSchema.java
@@ -0,0 +1,168 @@
+/*
+ * 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.metastore.iceberg.schema;
+
+import org.apache.drill.metastore.MetastoreFieldDefinition;
+import org.apache.drill.metastore.iceberg.exceptions.IcebergMetastoreException;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Stream;
+
+/**
+ * Provides Iceberg table schema and its partition specification for specific component.
+ */
+public class IcebergTableSchema {
+
+  private static final Logger logger = LoggerFactory.getLogger(IcebergTableSchema.class);
+
+  public static final int STARTING_SCHEMA_INDEX = 1;
+  public static final int STARTING_COMPLEX_TYPES_INDEX = 10_000;
+
+  private static final Map<String, org.apache.iceberg.types.Type> JAVA_TO_ICEBERG_TYPE_MAP =
+    ImmutableMap.<String, org.apache.iceberg.types.Type>builder()
+      .put("string", Types.StringType.get())
+      .put("int", Types.IntegerType.get())
+      .put("integer", Types.IntegerType.get())
+      .put("long", Types.LongType.get())
+      .put("float", Types.FloatType.get())
+      .put("double", Types.DoubleType.get())
+      .put("boolean", Types.BooleanType.get())
+      .build();
+
+  private final Schema tableSchema;
+  private final PartitionSpec partitionSpec;
+
+  public IcebergTableSchema(Schema tableSchema, PartitionSpec partitionSpec) {
+    this.tableSchema = tableSchema;
+    this.partitionSpec = partitionSpec;
+  }
+
+  /**
+   * Based on given class fields annotated with {@link MetastoreFieldDefinition}
+   * generates Iceberg table schema and its partition specification.
+   *
+   * @param clazz base class for Iceberg schema
+   * @param partitionKeys list of partition keys
+   * @return instance of Iceberg table schema
+   */
+  public static IcebergTableSchema of(Class<?> clazz, List<String> partitionKeys) {
+    List<Types.NestedField> tableSchemaFields = new ArrayList<>();
+    Types.NestedField[] partitionSpecSchemaFields = new Types.NestedField[partitionKeys.size()];
+
+    int schemaIndex = STARTING_SCHEMA_INDEX;
+    int complexTypesIndex = STARTING_COMPLEX_TYPES_INDEX;
+
+    for (Field field : clazz.getDeclaredFields()) {
+      if (!field.isAnnotationPresent(MetastoreFieldDefinition.class)) {
+        continue;
+      }
+
+      String typeSimpleName = field.getType().getSimpleName().toLowerCase();
+      org.apache.iceberg.types.Type icebergType = JAVA_TO_ICEBERG_TYPE_MAP.get(typeSimpleName);
+
+      if (icebergType == null && field.getAnnotatedType().getType() instanceof ParameterizedType) {
+        Type[] actualTypeArguments = ((ParameterizedType) field.getAnnotatedType().getType()).getActualTypeArguments();
+        switch (typeSimpleName) {
+          case "list":
+            org.apache.iceberg.types.Type listIcebergType = getGenericsType(actualTypeArguments[0]);
+            icebergType = Types.ListType.ofOptional(complexTypesIndex++, listIcebergType);
+            break;
+          case "map":
+            org.apache.iceberg.types.Type keyIcebergType = getGenericsType(actualTypeArguments[0]);
+            org.apache.iceberg.types.Type valueIcebergType = getGenericsType(actualTypeArguments[1]);
+            icebergType = Types.MapType.ofOptional(complexTypesIndex++, complexTypesIndex++, keyIcebergType, valueIcebergType);
+            break;
+          default:
+            throw new IcebergMetastoreException(String.format(
+              "Unexpected parametrized type for class [%s]: %s", clazz.getCanonicalName(), typeSimpleName));
+        }
+      }
+
+      if (icebergType == null) {
+        throw new IcebergMetastoreException(String.format(
+          "Unexpected type for class [%s]: %s", clazz.getCanonicalName(), typeSimpleName));
+      }
+
+      Types.NestedField icebergField = Types.NestedField.optional(schemaIndex++, field.getName(), icebergType);
+
+      tableSchemaFields.add(icebergField);
+
+      int partitionIndex = partitionKeys.indexOf(field.getName());
+      if (partitionIndex != -1) {
+        partitionSpecSchemaFields[partitionIndex] = icebergField;
+      }
+    }
+
+    if (Stream.of(partitionSpecSchemaFields).anyMatch(Objects::isNull)) {
+      throw new IcebergMetastoreException(String.format(
+        "Some of partition fields are missing in the class [%s]. Partition keys: %s. Partition values: %s.",
+        clazz.getCanonicalName(), partitionKeys, Arrays.asList(partitionSpecSchemaFields)));
+    }
+
+    Schema tableSchema = new Schema(tableSchemaFields);
+    PartitionSpec partitionSpec = buildPartitionSpec(partitionSpecSchemaFields);
+    logger.debug("Constructed Iceberg table schema for class [{}]. Table schema : {}. Partition spec: {}.",
+      clazz.getCanonicalName(), tableSchema, partitionSpec);
+    return new IcebergTableSchema(tableSchema, partitionSpec);
+  }
+
+  public Schema tableSchema() {
+    return tableSchema;
+  }
+
+  public PartitionSpec partitionSpec() {
+    return partitionSpec;
+  }
+
+  private static org.apache.iceberg.types.Type getGenericsType(Type type) {
+    if (!(type instanceof Class)) {
+      throw new IcebergMetastoreException("Unexpected generics type: " + type.getTypeName());
+    }
+    Class typeArgument = (Class) type;
+    String typeSimpleName = typeArgument.getSimpleName().toLowerCase();
+    org.apache.iceberg.types.Type icebergType = JAVA_TO_ICEBERG_TYPE_MAP.get(typeSimpleName);
+    if (icebergType == null) {
+      throw new IcebergMetastoreException("Unexpected type: " + typeSimpleName);
+    }
+    return icebergType;
+  }
+
+  private static PartitionSpec buildPartitionSpec(Types.NestedField[] partitionFields) {
+    Schema schema = new Schema(partitionFields);
+    if (schema.columns().isEmpty()) {
+      return PartitionSpec.unpartitioned();
+    }
+    PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);
+    schema.columns().
+      forEach(column -> builder.identity(column.name()));
+    return builder.build();
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterExpressionVisitor.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterExpressionVisitor.java
new file mode 100644
index 0000000..cad02d5
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterExpressionVisitor.java
@@ -0,0 +1,118 @@
+/*
+ * 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.metastore.iceberg.transform;
+
+import org.apache.drill.metastore.expressions.DoubleExpressionPredicate;
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.expressions.IsPredicate;
+import org.apache.drill.metastore.expressions.ListPredicate;
+import org.apache.drill.metastore.expressions.SimplePredicate;
+import org.apache.drill.metastore.expressions.SingleExpressionPredicate;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import java.util.List;
+
+/**
+ * Visits {@link FilterExpression} implementations and transforms them into Iceberg {@link Expression}.
+ */
+public class FilterExpressionVisitor implements FilterExpression.Visitor<Expression> {
+
+  private static final FilterExpressionVisitor INSTANCE = new FilterExpressionVisitor();
+
+  public static FilterExpression.Visitor<Expression> get() {
+    return INSTANCE;
+  }
+
+  @Override
+  public Expression visit(SimplePredicate.Equal expression) {
+    return Expressions.equal(expression.reference(), expression.value());
+  }
+
+  @Override
+  public Expression visit(SimplePredicate.NotEqual expression) {
+    return Expressions.notEqual(expression.reference(), expression.value());
+  }
+
+  @Override
+  public Expression visit(SimplePredicate.LessThan expression) {
+    return Expressions.lessThan(expression.reference(), expression.value());
+  }
+
+  @Override
+  public Expression visit(SimplePredicate.LessThanOrEqual expression) {
+    return Expressions.lessThanOrEqual(expression.reference(), expression.value());
+  }
+
+  @Override
+  public Expression visit(SimplePredicate.GreaterThan expression) {
+    return Expressions.greaterThan(expression.reference(), expression.value());
+  }
+
+  @Override
+  public Expression visit(SimplePredicate.GreaterThanOrEqual expression) {
+    return Expressions.greaterThanOrEqual(expression.reference(), expression.value());
+  }
+
+  @Override
+  public Expression visit(ListPredicate.In expression) {
+    return toInExpression(expression.reference(), expression.values());
+  }
+
+  @Override
+  public Expression visit(ListPredicate.NotIn expression) {
+    Expression in = toInExpression(expression.reference(), expression.values());
+    return Expressions.not(in);
+  }
+
+  @Override
+  public Expression visit(IsPredicate.IsNull expression) {
+    return Expressions.isNull(expression.reference());
+  }
+
+  @Override
+  public Expression visit(IsPredicate.IsNotNull expression) {
+    return Expressions.notNull(expression.reference());
+  }
+
+  @Override
+  public Expression visit(SingleExpressionPredicate.Not expression) {
+    Expression child = expression.expression().accept(this);
+    return Expressions.not(child);
+  }
+
+  @Override
+  public Expression visit(DoubleExpressionPredicate.And expression) {
+    Expression right = expression.right().accept(this);
+    Expression left = expression.left().accept(this);
+    return Expressions.and(right, left);
+  }
+
+  @Override
+  public Expression visit(DoubleExpressionPredicate.Or expression) {
+    Expression right = expression.right().accept(this);
+    Expression left = expression.left().accept(this);
+    return Expressions.or(right, left);
+  }
+
+  private Expression toInExpression(String reference, List<?> values) {
+    return values.stream()
+      .map(value -> (Expression) Expressions.equal(reference, value))
+      .reduce(Expressions.alwaysFalse(), Expressions::or);
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterTransformer.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterTransformer.java
new file mode 100644
index 0000000..5ee753c
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterTransformer.java
@@ -0,0 +1,56 @@
+/*
+ * 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.metastore.iceberg.transform;
+
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Transforms given input into Iceberg {@link Expression} which is used as filter
+ * to retrieve, overwrite or delete Metastore component data.
+ */
+public class FilterTransformer {
+
+  private final FilterExpression.Visitor<Expression> visitor = FilterExpressionVisitor.get();
+
+  public Expression transform(FilterExpression filter) {
+    return filter == null ? Expressions.alwaysTrue() : filter.accept(visitor);
+  }
+
+  public Expression transform(Map<String, Object> conditions) {
+    if (conditions == null || conditions.isEmpty()) {
+      return Expressions.alwaysTrue();
+    }
+
+    List<Expression> expressions = conditions.entrySet().stream()
+      .map(entry -> Expressions.equal(entry.getKey(), entry.getValue()))
+      .collect(Collectors.toList());
+
+    if (expressions.size() == 1) {
+      return expressions.get(0);
+    }
+
+    return Expressions.and(expressions.get(0), expressions.get(1),
+      expressions.subList(2, expressions.size()).toArray(new Expression[0]));
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/InputDataTransformer.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/InputDataTransformer.java
new file mode 100644
index 0000000..ca0350e
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/InputDataTransformer.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.metastore.iceberg.transform;
+
+
+import org.apache.drill.metastore.iceberg.exceptions.IcebergMetastoreException;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.types.Types;
+
+import java.lang.invoke.MethodHandle;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Converts list of into Metastore component units into {@link WriteData} holder.
+ *
+ * @param <T> Metastore component unit type
+ */
+public class InputDataTransformer<T> {
+
+  private final Schema tableSchema;
+  private final Schema partitionSpecSchema;
+  private final Map<String, MethodHandle> unitGetters;
+  private final List<T> units = new ArrayList<>();
+
+  public InputDataTransformer(Schema tableSchema,
+                              Schema partitionSpecSchema,
+                              Map<String, MethodHandle> unitGetters) {
+    this.tableSchema = tableSchema;
+    this.partitionSpecSchema = partitionSpecSchema;
+    this.unitGetters = unitGetters;
+  }
+
+  public InputDataTransformer<T> units(List<T> units) {
+    this.units.addAll(units);
+    return this;
+  }
+
+  public WriteData execute() {
+    List<Record> records = new ArrayList<>();
+    Set<Record> partitions = new HashSet<>();
+    for (T unit : units) {
+      partitions.add(getPartition(unit, partitionSpecSchema, unitGetters));
+
+      Record record = GenericRecord.create(tableSchema);
+      for (Types.NestedField column : tableSchema.columns()) {
+        String name = column.name();
+        MethodHandle methodHandle = unitGetters.get(name);
+        if (methodHandle == null) {
+          // ignore absent getters
+          continue;
+        }
+        try {
+          record.setField(name, methodHandle.invoke(unit));
+        } catch (Throwable e) {
+          throw new IcebergMetastoreException(
+            String.format("Unable to invoke getter for column [%s] using [%s]", name, methodHandle), e);
+        }
+      }
+      records.add(record);
+    }
+
+    if (partitions.size() > 1) {
+      throw new IcebergMetastoreException(String.format(
+        "Partition keys values must be the same for all records in the partition. " +
+          "Partition schema: [%s]. Received partition values: %s", partitionSpecSchema, partitions));
+    }
+
+    return new WriteData(records, partitions.isEmpty() ? null : partitions.iterator().next());
+  }
+
+  /**
+   * Generates record with partition values based on given partition schema
+   * and Metastore component unit instance.
+   *
+   * @param unit specific Metastore component unit
+   * @param schema partition schema
+   * @param unitGetters specific Metastore component unit getters
+   * @return {@link Record} with partition values
+   * @throws IcebergMetastoreException if getter to partition column is absent or
+   *         partition column value is null
+   */
+  private Record getPartition(T unit, Schema schema, Map<String, MethodHandle> unitGetters) {
+    Record partitionRecord = GenericRecord.create(schema);
+    for (Types.NestedField column : schema.columns()) {
+      String name = column.name();
+      MethodHandle methodHandle = unitGetters.get(name);
+      if (methodHandle == null) {
+        throw new IcebergMetastoreException(
+          String.format("Getter for partition key [%s::%s] must be declared in [%s] class",
+            name, column.type(), unit.getClass().getSimpleName()));
+      }
+      Object value;
+      try {
+        value = methodHandle.invoke(unit);
+      } catch (Throwable e) {
+        throw new IcebergMetastoreException(
+          String.format("Unable to invoke getter for column [%s] using [%s]", name, methodHandle), e);
+      }
+
+      if (value == null) {
+        throw new IcebergMetastoreException(
+          String.format("Partition key [%s::%s] value must be set", name, column.type()));
+      }
+      partitionRecord.setField(name, value);
+    }
+    return partitionRecord;
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/OperationTransformer.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/OperationTransformer.java
new file mode 100644
index 0000000..c410c76
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/OperationTransformer.java
@@ -0,0 +1,85 @@
+/*
+ * 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.metastore.iceberg.transform;
+
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.iceberg.MetastoreContext;
+import org.apache.drill.metastore.iceberg.operate.Delete;
+import org.apache.drill.metastore.iceberg.operate.Overwrite;
+import org.apache.drill.metastore.iceberg.write.File;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.expressions.Expression;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+/**
+ * Base class to transforms given input into
+ * {@link org.apache.drill.metastore.iceberg.operate.IcebergOperation} implementations.
+ *
+ * @param <T> Metastore component unit type
+ */
+public abstract class OperationTransformer<T> {
+
+  protected final MetastoreContext<T> context;
+
+  protected OperationTransformer(MetastoreContext<T> context) {
+    this.context = context;
+  }
+
+  public Overwrite toOverwrite(String location, Expression expression, List<T> units) {
+    WriteData writeData = context.transformer().inputData()
+      .units(units)
+      .execute();
+
+    File file = context.fileWriter()
+      .records(writeData.records())
+      .location(location)
+      .name(UUID.randomUUID().toString())
+      .write();
+
+    DataFile dataFile = DataFiles.builder(context.table().spec())
+      .withInputFile(file.input())
+      .withMetrics(file.metrics())
+      .withPartition(writeData.partition())
+      .build();
+
+    return new Overwrite(dataFile, expression);
+  }
+
+  public List<Delete> toDelete(List<FilterExpression> filters) {
+    return filters.stream()
+      .map(this::toDelete)
+      .collect(Collectors.toList());
+  }
+
+  public Delete toDelete(FilterExpression filter) {
+    return new Delete(context.transformer().filter().transform(filter));
+  }
+
+  /**
+   * Converts given list of Metastore components units into list of overwrite operations.
+   * Specific for each Metastore component.
+   *
+   * @param units Metastore component units
+   * @return list of overwrite operations
+   */
+  public abstract List<Overwrite> toOverwrite(List<T> units);
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/OutputDataTransformer.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/OutputDataTransformer.java
new file mode 100644
index 0000000..047c455
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/OutputDataTransformer.java
@@ -0,0 +1,85 @@
+/*
+ * 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.metastore.iceberg.transform;
+
+import org.apache.iceberg.data.Record;
+
+import java.lang.invoke.MethodHandle;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Base class to convert list of {@link Record}
+ * into Metastore component units for the given list of column names.
+ *
+ * @param <T> Metastore component unit type
+ */
+public abstract class OutputDataTransformer<T> {
+
+  private final Map<String, MethodHandle> unitSetters;
+  private final List<String> columns = new ArrayList<>();
+  private final List<Record> records = new ArrayList<>();
+
+  protected OutputDataTransformer(Map<String, MethodHandle> unitSetters) {
+    this.unitSetters = unitSetters;
+  }
+
+  public OutputDataTransformer<T> columns(List<String> columns) {
+    this.columns.addAll(columns);
+    return this;
+  }
+
+  public OutputDataTransformer<T> columns(String... columns) {
+    return columns(Arrays.asList(columns));
+  }
+
+  public OutputDataTransformer<T> records(List<Record> records) {
+    this.records.addAll(records);
+    return this;
+  }
+
+  /**
+   * Converts given list of {@link Record} into Metastore component units.
+   * Specific for each Metastore component.
+   *
+   * @return list of Metastore component units
+   */
+  public abstract List<T> execute();
+
+  /**
+   * For each given record prepares specific methods handler and its value
+   * to be set into Metastore specific component unit.
+   * Ignores absent setters for columns and null values.
+   *
+   * @return list of methods handlers and values to set
+   */
+  protected List<Map<MethodHandle, Object>> valuesToSet() {
+    return records.stream()
+      .map(record -> columns.stream()
+        .filter(column -> unitSetters.get(column) != null)
+        .filter(column -> record.getField(column) != null)
+        .collect(Collectors.toMap(
+          unitSetters::get,
+          record::getField,
+          (o, n) -> n)))
+      .collect(Collectors.toList());
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/Transformer.java
similarity index 51%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java
copy to metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/Transformer.java
index da813d7..1b46900 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/Transformer.java
@@ -15,47 +15,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.iceberg.transform;
 
 /**
- * Enum with possible types of metadata.
+ * Provides various mechanism implementations to transform filters, data and operations.
+ *
+ * @param <T> component unit type
  */
-public enum MetadataType {
-
-  /**
-   * Metadata that can be applicable to any type.
-   */
-  ALL,
-
-  /**
-   * Table level metadata type.
-   */
-  TABLE,
-
-  /**
-   * Segment level metadata type. It corresponds to the metadata
-   * within specific directory for FS tables, or may correspond to partition for hive tables.
-   */
-  SEGMENT,
+public interface Transformer<T> {
 
   /**
-   * Drill partition level metadata type. It corresponds to parts of table data which has the same
-   * values within specific column, i.e. partitions discovered by Drill.
+   * Creates filter transformer. Since filter transformer does not
+   * depend on specific Metastore component implementation, provides
+   * it as default method.
+   *
+   * @return filter transformer
    */
-  PARTITION,
+  default FilterTransformer filter() {
+    return new FilterTransformer();
+  }
 
   /**
-   * File level metadata type.
+   * @return input data transformer for specific Metastore component
    */
-  FILE,
+  InputDataTransformer<T> inputData();
 
   /**
-   * Row group level metadata type. Used for parquet tables.
+   * @return output data transformer for specific Metastore component
    */
-  ROW_GROUP,
+  OutputDataTransformer<T> outputData();
 
   /**
-   * Metadata type which helps to indicate that there is no overflow of metadata.
+   * @return operation transformer for specific Metastore component
    */
-  NONE
+  OperationTransformer<T> operation();
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/WriteData.java
similarity index 59%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/WriteData.java
index 7db95fb..234acc3 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/WriteData.java
@@ -15,19 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.iceberg.transform;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.data.Record;
+
+import java.util.List;
 
 /**
- * A metadata which has specific location.
+ * Provides records in format used to store them in Iceberg and their partition information.
  */
-public interface LocationProvider {
+public class WriteData {
+
+  private final List<Record> records;
+  private final Record partition;
+
+  public WriteData(List<Record> records, Record partition) {
+    this.records = records;
+    this.partition = partition;
+  }
+
+  public List<Record> records() {
+    return records;
+  }
 
-  /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
-   */
-  Path getPath();
+  public Record partition() {
+    return partition;
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/write/File.java
similarity index 52%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/write/File.java
index 7db95fb..8b753ff 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/write/File.java
@@ -15,19 +15,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.iceberg.write;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
 
 /**
- * A metadata which has specific location.
+ * Written file information holder.
+ * Provides input / output file instances, file location and its metrics.
  */
-public interface LocationProvider {
-
-  /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
-   */
-  Path getPath();
+public class File {
+
+  private final OutputFile outputFile;
+  private final Metrics metrics;
+
+  public File(OutputFile outputFile, Metrics metrics) {
+    this.outputFile = outputFile;
+    this.metrics = metrics;
+  }
+
+  public OutputFile output() {
+    return outputFile;
+  }
+
+  public InputFile input() {
+    return outputFile.toInputFile();
+  }
+
+  public Metrics metrics() {
+    return metrics;
+  }
+
+  public String location() {
+    return outputFile.location();
+  }
 }
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/write/FileWriter.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/write/FileWriter.java
new file mode 100644
index 0000000..73290f3
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/write/FileWriter.java
@@ -0,0 +1,60 @@
+/*
+ * 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.metastore.iceberg.write;
+
+import org.apache.iceberg.data.Record;
+
+import java.util.List;
+
+/**
+ * Generic interface that provides functionality to write data into the file.
+ * File format will be determined by the implementation.
+ */
+public interface FileWriter {
+
+  /**
+   * Indicates list of records to be written.
+   *
+   * @param records list of records to be written into file
+   * @return current File Writer instance
+   */
+  FileWriter records(List<Record> records);
+
+  /**
+   * Indicates location where new file will be written to.
+   *
+   * @param location file location
+   * @return current File Writer instance
+   */
+  FileWriter location(String location);
+
+  /**
+   * Indicates name with which new file will be created.
+   *
+   * @param name file name
+   * @return current File Writer instance
+   */
+  FileWriter name(String name);
+
+  /**
+   * Writes provided list of records into the file.
+   *
+   * @return written file information holder
+   */
+  File write();
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/write/ParquetFileWriter.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/write/ParquetFileWriter.java
new file mode 100644
index 0000000..3c1604d
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/write/ParquetFileWriter.java
@@ -0,0 +1,97 @@
+/*
+ * 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.metastore.iceberg.write;
+
+import org.apache.drill.metastore.iceberg.exceptions.IcebergMetastoreException;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.parquet.Parquet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Parquet File Writer implementation. Creates Parquet file in given location and name
+ * and '.parquet' extension and writes given data into it.
+ * Note: if file already exists, it will be overwritten.
+ */
+public class ParquetFileWriter implements FileWriter {
+
+  private final Table table;
+  private final List<Record> records = new ArrayList<>();
+  private String location;
+  private String name;
+
+  public ParquetFileWriter(Table table) {
+    this.table = table;
+  }
+
+  @Override
+  public FileWriter records(List<Record> records) {
+    this.records.addAll(records);
+    return this;
+  }
+
+  @Override
+  public FileWriter location(String location) {
+    this.location = location;
+    return this;
+  }
+
+  @Override
+  public FileWriter name(String name) {
+    this.name = name;
+    return this;
+  }
+
+  @Override
+  public File write() {
+    Objects.requireNonNull(location, "File create location must be specified");
+    Objects.requireNonNull(name, "File name must be specified");
+
+    OutputFile outputFile = table.io().newOutputFile(new Path(location, FileFormat.PARQUET.addExtension(name)).toUri().getPath());
+
+    FileAppender<Record> fileAppender = null;
+    try {
+      fileAppender = Parquet.write(outputFile)
+        .forTable(table)
+        .createWriterFunc(GenericParquetWriter::buildWriter)
+        .build();
+      fileAppender.addAll(records);
+      fileAppender.close();
+      // metrics are available only when file was written (i.e. close method was executed)
+      return new File(outputFile, fileAppender.metrics());
+    } catch (IOException | ClassCastException e) {
+      if (fileAppender != null) {
+        try {
+          fileAppender.close();
+        } catch (Exception ex) {
+          // write has failed anyway, ignore closing exception if any and throw initial one
+        }
+      }
+      throw new IcebergMetastoreException("Unable to write data into parquet file", e);
+    }
+  }
+}
diff --git a/metastore/iceberg-metastore/src/main/resources/drill-metastore-module.conf b/metastore/iceberg-metastore/src/main/resources/drill-metastore-module.conf
new file mode 100644
index 0000000..182d029
--- /dev/null
+++ b/metastore/iceberg-metastore/src/main/resources/drill-metastore-module.conf
@@ -0,0 +1,60 @@
+// 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.
+
+drill.metastore.iceberg: {
+
+    // File system config can be specified
+    config.properties: {
+      // Iceberg tables location will be created under default file system
+      // taken from Hadoop configuration unless the below property is set
+      // fs.defaultFS: "file:///"
+    }
+
+    // Iceberg Metastore location is constructed based on
+    // combination of base_path and relative_path config values
+    location: {
+      // Iceberg table base path, if not indicated, user home directory will be used
+      // base_path: "/app",
+
+      // Iceberg table relative path unique among clusters
+      // ${drill.exec.zk.root} value will be substituted from Drill main config
+       relative_path: ${drill.exec.zk.root}"/metastore/iceberg"
+    }
+
+    components: {
+        // Common properties for all Iceberg tables from org.apache.iceberg.TableProperties can be specified
+       common.properties: {
+          write.format.default: "parquet",
+          write.metadata.compression-codec: "none"
+       },
+
+       tables: {
+          // Iceberg table location in Iceberg Metastore
+          location: "tables",
+
+          // Specific tables properties from org.apache.iceberg.TableProperties can be specified
+          properties: {}
+       },
+
+       views: {
+          // Iceberg table location in Iceberg Metastore
+          location: "views",
+
+          // Specific views properties from org.apache.iceberg.TableProperties can be specified
+          properties: {}
+       }
+    }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/IcebergBaseTest.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/IcebergBaseTest.java
new file mode 100644
index 0000000..0250ce9
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/IcebergBaseTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.metastore.iceberg;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigValueFactory;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.GuavaPatcher;
+import org.apache.drill.metastore.iceberg.config.IcebergConfigConstants;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+
+public abstract class IcebergBaseTest {
+
+  @ClassRule
+  public static TemporaryFolder defaultFolder = new TemporaryFolder();
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @BeforeClass
+  public static void setup() {
+    // patches Guava Preconditions class with missing methods
+    GuavaPatcher.patch();
+  }
+
+  /**
+   * Creates Hadoop configuration and sets local file system as default.
+   *
+   * @return {@link Configuration} instance
+   */
+  protected static Configuration baseHadoopConfig() {
+    Configuration config = new Configuration();
+    config.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
+    return config;
+  }
+
+  /**
+   * Creates default configuration for Iceberg Metastore based on given base path.
+   * Sets local file system as default.
+   *
+   * @param base Iceberg Metastore base path
+   * @return {@link Config} instance
+   */
+  protected static Config baseIcebergConfig(File base) {
+    return DrillConfig.create()
+      .withValue(IcebergConfigConstants.BASE_PATH,
+        ConfigValueFactory.fromAnyRef(new Path(base.toURI().getPath()).toUri().getPath()))
+      .withValue(IcebergConfigConstants.RELATIVE_PATH,
+        ConfigValueFactory.fromAnyRef("drill/metastore/iceberg"))
+      .withValue(String.format("%s.%s", IcebergConfigConstants.CONFIG_PROPERTIES, FileSystem.FS_DEFAULT_NAME_KEY),
+        ConfigValueFactory.fromAnyRef(FileSystem.DEFAULT_FS))
+      .withValue(IcebergConfigConstants.COMPONENTS_TABLES_LOCATION,
+        ConfigValueFactory.fromAnyRef("tables"))
+      .withValue(IcebergConfigConstants.COMPONENTS_VIEWS_LOCATION,
+        ConfigValueFactory.fromAnyRef("views"));
+  }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestBasicRequests.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestBasicRequests.java
new file mode 100644
index 0000000..9862c9b
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestBasicRequests.java
@@ -0,0 +1,484 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.metastore.components.tables.BasicTablesTransformer;
+import org.apache.drill.metastore.components.tables.BasicTablesRequests;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.components.tables.MetastoreTableInfo;
+import org.apache.drill.metastore.components.tables.Tables;
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.iceberg.IcebergBaseTest;
+import org.apache.drill.metastore.iceberg.IcebergMetastore;
+import org.apache.drill.metastore.metadata.BaseTableMetadata;
+import org.apache.drill.metastore.metadata.FileMetadata;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.PartitionMetadata;
+import org.apache.drill.metastore.metadata.RowGroupMetadata;
+import org.apache.drill.metastore.metadata.SegmentMetadata;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestBasicRequests extends IcebergBaseTest {
+
+  private static Tables tables;
+  private static BasicTablesRequests basicRequests;
+  private static TableMetadataUnit nationTable;
+  private static TableInfo nationTableInfo;
+
+  @BeforeClass
+  public static void init() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(defaultFolder.getRoot()));
+    tables = new IcebergMetastore(config).tables();
+    prepareData(tables);
+    basicRequests = tables.basicRequests();
+  }
+
+  @Test
+  public void testMetastoreTableInfoAbsentTable() {
+    TableInfo tableInfo = TableInfo.builder().storagePlugin("dfs").workspace("tmp").name("absent").build();
+    MetastoreTableInfo metastoreTableInfo = basicRequests.metastoreTableInfo(tableInfo);
+    assertFalse(metastoreTableInfo.isExists());
+    assertEquals(tableInfo, metastoreTableInfo.tableInfo());
+    assertNull(metastoreTableInfo.lastModifiedTime());
+  }
+
+  @Test
+  public void testMetastoreTableInfoExistingTable() {
+    MetastoreTableInfo metastoreTableInfo = basicRequests.metastoreTableInfo(nationTableInfo);
+    assertTrue(metastoreTableInfo.isExists());
+    assertEquals(nationTableInfo, metastoreTableInfo.tableInfo());
+    assertEquals(nationTable.lastModifiedTime(), metastoreTableInfo.lastModifiedTime());
+    assertTrue(metastoreTableInfo.metastoreVersion() > 0);
+  }
+
+  @Test
+  public void testHasMetastoreTableInfoChangedFalse() {
+    MetastoreTableInfo metastoreTableInfo = basicRequests.metastoreTableInfo(nationTableInfo);
+    assertFalse(basicRequests.hasMetastoreTableInfoChanged(metastoreTableInfo));
+  }
+
+  @Test
+  public void testHasMetastoreTableInfoChangedTrue() {
+    TableMetadataUnit unit = nationTable.toBuilder()
+      .tableName("changingTable")
+      .lastModifiedTime(1L)
+      .build();
+    tables.modify()
+      .overwrite(unit)
+      .execute();
+    TableInfo tableInfo = TableInfo.builder().metadataUnit(unit).build();
+    MetastoreTableInfo metastoreTableInfo = basicRequests.metastoreTableInfo(tableInfo);
+
+    TableMetadataUnit updatedUnit = unit.toBuilder()
+      .lastModifiedTime(2L)
+      .build();
+    tables.modify()
+      .overwrite(updatedUnit)
+      .execute();
+
+    assertTrue(basicRequests.hasMetastoreTableInfoChanged(metastoreTableInfo));
+  }
+
+  @Test
+  public void testTablesMetadataAbsent() {
+    List<BaseTableMetadata> tablesMetadata = basicRequests.tablesMetadata(
+      FilterExpression.and(
+        FilterExpression.equal(IcebergTables.STORAGE_PLUGIN, "dfs"),
+        FilterExpression.equal(IcebergTables.WORKSPACE, "absent")));
+    assertTrue(tablesMetadata.isEmpty());
+  }
+
+  @Test
+  public void testTablesMetadataExisting() {
+    List<BaseTableMetadata> baseTableMetadata = basicRequests.tablesMetadata(
+      FilterExpression.and(
+        FilterExpression.equal(IcebergTables.STORAGE_PLUGIN, "dfs"),
+        FilterExpression.equal(IcebergTables.WORKSPACE, "tmp")));
+    assertTrue(baseTableMetadata.size() > 1);
+  }
+
+  @Test
+  public void testTableMetadataAbsent() {
+    TableInfo tableInfo = TableInfo.builder().storagePlugin("dfs").workspace("tmp").name("absent").build();
+    BaseTableMetadata tableMetadata = basicRequests.tableMetadata(tableInfo);
+    assertNull(tableMetadata);
+  }
+
+  @Test
+  public void testTableMetadataExisting() {
+    BaseTableMetadata tableMetadata = basicRequests.tableMetadata(nationTableInfo);
+    assertNotNull(tableMetadata);
+  }
+
+  @Test
+  public void testSegmentsMetadataByMetadataKeyAbsent() {
+    List<SegmentMetadata> segmentMetadata = basicRequests.segmentsMetadataByMetadataKey(
+      nationTableInfo,
+      Collections.singletonList("/tmp/nation/part_int=3/d6"),
+      "part_int=3");
+    assertTrue(segmentMetadata.isEmpty());
+  }
+
+  @Test
+  public void testSegmentsMetadataByMetadataKeyExisting() {
+    List<SegmentMetadata> segmentMetadata = basicRequests.segmentsMetadataByMetadataKey(
+      nationTableInfo,
+      Arrays.asList("/tmp/nation/part_int=3/d3", "/tmp/nation/part_int=3/d4"),
+      "part_int=3");
+    assertEquals(2, segmentMetadata.size());
+  }
+
+  @Test
+  public void testSegmentsMetadataByColumnAbsent() {
+    List<SegmentMetadata> segmentMetadata = basicRequests.segmentsMetadataByColumn(
+      nationTableInfo,
+      Arrays.asList("/tmp/nation/part_int=3/d4", "/tmp/nation/part_int=3/d5"),
+      "n_region");
+    assertTrue(segmentMetadata.isEmpty());
+  }
+
+  @Test
+  public void testSegmentsMetadataByColumnExisting() {
+    List<SegmentMetadata> segmentMetadata = basicRequests.segmentsMetadataByColumn(
+      nationTableInfo,
+      Arrays.asList("/tmp/nation/part_int=3/d3", "/tmp/nation/part_int=3/d4"),
+      "n_nation");
+    assertEquals(2, segmentMetadata.size());
+  }
+
+  @Test
+  public void testPartitionsMetadataAbsent() {
+    List<PartitionMetadata> partitionMetadata = basicRequests.partitionsMetadata(
+      nationTableInfo,
+      Arrays.asList("part_int=3", "part_int=4"),
+      "id");
+    assertTrue(partitionMetadata.isEmpty());
+  }
+
+  @Test
+  public void testPartitionsMetadataExisting() {
+    List<PartitionMetadata> partitionMetadata = basicRequests.partitionsMetadata(
+      nationTableInfo,
+      Arrays.asList("part_int=3", "part_int=4"),
+      "n_nation");
+    assertEquals(2, partitionMetadata.size());
+  }
+
+  @Test
+  public void testFilesMetadataAbsent() {
+    List<FileMetadata> fileMetadata = basicRequests.filesMetadata(
+      nationTableInfo,
+      "part_int=3",
+      Collections.singletonList("/tmp/nation/part_int=3/part_varchar=g/0_0_2.parquet"));
+    assertTrue(fileMetadata.isEmpty());
+  }
+
+  @Test
+  public void testFilesMetadataExisting() {
+    List<FileMetadata> fileMetadata = basicRequests.filesMetadata(
+      nationTableInfo,
+      "part_int=3",
+      Arrays.asList("/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet",
+        "/tmp/nation/part_int=3/part_varchar=g/0_0_1.parquet"));
+    assertEquals(2, fileMetadata.size());
+  }
+
+  @Test
+  public void testFileMetadataAbsent() {
+    FileMetadata fileMetadata = basicRequests.fileMetadata(
+      nationTableInfo,
+      "part_int=3",
+      "/tmp/nation/part_int=3/part_varchar=g/0_0_2.parquet");
+    assertNull(fileMetadata);
+  }
+
+  @Test
+  public void testFileMetadataExisting() {
+    FileMetadata fileMetadata = basicRequests.fileMetadata(
+      nationTableInfo,
+      "part_int=3",
+      "/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet");
+    assertNotNull(fileMetadata);
+  }
+
+  @Test
+  public void testRowGroupsMetadataAbsent() {
+    List<RowGroupMetadata> rowGroupMetadata = basicRequests.rowGroupsMetadata(
+      nationTableInfo,
+      "part_int=3",
+      "/tmp/nation/part_int=3/part_varchar=g/0_0_2.parquet");
+    assertTrue(rowGroupMetadata.isEmpty());
+  }
+
+  @Test
+  public void testRowGroupsMetadataExisting() {
+    List<RowGroupMetadata> rowGroupMetadata = basicRequests.rowGroupsMetadata(
+      nationTableInfo,
+      "part_int=3",
+      "/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet");
+    assertEquals(2, rowGroupMetadata.size());
+  }
+
+  @Test
+  public void testFullSegmentsMetadataWithoutPartitions() {
+    BasicTablesTransformer.MetadataHolder metadataHolder = basicRequests.fullSegmentsMetadataWithoutPartitions(
+      nationTableInfo,
+      Arrays.asList("part_int=4", "part_int=5"),
+      Arrays.asList("/tmp/nation/part_int=4/d5", "/tmp/nation/part_int=4/part_varchar=g"));
+    assertTrue(metadataHolder.tables().isEmpty());
+    assertTrue(metadataHolder.partitions().isEmpty());
+    assertEquals(1, metadataHolder.segments().size());
+    assertEquals(1, metadataHolder.files().size());
+    assertEquals(2, metadataHolder.rowGroups().size());
+  }
+
+  @Test
+  public void testFilesLastModifiedTime() {
+    Map<String, Long> result = basicRequests.filesLastModifiedTime(
+      nationTableInfo,
+      "part_int=3",
+      Collections.singletonList("/tmp/nation/part_int=3/part_varchar=g"));
+
+    Map<String, Long> expected = new HashMap<>();
+    expected.put("/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet", 1L);
+    expected.put("/tmp/nation/part_int=3/part_varchar=g/0_0_1.parquet", 2L);
+
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testSegmentsLastModifiedTime() {
+    Map<String, Long> result = basicRequests.segmentsLastModifiedTime(
+      nationTableInfo,
+      Arrays.asList("/tmp/nation/part_int=3/d3", "/tmp/nation/part_int=4/d5"));
+
+    Map<String, Long> expected = new HashMap<>();
+    expected.put("part_int=3", 1L);
+    expected.put("part_int=4", 3L);
+
+    assertEquals(expected, result);
+  }
+
+  @Test
+  public void testInterestingColumnsAndPartitionKeys() {
+    TableMetadataUnit result = basicRequests.interestingColumnsAndPartitionKeys(nationTableInfo);
+    assertEquals(nationTable.interestingColumns(), result.interestingColumns());
+    assertEquals(nationTable.partitionKeys(), result.partitionKeys());
+    assertNull(result.tableName());
+    assertNull(result.lastModifiedTime());
+  }
+
+  @Test
+  public void testCustomRequest() {
+    BasicTablesRequests.RequestMetadata requestMetadata = BasicTablesRequests.RequestMetadata.builder()
+      .column("n_nation")
+      .metadataType(MetadataType.PARTITION.name())
+      .build();
+
+    List<TableMetadataUnit> units = basicRequests.request(requestMetadata);
+    assertEquals(2, units.size());
+  }
+
+  /**
+   * Prepares data which will be used in the unit tests.
+   * Note: data is filled to check basic request results and might not be exactly true to reality.
+   *
+   * @param tables Drill Metastore Tables instance
+   */
+  private static void prepareData(Tables tables) {
+    nationTable = basicUnit().toBuilder()
+      .tableName("nation")
+      .metadataType(MetadataType.TABLE.name())
+      .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+      .build();
+
+    nationTableInfo = TableInfo.builder().metadataUnit(nationTable).build();
+
+    TableMetadataUnit nationSegment1 = nationTable.toBuilder()
+      .metadataType(MetadataType.SEGMENT.name())
+      .metadataKey("part_int=3")
+      .location("/tmp/nation/part_int=3/d3")
+      .column("n_nation")
+      .lastModifiedTime(1L)
+      .build();
+
+    TableMetadataUnit nationSegment2 = nationTable.toBuilder()
+      .metadataType(MetadataType.SEGMENT.name())
+      .metadataKey("part_int=3")
+      .location("/tmp/nation/part_int=3/d4")
+      .column("n_nation")
+      .lastModifiedTime(2L)
+      .build();
+
+    TableMetadataUnit nationSegment3 = nationTable.toBuilder()
+      .metadataType(MetadataType.SEGMENT.name())
+      .metadataKey("part_int=4")
+      .location("/tmp/nation/part_int=4/d5")
+      .column("n_nation")
+      .lastModifiedTime(3L)
+      .build();
+
+    TableMetadataUnit nationPartition1 = nationTable.toBuilder()
+      .metadataType(MetadataType.PARTITION.name())
+      .metadataKey("part_int=3")
+      .location("/tmp/nation/part_int=3/d5")
+      .column("n_nation")
+      .build();
+
+    TableMetadataUnit nationPartition2 = nationTable.toBuilder()
+      .metadataType(MetadataType.PARTITION.name())
+      .metadataKey("part_int=4")
+      .location("/tmp/nation/part_int=4/d5")
+      .column("n_nation")
+      .build();
+
+    TableMetadataUnit nationPartition3 = nationTable.toBuilder()
+      .metadataType(MetadataType.PARTITION.name())
+      .metadataKey("part_int=4")
+      .column("n_region")
+      .location("/tmp/nation/part_int=4/d6")
+      .build();
+
+    TableMetadataUnit nationFile1 = nationTable.toBuilder()
+      .metadataType(MetadataType.FILE.name())
+      .metadataKey("part_int=3")
+      .location("/tmp/nation/part_int=3/part_varchar=g")
+      .path("/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet")
+      .lastModifiedTime(1L)
+      .build();
+
+    TableMetadataUnit nationFile2 = nationTable.toBuilder()
+      .metadataType(MetadataType.FILE.name())
+      .metadataKey("part_int=3")
+      .location("/tmp/nation/part_int=3/part_varchar=g")
+      .path("/tmp/nation/part_int=3/part_varchar=g/0_0_1.parquet")
+      .lastModifiedTime(System.currentTimeMillis())
+      .lastModifiedTime(2L)
+      .build();
+
+    TableMetadataUnit nationFile3 = nationTable.toBuilder()
+      .metadataType(MetadataType.FILE.name())
+      .metadataKey("part_int=4")
+      .location("/tmp/nation/part_int=4/part_varchar=g")
+      .path("/tmp/nation/part_int=4/part_varchar=g/0_0_0.parquet")
+      .lastModifiedTime(3L)
+      .build();
+
+    TableMetadataUnit nationRowGroup1 = nationTable.toBuilder()
+      .metadataType(MetadataType.ROW_GROUP.name())
+      .metadataKey("part_int=3")
+      .location("/tmp/nation/part_int=3/part_varchar=g")
+      .path("/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet")
+      .rowGroupIndex(1)
+      .build();
+
+    TableMetadataUnit nationRowGroup2 = nationTable.toBuilder()
+      .metadataType(MetadataType.ROW_GROUP.name())
+      .metadataKey("part_int=3")
+      .location("/tmp/nation/part_int=3/part_varchar=g")
+      .path("/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet")
+      .rowGroupIndex(2)
+      .build();
+
+    TableMetadataUnit nationRowGroup3 = nationTable.toBuilder()
+      .metadataType(MetadataType.ROW_GROUP.name())
+      .metadataKey("part_int=4")
+      .location("/tmp/nation/part_int=4/part_varchar=g")
+      .path("/tmp/nation/part_int=4/part_varchar=g/0_0_0.parquet")
+      .rowGroupIndex(1)
+      .build();
+
+    TableMetadataUnit nationRowGroup4 = nationTable.toBuilder()
+      .metadataType(MetadataType.ROW_GROUP.name())
+      .metadataKey("part_int=4")
+      .location("/tmp/nation/part_int=4/part_varchar=g")
+      .path("/tmp/nation/part_int=4/part_varchar=g/0_0_0.parquet")
+      .rowGroupIndex(2)
+      .build();
+
+    TableMetadataUnit regionTable = basicUnit().toBuilder()
+      .tableName("region")
+      .metadataType(MetadataType.TABLE.name())
+      .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+      .build();
+
+    tables.modify()
+      .overwrite(nationTable,
+        nationSegment1, nationSegment2, nationSegment3,
+        nationPartition1, nationPartition2, nationPartition3,
+        nationFile1, nationFile2, nationFile3,
+        nationRowGroup1, nationRowGroup2, nationRowGroup3, nationRowGroup4,
+        regionTable)
+      .execute();
+  }
+
+  /**
+   * Returns metadata unit where all fields are filled in.
+   * Note: data in the fields may be not exactly true to reality.
+   *
+   * @return basic metadata unit
+   */
+  private static TableMetadataUnit basicUnit() {
+    return TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("test")
+      .owner("user")
+      .tableType("parquet")
+      .metadataType(MetadataType.NONE.name())
+      .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+      .location("/tmp/nation")
+      .interestingColumns(Arrays.asList("`id`", "`name`"))
+      .schema("{\"type\":\"tuple_schema\"," +
+        "\"columns\":[{\"name\":\"id\",\"type\":\"INT\",\"mode\":\"REQUIRED\"}," +
+        "{\"name\":\"name\",\"type\":\"VARCHAR\",\"mode\":\"REQUIRED\"}]," +
+        "\"properties\":{\"drill.strict\":\"true\"}}\n")
+      .columnsStatistics(Collections.singletonMap("`name`", "{\"statistics\":[{\"statisticsValue\":\"aaa\"," +
+        "\"statisticsKind\":{\"exact\":true,\"name\":\"minValue\"}},{\"statisticsValue\":\"zzz\"," +
+        "\"statisticsKind\":{\"exact\":true,\"name\":\"maxValue\"}}],\"type\":\"VARCHAR\"}"))
+      .metadataStatistics(Collections.singletonList("{\"statisticsValue\":2.1," +
+        "\"statisticsKind\":{\"name\":\"approx_count_distinct\"}}"))
+      .lastModifiedTime(System.currentTimeMillis())
+      .partitionKeys(Collections.singletonMap("dir0", "2018"))
+      .additionalMetadata("test table metadata")
+      .metadataIdentifier("part_int=3/part_varchar=g/0_0_0.parquet")
+      .column("`id`")
+      .locations(Arrays.asList("/tmp/nation/1", "/tmp/nation/2"))
+      .partitionValues(Arrays.asList("1", "2"))
+      .path("/tmp/nation/1/0_0_0.parquet")
+      .rowGroupIndex(0)
+      .hostAffinity(Collections.singletonMap("host1", 0.1F))
+      .build();
+  }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestIcebergTablesMetastore.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestIcebergTablesMetastore.java
new file mode 100644
index 0000000..cf69805
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestIcebergTablesMetastore.java
@@ -0,0 +1,486 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import com.typesafe.config.ConfigValueFactory;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.metastore.components.tables.Tables;
+import org.apache.drill.metastore.operate.Metadata;
+import org.apache.drill.metastore.Metastore;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.iceberg.IcebergBaseTest;
+import org.apache.drill.metastore.iceberg.IcebergMetastore;
+import org.apache.drill.metastore.iceberg.config.IcebergConfigConstants;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.apache.iceberg.TableProperties;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+
+public class TestIcebergTablesMetastore extends IcebergBaseTest {
+
+  private static final String COMPONENTS_COMMON_PROPERTIES_PATTERN = IcebergConfigConstants.COMPONENTS_COMMON_PROPERTIES + ".%s";
+  private static final String COMPONENTS_TABLES_PROPERTIES_PATTERN = IcebergConfigConstants.COMPONENTS_TABLES_PROPERTIES + ".%s";
+
+  @Rule
+  public TemporaryFolder baseLocation = new TemporaryFolder();
+
+  @Test
+  public void testCreationWithoutProperties() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+
+    Metastore metastore = new IcebergMetastore(config);
+    assertTrue(metastore.tables().metadata().properties().isEmpty());
+  }
+
+  @Test
+  public void testCreationWithCommonProperties() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot())
+      .withValue(String.format(COMPONENTS_COMMON_PROPERTIES_PATTERN, TableProperties.SPLIT_SIZE),
+        ConfigValueFactory.fromAnyRef(10))
+      .withValue(String.format(COMPONENTS_COMMON_PROPERTIES_PATTERN, TableProperties.MANIFEST_MIN_MERGE_COUNT),
+        ConfigValueFactory.fromAnyRef(2)));
+
+    Metastore metastore = new IcebergMetastore(config);
+    Map<String, String> expected = new HashMap<>();
+    expected.put(TableProperties.SPLIT_SIZE, "10");
+    expected.put(TableProperties.MANIFEST_MIN_MERGE_COUNT, "2");
+    assertEquals(expected, metastore.tables().metadata().properties());
+  }
+
+  @Test
+  public void testCreationWithCommonAndComponentProperties() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot())
+      .withValue(String.format(COMPONENTS_COMMON_PROPERTIES_PATTERN, TableProperties.SPLIT_SIZE),
+        ConfigValueFactory.fromAnyRef(10))
+      .withValue(String.format(COMPONENTS_TABLES_PROPERTIES_PATTERN, TableProperties.MANIFEST_MIN_MERGE_COUNT),
+        ConfigValueFactory.fromAnyRef(2)));
+
+    Metastore metastore = new IcebergMetastore(config);
+    Map<String, String> expected = new HashMap<>();
+    expected.put(TableProperties.SPLIT_SIZE, "10");
+    expected.put(TableProperties.MANIFEST_MIN_MERGE_COUNT, "2");
+    assertEquals(expected, metastore.tables().metadata().properties());
+  }
+
+  @Test
+  public void testCreationWithComponentPropertiesPrecedence() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot())
+      .withValue(String.format(COMPONENTS_COMMON_PROPERTIES_PATTERN, TableProperties.SPLIT_SIZE),
+        ConfigValueFactory.fromAnyRef(10))
+      .withValue(String.format(COMPONENTS_TABLES_PROPERTIES_PATTERN, TableProperties.SPLIT_SIZE),
+        ConfigValueFactory.fromAnyRef(100)));
+
+    Metastore metastore = new IcebergMetastore(config);
+    assertEquals(Collections.singletonMap(TableProperties.SPLIT_SIZE, "100"),
+      metastore.tables().metadata().properties());
+  }
+
+  @Test
+  public void testLoadWithoutProperties() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+
+    Metastore initialMetastore = new IcebergMetastore(config);
+    assertTrue(initialMetastore.tables().metadata().properties().isEmpty());
+
+    Metastore newMetastore = new IcebergMetastore(config);
+    assertTrue(newMetastore.tables().metadata().properties().isEmpty());
+  }
+
+  @Test
+  public void testLoadWithSameProperties() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot())
+      .withValue(String.format(COMPONENTS_COMMON_PROPERTIES_PATTERN, TableProperties.SPLIT_SIZE),
+        ConfigValueFactory.fromAnyRef(10)));
+
+    Map<String, String> initialProperties = Collections.singletonMap(TableProperties.SPLIT_SIZE, "10");
+
+    Metastore initialMetastore = new IcebergMetastore(config);
+    assertEquals(initialProperties, initialMetastore.tables().metadata().properties());
+
+    Metastore newMetastore = new IcebergMetastore(config);
+    assertEquals(initialProperties, newMetastore.tables().metadata().properties());
+  }
+
+  @Test
+  public void testLoadWithUpdatedProperties() {
+    DrillConfig initialConfig = new DrillConfig(baseIcebergConfig(baseLocation.getRoot())
+      .withValue(String.format(COMPONENTS_COMMON_PROPERTIES_PATTERN, TableProperties.SPLIT_SIZE),
+        ConfigValueFactory.fromAnyRef(10))
+      .withValue(String.format(COMPONENTS_TABLES_PROPERTIES_PATTERN, TableProperties.MANIFEST_MIN_MERGE_COUNT),
+        ConfigValueFactory.fromAnyRef(2)));
+
+    Map<String, String> initialProperties = new HashMap<>();
+    initialProperties.put(TableProperties.SPLIT_SIZE, "10");
+    initialProperties.put(TableProperties.MANIFEST_MIN_MERGE_COUNT, "2");
+
+    Metastore initialMetastore = new IcebergMetastore(initialConfig);
+    assertEquals(initialProperties, initialMetastore.tables().metadata().properties());
+
+    DrillConfig newConfig = new DrillConfig(baseIcebergConfig(baseLocation.getRoot())
+      .withValue(String.format(COMPONENTS_COMMON_PROPERTIES_PATTERN, TableProperties.SPLIT_SIZE),
+        ConfigValueFactory.fromAnyRef(100))
+      .withValue(String.format(COMPONENTS_TABLES_PROPERTIES_PATTERN, TableProperties.COMMIT_NUM_RETRIES),
+        ConfigValueFactory.fromAnyRef(5)));
+
+    Map<String, String> newProperties = new HashMap<>();
+    newProperties.put(TableProperties.SPLIT_SIZE, "100");
+    newProperties.put(TableProperties.COMMIT_NUM_RETRIES, "5");
+
+    Metastore newMetastore = new IcebergMetastore(newConfig);
+    assertEquals(newProperties, newMetastore.tables().metadata().properties());
+  }
+
+  @Test
+  public void testNewInstance() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+    Metastore metastore = new IcebergMetastore(config);
+
+    assertNotSame(metastore.tables(), metastore.tables());
+  }
+
+  @Test
+  public void testVersionInitial() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+    Metastore metastore = new IcebergMetastore(config);
+    Metadata metadata = metastore.tables().metadata();
+    assertTrue(metadata.supportsVersioning());
+    assertEquals(0, metadata.version());
+  }
+
+  @Test
+  public void testVersionUpdate() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+
+    Tables tables = new IcebergMetastore(config).tables();
+    Metadata metadata = tables.metadata();
+
+    assertTrue(metadata.supportsVersioning());
+    assertEquals(0, metadata.version());
+
+    tables.modify()
+      .overwrite(TableMetadataUnit.builder()
+        .storagePlugin("dfs")
+        .workspace("tmp")
+        .tableName("nation")
+        .metadataKey("dir0")
+        .build())
+      .execute();
+
+    assertNotEquals(0, metadata.version());
+  }
+
+  @Test
+  public void testWriteReadAllFieldTypes() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+    Tables tables = new IcebergMetastore(config).tables();
+
+    Map<String, String> columnStatistics = new HashMap<>();
+    columnStatistics.put("stat1", "val1");
+    columnStatistics.put("stat2", "val2");
+
+    TableInfo tableInfo = TableInfo.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .name("nation")
+      .build();
+
+    TableMetadataUnit unit = TableMetadataUnit.builder()
+      .storagePlugin(tableInfo.storagePlugin())
+      .workspace(tableInfo.workspace())
+      .tableName(tableInfo.name())
+      .metadataKey("dir0")
+      .rowGroupIndex(1)
+      .lastModifiedTime(System.currentTimeMillis())
+      .partitionValues(Collections.emptyList())
+      .locations(Arrays.asList("a", "b", "c"))
+      .hostAffinity(Collections.emptyMap())
+      .columnsStatistics(columnStatistics)
+      .build();
+
+    tables.modify()
+      .overwrite(unit)
+      .execute();
+
+    List<TableMetadataUnit> units = tables.read()
+      .filter(tableInfo.toFilter())
+      .execute();
+
+    assertEquals(1, units.size());
+    assertEquals(unit, units.get(0));
+  }
+
+  @Test
+  public void testReadSelectedColumns() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+    Tables tables = new IcebergMetastore(config).tables();
+
+    TableInfo tableInfo = TableInfo.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .name("nation")
+      .build();
+
+    TableMetadataUnit unit = TableMetadataUnit.builder()
+      .storagePlugin(tableInfo.storagePlugin())
+      .workspace(tableInfo.workspace())
+      .tableName(tableInfo.name())
+      .metadataKey("dir0")
+      .build();
+
+    tables.modify()
+      .overwrite(unit)
+      .execute();
+
+    List<TableMetadataUnit> units = tables.read()
+      .filter(tableInfo.toFilter())
+      .columns("tableName", "metadataKey")
+      .execute();
+
+    assertEquals(1, units.size());
+    assertEquals(TableMetadataUnit.builder().tableName("nation").metadataKey("dir0").build(), units.get(0));
+  }
+
+  @Test
+  public void testReadNoResult() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+    Tables tables = new IcebergMetastore(config).tables();
+
+    List<TableMetadataUnit> units = tables.read()
+      .filter(FilterExpression.equal("storagePlugin", "dfs"))
+      .columns("tableName", "metadataKey")
+      .execute();
+
+    assertTrue(units.isEmpty());
+  }
+
+  @Test
+  public void testOverwrite() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+    Tables tables = new IcebergMetastore(config).tables();
+
+    TableInfo tableInfo = TableInfo.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .name("nation")
+      .build();
+
+    TableMetadataUnit initialUnit = TableMetadataUnit.builder()
+      .storagePlugin(tableInfo.storagePlugin())
+      .workspace(tableInfo.workspace())
+      .tableName(tableInfo.name())
+      .metadataKey("dir0")
+      .tableType("parquet")
+      .build();
+
+    tables.modify()
+      .overwrite(initialUnit)
+      .execute();
+
+    List<TableMetadataUnit> units = tables.read()
+      .filter(tableInfo.toFilter())
+      .execute();
+
+    assertEquals(1, units.size());
+    assertEquals(initialUnit, units.get(0));
+
+    TableMetadataUnit updatedUnit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey("dir0")
+      .tableType("text")
+      .build();
+
+    tables.modify()
+      .overwrite(updatedUnit)
+      .execute();
+
+    List<TableMetadataUnit> updatedUnits = tables.read()
+      .filter(tableInfo.toFilter())
+      .execute();
+
+    assertEquals(1, updatedUnits.size());
+    assertEquals(updatedUnit, updatedUnits.get(0));
+  }
+
+  @Test
+  public void testDelete() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+    Tables tables = new IcebergMetastore(config).tables();
+
+    TableInfo tableInfo = TableInfo.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .name("nation")
+      .build();
+
+    TableMetadataUnit firstUnit = TableMetadataUnit.builder()
+      .storagePlugin(tableInfo.storagePlugin())
+      .workspace(tableInfo.workspace())
+      .tableName(tableInfo.name())
+      .metadataKey("dir0")
+      .build();
+
+    TableMetadataUnit secondUnit = TableMetadataUnit.builder()
+      .storagePlugin(tableInfo.storagePlugin())
+      .workspace(tableInfo.workspace())
+      .tableName(tableInfo.name())
+      .metadataKey("dir1")
+      .build();
+
+    tables.modify()
+      .overwrite(firstUnit, secondUnit)
+      .execute();
+
+    List<TableMetadataUnit> units = tables.read()
+      .filter(tableInfo.toFilter())
+      .execute();
+
+    assertEquals(2, units.size());
+
+    FilterExpression deleteFilter = FilterExpression.and(
+      tableInfo.toFilter(),
+      FilterExpression.equal("metadataKey", "dir0"));
+
+    tables.modify()
+      .delete(deleteFilter)
+      .execute();
+
+    List<TableMetadataUnit> updatedUnits = tables.read()
+      .filter(tableInfo.toFilter())
+      .execute();
+
+    assertEquals(1, updatedUnits.size());
+    assertEquals(secondUnit, updatedUnits.get(0));
+  }
+
+  @Test
+  public void testOverwriteAndDeleteInOneTransaction() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+    Tables tables = new IcebergMetastore(config).tables();
+
+    TableInfo tableInfo = TableInfo.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .name("nation")
+      .build();
+
+    TableMetadataUnit firstUnit = TableMetadataUnit.builder()
+      .storagePlugin(tableInfo.storagePlugin())
+      .workspace(tableInfo.workspace())
+      .tableName(tableInfo.name())
+      .metadataKey("dir0")
+      .tableType("parquet")
+      .build();
+
+    TableMetadataUnit secondUnit = TableMetadataUnit.builder()
+      .storagePlugin(tableInfo.storagePlugin())
+      .workspace(tableInfo.workspace())
+      .tableName(tableInfo.name())
+      .metadataKey("dir1")
+      .tableType("parquet")
+      .build();
+
+    tables.modify()
+      .overwrite(firstUnit, secondUnit)
+      .execute();
+
+    List<TableMetadataUnit> units = tables.read()
+      .filter(tableInfo.toFilter())
+      .execute();
+
+    assertEquals(2, units.size());
+
+    FilterExpression deleteFilter = FilterExpression.and(
+      tableInfo.toFilter(),
+      FilterExpression.equal("metadataKey", "dir0"));
+
+    TableMetadataUnit updatedUnit = TableMetadataUnit.builder()
+      .storagePlugin(tableInfo.storagePlugin())
+      .workspace(tableInfo.workspace())
+      .tableName(tableInfo.name())
+      .metadataKey("dir1")
+      .tableType("text")
+      .build();
+
+    tables.modify()
+      .delete(deleteFilter)
+      .overwrite(updatedUnit)
+      .execute();
+
+    List<TableMetadataUnit> updatedUnits = tables.read()
+      .filter(tableInfo.toFilter())
+      .execute();
+
+    assertEquals(1, updatedUnits.size());
+    assertEquals(updatedUnit, updatedUnits.get(0));
+  }
+
+  @Test
+  public void testPurge() {
+    DrillConfig config = new DrillConfig(baseIcebergConfig(baseLocation.getRoot()));
+    Tables tables = new IcebergMetastore(config).tables();
+
+    TableMetadataUnit firstUnit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey("dir0")
+      .tableType("parquet")
+      .build();
+
+    TableMetadataUnit secondUnit = TableMetadataUnit.builder()
+      .storagePlugin("s3")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey("dir0")
+      .tableType("parquet")
+      .build();
+
+    tables.modify()
+      .overwrite(firstUnit, secondUnit)
+      .execute();
+
+    List<TableMetadataUnit> initialUnits = tables.read()
+      .execute();
+
+    assertEquals(2, initialUnits.size());
+
+    tables.modify()
+      .purge()
+      .execute();
+
+    List<TableMetadataUnit> resultingUnits = tables.read()
+      .execute();
+
+    assertTrue(resultingUnits.isEmpty());
+  }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTableKey.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTableKey.java
new file mode 100644
index 0000000..c84a3b9
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTableKey.java
@@ -0,0 +1,68 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.iceberg.IcebergBaseTest;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestTableKey extends IcebergBaseTest {
+
+  @Test
+  public void testCreation() {
+    TableMetadataUnit unit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .build();
+
+    TableKey expected = new TableKey("dfs", "tmp", "nation");
+
+    assertEquals(expected, TableKey.of(unit));
+  }
+
+  @Test
+  public void testToLocation() {
+    TableKey tableKey = new TableKey("dfs", "tmp", "nation");
+
+    String expected = new Path(
+      Paths.get("/metastore", "dfs", "tmp", "nation").toUri().getPath())
+      .toUri().getPath();
+
+    assertEquals(expected, tableKey.toLocation("/metastore"));
+  }
+
+  @Test
+  public void testToFilterConditions() {
+    TableKey tableKey = new TableKey("dfs", "tmp", "nation");
+
+    Map<String, Object> expected = new HashMap<>();
+    expected.put(IcebergTables.STORAGE_PLUGIN, "dfs");
+    expected.put(IcebergTables.WORKSPACE, "tmp");
+    expected.put(IcebergTables.TABLE_NAME, "nation");
+
+    assertEquals(expected, tableKey.toFilterConditions());
+  }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTablesInputDataTransformer.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTablesInputDataTransformer.java
new file mode 100644
index 0000000..a33516a
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTablesInputDataTransformer.java
@@ -0,0 +1,209 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.iceberg.IcebergBaseTest;
+import org.apache.drill.metastore.iceberg.exceptions.IcebergMetastoreException;
+import org.apache.drill.metastore.iceberg.transform.InputDataTransformer;
+import org.apache.drill.metastore.iceberg.transform.WriteData;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.lang.invoke.MethodHandle;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestTablesInputDataTransformer extends IcebergBaseTest {
+
+  private static Schema metastoreSchema;
+  private static Schema partitionSchema;
+  private static Map<String, MethodHandle> unitGetters;
+
+  @BeforeClass
+  public static void init() {
+    metastoreSchema = IcebergTables.SCHEMA.tableSchema();
+    partitionSchema = new Schema(IcebergTables.SCHEMA.partitionSpec().partitionType().fields());
+    unitGetters = TableMetadataUnit.SCHEMA.unitGetters();
+  }
+
+  @Test
+  public void testNoData() {
+    WriteData writeData = new InputDataTransformer<TableMetadataUnit>(metastoreSchema, partitionSchema, unitGetters)
+      .units(Collections.emptyList())
+      .execute();
+
+    assertEquals(Collections.emptyList(), writeData.records());
+    assertNull(writeData.partition());
+  }
+
+  @Test
+  public void testValidDataOneRecord() {
+    Map<String, String> partitionKeys = new HashMap<>();
+    partitionKeys.put("dir0", "2018");
+    partitionKeys.put("dir1", "2019");
+    List<String> partitionValues = Arrays.asList("a", "b", "c");
+    Long lastModifiedTime = System.currentTimeMillis();
+
+    TableMetadataUnit unit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+      .partitionKeys(partitionKeys)
+      .partitionValues(partitionValues)
+      .lastModifiedTime(lastModifiedTime)
+      .build();
+
+    WriteData writeData = new InputDataTransformer<TableMetadataUnit>(metastoreSchema, partitionSchema, unitGetters)
+      .units(Collections.singletonList(unit))
+      .execute();
+
+    Record tableRecord = GenericRecord.create(metastoreSchema);
+    tableRecord.setField("storagePlugin", "dfs");
+    tableRecord.setField("workspace", "tmp");
+    tableRecord.setField("tableName", "nation");
+    tableRecord.setField("metadataKey", MetadataInfo.GENERAL_INFO_KEY);
+    tableRecord.setField("partitionKeys", partitionKeys);
+    tableRecord.setField("partitionValues", partitionValues);
+    tableRecord.setField("lastModifiedTime", lastModifiedTime);
+
+    Record partitionRecord = GenericRecord.create(partitionSchema);
+    partitionRecord.setField("storagePlugin", "dfs");
+    partitionRecord.setField("workspace", "tmp");
+    partitionRecord.setField("tableName", "nation");
+    partitionRecord.setField("metadataKey", MetadataInfo.GENERAL_INFO_KEY);
+
+    assertEquals(Collections.singletonList(tableRecord), writeData.records());
+    assertEquals(partitionRecord, writeData.partition());
+  }
+
+  @Test
+  public void testValidDataSeveralRecords() {
+    List<TableMetadataUnit> units = Arrays.asList(
+      TableMetadataUnit.builder()
+        .storagePlugin("dfs")
+        .workspace("tmp")
+        .tableName("nation")
+        .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+        .column("a")
+        .build(),
+      TableMetadataUnit.builder()
+        .storagePlugin("dfs")
+        .workspace("tmp")
+        .tableName("nation")
+        .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+        .column("b")
+        .build(),
+      TableMetadataUnit.builder()
+        .storagePlugin("dfs")
+        .workspace("tmp")
+        .tableName("nation")
+        .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+        .column("c")
+        .build());
+
+    WriteData writeData = new InputDataTransformer<TableMetadataUnit>(metastoreSchema, partitionSchema, unitGetters)
+      .units(units)
+      .execute();
+
+    Record tableRecord1 = GenericRecord.create(metastoreSchema);
+    tableRecord1.setField("storagePlugin", "dfs");
+    tableRecord1.setField("workspace", "tmp");
+    tableRecord1.setField("tableName", "nation");
+    tableRecord1.setField("metadataKey", MetadataInfo.GENERAL_INFO_KEY);
+    tableRecord1.setField("column", "a");
+
+    Record tableRecord2 = GenericRecord.create(metastoreSchema);
+    tableRecord2.setField("storagePlugin", "dfs");
+    tableRecord2.setField("workspace", "tmp");
+    tableRecord2.setField("tableName", "nation");
+    tableRecord2.setField("metadataKey", MetadataInfo.GENERAL_INFO_KEY);
+    tableRecord2.setField("column", "b");
+
+    Record tableRecord3 = GenericRecord.create(metastoreSchema);
+    tableRecord3.setField("storagePlugin", "dfs");
+    tableRecord3.setField("workspace", "tmp");
+    tableRecord3.setField("tableName", "nation");
+    tableRecord3.setField("metadataKey", MetadataInfo.GENERAL_INFO_KEY);
+    tableRecord3.setField("column", "c");
+
+    Record partitionRecord = GenericRecord.create(partitionSchema);
+    partitionRecord.setField("storagePlugin", "dfs");
+    partitionRecord.setField("workspace", "tmp");
+    partitionRecord.setField("tableName", "nation");
+    partitionRecord.setField("metadataKey", MetadataInfo.GENERAL_INFO_KEY);
+
+    assertEquals(Arrays.asList(tableRecord1, tableRecord2, tableRecord3), writeData.records());
+    assertEquals(partitionRecord, writeData.partition());
+  }
+
+  @Test
+  public void testInvalidPartition() {
+    TableMetadataUnit unit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .build();
+
+    thrown.expect(IcebergMetastoreException.class);
+
+    new InputDataTransformer<TableMetadataUnit>(metastoreSchema, partitionSchema, unitGetters)
+      .units(Collections.singletonList(unit))
+      .execute();
+  }
+
+  @Test
+  public void testNonMatchingPartitionKey() {
+    List<TableMetadataUnit> units = Arrays.asList(
+      TableMetadataUnit.builder()
+        .storagePlugin("dfs")
+        .workspace("tmp")
+        .tableName("a")
+        .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+        .build(),
+      TableMetadataUnit.builder()
+        .storagePlugin("dfs")
+        .workspace("tmp")
+        .tableName("b")
+        .metadataKey(MetadataInfo.DEFAULT_SEGMENT_KEY)
+        .build(),
+      TableMetadataUnit.builder()
+        .storagePlugin("dfs")
+        .workspace("tmp")
+        .tableName("c")
+        .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+        .build());
+
+    thrown.expect(IcebergMetastoreException.class);
+
+    new InputDataTransformer<TableMetadataUnit>(metastoreSchema, partitionSchema, unitGetters)
+      .units(units)
+      .execute();
+  }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTablesOperationTransformer.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTablesOperationTransformer.java
new file mode 100644
index 0000000..d7b72eb
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTablesOperationTransformer.java
@@ -0,0 +1,116 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.iceberg.IcebergBaseTest;
+import org.apache.drill.metastore.iceberg.IcebergMetastore;
+import org.apache.drill.metastore.iceberg.operate.Delete;
+import org.apache.drill.metastore.iceberg.operate.Overwrite;
+import org.apache.drill.metastore.iceberg.transform.FilterTransformer;
+import org.apache.drill.metastore.iceberg.transform.OperationTransformer;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestTablesOperationTransformer extends IcebergBaseTest {
+
+  private static String location;
+  private static OperationTransformer<TableMetadataUnit> transformer;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    location = new Path(defaultFolder.newFolder(TestTablesOperationTransformer.class.getSimpleName()).toURI().getPath()).toUri().getPath();
+    IcebergMetastore metastore = new IcebergMetastore(new DrillConfig(baseIcebergConfig(new File(location))));
+    transformer = new TablesOperationTransformer((IcebergTables) metastore.tables());
+  }
+
+  @Test
+  public void testToOverwriteOperation() {
+    TableMetadataUnit unit = TableMetadataUnit.builder()
+      .storagePlugin("dfs").workspace("tmp").tableName("nation").metadataKey("dir0").build();
+
+    TableKey tableKey = new TableKey(unit.storagePlugin(), unit.workspace(), unit.tableName());
+
+    Map<String, Object> filterConditions = new HashMap<>(tableKey.toFilterConditions());
+    filterConditions.put(IcebergTables.METADATA_KEY, unit.metadataKey());
+
+    String location = tableKey.toLocation(TestTablesOperationTransformer.location);
+    Expression expression = new FilterTransformer().transform(filterConditions);
+    Overwrite operation = transformer.toOverwrite(location, expression, Collections.singletonList(unit));
+
+    assertEquals(expression.toString(), operation.filter().toString());
+
+    File file = new File(new Path(String.valueOf(operation.dataFile().path())).toUri().getPath());
+    assertTrue(file.exists());
+    assertEquals(location, file.getParent());
+  }
+
+  @Test
+  public void testToOverwriteOperations() {
+    List<TableMetadataUnit> units = Arrays.asList(
+      TableMetadataUnit.builder().storagePlugin("dfs").workspace("tmp").tableName("nation").metadataKey("dir0").build(),
+      TableMetadataUnit.builder().storagePlugin("dfs").workspace("tmp").tableName("nation").metadataKey("dir0").build(),
+      TableMetadataUnit.builder().storagePlugin("dfs").workspace("tmp").tableName("nation").metadataKey("dir2").build(),
+      TableMetadataUnit.builder().storagePlugin("dfs").workspace("tmp").tableName("nation").metadataKey("dir2").build(),
+      TableMetadataUnit.builder().storagePlugin("dfs").workspace("tmp").tableName("region").metadataKey("dir0").build(),
+      TableMetadataUnit.builder().storagePlugin("s3").workspace("tmp").tableName("region").metadataKey("dir0").build());
+
+    List<Overwrite> operations = transformer.toOverwrite(units);
+    assertEquals(4, operations.size());
+  }
+
+  @Test
+  public void testToDeleteOperation() {
+    FilterExpression filter = FilterExpression.and(
+      FilterExpression.equal("storagePlugin", "dfs"),
+      FilterExpression.equal("workspace", "tmp"));
+
+    Expression expected = Expressions.and(
+      Expressions.equal(IcebergTables.STORAGE_PLUGIN, "dfs"),
+      Expressions.equal(IcebergTables.WORKSPACE, "tmp"));
+
+    Delete operation = transformer.toDelete(filter);
+
+    assertEquals(expected.toString(), operation.filter().toString());
+  }
+
+  @Test
+  public void testToDeleteOperations() {
+    FilterExpression dfs = FilterExpression.equal("storagePlugin", "dfs");
+    FilterExpression s3 = FilterExpression.equal("storagePlugin", "s3");
+
+    List<Delete> operations = transformer.toDelete(Arrays.asList(dfs, s3));
+
+    assertEquals(2, operations.size());
+  }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTablesOutputDataTransformer.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTablesOutputDataTransformer.java
new file mode 100644
index 0000000..40215f7
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/components/tables/TestTablesOutputDataTransformer.java
@@ -0,0 +1,131 @@
+/*
+ * 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.metastore.iceberg.components.tables;
+
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.iceberg.IcebergBaseTest;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.lang.invoke.MethodHandle;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestTablesOutputDataTransformer extends IcebergBaseTest {
+
+  private static Map<String, MethodHandle> unitSetters;
+  private static Schema schema;
+
+  @BeforeClass
+  public static void init() {
+    unitSetters = TableMetadataUnit.SCHEMA.unitBuilderSetters();
+    schema = IcebergTables.SCHEMA.tableSchema();
+  }
+
+  @Test
+  public void testNoData() {
+    List<TableMetadataUnit> actualResult = new TablesOutputDataTransformer(unitSetters)
+      .columns("storagePlugin", "workspace", "tableName")
+      .records(Collections.emptyList())
+      .execute();
+
+    assertEquals(Collections.emptyList(), actualResult);
+  }
+
+  @Test
+  public void testValidDataOneRecord() {
+    Map<String, String> partitionKeys = new HashMap<>();
+    partitionKeys.put("dir0", "2018");
+    partitionKeys.put("dir1", "2019");
+    List<String> partitionValues = Arrays.asList("a", "b", "c");
+    Long lastModifiedTime = System.currentTimeMillis();
+
+    Record record = GenericRecord.create(schema);
+    record.setField("storagePlugin", "dfs");
+    record.setField("workspace", "tmp");
+    record.setField("tableName", "nation");
+    record.setField("partitionKeys", partitionKeys);
+    record.setField("partitionValues", partitionValues);
+    record.setField("lastModifiedTime", lastModifiedTime);
+
+    List<TableMetadataUnit> actualResult = new TablesOutputDataTransformer(unitSetters)
+      .columns(Arrays.asList("storagePlugin", "workspace", "tableName",
+        "partitionKeys", "partitionValues", "lastModifiedTime"))
+      .records(Collections.singletonList(record))
+      .execute();
+
+    List<TableMetadataUnit> expectedResult = Collections.singletonList(TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .partitionKeys(partitionKeys)
+      .partitionValues(partitionValues)
+      .lastModifiedTime(lastModifiedTime)
+      .build());
+
+    assertEquals(expectedResult, actualResult);
+  }
+
+  @Test
+  public void testValidDataSeveralRecords() {
+    Record record1 = GenericRecord.create(schema);
+    record1.setField("tableName", "a");
+
+    Record record2 = GenericRecord.create(schema);
+    record2.setField("tableName", "b");
+
+    Record record3 = GenericRecord.create(schema);
+    record3.setField("tableName", "c");
+
+
+    List<TableMetadataUnit> actualResult = new TablesOutputDataTransformer(unitSetters)
+      .columns(Collections.singletonList("tableName"))
+      .records(Arrays.asList(record1, record2, record3))
+      .execute();
+
+    List<TableMetadataUnit> expectedResult = Arrays.asList(
+      TableMetadataUnit.builder().tableName("a").build(),
+      TableMetadataUnit.builder().tableName("b").build(),
+      TableMetadataUnit.builder().tableName("c").build());
+
+    assertEquals(expectedResult, actualResult);
+  }
+
+  @Test
+  public void testInvalidColumns() {
+    Record record = GenericRecord.create(schema);
+    record.setField("tableName", "a");
+
+    List<TableMetadataUnit> actualResult = new TablesOutputDataTransformer(unitSetters)
+      .records(Collections.singletonList(record))
+      .columns(Arrays.asList("a", "b"))
+      .execute();
+
+    List<TableMetadataUnit> expectedResult = Collections.singletonList(TableMetadataUnit.builder().build());
+
+    assertEquals(expectedResult, actualResult);
+  }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/schema/TestIcebergTableSchema.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/schema/TestIcebergTableSchema.java
new file mode 100644
index 0000000..c95b9a0
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/schema/TestIcebergTableSchema.java
@@ -0,0 +1,300 @@
+/*
+ * 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.metastore.iceberg.schema;
+
+import com.sun.codemodel.CodeWriter;
+import com.sun.codemodel.JAnnotationArrayMember;
+import com.sun.codemodel.JAnnotationUse;
+import com.sun.codemodel.JClass;
+import com.sun.codemodel.JClassAlreadyExistsException;
+import com.sun.codemodel.JCodeModel;
+import com.sun.codemodel.JDefinedClass;
+import com.sun.codemodel.JFieldVar;
+import com.sun.codemodel.JMod;
+import com.sun.codemodel.JPackage;
+import net.openhft.compiler.CompilerUtils;
+import org.apache.drill.metastore.MetastoreFieldDefinition;
+import org.apache.drill.metastore.iceberg.IcebergBaseTest;
+import org.apache.drill.metastore.iceberg.exceptions.IcebergMetastoreException;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Types;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class TestIcebergTableSchema extends IcebergBaseTest {
+
+  @Test
+  public void testAllTypes() throws Exception {
+    Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "AllTypes") {
+
+      @Override
+      void addFields(JDefinedClass jDefinedClass) {
+        JFieldVar stringField = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "stringField");
+        annotate(stringField);
+
+        JFieldVar intField = jDefinedClass.field(DEFAULT_FIELD_MODE, int.class, "intField");
+        annotate(intField);
+
+        JFieldVar integerField = jDefinedClass.field(DEFAULT_FIELD_MODE, Integer.class, "integerField");
+        annotate(integerField);
+
+        JFieldVar longField = jDefinedClass.field(DEFAULT_FIELD_MODE, Long.class, "longField");
+        annotate(longField);
+
+        JFieldVar floatField = jDefinedClass.field(DEFAULT_FIELD_MODE, Float.class, "floatField");
+        annotate(floatField);
+
+        JFieldVar doubleField = jDefinedClass.field(DEFAULT_FIELD_MODE, Double.class, "doubleField");
+        annotate(doubleField);
+
+        JFieldVar booleanField = jDefinedClass.field(DEFAULT_FIELD_MODE, Boolean.class, "booleanField");
+        annotate(booleanField);
+
+        JCodeModel jCodeModel = jDefinedClass.owner();
+
+        JClass listRef = jCodeModel.ref(List.class).narrow(String.class);
+        JFieldVar listField = jDefinedClass.field(DEFAULT_FIELD_MODE, listRef, "listField");
+        annotate(listField);
+
+        JClass mapRef = jCodeModel.ref(Map.class).narrow(String.class, Float.class);
+        JFieldVar mapField = jDefinedClass.field(DEFAULT_FIELD_MODE, mapRef, "mapField");
+        annotate(mapField);
+      }
+
+    }.generate();
+
+    IcebergTableSchema schema = IcebergTableSchema.of(clazz, Collections.emptyList());
+
+    int schemaIndex = IcebergTableSchema.STARTING_SCHEMA_INDEX;
+    int complexTypesIndex = IcebergTableSchema.STARTING_COMPLEX_TYPES_INDEX;
+
+    Schema expectedSchema = new Schema(
+      Types.NestedField.optional(schemaIndex++, "stringField", Types.StringType.get()),
+      Types.NestedField.optional(schemaIndex++, "intField", Types.IntegerType.get()),
+      Types.NestedField.optional(schemaIndex++, "integerField", Types.IntegerType.get()),
+      Types.NestedField.optional(schemaIndex++, "longField", Types.LongType.get()),
+      Types.NestedField.optional(schemaIndex++, "floatField", Types.FloatType.get()),
+      Types.NestedField.optional(schemaIndex++, "doubleField", Types.DoubleType.get()),
+      Types.NestedField.optional(schemaIndex++, "booleanField", Types.BooleanType.get()),
+      Types.NestedField.optional(schemaIndex++, "listField",
+        Types.ListType.ofOptional(complexTypesIndex++, Types.StringType.get())),
+      Types.NestedField.optional(schemaIndex, "mapField",
+        Types.MapType.ofOptional(complexTypesIndex++, complexTypesIndex, Types.StringType.get(), Types.FloatType.get())));
+
+    assertEquals(expectedSchema.asStruct(), schema.tableSchema().asStruct());
+  }
+
+  @Test
+  public void testIgnoreUnannotatedFields() throws Exception {
+    Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "IgnoreUnannotatedFields") {
+
+      @Override
+      void addFields(JDefinedClass jDefinedClass) {
+        JFieldVar stringField = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "stringField");
+        annotate(stringField);
+
+        jDefinedClass.field(DEFAULT_FIELD_MODE, Integer.class, "integerField");
+      }
+    }.generate();
+
+    IcebergTableSchema schema = IcebergTableSchema.of(clazz, Collections.emptyList());
+    assertNotNull(schema.tableSchema().findField("stringField"));
+    assertNull(schema.tableSchema().findField("integerField"));
+  }
+
+  @Test
+  public void testNestedComplexType() throws Exception {
+    Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "NestedComplexType") {
+
+      @Override
+      void addFields(JDefinedClass jDefinedClass) {
+        JCodeModel jCodeModel = jDefinedClass.owner();
+
+        JClass nestedListRef = jCodeModel.ref(List.class).narrow(String.class);
+        JClass listRef = jCodeModel.ref(List.class).narrow(nestedListRef);
+        JFieldVar listField = jDefinedClass.field(DEFAULT_FIELD_MODE, listRef, "listField");
+        annotate(listField);
+      }
+    }.generate();
+
+    thrown.expect(IcebergMetastoreException.class);
+
+    IcebergTableSchema.of(clazz, Collections.emptyList());
+  }
+
+  @Test
+  public void testUnpartitionedPartitionSpec() throws Exception {
+    Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "UnpartitionedPartitionSpec") {
+
+      @Override
+      void addFields(JDefinedClass jDefinedClass) {
+        JFieldVar stringField = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "stringField");
+        annotate(stringField);
+      }
+    }.generate();
+
+    IcebergTableSchema schema = IcebergTableSchema.of(clazz, Collections.emptyList());
+    assertNotNull(schema.tableSchema().findField("stringField"));
+
+    assertEquals(PartitionSpec.unpartitioned(), schema.partitionSpec());
+  }
+
+  @Test
+  public void testPartitionedPartitionSpec() throws Exception {
+    Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "PartitionedPartitionSpec") {
+
+      @Override
+      void addFields(JDefinedClass jDefinedClass) {
+        JFieldVar partKey1 = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "partKey1");
+        annotate(partKey1);
+
+        JFieldVar partKey2 = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "partKey2");
+        annotate(partKey2);
+
+        JFieldVar partKey3 = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "partKey3");
+        annotate(partKey3);
+
+        JFieldVar integerField = jDefinedClass.field(DEFAULT_FIELD_MODE, Integer.class, "integerField");
+        annotate(integerField);
+
+        JFieldVar booleanField = jDefinedClass.field(DEFAULT_FIELD_MODE, Boolean.class, "booleanField");
+        annotate(booleanField);
+      }
+    }.generate();
+
+    IcebergTableSchema schema = IcebergTableSchema.of(clazz, Arrays.asList("partKey1", "partKey2", "partKey3"));
+
+    Types.NestedField partKey1 = schema.tableSchema().findField("partKey1");
+    assertNotNull(partKey1);
+
+    Types.NestedField partKey2 = schema.tableSchema().findField("partKey2");
+    assertNotNull(partKey2);
+
+    Types.NestedField partKey3 = schema.tableSchema().findField("partKey3");
+    assertNotNull(partKey3);
+
+    assertNotNull(schema.tableSchema().findField("integerField"));
+    assertNotNull(schema.tableSchema().findField("booleanField"));
+
+    Schema partitionSchema = new Schema(partKey1, partKey2, partKey3);
+    PartitionSpec expectedPartitionSpec = PartitionSpec.builderFor(partitionSchema)
+      .identity(partKey1.name())
+      .identity(partKey2.name())
+      .identity(partKey3.name())
+      .build();
+
+    assertEquals(expectedPartitionSpec, schema.partitionSpec());
+  }
+
+  @Test
+  public void testUnMatchingPartitionSpec() throws Exception {
+    Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "UnMatchingPartitionSpec") {
+
+      @Override
+      void addFields(JDefinedClass jDefinedClass) {
+        JFieldVar partKey1 = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "partKey1");
+        annotate(partKey1);
+
+        JFieldVar integerField = jDefinedClass.field(DEFAULT_FIELD_MODE, Integer.class, "integerField");
+        annotate(integerField);
+      }
+    }.generate();
+
+    thrown.expect(IcebergMetastoreException.class);
+
+    IcebergTableSchema.of(clazz, Arrays.asList("partKey1", "partKey2"));
+  }
+
+  /**
+   * Generates and loads class at the runtime with specified fields.
+   * Fields may or may not be annotated.
+   */
+  private abstract class ClassGenerator {
+
+    final int DEFAULT_FIELD_MODE = JMod.PRIVATE;
+
+    private final String name;
+
+    ClassGenerator(String name) {
+      this.name = name;
+    }
+
+    Class<?> generate() throws JClassAlreadyExistsException, IOException, ClassNotFoundException {
+      JCodeModel jCodeModel = prepareModel();
+      ByteArrayStreamCodeWriter codeWriter = new ByteArrayStreamCodeWriter();
+      jCodeModel.build(codeWriter);
+
+      String sourceCode = codeWriter.sourceCode();
+      return CompilerUtils.CACHED_COMPILER.loadFromJava(name, sourceCode);
+    }
+
+    private JCodeModel prepareModel() throws JClassAlreadyExistsException {
+      JCodeModel jCodeModel = new JCodeModel();
+      JPackage jPackage = jCodeModel._package("");
+      JDefinedClass jDefinedClass = jPackage._class(name);
+      addFields(jDefinedClass);
+      return jCodeModel;
+    }
+
+    void annotate(JFieldVar field) {
+      annotate(field, MetadataType.ALL);
+    }
+
+    void annotate(JFieldVar field, MetadataType... scopes) {
+      JAnnotationUse annotate = field.annotate(MetastoreFieldDefinition.class);
+      assert scopes.length != 0;
+      JAnnotationArrayMember scopesParam = annotate.paramArray("scopes");
+      Stream.of(scopes).forEach(scopesParam::param);
+    }
+
+    abstract void addFields(JDefinedClass jDefinedClass);
+
+    private class ByteArrayStreamCodeWriter extends CodeWriter {
+
+      private final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+
+      @Override
+      public OutputStream openBinary(JPackage pkg, String fileName) {
+        return outputStream;
+      }
+
+      @Override
+      public void close() {
+        // no need to close byte array stream
+      }
+
+      String sourceCode() {
+        return new String(outputStream.toByteArray());
+      }
+    }
+  }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/transform/TestFilterTransformer.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/transform/TestFilterTransformer.java
new file mode 100644
index 0000000..112b07b
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/transform/TestFilterTransformer.java
@@ -0,0 +1,222 @@
+/*
+ * 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.metastore.iceberg.transform;
+
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.iceberg.IcebergBaseTest;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestFilterTransformer extends IcebergBaseTest {
+
+  private static FilterTransformer transformer;
+
+  @BeforeClass
+  public static void init() {
+    transformer = new FilterTransformer();
+  }
+
+  @Test
+  public void testToFilterNull() {
+    Expression expected = Expressions.alwaysTrue();
+    Expression actual = transformer.transform((FilterExpression) null);
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterEqual() {
+    Expression expected = Expressions.equal("a", 1);
+    Expression actual = transformer.transform(FilterExpression.equal("a", 1));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterNotEqual() {
+    Expression expected = Expressions.notEqual("a", 1);
+    Expression actual = transformer.transform(FilterExpression.notEqual("a", 1));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterLessThan() {
+    Expression expected = Expressions.lessThan("a", 1);
+    Expression actual = transformer.transform(FilterExpression.lessThan("a", 1));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterLessThanOrEqual() {
+    Expression expected = Expressions.lessThanOrEqual("a", 1);
+    Expression actual = transformer.transform(FilterExpression.lessThanOrEqual("a", 1));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterGreaterThan() {
+    Expression expected = Expressions.greaterThan("a", 1);
+    Expression actual = transformer.transform(FilterExpression.greaterThan("a", 1));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterGreaterThanOrEqual() {
+    Expression expected = Expressions.greaterThanOrEqual("a", 1);
+    Expression actual = transformer.transform(FilterExpression.greaterThanOrEqual("a", 1));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterIn() {
+    Expression expected = Expressions.or(Expressions.equal("a", 1), Expressions.equal("a", 2));
+    Expression actual = transformer.transform(FilterExpression.in("a", 1, 2));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterNotIn() {
+    Expression expected = Expressions.not(
+      Expressions.or(Expressions.equal("a", 1), Expressions.equal("a", 2)));
+    Expression actual = transformer.transform(FilterExpression.notIn("a", 1, 2));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterIsNull() {
+    Expression expected = Expressions.isNull("a");
+    Expression actual = transformer.transform(FilterExpression.isNull("a"));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterIsNotNull() {
+    Expression expected = Expressions.notNull("a");
+    Expression actual = transformer.transform(FilterExpression.isNotNull("a"));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterNot() {
+    Expression expected = Expressions.not(Expressions.equal("a", 1));
+    Expression actual = transformer.transform(FilterExpression.not(FilterExpression.equal("a", 1)));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterAnd() {
+    Expression expected = Expressions.and(
+      Expressions.equal("a", 1), Expressions.equal("b", 2),
+      Expressions.equal("c", 3), Expressions.equal("d", 4));
+
+    Expression actual = transformer.transform(FilterExpression.and(
+      FilterExpression.equal("a", 1), FilterExpression.equal("b", 2),
+      FilterExpression.equal("c", 3), FilterExpression.equal("d", 4)));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterOr() {
+    Expression expected = Expressions.or(Expressions.equal("a", 1), Expressions.equal("a", 2));
+    Expression actual = transformer.transform(
+      FilterExpression.or(FilterExpression.equal("a", 1), FilterExpression.equal("a", 2)));
+
+    assertEquals(expected.toString(), actual.toString());
+  }
+
+  @Test
+  public void testToFilterUnsupported() {
+    thrown.expect(UnsupportedOperationException.class);
+
+    transformer.transform(new FilterExpression() {
+      @Override
+      public Operator operator() {
+        return null;
+      }
+
+      @Override
+      public <T> T accept(Visitor<T> visitor) {
+        return visitor.visit(this);
+      }
+    });
+  }
+
+  @Test
+  public void testToFilterConditionsNull() {
+    assertEquals(Expressions.alwaysTrue().toString(), transformer.transform((Map<String, Object>) null).toString());
+  }
+
+  @Test
+  public void testToFilterConditionsEmpty() {
+    assertEquals(Expressions.alwaysTrue().toString(), transformer.transform(Collections.emptyMap()).toString());
+  }
+
+  @Test
+  public void testToFilterConditionsOne() {
+    Map<String, Object> conditions = new HashMap<>();
+    conditions.put("a", 1);
+
+    assertEquals(Expressions.equal("a", 1).toString(), transformer.transform(conditions).toString());
+  }
+
+  @Test
+  public void testToFilterConditionsTwo() {
+    Map<String, Object> conditions = new HashMap<>();
+    conditions.put("a", 1);
+    conditions.put("b", 2);
+
+    Expression expected = Expressions.and(
+      Expressions.equal("a", 1), Expressions.equal("b", 2));
+
+    assertEquals(expected.toString(), transformer.transform(conditions).toString());
+  }
+
+  @Test
+  public void testToFilterConditionsFour() {
+    Map<String, Object> conditions = new HashMap<>();
+    conditions.put("a", 1);
+    conditions.put("b", 2);
+    conditions.put("c", 3);
+    conditions.put("d", 4);
+
+    Expression expected = Expressions.and(
+      Expressions.equal("a", 1), Expressions.equal("b", 2),
+      Expressions.equal("c", 3), Expressions.equal("d", 4));
+
+    assertEquals(expected.toString(), transformer.transform(conditions).toString());
+  }
+}
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/write/TestParquetFileWriter.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/write/TestParquetFileWriter.java
new file mode 100644
index 0000000..6943185
--- /dev/null
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/write/TestParquetFileWriter.java
@@ -0,0 +1,283 @@
+/*
+ * 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.metastore.iceberg.write;
+
+import org.apache.drill.metastore.iceberg.IcebergBaseTest;
+import org.apache.drill.metastore.iceberg.exceptions.IcebergMetastoreException;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.Tables;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.Types;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestParquetFileWriter extends IcebergBaseTest {
+
+  private static Tables tables;
+
+  @BeforeClass
+  public static void init() {
+    tables = new HadoopTables(baseHadoopConfig());
+  }
+
+  @Test
+  public void testAllTypes() throws Exception {
+    Schema schema = new Schema(
+      Types.NestedField.optional(1, "int_field", Types.IntegerType.get()),
+      Types.NestedField.optional(2, "long_field", Types.LongType.get()),
+      Types.NestedField.optional(3, "float_field", Types.FloatType.get()),
+      Types.NestedField.optional(4, "double_field", Types.DoubleType.get()),
+      Types.NestedField.optional(5, "string_field", Types.StringType.get()),
+      Types.NestedField.optional(6, "boolean_field", Types.BooleanType.get()),
+      Types.NestedField.optional(7, "list_field", Types.ListType.ofOptional(
+        9, Types.StringType.get())),
+      Types.NestedField.optional(8, "map_field", Types.MapType.ofOptional(
+        10, 11, Types.StringType.get(), Types.FloatType.get())));
+
+    List<String> listValue = Arrays.asList("a", "b", "c");
+
+    Map<String, Float> mapValue = new HashMap<>();
+    mapValue.put("a", 0.1F);
+    mapValue.put("b", 0.2F);
+
+    Record record = GenericRecord.create(schema);
+    record.setField("int_field", 1);
+    record.setField("long_field", 100L);
+    record.setField("float_field", 0.5F);
+    record.setField("double_field", 1.5D);
+    record.setField("string_field", "abc");
+    record.setField("boolean_field", true);
+    record.setField("list_field", listValue);
+    record.setField("map_field", mapValue);
+
+    String location = defaultFolder.newFolder("testAllTypes").toURI().getPath();
+    String fileName = "allTypes";
+    Table table = tables.create(schema, location);
+
+    org.apache.drill.metastore.iceberg.write.File result = new ParquetFileWriter(table)
+      .records(Collections.singletonList(record))
+      .location(location)
+      .name(fileName)
+      .write();
+
+    String writePath = new Path(location, FileFormat.PARQUET.addExtension(fileName)).toUri().getPath();
+    assertEquals(new Path( FileFormat.PARQUET.addExtension(writePath)), new Path(result.location()));
+    assertEquals(Long.valueOf(1), result.metrics().recordCount());
+
+    List<Record> rows = readData(result.input(), schema);
+
+    assertEquals(1, rows.size());
+
+    Record row = rows.get(0);
+    assertEquals(1, row.getField("int_field"));
+    assertEquals(100L, row.getField("long_field"));
+    assertEquals(0.5F, row.getField("float_field"));
+    assertEquals(1.5D, row.getField("double_field"));
+    assertEquals("abc", row.getField("string_field"));
+    assertEquals(true, row.getField("boolean_field"));
+    assertEquals(listValue, row.getField("list_field"));
+    assertEquals(mapValue, row.getField("map_field"));
+  }
+
+  @Test
+  public void testNullAndEmptyValues() throws Exception {
+    Schema schema = new Schema(
+      Types.NestedField.optional(1, "int_null_field", Types.IntegerType.get()),
+      Types.NestedField.optional(2, "string_null_field", Types.StringType.get()),
+      Types.NestedField.optional(3, "string_empty_field", Types.StringType.get()),
+      Types.NestedField.optional(4, "boolean_null_field", Types.BooleanType.get()),
+      Types.NestedField.optional(5, "list_null_field", Types.ListType.ofOptional(
+        9, Types.StringType.get())),
+      Types.NestedField.optional(6, "list_empty_field", Types.ListType.ofOptional(
+        10, Types.StringType.get())),
+      Types.NestedField.optional(7, "map_null_field", Types.MapType.ofOptional(
+      11, 12, Types.StringType.get(), Types.FloatType.get())),
+    Types.NestedField.optional(8, "map_empty_field", Types.MapType.ofOptional(
+      13, 14, Types.StringType.get(), Types.FloatType.get())));
+
+    Record record = GenericRecord.create(schema);
+    record.setField("int_null_field", null);
+    record.setField("string_null_field", null);
+    record.setField("string_empty_field", "");
+    record.setField("boolean_null_field", null);
+    record.setField("list_null_field", null);
+    record.setField("list_empty_field", Collections.emptyList());
+    record.setField("map_null_field", null);
+    record.setField("map_empty_field", Collections.emptyMap());
+
+    String location = defaultFolder.newFolder("testNullAndEmptyValues").toURI().getPath();
+    Table table = tables.create(schema, location);
+
+    org.apache.drill.metastore.iceberg.write.File result = new ParquetFileWriter(table)
+      .records(Collections.singletonList(record))
+      .location(location)
+      .name("nullEmptyValues")
+      .write();
+
+    assertEquals(Long.valueOf(1), result.metrics().recordCount());
+
+    List<Record> rows = readData(result.input(), schema);
+    assertEquals(1, rows.size());
+
+    Record row = rows.get(0);
+    assertNull(row.getField("int_null_field"));
+    assertNull(row.getField("string_null_field"));
+    assertEquals("", row.getField("string_empty_field"));
+    assertNull(row.getField("boolean_null_field"));
+    assertNull(row.getField("list_null_field"));
+    assertEquals(Collections.emptyList(), row.getField("list_empty_field"));
+    assertNull(row.getField("map_null_field"));
+    assertEquals(Collections.emptyMap(), row.getField("map_empty_field"));
+  }
+
+  @Test
+  public void testEmptyFile() throws Exception {
+    Schema schema = new Schema(
+      Types.NestedField.optional(1, "int_field", Types.IntegerType.get()));
+
+    String location = defaultFolder.newFolder("testEmptyFile").toURI().getPath();
+    Table table = tables.create(schema, location);
+
+    org.apache.drill.metastore.iceberg.write.File result = new ParquetFileWriter(table)
+      .location(location)
+      .name("emptyFile")
+      .write();
+
+    assertEquals(Long.valueOf(0), result.metrics().recordCount());
+
+    List<Record> rows = readData(result.input(), schema);
+    assertEquals(0, rows.size());
+  }
+
+  @Test
+  public void testSeveralRecords() throws Exception {
+    int fieldIndex = 1;
+    Schema schema = new Schema(
+      Types.NestedField.optional(fieldIndex, "int_field", Types.IntegerType.get()));
+
+    List<Integer> values = Arrays.asList(1, 2, 3, 3, null, null, null);
+
+    List<Record> records = values.stream()
+      .map(value -> {
+        Record record = GenericRecord.create(schema);
+        record.setField("int_field", value);
+        return record;
+      })
+      .collect(Collectors.toList());
+
+    String location = defaultFolder.newFolder("testSeveralRecords").toURI().getPath();
+    Table table = tables.create(schema, location);
+
+    org.apache.drill.metastore.iceberg.write.File result = new ParquetFileWriter(table)
+      .records(records)
+      .location(location)
+      .name("severalRecords")
+      .write();
+
+    assertEquals(Long.valueOf(7), result.metrics().recordCount());
+    assertEquals(Long.valueOf(7), result.metrics().valueCounts().get(fieldIndex));
+    assertEquals(Long.valueOf(3), result.metrics().nullValueCounts().get(fieldIndex));
+
+    List<Record> rows = readData(result.input(), schema);
+    assertEquals(7, rows.size());
+    List<Integer> actual = rows.stream()
+      .map(row -> (Integer) row.getField("int_field"))
+      .collect(Collectors.toList());
+    assertEquals(values, actual);
+  }
+
+  @Test
+  public void testTypeMismatch() throws Exception {
+    Schema schema = new Schema(
+      Types.NestedField.optional(1, "int_field", Types.IntegerType.get()));
+
+    Record record = GenericRecord.create(schema);
+    record.setField("int_field", 1);
+    record.setField("int_field", "abc");
+
+    String location = defaultFolder.newFolder("testTypeMismatch").toURI().getPath();
+    Table table = tables.create(schema, location);
+
+    thrown.expect(IcebergMetastoreException.class);
+
+    new ParquetFileWriter(table)
+      .records(Collections.singletonList(record))
+      .location(location)
+      .name("typeMismatch")
+      .write();
+  }
+
+  @Test
+  public void testWriteIntoExistingFile() throws Exception {
+    Schema schema = new Schema(
+      Types.NestedField.optional(1, "int_field", Types.IntegerType.get()));
+
+    Record record = GenericRecord.create(schema);
+    record.setField("int_field", 1);
+
+    String fileName = "existingFile";
+    String location = defaultFolder.newFolder("testWriteIntoExistingFile").toURI().getPath();
+    Table table = tables.create(schema, location);
+
+    java.nio.file.Path file = Paths.get(new File(location, FileFormat.PARQUET.addExtension(fileName)).getPath());
+    Files.write(file, Collections.singletonList("abc"));
+
+    org.apache.drill.metastore.iceberg.write.File result = new ParquetFileWriter(table)
+      .records(Collections.singletonList(record))
+      .location(location)
+      .name(fileName)
+      .write();
+
+    List<Record> rows = readData(result.input(), schema);
+    assertEquals(1, rows.size());
+    assertEquals(1, rows.get(0).getField("int_field"));
+  }
+
+  private List<Record> readData(InputFile inputFile, Schema schema) throws IOException {
+    try (CloseableIterable<Record> reader = Parquet.read(inputFile)
+      .project(schema)
+      .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema))
+      .build()) {
+      return Lists.newArrayList(reader);
+    }
+  }
+}
diff --git a/metastore/metastore-api/README.md b/metastore/metastore-api/README.md
new file mode 100644
index 0000000..73e5831
--- /dev/null
+++ b/metastore/metastore-api/README.md
@@ -0,0 +1,334 @@
+# Metastore
+
+Metastore can store metadata information for tables, views, etc.
+Such metadata can be used during queries execution for better planning,
+obtaining additional information, etc.
+
+### Configuration
+
+All configuration properties should reside in `drill.metastore` namespace.
+
+Default Metastore configuration is defined in `drill-metastore-default.conf` file.
+It can be overridden in `drill-metastore-override.conf`. Distribution configuration can be
+indicated in `drill-metastore-distrib.conf`. Metastore implementations configuration can be
+indicated in `drill-metastore-module.conf`.
+
+## Initialization
+
+`MetastoreRegistry` is initialized during Drillbit start up and is accessible though `DrillbitContext`.
+It lazily initializes Metastore implementation based
+on class implementation config property `drill.metastore.implementation.class`.
+
+Metastore implementation must implement `Metastore` interface and
+have constructor which accepts `DrillConfig`.
+
+### Metastore Components
+
+Metastore can store metadata for various components: tables, views etc.
+Current implementation provides fully functioning support for tables component.
+Views component support is not implemented but contains stub methods to show
+how new Metastore components like udfs, storage plugins, etc. be added in future.
+
+### Metastore Tables
+
+Metastore Tables component contains metadata about Drill tables, including general information, as well as
+information about table segments, files, row groups, partitions.
+
+`TableMetadataUnit` is a generic representation of Metastore Tables metadata unit, 
+suitable to any table metadata type (table, segment, file, row group, partition).
+`BaseTableMetadata`, `SegmentMetadata`, `FileMetadata`, `RowGroupMetadata` and `PartitionMetadata`
+classes which are located in `org.apache.drill.metastore.metadata` package, have methods 
+to convert to / from `TableMetadataUnit`.
+
+Full table metadata consists of two major concepts: general information and top-level segments metadata.
+Table general information contains basic table information and corresponds to `BaseTableMetadata` class.
+
+Table can be non-partitioned and partitioned. Non-partitioned tables, have only one top-level segment 
+which is called default (`MetadataInfo#DEFAULT_SEGMENT_KEY`). Partitioned tables may have several top-level segments.
+Each top-level segment can include metadata about inner segments, files, row groups and partitions.
+
+Unique table identifier in Metastore Tables is combination of storage plugin, workspace and table name.
+Table metadata inside is grouped by top-level segments, unique identifier of the top-level segment and its metadata
+is storage plugin, workspace, table name and metadata key.
+
+### Metadata
+
+In order to provide Metastore component metadata functionality `Metadata` interface must be implemented.
+`Metadata` interface implementation can be specific for each Metastore component or shared.
+For example, for Iceberg Metastore, each component will provide its own `Metadata` interface implementation
+since each component is stored in separate Iceberg tables. For other Metastore implementations, if components reside
+in the same storage (database), `Metadata` interface implementation can be shared.
+
+#### Versioning
+
+Metastore component may or may not support versioning depending on the implementation.
+`Metadata#supportsVersioning` and `Metastore.Metadata#version` methods 
+are used to indicate versioning support. If Metastore component does not support versioning, 
+`Metadata#version` returns undefined version (`Metadata#UNDEFINED`).
+If Metastore component supports versioning, it is assumed that version is changed each time
+data in the Metastore component is modified and remains the same during read operations.
+Metastore component version is used to determine if metadata has changed after last access of the Metastore component.
+
+#### Properties
+
+Metastore component may or may not support properties depending on the implementation.
+If properties are supported, map with properties names and values is returned.
+otherwise empty map is returned. `Metadata#properties` is used to obtain properties information. 
+
+### Filter expression
+
+Metastore data can be read or deleted based on the filter expression.
+All filter expressions implement `FilterExpression` interface. 
+List of supported filter operators is indicated in `FilterExpression.Operator` enum.
+When filter expression is provided in read or delete operation, it's up to Metastore
+implementation to convert it into suitable representation for storage.
+For convenience, `FilterExpression.Visitor` can be implemented to traverse filter expression.
+
+Filter expression can be simple and contain only one condition:
+
+```
+FilterExpression storagePlugin = FilterExpression.equal("storagePlugin", "dfs");
+FilterExpression workspaces = FilterExpression.in("workspace", "root", "tmp");
+
+```
+
+Or it can be complex and contain several conditions combined with `AND` or `OR` operators.
+
+```
+  FilterExpression filter = FilterExpression.and(
+    FilterExpression.equal("storagePlugin", "dfs"),
+    FilterExpression.in("workspace", "root", "tmp"));
+  
+  metastore.tables().read()
+    .filters(filter)
+    .execute();
+```
+
+SQL-like equivalent for the above operation is:
+
+```
+  select * from METASTORE.TABLES
+  where storagePlugin = 'dfs'
+  and workspace in ('root', 'tmp')
+```
+
+### Metastore Read
+
+In order to provide read functionality each component must implement `Read`.
+During implementation component unit type must be indicated.
+`Metastore.Read#columns` allows to specify list of columns to be retrieved from the Metastore component.
+`Metastore.Read#filter` allows to specify filter expression by which data will be retrieved.
+`Metastore.Read#execute` executes read operation and returns the results.
+Data is returned in a form of list of component metadata units, it is caller responsibility to transform received
+data into suitable representation. It is expected, if no result is found, empty of list of units
+will be returned, not null instance.
+
+For `tables` component, metadata unit is represented by `TableMetadataUnit`. To retrieve `lastModifiedTime` 
+for all tables in the `dfs` storage plugin the following code can be used:
+
+```
+  List<TableMetadataUnit> units = metastore.tables().read()
+    .columns("tableName", "lastModifiedTime")
+    .filter(FilterExpression.equal("storagePlugin", "dfs")
+    .execute();
+```
+
+### Metastore Tables Basic Requests
+
+`Tables#basicRequests` provides list of most frequent requests to the Metastore Tables without need 
+to write filters and transformers from `TableMetadataUnit` class.
+
+Assume caller needs to obtain general metadata about `dfs.tmp.nation` table: 
+
+```
+    TableInfo tableInfo = TableInfo.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .name("nation")
+      .build();
+
+    BaseTableMetadata metastoreTableInfo = metastore.tables().basicRequests()
+      .tableMetadata(tableInfo);
+```
+
+### Metastore Modify
+
+In order to provide read functionality each component must implement `Modify`.
+During implementation component unit type must be indicated.
+
+If Metastore component implementation supports versioning, it is assumed that each modify operation will
+change Metastore component version.
+
+#### Overwrite
+
+`Read#overwrite` writes data into Metastore component or overwrites existing data by unique partition 
+keys combination. 
+
+For example, for `tables` component, partition keys are storage plugin, workspace, table name 
+and metadata key. Caller provides only list of `TableMetadataUnit` to be written 
+and Metastore component implementation will decide how data will be stored or overwritten.
+
+##### Adding new table metadata
+
+Assume there is non-partitioned table which metadata is represented with two units.
+
+1. Unit with table general information:
+
+```
+    TableMetadataUnit tableUnit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+      .metadataType(MetadataType.TABLE.name())
+      ...
+      .build();
+```
+
+2. Unit with default segment information with one file:
+
+```
+    TableMetadataUnit segmentUnit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey(MetadataInfo.DEFAULT_SEGMENT_KEY)
+      .metadataType(MetadataType.SEGMENT.name())
+      ...
+      .build();
+      
+    TableMetadataUnit fileUnit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey(MetadataInfo.DEFAULT_SEGMENT_KEY)
+      .metadataType(MetadataType.FILE.name())
+      ...
+      .build();
+```
+
+To add this table metadata in the Metastore, the following code can be executed:
+
+```
+    metastore.tables().modify()
+      .overwrite(tableUnit, segmentUnit)
+      .execute();
+```
+
+##### Overwriting table metadata
+
+Metastore allows only to overwrite metadata by unique combination of table identifier 
+or table metadata identifiers (general info or top-level segments).
+
+When only general table information has changed but segments default metadata did not change,
+it is enough to overwrite only general information.
+
+```
+    TableMetadataUnit updatedTableUnit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+      .metadataType(MetadataType.TABLE.name())
+      ...
+      .build();
+      
+    metastore.tables().modify()
+      .overwrite(updatedTableUnit)
+      .execute();
+```
+
+If segment metadata was changed, new file was added to the default segment, 
+all segment information must be overwritten.
+
+```   
+    TableMetadataUnit segmentUnit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey(MetadataInfo.DEFAULT_SEGMENT_KEY)
+      .metadataType(MetadataType.SEGMENT.name())
+      ...
+      .build();
+      
+    TableMetadataUnit initialFileUnit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey(MetadataInfo.DEFAULT_SEGMENT_KEY)
+      .metadataType(MetadataType.FILE.name())
+      ...
+      .build();
+      
+    TableMetadataUnit newFileUnit = TableMetadataUnit.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .tableName("nation")
+      .metadataKey(MetadataInfo.DEFAULT_SEGMENT_KEY)
+      .metadataType(MetadataType.FILE.name())
+      ...
+      .build();  
+      
+    metastore.tables().modify()
+      .overwrite(segmentUnit, initialFileUnit, newFileUnit)
+      .execute();
+```
+
+#### Delete
+
+`Read#delete` deletes data from the Metastore component based on the provided filter expression.
+
+Assume metadata for table `dfs.tmp.nation` already exists in the Metastore `tables` component
+and caller needs to delete it. First, deletion filter must be created:
+
+```
+    FilterExpression filter = FilterExpression.and(
+      FilterExpression.equal("storagePlugin", "dfs"),
+      FilterExpression.equal("workspace", "tmp"),
+      FilterExpression.equal("tableName", "nation"));
+```
+
+Such filter can be also generated using `TableInfo` class:
+
+```
+    TableInfo tableInfo = TableInfo.builder()
+      .storagePlugin("dfs")
+      .workspace("tmp")
+      .name("nation")
+      .build();
+      
+    FilterExpression filter = tableInfo.toFilter();  
+```
+
+Delete operation can be executed using the following code:
+
+```
+    metastore.tables().modify()
+      .delete(filter)
+      .execute();
+```
+
+#### Purge
+
+`Read#purge` deletes all data from the Metastore component.
+
+```
+    metastore.tables().modify()
+      .purge()
+      .execute();
+```
+
+#### Transactions
+
+Metastore component implementation may or may not support transactions. If transactions are supported,
+all operations in one `Modify` instance will be executed fully or not executed at all.
+If Metastore implementation does not support transactions, all operations will be executed consequently.
+
+```
+    metastore.tables().modify()
+      .overwrite(tableUnit1, segmentUnit1)
+      .overwrite(tableUnit2, segmentUnit2)
+      .delete(table3Filter)
+      .delete(table4Filter)
+      .execute();
+```
diff --git a/metastore/metastore-api/pom.xml b/metastore/metastore-api/pom.xml
index d3f3aed..869cdd8 100644
--- a/metastore/metastore-api/pom.xml
+++ b/metastore/metastore-api/pom.xml
@@ -28,6 +28,42 @@
   </parent>
 
   <artifactId>drill-metastore-api</artifactId>
-  <name>Drill Metastore API</name>
+  <name>metastore/Drill Metastore API</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-logical</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.drill.exec</groupId>
+      <artifactId>vector</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-codec</groupId>
+          <artifactId>commons-codec</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-all</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
 
 </project>
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/Metastore.java
similarity index 53%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/Metastore.java
index 7db95fb..7dad82d 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/Metastore.java
@@ -15,19 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.drill.metastore.components.tables.Tables;
+import org.apache.drill.metastore.components.views.Views;
 
 /**
- * A metadata which has specific location.
+ * Drill Metastore interface contains methods needed to be implemented by Metastore implementations.
+ *
+ * Drill Metastore main goal is to read and write Metastore data from / to Metastore components:
+ * tables, views, etc.
+ *
+ * Besides implementing {@link Metastore}, Metastore implementation must have constructor
+ * which accepts {@link org.apache.drill.common.config.DrillConfig}.
  */
-public interface LocationProvider {
+public interface Metastore {
+
+  /**
+   * @return Metastore Tables component implementation
+   */
+  Tables tables();
 
   /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
+   * @return Metastore Views component implementation
    */
-  Path getPath();
+  Views views();
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/MetastoreFieldDefinition.java
similarity index 51%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/MetastoreFieldDefinition.java
index 7db95fb..c81b17c 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/MetastoreFieldDefinition.java
@@ -15,19 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.drill.metastore.metadata.MetadataType;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
 /**
- * A metadata which has specific location.
+ * Annotation used to determine to which metadata types Metastore units fields belong.
+ * Mainly used when obtaining field information using reflection.
+ * Only fields annotated with this annotation are considered to be part of component
+ * metadata unit, non-annotated fields will be ignored.
  */
-public interface LocationProvider {
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.FIELD})
+public @interface MetastoreFieldDefinition {
 
   /**
-   * Returns location of this metadata.
+   * Indicated metadata types to which field belongs to.
    *
-   * @return metadata location
+   * @return array of metadata types field belongs to
    */
-  Path getPath();
+  MetadataType[] scopes();
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/MetastoreRegistry.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/MetastoreRegistry.java
new file mode 100644
index 0000000..0343463
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/MetastoreRegistry.java
@@ -0,0 +1,107 @@
+/*
+ * 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.metastore;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.metastore.config.MetastoreConfigConstants;
+import org.apache.drill.metastore.config.MetastoreConfigFileInfo;
+import org.apache.drill.metastore.exceptions.MetastoreException;
+
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
+
+/**
+ * Class is responsible for returning instance of {@link Metastore} class
+ * which will be initialized based on {@link MetastoreConfigConstants#IMPLEMENTATION_CLASS} config property value.
+ * Metastore initialization is delayed until {@link #get()} method is called.
+ * Metastore implementation must have constructor which accepts {@link DrillConfig}.
+ */
+public class MetastoreRegistry {
+
+  private DrillConfig config;
+  private volatile Metastore metastore;
+
+  public MetastoreRegistry(DrillConfig config) {
+    this.config = config;
+  }
+
+  public Metastore get() {
+    if (metastore == null) {
+      synchronized (this) {
+        if (metastore == null) {
+          metastore = initMetastore();
+        }
+      }
+    }
+    return metastore;
+  }
+
+  private Metastore initMetastore() {
+    DrillConfig metastoreConfig = createMetastoreConfig(config);
+    String metastoreClass = metastoreConfig.getString(MetastoreConfigConstants.IMPLEMENTATION_CLASS);
+    if (metastoreClass == null) {
+      throw new MetastoreException(
+        String.format("Drill Metastore class config is absent [%s]", MetastoreConfigConstants.IMPLEMENTATION_CLASS));
+    }
+    MethodHandles.Lookup publicLookup = MethodHandles.publicLookup();
+    MethodHandle constructor;
+    try {
+      MethodType methodType = MethodType.methodType(void.class, DrillConfig.class);
+      constructor = publicLookup.findConstructor(Class.forName(metastoreClass), methodType);
+    } catch (ClassNotFoundException e) {
+      throw new MetastoreException(
+        String.format("Unable to find Metastore implementation class [%s]", metastoreClass));
+    } catch (NoSuchMethodException | IllegalAccessException e) {
+      throw new MetastoreException(
+        String.format("Metastore implementation class [%s] must have constructor which accepts [%s]",
+          metastoreClass, metastoreConfig.getClass().getSimpleName()));
+    }
+
+    Object instance;
+    try {
+      instance = constructor.invokeWithArguments(metastoreConfig);
+    } catch (Throwable e) {
+      throw new MetastoreException(
+        String.format("Unable to init Drill Metastore class [%s]", metastoreClass), e);
+    }
+
+    if (!(instance instanceof Metastore)) {
+      throw new MetastoreException(
+        String.format("Created instance of [%s] does not implement [%s] interface",
+          instance.getClass().getSimpleName(), Metastore.class.getSimpleName()));
+    }
+    return (Metastore) instance;
+  }
+
+  /**
+   * Creates Metastore Config and substitutes config values from Drill main config
+   * for default and module configs only.
+   *
+   * For example, if Iceberg module config defines relative path based on Drill Zk root:
+   * drill.metastore.iceberg.location.relative_path: ${drill.exec.zk.root}"/metastore/iceberg",
+   * and Drill main config defines drill.exec.zk.root as "drill",
+   * resulting Iceberg table relative path will be drill/metastore/iceberg.
+   *
+   * @param config main Drill config
+   * @return metastore config
+   */
+  private DrillConfig createMetastoreConfig(DrillConfig config) {
+    return DrillConfig.create(null, null, true, new MetastoreConfigFileInfo(), config.root());
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesRequests.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesRequests.java
new file mode 100644
index 0000000..6cc84fc
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesRequests.java
@@ -0,0 +1,657 @@
+/*
+ * 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.metastore.components.tables;
+
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.metadata.BaseTableMetadata;
+import org.apache.drill.metastore.metadata.FileMetadata;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.PartitionMetadata;
+import org.apache.drill.metastore.metadata.RowGroupMetadata;
+import org.apache.drill.metastore.metadata.SegmentMetadata;
+import org.apache.drill.metastore.metadata.TableInfo;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Provides handy methods to retrieve Metastore Tables data for analysis.
+ * Contains list of most frequent requests to the Metastore Tables without a need
+ * to write filters and transformers from {@link TableMetadataUnit} class.
+ */
+public class BasicTablesRequests {
+
+  public static final String LAST_MODIFIED_TIME = "lastModifiedTime";
+  public static final String PATH = "path";
+  public static final String METADATA_KEY = "metadataKey";
+  public static final String LOCATION = "location";
+  public static final String COLUMN = "column";
+  public static final String METADATA_TYPE = "metadataType";
+  public static final String INTERESTING_COLUMNS = "interestingColumns";
+  public static final String PARTITION_KEYS = "partitionKeys";
+
+  private final Tables tables;
+
+  public BasicTablesRequests(Tables tables) {
+    this.tables = tables;
+  }
+
+  /**
+   * Returns metastore table information, including metastore version and table last modified time.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select lastModifiedTime from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and metadataKey = 'GENERAL_INFO'
+   *   and metadataType = 'TABLE'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @return {@link MetastoreTableInfo} instance
+   */
+  public MetastoreTableInfo metastoreTableInfo(TableInfo tableInfo) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+      .metadataType(MetadataType.TABLE.name())
+      .requestColumns(LAST_MODIFIED_TIME)
+      .build();
+
+    long version = tables.metadata().version();
+    TableMetadataUnit unit = retrieveSingleElement(request(requestMetadata));
+    return MetastoreTableInfo.of(tableInfo, unit, version);
+  }
+
+  /**
+   * Checks if given metastore table information is the same with current one.
+   * If Metastore supports versioning, first checks metastore versions,
+   * if metastore version did not change, it is assumed table metadata did not change as well.
+   * If Metastore version has changed or Metastore does not support versioning,
+   * retrieves current metastore table info and checks against given one.
+   *
+   * @param metastoreTableInfo metastore table information
+   * @return true is metastore table information has changed, false otherwise
+   */
+  public boolean hasMetastoreTableInfoChanged(MetastoreTableInfo metastoreTableInfo) {
+    if (tables.metadata().supportsVersioning()
+      && metastoreTableInfo.metastoreVersion() == tables.metadata().version()) {
+      return false;
+    }
+
+    MetastoreTableInfo current = metastoreTableInfo(metastoreTableInfo.tableInfo());
+    return metastoreTableInfo.hasChanged(current.isExists(), current.lastModifiedTime());
+  }
+
+  /**
+   * Returns tables general information metadata based on given filter.
+   * For example, can return list of tables that belong to particular storage plugin or
+   * storage plugin and workspace combination.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select [$TABLE_METADATA$] from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp'
+   *   and metadataKey = 'GENERAL_INFO'
+   *   and metadataType = 'TABLE'
+   * </pre>
+   *
+   * @param filter filter expression
+   * @return list of table metadata
+   */
+  public List<BaseTableMetadata> tablesMetadata(FilterExpression filter) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .customFilter(filter)
+      .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+      .metadataType(MetadataType.TABLE.name())
+      .requestColumns(TableMetadataUnit.SCHEMA.tableColumns())
+      .build();
+
+    List<TableMetadataUnit> units = request(requestMetadata);
+    return BasicTablesTransformer.tables(units);
+  }
+
+  /**
+   * Returns table general information metadata based on given table information.
+   * Expects only one qualified result, otherwise will fail.
+   * If no data is returned, will return null.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select [$TABLE_METADATA$] from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and metadataKey = 'GENERAL_INFO'
+   *   and metadataType = 'TABLE'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @return table metadata
+   */
+  public BaseTableMetadata tableMetadata(TableInfo tableInfo) {
+    List<BaseTableMetadata> tables = tablesMetadata(tableInfo.toFilter());
+    return retrieveSingleElement(tables);
+  }
+
+  /**
+   * Returns segments metadata based on given table information, locations and column name.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select [$SEGMENT_METADATA$] from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and location in ('.../part_int=3/d3', '.../part_int=3/d4')
+   *   and metadataKey = 'part_int=3'
+   *   and metadataType = 'SEGMENT'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @param locations segments locations
+   * @param metadataKey metadata key
+   * @return list of segment metadata
+   */
+  public List<SegmentMetadata> segmentsMetadataByMetadataKey(TableInfo tableInfo, List<String> locations, String metadataKey) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .locations(locations)
+      .metadataKey(metadataKey)
+      .metadataType(MetadataType.SEGMENT.name())
+      .requestColumns(TableMetadataUnit.SCHEMA.segmentColumns())
+      .build();
+
+    List<TableMetadataUnit> units = request(requestMetadata);
+    return BasicTablesTransformer.segments(units);
+  }
+
+  /**
+   * Returns segments metadata based on given table information, locations and column name.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select [$SEGMENT_METADATA$] from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and location in ('.../dir0', '.../dir1')
+   *   and column = 'n_nation'
+   *   and metadataType = 'SEGMENT'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @param locations segments locations
+   * @param column column name
+   * @return list of segment metadata
+   */
+  public List<SegmentMetadata> segmentsMetadataByColumn(TableInfo tableInfo, List<String> locations, String column) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .locations(locations)
+      .column(column)
+      .metadataType(MetadataType.SEGMENT.name())
+      .requestColumns(TableMetadataUnit.SCHEMA.segmentColumns())
+      .build();
+
+    List<TableMetadataUnit> units = request(requestMetadata);
+    return BasicTablesTransformer.segments(units);
+  }
+
+  /**
+   * Returns partitions metadata based on given table information, metadata keys and column name.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select [$PARTITION_METADATA$] from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and metadataKey in ('part_int=3', 'part_int=4')
+   *   and column = 'n_nation'
+   *   and metadataType = 'PARTITION'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @param metadataKeys list of metadata keys
+   * @param column partition column
+   * @return list of partition metadata
+   */
+  public List<PartitionMetadata> partitionsMetadata(TableInfo tableInfo, List<String> metadataKeys, String column) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .metadataKeys(metadataKeys)
+      .column(column)
+      .metadataType(MetadataType.PARTITION.name())
+      .requestColumns(TableMetadataUnit.SCHEMA.partitionColumns())
+      .build();
+
+    List<TableMetadataUnit> units = request(requestMetadata);
+    return BasicTablesTransformer.partitions(units);
+  }
+
+  /**
+   * Returns files metadata based on given table information, metadata key and files paths.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select [$FILE_METADATA$] from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and metadataKey = 'part_int=3'
+   *   and path in ('/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet', …)
+   *   and metadataType = 'FILE'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @param metadataKey metadata key
+   * @param paths list of full file paths
+   * @return list of files metadata
+   */
+  public List<FileMetadata> filesMetadata(TableInfo tableInfo, String metadataKey, List<String> paths) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .metadataKey(metadataKey)
+      .paths(paths)
+      .metadataType(MetadataType.FILE.name())
+      .requestColumns(TableMetadataUnit.SCHEMA.fileColumns())
+      .build();
+
+    List<TableMetadataUnit> units = request(requestMetadata);
+    return BasicTablesTransformer.files(units);
+  }
+
+  /**
+   * Returns file metadata based on given table information, metadata key and full path.
+   * Expects only one qualified result, otherwise will fail.
+   * If no data is returned, will return null.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select [$FILE_METADATA$] from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and metadataKey = 'part_int=3'
+   *   and path = '/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet'
+   *   and metadataType = 'FILE'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @param metadataKey metadata key
+   * @param path full file path
+   * @return list of files metadata
+   */
+  public FileMetadata fileMetadata(TableInfo tableInfo, String metadataKey, String path) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .metadataKey(metadataKey)
+      .path(path)
+      .metadataType(MetadataType.FILE.name())
+      .requestColumns(TableMetadataUnit.SCHEMA.fileColumns())
+      .build();
+
+    List<TableMetadataUnit> units = request(requestMetadata);
+    return retrieveSingleElement(BasicTablesTransformer.files(units));
+  }
+
+  /**
+   * Returns row groups metadata based on given table information, metadata key and location.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select [$ROW_GROUP_METADATA$] from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and metadataKey = 'part_int=3'
+   *   and path = '/tmp/nation/part_int=3/part_varchar=g/0_0_0.parquet'
+   *   and metadataType = 'ROW_GROUP'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @param metadataKey metadata key
+   * @param path full path to the file of the row group
+   * @return list of row group metadata
+   */
+  public List<RowGroupMetadata> rowGroupsMetadata(TableInfo tableInfo, String metadataKey, String path) {
+   RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .metadataKey(metadataKey)
+      .path(path)
+      .metadataType(MetadataType.ROW_GROUP.name())
+      .requestColumns(TableMetadataUnit.SCHEMA.rowGroupColumns())
+      .build();
+
+    List<TableMetadataUnit> units = request(requestMetadata);
+    return BasicTablesTransformer.rowGroups(units);
+  }
+
+  /**
+   * Returns metadata for segments, files and row groups based on given metadata keys and locations.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select [$SEGMENT_METADATA$] / [$FILE_METADATA$] / [$ROW_GROUP_METADATA$] from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and metadataKey in ('part_int=1', 'part_int=2', 'part_int=5')
+   *   and location in ('.../dir0/d3', '.../dir0/d4', '.../part_int=3/d3', '.../part_int=4/d4')
+   *   and metadataType in ('SEGMENT', 'FILE', 'ROW_GROUP')
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @param metadataKeys metadata keys
+   * @param locations locations
+   * @return list of segments / files / rows groups metadata in {@link BasicTablesTransformer.MetadataHolder} instance
+   */
+  public BasicTablesTransformer.MetadataHolder fullSegmentsMetadataWithoutPartitions(TableInfo tableInfo, List<String> metadataKeys, List<String> locations) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .metadataKeys(metadataKeys)
+      .locations(locations)
+      .metadataTypes(Arrays.asList(MetadataType.SEGMENT.name(), MetadataType.FILE.name(), MetadataType.ROW_GROUP.name()))
+      .build();
+
+    List<TableMetadataUnit> units = request(requestMetadata);
+    return BasicTablesTransformer.all(units);
+  }
+
+  /**
+   * Returns map of file full paths and their last modified time.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select path, lastModifiedTime from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and metadataKey = 'part_int=3'
+   *   and location in ('/tmp/nation/part_int=3/part_varchar=g', ...)
+   *   and metadataType = 'FILE'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @param metadataKey metadata key
+   * @param locations files locations
+   * @return result map where key is file full path and value is file last modification time
+   */
+  public Map<String, Long> filesLastModifiedTime(TableInfo tableInfo, String metadataKey, List<String> locations) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .metadataKey(metadataKey)
+      .locations(locations)
+      .metadataType(MetadataType.FILE.name())
+      .requestColumns(PATH, LAST_MODIFIED_TIME)
+      .build();
+
+    return request(requestMetadata).stream()
+      .collect(Collectors.toMap(
+        TableMetadataUnit::path,
+        TableMetadataUnit::lastModifiedTime,
+        (o, n) -> n));
+  }
+
+  /**
+   * Returns map of segments metadata keys and their last modified time.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select metadataKey, lastModifiedTime from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and location in ('.../dir0', '.../dir1')
+   *   and metadataType = 'SEGMENT'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @param locations segments locations
+   * @return result map where key is metadata key and value is its last modification time
+   */
+  public Map<String, Long> segmentsLastModifiedTime(TableInfo tableInfo, List<String> locations) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .locations(locations)
+      .metadataType(MetadataType.SEGMENT.name())
+      .requestColumns(METADATA_KEY, LAST_MODIFIED_TIME)
+      .build();
+
+    return request(requestMetadata).stream()
+      .collect(Collectors.toMap(
+        TableMetadataUnit::metadataKey,
+        TableMetadataUnit::lastModifiedTime,
+        (o, n) -> n));
+  }
+
+  /**
+   * Returns tables interesting columns and partition keys based on given table information.
+   * Expects only one qualified result, otherwise will fail.
+   * If no data is returned, will return null.
+   *
+   * Schematic SQL request:
+   * <pre>
+   *   select interestingColumns, partitionKeys from METASTORE
+   *   where storage = 'dfs' and workspace = 'tmp' and tableName = 'nation'
+   *   and metadataKey = 'GENERAL_INFO'
+   *   and metadataType = 'TABLE'
+   * </pre>
+   *
+   * @param tableInfo table information
+   * @return {@link TableMetadataUnit} instance with set interesting columns and partition keys if present
+   */
+  public TableMetadataUnit interestingColumnsAndPartitionKeys(TableInfo tableInfo) {
+    RequestMetadata requestMetadata = RequestMetadata.builder()
+      .tableInfo(tableInfo)
+      .metadataKey(MetadataInfo.GENERAL_INFO_KEY)
+      .metadataType(MetadataType.TABLE.name())
+      .requestColumns(INTERESTING_COLUMNS, PARTITION_KEYS)
+      .build();
+
+    return retrieveSingleElement(request(requestMetadata));
+  }
+
+  /**
+   * Executes Metastore Tables read request based on given information in {@link RequestMetadata}.
+   *
+   * @param requestMetadata request metadata
+   * @return list of metadata units
+   */
+  public List<TableMetadataUnit> request(RequestMetadata requestMetadata) {
+    return tables.read()
+      .filter(requestMetadata.filter())
+      .columns(requestMetadata.columns())
+      .execute();
+  }
+
+  /**
+   * Retrieves one element from the given list of elements.
+   * If given list of elements is null or empty, returns null.
+   * Will fail, if given list of elements contains more than one element.
+   *
+   * @param elements list of elements
+   * @param <T> elements type
+   * @return single element
+   * @throws IllegalArgumentException if given more than one element
+   */
+  private <T> T retrieveSingleElement(List<T> elements) {
+    if (elements == null || elements.isEmpty()) {
+      return null;
+    }
+    if (elements.size() > 1) {
+      throw new IllegalArgumentException(String.format("Expected one element but received [%s]", elements.size()));
+    }
+    return elements.get(0);
+  }
+
+  /**
+   * Request metadata holder that provides request filters and columns.
+   * Combines given filters using {@link FilterExpression.Operator#AND} operator.
+   * Supports only {@link FilterExpression.Operator#EQUAL} and {@link FilterExpression.Operator#IN}
+   * operators for predefined filter references, for other cases custom filter can be used.
+   */
+  public static class RequestMetadata {
+
+    private final FilterExpression filter;
+    private final List<String> columns;
+
+    private RequestMetadata(FilterExpression filter, List<String> columns) {
+      this.filter = filter;
+      this.columns = columns;
+    }
+
+    public FilterExpression filter() {
+      return filter;
+    }
+
+    public List<String> columns() {
+      return columns;
+    }
+
+    public static RequestMetadata.Builder builder() {
+      return new RequestMetadata.Builder();
+    }
+
+    public static class Builder {
+
+      private TableInfo tableInfo;
+      private String location;
+      private List<String> locations;
+      private String column;
+      private String metadataType;
+      private List<String> metadataTypes;
+      private String metadataKey;
+      private List<String> metadataKeys;
+      private String path;
+      private List<String> paths;
+      private FilterExpression customFilter;
+      private final List<String> requestColumns = new ArrayList<>();
+
+      public RequestMetadata.Builder tableInfo(TableInfo tableInfo) {
+        this.tableInfo = tableInfo;
+        return this;
+      }
+
+      public RequestMetadata.Builder location(String location) {
+        this.location = location;
+        return this;
+      }
+
+      public RequestMetadata.Builder locations(List<String> locations) {
+        this.locations = locations;
+        return this;
+      }
+
+      public RequestMetadata.Builder column(String column) {
+        this.column = column;
+        return this;
+      }
+
+      public RequestMetadata.Builder metadataType(String metadataType) {
+        this.metadataType = metadataType;
+        return this;
+      }
+
+      public RequestMetadata.Builder metadataTypes(List<String> metadataTypes) {
+        this.metadataTypes = metadataTypes;
+        return this;
+      }
+
+      public RequestMetadata.Builder metadataKey(String metadataKey) {
+        this.metadataKey = metadataKey;
+        return this;
+      }
+
+      public RequestMetadata.Builder metadataKeys(List<String> metadataKeys) {
+        this.metadataKeys = metadataKeys;
+        return this;
+      }
+
+      public RequestMetadata.Builder path(String path) {
+        this.path = path;
+        return this;
+      }
+
+      public RequestMetadata.Builder paths(List<String> paths) {
+        this.paths = paths;
+        return this;
+      }
+
+      public RequestMetadata.Builder customFilter(FilterExpression customFilter) {
+        this.customFilter = customFilter;
+        return this;
+      }
+
+      public RequestMetadata.Builder requestColumns(List<String> requestColumns) {
+        this.requestColumns.addAll(requestColumns);
+        return this;
+      }
+
+      public RequestMetadata.Builder requestColumns(String... requestColumns) {
+        return requestColumns(Arrays.asList(requestColumns));
+      }
+
+      public RequestMetadata build() {
+        return new RequestMetadata(createFilter(), requestColumns);
+      }
+
+      private FilterExpression createFilter() {
+        List<FilterExpression> filters = new ArrayList<>();
+        if (tableInfo != null) {
+          filters.add(tableInfo.toFilter());
+        }
+        addFilter(LOCATION, location, filters);
+        addFilter(LOCATION, locations, filters);
+        addFilter(COLUMN, column, filters);
+        addFilter(METADATA_TYPE, metadataType, filters);
+        addFilter(METADATA_TYPE, metadataTypes, filters);
+        addFilter(METADATA_KEY, metadataKey, filters);
+        addFilter(METADATA_KEY, metadataKeys, filters);
+        addFilter(PATH, path, filters);
+        addFilter(PATH, paths, filters);
+        if (customFilter != null) {
+          filters.add(customFilter);
+        }
+
+        if (filters.isEmpty()) {
+          return null;
+        }
+
+        if (filters.size() == 1) {
+          return filters.get(0);
+        }
+
+        return FilterExpression.and(filters.get(0), filters.get(1),
+          filters.subList(2, filters.size()).toArray(new FilterExpression[0]));
+      }
+
+      /**
+       * Creates filter based on given parameters and adds to the given list of filters.
+       * If given filter value is null, does nothing. If given filter value is List and is not empty,
+       * creates {@link FilterExpression.Operator#IN} filter, if List is empty, does nothing.
+       * For all other cases, creates {@link FilterExpression.Operator#EQUAL} filter.
+       *
+       * @param reference filter reference
+       * @param value filter value
+       * @param filters current list of filters
+       */
+      private <T> void addFilter(String reference, T value, List<FilterExpression> filters) {
+        if (value == null) {
+          return;
+        }
+
+        if (value instanceof List) {
+          List<?> list = (List) value;
+          if (list.isEmpty()) {
+            return;
+          }
+          filters.add(FilterExpression.in(reference, list));
+          return;
+        }
+
+        filters.add(FilterExpression.equal(reference, value));
+      }
+    }
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesTransformer.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesTransformer.java
new file mode 100644
index 0000000..863ba9d
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesTransformer.java
@@ -0,0 +1,147 @@
+/*
+ * 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.metastore.components.tables;
+
+import org.apache.drill.metastore.metadata.BaseTableMetadata;
+import org.apache.drill.metastore.metadata.FileMetadata;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.PartitionMetadata;
+import org.apache.drill.metastore.metadata.RowGroupMetadata;
+import org.apache.drill.metastore.metadata.SegmentMetadata;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Basic metadata transformer class which can transform given list of {@link TableMetadataUnit}
+ * into {@link BaseTableMetadata}, {@link SegmentMetadata}, {@link FileMetadata},
+ * {@link RowGroupMetadata}, {@link PartitionMetadata} or all metadata types returned in one holder
+ * ({@link MetadataHolder}).
+ */
+public class BasicTablesTransformer {
+
+  public static List<BaseTableMetadata> tables(List<TableMetadataUnit> units) {
+    return units.stream()
+      .filter(unit -> MetadataType.TABLE == MetadataType.fromValue(unit.metadataType()))
+      .map(unit -> BaseTableMetadata.builder().metadataUnit(unit).build())
+      .collect(Collectors.toList());
+  }
+
+  public static List<SegmentMetadata> segments(List<TableMetadataUnit> units) {
+    return units.stream()
+      .filter(unit -> MetadataType.SEGMENT == MetadataType.fromValue(unit.metadataType()))
+      .map(unit -> SegmentMetadata.builder().metadataUnit(unit).build())
+      .collect(Collectors.toList());
+  }
+
+  public static List<FileMetadata> files(List<TableMetadataUnit> units) {
+    return units.stream()
+      .filter(unit -> MetadataType.FILE == MetadataType.fromValue(unit.metadataType()))
+      .map(unit -> FileMetadata.builder().metadataUnit(unit).build())
+      .collect(Collectors.toList());
+  }
+
+  public static List<RowGroupMetadata> rowGroups(List<TableMetadataUnit> units) {
+    return units.stream()
+      .filter(unit -> MetadataType.ROW_GROUP == MetadataType.fromValue(unit.metadataType()))
+      .map(unit -> RowGroupMetadata.builder().metadataUnit(unit).build())
+      .collect(Collectors.toList());
+  }
+
+  public static List<PartitionMetadata> partitions(List<TableMetadataUnit> units) {
+    return units.stream()
+      .filter(unit -> MetadataType.PARTITION == MetadataType.fromValue(unit.metadataType()))
+      .map(unit -> PartitionMetadata.builder().metadataUnit(unit).build())
+      .collect(Collectors.toList());
+  }
+
+  public static MetadataHolder all(List<TableMetadataUnit> units) {
+    List<BaseTableMetadata> tables = new ArrayList<>();
+    List<SegmentMetadata> segments = new ArrayList<>();
+    List<FileMetadata> files = new ArrayList<>();
+    List<RowGroupMetadata> rowGroups = new ArrayList<>();
+    List<PartitionMetadata> partitions = new ArrayList<>();
+
+    for (TableMetadataUnit unit : units) {
+      MetadataType metadataType = MetadataType.fromValue(unit.metadataType());
+      if (metadataType == null) {
+        continue;
+      }
+      switch (metadataType) {
+        case TABLE:
+          tables.add(BaseTableMetadata.builder().metadataUnit(unit).build());
+          break;
+        case SEGMENT:
+          segments.add(SegmentMetadata.builder().metadataUnit(unit).build());
+          break;
+        case FILE:
+          files.add(FileMetadata.builder().metadataUnit(unit).build());
+          break;
+        case ROW_GROUP:
+          rowGroups.add(RowGroupMetadata.builder().metadataUnit(unit).build());
+          break;
+        case PARTITION:
+          partitions.add(PartitionMetadata.builder().metadataUnit(unit).build());
+          break;
+      }
+    }
+    return new MetadataHolder(tables, segments, files, rowGroups, partitions);
+  }
+
+  public static class MetadataHolder {
+
+    private final List<BaseTableMetadata> tables;
+    private final List<SegmentMetadata> segments;
+    private final List<FileMetadata> files;
+    private final List<RowGroupMetadata> rowGroups;
+    private final List<PartitionMetadata> partitions;
+
+    public MetadataHolder(List<BaseTableMetadata> tables,
+                          List<SegmentMetadata> segments,
+                          List<FileMetadata> files,
+                          List<RowGroupMetadata> rowGroups,
+                          List<PartitionMetadata> partitions) {
+      this.tables = tables;
+      this.segments = segments;
+      this.files = files;
+      this.rowGroups = rowGroups;
+      this.partitions = partitions;
+    }
+
+    public List<BaseTableMetadata> tables() {
+      return tables;
+    }
+
+    public List<SegmentMetadata> segments() {
+      return segments;
+    }
+
+    public List<FileMetadata> files() {
+      return files;
+    }
+
+    public List<RowGroupMetadata> rowGroups() {
+      return rowGroups;
+    }
+
+    public List<PartitionMetadata> partitions() {
+      return partitions;
+    }
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/MetastoreTableInfo.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/MetastoreTableInfo.java
new file mode 100644
index 0000000..064c391
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/MetastoreTableInfo.java
@@ -0,0 +1,118 @@
+/*
+ * 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.metastore.components.tables;
+
+import org.apache.drill.metastore.metadata.TableInfo;
+
+import java.util.Objects;
+import java.util.StringJoiner;
+
+/**
+ * Holds metastore table metadata information, including table information, exists status,
+ * last modified time and metastore version.
+ */
+public class MetastoreTableInfo {
+
+  private final TableInfo tableInfo;
+  private final Long lastModifiedTime;
+  private final boolean exists;
+  private final long metastoreVersion;
+
+  private MetastoreTableInfo(TableInfo tableInfo, Long lastModifiedTime, boolean exists, long metastoreVersion) {
+    this.tableInfo = tableInfo;
+    this.lastModifiedTime = lastModifiedTime;
+    this.exists = exists;
+    this.metastoreVersion = metastoreVersion;
+  }
+
+  public static MetastoreTableInfo of(TableInfo tableInfo, TableMetadataUnit unit, long metastoreVersion) {
+    boolean exists = unit != null;
+    Long lastModifiedTime = exists ? unit.lastModifiedTime() : null;
+    return new MetastoreTableInfo(tableInfo, lastModifiedTime, exists, metastoreVersion);
+  }
+
+  public TableInfo tableInfo() {
+    return tableInfo;
+  }
+
+  public Long lastModifiedTime() {
+    return lastModifiedTime;
+  }
+
+  public boolean isExists() {
+    return exists;
+  }
+
+  public long metastoreVersion() {
+    return metastoreVersion;
+  }
+
+  /**
+   * Checks if table metadata has changed or not, based on given exists status and last modified time.
+   * Checks are done based on the following rules and order:
+   * <ul>
+   *   <li>If table did not exist but now does not, return true.</li>
+   *   <li>If table existed but now does, return true.</li>
+   *   <li>If both last modified times are null, return false.</li>
+   *   <li>If one last modified time is null and other is not, return true.</li>
+   *   <li>If both last modified times are the same, return false.</li>
+   *   <li>If both last modified times are different, return true.</li>
+   * </ul>
+   *
+   * @param currentExists current table exists status
+   * @param currentLastModifiedTime current table lat modified time
+   * @return true if table metadata has changed, false otherwise
+   */
+  public boolean hasChanged(boolean currentExists, Long currentLastModifiedTime) {
+    if (exists && currentExists) {
+      return !Objects.equals(lastModifiedTime, currentLastModifiedTime);
+    } else {
+      return exists || currentExists;
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(tableInfo, lastModifiedTime, exists, metastoreVersion);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    MetastoreTableInfo that = (MetastoreTableInfo) o;
+    return exists == that.exists
+      && metastoreVersion == that.metastoreVersion
+      && Objects.equals(tableInfo, that.tableInfo)
+      && Objects.equals(lastModifiedTime, that.lastModifiedTime);
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", MetastoreTableInfo.class.getSimpleName() + "[", "]")
+      .add("tableInfo=" + tableInfo)
+      .add("lastModifiedTime=" + lastModifiedTime)
+      .add("exists=" + exists)
+      .add("metastoreVersion=" + metastoreVersion)
+      .toString();
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/TableMetadataUnit.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/TableMetadataUnit.java
new file mode 100644
index 0000000..7863a4d
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/TableMetadataUnit.java
@@ -0,0 +1,561 @@
+/*
+ * 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.metastore.components.tables;
+
+import org.apache.drill.metastore.MetastoreFieldDefinition;
+import org.apache.drill.metastore.exceptions.MetastoreException;
+import org.apache.drill.metastore.metadata.MetadataType;
+
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.StringJoiner;
+
+import static org.apache.drill.metastore.metadata.MetadataType.ALL;
+import static org.apache.drill.metastore.metadata.MetadataType.FILE;
+import static org.apache.drill.metastore.metadata.MetadataType.PARTITION;
+import static org.apache.drill.metastore.metadata.MetadataType.ROW_GROUP;
+import static org.apache.drill.metastore.metadata.MetadataType.SEGMENT;
+import static org.apache.drill.metastore.metadata.MetadataType.TABLE;
+
+/**
+ * Class that represents one row in Drill Metastore Tables which is a generic representation of metastore metadata
+ * suitable to any metastore table metadata type (table, segment, file, row group, partition).
+ * Is used to read and write data from / to Drill Metastore Tables implementation.
+ *
+ * Note: changing field order and adding new fields may break backward compatibility in existing
+ * Metastore implementations. It is recommended to add new information into {@link #additionalMetadata}
+ * field instead.
+ */
+public class TableMetadataUnit {
+
+  public static final Schema SCHEMA = Schema.of(TableMetadataUnit.class, Builder.class);
+
+  @MetastoreFieldDefinition(scopes = {ALL}) private final String storagePlugin;
+  @MetastoreFieldDefinition(scopes = {ALL}) private final String workspace;
+  @MetastoreFieldDefinition(scopes = {ALL}) private final String tableName;
+  @MetastoreFieldDefinition(scopes = {TABLE}) private final String owner;
+  @MetastoreFieldDefinition(scopes = {TABLE}) private final String tableType;
+  @MetastoreFieldDefinition(scopes = {ALL}) private final String metadataType;
+  @MetastoreFieldDefinition(scopes = {ALL}) private final String metadataKey;
+  @MetastoreFieldDefinition(scopes = {TABLE, SEGMENT, FILE, ROW_GROUP}) private final String location;
+  @MetastoreFieldDefinition(scopes = {TABLE}) private final List<String> interestingColumns;
+  @MetastoreFieldDefinition(scopes = {ALL}) private final String schema;
+  @MetastoreFieldDefinition(scopes = {ALL}) private final Map<String, String> columnsStatistics;
+  @MetastoreFieldDefinition(scopes = {ALL}) private final List<String> metadataStatistics;
+  @MetastoreFieldDefinition(scopes = {ALL}) private final Long lastModifiedTime;
+  @MetastoreFieldDefinition(scopes = {TABLE}) private final Map<String, String> partitionKeys;
+  @MetastoreFieldDefinition(scopes = {ALL}) private final String additionalMetadata;
+
+  @MetastoreFieldDefinition(scopes = {SEGMENT, FILE, ROW_GROUP, PARTITION}) private final String metadataIdentifier;
+  @MetastoreFieldDefinition(scopes = {SEGMENT, PARTITION}) private final String column;
+  @MetastoreFieldDefinition(scopes = {SEGMENT, PARTITION}) private final List<String> locations;
+  @MetastoreFieldDefinition(scopes = {SEGMENT, PARTITION}) private final List<String> partitionValues;
+
+  @MetastoreFieldDefinition(scopes = {SEGMENT, FILE, ROW_GROUP}) private final String path;
+
+  @MetastoreFieldDefinition(scopes = {ROW_GROUP}) private final Integer rowGroupIndex;
+  @MetastoreFieldDefinition(scopes = {ROW_GROUP}) private final Map<String, Float> hostAffinity;
+
+  private TableMetadataUnit(Builder builder) {
+    this.storagePlugin = builder.storagePlugin;
+    this.workspace = builder.workspace;
+    this.tableName = builder.tableName;
+    this.owner = builder.owner;
+    this.tableType = builder.tableType;
+    this.metadataType = builder.metadataType;
+    this.metadataKey = builder.metadataKey;
+    this.location = builder.location;
+    this.interestingColumns = builder.interestingColumns;
+    this.schema = builder.schema;
+    this.columnsStatistics = builder.columnsStatistics;
+    this.metadataStatistics = builder.metadataStatistics;
+    this.lastModifiedTime = builder.lastModifiedTime;
+    this.partitionKeys = builder.partitionKeys;
+    this.additionalMetadata = builder.additionalMetadata;
+    this.metadataIdentifier = builder.metadataIdentifier;
+    this.column = builder.column;
+    this.locations = builder.locations;
+    this.partitionValues = builder.partitionValues;
+    this.path = builder.path;
+    this.rowGroupIndex = builder.rowGroupIndex;
+    this.hostAffinity = builder.hostAffinity;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public String storagePlugin() {
+    return storagePlugin;
+  }
+
+  public String workspace() {
+    return workspace;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public String owner() {
+    return owner;
+  }
+
+  public String tableType() {
+    return tableType;
+  }
+
+  public String metadataType() {
+    return metadataType;
+  }
+
+  public String metadataKey() {
+    return metadataKey;
+  }
+
+  public String location() {
+    return location;
+  }
+
+  public List<String> interestingColumns() {
+    return interestingColumns;
+  }
+
+  public String schema() {
+    return schema;
+  }
+
+  public Map<String, String> columnsStatistics() {
+    return columnsStatistics;
+  }
+
+  public List<String> metadataStatistics() {
+    return metadataStatistics;
+  }
+
+  public Long lastModifiedTime() {
+    return lastModifiedTime;
+  }
+
+  public Map<String, String> partitionKeys() {
+    return partitionKeys;
+  }
+
+  public String additionalMetadata() {
+    return additionalMetadata;
+  }
+
+  public String metadataIdentifier() {
+    return metadataIdentifier;
+  }
+
+  public String column() {
+    return column;
+  }
+
+  public List<String> locations() {
+    return locations;
+  }
+
+  public List<String> partitionValues() {
+    return partitionValues;
+  }
+
+  public String path() {
+    return path;
+  }
+
+  public Integer rowGroupIndex() {
+    return rowGroupIndex;
+  }
+
+  public Map<String, Float> hostAffinity() {
+    return hostAffinity;
+  }
+
+  public Builder toBuilder() {
+    return TableMetadataUnit.builder()
+      .storagePlugin(storagePlugin)
+      .workspace(workspace)
+      .tableName(tableName)
+      .owner(owner)
+      .tableType(tableType)
+      .metadataType(metadataType)
+      .metadataKey(metadataKey)
+      .location(location)
+      .interestingColumns(interestingColumns)
+      .schema(schema)
+      .columnsStatistics(columnsStatistics)
+      .metadataStatistics(metadataStatistics)
+      .lastModifiedTime(lastModifiedTime)
+      .partitionKeys(partitionKeys)
+      .additionalMetadata(additionalMetadata)
+      .metadataIdentifier(metadataIdentifier)
+      .column(column)
+      .locations(locations)
+      .partitionValues(partitionValues)
+      .path(path)
+      .rowGroupIndex(rowGroupIndex)
+      .hostAffinity(hostAffinity);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(storagePlugin, workspace, tableName, owner, tableType, metadataType,
+      metadataKey, location, interestingColumns, schema, columnsStatistics, metadataStatistics,
+      lastModifiedTime, partitionKeys, additionalMetadata, metadataIdentifier, column, locations,
+      partitionValues, path, rowGroupIndex, hostAffinity);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o)  {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    TableMetadataUnit that = (TableMetadataUnit) o;
+    return Objects.equals(storagePlugin, that.storagePlugin)
+      && Objects.equals(workspace, that.workspace)
+      && Objects.equals(tableName, that.tableName)
+      && Objects.equals(owner, that.owner)
+      && Objects.equals(tableType, that.tableType)
+      && Objects.equals(metadataType, that.metadataType)
+      && Objects.equals(metadataKey, that.metadataKey)
+      && Objects.equals(location, that.location)
+      && Objects.equals(interestingColumns, that.interestingColumns)
+      && Objects.equals(schema, that.schema)
+      && Objects.equals(columnsStatistics, that.columnsStatistics)
+      && Objects.equals(metadataStatistics, that.metadataStatistics)
+      && Objects.equals(lastModifiedTime, that.lastModifiedTime)
+      && Objects.equals(partitionKeys, that.partitionKeys)
+      && Objects.equals(additionalMetadata, that.additionalMetadata)
+      && Objects.equals(metadataIdentifier, that.metadataIdentifier)
+      && Objects.equals(column, that.column)
+      && Objects.equals(locations, that.locations)
+      && Objects.equals(partitionValues, that.partitionValues)
+      && Objects.equals(path, that.path)
+      && Objects.equals(rowGroupIndex, that.rowGroupIndex)
+      && Objects.equals(hostAffinity, that.hostAffinity);
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(",\n", TableMetadataUnit.class.getSimpleName() + "[", "]")
+      .add("storagePlugin=" + storagePlugin)
+      .add("workspace=" + workspace)
+      .add("tableName=" + tableName)
+      .add("owner=" + owner)
+      .add("tableType=" + tableType)
+      .add("metadataType=" + metadataType)
+      .add("metadataKey=" + metadataKey)
+      .add("location=" + location)
+      .add("interestingColumns=" + interestingColumns)
+      .add("schema=" + schema)
+      .add("columnsStatistics=" + columnsStatistics)
+      .add("metadataStatistics=" + metadataStatistics)
+      .add("lastModifiedTime=" + lastModifiedTime)
+      .add("partitionKeys=" + partitionKeys)
+      .add("additionalMetadata=" + additionalMetadata)
+      .add("metadataIdentifier=" + metadataIdentifier)
+      .add("column=" + column)
+      .add("locations=" + locations)
+      .add("partitionValues=" + partitionValues)
+      .add("path=" + path)
+      .add("rowGroupIndex=" + rowGroupIndex)
+      .add("hostAffinity=" + hostAffinity)
+      .toString();
+  }
+
+  public static class Builder {
+
+    private String storagePlugin;
+    private String workspace;
+    private String tableName;
+    private String owner;
+    private String tableType;
+    private String metadataType;
+    private String metadataKey;
+    private String location;
+    private List<String> interestingColumns;
+    private String schema;
+    private Map<String, String> columnsStatistics;
+    private List<String> metadataStatistics;
+    private Long lastModifiedTime;
+    private Map<String, String> partitionKeys;
+    private String additionalMetadata;
+    private String metadataIdentifier;
+    private String column;
+    private List<String> locations;
+    private List<String> partitionValues;
+    private String path;
+    private Integer rowGroupIndex;
+    private Map<String, Float> hostAffinity;
+
+    public Builder storagePlugin(String storagePlugin) {
+      this.storagePlugin = storagePlugin;
+      return this;
+    }
+
+    public Builder workspace(String workspace) {
+      this.workspace = workspace;
+      return this;
+    }
+
+    public Builder tableName(String tableName) {
+      this.tableName = tableName;
+      return this;
+    }
+
+    public Builder owner(String owner) {
+      this.owner = owner;
+      return this;
+    }
+
+    public Builder tableType(String tableType) {
+      this.tableType = tableType;
+      return this;
+    }
+
+    public Builder metadataType(String metadataType) {
+      this.metadataType = metadataType;
+      return this;
+    }
+
+    public Builder metadataKey(String metadataKey) {
+      this.metadataKey = metadataKey;
+      return this;
+    }
+
+    public Builder location(String location) {
+      this.location = location;
+      return this;
+    }
+
+    public Builder interestingColumns(List<String> interestingColumns) {
+      this.interestingColumns = interestingColumns;
+      return this;
+    }
+
+    public Builder schema(String schema) {
+      this.schema = schema;
+      return this;
+    }
+
+    public Builder columnsStatistics(Map<String, String> columnsStatistics) {
+      this.columnsStatistics = columnsStatistics;
+      return this;
+    }
+
+    public Builder metadataStatistics(List<String> metadataStatistics) {
+      this.metadataStatistics = metadataStatistics;
+      return this;
+    }
+
+    public Builder lastModifiedTime(Long lastModifiedTime) {
+      this.lastModifiedTime = lastModifiedTime;
+      return this;
+    }
+
+    public Builder partitionKeys(Map<String, String> partitionKeys) {
+      this.partitionKeys = partitionKeys;
+      return this;
+    }
+
+    public Builder additionalMetadata(String additionalMetadata) {
+      this.additionalMetadata = additionalMetadata;
+      return this;
+    }
+
+    public Builder metadataIdentifier(String metadataIdentifier) {
+      this.metadataIdentifier = metadataIdentifier;
+      return this;
+    }
+
+    public Builder column(String column) {
+      this.column = column;
+      return this;
+    }
+
+    public Builder locations(List<String> locations) {
+      this.locations = locations;
+      return this;
+    }
+
+    public Builder partitionValues(List<String> partitionValues) {
+      this.partitionValues = partitionValues;
+      return this;
+    }
+
+    public Builder path(String path) {
+      this.path = path;
+      return this;
+    }
+
+    public Builder rowGroupIndex(Integer rowGroupIndex) {
+      this.rowGroupIndex = rowGroupIndex;
+      return this;
+    }
+
+    public Builder hostAffinity(Map<String, Float> hostAffinity) {
+      this.hostAffinity = hostAffinity;
+      return this;
+    }
+
+    public TableMetadataUnit build() {
+      return new TableMetadataUnit(this);
+    }
+  }
+
+  /**
+   * Contains schema metadata, including lists of columns which belong to table, segment, file, row group
+   * or partition. Also provides unit class getters and its builder setters method handlers
+   * to optimize reflection calls.
+   */
+  public static class Schema {
+
+    private final List<String> tableColumns;
+    private final List<String> segmentColumns;
+    private final List<String> fileColumns;
+    private final List<String> rowGroupColumns;
+    private final List<String> partitionColumns;
+    private final Map<String, MethodHandle> unitGetters;
+    private final Map<String, MethodHandle> unitBuilderSetters;
+
+    private Schema(List<String> tableColumns,
+                   List<String> segmentColumns,
+                   List<String> fileColumns,
+                   List<String> rowGroupColumns,
+                   List<String> partitionColumns,
+                   Map<String, MethodHandle> unitGetters,
+                   Map<String, MethodHandle> unitBuilderSetters) {
+      this.tableColumns = tableColumns;
+      this.segmentColumns = segmentColumns;
+      this.fileColumns = fileColumns;
+      this.rowGroupColumns = rowGroupColumns;
+      this.partitionColumns = partitionColumns;
+      this.unitGetters = unitGetters;
+      this.unitBuilderSetters = unitBuilderSetters;
+    }
+
+    /**
+     * Obtains field information for the given unit class and its builder.
+     * Traverses through the list of unit class fields which are annotated with {@link MetastoreFieldDefinition}
+     * and creates instance of {@link Schema} class that holds unit class schema metadata.
+     * Assumes that given unit class and its builder getters and setters method names
+     * are the same as annotated fields names.
+     */
+    public static Schema of(Class<?> unitClass, Class<?> builderClass) {
+      List<String> tableColumns = new ArrayList<>();
+      List<String> segmentColumns = new ArrayList<>();
+      List<String> fileColumns = new ArrayList<>();
+      List<String> rowGroupColumns = new ArrayList<>();
+      List<String> partitionColumns = new ArrayList<>();
+      Map<String, MethodHandle> unitGetters = new HashMap<>();
+      Map<String, MethodHandle> unitBuilderSetters = new HashMap<>();
+
+      MethodHandles.Lookup gettersLookup = MethodHandles.publicLookup().in(unitClass);
+      MethodHandles.Lookup settersLookup = MethodHandles.publicLookup().in(builderClass);
+
+      for (Field field : unitClass.getDeclaredFields()) {
+        MetastoreFieldDefinition definition = field.getAnnotation(MetastoreFieldDefinition.class);
+        if (definition == null) {
+          continue;
+        }
+
+        String name = field.getName();
+        for (MetadataType scope : definition.scopes()) {
+          switch (scope) {
+            case TABLE:
+              tableColumns.add(name);
+              break;
+            case SEGMENT:
+              segmentColumns.add(name);
+              break;
+            case FILE:
+              fileColumns.add(name);
+              break;
+            case ROW_GROUP:
+              rowGroupColumns.add(name);
+              break;
+            case PARTITION:
+              partitionColumns.add(name);
+              break;
+            case ALL:
+              tableColumns.add(name);
+              segmentColumns.add(name);
+              fileColumns.add(name);
+              rowGroupColumns.add(name);
+              partitionColumns.add(name);
+              break;
+          }
+        }
+
+        Class<?> type = field.getType();
+        try {
+          MethodHandle getter = gettersLookup.findVirtual(unitClass, name, MethodType.methodType(type));
+          unitGetters.put(name, getter);
+          MethodHandle setter = settersLookup.findVirtual(builderClass, name, MethodType.methodType(builderClass, type));
+          unitBuilderSetters.put(name, setter);
+        } catch (ReflectiveOperationException e) {
+          throw new MetastoreException(String.format("Unable to init unit setter / getter method handlers " +
+              "for unit [%s] and its builder [%s] classes", unitClass.getSimpleName(), builderClass.getSimpleName()), e);
+        }
+      }
+
+      return new Schema(tableColumns, segmentColumns, fileColumns, rowGroupColumns, partitionColumns,
+        unitGetters, unitBuilderSetters);
+    }
+
+    public List<String> tableColumns() {
+      return tableColumns;
+    }
+
+    public List<String> segmentColumns() {
+      return segmentColumns;
+    }
+
+    public List<String> fileColumns() {
+      return fileColumns;
+    }
+
+    public List<String> rowGroupColumns() {
+      return rowGroupColumns;
+    }
+
+    public List<String> partitionColumns() {
+      return partitionColumns;
+    }
+
+    public Map<String, MethodHandle> unitGetters() {
+      return unitGetters;
+    }
+
+    public Map<String, MethodHandle> unitBuilderSetters() {
+      return unitBuilderSetters;
+    }
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/Tables.java
similarity index 60%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/Tables.java
index 7db95fb..74c1f95 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/Tables.java
@@ -15,19 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.components.tables;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.drill.metastore.operate.Operations;
 
 /**
- * A metadata which has specific location.
+ * Metastore Tables component implementation which allows
+ * to read / write tables metadata.
  */
-public interface LocationProvider {
+public interface Tables extends Operations<TableMetadataUnit> {
 
   /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
+   * @return new basic tables requests instance that provides methods to make most frequent
+   * calls to Metastore Tables in order to obtain metadata needed for analysis
    */
-  Path getPath();
+  default BasicTablesRequests basicRequests() {
+    return new BasicTablesRequests(this);
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/views/ViewMetadataUnit.java
similarity index 75%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/views/ViewMetadataUnit.java
index 7db95fb..e9e8dc2 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/views/ViewMetadataUnit.java
@@ -15,19 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
-
-import org.apache.hadoop.fs.Path;
+package org.apache.drill.metastore.components.views;
 
 /**
- * A metadata which has specific location.
+ * Class that represents one row in Drill Metastore Views
+ * which is a representation of metastore view metadata.
+ *
+ * //todo to be implemented
  */
-public interface LocationProvider {
-
-  /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
-   */
-  Path getPath();
+public class ViewMetadataUnit {
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/views/Views.java
similarity index 75%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/views/Views.java
index 7db95fb..63a3643 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/views/Views.java
@@ -15,19 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.components.views;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.drill.metastore.operate.Operations;
 
 /**
- * A metadata which has specific location.
+ * Metastore Views component implementation which allows
+ * to read / write views metadata.
  */
-public interface LocationProvider {
-
-  /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
-   */
-  Path getPath();
+public interface Views extends Operations<ViewMetadataUnit> {
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/config/MetastoreConfigConstants.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/config/MetastoreConfigConstants.java
new file mode 100644
index 0000000..f27eea5
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/config/MetastoreConfigConstants.java
@@ -0,0 +1,57 @@
+/*
+ * 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.metastore.config;
+
+/**
+ * Holds Metastore configuration files names and their properties names.
+ */
+public interface MetastoreConfigConstants {
+
+  /**
+   * Indicates file which provides default Metastore configuration, usually shipped in the project.
+   */
+  String DEFAULT_RESOURCE_FILE_NAME = "drill-metastore-default.conf";
+
+  /**
+   * Indicates file which provides configuration of the Metastore implementations,
+   * usually shipped with Metastore implementations.
+   */
+  String MODULE_RESOURCE_FILE_NAME = "drill-metastore-module.conf";
+
+  /**
+   * Indicates file which provides distribution specific Metastore configuration.
+   */
+  String DISTRIBUTION_RESOURCE_FILE_NAME = "drill-metastore-distrib.conf";
+
+  /**
+   * Indicates file which can override default Metastore configuration
+   * or Metastore implementations configuration.
+   */
+  String OVERRIDE_RESOURCE_FILE_NAME = "drill-metastore-override.conf";
+
+
+  /**
+   * Metastore configuration properties namespace.
+   */
+  String BASE = "drill.metastore.";
+
+  /**
+   * Indicates canonical class name of the Metastore implementation class.
+   */
+  String IMPLEMENTATION_CLASS = BASE + "implementation.class";
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/config/MetastoreConfigFileInfo.java
similarity index 51%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/config/MetastoreConfigFileInfo.java
index 7db95fb..689f575 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/config/MetastoreConfigFileInfo.java
@@ -15,19 +15,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
+package org.apache.drill.metastore.config;
 
-import org.apache.hadoop.fs.Path;
+import org.apache.drill.common.config.ConfigFileInfo;
 
 /**
- * A metadata which has specific location.
+ * Metastore {@link ConfigFileInfo} implementation which provides names of Metastore specific configuration files.
  */
-public interface LocationProvider {
+public class MetastoreConfigFileInfo implements ConfigFileInfo {
 
-  /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
-   */
-  Path getPath();
+  @Override
+  public String getDefaultFileName() {
+    return MetastoreConfigConstants.DEFAULT_RESOURCE_FILE_NAME;
+  }
+
+  @Override
+  public String getModuleFileName() {
+    return MetastoreConfigConstants.MODULE_RESOURCE_FILE_NAME;
+  }
+
+  @Override
+  public String getDistributionFileName() {
+    return MetastoreConfigConstants.DISTRIBUTION_RESOURCE_FILE_NAME;
+  }
+
+  @Override
+  public String getOverrideFileName() {
+    return MetastoreConfigConstants.OVERRIDE_RESOURCE_FILE_NAME;
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/exceptions/MetastoreException.java
similarity index 63%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/exceptions/MetastoreException.java
index 7db95fb..96516df 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/exceptions/MetastoreException.java
@@ -15,19 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
-
-import org.apache.hadoop.fs.Path;
+package org.apache.drill.metastore.exceptions;
 
 /**
- * A metadata which has specific location.
+ * Drill Metastore runtime exception to indicate that exception was caused by Drill Metastore.
+ * Drill Metastore implementations can use or extend it to throw Metastore specific exceptions.
  */
-public interface LocationProvider {
+public class MetastoreException extends RuntimeException {
+
+  public MetastoreException(String message, Throwable cause) {
+    super(message, cause);
+  }
 
-  /**
-   * Returns location of this metadata.
-   *
-   * @return metadata location
-   */
-  Path getPath();
+  public MetastoreException(String message) {
+    super(message);
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/DoubleExpressionPredicate.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/DoubleExpressionPredicate.java
new file mode 100644
index 0000000..04fab93
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/DoubleExpressionPredicate.java
@@ -0,0 +1,90 @@
+/*
+ * 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.metastore.expressions;
+
+import java.util.StringJoiner;
+
+/**
+ * Indicates double expression predicate implementations.
+ */
+public abstract class DoubleExpressionPredicate implements FilterExpression {
+
+  private final FilterExpression right;
+  private final Operator operator;
+  private final FilterExpression left;
+
+  protected DoubleExpressionPredicate(FilterExpression right, Operator operator, FilterExpression left) {
+    this.right = right;
+    this.operator = operator;
+    this.left = left;
+  }
+
+  public FilterExpression right() {
+    return right;
+  }
+
+  public FilterExpression left() {
+    return left;
+  }
+
+  @Override
+  public Operator operator() {
+    return operator;
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", DoubleExpressionPredicate.class.getSimpleName() + "[", "]")
+      .add("right=" + right)
+      .add("operator=" + operator)
+      .add("left=" + left)
+      .toString();
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#AND} operator expression:
+   * storagePlugin = 'dfs' and workspace = 'tmp'.
+   */
+  public static class And extends DoubleExpressionPredicate {
+
+    public And(FilterExpression right, FilterExpression left) {
+      super(right, Operator.AND, left);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#OR} operator expression:
+   * storagePlugin = 'dfs' or storagePlugin = 's3'.
+   */
+  public static class Or extends DoubleExpressionPredicate {
+
+    public Or(FilterExpression right, FilterExpression left) {
+      super(right, Operator.OR, left);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/FilterExpression.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/FilterExpression.java
new file mode 100644
index 0000000..45b75db
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/FilterExpression.java
@@ -0,0 +1,146 @@
+/*
+ * 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.metastore.expressions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+/**
+ * Interface which defines filter expression types by which Metastore data can be read or deleted.
+ */
+public interface FilterExpression {
+
+  Operator operator();
+
+  <T> T accept(Visitor<T> visitor);
+
+  /**
+   * Indicates list of supported operators that can be used in filter expressions.
+   */
+  enum Operator {
+    EQUAL,
+    NOT_EQUAL,
+    LESS_THAN,
+    LESS_THAN_OR_EQUAL,
+    GREATER_THAN,
+    GREATER_THAN_OR_EQUAL,
+    IN,
+    NOT_IN,
+    IS_NULL,
+    IS_NOT_NULL,
+    NOT,
+    AND,
+    OR
+  }
+
+  /**
+   * Transforms {@link FilterExpression} implementations into suitable
+   * for Metastore implementation representation.
+   * Is handy when needed to traverse through complex filter expressions.
+   *
+   * @param <T> type into which {@link FilterExpression} will be transformed
+   */
+  interface Visitor<T> {
+
+    T visit(SimplePredicate.Equal expression);
+    T visit(SimplePredicate.NotEqual expression);
+    T visit(SimplePredicate.LessThan expression);
+    T visit(SimplePredicate.LessThanOrEqual expression);
+    T visit(SimplePredicate.GreaterThan expression);
+    T visit(SimplePredicate.GreaterThanOrEqual expression);
+    T visit(ListPredicate.In expression);
+    T visit(ListPredicate.NotIn expression);
+    T visit(IsPredicate.IsNull expression);
+    T visit(IsPredicate.IsNotNull expression);
+    T visit(SingleExpressionPredicate.Not expression);
+    T visit(DoubleExpressionPredicate.And expression);
+    T visit(DoubleExpressionPredicate.Or expression);
+
+    default T visit(FilterExpression expression) {
+      throw new UnsupportedOperationException("Unsupported filter expression: " + expression);
+    }
+  }
+
+  static <T> FilterExpression equal(String reference, T value) {
+    return new SimplePredicate.Equal<>(reference, value);
+  }
+
+  static <T> FilterExpression notEqual(String reference, T value) {
+    return new SimplePredicate.NotEqual<>(reference, value);
+  }
+
+  static <T> FilterExpression lessThan(String reference, T value) {
+    return new SimplePredicate.LessThan<>(reference, value);
+  }
+
+  static <T> FilterExpression lessThanOrEqual(String reference, T value) {
+    return new SimplePredicate.LessThanOrEqual<>(reference, value);
+  }
+
+  static <T> FilterExpression greaterThan(String reference, T value) {
+    return new SimplePredicate.GreaterThan<>(reference, value);
+  }
+
+  static <T> FilterExpression greaterThanOrEqual(String reference, T value) {
+    return new SimplePredicate.GreaterThanOrEqual<>(reference, value);
+  }
+
+  static <T> FilterExpression in(String reference, List<T> values) {
+    return new ListPredicate.In<>(reference, values);
+  }
+
+  @SafeVarargs
+  static <T> FilterExpression in(String reference, T... values) {
+    return in(reference, Arrays.asList(values));
+  }
+
+  static <T> FilterExpression notIn(String reference, List<T> values) {
+    return new ListPredicate.NotIn<>(reference, values);
+  }
+
+  @SafeVarargs
+  static <T> FilterExpression notIn(String reference, T... values) {
+    return notIn(reference, Arrays.asList(values));
+  }
+
+  static FilterExpression isNull(String reference) {
+    return new IsPredicate.IsNull(reference);
+  }
+
+  static FilterExpression isNotNull(String reference) {
+    return new IsPredicate.IsNotNull(reference);
+  }
+
+  static FilterExpression not(FilterExpression expression) {
+    return new SingleExpressionPredicate.Not(expression);
+  }
+
+  static FilterExpression and(FilterExpression right, FilterExpression left) {
+    return new DoubleExpressionPredicate.And(right, left);
+  }
+
+  static FilterExpression and(FilterExpression right, FilterExpression left, FilterExpression... expressions) {
+    return Stream.of(expressions)
+      .reduce(and(right, left), FilterExpression::and);
+  }
+
+  static FilterExpression or(FilterExpression right, FilterExpression left) {
+    return new DoubleExpressionPredicate.Or(right, left);
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/IsPredicate.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/IsPredicate.java
new file mode 100644
index 0000000..d3a26e2
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/IsPredicate.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.metastore.expressions;
+
+import java.util.StringJoiner;
+
+/**
+ * Indicates IS predicate implementations.
+ */
+public abstract class IsPredicate implements FilterExpression {
+
+  private final String reference;
+  private final Operator operator;
+
+  protected IsPredicate(String reference, Operator operator) {
+    this.reference = reference;
+    this.operator = operator;
+  }
+
+  public String reference() {
+    return reference;
+  }
+
+  @Override
+  public Operator operator() {
+    return operator;
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", IsPredicate.class.getSimpleName() + "[", "]")
+      .add("reference=" + reference)
+      .add("operator=" + operator)
+      .toString();
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#IS_NULL} operator expression:
+   * storagePlugin is null.
+   */
+  public static class IsNull extends IsPredicate {
+
+    public IsNull(String reference) {
+      super(reference, Operator.IS_NULL);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#IS_NOT_NULL} operator expression:
+   * storagePlugin is not null.
+   */
+  public static class IsNotNull extends IsPredicate {
+
+    public IsNotNull(String reference) {
+      super(reference, Operator.IS_NOT_NULL);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/ListPredicate.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/ListPredicate.java
new file mode 100644
index 0000000..18a4433
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/ListPredicate.java
@@ -0,0 +1,97 @@
+/*
+ * 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.metastore.expressions;
+
+import java.util.List;
+import java.util.StringJoiner;
+
+/**
+ * Indicates list predicate implementations which have reference and list of values.
+ *
+ * @param <T> predicate value type
+ */
+public abstract class ListPredicate<T> implements FilterExpression {
+
+  private final String reference;
+  private final Operator operator;
+  private final List<T> values;
+
+  protected ListPredicate(String reference, Operator operator, List<T> values) {
+    this.reference = reference;
+    this.operator = operator;
+    this.values = values;
+  }
+
+  public String reference() {
+    return reference;
+  }
+
+  public List<T> values() {
+    return values;
+  }
+
+  @Override
+  public Operator operator() {
+    return operator;
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", ListPredicate.class.getSimpleName() + "[", "]")
+      .add("reference=" + reference)
+      .add("operator=" + operator)
+      .add("values=" + values)
+      .toString();
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#IN} operator expression:
+   * storagePlugin in ('dfs', 's3').
+   *
+   * @param <T> expression value type
+   */
+  public static class In<T> extends ListPredicate<T> {
+
+    public In(String reference, List<T> values) {
+      super(reference, Operator.IN, values);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#NOT_IN} operator expression:
+   * storagePlugin not in ('dfs', 's3').
+   *
+   * @param <T> expression value type
+   */
+  public static class NotIn<T> extends ListPredicate<T> {
+
+    public NotIn(String reference, List<T> values) {
+      super(reference, Operator.NOT_IN, values);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/SimplePredicate.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/SimplePredicate.java
new file mode 100644
index 0000000..d18e7cf
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/SimplePredicate.java
@@ -0,0 +1,168 @@
+/*
+ * 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.metastore.expressions;
+
+import java.util.StringJoiner;
+
+/**
+ * Indicates simple predicate implementations which have reference and one value.
+ *
+ * @param <T> predicate value type
+ */
+public abstract class SimplePredicate<T> implements FilterExpression {
+
+  private final String reference;
+  private final Operator operator;
+  private final T value;
+
+  protected SimplePredicate(String reference, Operator operator, T value) {
+    this.reference = reference;
+    this.operator = operator;
+    this.value = value;
+  }
+
+  public String reference() {
+    return reference;
+  }
+
+  public T value() {
+    return value;
+  }
+
+  @Override
+  public Operator operator() {
+    return operator;
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", SimplePredicate.class.getSimpleName() + "[", "]")
+      .add("reference=" + reference)
+      .add("operator=" + operator)
+      .add("value=" + value)
+      .toString();
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#EQUAL} operator expression:
+   * storagePlugin = 'dfs'.
+   *
+   * @param <T> expression value type
+   */
+  public static class Equal<T> extends SimplePredicate<T> {
+
+    public Equal(String reference, T value) {
+      super(reference, Operator.EQUAL, value);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#NOT_EQUAL} operator expression:
+   * storagePlugin != 'dfs'.
+   *
+   * @param <T> expression value type
+   */
+  public static class NotEqual<T> extends SimplePredicate<T> {
+
+    public NotEqual(String reference, T value) {
+      super(reference, Operator.NOT_EQUAL, value);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#LESS_THAN} operator expression:
+   * index < 1.
+   *
+   * @param <T> expression value type
+   */
+  public static class LessThan<T> extends SimplePredicate<T> {
+
+    public LessThan(String reference, T value) {
+      super(reference, Operator.LESS_THAN, value);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#LESS_THAN_OR_EQUAL} operator expression:
+   * index <= 1.
+   *
+   * @param <T> expression value type
+   */
+  public static class LessThanOrEqual<T> extends SimplePredicate<T> {
+
+    public LessThanOrEqual(String reference, T value) {
+      super(reference, Operator.LESS_THAN_OR_EQUAL, value);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#GREATER_THAN} operator expression:
+   * index > 1.
+   *
+   * @param <T> expression value type
+   */
+  public static class GreaterThan<T> extends SimplePredicate<T> {
+
+    public GreaterThan(String reference, T value) {
+      super(reference, Operator.GREATER_THAN, value);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#GREATER_THAN_OR_EQUAL} operator expression:
+   * index >= 1.
+   *
+   * @param <T> expression value type
+   */
+  public static class GreaterThanOrEqual<T> extends SimplePredicate<T> {
+
+    public GreaterThanOrEqual(String reference, T value) {
+      super(reference, Operator.GREATER_THAN_OR_EQUAL, value);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/SingleExpressionPredicate.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/SingleExpressionPredicate.java
new file mode 100644
index 0000000..6b7318a
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/SingleExpressionPredicate.java
@@ -0,0 +1,67 @@
+/*
+ * 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.metastore.expressions;
+
+import java.util.StringJoiner;
+
+/**
+ * Indicates single expression predicate implementations.
+ */
+public abstract class SingleExpressionPredicate implements FilterExpression {
+
+  private final FilterExpression expression;
+  private final Operator operator;
+
+  protected SingleExpressionPredicate(FilterExpression expression, Operator operator) {
+    this.expression = expression;
+    this.operator = operator;
+  }
+
+  public FilterExpression expression() {
+    return expression;
+  }
+
+  @Override
+  public Operator operator() {
+    return operator;
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", SingleExpressionPredicate.class.getSimpleName() + "[", "]")
+      .add("expression=" + expression)
+      .add("operator=" + operator)
+      .toString();
+  }
+
+  /**
+   * Indicates {@link FilterExpression.Operator#NOT} operator expression:
+   * not(storagePlugin = 'dfs').
+   */
+  public static class Not extends SingleExpressionPredicate {
+
+    public Not(FilterExpression expression) {
+      super(expression, Operator.NOT);
+    }
+
+    @Override
+    public <V> V accept(Visitor<V> visitor) {
+      return visitor.visit(this);
+    }
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseMetadata.java
index e66d75e..b9d71fe 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseMetadata.java
@@ -20,10 +20,11 @@ package org.apache.drill.metastore.metadata;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.metastore.util.SchemaPathUtils;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
 import org.apache.drill.metastore.statistics.ColumnStatistics;
 import org.apache.drill.metastore.statistics.StatisticsHolder;
 import org.apache.drill.metastore.statistics.StatisticsKind;
+import org.apache.drill.metastore.util.SchemaPathUtils;
 
 import java.util.Collection;
 import java.util.Map;
@@ -35,11 +36,15 @@ import java.util.stream.Collectors;
  * Common provider of tuple schema, column metadata, and statistics for table, partition, file or row group.
  */
 public abstract class BaseMetadata implements Metadata {
+
+  public static final long UNDEFINED_TIME = -1;
+
   protected final TableInfo tableInfo;
   protected final MetadataInfo metadataInfo;
   protected final TupleMetadata schema;
   protected final Map<SchemaPath, ColumnStatistics> columnsStatistics;
   protected final Map<String, StatisticsHolder> metadataStatistics;
+  protected final long lastModifiedTime;
 
   protected <T extends BaseMetadataBuilder<T>> BaseMetadata(BaseMetadataBuilder<T> builder) {
     this.tableInfo = builder.tableInfo;
@@ -51,6 +56,7 @@ public abstract class BaseMetadata implements Metadata {
             statistic -> statistic.getStatisticsKind().getName(),
             Function.identity(),
             (a, b) -> a.getStatisticsKind().isExact() ? a : b));
+    this.lastModifiedTime = builder.lastModifiedTime;
   }
 
   @Override
@@ -102,12 +108,53 @@ public abstract class BaseMetadata implements Metadata {
     return metadataInfo;
   }
 
+  /**
+   * Allows to check the time, when any files were modified.
+   * It is in Unix Timestamp, unit of measurement is millisecond.
+   * Undefined time value will be indicated using {@link #UNDEFINED_TIME} value.
+   *
+   * @return last modified time of files
+   */
+  public long getLastModifiedTime() {
+    return lastModifiedTime;
+  }
+
+  @Override
+  public TableMetadataUnit toMetadataUnit() {
+    TableMetadataUnit.Builder builder = TableMetadataUnit.builder();
+
+    tableInfo.toMetadataUnitBuilder(builder);
+    metadataInfo.toMetadataUnitBuilder(builder);
+
+    if (schema != null) {
+      builder.schema(schema.jsonString());
+    }
+
+    builder.columnsStatistics(columnsStatistics.entrySet().stream()
+      .collect(Collectors.toMap(
+        entry -> entry.getKey().toString(),
+        entry -> entry.getValue().jsonString(),
+        (o, n) -> n)));
+
+    builder.metadataStatistics(metadataStatistics.values().stream()
+      .map(StatisticsHolder::jsonString)
+      .collect(Collectors.toList()));
+
+    builder.lastModifiedTime(lastModifiedTime);
+
+    toMetadataUnitBuilder(builder);
+    return builder.build();
+  }
+
+  protected abstract void toMetadataUnitBuilder(TableMetadataUnit.Builder builder);
+
   public static abstract class BaseMetadataBuilder<T extends BaseMetadataBuilder<T>> {
     protected TableInfo tableInfo;
     protected MetadataInfo metadataInfo;
     protected TupleMetadata schema;
     protected Map<SchemaPath, ColumnStatistics> columnsStatistics;
     protected Collection<StatisticsHolder> metadataStatistics;
+    protected long lastModifiedTime = UNDEFINED_TIME;
 
     public T tableInfo(TableInfo tableInfo) {
       this.tableInfo = tableInfo;
@@ -134,6 +181,37 @@ public abstract class BaseMetadata implements Metadata {
       return self();
     }
 
+    public T lastModifiedTime(long lastModifiedTime) {
+      this.lastModifiedTime = lastModifiedTime;
+      return self();
+    }
+
+    public T metadataUnit(TableMetadataUnit unit) {
+      tableInfo(TableInfo.builder().metadataUnit(unit).build());
+      metadataInfo(MetadataInfo.builder().metadataUnit(unit).build());
+      schema(TupleMetadata.of(unit.schema()));
+
+      if (unit.columnsStatistics() != null) {
+        columnsStatistics(unit.columnsStatistics().entrySet().stream()
+          .collect(Collectors.toMap(
+            entry -> SchemaPath.parseFromString(entry.getKey()),
+            entry -> ColumnStatistics.of(entry.getValue()),
+            (o, n) -> n)));
+      }
+
+      if (unit.metadataStatistics() != null) {
+        metadataStatistics(unit.metadataStatistics().stream()
+          .map(StatisticsHolder::of)
+          .collect(Collectors.toList()));
+      }
+
+      if (unit.lastModifiedTime() != null) {
+        lastModifiedTime(unit.lastModifiedTime());
+      }
+
+      return metadataUnitInternal(unit);
+    }
+
     protected void checkRequiredValues() {
       Objects.requireNonNull(tableInfo, "tableInfo was not set");
       Objects.requireNonNull(metadataInfo, "metadataInfo was not set");
@@ -144,5 +222,7 @@ public abstract class BaseMetadata implements Metadata {
     public abstract BaseMetadata build();
 
     protected abstract T self();
+
+    protected abstract T metadataUnitInternal(TableMetadataUnit unit);
   }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseTableMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseTableMetadata.java
index 690b44a..ebcfc2a 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseTableMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseTableMetadata.java
@@ -18,6 +18,7 @@
 package org.apache.drill.metastore.metadata;
 
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
 import org.apache.drill.metastore.statistics.ColumnStatistics;
 import org.apache.drill.metastore.statistics.StatisticsHolder;
 import org.apache.hadoop.fs.Path;
@@ -26,16 +27,14 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.stream.Collectors;
 
 /**
  * Base implementation of {@link TableMetadata} interface.
  */
 public class BaseTableMetadata extends BaseMetadata implements TableMetadata {
 
-  public static final long NON_DEFINED_LAST_MODIFIED_TIME = -1;
-
   private final Path location;
-  private final long lastModifiedTime;
   private final Map<String, String> partitionKeys;
   private final List<SchemaPath> interestingColumns;
 
@@ -44,7 +43,6 @@ public class BaseTableMetadata extends BaseMetadata implements TableMetadata {
     this.location = builder.location;
     this.partitionKeys = builder.partitionKeys;
     this.interestingColumns = builder.interestingColumns;
-    this.lastModifiedTime = builder.lastModifiedTime;
   }
 
   public boolean isPartitionColumn(String fieldName) {
@@ -61,11 +59,6 @@ public class BaseTableMetadata extends BaseMetadata implements TableMetadata {
   }
 
   @Override
-  public long getLastModifiedTime() {
-    return lastModifiedTime;
-  }
-
-  @Override
   public List<SchemaPath> getInterestingColumns() {
     return interestingColumns;
   }
@@ -98,13 +91,25 @@ public class BaseTableMetadata extends BaseMetadata implements TableMetadata {
         .build();
   }
 
+  @Override
+  protected void toMetadataUnitBuilder(TableMetadataUnit.Builder builder) {
+    if (location != null) {
+      builder.location(location.toUri().getPath());
+    }
+    builder.partitionKeys(partitionKeys);
+    if (interestingColumns != null) {
+    builder.interestingColumns(interestingColumns.stream()
+      .map(SchemaPath::toString)
+      .collect(Collectors.toList()));
+    }
+  }
+
   public static BaseTableMetadataBuilder builder() {
     return new BaseTableMetadataBuilder();
   }
 
   public static class BaseTableMetadataBuilder extends BaseMetadataBuilder<BaseTableMetadataBuilder> {
     private Path location;
-    private long lastModifiedTime = NON_DEFINED_LAST_MODIFIED_TIME;
     private Map<String, String> partitionKeys;
     private List<SchemaPath> interestingColumns;
 
@@ -113,11 +118,6 @@ public class BaseTableMetadata extends BaseMetadata implements TableMetadata {
       return self();
     }
 
-    public BaseTableMetadataBuilder lastModifiedTime(long lastModifiedTime) {
-      this.lastModifiedTime = lastModifiedTime;
-      return self();
-    }
-
     public BaseTableMetadataBuilder partitionKeys(Map<String, String> partitionKeys) {
       this.partitionKeys = partitionKeys;
       return self();
@@ -144,5 +144,19 @@ public class BaseTableMetadata extends BaseMetadata implements TableMetadata {
     protected BaseTableMetadataBuilder self() {
       return this;
     }
+
+    @Override
+    protected BaseTableMetadataBuilder metadataUnitInternal(TableMetadataUnit unit) {
+      if (unit.location() != null) {
+        location(new Path(unit.location()));
+      }
+      partitionKeys(unit.partitionKeys());
+      if (unit.interestingColumns() != null) {
+        interestingColumns(unit.interestingColumns().stream()
+          .map(SchemaPath::parseFromString)
+          .collect(Collectors.toList()));
+      }
+      return self();
+    }
   }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/FileMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/FileMetadata.java
index f99b7eb..a8b5a72 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/FileMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/FileMetadata.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.metastore.metadata;
 
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
 import org.apache.hadoop.fs.Path;
 
 import java.util.Objects;
@@ -26,12 +27,10 @@ import java.util.Objects;
  */
 public class FileMetadata extends BaseMetadata implements LocationProvider {
   private final Path path;
-  private final long lastModifiedTime;
 
   private FileMetadata(FileMetadataBuilder builder) {
     super(builder);
     this.path = builder.path;
-    this.lastModifiedTime = builder.lastModifiedTime;
   }
 
   @Override
@@ -39,38 +38,29 @@ public class FileMetadata extends BaseMetadata implements LocationProvider {
     return path;
   }
 
-  /**
-   * Allows to check the time, when any files were modified.
-   * It is in Unix Timestamp, unit of measurement is millisecond.
-   *
-   * @return last modified time of files
-   */
-  public long getLastModifiedTime() {
-    return lastModifiedTime;
-  }
-
+  @Override
   public Path getLocation() {
     return path.getParent();
   }
 
+  @Override
+  protected void toMetadataUnitBuilder(TableMetadataUnit.Builder builder) {
+    builder.path(path.toUri().getPath());
+    builder.location(getLocation().toUri().getPath());
+  }
+
   public static FileMetadataBuilder builder() {
     return new FileMetadataBuilder();
   }
 
   public static class FileMetadataBuilder extends BaseMetadataBuilder<FileMetadataBuilder> {
     private Path path;
-    private long lastModifiedTime;
 
     public FileMetadataBuilder path(Path path) {
       this.path = path;
       return self();
     }
 
-    public FileMetadataBuilder lastModifiedTime(long lastModifiedTime) {
-      this.lastModifiedTime = lastModifiedTime;
-      return self();
-    }
-
     @Override
     protected void checkRequiredValues() {
       super.checkRequiredValues();
@@ -87,5 +77,13 @@ public class FileMetadata extends BaseMetadata implements LocationProvider {
     protected FileMetadataBuilder self() {
       return this;
     }
+
+    @Override
+    protected FileMetadataBuilder metadataUnitInternal(TableMetadataUnit unit) {
+      if (unit.path() != null) {
+        path(new Path(unit.path()));
+      }
+      return self();
+    }
   }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
index 7db95fb..023ceb2 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
@@ -25,9 +25,20 @@ import org.apache.hadoop.fs.Path;
 public interface LocationProvider {
 
   /**
-   * Returns location of this metadata.
+   * Returns path of the metadata.
+   * For files and row groups - full path to the file including file name.
+   * For segments - path to the segment directory.
    *
-   * @return metadata location
+   * @return metadata path
    */
   Path getPath();
+
+  /**
+   * Returns location of the metadata.
+   * For files and row groups - path to the parent directory they reside it.
+   * For segments - path to the segment directory.
+   *
+   * @return metadata location
+   */
+  Path getLocation();
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java
index 079ec42..6b40d95 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java
@@ -20,6 +20,7 @@ package org.apache.drill.metastore.metadata;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
 import org.apache.drill.metastore.statistics.ColumnStatistics;
 import org.apache.drill.metastore.statistics.StatisticsKind;
 
@@ -92,4 +93,12 @@ public interface Metadata {
   TableInfo getTableInfo();
 
   MetadataInfo getMetadataInfo();
+
+  /**
+   * Converts {@link Metadata} implementation into {@link TableMetadataUnit} instance
+   * which will be used to write data into Drill Metastore Tables.
+   *
+   * @return metadata unit instance
+   */
+  TableMetadataUnit toMetadataUnit();
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataInfo.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataInfo.java
index f59c99b..2eeb8e1 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataInfo.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataInfo.java
@@ -17,12 +17,17 @@
  */
 package org.apache.drill.metastore.metadata;
 
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+
+import java.util.Objects;
+import java.util.StringJoiner;
+
 /**
  * Class that specifies metadata type and metadata information
  * which will be used for obtaining specific metadata from metastore.
  *
  * For example, for table-level metadata, it will be
- * {@code MetadataInfo(MetadataType.TABLE, MetadataInfo.GENERAL_INFO_KEY, null)}.
+ * {@code MetadataInfo[MetadataType.TABLE, MetadataInfo.GENERAL_INFO_KEY, null]}.
  */
 public class MetadataInfo {
 
@@ -34,21 +39,94 @@ public class MetadataInfo {
   private final String key;
   private final String identifier;
 
-  public MetadataInfo(MetadataType type, String key, String identifier) {
-    this.type = type;
-    this.key = key;
-    this.identifier = identifier;
+  private MetadataInfo(MetadataInfoBuilder builder) {
+    this.type = builder.type;
+    this.key = builder.key;
+    this.identifier = builder.identifier;
   }
 
-  public MetadataType getType() {
+  public MetadataType type() {
     return type;
   }
 
-  public String getKey() {
+  public String key() {
     return key;
   }
 
-  public String getIdentifier() {
+  public String identifier() {
     return identifier;
   }
+
+  public void toMetadataUnitBuilder(TableMetadataUnit.Builder builder) {
+    if (type != null) {
+      builder.metadataType(type.name());
+    }
+    builder.metadataKey(key);
+    builder.metadataIdentifier(identifier);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(type, key, identifier);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    MetadataInfo that = (MetadataInfo) o;
+    return type == that.type
+      && Objects.equals(key, that.key)
+      && Objects.equals(identifier, that.identifier);
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", MetadataInfo.class.getSimpleName() + "[", "]")
+      .add("type=" + type)
+      .add("key=" + key)
+      .add("identifier=" + identifier)
+      .toString();
+  }
+
+  public static MetadataInfoBuilder builder() {
+    return new MetadataInfoBuilder();
+  }
+
+  public static class MetadataInfoBuilder {
+    private MetadataType type;
+    private String key;
+    private String identifier;
+
+    public MetadataInfoBuilder type(MetadataType type) {
+      this.type = type;
+      return this;
+    }
+
+    public MetadataInfoBuilder key(String key) {
+      this.key = key;
+      return this;
+    }
+
+    public MetadataInfoBuilder identifier(String identifier) {
+      this.identifier = identifier;
+      return this;
+    }
+
+    public MetadataInfoBuilder metadataUnit(TableMetadataUnit unit) {
+      type(MetadataType.fromValue(unit.metadataType()));
+      key(unit.metadataKey());
+      identifier(unit.metadataIdentifier());
+      return this;
+    }
+
+    public MetadataInfo build() {
+      Objects.requireNonNull(type, "type was not set");
+      return new MetadataInfo(this);
+    }
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java
index da813d7..34ab6e2 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.metastore.metadata;
 
+import java.util.stream.Stream;
+
 /**
  * Enum with possible types of metadata.
  */
@@ -57,5 +59,23 @@ public enum MetadataType {
   /**
    * Metadata type which helps to indicate that there is no overflow of metadata.
    */
-  NONE
+  NONE,
+
+  /**
+   * Metadata type which belongs to views.
+   */
+  VIEW;
+
+  /**
+   * Converts metadata type string representation into {@link MetadataType} instance.
+   *
+   * @param value metadata
+   * @return metadata type instance, null otherwise
+   */
+  public static MetadataType fromValue(String value) {
+    return Stream.of(values())
+      .filter(type -> type.name().equalsIgnoreCase(value))
+      .findAny()
+      .orElse(null);
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java
index 83fb5a2..6944ab0 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java
@@ -20,6 +20,7 @@ package org.apache.drill.metastore.metadata;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
 import org.apache.drill.metastore.statistics.ColumnStatistics;
 import org.apache.drill.metastore.statistics.StatisticsKind;
 
@@ -84,4 +85,9 @@ public class NonInterestingColumnsMetadata implements Metadata {
   public MetadataInfo getMetadataInfo() {
     return null;
   }
+
+  @Override
+  public TableMetadataUnit toMetadataUnit() {
+    return null;
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/PartitionMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/PartitionMetadata.java
index 938e5fd..bf14510 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/PartitionMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/PartitionMetadata.java
@@ -18,11 +18,13 @@
 package org.apache.drill.metastore.metadata;
 
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
 import org.apache.hadoop.fs.Path;
 
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 /**
  * Represents a metadata for the table part, which corresponds to the specific partition key.
@@ -31,14 +33,12 @@ public class PartitionMetadata extends BaseMetadata {
   private final SchemaPath column;
   private final List<String> partitionValues;
   private final Set<Path> locations;
-  private final long lastModifiedTime;
 
   private PartitionMetadata(PartitionMetadataBuilder builder) {
     super(builder);
     this.column = builder.column;
     this.partitionValues = builder.partitionValues;
     this.locations = builder.locations;
-    this.lastModifiedTime = builder.lastModifiedTime;
   }
 
   /**
@@ -59,20 +59,19 @@ public class PartitionMetadata extends BaseMetadata {
     return locations;
   }
 
-  /**
-   * Allows to check the time, when any files were modified.
-   * It is in Unix Timestamp, unit of measurement is millisecond.
-   *
-   * @return last modified time of files
-   */
-  public long getLastModifiedTime() {
-    return lastModifiedTime;
-  }
-
   public List<String> getPartitionValues() {
     return partitionValues;
   }
 
+  @Override
+  protected void toMetadataUnitBuilder(TableMetadataUnit.Builder builder) {
+    builder.column(column.toString());
+    builder.partitionValues(partitionValues);
+    builder.locations(locations.stream()
+      .map(location -> location.toUri().getPath())
+      .collect(Collectors.toList()));
+  }
+
   public static PartitionMetadataBuilder builder() {
     return new PartitionMetadataBuilder();
   }
@@ -81,18 +80,12 @@ public class PartitionMetadata extends BaseMetadata {
     private SchemaPath column;
     private List<String> partitionValues;
     private Set<Path> locations;
-    private long lastModifiedTime = BaseTableMetadata.NON_DEFINED_LAST_MODIFIED_TIME;
 
     public PartitionMetadataBuilder locations(Set<Path> locations) {
       this.locations = locations;
       return self();
     }
 
-    public PartitionMetadataBuilder lastModifiedTime(long lastModifiedTime) {
-      this.lastModifiedTime = lastModifiedTime;
-      return self();
-    }
-
     public PartitionMetadataBuilder partitionValues(List<String> partitionValues) {
       this.partitionValues = partitionValues;
       return self();
@@ -121,5 +114,17 @@ public class PartitionMetadata extends BaseMetadata {
     protected PartitionMetadataBuilder self() {
       return this;
     }
+
+    @Override
+    protected PartitionMetadataBuilder metadataUnitInternal(TableMetadataUnit unit) {
+      if (unit.locations() != null) {
+        locations(unit.locations().stream()
+          .map(Path::new)
+          .collect(Collectors.toSet()));
+      }
+      partitionValues(unit.partitionValues());
+      column(SchemaPath.parseFromString(unit.column()));
+      return self();
+    }
   }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/RowGroupMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/RowGroupMetadata.java
index af37d5e..787dd84 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/RowGroupMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/RowGroupMetadata.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.metastore.metadata;
 
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
 import org.apache.hadoop.fs.Path;
 
 import java.util.Map;
@@ -42,6 +43,7 @@ public class RowGroupMetadata extends BaseMetadata implements LocationProvider {
     return path;
   }
 
+  @Override
   public Path getLocation() {
     return path.getParent();
   }
@@ -64,13 +66,21 @@ public class RowGroupMetadata extends BaseMetadata implements LocationProvider {
     return hostAffinity;
   }
 
+  @Override
+  protected void toMetadataUnitBuilder(TableMetadataUnit.Builder builder) {
+    builder.hostAffinity(hostAffinity);
+    builder.rowGroupIndex(rowGroupIndex);
+    builder.path(path.toUri().getPath());
+    builder.location(getLocation().toUri().getPath());
+  }
+
   public static RowGroupMetadataBuilder builder() {
     return new RowGroupMetadataBuilder();
   }
 
   public static class RowGroupMetadataBuilder extends BaseMetadataBuilder<RowGroupMetadataBuilder> {
     private Map<String, Float> hostAffinity;
-    private int rowGroupIndex;
+    private Integer rowGroupIndex;
     private Path path;
 
     public RowGroupMetadataBuilder hostAffinity(Map<String, Float> hostAffinity) {
@@ -91,6 +101,8 @@ public class RowGroupMetadata extends BaseMetadata implements LocationProvider {
     @Override
     protected void checkRequiredValues() {
       super.checkRequiredValues();
+      Objects.requireNonNull(rowGroupIndex, "rowGroupIndex was not set");
+      Objects.requireNonNull(hostAffinity, "hostAffinity was not set");
       Objects.requireNonNull(path, "path was not set");
     }
 
@@ -104,5 +116,15 @@ public class RowGroupMetadata extends BaseMetadata implements LocationProvider {
     protected RowGroupMetadataBuilder self() {
       return this;
     }
+
+    @Override
+    protected RowGroupMetadataBuilder metadataUnitInternal(TableMetadataUnit unit) {
+      hostAffinity(unit.hostAffinity());
+      rowGroupIndex(unit.rowGroupIndex());
+      if (unit.path() != null) {
+        path(new Path(unit.path()));
+      }
+      return self();
+    }
   }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/SegmentMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/SegmentMetadata.java
index 9a85b0e..9b30792 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/SegmentMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/SegmentMetadata.java
@@ -18,10 +18,13 @@
 package org.apache.drill.metastore.metadata;
 
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
 import org.apache.hadoop.fs.Path;
 
 import java.util.List;
 import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
 
 /**
  * Metadata which corresponds to the segment level of table.
@@ -30,8 +33,7 @@ public class SegmentMetadata extends BaseMetadata implements LocationProvider {
   private final SchemaPath column;
   private final Path path;
   private final List<String> partitionValues;
-  private final List<Path> locations;
-  private final long lastModifiedTime;
+  private final Set<Path> locations;
 
   private SegmentMetadata(SegmentMetadataBuilder builder) {
     super(builder);
@@ -39,33 +41,41 @@ public class SegmentMetadata extends BaseMetadata implements LocationProvider {
     this.path = builder.path;
     this.partitionValues = builder.partitionValues;
     this.locations = builder.locations;
-    this.lastModifiedTime = builder.lastModifiedTime;
   }
 
-  public SchemaPath getSegmentColumn() {
+  public SchemaPath getColumn() {
     return column;
   }
 
+  @Override
   public Path getPath() {
     return path;
   }
 
+  @Override
+  public Path getLocation() {
+    return path;
+  }
+
   public List<String> getPartitionValues() {
     return partitionValues;
   }
 
-  public List<Path> getLocations() {
+  public Set<Path> getLocations() {
     return locations;
   }
 
-  /**
-   * Allows to check the time, when any files were modified.
-   * It is in Unix Timestamp, unit of measurement is millisecond.
-   *
-   * @return last modified time of files
-   */
-  public long getLastModifiedTime() {
-    return lastModifiedTime;
+  @Override
+  protected void toMetadataUnitBuilder(TableMetadataUnit.Builder builder) {
+    if (column != null) {
+      builder.column(column.toString());
+    }
+    builder.path(path.toUri().getPath());
+    builder.location(getLocation().toUri().getPath());
+    builder.partitionValues(partitionValues);
+    builder.locations(locations.stream()
+      .map(location -> location.toUri().getPath())
+      .collect(Collectors.toList()));
   }
 
   public static SegmentMetadataBuilder builder() {
@@ -76,10 +86,9 @@ public class SegmentMetadata extends BaseMetadata implements LocationProvider {
     private SchemaPath column;
     private List<String> partitionValues;
     private Path path;
-    private List<Path> locations;
-    private long lastModifiedTime = BaseTableMetadata.NON_DEFINED_LAST_MODIFIED_TIME;
+    private Set<Path> locations;
 
-    public SegmentMetadataBuilder locations(List<Path> locations) {
+    public SegmentMetadataBuilder locations(Set<Path> locations) {
       this.locations = locations;
       return self();
     }
@@ -89,11 +98,6 @@ public class SegmentMetadata extends BaseMetadata implements LocationProvider {
       return self();
     }
 
-    public SegmentMetadataBuilder lastModifiedTime(long lastModifiedTime) {
-      this.lastModifiedTime = lastModifiedTime;
-      return self();
-    }
-
     public SegmentMetadataBuilder partitionValues(List<String> partitionValues) {
       this.partitionValues = partitionValues;
       return self();
@@ -107,10 +111,8 @@ public class SegmentMetadata extends BaseMetadata implements LocationProvider {
     @Override
     protected void checkRequiredValues() {
       super.checkRequiredValues();
-      Objects.requireNonNull(column, "column was not set");
-      Objects.requireNonNull(partitionValues, "partitionValues were not set");
-      Objects.requireNonNull(locations, "locations were not set");
       Objects.requireNonNull(path, "path was not set");
+      Objects.requireNonNull(locations, "locations were not set");
     }
 
     @Override
@@ -123,5 +125,20 @@ public class SegmentMetadata extends BaseMetadata implements LocationProvider {
     protected SegmentMetadataBuilder self() {
       return this;
     }
+
+    @Override
+    protected SegmentMetadataBuilder metadataUnitInternal(TableMetadataUnit unit) {
+      if (unit.locations() != null) {
+        locations(unit.locations().stream()
+          .map(Path::new)
+          .collect(Collectors.toSet()));
+      }
+      if (unit.path() != null) {
+        path(new Path(unit.path()));
+      }
+      partitionValues(unit.partitionValues());
+      column(SchemaPath.parseFromString(unit.column()));
+      return self();
+    }
   }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableInfo.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableInfo.java
index 3e73851..1212d2b 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableInfo.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableInfo.java
@@ -17,6 +17,12 @@
  */
 package org.apache.drill.metastore.metadata;
 
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.expressions.FilterExpression;
+
+import java.util.Objects;
+import java.util.StringJoiner;
+
 /**
  * General table information.
  */
@@ -30,6 +36,10 @@ public class TableInfo {
       .owner(UNKNOWN)
       .build();
 
+  public static final String STORAGE_PLUGIN = "storagePlugin";
+  public static final String WORKSPACE = "workspace";
+  public static final String TABLE_NAME = "tableName";
+
   private final String storagePlugin;
   private final String workspace;
   private final String name;
@@ -44,26 +54,73 @@ public class TableInfo {
     this.owner = builder.owner;
   }
 
-  public String getStoragePlugin() {
+  public String storagePlugin() {
     return storagePlugin;
   }
 
-  public String getWorkspace() {
+  public String workspace() {
     return workspace;
   }
 
-  public String getName() {
+  public String name() {
     return name;
   }
 
-  public String getType() {
+  public String type() {
     return type;
   }
 
-  public String getOwner() {
+  public String owner() {
     return owner;
   }
 
+  public FilterExpression toFilter() {
+    FilterExpression storagePluginFilter = FilterExpression.equal(STORAGE_PLUGIN, storagePlugin);
+    FilterExpression workspaceFilter = FilterExpression.equal(WORKSPACE, workspace);
+    FilterExpression tableNameFilter = FilterExpression.equal(TABLE_NAME, name);
+    return FilterExpression.and(storagePluginFilter, workspaceFilter, tableNameFilter);
+  }
+
+  public void toMetadataUnitBuilder(TableMetadataUnit.Builder builder) {
+    builder.storagePlugin(storagePlugin);
+    builder.workspace(workspace);
+    builder.tableName(name);
+    builder.tableType(type);
+    builder.owner(owner);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(storagePlugin, workspace, name, type, owner);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    TableInfo tableInfo = (TableInfo) o;
+    return Objects.equals(storagePlugin, tableInfo.storagePlugin)
+      && Objects.equals(workspace, tableInfo.workspace)
+      && Objects.equals(name, tableInfo.name)
+      && Objects.equals(type, tableInfo.type)
+      && Objects.equals(owner, tableInfo.owner);
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(", ", TableInfo.class.getSimpleName() + "[", "]")
+      .add("storagePlugin=" + storagePlugin)
+      .add("workspace=" + workspace)
+      .add("name=" + name)
+      .add("type=" + type)
+      .add("owner=" + owner)
+      .toString();
+  }
+
   public static TableInfoBuilder builder() {
     return new TableInfoBuilder();
   }
@@ -100,7 +157,18 @@ public class TableInfo {
       return this;
     }
 
+    public TableInfoBuilder metadataUnit(TableMetadataUnit unit) {
+      return storagePlugin(unit.storagePlugin())
+        .workspace(unit.workspace())
+        .name(unit.tableName())
+        .type(unit.tableType())
+        .owner(unit.owner());
+    }
+
     public TableInfo build() {
+      Objects.requireNonNull(storagePlugin, "storagePlugin was not set");
+      Objects.requireNonNull(workspace, "workspace was not set");
+      Objects.requireNonNull(name, "name was not set");
       return new TableInfo(this);
     }
 
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/operate/Metadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/operate/Metadata.java
new file mode 100644
index 0000000..4630d40
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/operate/Metadata.java
@@ -0,0 +1,64 @@
+/*
+ * 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.metastore.operate;
+
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Provides Metastore component implementation metadata,
+ * including information about versioning support if any
+ * and current properties applicable to the Metastore component instance.
+ */
+public interface Metadata {
+
+  int UNDEFINED = -1;
+
+  /**
+   * Indicates if Metastore component supports versioning,
+   * i.e. Metastore component version is changed each time write operation is executed.
+   *
+   * @return true if Metastore component supports versioning, false otherwise
+   */
+  boolean supportsVersioning();
+
+  /**
+   * Depending on Metastore component implementation, it may have version which can be used to determine
+   * if anything has changed during last call to the Metastore component.
+   * If Metastore component implementation, supports versioning,
+   * version is changed each time Metastore component data has changed.
+   * {@link #supportsVersioning()} indicates if Metastore component supports versioning.
+   * If versioning is not supported, {@link #UNDEFINED} is returned.
+   *
+   * @return current metastore version
+   */
+  default long version() {
+    return UNDEFINED;
+  }
+
+  /**
+   * Depending on Metastore component implementation, it may have properties.
+   * If Metastore component supports properties, map with properties names and values are returned,
+   * otherwise empty map is returned.
+   *
+   * @return Metastore component implementation properties
+   */
+  default Map<String, String> properties() {
+    return Collections.emptyMap();
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/operate/Modify.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/operate/Modify.java
new file mode 100644
index 0000000..2b56662
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/operate/Modify.java
@@ -0,0 +1,74 @@
+/*
+ * 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.metastore.operate;
+
+import org.apache.drill.metastore.expressions.FilterExpression;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Drill Metastore Modify interface contains methods to be implemented in order
+ * to provide modify functionality in the Metastore component.
+ *
+ * @param <T> component unit type
+ */
+public interface Modify<T> {
+
+  /**
+   * Adds overwrite operation for the Metastore component. For Metastore Tables compoenent,
+   * can be used to add new table data or replace partially / fully existing.
+   * For example, if one of the table segments has changed,
+   * all this segment data and table general information must be replaced with updated data.
+   * Thus provided units must include updated data, filter by which existing data will be overwritten
+   * will be determined based on given data.
+   *
+   * @param units component units to be overwritten
+   * @return current instance of Modify interface implementation
+   */
+  Modify<T> overwrite(List<T> units);
+
+  default Modify<T> overwrite(T... units) {
+    return overwrite(Arrays.asList(units));
+  }
+
+  /**
+   * Adds delete operation for the Metastore component based on the given filter expression.
+   * For example for Metastore Tables component, if table has two segments
+   * and data for one of the segments needs to be deleted.
+   * Thus filter must be based on unique identifier of the table's top-level segment:
+   * storagePlugin = 'dfs' and workspace = 'tmp' and tableName = 'nation' and metadataKey = 'part_int=3'
+   *
+   * @param filter filter expression
+   * @return current instance of Modify interface implementation
+   */
+  Modify<T> delete(FilterExpression filter);
+
+  /**
+   * Deletes all data from the Metastore component.
+   *
+   * @return current instance of Modify interface implementation
+   */
+  Modify<T> purge();
+
+  /**
+   * Executes list of provided metastore operations in one transaction if Metastore implementation
+   * supports transactions, otherwise executes operations consecutively.
+   */
+  void execute();
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/operate/Operations.java
similarity index 62%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/operate/Operations.java
index 7db95fb..dd6a00e 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/operate/Operations.java
@@ -15,19 +15,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore.metadata;
-
-import org.apache.hadoop.fs.Path;
+package org.apache.drill.metastore.operate;
 
 /**
- * A metadata which has specific location.
+ * Each Metastore component must provide mechanisms which return
+ * component metadata, allow reading / writing data from / into Metastore.
+ *
+ * @param <T> component unit type
  */
-public interface LocationProvider {
+public interface Operations<T> {
+
+  /**
+   * @return component metadata instance
+   */
+  Metadata metadata();
+
... 1358 lines suppressed ...