You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by vo...@apache.org on 2019/06/25 08:39:23 UTC

[drill] branch master updated: DRILL-7271: Refactor Metadata interfaces and classes to contain all needed information for the File based Metastore

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

volodymyr 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 dc865e8  DRILL-7271: Refactor Metadata interfaces and classes to contain all needed information for the File based Metastore
dc865e8 is described below

commit dc865e80d5cd67eea95e0a509ce5efd1ee521385
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Wed Jun 19 19:31:06 2019 +0300

    DRILL-7271: Refactor Metadata interfaces and classes to contain all needed information for the File based Metastore
---
 .../store/hive/HiveDrillNativeParquetScan.java     |  11 +-
 exec/java-exec/pom.xml                             |   5 -
 .../drill/exec/expr/ComparisonPredicate.java       |  22 +-
 .../exec/expr/ExpressionTreeMaterializer.java      |   2 +-
 .../org/apache/drill/exec/expr/IsPredicate.java    |  42 +--
 .../apache/drill/exec/expr/StatisticsProvider.java | 195 +++++------
 .../exec/physical/base/AbstractGroupScan.java      |   3 +-
 .../base/AbstractGroupScanWithMetadata.java        | 368 +++++++++++++++------
 .../apache/drill/exec/physical/base/GroupScan.java |   3 +-
 .../impl/statistics/AbstractMergedStatistic.java   |   1 +
 .../impl/statistics/AvgWidthMergedStatistic.java   |   1 +
 .../impl/statistics/CntDupsMergedStatistic.java    |   1 +
 .../impl/statistics/ColTypeMergedStatistic.java    |   1 +
 .../impl/statistics/ColumnMergedStatistic.java     |   1 +
 .../impl/statistics/HLLMergedStatistic.java        |   1 +
 .../impl/statistics/MergedStatisticFactory.java    |   2 +
 .../impl/statistics/NDVMergedStatistic.java        |   1 +
 .../impl/statistics/NNRowCountMergedStatistic.java |   1 +
 .../impl/statistics/RowCountMergedStatistic.java   |   1 +
 .../impl/statistics/StatisticsAggBatch.java        |   1 +
 .../impl/statistics/StatisticsMergeBatch.java      |   1 +
 .../impl/statistics/TDigestMergedStatistic.java    |   1 +
 .../drill/exec/planner/common/DrillRelOptUtil.java |   6 +-
 .../drill/exec/planner/common/DrillStatsTable.java |  43 +--
 .../drill/exec/planner/common/HistogramUtils.java  |   1 +
 .../planner/common/NumericEquiDepthHistogram.java  |   5 +-
 .../planner/cost/DrillRelMdDistinctRowCount.java   |  10 +-
 .../exec/planner/cost/DrillRelMdRowCount.java      |   6 +-
 .../exec/planner/cost/DrillRelMdSelectivity.java   |  24 +-
 .../logical/ConvertCountToDirectScanRule.java      |   6 +-
 .../drill/exec/planner/logical/DrillTable.java     |   6 +-
 .../exec/planner/logical/DynamicDrillTable.java    |   2 +-
 .../drill/exec/planner/physical/AnalyzePrule.java  |   2 +-
 .../physical/ConvertCountToDirectScanPrule.java    |   8 +-
 .../drill/exec/planner/sql/SqlConverter.java       |   4 +-
 .../drill/exec/store/AbstractStoragePlugin.java    |   2 +-
 .../apache/drill/exec/store/ColumnExplorer.java    |   2 +-
 .../org/apache/drill/exec/store/StoragePlugin.java |   2 +-
 .../drill/exec/store/dfs/FileSystemPlugin.java     |   2 +-
 .../apache/drill/exec/store/dfs/FormatPlugin.java  |   2 +-
 .../exec/store/dfs/WorkspaceSchemaFactory.java     |   4 +-
 .../exec/store/dfs/easy/EasyFormatPlugin.java      |   2 +-
 .../drill/exec/store/dfs/easy/EasyGroupScan.java   |   8 +-
 .../SimpleFileTableMetadataProviderBuilder.java    |   5 +-
 .../easy/json/JsonStatisticsRecordWriter.java      |   2 +-
 .../exec/store/easy/text/TextFormatPlugin.java     |   2 +-
 .../store/parquet/AbstractParquetGroupScan.java    | 213 +++++++-----
 .../parquet/AbstractParquetScanBatchCreator.java   |   2 +-
 .../store/parquet/BaseParquetMetadataProvider.java | 251 ++++++++++----
 .../exec/store/parquet/FilterEvaluatorUtils.java   |  18 +-
 .../ParquetFileTableMetadataProviderBuilder.java   |   4 +-
 .../exec/store/parquet/ParquetFormatPlugin.java    |   2 +-
 .../drill/exec/store/parquet/ParquetGroupScan.java |  17 +-
 .../store/parquet/ParquetGroupScanStatistics.java  |  39 +--
 .../exec/store/parquet/ParquetReaderUtility.java   |  52 ++-
 .../parquet/ParquetTableMetadataProviderImpl.java  |   6 +-
 .../store/parquet/ParquetTableMetadataUtils.java   | 276 +++++-----------
 .../FileSystemMetadataProviderManager.java         |   4 +-
 .../MetadataProviderManager.java                   |   4 +-
 .../ParquetMetadataProvider.java                   |  14 +-
 .../ParquetTableMetadataProvider.java              |   2 +-
 .../SimpleFileTableMetadataProvider.java           |  50 +--
 .../store/parquet/TestParquetFilterPushDown.java   |  34 +-
 metastore/file-metadata/pom.xml                    |  41 ---
 .../org/apache/drill/metastore/FileMetadata.java   | 114 -------
 .../apache/drill/metastore/FileTableMetadata.java  | 155 ---------
 .../apache/drill/metastore/ColumnStatistics.java   |  82 -----
 .../drill/metastore/ColumnStatisticsImpl.java      |  83 -----
 .../drill/metastore/ColumnStatisticsKind.java      | 184 -----------
 .../apache/drill/metastore/PartitionMetadata.java  | 135 --------
 .../apache/drill/metastore/RowGroupMetadata.java   | 118 -------
 .../drill/metastore/TableStatisticsKind.java       | 122 -------
 .../drill/metastore/metadata/BaseMetadata.java     | 148 +++++++++
 .../metastore/metadata/BaseTableMetadata.java      | 148 +++++++++
 .../drill/metastore/metadata/FileMetadata.java     |  91 +++++
 .../metastore/{ => metadata}/LocationProvider.java |   4 +-
 .../{BaseMetadata.java => metadata/Metadata.java}  |  16 +-
 .../drill/metastore/metadata/MetadataInfo.java     |  54 +++
 .../MetadataType.java}                             |  51 ++-
 .../NonInterestingColumnsMetadata.java             |  29 +-
 .../metastore/metadata/PartitionMetadata.java      | 125 +++++++
 .../drill/metastore/metadata/RowGroupMetadata.java | 108 ++++++
 .../drill/metastore/metadata/SegmentMetadata.java  | 127 +++++++
 .../apache/drill/metastore/metadata/TableInfo.java | 108 ++++++
 .../metastore/{ => metadata}/TableMetadata.java    |  12 +-
 .../metastore/metadata}/TableMetadataProvider.java |  22 +-
 .../metadata}/TableMetadataProviderBuilder.java    |   2 +-
 .../metastore/statistics/BaseStatisticsKind.java   |  44 +--
 .../CollectableColumnStatisticsKind.java           |   4 +-
 .../CollectableTableStatisticsKind.java            |   8 +-
 .../metastore/statistics/ColumnStatistics.java     | 179 ++++++++++
 .../metastore/statistics/ColumnStatisticsKind.java | 172 ++++++++++
 .../statistics}/ExactStatisticsConstants.java      |   2 +-
 .../drill/metastore/statistics}/Histogram.java     |  15 +-
 .../impl => metastore}/statistics/Statistic.java   |   2 +-
 .../metastore/statistics/StatisticsHolder.java     |  76 +++++
 .../metastore/{ => statistics}/StatisticsKind.java |  22 +-
 .../metastore/statistics/TableStatisticsKind.java  | 116 +++++++
 .../util}/SchemaPathUtils.java                     |   4 +-
 .../drill/metastore/util/TableMetadataUtils.java   | 135 ++++++++
 .../metastore/metadata/MetadataSerDeTest.java      | 152 +++++++++
 metastore/pom.xml                                  |   1 -
 102 files changed, 2895 insertions(+), 1925 deletions(-)

diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
index 94014fe..710ab7f 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
@@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.parquet.ParquetReaderConfig;
-import org.apache.drill.metastore.LocationProvider;
+import org.apache.drill.metastore.metadata.LocationProvider;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.LogicalExpression;
@@ -211,14 +211,14 @@ public class HiveDrillNativeParquetScan extends AbstractParquetGroupScan {
 
   @Override
   protected List<String> getPartitionValues(LocationProvider locationProvider) {
-    return hivePartitionHolder.get(locationProvider.getLocation());
+    return hivePartitionHolder.get(locationProvider.getPath());
   }
 
   /**
    * Implementation of RowGroupScanFilterer which uses {@link HiveDrillNativeParquetScanFilterer} as source and
    * builds {@link HiveDrillNativeParquetScanFilterer} instance with filtered metadata.
    */
-  private class HiveDrillNativeParquetScanFilterer extends RowGroupScanFilterer {
+  private class HiveDrillNativeParquetScanFilterer extends RowGroupScanFilterer<HiveDrillNativeParquetScanFilterer> {
 
     public HiveDrillNativeParquetScanFilterer(HiveDrillNativeParquetScan source) {
       super(source);
@@ -228,5 +228,10 @@ public class HiveDrillNativeParquetScan extends AbstractParquetGroupScan {
     protected AbstractParquetGroupScan getNewScan() {
       return new HiveDrillNativeParquetScan((HiveDrillNativeParquetScan) source);
     }
+
+    @Override
+    protected HiveDrillNativeParquetScanFilterer self() {
+      return this;
+    }
   }
 }
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 366b174..6387960 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -292,11 +292,6 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.apache.drill.metastore</groupId>
-      <artifactId>drill-file-metastore-plugin</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
       <groupId>com.beust</groupId>
       <artifactId>jcommander</artifactId>
       <version>1.30</version>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ComparisonPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ComparisonPredicate.java
index 43729e0..fa51780 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ComparisonPredicate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ComparisonPredicate.java
@@ -23,8 +23,8 @@ import org.apache.drill.common.expression.visitors.ExprVisitor;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
 import org.apache.drill.exec.expr.stat.RowsMatch;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -86,12 +86,13 @@ public class ComparisonPredicate<C extends Comparable<C>> extends LogicalExpress
    * where Column1 and Column2 are from same parquet table.
    */
   @Override
+  @SuppressWarnings("unchecked")
   public RowsMatch matches(StatisticsProvider<C> evaluator) {
-    ColumnStatistics<C> leftStat = left.accept(evaluator, null);
+    ColumnStatistics leftStat = left.accept(evaluator, null);
     if (IsPredicate.isNullOrEmpty(leftStat)) {
       return RowsMatch.SOME;
     }
-    ColumnStatistics<C> rightStat = right.accept(evaluator, null);
+    ColumnStatistics rightStat = right.accept(evaluator, null);
     if (IsPredicate.isNullOrEmpty(rightStat)) {
       return RowsMatch.SOME;
     }
@@ -126,14 +127,13 @@ public class ComparisonPredicate<C extends Comparable<C>> extends LogicalExpress
    * @param scale adjustment scale
    * @return adjusted statistics
    */
-  @SuppressWarnings("unchecked")
-  private ColumnStatistics<C> adjustDecimalStatistics(ColumnStatistics<C> statistics, int scale) {
-    BigInteger min = new BigDecimal((BigInteger) statistics.getStatistic(ColumnStatisticsKind.MIN_VALUE))
+  private ColumnStatistics adjustDecimalStatistics(ColumnStatistics<BigInteger> statistics, int scale) {
+    BigInteger min = new BigDecimal(ColumnStatisticsKind.MIN_VALUE.getValueStatistic(statistics))
         .setScale(scale, RoundingMode.HALF_UP).unscaledValue();
-    BigInteger max = new BigDecimal((BigInteger) statistics.getStatistic(ColumnStatisticsKind.MAX_VALUE))
+    BigInteger max = new BigDecimal(ColumnStatisticsKind.MAX_VALUE.getValueStatistic(statistics))
         .setScale(scale, RoundingMode.HALF_UP).unscaledValue();
 
-    return new StatisticsProvider.MinMaxStatistics(min, max, Comparator.nullsFirst(Comparator.naturalOrder()));
+    return StatisticsProvider.getColumnStatistics(min, max, ColumnStatisticsKind.NULLS_COUNT.getFrom(statistics), TypeProtos.MinorType.VARDECIMAL);
   }
 
   /**
@@ -188,11 +188,11 @@ public class ComparisonPredicate<C extends Comparable<C>> extends LogicalExpress
   }
 
   static <C> C getMaxValue(ColumnStatistics<C> leftStat) {
-    return leftStat.getValueStatistic(ColumnStatisticsKind.MAX_VALUE);
+    return ColumnStatisticsKind.MAX_VALUE.getValueStatistic(leftStat);
   }
 
   static <C> C getMinValue(ColumnStatistics<C> leftStat) {
-    return leftStat.getValueStatistic(ColumnStatisticsKind.MIN_VALUE);
+    return ColumnStatisticsKind.MIN_VALUE.getValueStatistic(leftStat);
   }
 
   /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index 9d2424a..8bb2d9e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -27,7 +27,7 @@ import java.util.Optional;
 import java.util.Queue;
 
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.SchemaPathUtils;
+import org.apache.drill.metastore.util.SchemaPathUtils;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/IsPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/IsPredicate.java
index 37a8d0d..0a7667d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/IsPredicate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/IsPredicate.java
@@ -18,9 +18,9 @@
 package org.apache.drill.exec.expr;
 
 import org.apache.drill.exec.expr.stat.RowsMatch;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.Statistic;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.LogicalExpressionBase;
@@ -73,10 +73,10 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
    */
   static boolean isNullOrEmpty(ColumnStatistics stat) {
     return stat == null
-        || !stat.containsStatistic(ColumnStatisticsKind.MIN_VALUE)
-        || !stat.containsStatistic(ColumnStatisticsKind.MAX_VALUE)
-        || !stat.containsStatistic(ColumnStatisticsKind.NULLS_COUNT)
-        || (long) stat.getStatistic(ColumnStatisticsKind.NULLS_COUNT) == Statistic.NO_COLUMN_STATS;
+        || !stat.contains(ColumnStatisticsKind.MIN_VALUE)
+        || !stat.contains(ColumnStatisticsKind.MAX_VALUE)
+        || !stat.contains(ColumnStatisticsKind.NULLS_COUNT)
+        || ColumnStatisticsKind.NULLS_COUNT.getFrom(stat) == Statistic.NO_COLUMN_STATS;
   }
 
   /**
@@ -96,7 +96,7 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
    * @return <tt>true</tt> if the statistics does not have nulls and <tt>false</tt> otherwise
    */
   static boolean hasNoNulls(ColumnStatistics stat) {
-    return (long) stat.getStatistic(ColumnStatisticsKind.NULLS_COUNT) == 0;
+    return ColumnStatisticsKind.NULLS_COUNT.getFrom(stat) == 0;
   }
 
   /**
@@ -129,13 +129,13 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
    */
   static boolean isAllNulls(ColumnStatistics stat, long rowCount) {
     Preconditions.checkArgument(rowCount >= 0, String.format("negative rowCount %d is not valid", rowCount));
-    return (long) stat.getStatistic(ColumnStatisticsKind.NULLS_COUNT) == rowCount;
+    return ColumnStatisticsKind.NULLS_COUNT.getFrom(stat) == rowCount;
   }
 
-  static boolean hasNonNullValues(ColumnStatistics stat, long rowCount) {
-    return rowCount > (long) stat.getStatistic(ColumnStatisticsKind.NULLS_COUNT)
-      && stat.getValueStatistic(ColumnStatisticsKind.MIN_VALUE) != null
-      && stat.getValueStatistic(ColumnStatisticsKind.MAX_VALUE) != null;
+  static <T> boolean hasNonNullValues(ColumnStatistics<T> stat, long rowCount) {
+    return rowCount > ColumnStatisticsKind.NULLS_COUNT.getFrom(stat)
+        && ColumnStatisticsKind.MIN_VALUE.getValueStatistic(stat) != null
+        && ColumnStatisticsKind.MAX_VALUE.getValueStatistic(stat) != null;
   }
 
   /**
@@ -158,10 +158,10 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
       if (!hasNonNullValues(exprStat, evaluator.getRowCount())) {
         return RowsMatch.SOME;
       }
-      if (!exprStat.getValueStatistic(ColumnStatisticsKind.MAX_VALUE)) {
+      if (!ColumnStatisticsKind.MAX_VALUE.getValueStatistic(exprStat)) {
         return RowsMatch.NONE;
       }
-      return exprStat.getValueStatistic(ColumnStatisticsKind.MIN_VALUE) ? checkNull(exprStat) : RowsMatch.SOME;
+      return ColumnStatisticsKind.MIN_VALUE.getValueStatistic(exprStat) ? checkNull(exprStat) : RowsMatch.SOME;
     });
   }
 
@@ -176,10 +176,10 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
       if (!hasNonNullValues(exprStat, evaluator.getRowCount())) {
         return RowsMatch.SOME;
       }
-      if (exprStat.getValueStatistic(ColumnStatisticsKind.MIN_VALUE)) {
+      if (ColumnStatisticsKind.MIN_VALUE.getValueStatistic(exprStat)) {
         return RowsMatch.NONE;
       }
-      return exprStat.getValueStatistic(ColumnStatisticsKind.MAX_VALUE) ? RowsMatch.SOME : checkNull(exprStat);
+      return ColumnStatisticsKind.MAX_VALUE.getValueStatistic(exprStat) ? RowsMatch.SOME : checkNull(exprStat);
     });
   }
 
@@ -194,10 +194,10 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
       if (!hasNonNullValues(exprStat, evaluator.getRowCount())) {
         return RowsMatch.SOME;
       }
-      if (exprStat.getValueStatistic(ColumnStatisticsKind.MIN_VALUE)) {
+      if (ColumnStatisticsKind.MIN_VALUE.getValueStatistic(exprStat)) {
         return hasNoNulls(exprStat) ? RowsMatch.NONE : RowsMatch.SOME;
       }
-      return exprStat.getValueStatistic(ColumnStatisticsKind.MAX_VALUE) ? RowsMatch.SOME : RowsMatch.ALL;
+      return ColumnStatisticsKind.MAX_VALUE.getValueStatistic(exprStat) ? RowsMatch.SOME : RowsMatch.ALL;
     });
   }
 
@@ -212,10 +212,10 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
       if (!hasNonNullValues(exprStat, evaluator.getRowCount())) {
         return RowsMatch.SOME;
       }
-      if (!exprStat.getValueStatistic(ColumnStatisticsKind.MAX_VALUE)) {
+      if (!ColumnStatisticsKind.MAX_VALUE.getValueStatistic(exprStat)) {
         return hasNoNulls(exprStat) ? RowsMatch.NONE : RowsMatch.SOME;
       }
-      return exprStat.getValueStatistic(ColumnStatisticsKind.MIN_VALUE) ? RowsMatch.ALL : RowsMatch.SOME;
+      return ColumnStatisticsKind.MIN_VALUE.getValueStatistic(exprStat) ? RowsMatch.ALL : RowsMatch.SOME;
     });
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/StatisticsProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/StatisticsProvider.java
index d93ede0..5b3cfc8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/StatisticsProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/StatisticsProvider.java
@@ -37,13 +37,12 @@ import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.ValueHolder;
 import org.apache.drill.exec.vector.ValueHolderHelper;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsImpl;
-import org.apache.drill.metastore.ColumnStatisticsKind;
-import org.apache.drill.metastore.StatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
 
 import java.math.BigInteger;
-import java.util.Comparator;
+import java.util.Arrays;
 import java.util.EnumMap;
 import java.util.EnumSet;
 import java.util.Map;
@@ -64,7 +63,7 @@ public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVis
   }
 
   @Override
-  public ColumnStatisticsImpl visitUnknown(LogicalExpression e, Void value) {
+  public ColumnStatistics visitUnknown(LogicalExpression e, Void value) {
     // do nothing for the unknown expression
     return null;
   }
@@ -76,74 +75,59 @@ public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVis
       return columnStatistics;
     } else if (typedFieldExpr.getMajorType().equals(Types.OPTIONAL_INT)) {
       // field does not exist.
-      MinMaxStatistics<Integer> statistics = new MinMaxStatistics<>(null, null, Integer::compareTo);
-      statistics.setNullsCount(rowCount); // all values are nulls
-      return statistics;
+      return StatisticsProvider.getColumnStatistics(null, null, rowCount, typedFieldExpr.getMajorType().getMinorType());
     }
     return null;
   }
 
   @Override
   public ColumnStatistics<Integer> visitIntConstant(ValueExpressions.IntExpression expr, Void value) {
-    int exprValue = expr.getInt();
-    return new MinMaxStatistics<>(exprValue, exprValue, Integer::compareTo);
+    return getConstantColumnStatistics(expr.getInt(), expr);
   }
 
   @Override
   public ColumnStatistics<Boolean> visitBooleanConstant(ValueExpressions.BooleanExpression expr, Void value) {
-    boolean exprValue = expr.getBoolean();
-    return new MinMaxStatistics<>(exprValue, exprValue, Boolean::compareTo);
+    return getConstantColumnStatistics(expr.getBoolean(), expr);
   }
 
   @Override
   public ColumnStatistics<Long> visitLongConstant(ValueExpressions.LongExpression expr, Void value) {
-    long exprValue = expr.getLong();
-    return new MinMaxStatistics<>(exprValue, exprValue, Long::compareTo);
+    return getConstantColumnStatistics(expr.getLong(), expr);
   }
 
   @Override
   public ColumnStatistics<Float> visitFloatConstant(ValueExpressions.FloatExpression expr, Void value) {
-    float exprValue = expr.getFloat();
-    return new MinMaxStatistics<>(exprValue, exprValue, Float::compareTo);
+    return getConstantColumnStatistics(expr.getFloat(), expr);
   }
 
   @Override
   public ColumnStatistics<Double> visitDoubleConstant(ValueExpressions.DoubleExpression expr, Void value) {
-    double exprValue = expr.getDouble();
-    return new MinMaxStatistics<>(exprValue, exprValue, Double::compareTo);
+    return getConstantColumnStatistics(expr.getDouble(), expr);
   }
 
   @Override
   public ColumnStatistics<Long> visitDateConstant(ValueExpressions.DateExpression expr, Void value) {
-    long exprValue = expr.getDate();
-    return new MinMaxStatistics<>(exprValue, exprValue, Long::compareTo);
+    return getConstantColumnStatistics(expr.getDate(), expr);
   }
 
   @Override
-  public ColumnStatistics<Long> visitTimeStampConstant(ValueExpressions.TimeStampExpression tsExpr, Void value) {
-    long exprValue = tsExpr.getTimeStamp();
-    return new MinMaxStatistics<>(exprValue, exprValue, Long::compareTo);
+  public ColumnStatistics<Long> visitTimeStampConstant(ValueExpressions.TimeStampExpression expr, Void value) {
+    return getConstantColumnStatistics(expr.getTimeStamp(), expr);
   }
 
   @Override
-  public ColumnStatistics<Integer> visitTimeConstant(ValueExpressions.TimeExpression timeExpr, Void value) {
-    int exprValue = timeExpr.getTime();
-    return new MinMaxStatistics<>(exprValue, exprValue, Integer::compareTo);
+  public ColumnStatistics<Integer> visitTimeConstant(ValueExpressions.TimeExpression expr, Void value) {
+    return getConstantColumnStatistics(expr.getTime(), expr);
   }
 
   @Override
-  public ColumnStatistics<String> visitQuotedStringConstant(ValueExpressions.QuotedString quotedString, Void value) {
-    String binary = quotedString.getString();
-    return new MinMaxStatistics<>(binary, binary, Comparator.nullsFirst(Comparator.naturalOrder()));
+  public ColumnStatistics<String> visitQuotedStringConstant(ValueExpressions.QuotedString expr, Void value) {
+    return getConstantColumnStatistics(expr.getString(), expr);
   }
 
   @Override
-  public ColumnStatistics<BigInteger> visitVarDecimalConstant(ValueExpressions.VarDecimalExpression decExpr, Void value) {
-    BigInteger unscaled = decExpr.getBigDecimal().unscaledValue();
-    return new MinMaxStatistics<>(
-        unscaled,
-        unscaled,
-        Comparator.nullsFirst(Comparator.naturalOrder()));
+  public ColumnStatistics<BigInteger> visitVarDecimalConstant(ValueExpressions.VarDecimalExpression expr, Void value) {
+    return getConstantColumnStatistics(expr.getBigDecimal().unscaledValue(), expr);
   }
 
   @Override
@@ -167,8 +151,7 @@ public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVis
     return null;
   }
 
-  @SuppressWarnings("unchecked")
-  private ColumnStatistics<T> evalCastFunc(FunctionHolderExpression holderExpr, ColumnStatistics<T> input) {
+  private ColumnStatistics evalCastFunc(FunctionHolderExpression holderExpr, ColumnStatistics<T> input) {
     try {
       DrillSimpleFuncHolder funcHolder = (DrillSimpleFuncHolder) holderExpr.getHolder();
 
@@ -218,89 +201,89 @@ public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVis
       ValueHolder minFuncHolder = InterpreterEvaluator.evaluateFunction(interpreter, args1, holderExpr.getName());
       ValueHolder maxFuncHolder = InterpreterEvaluator.evaluateFunction(interpreter, args2, holderExpr.getName());
 
-      MinMaxStatistics statistics;
       switch (destType) {
         case INT:
-          statistics = new MinMaxStatistics<>(((IntHolder) minFuncHolder).value, ((IntHolder) maxFuncHolder).value, Integer::compareTo);
-          break;
+          return StatisticsProvider.getColumnStatistics(
+              ((IntHolder) minFuncHolder).value,
+              ((IntHolder) maxFuncHolder).value,
+              ColumnStatisticsKind.NULLS_COUNT.getFrom(input),
+              destType);
         case BIGINT:
-          statistics = new MinMaxStatistics<>(((BigIntHolder) minFuncHolder).value, ((BigIntHolder) maxFuncHolder).value, Long::compareTo);
-          break;
+          return StatisticsProvider.getColumnStatistics(
+              ((BigIntHolder) minFuncHolder).value,
+              ((BigIntHolder) maxFuncHolder).value,
+              ColumnStatisticsKind.NULLS_COUNT.getFrom(input),
+              destType);
         case FLOAT4:
-          statistics = new MinMaxStatistics<>(((Float4Holder) minFuncHolder).value, ((Float4Holder) maxFuncHolder).value, Float::compareTo);
-          break;
+          return StatisticsProvider.getColumnStatistics(
+              ((Float4Holder) minFuncHolder).value,
+              ((Float4Holder) maxFuncHolder).value,
+              ColumnStatisticsKind.NULLS_COUNT.getFrom(input),
+              destType);
         case FLOAT8:
-          statistics = new MinMaxStatistics<>(((Float8Holder) minFuncHolder).value, ((Float8Holder) maxFuncHolder).value, Double::compareTo);
-          break;
+          return StatisticsProvider.getColumnStatistics(
+              ((Float8Holder) minFuncHolder).value,
+              ((Float8Holder) maxFuncHolder).value,
+              ColumnStatisticsKind.NULLS_COUNT.getFrom(input),
+              destType);
         case TIMESTAMP:
-          statistics = new MinMaxStatistics<>(((TimeStampHolder) minFuncHolder).value, ((TimeStampHolder) maxFuncHolder).value, Long::compareTo);
-          break;
+          return StatisticsProvider.getColumnStatistics(
+              ((TimeStampHolder) minFuncHolder).value,
+              ((TimeStampHolder) maxFuncHolder).value,
+              ColumnStatisticsKind.NULLS_COUNT.getFrom(input),
+              destType);
         default:
           return null;
       }
-      statistics.setNullsCount((long) input.getStatistic(ColumnStatisticsKind.NULLS_COUNT));
-      return statistics;
     } catch (Exception e) {
-      throw new DrillRuntimeException("Error in evaluating function of " + holderExpr.getName() );
+      throw new DrillRuntimeException("Error in evaluating function of " + holderExpr.getName());
     }
   }
 
-  public static class MinMaxStatistics<V> implements ColumnStatistics<V> {
-    private final V minVal;
-    private final V maxVal;
-    private final Comparator<V> valueComparator;
-    private long nullsCount;
-
-    public MinMaxStatistics(V minVal, V maxVal, Comparator<V> valueComparator) {
-      this.minVal = minVal;
-      this.maxVal = maxVal;
-      this.valueComparator = valueComparator;
-    }
-
-    @Override
-    public Object getStatistic(StatisticsKind statisticsKind) {
-      switch (statisticsKind.getName()) {
-        case ExactStatisticsConstants.MIN_VALUE:
-          return minVal;
-        case ExactStatisticsConstants.MAX_VALUE:
-          return maxVal;
-        case ExactStatisticsConstants.NULLS_COUNT:
-          return nullsCount;
-        default:
-          return null;
-      }
-    }
-
-    @Override
-    public boolean containsStatistic(StatisticsKind statisticsKind) {
-      switch (statisticsKind.getName()) {
-        case ExactStatisticsConstants.MIN_VALUE:
-        case ExactStatisticsConstants.MAX_VALUE:
-        case ExactStatisticsConstants.NULLS_COUNT:
-          return true;
-        default:
-          return false;
-      }
-    }
-
-    @Override
-    public boolean containsExactStatistics(StatisticsKind statisticsKind) {
-      return true;
-    }
-
-    @Override
-    public Comparator<V> getValueComparator() {
-      return valueComparator;
-    }
+  /**
+   * Returns {@link ColumnStatistics} instance with set min, max values and nulls count statistics specified in the arguments.
+   *
+   * @param minVal     min value
+   * @param maxVal     max value
+   * @param nullsCount nulls count
+   * @param type       type of the column
+   * @param <V>        type of min and max values
+   * @return {@link ColumnStatistics} instance with set min, max values and nulls count statistics
+   */
+  public static <V> ColumnStatistics<V> getColumnStatistics(V minVal, V maxVal, long nullsCount, TypeProtos.MinorType type) {
+    return new ColumnStatistics<>(
+        Arrays.asList(new StatisticsHolder<>(minVal, ColumnStatisticsKind.MIN_VALUE),
+            new StatisticsHolder<>(maxVal, ColumnStatisticsKind.MAX_VALUE),
+            new StatisticsHolder<>(nullsCount, ColumnStatisticsKind.NULLS_COUNT)),
+        type);
+  }
 
-    @Override
-    public ColumnStatistics<V> cloneWithStats(ColumnStatistics statistics) {
-      throw new UnsupportedOperationException("MinMaxStatistics does not support cloneWithStats");
-    }
+  /**
+   * Returns {@link ColumnStatistics} instance with min and max values set to {@code minMaxValue}
+   * and nulls count set to 0. Resulting {@link ColumnStatistics} instance corresponds
+   * to a constant value, so nulls count is set to 0.
+   *
+   * @param minMaxValue value of min and max statistics
+   * @param expr        source of column type
+   * @param <V>         type of min and max values
+   * @return {@link ColumnStatistics} instance with min and max values set to {@code minMaxValue} and nulls count set to 0
+   */
+  public static <V> ColumnStatistics<V> getConstantColumnStatistics(V minMaxValue, LogicalExpression expr) {
+    return getConstantColumnStatistics(minMaxValue, expr.getMajorType().getMinorType());
+  }
 
-    void setNullsCount(long nullsCount) {
-      this.nullsCount = nullsCount;
-    }
+  /**
+   * Returns {@link ColumnStatistics} instance with min and max values set to {@code minMaxValue}
+   * and nulls count set to 0. Resulting {@link ColumnStatistics} instance corresponds
+   * to a constant value, so nulls count is set to 0.
+   *
+   * @param minMaxValue value of min and max statistics
+   * @param type        column type
+   * @param <V>         type of min and max values
+   * @return {@link ColumnStatistics} instance with min and max values set to {@code minMaxValue} and nulls count set to 0
+   */
+  public static <V> ColumnStatistics<V> getConstantColumnStatistics(V minMaxValue, TypeProtos.MinorType type) {
+    return getColumnStatistics(minMaxValue, minMaxValue, 0, type);
   }
 
   private static final Map<TypeProtos.MinorType, Set<TypeProtos.MinorType>> CAST_FUNC = new EnumMap<>(TypeProtos.MinorType.class);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
index 126ad07..4916370 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
@@ -22,7 +22,8 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.drill.metastore.TableMetadata;
+import org.apache.drill.metastore.metadata.TableMetadata;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 import org.apache.drill.common.expression.LogicalExpression;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScanWithMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScanWithMetadata.java
index 3be82df..98646b4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScanWithMetadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScanWithMetadata.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.physical.base;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
@@ -34,11 +35,14 @@ import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.stat.RowsMatch;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
 import org.apache.drill.exec.ops.UdfUtilities;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
+import org.apache.drill.metastore.metadata.Metadata;
+import org.apache.drill.metastore.metadata.SegmentMetadata;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.SchemaPathUtils;
+import org.apache.drill.metastore.util.SchemaPathUtils;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.server.options.OptionManager;
@@ -46,15 +50,16 @@ import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.store.parquet.FilterEvaluatorUtils;
 import org.apache.drill.exec.store.parquet.ParquetTableMetadataUtils;
-import org.apache.drill.metastore.BaseMetadata;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsKind;
-import org.apache.drill.metastore.FileMetadata;
-import org.apache.drill.metastore.LocationProvider;
-import org.apache.drill.metastore.NonInterestingColumnsMetadata;
-import org.apache.drill.metastore.PartitionMetadata;
-import org.apache.drill.metastore.TableMetadata;
-import org.apache.drill.metastore.TableStatisticsKind;
+import org.apache.drill.metastore.metadata.BaseMetadata;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.metadata.FileMetadata;
+import org.apache.drill.metastore.metadata.LocationProvider;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.NonInterestingColumnsMetadata;
+import org.apache.drill.metastore.metadata.PartitionMetadata;
+import org.apache.drill.metastore.metadata.TableMetadata;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
 import org.apache.drill.exec.expr.FilterBuilder;
 import org.apache.drill.exec.expr.FilterPredicate;
 import org.apache.hadoop.fs.Path;
@@ -63,6 +68,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -71,7 +77,6 @@ import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
-// import static org.apache.drill.exec.ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS;
 import static org.apache.drill.exec.ExecConstants.SKIP_RUNTIME_ROWGROUP_PRUNING_KEY;
 
 /**
@@ -87,6 +92,8 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
   // partition metadata info: mixed partition values for all partition keys in the same list
   protected List<PartitionMetadata> partitions;
 
+  protected Map<Path, SegmentMetadata> segments;
+
   protected NonInterestingColumnsMetadata nonInterestingColumnsMetadata;
   protected List<SchemaPath> partitionColumns;
   protected LogicalExpression filter;
@@ -116,6 +123,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     this.tableMetadata = that.tableMetadata;
     this.partitionColumns = that.partitionColumns;
     this.partitions = that.partitions;
+    this.segments = that.segments;
     this.files = that.files;
     this.nonInterestingColumnsMetadata = that.nonInterestingColumnsMetadata;
     this.fileSet = that.fileSet == null ? null : new HashSet<>(that.fileSet);
@@ -162,15 +170,15 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     }
 
     if (columnStats != null) {
-      tableRowCount = (long) TableStatisticsKind.ROW_COUNT.getValue(getTableMetadata());
+      tableRowCount = TableStatisticsKind.ROW_COUNT.getValue(getTableMetadata());
     } else if (nonInterestingColStats != null) {
-      tableRowCount = (long) TableStatisticsKind.ROW_COUNT.getValue(getNonInterestingColumnsMetadata());
+      tableRowCount = TableStatisticsKind.ROW_COUNT.getValue(getNonInterestingColumnsMetadata());
     } else {
       return 0; // returns 0 if the column doesn't exist in the table.
     }
 
     columnStats = columnStats != null ? columnStats : nonInterestingColStats;
-    nulls = (Long) columnStats.getStatistic(ColumnStatisticsKind.NULLS_COUNT);
+    nulls = ColumnStatisticsKind.NULLS_COUNT.getFrom(columnStats);
     colNulls = nulls != null ? nulls : Statistic.NO_COLUMN_STATS;
 
     return Statistic.NO_COLUMN_STATS == tableRowCount
@@ -186,7 +194,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
   @Override
   public ScanStats getScanStats() {
     int columnCount = columns == null ? 20 : columns.size();
-    double rowCount = (long) TableStatisticsKind.ROW_COUNT.getValue(getTableMetadata());
+    double rowCount = TableStatisticsKind.ROW_COUNT.getValue(getTableMetadata());
 
     ScanStats scanStats = new ScanStats(ScanStats.GroupScanProperty.EXACT_ROW_COUNT, rowCount, 1, rowCount * columnCount);
     logger.trace("Drill parquet scan statistics: {}", scanStats);
@@ -221,6 +229,31 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     if ( ! skipRuntimePruning ) { setFilter(filterExpr); }
   }
 
+  /**
+   * Applies specified filter {@code filterExpr} to current group scan and produces filtering at:
+   * <ul>
+   * <li>table level:
+   * <ul><li>if filter matches all the the data or prunes all the data, sets corresponding value to
+   * {@link AbstractGroupScanWithMetadata#isMatchAllMetadata()} and returns null</li></ul></li>
+   * <li>segment level:
+   * <ul><li>if filter matches all the the data or prunes all the data, sets corresponding value to
+   * {@link AbstractGroupScanWithMetadata#isMatchAllMetadata()} and returns null</li>
+   * <li>if segment metadata was pruned, prunes underlying metadata</li></ul></li>
+   * <li>partition level:
+   * <ul><li>if filter matches all the the data or prunes all the data, sets corresponding value to
+   * {@link AbstractGroupScanWithMetadata#isMatchAllMetadata()} and returns null</li>
+   * <li>if partition metadata was pruned, prunes underlying metadata</li></ul></li>
+   * <li>file level:
+   * <ul><li>if filter matches all the the data or prunes all the data, sets corresponding value to
+   * {@link AbstractGroupScanWithMetadata#isMatchAllMetadata()} and returns null</li></ul></li>
+   * </ul>
+   *
+   * @param filterExpr                     filter expression to build
+   * @param udfUtilities                   udf utilities
+   * @param functionImplementationRegistry context to find drill function holder
+   * @param optionManager                  option manager
+   * @return group scan with applied filter expression
+   */
   @Override
   public AbstractGroupScanWithMetadata applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities,
       FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) {
@@ -238,35 +271,13 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
 
     GroupScanWithMetadataFilterer filteredMetadata = getFilterer().getFiltered(optionManager, filterPredicate, schemaPathsInExpr);
 
-    if (!getFilesMetadata().isEmpty()) {
-      if (!filteredMetadata.getFiles().isEmpty() && getFilesMetadata().size() == filteredMetadata.getFiles().size()) {
-        // There is no reduction of files, but filter may be omitted.
-        logger.debug("applyFilter() does not have any pruning since GroupScan fully matches filter");
-        matchAllMetadata = filteredMetadata.isMatchAllMetadata();
-        return null;
-      }
-    } else if (!getPartitionsMetadata().isEmpty()) {
-      // for the case when files metadata wasn't created, check partition metadata
-      if (!filteredMetadata.getPartitions().isEmpty() && getPartitionsMetadata().size() == filteredMetadata.getPartitions().size()) {
-        // There is no reduction of partitions, but filter may be omitted.
-        logger.debug("applyFilter() does not have any pruning since GroupScan fully matches filter");
-        matchAllMetadata = filteredMetadata.isMatchAllMetadata();
-        return null;
-      }
-    } else if (getTableMetadata() != null) {
-      // There is no reduction of files or partitions, but filter may be omitted.
+    if (isGroupScanFullyMatchesFilter(filteredMetadata)) {
       logger.debug("applyFilter() does not have any pruning since GroupScan fully matches filter");
       matchAllMetadata = filteredMetadata.isMatchAllMetadata();
       return null;
     }
 
-    if (!filteredMetadata.isMatchAllMetadata()
-      // filter returns empty result using table metadata
-      && ((filteredMetadata.getTableMetadata() == null && getTableMetadata() != null)
-          // all partitions are pruned if partition metadata is available
-          || filteredMetadata.getPartitions().isEmpty() && !getPartitionsMetadata().isEmpty()
-          // all files are pruned if file metadata is available
-          || filteredMetadata.getFiles().isEmpty() && !getFilesMetadata().isEmpty())) {
+    if (isAllDataPruned(filteredMetadata)) {
       if (getFilesMetadata().size() == 1) {
         // For the case when group scan has single file and it was filtered,
         // no need to create new group scan with the same file.
@@ -274,18 +285,46 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
       }
       logger.debug("All files have been filtered out. Add back one to get schema from scanner");
       Map<Path, FileMetadata> filesMap = getNextOrEmpty(getFilesMetadata().values()).stream()
-          .collect(Collectors.toMap(FileMetadata::getLocation, Function.identity()));
+          .collect(Collectors.toMap(FileMetadata::getPath, Function.identity()));
+
+      Map<Path, SegmentMetadata> segmentsMap = getNextOrEmpty(getSegmentsMetadata().values()).stream()
+          .collect(Collectors.toMap(SegmentMetadata::getPath, Function.identity()));
 
-      filteredMetadata.withTable(getTableMetadata())
-          .withPartitions(getNextOrEmpty(getPartitionsMetadata()))
-          .withFiles(filesMap)
-          .withNonInterestingColumns(getNonInterestingColumnsMetadata())
-          .withMatching(false);
+      filteredMetadata.table(getTableMetadata())
+          .segments(segmentsMap)
+          .partitions(getNextOrEmpty(getPartitionsMetadata()))
+          .files(filesMap)
+          .nonInterestingColumns(getNonInterestingColumnsMetadata())
+          .matching(false);
     }
 
     return filteredMetadata.build();
   }
 
+  protected boolean isAllDataPruned(GroupScanWithMetadataFilterer filteredMetadata) {
+    return !filteredMetadata.isMatchAllMetadata()
+        // filter returns empty result using table metadata
+        && (filteredMetadata.getTableMetadata() == null && getTableMetadata() != null)
+            // all partitions are pruned if segment metadata is available
+            || filteredMetadata.getSegments().isEmpty() && !getSegmentsMetadata().isEmpty()
+            // all segments are pruned if partition metadata is available
+            || filteredMetadata.getPartitions().isEmpty() && !getPartitionsMetadata().isEmpty()
+            // all files are pruned if file metadata is available
+            || filteredMetadata.getFiles().isEmpty() && !getFilesMetadata().isEmpty();
+  }
+
+  protected boolean isGroupScanFullyMatchesFilter(GroupScanWithMetadataFilterer filteredMetadata) {
+    if (MapUtils.isNotEmpty(getFilesMetadata())) {
+      return getFilesMetadata().size() == filteredMetadata.getFiles().size();
+    } else if (CollectionUtils.isNotEmpty(getPartitionsMetadata())) {
+      return getPartitionsMetadata().size() == filteredMetadata.getPartitions().size();
+    } else if (MapUtils.isNotEmpty(getSegmentsMetadata())) {
+      return getSegmentsMetadata().size() == filteredMetadata.getSegments().size();
+    } else {
+      return getTableMetadata() != null;
+    }
+  }
+
   /**
    * Returns list with the first element of input list or empty list if input one was empty.
    *
@@ -311,14 +350,18 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     return getFilterPredicate(filterExpr, udfUtilities, functionImplementationRegistry, optionManager,
             omitUnsupportedExprs, supportsFileImplicitColumns(), (TupleSchema) getTableMetadata().getSchema());
   }
+
   /**
    * Returns parquet filter predicate built from specified {@code filterExpr}.
    *
    * @param filterExpr                     filter expression to build
    * @param udfUtilities                   udf utilities
    * @param functionImplementationRegistry context to find drill function holder
+   * @param optionManager                  option manager
    * @param omitUnsupportedExprs           whether expressions which cannot be converted
    *                                       may be omitted from the resulting expression
+   * @param supportsFileImplicitColumns    whether implicit columns are supported
+   * @param tupleSchema                    schema
    * @return parquet filter predicate
    */
   public static FilterPredicate getFilterPredicate(LogicalExpression filterExpr,
@@ -350,7 +393,9 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
         errorCollector.getErrorCount(), errorCollector.toErrorString());
       return null;
     }
-    logger.debug("materializedFilter : {}", ExpressionStringBuilder.toString(materializedFilter));
+    if (logger.isDebugEnabled()) {
+      logger.debug("materializedFilter : {}", ExpressionStringBuilder.toString(materializedFilter));
+    }
 
     Set<LogicalExpression> constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(materializedFilter);
     return FilterBuilder.buildFilterPredicate(materializedFilter, constantBoundaries, udfUtilities, omitUnsupportedExprs);
@@ -377,7 +422,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     maxRecords = Math.max(maxRecords, 1); // Make sure it request at least 1 row -> 1 file.
     GroupScanWithMetadataFilterer prunedMetadata = getFilterer();
     if (getTableMetadata() != null) {
-      long tableRowCount = (long) TableStatisticsKind.ROW_COUNT.getValue(getTableMetadata());
+      long tableRowCount = TableStatisticsKind.ROW_COUNT.getValue(getTableMetadata());
       if (tableRowCount == Statistic.NO_COLUMN_STATS || tableRowCount <= maxRecords) {
         logger.debug("limit push down does not apply, since total number of rows [{}] is less or equal to the required [{}].",
             tableRowCount, maxRecords);
@@ -395,14 +440,15 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     }
 
     Map<Path, FileMetadata> filesMap = qualifiedFiles.stream()
-        .collect(Collectors.toMap(FileMetadata::getLocation, Function.identity()));
+        .collect(Collectors.toMap(FileMetadata::getPath, Function.identity()));
 
     return prunedMetadata
-        .withTable(getTableMetadata())
-        .withPartitions(getPartitionsMetadata())
-        .withFiles(filesMap)
-        .withNonInterestingColumns(getNonInterestingColumnsMetadata())
-        .withMatching(matchAllMetadata)
+        .table(getTableMetadata())
+        .segments(getSegmentsMetadata())
+        .partitions(getPartitionsMetadata())
+        .files(filesMap)
+        .nonInterestingColumns(getNonInterestingColumnsMetadata())
+        .matching(matchAllMetadata)
         .build();
   }
 
@@ -443,7 +489,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     List<T> qualifiedMetadata = new ArrayList<>();
     int currentRowCount = 0;
     for (T metadata : metadataList) {
-      long rowCount = (long) TableStatisticsKind.ROW_COUNT.getValue(metadata);
+      long rowCount = TableStatisticsKind.ROW_COUNT.getValue(metadata);
       if (rowCount == Statistic.NO_COLUMN_STATS) {
         return null;
       } else if (currentRowCount + rowCount <= maxRecords) {
@@ -476,7 +522,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     return getPartitionsMetadata().stream()
         .filter(partition -> partition.getColumn().equals(column) && partition.getLocations().contains(path))
         .findAny()
-        .map(metadata -> clazz.cast(metadata.getColumnsStatistics().get(column).getStatistic(ColumnStatisticsKind.MAX_VALUE)))
+        .map(metadata -> clazz.cast(metadata.getColumnsStatistics().get(column).get(ColumnStatisticsKind.MAX_VALUE)))
         .orElse(null);
   }
 
@@ -515,7 +561,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
   // protected methods for internal usage
   protected Map<Path, FileMetadata> getFilesMetadata() {
     if (files == null) {
-      files = ((ParquetMetadataProvider) metadataProvider).getFilesMetadataMap();
+      files = metadataProvider.getFilesMetadataMap();
     }
     return files;
   }
@@ -535,10 +581,17 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     return partitions;
   }
 
+  protected Map<Path, SegmentMetadata> getSegmentsMetadata() {
+    if (segments == null) {
+      segments = metadataProvider.getSegmentsMetadataMap();
+    }
+    return segments;
+  }
+
   @JsonIgnore
   public NonInterestingColumnsMetadata getNonInterestingColumnsMetadata() {
     if (nonInterestingColumnsMetadata == null) {
-      nonInterestingColumnsMetadata = metadataProvider.getNonInterestingColumnsMeta();
+      nonInterestingColumnsMetadata = metadataProvider.getNonInterestingColumnsMetadata();
     }
     return nonInterestingColumnsMetadata;
   }
@@ -546,20 +599,21 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
   /**
    * This class is responsible for filtering different metadata levels.
    */
-  protected abstract static class GroupScanWithMetadataFilterer {
+  protected abstract static class GroupScanWithMetadataFilterer<B extends GroupScanWithMetadataFilterer<B>> {
     protected final AbstractGroupScanWithMetadata source;
 
     protected boolean matchAllMetadata = false;
 
     protected TableMetadata tableMetadata;
     protected List<PartitionMetadata> partitions = Collections.emptyList();
+    protected Map<Path, SegmentMetadata> segments = Collections.emptyMap();
     protected Map<Path, FileMetadata> files = Collections.emptyMap();
     protected NonInterestingColumnsMetadata nonInterestingColumnsMetadata;
 
     // for the case when filtering is possible for partitions, but files count exceeds
     // PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD, new group scan with at least filtered partitions
     // and files which belongs to that partitions may be returned
-    protected MetadataLevel overflowLevel = MetadataLevel.NONE;
+    protected MetadataType overflowLevel = MetadataType.NONE;
 
     public GroupScanWithMetadataFilterer(AbstractGroupScanWithMetadata source) {
       this.source = source;
@@ -572,34 +626,39 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
      */
     public abstract AbstractGroupScanWithMetadata build();
 
-    public GroupScanWithMetadataFilterer withTable(TableMetadata tableMetadata) {
+    public B table(TableMetadata tableMetadata) {
       this.tableMetadata = tableMetadata;
-      return this;
+      return self();
     }
 
-    public GroupScanWithMetadataFilterer withPartitions(List<PartitionMetadata> partitions) {
+    public B partitions(List<PartitionMetadata> partitions) {
       this.partitions = partitions;
-      return this;
+      return self();
     }
 
-    public GroupScanWithMetadataFilterer withNonInterestingColumns(NonInterestingColumnsMetadata nonInterestingColumns) {
+    public B segments(Map<Path, SegmentMetadata> segments) {
+      this.segments = segments;
+      return self();
+    }
+
+    public B nonInterestingColumns(NonInterestingColumnsMetadata nonInterestingColumns) {
       this.nonInterestingColumnsMetadata = nonInterestingColumns;
-      return this;
+      return self();
     }
 
-    public GroupScanWithMetadataFilterer withFiles(Map<Path, FileMetadata> files) {
+    public B files(Map<Path, FileMetadata> files) {
       this.files = files;
-      return this;
+      return self();
     }
 
-    public GroupScanWithMetadataFilterer withMatching(boolean matchAllMetadata) {
+    public B matching(boolean matchAllMetadata) {
       this.matchAllMetadata = matchAllMetadata;
-      return this;
+      return self();
     }
 
-    public GroupScanWithMetadataFilterer withOverflow(MetadataLevel overflowLevel) {
+    public B overflow(MetadataType overflowLevel) {
       this.overflowLevel = overflowLevel;
-      return this;
+      return self();
     }
 
     public boolean isMatchAllMetadata() {
@@ -614,11 +673,15 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
       return partitions;
     }
 
+    public Map<Path, SegmentMetadata> getSegments() {
+      return segments;
+    }
+
     public Map<Path, FileMetadata> getFiles() {
       return files;
     }
 
-    public MetadataLevel getOverflowLevel() {
+    public MetadataType getOverflowLevel() {
       return overflowLevel;
     }
 
@@ -631,13 +694,17 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
      * @param schemaPathsInExpr columns used in filter expression
      * @return this instance with filtered metadata
      */
-    protected GroupScanWithMetadataFilterer getFiltered(OptionManager optionManager,
+    protected B getFiltered(OptionManager optionManager,
                                                         FilterPredicate filterPredicate,
                                                         Set<SchemaPath> schemaPathsInExpr) {
       if (source.getTableMetadata() != null) {
         filterTableMetadata(filterPredicate, schemaPathsInExpr);
       }
 
+      if (source.getSegmentsMetadata() != null) {
+        filterSegmentMetadata(optionManager, filterPredicate, schemaPathsInExpr);
+      }
+
       if (source.getPartitionsMetadata() != null) {
         filterPartitionMetadata(optionManager, filterPredicate, schemaPathsInExpr);
       }
@@ -645,7 +712,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
       if (source.getFilesMetadata() != null) {
         filterFileMetadata(optionManager, filterPredicate, schemaPathsInExpr);
       }
-      return this;
+      return self();
     }
 
     /**
@@ -666,27 +733,69 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     }
 
     /**
-     * Produces filtering of metadata at partition level.
+     * Produces filtering of metadata at segment level.
      *
      * @param optionManager     option manager
      * @param filterPredicate   filter expression
      * @param schemaPathsInExpr columns used in filter expression
      */
-    protected void filterPartitionMetadata(OptionManager optionManager,
-                                           FilterPredicate filterPredicate,
-                                           Set<SchemaPath> schemaPathsInExpr) {
+    protected void filterSegmentMetadata(OptionManager optionManager,
+                                         FilterPredicate filterPredicate,
+                                         Set<SchemaPath> schemaPathsInExpr) {
       if (!matchAllMetadata) {
-        if (!source.getPartitionsMetadata().isEmpty()) {
-          if (source.getPartitionsMetadata().size() <= optionManager.getOption(
-            PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD)) {
+        if (!source.getSegmentsMetadata().isEmpty()) {
+          if (source.getSegmentsMetadata().size() <= optionManager.getOption(
+              PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD)) {
             matchAllMetadata = true;
-            partitions = filterAndGetMetadata(schemaPathsInExpr, source.getPartitionsMetadata(), filterPredicate, optionManager);
+            segments = filterAndGetMetadata(schemaPathsInExpr,
+                source.getSegmentsMetadata().values(),
+                filterPredicate,
+                optionManager).stream()
+                    .collect(Collectors.toMap(
+                        SegmentMetadata::getPath,
+                        Function.identity(),
+                        (first, second) -> second));
           } else {
-            overflowLevel = MetadataLevel.PARTITION;
+            overflowLevel = MetadataType.SEGMENT;
           }
         }
       } else {
-        partitions = source.getPartitionsMetadata();
+        segments = source.getSegmentsMetadata();
+      }
+    }
+
+    /**
+     * Produces filtering of metadata at partition level.
+     *
+     * @param optionManager     option manager
+     * @param filterPredicate   filter expression
+     * @param schemaPathsInExpr columns used in filter expression
+     */
+    protected void filterPartitionMetadata(OptionManager optionManager,
+                                           FilterPredicate filterPredicate,
+                                           Set<SchemaPath> schemaPathsInExpr) {
+      List<PartitionMetadata> prunedPartitions;
+      if (!source.getSegmentsMetadata().isEmpty()
+          && source.getSegmentsMetadata().size() > getSegments().size()) {
+        // prunes row groups to leave only row groups which are contained by pruned segments
+        prunedPartitions = pruneForSegments(source.getPartitionsMetadata(), getSegments());
+      } else {
+        prunedPartitions = source.getPartitionsMetadata();
+      }
+
+      if (isMatchAllMetadata()) {
+        partitions = prunedPartitions;
+        return;
+      }
+
+      if (!source.getPartitionsMetadata().isEmpty()) {
+        if (source.getPartitionsMetadata().size() <= optionManager.getOption(
+          PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD)) {
+          matchAllMetadata = true;
+          partitions = filterAndGetMetadata(schemaPathsInExpr, prunedPartitions, filterPredicate, optionManager);
+        } else {
+          overflowLevel = MetadataType.PARTITION;
+        }
       }
     }
 
@@ -701,16 +810,19 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
                                       FilterPredicate filterPredicate,
                                       Set<SchemaPath> schemaPathsInExpr) {
       Map<Path, FileMetadata> prunedFiles;
-      if (CollectionUtils.isEmpty(source.getPartitionsMetadata())
-          || source.getPartitionsMetadata().size() == getPartitions().size()) {
-        // no partition pruning happened, no need to prune initial files list
-        prunedFiles = source.getFilesMetadata();
-      } else {
+      if (!source.getPartitionsMetadata().isEmpty()
+          && source.getPartitionsMetadata().size() > getPartitions().size()) {
         // prunes files to leave only files which are contained by pruned partitions
-        prunedFiles = pruneForPartitions(source.getFilesMetadata(), partitions);
+        prunedFiles = pruneForPartitions(source.getFilesMetadata(), getPartitions());
+      } else if (!source.getSegmentsMetadata().isEmpty()
+          && source.getSegmentsMetadata().size() > getSegments().size()) {
+        // prunes row groups to leave only row groups which are contained by pruned segments
+        prunedFiles = pruneForSegments(source.getFilesMetadata(), getSegments());
+      } else {
+        prunedFiles = source.getFilesMetadata();
       }
 
-      if (matchAllMetadata) {
+      if (isMatchAllMetadata()) {
         files = prunedFiles;
         return;
       }
@@ -718,20 +830,69 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
       // Stop files pruning for the case:
       //    -  # of files is beyond PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD.
       if (prunedFiles.size() <= optionManager.getOption(
-        PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD)) {
+          PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD)) {
 
         matchAllMetadata = true;
         files = filterAndGetMetadata(schemaPathsInExpr, prunedFiles.values(), filterPredicate, optionManager).stream()
-            .collect(Collectors.toMap(FileMetadata::getLocation, Function.identity()));
+            .collect(Collectors.toMap(FileMetadata::getPath, Function.identity()));
 
       } else {
         matchAllMetadata = false;
         files = prunedFiles;
-        overflowLevel = MetadataLevel.FILE;
+        overflowLevel = MetadataType.FILE;
       }
     }
 
     /**
+     * Removes metadata which does not belong to any of partitions in metadata list.
+     *
+     * @param metadataToPrune         list of metadata which should be pruned
+     * @param filteredSegmentMetadata list of segment metadata which was pruned
+     * @param <T>                     type of metadata to filter
+     * @return map with metadata which belongs to pruned partitions
+     */
+    protected static <T extends BaseMetadata & LocationProvider> Map<Path, T> pruneForSegments(
+        Map<Path, T> metadataToPrune, Map<Path, SegmentMetadata> filteredSegmentMetadata) {
+      Map<Path, T> prunedFiles = new HashMap<>();
+      if (metadataToPrune != null) {
+        for (Map.Entry<Path, T> entry : metadataToPrune.entrySet()) {
+          for (SegmentMetadata filteredPartition : filteredSegmentMetadata.values()) {
+            if (filteredPartition.getLocations().contains(entry.getKey())) {
+              prunedFiles.put(entry.getKey(), entry.getValue());
+              break;
+            }
+          }
+        }
+      }
+
+      return prunedFiles;
+    }
+
+    /**
+     * Removes metadata which does not belong to any of partitions in metadata list.
+     *
+     * @param metadataToPrune         list of partition metadata which should be pruned
+     * @param filteredSegmentMetadata list of segment metadata which was pruned
+     * @return list with metadata which belongs to pruned partitions
+     */
+    protected List<PartitionMetadata> pruneForSegments(
+        List<PartitionMetadata> metadataToPrune, Map<Path, SegmentMetadata> filteredSegmentMetadata) {
+      List<PartitionMetadata> prunedPartitions = new ArrayList<>();
+      if (metadataToPrune != null) {
+        for (PartitionMetadata partition : metadataToPrune) {
+          for (SegmentMetadata segment : filteredSegmentMetadata.values()) {
+            if (!Collections.disjoint(segment.getLocations(), partition.getLocations())) {
+              prunedPartitions.add(partition);
+              break;
+            }
+          }
+        }
+      }
+
+      return prunedPartitions;
+    }
+
+    /**
      * Produces filtering of specified metadata using specified filter expression and returns filtered metadata.
      *
      * @param schemaPathsInExpr columns used in filter expression
@@ -741,10 +902,10 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
      * @param <T>               type of metadata to filter
      * @return filtered metadata
      */
-    public <T extends BaseMetadata> List<T> filterAndGetMetadata(Set<SchemaPath> schemaPathsInExpr,
-                                                                 Iterable<T> metadataList,
-                                                                 FilterPredicate filterPredicate,
-                                                                 OptionManager optionManager) {
+    public <T extends Metadata> List<T> filterAndGetMetadata(Set<SchemaPath> schemaPathsInExpr,
+                                                             Iterable<T> metadataList,
+                                                             FilterPredicate filterPredicate,
+                                                             OptionManager optionManager) {
       List<T> qualifiedFiles = new ArrayList<>();
 
       for (T metadata : metadataList) {
@@ -755,14 +916,14 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
           LocationProvider locationProvider = (LocationProvider) metadata;
           columnsStatistics = ParquetTableMetadataUtils.addImplicitColumnsStatistics(columnsStatistics,
               source.columns, source.getPartitionValues(locationProvider), optionManager,
-              locationProvider.getLocation(), source.supportsFileImplicitColumns());
+              locationProvider.getPath(), source.supportsFileImplicitColumns());
         }
 
         if (source.getNonInterestingColumnsMetadata() != null) {
           columnsStatistics.putAll(source.getNonInterestingColumnsMetadata().getColumnsStatistics());
         }
         RowsMatch match = FilterEvaluatorUtils.matches(filterPredicate,
-            columnsStatistics, (long) metadata.getStatistic(TableStatisticsKind.ROW_COUNT),
+            columnsStatistics, TableStatisticsKind.ROW_COUNT.getValue(metadata),
             metadata.getSchema(), schemaPathsInExpr);
         if (match == RowsMatch.NONE) {
           continue; // No file comply to the filter => drop the file
@@ -777,9 +938,8 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
       }
       return qualifiedFiles;
     }
-  }
 
-  public enum MetadataLevel {
-    TABLE, PARTITION, FILE, ROW_GROUP, NONE
+    protected abstract B self();
   }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
index cd49a1e..1ba1dd9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
@@ -29,7 +29,8 @@ import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.metastore.TableMetadata;
+import org.apache.drill.metastore.metadata.TableMetadata;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.fs.Path;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
index 4e9a762..30674e8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.physical.impl.statistics;
 
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 public abstract class AbstractMergedStatistic implements MergedStatistic, Statistic {
   protected String name;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
index 0a01e74..0b07213 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.record.MajorTypeSerDe;
 import org.apache.drill.exec.vector.NullableFloat8Vector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
   private Map<String, Double> sumHolder;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java
index 2d7194d..8d0a130 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java
@@ -24,6 +24,7 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 public class CntDupsMergedStatistic extends AbstractMergedStatistic {
     private Map<String, Long> sumHolder;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java
index 296d055..1be7c4a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java
@@ -23,6 +23,7 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VarCharVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 public class ColTypeMergedStatistic extends AbstractMergedStatistic {
   private Map<String, byte[]> typeHolder;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java
index 90d3b27..7eb3bf3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java
@@ -21,6 +21,7 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VarCharVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 public class ColumnMergedStatistic extends AbstractMergedStatistic {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java
index 68854f7..d01e604 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java
@@ -31,6 +31,7 @@ import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.vector.NullableVarBinaryVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 public class HLLMergedStatistic extends AbstractMergedStatistic {
   private Map<String, HyperLogLog> hllHolder;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java
index 7ffd0ae..8bfb765 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.physical.impl.statistics;
 
+import org.apache.drill.metastore.statistics.Statistic;
+
 import java.util.HashMap;
 
 public class MergedStatisticFactory {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
index 88e93d4..ad77b21 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
@@ -36,6 +36,7 @@ import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.apache.drill.exec.vector.NullableVarBinaryVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 public class NDVMergedStatistic extends AbstractMergedStatistic {
   private Map<String, HyperLogLog> hllHolder;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java
index a930502..1f28c14 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java
@@ -23,6 +23,7 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 public class NNRowCountMergedStatistic extends AbstractMergedStatistic {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java
index 9993031..f9dee3d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java
@@ -23,6 +23,7 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 public class RowCountMergedStatistic extends AbstractMergedStatistic {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java
index 40fc445..dcb6a2c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java
@@ -49,6 +49,7 @@ import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.FieldIdUtil;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 /*
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
index acf167b..15962ad 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
@@ -44,6 +44,7 @@ import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.DateVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/TDigestMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/TDigestMergedStatistic.java
index dc84eba..e06f129 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/TDigestMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/TDigestMergedStatistic.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.vector.NullableVarBinaryVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.metastore.statistics.Statistic;
 
 import java.util.HashMap;
 import java.util.Map;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
index 82e406a..03a4322 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
@@ -28,8 +28,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.drill.metastore.TableMetadata;
-import org.apache.drill.metastore.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.metastore.metadata.TableMetadata;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.hep.HepRelVertex;
 import org.apache.calcite.plan.volcano.RelSubset;
@@ -593,7 +593,7 @@ public abstract class DrillRelOptUtil {
         TableMetadata tableMetadata;
         return table == null
             || (tableMetadata = table.getGroupScan().getTableMetadata()) == null
-            || !((Boolean) TableStatisticsKind.HAS_STATISTICS.getValue(tableMetadata));
+            || !TableStatisticsKind.HAS_DESCRIPTIVE_STATISTICS.getValue(tableMetadata);
       } catch (IOException e) {
         RelOptPlanner.LOGGER.debug("Unable to obtain table metadata due to exception:", e);
         return true;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
index 7565abf..695cff6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
@@ -25,11 +25,13 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
 import com.fasterxml.jackson.databind.module.SimpleModule;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -50,9 +52,11 @@ import org.apache.drill.exec.store.dfs.FormatPlugin;
 import org.apache.drill.exec.store.dfs.FormatSelection;
 import org.apache.drill.exec.store.parquet.ParquetFormatConfig;
 import org.apache.drill.exec.util.ImpersonationUtil;
-import org.apache.drill.metastore.ColumnStatisticsKind;
-import org.apache.drill.metastore.StatisticsKind;
-import org.apache.drill.metastore.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.Histogram;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
+import org.apache.drill.metastore.statistics.StatisticsKind;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
@@ -452,53 +456,54 @@ public class DrillStatsTable {
         .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De())
         .addDeserializer(SchemaPath.class, new SchemaPath.De());
     mapper.registerModule(deModule);
+    mapper.registerSubtypes(new NamedType(NumericEquiDepthHistogram.class, "numeric-equi-depth"));
     return mapper;
   }
 
   /**
-   * Returns map of {@link StatisticsKind} and statistics values obtained from specified {@link DrillStatsTable}.
+   * Returns list of {@link StatisticsKind} and statistics values obtained from specified {@link DrillStatsTable}.
    *
    * @param statsProvider the source of statistics
-   * @return map of {@link StatisticsKind} and statistics values
+   * @return list of {@link StatisticsKind} and statistics values
    */
-  public static Map<StatisticsKind, Object> getEstimatedTableStats(DrillStatsTable statsProvider) {
+  public static List<StatisticsHolder> getEstimatedTableStats(DrillStatsTable statsProvider) {
     if (statsProvider != null && statsProvider.isMaterialized()) {
-      Map<StatisticsKind, Object> tableStatistics = new HashMap<>();
-      tableStatistics.put(TableStatisticsKind.EST_ROW_COUNT, statsProvider.getRowCount());
-      tableStatistics.put(TableStatisticsKind.HAS_STATISTICS, Boolean.TRUE);
+      List<StatisticsHolder> tableStatistics = Arrays.asList(
+          new StatisticsHolder<>(statsProvider.getRowCount(), TableStatisticsKind.EST_ROW_COUNT),
+          new StatisticsHolder<>(Boolean.TRUE, TableStatisticsKind.HAS_DESCRIPTIVE_STATISTICS));
       return tableStatistics;
     }
-    return Collections.emptyMap();
+    return Collections.emptyList();
   }
 
   /**
-   * Returns map of {@link StatisticsKind} and statistics values obtained from specified {@link DrillStatsTable} for specified column.
+   * Returns list of {@link StatisticsKind} and statistics values obtained from specified {@link DrillStatsTable} for specified column.
    *
    * @param statsProvider the source of statistics
    * @param fieldName     name of the columns whose statistics should be obtained
-   * @return map of {@link StatisticsKind} and statistics values
+   * @return list of {@link StatisticsKind} and statistics values
    */
-  public static Map<StatisticsKind, Object> getEstimatedColumnStats(DrillStatsTable statsProvider, SchemaPath fieldName) {
+  public static List<StatisticsHolder> getEstimatedColumnStats(DrillStatsTable statsProvider, SchemaPath fieldName) {
     if (statsProvider != null && statsProvider.isMaterialized()) {
-      Map<StatisticsKind, Object> statisticsValues = new HashMap<>();
+      List<StatisticsHolder> statisticsValues = new ArrayList<>();
       Double ndv = statsProvider.getNdv(fieldName);
       if (ndv != null) {
-        statisticsValues.put(ColumnStatisticsKind.NDV, ndv);
+        statisticsValues.add(new StatisticsHolder<>(ndv, ColumnStatisticsKind.NDV));
       }
       Double nonNullCount = statsProvider.getNNRowCount(fieldName);
       if (nonNullCount != null) {
-        statisticsValues.put(ColumnStatisticsKind.NON_NULL_COUNT, nonNullCount);
+        statisticsValues.add(new StatisticsHolder<>(nonNullCount, ColumnStatisticsKind.NON_NULL_COUNT));
       }
       Histogram histogram = statsProvider.getHistogram(fieldName);
       if (histogram != null) {
-        statisticsValues.put(ColumnStatisticsKind.HISTOGRAM, histogram);
+        statisticsValues.add(new StatisticsHolder<>(histogram, ColumnStatisticsKind.HISTOGRAM));
       }
       Double rowcount = statsProvider.getRowCount();
       if (rowcount != null) {
-        statisticsValues.put(ColumnStatisticsKind.ROWCOUNT, rowcount);
+        statisticsValues.add(new StatisticsHolder<>(rowcount, ColumnStatisticsKind.ROWCOUNT));
       }
       return statisticsValues;
     }
-    return Collections.emptyMap();
+    return Collections.emptyList();
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/HistogramUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/HistogramUtils.java
index f19b259..7f97fea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/HistogramUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/HistogramUtils.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.planner.common;
 
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.metastore.statistics.Histogram;
 
 public class HistogramUtils {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/NumericEquiDepthHistogram.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/NumericEquiDepthHistogram.java
index e78b437..69ec289 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/NumericEquiDepthHistogram.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/NumericEquiDepthHistogram.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.planner.common;
 
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 
@@ -30,6 +31,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexLiteral;
 import com.tdunning.math.stats.MergingDigest;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.drill.metastore.statistics.Histogram;
 import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.BoundType;
@@ -46,7 +48,7 @@ public class NumericEquiDepthHistogram implements Histogram {
    * histogram boundaries are approximate and even if some values lie outside the
    * range, we cannot be absolutely sure
    */
-  static final double SMALL_SELECTIVITY = 0.0001;
+  private static final double SMALL_SELECTIVITY = 0.0001;
 
   /** For equi-depth, all buckets will have same (approx) number of rows */
   @JsonProperty("numRowsPerBucket")
@@ -97,6 +99,7 @@ public class NumericEquiDepthHistogram implements Histogram {
    * number of buckets is 1 less than the total # entries in the buckets array since last
    * entry is the end point of the last bucket
    */
+  @JsonIgnore
   public int getNumBuckets() {
     return buckets.length - 1;
   }
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 d7f701e..6ddfd48 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
@@ -52,9 +52,9 @@ import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.util.Utilities;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsKind;
-import org.apache.drill.metastore.TableMetadata;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.metadata.TableMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -165,7 +165,7 @@ public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount{
       }
       ColumnStatistics columnStatistics = tableMetadata != null ?
           tableMetadata.getColumnStatistics(SchemaPath.getSimplePath(colName)) : null;
-      Double ndv = columnStatistics != null ? (Double) columnStatistics.getStatistic(ColumnStatisticsKind.NDV) : null;
+      Double ndv = columnStatistics != null ? ColumnStatisticsKind.NDV.getFrom(columnStatistics) : null;
       // Skip NDV, if not available
       if (ndv == null) {
         allColsHaveNDV = false;
@@ -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.getTableName() : ""), colName, groupKey.toString()));
+            (tableMetadata != null ? tableMetadata.getTableInfo().getName() : ""), 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/planner/cost/DrillRelMdRowCount.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
index b65c582..a24f5d1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
@@ -40,7 +40,7 @@ import org.apache.drill.exec.planner.physical.AggPrelBase;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.util.Utilities;
-import org.apache.drill.metastore.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
 
 
 public class DrillRelMdRowCount extends RelMdRowCount{
@@ -120,14 +120,14 @@ public class DrillRelMdRowCount extends RelMdRowCount{
     try {
       if (table != null
           && table.getGroupScan().getTableMetadata() != null
-          && (boolean) TableStatisticsKind.HAS_STATISTICS.getValue(table.getGroupScan().getTableMetadata())) {
+          && TableStatisticsKind.HAS_DESCRIPTIVE_STATISTICS.getValue(table.getGroupScan().getTableMetadata())) {
           /* For GroupScan rely on accurate count from the scan, if available, instead of
            * statistics since partition pruning/filter pushdown might have occurred.
            * e.g. ParquetGroupScan returns accurate rowcount. The other way would be to
            * iterate over the rowgroups present in the GroupScan to compute the rowcount.
            */
         if (!table.getGroupScan().getScanStats(settings).getGroupScanProperty().hasExactRowCount()) {
-          return (Double) TableStatisticsKind.EST_ROW_COUNT.getValue(table.getGroupScan().getTableMetadata());
+          return TableStatisticsKind.EST_ROW_COUNT.getValue(table.getGroupScan().getTableMetadata());
         } else {
           if (!(rel instanceof DrillScanRelBase)) {
             return table.getGroupScan().getScanStats(settings).getRecordCount();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
index 9a9b265..49d2129 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
@@ -54,17 +54,17 @@ import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.planner.common.DrillJoinRelBase;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.common.DrillScanRelBase;
-import org.apache.drill.exec.planner.common.Histogram;
+import org.apache.drill.metastore.statistics.Histogram;
 import org.apache.drill.exec.planner.logical.DrillScanRel;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.planner.physical.ScanPrel;
 import org.apache.drill.exec.util.Utilities;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsKind;
-import org.apache.drill.metastore.TableMetadata;
-import org.apache.drill.metastore.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.metadata.TableMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -145,7 +145,7 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
       try {
         TableMetadata tableMetadata;
         if (table != null && (tableMetadata = table.getGroupScan().getTableMetadata()) != null
-            && (boolean) TableStatisticsKind.HAS_STATISTICS.getValue(tableMetadata)) {
+            && TableStatisticsKind.HAS_DESCRIPTIVE_STATISTICS.getValue(tableMetadata)) {
           List<SchemaPath> fieldNames;
           if (rel instanceof DrillScanRelBase) {
             fieldNames = ((DrillScanRelBase) rel).getGroupScan().getColumns();
@@ -295,7 +295,7 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
     SchemaPath col = getColumn(orPred, fieldNames);
     if (col != null) {
       ColumnStatistics columnStatistics = tableMetadata != null ? tableMetadata.getColumnStatistics(col) : null;
-      Double ndv = columnStatistics != null ? (Double) columnStatistics.getStatistic(ColumnStatisticsKind.NDV) : null;
+      Double ndv = columnStatistics != null ? ColumnStatisticsKind.NDV.getFrom(columnStatistics) : null;
       if (ndv != null) {
         return 1.00 / ndv;
       }
@@ -308,10 +308,10 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
     SchemaPath col = getColumn(orPred, fieldNames);
     if (col != null) {
       ColumnStatistics columnStatistics = tableMetadata != null ? tableMetadata.getColumnStatistics(col) : null;
-      Histogram histogram = columnStatistics != null ? (Histogram) columnStatistics.getStatistic(ColumnStatisticsKind.HISTOGRAM) : null;
+      Histogram histogram = columnStatistics != null ? ColumnStatisticsKind.HISTOGRAM.getFrom(columnStatistics) : null;
       if (histogram != null) {
-        Double totalCount = (Double) columnStatistics.getStatistic(ColumnStatisticsKind.ROWCOUNT);
-        Double ndv = (Double) columnStatistics.getStatistic(ColumnStatisticsKind.NDV);
+        Double totalCount = ColumnStatisticsKind.ROWCOUNT.getFrom(columnStatistics);
+        Double ndv = ColumnStatisticsKind.NDV.getFrom(columnStatistics);
         Double sel = histogram.estimatedSelectivity(orPred, totalCount.longValue(), ndv.longValue());
         if (sel != null) {
           return sel;
@@ -325,10 +325,10 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
     SchemaPath col = getColumn(orPred, fieldNames);
     if (col != null) {
       ColumnStatistics columnStatistics = tableMetadata != null ? tableMetadata.getColumnStatistics(col) : null;
-      Double nonNullCount = columnStatistics != null ? (Double) columnStatistics.getStatistic(ColumnStatisticsKind.NON_NULL_COUNT) : null;
+      Double nonNullCount = columnStatistics != null ? ColumnStatisticsKind.NON_NULL_COUNT.getFrom(columnStatistics) : null;
       if (nonNullCount != null) {
         // Cap selectivity below Calcite Guess
-        return Math.min(nonNullCount / (Double) TableStatisticsKind.EST_ROW_COUNT.getValue(tableMetadata),
+        return Math.min(nonNullCount / TableStatisticsKind.EST_ROW_COUNT.getValue(tableMetadata),
             RelMdUtil.guessSelectivity(orPred));
       }
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ConvertCountToDirectScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ConvertCountToDirectScanRule.java
index 3a4e6ab..fb1bd2f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ConvertCountToDirectScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ConvertCountToDirectScanRule.java
@@ -32,7 +32,7 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfig;
 
 import org.apache.drill.exec.physical.base.ScanStats;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.exec.planner.common.CountToDirectScanUtils;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 
@@ -107,8 +107,8 @@ public class ConvertCountToDirectScanRule extends RelOptRule {
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final Aggregate agg = (Aggregate) call.rel(0);
-    final TableScan scan = (TableScan) call.rel(call.rels.length - 1);
+    final Aggregate agg = call.rel(0);
+    final TableScan scan = call.rel(call.rels.length - 1);
     final Project project = call.rels.length == 3 ? (Project) call.rel(1) : null;
 
     // Qualifying conditions for rule:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
index c53899e..14e7bac 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
@@ -30,9 +30,9 @@ import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
-import org.apache.drill.exec.physical.base.FileSystemMetadataProviderManager;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
-import org.apache.drill.exec.physical.base.TableMetadataProvider;
+import org.apache.drill.metastore.FileSystemMetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
 import org.apache.drill.exec.physical.base.SchemalessScan;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.server.options.SessionOptionManager;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
index a36af50..843fc9a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.planner.logical;
 
 import org.apache.calcite.schema.Schema;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.exec.planner.types.RelDataTypeDrillImpl;
 import org.apache.drill.exec.planner.types.RelDataTypeHolder;
 import org.apache.drill.exec.store.StoragePlugin;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java
index 86bed95..7efa0cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java
@@ -30,7 +30,7 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScanPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScanPrule.java
index 8d7022e..ac8f3ca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScanPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScanPrule.java
@@ -33,7 +33,7 @@ import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.ScanStats;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
 import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.DrillScanRel;
@@ -97,8 +97,8 @@ public class ConvertCountToDirectScanPrule extends Prule {
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final DrillAggregateRel agg = (DrillAggregateRel) call.rel(0);
-    final DrillScanRel scan = (DrillScanRel) call.rel(call.rels.length - 1);
+    final DrillAggregateRel agg = call.rel(0);
+    final DrillScanRel scan = call.rel(call.rels.length - 1);
     final DrillProjectRel project = call.rels.length == 3 ? (DrillProjectRel) call.rel(1) : null;
 
     final GroupScan oldGrpScan = scan.getGroupScan();
@@ -115,7 +115,7 @@ public class ConvertCountToDirectScanPrule extends Prule {
     }
 
     Map<String, Long> result = collectCounts(settings, agg, scan, project);
-    logger.trace("Calculated the following aggregate counts: ", result);
+    logger.trace("Calculated the following aggregate counts: {}", result);
     // if could not determine the counts, rule won't be applied
     if (result.isEmpty()) {
       return;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
index e6de62e..d766515 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -32,8 +32,8 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.drill.exec.planner.sql.parser.DrillParserUtil;
 import org.apache.drill.exec.planner.sql.parser.impl.DrillSqlParseException;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
-import org.apache.drill.exec.physical.base.TableMetadataProvider;
+import org.apache.drill.metastore.MetadataProviderManager;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
 import org.apache.drill.shaded.guava.com.google.common.cache.CacheBuilder;
 import org.apache.drill.shaded.guava.com.google.common.cache.CacheLoader;
 import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
index 2dfb382..c283c71 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
@@ -27,7 +27,7 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.exec.planner.PlannerPhase;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
index 1ae2d5e..57341cd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
@@ -71,7 +71,7 @@ public class ColumnExplorer {
 
   /**
    * Constructor for using the column explorer to probe existing columns in the
-   * {@link ProjectRecordBatch}.
+   * {@link org.apache.drill.exec.physical.impl.project.ProjectRecordBatch}.
    */
   // TODO: This is awkward. This class is being used for two distinct things:
   // 1. The definition of the metadata columns, and
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
index e850341..aa3bc53 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
@@ -28,7 +28,7 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.exec.server.options.SessionOptionManager;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
index 6de273a..02cc576 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
@@ -35,7 +35,7 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.SessionOptionManager;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
index eb50ed8..a839c1f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
@@ -26,7 +26,7 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.physical.base.AbstractWriter;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
 import org.apache.drill.exec.server.DrillbitContext;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index d11fb70..5588eda 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -51,8 +51,8 @@ import org.apache.drill.exec.dotdrill.DotDrillFile;
 import org.apache.drill.exec.dotdrill.DotDrillType;
 import org.apache.drill.exec.dotdrill.DotDrillUtil;
 import org.apache.drill.exec.dotdrill.View;
-import org.apache.drill.exec.physical.base.FileSystemMetadataProviderManager;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
+import org.apache.drill.metastore.FileSystemMetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.planner.logical.DrillTable;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index 4a2790e..302f61a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -33,7 +33,6 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.physical.base.AbstractWriter;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
@@ -64,6 +63,7 @@ import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.store.dfs.FormatMatcher;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
 import org.apache.drill.exec.store.schedule.CompleteFileWork;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.shaded.guava.com.google.common.base.Functions;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
index 8acd777..c71cc3d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
@@ -26,13 +26,13 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.EndpointAffinity;
 import org.apache.drill.exec.physical.base.AbstractFileGroupScan;
-import org.apache.drill.exec.physical.base.FileSystemMetadataProviderManager;
+import org.apache.drill.metastore.FileSystemMetadataProviderManager;
 import org.apache.drill.exec.physical.base.FileGroupScan;
 import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
-import org.apache.drill.exec.physical.base.TableMetadataProvider;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -53,7 +53,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.drill.metastore.TableMetadata;
+import org.apache.drill.metastore.metadata.TableMetadata;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Iterators;
 import org.apache.drill.shaded.guava.com.google.common.collect.ListMultimap;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/SimpleFileTableMetadataProviderBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/SimpleFileTableMetadataProviderBuilder.java
index 6746b97..a6432e1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/SimpleFileTableMetadataProviderBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/SimpleFileTableMetadataProviderBuilder.java
@@ -17,11 +17,12 @@
  */
 package org.apache.drill.exec.store.dfs.easy;
 
-import org.apache.drill.exec.physical.base.TableMetadataProviderBuilder;
+import org.apache.drill.metastore.SimpleFileTableMetadataProvider;
+import org.apache.drill.metastore.metadata.TableMetadataProviderBuilder;
 import org.apache.hadoop.fs.Path;
 
 /**
- * Builder for {@link org.apache.drill.exec.physical.base.SimpleFileTableMetadataProvider}.
+ * Builder for {@link SimpleFileTableMetadataProvider}.
  */
 public interface SimpleFileTableMetadataProviderBuilder extends TableMetadataProviderBuilder {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonStatisticsRecordWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonStatisticsRecordWriter.java
index f0b3bc4..3be8996 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonStatisticsRecordWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonStatisticsRecordWriter.java
@@ -26,7 +26,7 @@ import java.util.Map;
 
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.planner.common.DrillStatsTable.STATS_VERSION;
 import org.apache.drill.exec.record.VectorAccessible;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
index 89ef8f0..e0ec1c1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
@@ -33,7 +33,7 @@ import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.physical.impl.scan.columns.ColumnsScanFramework.ColumnsScanBuilder;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
index d5a752f..e9dc8a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
@@ -23,13 +23,16 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.drill.common.expression.ExpressionStringBuilder;
 import org.apache.drill.exec.physical.base.AbstractGroupScanWithMetadata;
-import org.apache.drill.exec.physical.base.ParquetMetadataProvider;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
-import org.apache.drill.metastore.BaseMetadata;
-import org.apache.drill.metastore.LocationProvider;
-import org.apache.drill.metastore.PartitionMetadata;
-import org.apache.drill.metastore.TableStatisticsKind;
-import org.apache.drill.exec.expr.ExactStatisticsConstants;
+import org.apache.drill.metastore.metadata.SegmentMetadata;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.ParquetMetadataProvider;
+import org.apache.drill.metastore.statistics.Statistic;
+import org.apache.drill.metastore.metadata.BaseMetadata;
+import org.apache.drill.metastore.metadata.LocationProvider;
+import org.apache.drill.metastore.metadata.PartitionMetadata;
+import org.apache.drill.metastore.util.TableMetadataUtils;
+import org.apache.drill.metastore.statistics.ExactStatisticsConstants;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.ArrayListMultimap;
 import org.apache.drill.shaded.guava.com.google.common.collect.LinkedListMultimap;
@@ -49,8 +52,8 @@ import org.apache.drill.exec.store.schedule.AffinityCreator;
 import org.apache.drill.exec.store.schedule.AssignmentCreator;
 import org.apache.drill.exec.store.schedule.EndpointByteMap;
 import org.apache.drill.exec.store.schedule.EndpointByteMapImpl;
-import org.apache.drill.metastore.FileMetadata;
-import org.apache.drill.metastore.RowGroupMetadata;
+import org.apache.drill.metastore.metadata.FileMetadata;
+import org.apache.drill.metastore.metadata.RowGroupMetadata;
 import org.apache.drill.exec.expr.FilterPredicate;
 import org.apache.drill.shaded.guava.com.google.common.collect.Multimap;
 import org.apache.hadoop.fs.Path;
@@ -154,11 +157,11 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
 
       rowGroupInfos = new ArrayList<>();
       for (RowGroupMetadata rowGroupMetadata : getRowGroupsMetadata().values()) {
-        RowGroupInfo rowGroupInfo = new RowGroupInfo(rowGroupMetadata.getLocation(),
-            (long) rowGroupMetadata.getStatistic(() -> ExactStatisticsConstants.START),
-            (long) rowGroupMetadata.getStatistic(() -> ExactStatisticsConstants.LENGTH),
+        RowGroupInfo rowGroupInfo = new RowGroupInfo(rowGroupMetadata.getPath(),
+            rowGroupMetadata.getStatistic(() -> ExactStatisticsConstants.START),
+            rowGroupMetadata.getStatistic(() -> ExactStatisticsConstants.LENGTH),
             rowGroupMetadata.getRowGroupIndex(),
-            (long) rowGroupMetadata.getStatistic(TableStatisticsKind.ROW_COUNT));
+            TableStatisticsKind.ROW_COUNT.getValue(rowGroupMetadata));
         rowGroupInfo.setNumRecordsToRead(rowGroupInfo.getRowCount());
 
         EndpointByteMap endpointByteMap = new EndpointByteMapImpl();
@@ -210,6 +213,17 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
     return readEntries;
   }
 
+  /**
+   * {@inheritDoc}
+   * <ul>
+   * <ul><li>file metadata was pruned, prunes underlying metadata</li></ul>
+   * <li>row group level:
+   * <ul><li>if filter matches all the the data or prunes all the data, sets corresponding value to
+   * {@link AbstractParquetGroupScan#isMatchAllMetadata()} and returns null</li></ul></li>
+   * </ul>
+   *
+   * @return group scan with applied filter expression
+   */
   @Override
   public AbstractGroupScanWithMetadata applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities,
       FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) {
@@ -223,46 +237,16 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
     Set<SchemaPath> schemaPathsInExpr =
         filterExpr.accept(new FilterEvaluatorUtils.FieldReferenceFinder(), null);
 
-    RowGroupScanFilterer builder = getFilterer().getFiltered(optionManager, filterPredicate, schemaPathsInExpr);
+    RowGroupScanFilterer filteredMetadata = getFilterer().getFiltered(optionManager, filterPredicate, schemaPathsInExpr);
 
     // checks whether metadata for specific level was available and there was no reduction of metadata
-    if (!getRowGroupsMetadata().isEmpty()) {
-      if (!builder.getRowGroups().isEmpty() && getRowGroupsMetadata().size() == builder.getRowGroups().size()) {
-        // There is no reduction of files
-        logger.debug("applyFilter() does not have any pruning");
-        matchAllMetadata = builder.isMatchAllMetadata();
-        return null;
-      }
-    } else if (!getFilesMetadata().isEmpty()) {
-      if (!builder.getFiles().isEmpty() && getFilesMetadata().size() == builder.getFiles().size()) {
-        // There is no reduction of files
-        logger.debug("applyFilter() does not have any pruning");
-        matchAllMetadata = builder.isMatchAllMetadata();
-        return null;
-      }
-    } else if (!getPartitionsMetadata().isEmpty()) {
-      if (!builder.getPartitions().isEmpty() && getPartitionsMetadata().size() == builder.getPartitions().size()) {
-        // There is no reduction of partitions
-        logger.debug("applyFilter() does not have any pruning ");
-        matchAllMetadata = builder.isMatchAllMetadata();
-        return null;
-      }
-    } else if (getTableMetadata() != null) {
-      // There is no reduction
+    if (isGroupScanFullyMatchesFilter(filteredMetadata)) {
       logger.debug("applyFilter() does not have any pruning");
-      matchAllMetadata = builder.isMatchAllMetadata();
+      matchAllMetadata = filteredMetadata.isMatchAllMetadata();
       return null;
     }
 
-    if (!builder.isMatchAllMetadata()
-        // filter returns empty result using table metadata
-        && ((builder.getTableMetadata() == null && getTableMetadata() != null)
-            // all partitions pruned if partition metadata is available
-            || builder.getPartitions().isEmpty() && !getPartitionsMetadata().isEmpty())
-            // all files are pruned if file metadata is available
-            || builder.getFiles().isEmpty() && !getFilesMetadata().isEmpty()
-            // all row groups are pruned if row group metadata is available
-            || builder.getRowGroups().isEmpty() && !getRowGroupsMetadata().isEmpty()) {
+    if (isAllDataPruned(filteredMetadata)) {
       if (getRowGroupsMetadata().size() == 1) {
         // For the case when group scan has single row group and it was filtered,
         // no need to create new group scan with the same row group.
@@ -281,31 +265,54 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
 
       logger.debug("All row groups have been filtered out. Add back one to get schema from scanner");
 
+      Map<Path, SegmentMetadata> segmentsMap = getNextOrEmpty(getSegmentsMetadata().values()).stream()
+          .collect(Collectors.toMap(SegmentMetadata::getPath, Function.identity()));
+
       Map<Path, FileMetadata> filesMap = getNextOrEmpty(getFilesMetadata().values()).stream()
-          .collect(Collectors.toMap(FileMetadata::getLocation, Function.identity()));
+          .collect(Collectors.toMap(FileMetadata::getPath, Function.identity()));
 
       Multimap<Path, RowGroupMetadata> rowGroupsMap = LinkedListMultimap.create();
-      getNextOrEmpty(getRowGroupsMetadata().values()).forEach(entry -> rowGroupsMap.put(entry.getLocation(), entry));
-
-      builder.withRowGroups(rowGroupsMap)
-          .withTable(getTableMetadata())
-          .withPartitions(getNextOrEmpty(getPartitionsMetadata()))
-          .withNonInterestingColumns(getNonInterestingColumnsMetadata())
-          .withFiles(filesMap)
-          .withMatching(false);
+      getNextOrEmpty(getRowGroupsMetadata().values()).forEach(entry -> rowGroupsMap.put(entry.getPath(), entry));
+
+      filteredMetadata.rowGroups(rowGroupsMap)
+          .table(getTableMetadata())
+          .segments(segmentsMap)
+          .partitions(getNextOrEmpty(getPartitionsMetadata()))
+          .nonInterestingColumns(getNonInterestingColumnsMetadata())
+          .files(filesMap)
+          .matching(false);
     }
 
-    if (builder.getOverflowLevel() != MetadataLevel.NONE) {
-      logger.warn("applyFilter {} wasn't able to do pruning for  all metadata levels filter condition, since metadata count for " +
-            "{} level exceeds `planner.store.parquet.rowgroup.filter.pushdown.threshold` value.\n" +
-            "But underlying metadata was pruned without filter expression according to the metadata with above level.",
-          ExpressionStringBuilder.toString(filterExpr), builder.getOverflowLevel());
+    if (filteredMetadata.getOverflowLevel() != MetadataType.NONE) {
+      if (logger.isWarnEnabled()) {
+        logger.warn("applyFilter {} wasn't able to do pruning for  all metadata levels filter condition, since metadata count for " +
+              "{} level exceeds `planner.store.parquet.rowgroup.filter.pushdown.threshold` value.\n" +
+              "But underlying metadata was pruned without filter expression according to the metadata with above level.",
+            ExpressionStringBuilder.toString(filterExpr), filteredMetadata.getOverflowLevel());
+      }
     }
 
-    logger.debug("applyFilter {} reduce row groups # from {} to {}",
-        ExpressionStringBuilder.toString(filterExpr), getRowGroupsMetadata().size(), builder.getRowGroups().size());
+    if (logger.isDebugEnabled()) {
+      logger.debug("applyFilter {} reduce row groups # from {} to {}",
+          ExpressionStringBuilder.toString(filterExpr), getRowGroupsMetadata().size(), filteredMetadata.getRowGroups().size());
+    }
 
-    return builder.build();
+    return filteredMetadata.build();
+  }
+
+  private boolean isAllDataPruned(RowGroupScanFilterer filteredMetadata) {
+    return !filteredMetadata.isMatchAllMetadata()
+        && (super.isAllDataPruned(filteredMetadata)
+            // all row groups are pruned if row group metadata is available
+            || filteredMetadata.getRowGroups().isEmpty() && !getRowGroupsMetadata().isEmpty());
+  }
+
+  private boolean isGroupScanFullyMatchesFilter(RowGroupScanFilterer filteredMetadata) {
+    if (!getRowGroupsMetadata().isEmpty()) {
+      return getRowGroupsMetadata().size() == filteredMetadata.getRowGroups().size();
+    } else {
+      return super.isGroupScanFullyMatchesFilter(filteredMetadata);
+    }
   }
 
   // narrows the return type
@@ -331,7 +338,7 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
   public GroupScan applyLimit(int maxRecords) {
     maxRecords = Math.max(maxRecords, 1); // Make sure it request at least 1 row -> 1 rowGroup.
     if (getTableMetadata() != null) {
-      long tableRowCount = (long) TableStatisticsKind.ROW_COUNT.getValue(getTableMetadata());
+      long tableRowCount = TableStatisticsKind.ROW_COUNT.getValue(getTableMetadata());
       if (tableRowCount == Statistic.NO_COLUMN_STATS || tableRowCount <= maxRecords) {
         logger.debug("limit push down does not apply, since total number of rows [{}] is less or equal to the required [{}].",
             tableRowCount, maxRecords);
@@ -346,26 +353,26 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
       return null;
     }
 
-    Map<Path, FileMetadata> qualifiedFiles;
     Map<Path, FileMetadata> filesMetadata = getFilesMetadata();
-    qualifiedFiles = qualifiedRowGroups.stream()
-        .map(rowGroup -> filesMetadata.get(rowGroup.getLocation()))
+    Map<Path, FileMetadata> qualifiedFiles = qualifiedRowGroups.stream()
+        .map(rowGroup -> filesMetadata.get(rowGroup.getPath()))
         .filter(Objects::nonNull)
-        .collect(Collectors.toMap(FileMetadata::getLocation, Function.identity()));
+        .collect(Collectors.toMap(FileMetadata::getPath, Function.identity()));
 
     Multimap<Path, RowGroupMetadata> prunedRowGroups = LinkedListMultimap.create();
 
     for (RowGroupMetadata qualifiedRowGroup : qualifiedRowGroups) {
-      prunedRowGroups.put(qualifiedRowGroup.getLocation(), qualifiedRowGroup);
+      prunedRowGroups.put(qualifiedRowGroup.getPath(), qualifiedRowGroup);
     }
 
     return getFilterer()
-        .withRowGroups(prunedRowGroups)
-        .withTable(getTableMetadata())
-        .withPartitions(getPartitionsMetadata())
-        .withFiles(qualifiedFiles)
-        .withNonInterestingColumns(getNonInterestingColumnsMetadata())
-        .withMatching(matchAllMetadata)
+        .rowGroups(prunedRowGroups)
+        .table(getTableMetadata())
+        .partitions(getPartitionsMetadata())
+        .segments(getSegmentsMetadata())
+        .files(qualifiedFiles)
+        .nonInterestingColumns(getNonInterestingColumnsMetadata())
+        .matching(matchAllMetadata)
         .build();
   }
   // limit push down methods end
@@ -391,7 +398,7 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
     }
     this.rowGroups = newRowGroups;
 
-    tableMetadata = ParquetTableMetadataUtils.updateRowCount(getTableMetadata(), getRowGroupsMetadata().values());
+    tableMetadata = TableMetadataUtils.updateRowCount(getTableMetadata(), getRowGroupsMetadata().values());
 
     if (!getFilesMetadata().isEmpty()) {
       this.files = getFilesMetadata().entrySet().stream()
@@ -414,6 +421,12 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
     }
     partitions = newPartitions;
 
+    if (!getSegmentsMetadata().isEmpty()) {
+      this.segments = getSegmentsMetadata().entrySet().stream()
+          .filter(entry -> fileSet.contains(entry.getKey()))
+          .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
     rowGroupInfos = null;
   }
 
@@ -466,16 +479,16 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
   /**
    * This class is responsible for filtering different metadata levels including row group level.
    */
-  protected abstract static class RowGroupScanFilterer extends GroupScanWithMetadataFilterer {
+  protected abstract static class RowGroupScanFilterer<B extends RowGroupScanFilterer<B>> extends GroupScanWithMetadataFilterer<B> {
     protected Multimap<Path, RowGroupMetadata> rowGroups = LinkedListMultimap.create();
 
     public RowGroupScanFilterer(AbstractGroupScanWithMetadata source) {
       super(source);
     }
 
-    public RowGroupScanFilterer withRowGroups(Multimap<Path, RowGroupMetadata> rowGroups) {
+    public B rowGroups(Multimap<Path, RowGroupMetadata> rowGroups) {
       this.rowGroups = rowGroups;
-      return this;
+      return self();
     }
 
     /**
@@ -496,9 +509,10 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
       newScan.tableMetadata = tableMetadata;
       // updates common row count and nulls counts for every column
       if (newScan.getTableMetadata() != null && rowGroups != null && newScan.getRowGroupsMetadata().size() != rowGroups.size()) {
-        newScan.tableMetadata = ParquetTableMetadataUtils.updateRowCount(newScan.getTableMetadata(), rowGroups.values());
+        newScan.tableMetadata = TableMetadataUtils.updateRowCount(newScan.getTableMetadata(), rowGroups.values());
       }
       newScan.partitions = partitions;
+      newScan.segments = segments;
       newScan.files = files;
       newScan.rowGroups = rowGroups;
       newScan.matchAllMetadata = matchAllMetadata;
@@ -524,13 +538,13 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
     }
 
     @Override
-    protected RowGroupScanFilterer getFiltered(OptionManager optionManager, FilterPredicate filterPredicate, Set<SchemaPath> schemaPathsInExpr) {
+    protected B getFiltered(OptionManager optionManager, FilterPredicate filterPredicate, Set<SchemaPath> schemaPathsInExpr) {
       super.getFiltered(optionManager, filterPredicate, schemaPathsInExpr);
 
       if (!((AbstractParquetGroupScan) source).getRowGroupsMetadata().isEmpty()) {
         filterRowGroupMetadata(optionManager, filterPredicate, schemaPathsInExpr);
       }
-      return this;
+      return self();
     }
 
     /**
@@ -553,6 +567,10 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
           && abstractParquetGroupScan.getPartitionsMetadata().size() > getPartitions().size()) {
         // prunes row groups to leave only row groups which are contained by pruned partitions
         prunedRowGroups = pruneForPartitions(abstractParquetGroupScan.getRowGroupsMetadata(), getPartitions());
+      } else if (!abstractParquetGroupScan.getSegmentsMetadata().isEmpty()
+          && abstractParquetGroupScan.getSegmentsMetadata().size() > getSegments().size()) {
+        // prunes row groups to leave only row groups which are contained by pruned segments
+        prunedRowGroups = pruneForSegments(abstractParquetGroupScan.getRowGroupsMetadata(), getSegments());
       } else {
         // no partition or file pruning happened, no need to prune initial row groups list
         prunedRowGroups = abstractParquetGroupScan.getRowGroupsMetadata();
@@ -571,13 +589,38 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
         List<RowGroupMetadata> filteredRowGroups = filterAndGetMetadata(schemaPathsInExpr, prunedRowGroups.values(), filterPredicate, optionManager);
 
         this.rowGroups = LinkedListMultimap.create();
-        filteredRowGroups.forEach(entry -> this.rowGroups.put(entry.getLocation(), entry));
+        filteredRowGroups.forEach(entry -> this.rowGroups.put(entry.getPath(), entry));
       } else {
         this.rowGroups = prunedRowGroups;
         matchAllMetadata = false;
-        overflowLevel = MetadataLevel.ROW_GROUP;
+        overflowLevel = MetadataType.ROW_GROUP;
       }
     }
+
+    /**
+     * Removes metadata which does not belong to any of segments in metadata list.
+     *
+     * @param metadataToPrune         list of metadata which should be pruned
+     * @param filteredSegmentMetadata list of segment metadata which was pruned
+     * @param <T>                     type of metadata to filter
+     * @return multimap with metadata which belongs to pruned segments
+     */
+    protected static <T extends BaseMetadata & LocationProvider> Multimap<Path, T> pruneForSegments(
+        Multimap<Path, T> metadataToPrune, Map<Path, SegmentMetadata> filteredSegmentMetadata) {
+        Multimap<Path, T> prunedFiles = LinkedListMultimap.create();
+      if (metadataToPrune != null) {
+        for (Map.Entry<Path, T> entry : metadataToPrune.entries()) {
+          for (SegmentMetadata filteredPartition : filteredSegmentMetadata.values()) {
+            if (filteredPartition.getLocations().contains(entry.getKey())) {
+              prunedFiles.put(entry.getKey(), entry.getValue());
+              break;
+            }
+          }
+        }
+      }
+
+      return prunedFiles;
+    }
   }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
index d95ee6d..586e094 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.store.CommonParquetRecordReader;
 import org.apache.drill.exec.store.parquet.metadata.Metadata;
 import org.apache.drill.exec.store.parquet.metadata.MetadataBase;
 import org.apache.drill.exec.store.parquet.metadata.Metadata_V4;
-import org.apache.drill.metastore.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
 import org.apache.drill.shaded.guava.com.google.common.base.Functions;
 import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
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 42b22ec..ac6190b 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
@@ -17,17 +17,22 @@
  */
 package org.apache.drill.exec.store.parquet;
 
-import org.apache.drill.exec.physical.base.ParquetMetadataProvider;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
+import org.apache.drill.metastore.ParquetMetadataProvider;
+import org.apache.drill.metastore.metadata.LocationProvider;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.SegmentMetadata;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.record.metadata.MetadataUtils;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.metastore.BaseMetadata;
-import org.apache.drill.metastore.ColumnStatisticsImpl;
-import org.apache.drill.metastore.NonInterestingColumnsMetadata;
-import org.apache.drill.metastore.StatisticsKind;
-import org.apache.drill.metastore.TableMetadata;
-import org.apache.drill.metastore.TableStatisticsKind;
+import org.apache.drill.metastore.metadata.BaseMetadata;
+import org.apache.drill.metastore.metadata.NonInterestingColumnsMetadata;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
+import org.apache.drill.metastore.metadata.TableMetadata;
+import org.apache.drill.metastore.util.TableMetadataUtils;
 import org.apache.drill.shaded.guava.com.google.common.collect.HashBasedTable;
 import org.apache.drill.shaded.guava.com.google.common.collect.HashMultimap;
 import org.apache.drill.shaded.guava.com.google.common.collect.LinkedListMultimap;
@@ -36,20 +41,21 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Table;
 
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.record.metadata.SchemaPathUtils;
+import org.apache.drill.metastore.util.SchemaPathUtils;
 import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.dfs.ReadEntryWithPath;
 import org.apache.drill.exec.store.parquet.metadata.MetadataBase;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsKind;
-import org.apache.drill.metastore.FileMetadata;
-import org.apache.drill.metastore.PartitionMetadata;
-import org.apache.drill.metastore.RowGroupMetadata;
-import org.apache.drill.metastore.FileTableMetadata;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.metadata.FileMetadata;
+import org.apache.drill.metastore.metadata.PartitionMetadata;
+import org.apache.drill.metastore.metadata.RowGroupMetadata;
+import org.apache.drill.metastore.metadata.BaseTableMetadata;
 import org.apache.hadoop.fs.Path;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -60,6 +66,8 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import static org.apache.drill.exec.store.parquet.ParquetTableMetadataUtils.PARQUET_COLUMN_STATISTICS;
+
 /**
  * Implementation of {@link ParquetMetadataProvider} which contains base methods for obtaining metadata from
  * parquet statistics.
@@ -89,6 +97,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
   private TableMetadata tableMetadata;
   private List<PartitionMetadata> partitions;
   private Map<Path, FileMetadata> files;
+  private Map<Path, SegmentMetadata> segments;
   private NonInterestingColumnsMetadata nonInterestingColumnsMetadata;
 
   // whether metadata for row groups should be collected to create files, partitions and table metadata
@@ -160,10 +169,11 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
             .forEach(entry -> rowGroups.put(entry.getKey(), entry.getValue()));
       }
       TableMetadata tableMetadata = getTableMetadata();
+      getSegmentsMetadataMap();
       getPartitionsMetadata();
       getRowGroupsMeta();
-      getNonInterestingColumnsMeta();
-      this.tableMetadata = ParquetTableMetadataUtils.updateRowCount(tableMetadata, getRowGroupsMeta());
+      getNonInterestingColumnsMetadata();
+      this.tableMetadata = TableMetadataUtils.updateRowCount(tableMetadata, getRowGroupsMeta());
       parquetTableMetadata = null;
     }
   }
@@ -173,20 +183,21 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
    * Once deserialization and serialization from/into metastore classes is done, this method should be removed
    * to allow lazy initialization.
    */
-  public void initializeMetadata() throws IOException {
+  public void initializeMetadata() {
     if (statsTable != null && !statsTable.isMaterialized()) {
       statsTable.materialize();
     }
     getTableMetadata();
-    getFilesMetadata();
+    getSegmentsMetadataMap();
+    getFilesMetadataMap();
     getPartitionsMetadata();
     getRowGroupsMeta();
-    getNonInterestingColumnsMeta();
+    getNonInterestingColumnsMetadata();
     parquetTableMetadata = null;
   }
 
   @Override
-  public NonInterestingColumnsMetadata getNonInterestingColumnsMeta() {
+  public NonInterestingColumnsMetadata getNonInterestingColumnsMetadata() {
     if (nonInterestingColumnsMetadata == null) {
       nonInterestingColumnsMetadata = ParquetTableMetadataUtils.getNonInterestingColumnsMeta(parquetTableMetadata);
     }
@@ -194,11 +205,9 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
   }
 
   @Override
-  @SuppressWarnings("unchecked")
   public TableMetadata getTableMetadata() {
     if (tableMetadata == null) {
-      Map<StatisticsKind, Object> tableStatistics = new HashMap<>(DrillStatsTable.getEstimatedTableStats(statsTable));
-      Set<String> partitionKeys = new HashSet<>();
+      List<StatisticsHolder> tableStatistics = new ArrayList<>(DrillStatsTable.getEstimatedTableStats(statsTable));
       Map<SchemaPath, TypeProtos.MajorType> fields = ParquetTableMetadataUtils.resolveFields(parquetTableMetadata);
 
       if (this.schema == null) {
@@ -215,41 +224,48 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
 
       Map<SchemaPath, ColumnStatistics> columnsStatistics;
       if (collectMetadata) {
-        List<? extends BaseMetadata> metadata = getFilesMetadata();
-        if (metadata == null || metadata.isEmpty()) {
+        Collection<? extends BaseMetadata> metadata = getFilesMetadataMap().values();
+        if (metadata.isEmpty()) {
           metadata = getRowGroupsMeta();
         }
-        tableStatistics.put(TableStatisticsKind.ROW_COUNT, TableStatisticsKind.ROW_COUNT.mergeStatistics(metadata));
-        columnsStatistics = ParquetTableMetadataUtils.mergeColumnsStatistics(metadata, fields.keySet(), ParquetTableMetadataUtils.PARQUET_STATISTICS, parquetTableMetadata);
+        tableStatistics.add(new StatisticsHolder<>(TableStatisticsKind.ROW_COUNT.mergeStatistics(metadata), TableStatisticsKind.ROW_COUNT));
+        columnsStatistics = TableMetadataUtils.mergeColumnsStatistics(metadata, fields.keySet(), PARQUET_COLUMN_STATISTICS);
       } else {
         columnsStatistics = new HashMap<>();
-        tableStatistics.put(TableStatisticsKind.ROW_COUNT, getParquetGroupScanStatistics().getRowCount());
+        tableStatistics.add(new StatisticsHolder<>(getParquetGroupScanStatistics().getRowCount(), TableStatisticsKind.ROW_COUNT));
 
         Set<SchemaPath> unhandledColumns = new HashSet<>();
         if (statsTable != null && statsTable.isMaterialized()) {
           unhandledColumns.addAll(statsTable.getColumns());
         }
 
-        for (SchemaPath partitionColumn : fields.keySet()) {
-          long columnValueCount = getParquetGroupScanStatistics().getColumnValueCount(partitionColumn);
+        fields.forEach((columnPath, value) -> {
+          long columnValueCount = getParquetGroupScanStatistics().getColumnValueCount(columnPath);
           // Adds statistics values itself if statistics is available
-          Map<StatisticsKind, Object> stats = new HashMap<>(DrillStatsTable.getEstimatedColumnStats(statsTable, partitionColumn));
-          unhandledColumns.remove(partitionColumn);
+          List<StatisticsHolder> stats = new ArrayList<>(DrillStatsTable.getEstimatedColumnStats(statsTable, columnPath));
+          unhandledColumns.remove(columnPath);
 
           // adds statistics for partition columns
-          stats.put(TableStatisticsKind.ROW_COUNT, columnValueCount);
-          stats.put(ColumnStatisticsKind.NULLS_COUNT, getParquetGroupScanStatistics().getRowCount() - columnValueCount);
-          columnsStatistics.put(partitionColumn, new ColumnStatisticsImpl(stats, ParquetTableMetadataUtils.getNaturalNullsFirstComparator()));
-        }
+          stats.add(new StatisticsHolder<>(columnValueCount, TableStatisticsKind.ROW_COUNT));
+          stats.add(new StatisticsHolder<>(getParquetGroupScanStatistics().getRowCount() - columnValueCount, ColumnStatisticsKind.NULLS_COUNT));
+          columnsStatistics.put(columnPath, new ColumnStatistics(stats, value.getMinorType()));
+        });
 
         for (SchemaPath column : unhandledColumns) {
           columnsStatistics.put(column,
-              new ColumnStatisticsImpl(DrillStatsTable.getEstimatedColumnStats(statsTable, column),
-                  ParquetTableMetadataUtils.getNaturalNullsFirstComparator()));
+              new ColumnStatistics(DrillStatsTable.getEstimatedColumnStats(statsTable, column)));
         }
       }
-      tableMetadata = new FileTableMetadata(tableName, tableLocation, schema, columnsStatistics, tableStatistics,
-          -1L, "", partitionKeys);
+      MetadataInfo metadataInfo = new MetadataInfo(MetadataType.TABLE, MetadataInfo.GENERAL_INFO_KEY, null);
+      tableMetadata = BaseTableMetadata.builder()
+          .tableInfo(TableInfo.UNKNOWN_TABLE_INFO)
+          .metadataInfo(metadataInfo)
+          .location(tableLocation)
+          .schema(schema)
+          .columnsStatistics(columnsStatistics)
+          .metadataStatistics(tableStatistics)
+          .partitionKeys(Collections.emptyMap())
+          .build();
     }
 
     return tableMetadata;
@@ -258,7 +274,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
   private ParquetGroupScanStatistics<? extends BaseMetadata> getParquetGroupScanStatistics() {
     if (parquetGroupScanStatistics == null) {
       if (collectMetadata) {
-        parquetGroupScanStatistics = new ParquetGroupScanStatistics<>(getFilesMetadata());
+        parquetGroupScanStatistics = new ParquetGroupScanStatistics<>(getFilesMetadataMap().values());
       } else {
         parquetGroupScanStatistics = new ParquetGroupScanStatistics<>(getRowGroupsMeta());
       }
@@ -275,18 +291,17 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
   }
 
   @Override
-  @SuppressWarnings("unchecked")
   public List<PartitionMetadata> getPartitionsMetadata() {
     if (partitions == null) {
       partitions = new ArrayList<>();
       if (collectMetadata) {
         Table<SchemaPath, Object, List<FileMetadata>> colValFile = HashBasedTable.create();
 
-        List<FileMetadata> filesMetadata = getFilesMetadata();
+        Collection<FileMetadata> filesMetadata = getFilesMetadataMap().values();
         partitionColumns = getParquetGroupScanStatistics().getPartitionColumns();
         for (FileMetadata fileMetadata : filesMetadata) {
           for (SchemaPath partitionColumn : partitionColumns) {
-            Object partitionValue = getParquetGroupScanStatistics().getPartitionValue(fileMetadata.getLocation(), partitionColumn);
+            Object partitionValue = getParquetGroupScanStatistics().getPartitionValue(fileMetadata.getPath(), partitionColumn);
             // Table cannot contain nulls
             partitionValue = partitionValue == null ? NULL_VALUE : partitionValue;
             List<FileMetadata> partitionFiles = colValFile.get(partitionColumn, partitionValue);
@@ -300,7 +315,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
 
         for (SchemaPath logicalExpressions : colValFile.rowKeySet()) {
           for (List<FileMetadata> partValues : colValFile.row(logicalExpressions).values()) {
-            partitions.add(ParquetTableMetadataUtils.getPartitionMetadata(logicalExpressions, partValues, tableName));
+            partitions.add(ParquetTableMetadataUtils.getPartitionMetadata(logicalExpressions, partValues));
           }
         }
       } else {
@@ -313,18 +328,30 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
           partitionsForValue.asMap().forEach((partitionKey, value) -> {
             Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
 
-            Map<StatisticsKind, Object> statistics = new HashMap<>();
+            List<StatisticsHolder> statistics = new ArrayList<>();
             partitionKey = partitionKey == NULL_VALUE ? null : partitionKey;
-            statistics.put(ColumnStatisticsKind.MIN_VALUE, partitionKey);
-            statistics.put(ColumnStatisticsKind.MAX_VALUE, partitionKey);
+            statistics.add(new StatisticsHolder<>(partitionKey, ColumnStatisticsKind.MIN_VALUE));
+            statistics.add(new StatisticsHolder<>(partitionKey, ColumnStatisticsKind.MAX_VALUE));
 
-            statistics.put(ColumnStatisticsKind.NULLS_COUNT, Statistic.NO_COLUMN_STATS);
-            statistics.put(TableStatisticsKind.ROW_COUNT, Statistic.NO_COLUMN_STATS);
+            statistics.add(new StatisticsHolder<>(Statistic.NO_COLUMN_STATS, ColumnStatisticsKind.NULLS_COUNT));
+            statistics.add(new StatisticsHolder<>(Statistic.NO_COLUMN_STATS, TableStatisticsKind.ROW_COUNT));
             columnsStatistics.put(partitionColumn,
-                new ColumnStatisticsImpl<>(statistics,
-                        ParquetTableMetadataUtils.getComparator(getParquetGroupScanStatistics().getTypeForColumn(partitionColumn).getMinorType())));
-            partitions.add(new PartitionMetadata(partitionColumn, getTableMetadata().getSchema(),
-                columnsStatistics, statistics, (Set<Path>) value, tableName, -1));
+                new ColumnStatistics<>(statistics,
+                    getParquetGroupScanStatistics().getTypeForColumn(partitionColumn).getMinorType()));
+            MetadataInfo metadataInfo = new MetadataInfo(MetadataType.PARTITION, MetadataInfo.GENERAL_INFO_KEY, null);
+            TableMetadata tableMetadata = getTableMetadata();
+            PartitionMetadata partitionMetadata = PartitionMetadata.builder()
+                .tableInfo(tableMetadata.getTableInfo())
+                .metadataInfo(metadataInfo)
+                .column(partitionColumn)
+                .schema(tableMetadata.getSchema())
+                .columnsStatistics(columnsStatistics)
+                .metadataStatistics(statistics)
+                .partitionValues(Collections.emptyList())
+                .locations(new HashSet<>(value))
+                .build();
+
+            partitions.add(partitionMetadata);
           });
         }
       }
@@ -342,23 +369,113 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
 
   @Override
   public FileMetadata getFileMetadata(Path location) {
-    return getFilesMetadata().stream()
-        .filter(Objects::nonNull)
-        .filter(fileMetadata -> location.equals(fileMetadata.getLocation()))
-        .findAny()
-        .orElse(null);
+    return getFilesMetadataMap().get(location);
   }
 
   @Override
   public List<FileMetadata> getFilesForPartition(PartitionMetadata partition) {
-    return getFilesMetadata().stream()
-        .filter(file -> partition.getLocations().contains(file.getLocation()))
+    return partition.getLocations().stream()
+        .map(location -> getFilesMetadataMap().get(location))
+        .filter(Objects::nonNull)
         .collect(Collectors.toList());
   }
 
   @Override
-  public List<FileMetadata> getFilesMetadata() {
-    return new ArrayList<>(getFilesMetadataMap().values());
+  public Map<Path, SegmentMetadata> getSegmentsMetadataMap() {
+    if (segments == null) {
+      if (entries.isEmpty() || !collectMetadata) {
+        return Collections.emptyMap();
+      }
+
+      segments = new LinkedHashMap<>();
+
+      Path fileLocation = getFilesMetadataMap().values().iterator().next().getPath();
+      int levelsCount = fileLocation.depth() - tableLocation.depth();
+
+      Map<Path, FileMetadata> filesMetadata = getFilesMetadataMap();
+      int segmentsIndex = levelsCount - 1;
+      Map<Path, SegmentMetadata> segmentMetadata = getSegmentsForMetadata(filesMetadata,
+          SchemaPath.getSimplePath(MetadataInfo.DEFAULT_COLUMN_PREFIX + segmentsIndex));
+      segments.putAll(segmentMetadata);
+      for (int i = segmentsIndex - 1; i >= 0; i--) {
+        String segmentColumn = MetadataInfo.DEFAULT_COLUMN_PREFIX + i;
+        segmentMetadata = getMetadataForSegments(segmentMetadata,
+            SchemaPath.getSimplePath(segmentColumn));
+        segments.putAll(segmentMetadata);
+      }
+
+    }
+    return segments;
+  }
+
+  private static <T extends BaseMetadata & LocationProvider> Map<Path, SegmentMetadata> getSegmentsForMetadata(
+      Map<Path, T> metadata, SchemaPath column) {
+    Multimap<Path, T> metadataMultimap = LinkedListMultimap.create();
+    metadata.forEach((key, value) -> metadataMultimap.put(key.getParent(), value));
+
+    Map<Path, SegmentMetadata> result = new HashMap<>();
+    metadataMultimap.asMap().forEach((key, value) -> result.put(key, combineToSegmentMetadata(value, column)));
+
+    return result;
+  }
+
+  private static Map<Path, SegmentMetadata> getMetadataForSegments(Map<Path, SegmentMetadata> metadata, SchemaPath column) {
+    Multimap<Path, SegmentMetadata> metadataMultimap = LinkedListMultimap.create();
+    metadata.forEach((key, value) -> metadataMultimap.put(key.getParent(), value));
+
+    Map<Path, SegmentMetadata> result = new HashMap<>();
+    metadataMultimap.asMap().forEach((key, value) -> result.put(key, combineSegmentMetadata(value, column)));
+
+    return result;
+  }
+
+  private static <T extends BaseMetadata & LocationProvider> SegmentMetadata combineToSegmentMetadata(Collection<T> metadataList, SchemaPath column) {
+    List<Path> metadataLocations = metadataList.stream()
+        .map(metadata -> metadata.getPath()) // used lambda instead of method reference due to JDK-8141508
+        .collect(Collectors.toList());
+    return combineToSegmentMetadata(metadataList, column, metadataLocations);
+  }
+
+  private static SegmentMetadata combineSegmentMetadata(Collection<SegmentMetadata> metadataList, SchemaPath column) {
+    List<Path> metadataLocations = metadataList.stream()
+        .flatMap(metadata -> metadata.getLocations().stream())
+        .collect(Collectors.toList());
+
+    return combineToSegmentMetadata(metadataList, column, metadataLocations);
+  }
+
+  /**
+   * Returns {@link SegmentMetadata} which is combined metadata of list of specified metadata
+   *
+   * @param metadataList      metadata to combine
+   * @param column            segment column
+   * @param metadataLocations locations of metadata combined in resulting segment
+   * @param <T>               type of metadata to combine
+   * @return {@link SegmentMetadata} from combined metadata
+   */
+  private static <T extends BaseMetadata & LocationProvider> SegmentMetadata combineToSegmentMetadata(Collection<T> metadataList,
+      SchemaPath column, List<Path> metadataLocations) {
+    List<StatisticsHolder> segmentStatistics =
+        Collections.singletonList(
+            new StatisticsHolder<>(
+                TableStatisticsKind.ROW_COUNT.mergeStatistics(metadataList),
+                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);
+    T firstMetadata = metadataList.iterator().next();
+
+    return SegmentMetadata.builder()
+        .tableInfo(firstMetadata.getTableInfo())
+        .metadataInfo(metadataInfo)
+        .column(column)
+        .schema(firstMetadata.getSchema())
+        .path(firstMetadata.getPath().getParent())
+        .columnsStatistics(TableMetadataUtils.mergeColumnsStatistics(metadataList, firstMetadata.getColumnsStatistics().keySet(), PARQUET_COLUMN_STATISTICS))
+        .metadataStatistics(segmentStatistics)
+        .partitionValues(Collections.emptyList())
+        .locations(metadataLocations)
+        .build();
   }
 
   @Override
@@ -381,12 +498,12 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
           fileRowGroups.add(rowGroupMetadata);
 
           if (addRowGroups) {
-            rowGroups.put(rowGroupMetadata.getLocation(), rowGroupMetadata);
+            rowGroups.put(rowGroupMetadata.getPath(), rowGroupMetadata);
           }
         }
 
-        FileMetadata fileMetadata = ParquetTableMetadataUtils.getFileMetadata(fileRowGroups, tableName, parquetTableMetadata);
-        files.put(fileMetadata.getLocation(), fileMetadata);
+        FileMetadata fileMetadata = ParquetTableMetadataUtils.getFileMetadata(fileRowGroups);
+        files.put(fileMetadata.getPath(), fileMetadata);
       }
     }
     return files;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FilterEvaluatorUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FilterEvaluatorUtils.java
index ab8f88f..cebb022 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FilterEvaluatorUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FilterEvaluatorUtils.java
@@ -18,12 +18,12 @@
 package org.apache.drill.exec.store.parquet;
 
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.SchemaPathUtils;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.metastore.util.SchemaPathUtils;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.store.parquet.metadata.MetadataBase;
-import org.apache.drill.metastore.NonInterestingColumnsMetadata;
-import org.apache.drill.metastore.RowGroupMetadata;
-import org.apache.drill.metastore.TableStatisticsKind;
+import org.apache.drill.metastore.metadata.NonInterestingColumnsMetadata;
+import org.apache.drill.metastore.metadata.RowGroupMetadata;
 import org.apache.drill.exec.expr.FilterBuilder;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
@@ -38,7 +38,7 @@ import org.apache.drill.exec.expr.stat.RowsMatch;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.UdfUtilities;
 import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.metastore.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
 import org.apache.drill.exec.expr.FilterPredicate;
 import org.apache.drill.exec.expr.StatisticsProvider;
 
@@ -67,14 +67,12 @@ public class FilterEvaluatorUtils {
     Map<SchemaPath, ColumnStatistics> columnsStatistics = rowGroupMetadata.getColumnsStatistics();
 
     // Add column statistics of non-interesting columns if there are any
-    if (nonInterestingColumnsMetadata != null) {
-      columnsStatistics.putAll(nonInterestingColumnsMetadata.getColumnsStatistics());
-    }
+    columnsStatistics.putAll(nonInterestingColumnsMetadata.getColumnsStatistics());
 
     columnsStatistics = ParquetTableMetadataUtils.addImplicitColumnsStatistics(columnsStatistics,
-        schemaPathsInExpr, Collections.emptyList(), options, rowGroupMetadata.getLocation(), true);
+        schemaPathsInExpr, Collections.emptyList(), options, rowGroupMetadata.getPath(), true);
 
-    return matches(expr, columnsStatistics, rowGroupMetadata.getSchema(), (Long) TableStatisticsKind.ROW_COUNT.getValue(rowGroupMetadata),
+    return matches(expr, columnsStatistics, rowGroupMetadata.getSchema(), TableStatisticsKind.ROW_COUNT.getValue(rowGroupMetadata),
         fragmentContext, fragmentContext.getFunctionRegistry());
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFileTableMetadataProviderBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFileTableMetadataProviderBuilder.java
index 9846187..ab809bc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFileTableMetadataProviderBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFileTableMetadataProviderBuilder.java
@@ -17,8 +17,8 @@
  */
 package org.apache.drill.exec.store.parquet;
 
-import org.apache.drill.exec.physical.base.ParquetTableMetadataProvider;
-import org.apache.drill.exec.physical.base.TableMetadataProviderBuilder;
+import org.apache.drill.metastore.ParquetTableMetadataProvider;
+import org.apache.drill.metastore.metadata.TableMetadataProviderBuilder;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.store.dfs.ReadEntryWithPath;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
index 3b0deec..e0726f1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
@@ -35,7 +35,7 @@ import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.AbstractFileGroupScan;
 import org.apache.drill.exec.physical.base.AbstractWriter;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.SchemalessScan;
 import org.apache.drill.exec.physical.impl.WriterRecordBatch;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index c35e21b..41ed1f5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -22,12 +22,12 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.drill.exec.physical.base.FileSystemMetadataProviderManager;
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
-import org.apache.drill.exec.physical.base.ParquetTableMetadataProvider;
+import org.apache.drill.metastore.FileSystemMetadataProviderManager;
+import org.apache.drill.metastore.MetadataProviderManager;
+import org.apache.drill.metastore.ParquetTableMetadataProvider;
 import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
-import org.apache.drill.metastore.LocationProvider;
+import org.apache.drill.metastore.metadata.LocationProvider;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
@@ -291,7 +291,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
 
   @Override
   protected List<String> getPartitionValues(LocationProvider locationProvider) {
-    return ColumnExplorer.listPartitionValues(locationProvider.getLocation(), selectionRoot, false);
+    return ColumnExplorer.listPartitionValues(locationProvider.getPath(), selectionRoot, false);
   }
 
   // overridden protected methods block end
@@ -300,7 +300,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
    * Implementation of RowGroupScanFilterer which uses {@link ParquetGroupScan} as source and
    * builds {@link ParquetGroupScan} instance with filtered metadata.
    */
-  private class ParquetGroupScanFilterer extends RowGroupScanFilterer {
+  private class ParquetGroupScanFilterer extends RowGroupScanFilterer<ParquetGroupScanFilterer> {
 
     public ParquetGroupScanFilterer(ParquetGroupScan source) {
       super(source);
@@ -310,5 +310,10 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
     protected AbstractParquetGroupScan getNewScan() {
       return new ParquetGroupScan((ParquetGroupScan) source);
     }
+
+    @Override
+    protected ParquetGroupScanFilterer self() {
+      return this;
+    }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java
index c53cee9..17a7c55 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java
@@ -20,19 +20,20 @@ package org.apache.drill.exec.store.parquet;
 import org.apache.commons.lang3.mutable.MutableLong;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.SchemaPathUtils;
-import org.apache.drill.metastore.BaseMetadata;
+import org.apache.drill.metastore.util.SchemaPathUtils;
+import org.apache.drill.metastore.metadata.BaseMetadata;
 import org.apache.hadoop.fs.Path;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsKind;
-import org.apache.drill.metastore.LocationProvider;
-import org.apache.drill.metastore.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.metadata.LocationProvider;
 import org.apache.drill.shaded.guava.com.google.common.collect.HashBasedTable;
 import org.apache.drill.shaded.guava.com.google.common.collect.Table;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -54,7 +55,7 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
   private long rowCount;
 
 
-  public ParquetGroupScanStatistics(List<T> rowGroupInfos) {
+  public ParquetGroupScanStatistics(Collection<T> rowGroupInfos) {
     collect(rowGroupInfos);
   }
 
@@ -95,11 +96,11 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
     return partitionValueMap.column(column);
   }
 
-  public void collect(List<T> metadataList) {
+  public void collect(Collection<T> metadataList) {
     resetHolders();
     boolean first = true;
     for (T metadata : metadataList) {
-      long localRowCount = (long) TableStatisticsKind.ROW_COUNT.getValue(metadata);
+      long localRowCount = TableStatisticsKind.ROW_COUNT.getValue(metadata);
       for (Map.Entry<SchemaPath, ColumnStatistics> columnsStatistics : metadata.getColumnsStatistics().entrySet()) {
         SchemaPath schemaPath = columnsStatistics.getKey();
         ColumnStatistics statistics = columnsStatistics.getValue();
@@ -108,7 +109,7 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
         if (previousCount == null) {
           previousCount = emptyCount;
         }
-        Long nullsNum = (Long) statistics.getStatistic(ColumnStatisticsKind.NULLS_COUNT);
+        Long nullsNum = ColumnStatisticsKind.NULLS_COUNT.getFrom(statistics);
         if (previousCount.longValue() != Statistic.NO_COLUMN_STATS && nullsNum != null && nullsNum != Statistic.NO_COLUMN_STATS) {
           previousCount.add(localRowCount - nullsNum);
         } else {
@@ -118,8 +119,8 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
         TypeProtos.MajorType majorType = columnMetadata != null ? columnMetadata.majorType() : null;
         boolean partitionColumn = checkForPartitionColumn(statistics, first, localRowCount, majorType, schemaPath);
         if (partitionColumn) {
-          Object value = partitionValueMap.get(metadata.getLocation(), schemaPath);
-          Object currentValue = statistics.getStatistic(ColumnStatisticsKind.MAX_VALUE);
+          Object value = partitionValueMap.get(metadata.getPath(), schemaPath);
+          Object currentValue = ColumnStatisticsKind.MAX_VALUE.getFrom(statistics);
           if (value != null && value != BaseParquetMetadataProvider.NULL_VALUE) {
             if (value != currentValue) {
               partitionColTypeMap.remove(schemaPath);
@@ -127,10 +128,10 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
           } else {
             // the value of a column with primitive type can not be null,
             // so checks that there are really null value and puts it to the map
-            if (localRowCount == (long) statistics.getStatistic(ColumnStatisticsKind.NULLS_COUNT)) {
-              partitionValueMap.put(metadata.getLocation(), schemaPath, BaseParquetMetadataProvider.NULL_VALUE);
+            if (localRowCount == ColumnStatisticsKind.NULLS_COUNT.getFrom(statistics)) {
+              partitionValueMap.put(metadata.getPath(), schemaPath, BaseParquetMetadataProvider.NULL_VALUE);
             } else {
-              partitionValueMap.put(metadata.getLocation(), schemaPath, currentValue);
+              partitionValueMap.put(metadata.getPath(), schemaPath, currentValue);
             }
           }
         } else {
@@ -206,10 +207,10 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
   }
 
   private boolean isSingleVal(ColumnStatistics columnStatistics, long rowCount) {
-    Long numNulls = (Long) columnStatistics.getStatistic(ColumnStatisticsKind.NULLS_COUNT);
+    Long numNulls = ColumnStatisticsKind.NULLS_COUNT.getFrom(columnStatistics);
     if (numNulls != null && numNulls != Statistic.NO_COLUMN_STATS) {
-      Object min = columnStatistics.getStatistic(ColumnStatisticsKind.MIN_VALUE);
-      Object max = columnStatistics.getStatistic(ColumnStatisticsKind.MAX_VALUE);
+      Object min = columnStatistics.get(ColumnStatisticsKind.MIN_VALUE);
+      Object max = columnStatistics.get(ColumnStatisticsKind.MAX_VALUE);
       if (min != null) {
         return (numNulls == 0 || numNulls == rowCount) && Objects.deepEquals(min, max);
       } else {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
index bafb08b..a453fea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
@@ -594,50 +594,66 @@ public class ParquetReaderUtility {
    * @return major type
    */
   public static TypeProtos.MajorType getType(PrimitiveTypeName type, OriginalType originalType, int scale, int precision) {
+    TypeProtos.MinorType minorType = getMinorType(type, originalType);
+    if (originalType == OriginalType.DECIMAL) {
+      return Types.withScaleAndPrecision(minorType, TypeProtos.DataMode.OPTIONAL, scale, precision);
+    }
+
+    return Types.optional(minorType);
+  }
+
+  /**
+   * Builds minor type using given {@code OriginalType originalType} or {@code PrimitiveTypeName type}.
+   *
+   * @param type         parquet primitive type
+   * @param originalType parquet original type
+   * @return minor type
+   */
+  public static TypeProtos.MinorType getMinorType(PrimitiveTypeName type, OriginalType originalType) {
     if (originalType != null) {
       switch (originalType) {
         case DECIMAL:
-          return Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, scale, precision);
+          return TypeProtos.MinorType.VARDECIMAL;
         case DATE:
-          return Types.optional(TypeProtos.MinorType.DATE);
+          return TypeProtos.MinorType.DATE;
         case TIME_MILLIS:
-          return Types.optional(TypeProtos.MinorType.TIME);
+          return TypeProtos.MinorType.TIME;
         case TIMESTAMP_MILLIS:
-          return Types.optional(TypeProtos.MinorType.TIMESTAMP);
+          return TypeProtos.MinorType.TIMESTAMP;
         case UTF8:
-          return Types.optional(TypeProtos.MinorType.VARCHAR);
+          return TypeProtos.MinorType.VARCHAR;
         case UINT_8:
-          return Types.optional(TypeProtos.MinorType.UINT1);
+          return TypeProtos.MinorType.UINT1;
         case UINT_16:
-          return Types.optional(TypeProtos.MinorType.UINT2);
+          return TypeProtos.MinorType.UINT2;
         case UINT_32:
-          return Types.optional(TypeProtos.MinorType.UINT4);
+          return TypeProtos.MinorType.UINT4;
         case UINT_64:
-          return Types.optional(TypeProtos.MinorType.UINT8);
+          return TypeProtos.MinorType.UINT8;
         case INT_8:
-          return Types.optional(TypeProtos.MinorType.TINYINT);
+          return TypeProtos.MinorType.TINYINT;
         case INT_16:
-          return Types.optional(TypeProtos.MinorType.SMALLINT);
+          return TypeProtos.MinorType.SMALLINT;
         case INTERVAL:
-          return Types.optional(TypeProtos.MinorType.INTERVAL);
+          return TypeProtos.MinorType.INTERVAL;
       }
     }
 
     switch (type) {
       case BOOLEAN:
-        return Types.optional(TypeProtos.MinorType.BIT);
+        return TypeProtos.MinorType.BIT;
       case INT32:
-        return Types.optional(TypeProtos.MinorType.INT);
+        return TypeProtos.MinorType.INT;
       case INT64:
-        return Types.optional(TypeProtos.MinorType.BIGINT);
+        return TypeProtos.MinorType.BIGINT;
       case FLOAT:
-        return Types.optional(TypeProtos.MinorType.FLOAT4);
+        return TypeProtos.MinorType.FLOAT4;
       case DOUBLE:
-        return Types.optional(TypeProtos.MinorType.FLOAT8);
+        return TypeProtos.MinorType.FLOAT8;
       case BINARY:
       case FIXED_LEN_BYTE_ARRAY:
       case INT96:
-        return Types.optional(TypeProtos.MinorType.VARBINARY);
+        return TypeProtos.MinorType.VARBINARY;
       default:
         // Should never hit this
         throw new UnsupportedOperationException("Unsupported type:" + type);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataProviderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataProviderImpl.java
index ec0f7af..7fc3230 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataProviderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataProviderImpl.java
@@ -17,9 +17,9 @@
  */
 package org.apache.drill.exec.store.parquet;
 
-import org.apache.drill.exec.physical.base.MetadataProviderManager;
-import org.apache.drill.exec.physical.base.ParquetMetadataProvider;
-import org.apache.drill.exec.physical.base.ParquetTableMetadataProvider;
+import org.apache.drill.metastore.MetadataProviderManager;
+import org.apache.drill.metastore.ParquetMetadataProvider;
+import org.apache.drill.metastore.ParquetTableMetadataProvider;
 import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.metadata.schema.SchemaProvider;
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 fb0ea88..4f12230 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
@@ -19,7 +19,12 @@ package org.apache.drill.exec.store.parquet;
 
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
+import org.apache.drill.metastore.statistics.BaseStatisticsKind;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.metastore.statistics.Statistic;
 import org.apache.drill.exec.record.metadata.MetadataUtils;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.metadata.TupleSchema;
@@ -29,30 +34,24 @@ import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.store.parquet.metadata.MetadataBase;
 import org.apache.drill.exec.store.parquet.metadata.MetadataVersion;
 import org.apache.drill.exec.store.parquet.metadata.Metadata_V4;
-import org.apache.drill.metastore.BaseMetadata;
-import org.apache.drill.metastore.CollectableColumnStatisticsKind;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsImpl;
-import org.apache.drill.metastore.ColumnStatisticsKind;
-import org.apache.drill.metastore.FileMetadata;
-import org.apache.drill.metastore.NonInterestingColumnsMetadata;
-import org.apache.drill.metastore.PartitionMetadata;
-import org.apache.drill.metastore.RowGroupMetadata;
-import org.apache.drill.metastore.StatisticsKind;
-import org.apache.drill.metastore.TableMetadata;
-import org.apache.drill.metastore.TableStatisticsKind;
-import org.apache.drill.exec.expr.ExactStatisticsConstants;
+import org.apache.drill.metastore.statistics.CollectableColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.metadata.FileMetadata;
+import org.apache.drill.metastore.metadata.NonInterestingColumnsMetadata;
+import org.apache.drill.metastore.metadata.PartitionMetadata;
+import org.apache.drill.metastore.metadata.RowGroupMetadata;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
+import org.apache.drill.metastore.util.TableMetadataUtils;
+import org.apache.drill.metastore.statistics.ExactStatisticsConstants;
 import org.apache.drill.exec.expr.StatisticsProvider;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
 import org.apache.drill.shaded.guava.com.google.common.collect.LinkedListMultimap;
 import org.apache.drill.shaded.guava.com.google.common.collect.Multimap;
 import org.apache.drill.shaded.guava.com.google.common.primitives.Longs;
-import org.apache.drill.shaded.guava.com.google.common.primitives.UnsignedBytes;
 import org.apache.hadoop.fs.Path;
 import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.schema.OriginalType;
-import org.apache.parquet.schema.PrimitiveComparator;
 import org.apache.parquet.schema.PrimitiveType;
 import org.joda.time.DateTimeConstants;
 
@@ -60,15 +59,13 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * Utility class for converting parquet metadata classes to metastore metadata classes.
@@ -76,10 +73,7 @@ import java.util.concurrent.ConcurrentHashMap;
 @SuppressWarnings("WeakerAccess")
 public class ParquetTableMetadataUtils {
 
-  private static final Comparator<byte[]> UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR = Comparator.nullsFirst((b1, b2) ->
-      PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR.compare(Binary.fromReusedByteArray(b1), Binary.fromReusedByteArray(b2)));
-
-  static final List<CollectableColumnStatisticsKind> PARQUET_STATISTICS =
+  static final List<CollectableColumnStatisticsKind> PARQUET_COLUMN_STATISTICS =
           ImmutableList.of(
               ColumnStatisticsKind.MAX_VALUE,
               ColumnStatisticsKind.MIN_VALUE,
@@ -111,8 +105,7 @@ public class ParquetTableMetadataUtils {
     columnsStatistics = new HashMap<>(columnsStatistics);
     for (Map.Entry<String, String> partitionValue : implicitColValues.entrySet()) {
       columnsStatistics.put(SchemaPath.getCompoundPath(partitionValue.getKey()),
-          new StatisticsProvider.MinMaxStatistics<>(partitionValue.getValue(),
-              partitionValue.getValue(), Comparator.nullsFirst(Comparator.naturalOrder())));
+          StatisticsProvider.getConstantColumnStatistics(partitionValue.getValue(), TypeProtos.MinorType.VARCHAR));
     }
     return columnsStatistics;
   }
@@ -148,78 +141,58 @@ public class ParquetTableMetadataUtils {
   public static RowGroupMetadata getRowGroupMetadata(MetadataBase.ParquetTableMetadataBase tableMetadata,
       MetadataBase.RowGroupMetadata rowGroupMetadata, int rgIndexInFile, Path location) {
     Map<SchemaPath, ColumnStatistics> columnsStatistics = getRowGroupColumnStatistics(tableMetadata, rowGroupMetadata);
-    Map<StatisticsKind, Object> rowGroupStatistics = new HashMap<>();
-    rowGroupStatistics.put(TableStatisticsKind.ROW_COUNT, rowGroupMetadata.getRowCount());
-    rowGroupStatistics.put(() -> ExactStatisticsConstants.START, rowGroupMetadata.getStart());
-    rowGroupStatistics.put(() -> ExactStatisticsConstants.LENGTH, rowGroupMetadata.getLength());
+    List<StatisticsHolder> rowGroupStatistics = new ArrayList<>();
+    rowGroupStatistics.add(new StatisticsHolder<>(rowGroupMetadata.getRowCount(), TableStatisticsKind.ROW_COUNT));
+    rowGroupStatistics.add(new StatisticsHolder<>(rowGroupMetadata.getStart(), new BaseStatisticsKind(ExactStatisticsConstants.START, true)));
+    rowGroupStatistics.add(new StatisticsHolder<>(rowGroupMetadata.getLength(), new BaseStatisticsKind(ExactStatisticsConstants.LENGTH, true)));
 
     Map<SchemaPath, TypeProtos.MajorType> columns = getRowGroupFields(tableMetadata, rowGroupMetadata);
 
     TupleSchema schema = new TupleSchema();
     columns.forEach((schemaPath, majorType) -> MetadataUtils.addColumnMetadata(schema, schemaPath, majorType));
 
-    return new RowGroupMetadata(
-        schema, columnsStatistics, rowGroupStatistics, rowGroupMetadata.getHostAffinity(), rgIndexInFile, location);
-  }
-
-  /**
-   * Merges list of specified metadata into the map of {@link ColumnStatistics} with columns as keys.
-   *
-   * @param <T>                 type of metadata to collect
-   * @param metadataList        list of metadata to be merged
-   * @param columns             set of columns whose statistics should be merged
-   * @param statisticsToCollect kinds of statistics that should be collected
-   * @param parquetTableMetadata ParquetTableMetadata object to fetch the non-interesting columns
-   * @return list of merged metadata
-   */
-  @SuppressWarnings("unchecked")
-  public static <T extends BaseMetadata> Map<SchemaPath, ColumnStatistics> mergeColumnsStatistics(
-          Collection<T> metadataList, Set<SchemaPath> columns, List<CollectableColumnStatisticsKind> statisticsToCollect, MetadataBase.ParquetTableMetadataBase parquetTableMetadata) {
-    Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
-
-    for (SchemaPath column : columns) {
-      List<ColumnStatistics> statisticsList = new ArrayList<>();
-      for (T metadata : metadataList) {
-        ColumnStatistics statistics = metadata.getColumnsStatistics().get(column);
-        if (statistics == null) {
-          // schema change happened, set statistics which represents all nulls
-          statistics = new ColumnStatisticsImpl(
-              ImmutableMap.of(ColumnStatisticsKind.NULLS_COUNT, metadata.getStatistic(TableStatisticsKind.ROW_COUNT)),
-              getNaturalNullsFirstComparator());
-        }
-        statisticsList.add(statistics);
-      }
-      Map<StatisticsKind, Object> statisticsMap = new HashMap<>();
-      for (CollectableColumnStatisticsKind statisticsKind : statisticsToCollect) {
-        Object mergedStatistic = statisticsKind.mergeStatistics(statisticsList);
-        statisticsMap.put(statisticsKind, mergedStatistic);
-      }
-      columnsStatistics.put(column, new ColumnStatisticsImpl(statisticsMap, statisticsList.iterator().next().getValueComparator()));
-    }
-    return columnsStatistics;
+    MetadataInfo metadataInfo = new MetadataInfo(MetadataType.ROW_GROUP, MetadataInfo.GENERAL_INFO_KEY, null);
+
+    return RowGroupMetadata.builder()
+        .tableInfo(TableInfo.UNKNOWN_TABLE_INFO)
+        .metadataInfo(metadataInfo)
+        .schema(schema)
+        .columnsStatistics(columnsStatistics)
+        .metadataStatistics(rowGroupStatistics)
+        .hostAffinity(rowGroupMetadata.getHostAffinity())
+        .rowGroupIndex(rgIndexInFile)
+        .path(location)
+        .build();
   }
 
   /**
    * Returns {@link FileMetadata} instance received by merging specified {@link RowGroupMetadata} list.
    *
    * @param rowGroups list of {@link RowGroupMetadata} to be merged
-   * @param tableName name of the table
-   * @param parquetTableMetadata the source of column metadata for non-interesting column's statistics
    * @return {@link FileMetadata} instance
    */
-  public static FileMetadata getFileMetadata(List<RowGroupMetadata> rowGroups, String tableName,
-      MetadataBase.ParquetTableMetadataBase parquetTableMetadata) {
+  public static FileMetadata getFileMetadata(List<RowGroupMetadata> rowGroups) {
     if (rowGroups.isEmpty()) {
       return null;
     }
-    Map<StatisticsKind, Object> fileStatistics = new HashMap<>();
-    fileStatistics.put(TableStatisticsKind.ROW_COUNT, TableStatisticsKind.ROW_COUNT.mergeStatistics(rowGroups));
+    List<StatisticsHolder> fileStatistics = new ArrayList<>();
+    fileStatistics.add(new StatisticsHolder<>(TableStatisticsKind.ROW_COUNT.mergeStatistics(rowGroups), TableStatisticsKind.ROW_COUNT));
+
+    RowGroupMetadata rowGroupMetadata = rowGroups.iterator().next();
+    TupleMetadata schema = rowGroupMetadata.getSchema();
+
+    Set<SchemaPath> columns = rowGroupMetadata.getColumnsStatistics().keySet();
 
-    TupleMetadata schema = rowGroups.iterator().next().getSchema();
+    MetadataInfo metadataInfo = new MetadataInfo(MetadataType.FILE, MetadataInfo.GENERAL_INFO_KEY, null);
 
-    return new FileMetadata(rowGroups.iterator().next().getLocation(), schema,
-      mergeColumnsStatistics(rowGroups, rowGroups.iterator().next().getColumnsStatistics().keySet(), PARQUET_STATISTICS, parquetTableMetadata),
-      fileStatistics, tableName, -1);
+    return FileMetadata.builder()
+        .tableInfo(rowGroupMetadata.getTableInfo())
+        .metadataInfo(metadataInfo)
+        .path(rowGroupMetadata.getPath())
+        .schema(schema)
+        .columnsStatistics(TableMetadataUtils.mergeColumnsStatistics(rowGroups, columns, PARQUET_COLUMN_STATISTICS))
+        .metadataStatistics(fileStatistics)
+        .build();
   }
 
   /**
@@ -227,33 +200,31 @@ public class ParquetTableMetadataUtils {
    *
    * @param partitionColumn partition column
    * @param files           list of files to be merged
-   * @param tableName       name of the table
    * @return {@link PartitionMetadata} instance
    */
-  public static PartitionMetadata getPartitionMetadata(SchemaPath partitionColumn, List<FileMetadata> files, String tableName) {
+  public static PartitionMetadata getPartitionMetadata(SchemaPath partitionColumn, List<FileMetadata> files) {
     Set<Path> locations = new HashSet<>();
     Set<SchemaPath> columns = new HashSet<>();
 
     for (FileMetadata file : files) {
       columns.addAll(file.getColumnsStatistics().keySet());
-      locations.add(file.getLocation());
+      locations.add(file.getPath());
     }
 
-    Map<StatisticsKind, Object> partStatistics = new HashMap<>();
-    partStatistics.put(TableStatisticsKind.ROW_COUNT, TableStatisticsKind.ROW_COUNT.mergeStatistics(files));
+    FileMetadata fileMetadata = files.iterator().next();
 
-    return new PartitionMetadata(partitionColumn, files.iterator().next().getSchema(),
-        mergeColumnsStatistics(files, columns, PARQUET_STATISTICS, null), partStatistics, locations, tableName, -1);
-  }
+    MetadataInfo metadataInfo = new MetadataInfo(MetadataType.PARTITION, MetadataInfo.GENERAL_INFO_KEY, null);
 
-  /**
-   * Returns "natural order" comparator which threads nulls as min values.
-   *
-   * @param <T> type to compare
-   * @return "natural order" comparator
-   */
-  public static <T extends Comparable<T>> Comparator<T> getNaturalNullsFirstComparator() {
-    return Comparator.nullsFirst(Comparator.naturalOrder());
+    return PartitionMetadata.builder()
+        .tableInfo(fileMetadata.getTableInfo())
+        .metadataInfo(metadataInfo)
+        .column(partitionColumn)
+        .schema(fileMetadata.getSchema())
+        .columnsStatistics(TableMetadataUtils.mergeColumnsStatistics(files, columns, PARQUET_COLUMN_STATISTICS))
+        .metadataStatistics(Collections.singletonList(new StatisticsHolder<>(TableStatisticsKind.ROW_COUNT.mergeStatistics(files), TableStatisticsKind.ROW_COUNT)))
+        .partitionValues(Collections.emptyList())
+        .locations(locations)
+        .build();
   }
 
   /**
@@ -264,7 +235,6 @@ public class ParquetTableMetadataUtils {
    * @param rowGroupMetadata metadata to convert
    * @return map with converted row group metadata
    */
-  @SuppressWarnings("unchecked")
   public static Map<SchemaPath, ColumnStatistics> getRowGroupColumnStatistics(
       MetadataBase.ParquetTableMetadataBase tableMetadata, MetadataBase.RowGroupMetadata rowGroupMetadata) {
 
@@ -279,13 +249,13 @@ public class ParquetTableMetadataUtils {
       }
       PrimitiveType.PrimitiveTypeName primitiveType = getPrimitiveTypeName(tableMetadata, column);
       OriginalType originalType = getOriginalType(tableMetadata, column);
-      Comparator comparator = getComparator(primitiveType, originalType);
+      TypeProtos.MinorType type = ParquetReaderUtility.getMinorType(primitiveType, originalType);
 
-      Map<StatisticsKind, Object> statistics = new HashMap<>();
-      statistics.put(ColumnStatisticsKind.MIN_VALUE, getValue(column.getMinValue(), primitiveType, originalType));
-      statistics.put(ColumnStatisticsKind.MAX_VALUE, getValue(column.getMaxValue(), primitiveType, originalType));
-      statistics.put(ColumnStatisticsKind.NULLS_COUNT, nulls);
-      columnsStatistics.put(colPath, new ColumnStatisticsImpl(statistics, comparator));
+      List<StatisticsHolder> statistics = new ArrayList<>();
+      statistics.add(new StatisticsHolder<>(getValue(column.getMinValue(), primitiveType, originalType), ColumnStatisticsKind.MIN_VALUE));
+      statistics.add(new StatisticsHolder<>(getValue(column.getMaxValue(), primitiveType, originalType), ColumnStatisticsKind.MAX_VALUE));
+      statistics.add(new StatisticsHolder<>(nulls, ColumnStatisticsKind.NULLS_COUNT));
+      columnsStatistics.put(colPath, new ColumnStatistics<>(statistics, type));
     }
     return columnsStatistics;
   }
@@ -298,7 +268,7 @@ public class ParquetTableMetadataUtils {
   public static NonInterestingColumnsMetadata getNonInterestingColumnsMeta(MetadataBase.ParquetTableMetadataBase parquetTableMetadata) {
     Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
     if (parquetTableMetadata instanceof Metadata_V4.ParquetTableMetadata_v4) {
-      ConcurrentHashMap<Metadata_V4.ColumnTypeMetadata_v4.Key, Metadata_V4.ColumnTypeMetadata_v4> columnTypeInfoMap =
+      Map<Metadata_V4.ColumnTypeMetadata_v4.Key, Metadata_V4.ColumnTypeMetadata_v4> columnTypeInfoMap =
               ((Metadata_V4.ParquetTableMetadata_v4) parquetTableMetadata).getColumnTypeInfoMap();
 
       if (columnTypeInfoMap == null) {
@@ -308,12 +278,12 @@ public class ParquetTableMetadataUtils {
       for (Metadata_V4.ColumnTypeMetadata_v4 columnTypeMetadata : columnTypeInfoMap.values()) {
         if (!columnTypeMetadata.isInteresting) {
           SchemaPath schemaPath = SchemaPath.getCompoundPath(columnTypeMetadata.name);
-          Map<StatisticsKind, Object> statistics = new HashMap<>();
-          statistics.put(ColumnStatisticsKind.NULLS_COUNT, Statistic.NO_COLUMN_STATS);
+          List<StatisticsHolder> statistics = new ArrayList<>();
+          statistics.add(new StatisticsHolder<>(Statistic.NO_COLUMN_STATS, ColumnStatisticsKind.NULLS_COUNT));
           PrimitiveType.PrimitiveTypeName primitiveType = columnTypeMetadata.primitiveType;
           OriginalType originalType = columnTypeMetadata.originalType;
-          Comparator comparator = getComparator(primitiveType, originalType);
-          columnsStatistics.put(schemaPath, new ColumnStatisticsImpl<>(statistics, comparator));
+          TypeProtos.MinorType type = ParquetReaderUtility.getMinorType(primitiveType, originalType);
+          columnsStatistics.put(schemaPath, new ColumnStatistics<>(statistics, type));
         }
       }
       return new NonInterestingColumnsMetadata(columnsStatistics);
@@ -482,80 +452,6 @@ public class ParquetTableMetadataUtils {
   }
 
   /**
-   * Returns {@link Comparator} instance considering specified {@code primitiveType} and {@code originalType}.
-   *
-   * @param primitiveType primitive type of the column
-   * @param originalType  original type og the column
-   * @return {@link Comparator} instance
-   */
-  public static Comparator getComparator(PrimitiveType.PrimitiveTypeName primitiveType, OriginalType originalType) {
-    if (originalType != null) {
-      switch (originalType) {
-        case UINT_8:
-        case UINT_16:
-        case UINT_32:
-          return getNaturalNullsFirstComparator();
-        case UINT_64:
-          return getNaturalNullsFirstComparator();
-        case DATE:
-        case INT_8:
-        case INT_16:
-        case INT_32:
-        case INT_64:
-        case TIME_MICROS:
-        case TIME_MILLIS:
-        case TIMESTAMP_MICROS:
-        case TIMESTAMP_MILLIS:
-        case DECIMAL:
-        case UTF8:
-          return getNaturalNullsFirstComparator();
-        case INTERVAL:
-          return UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR;
-        default:
-          return getNaturalNullsFirstComparator();
-      }
-    } else {
-      switch (primitiveType) {
-        case INT32:
-        case INT64:
-        case FLOAT:
-        case DOUBLE:
-        case BOOLEAN:
-        case BINARY:
-        case INT96:
-        case FIXED_LEN_BYTE_ARRAY:
-          return getNaturalNullsFirstComparator();
-        default:
-          throw new UnsupportedOperationException("Unsupported type: " + primitiveType);
-      }
-    }
-  }
-
-  /**
-   * Returns {@link Comparator} instance considering specified {@code type}.
-   *
-   * @param type type of the column
-   * @return {@link Comparator} instance
-   */
-  public static Comparator getComparator(TypeProtos.MinorType type) {
-    switch (type) {
-      case INTERVALDAY:
-      case INTERVAL:
-      case INTERVALYEAR:
-        return UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR;
-      case UINT1:
-        return Comparator.nullsFirst(UnsignedBytes::compare);
-      case UINT2:
-      case UINT4:
-        return Comparator.nullsFirst(Integer::compareUnsigned);
-      case UINT8:
-        return Comparator.nullsFirst(Long::compareUnsigned);
-      default:
-        return getNaturalNullsFirstComparator();
-    }
-  }
-
-  /**
    * Returns map of column names with their drill types for specified {@code file}.
    *
    * @param parquetTableMetadata the source of primitive and original column types
@@ -681,22 +577,4 @@ public class ParquetTableMetadataUtils {
     return columns;
   }
 
-  /**
-   * Updates row cont and column nulls count for specified table metadata and returns new {@link TableMetadata} instance with updated statistics.
-   *
-   * @param tableMetadata table statistics to update
-   * @param statistics    list of statistics whose row count should be considered
-   * @return new {@link TableMetadata} instance with updated statistics
-   */
-  public static TableMetadata updateRowCount(TableMetadata tableMetadata, Collection<? extends BaseMetadata> statistics) {
-    Map<StatisticsKind, Object> newStats = new HashMap<>();
-
-    newStats.put(TableStatisticsKind.ROW_COUNT, TableStatisticsKind.ROW_COUNT.mergeStatistics(statistics));
-
-    Map<SchemaPath, ColumnStatistics> columnsStatistics =
-        mergeColumnsStatistics(statistics, tableMetadata.getColumnsStatistics().keySet(),
-            ImmutableList.of(ColumnStatisticsKind.NULLS_COUNT), null);
-
-    return tableMetadata.cloneWithStats(columnsStatistics, newStats);
-  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileSystemMetadataProviderManager.java b/exec/java-exec/src/main/java/org/apache/drill/metastore/FileSystemMetadataProviderManager.java
similarity index 93%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileSystemMetadataProviderManager.java
rename to exec/java-exec/src/main/java/org/apache/drill/metastore/FileSystemMetadataProviderManager.java
index cfc6ef3..cbcf7fb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FileSystemMetadataProviderManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/metastore/FileSystemMetadataProviderManager.java
@@ -15,11 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.base;
+package org.apache.drill.metastore;
 
 import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.record.metadata.schema.SchemaProvider;
 import org.apache.drill.exec.store.parquet.ParquetTableMetadataProviderImpl;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
+import org.apache.drill.metastore.metadata.TableMetadataProviderBuilder;
 
 /**
  * Implementation of {@link MetadataProviderManager} which uses file system providers and returns
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/MetadataProviderManager.java b/exec/java-exec/src/main/java/org/apache/drill/metastore/MetadataProviderManager.java
similarity index 92%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/MetadataProviderManager.java
rename to exec/java-exec/src/main/java/org/apache/drill/metastore/MetadataProviderManager.java
index 8faab9f..af81683 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/MetadataProviderManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/metastore/MetadataProviderManager.java
@@ -15,10 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.base;
+package org.apache.drill.metastore;
 
 import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.record.metadata.schema.SchemaProvider;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
+import org.apache.drill.metastore.metadata.TableMetadataProviderBuilder;
 
 /**
  * Base interface for passing and obtaining {@link SchemaProvider}, {@link DrillStatsTable} and
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ParquetMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/metastore/ParquetMetadataProvider.java
similarity index 84%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ParquetMetadataProvider.java
rename to exec/java-exec/src/main/java/org/apache/drill/metastore/ParquetMetadataProvider.java
index 431ac01..8830a4c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ParquetMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/metastore/ParquetMetadataProvider.java
@@ -15,16 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.base;
+package org.apache.drill.metastore;
 
 import org.apache.drill.exec.store.dfs.ReadEntryWithPath;
-import org.apache.drill.metastore.FileMetadata;
-import org.apache.drill.metastore.RowGroupMetadata;
+import org.apache.drill.metastore.metadata.RowGroupMetadata;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
 import org.apache.drill.shaded.guava.com.google.common.collect.Multimap;
 import org.apache.hadoop.fs.Path;
 
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 /**
@@ -61,13 +60,6 @@ public interface ParquetMetadataProvider extends TableMetadataProvider {
   Multimap<Path, RowGroupMetadata> getRowGroupsMetadataMap();
 
   /**
-   * Returns list of {@link RowGroupMetadata} instances which provides metadata for specific row group and its columns.
-   *
-   * @return list of {@link RowGroupMetadata} instances
-   */
-  Map<Path, FileMetadata> getFilesMetadataMap();
-
-  /**
    * Returns list of file locations for table.
    *
    * @return list of file locations for table
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ParquetTableMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/metastore/ParquetTableMetadataProvider.java
similarity index 96%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ParquetTableMetadataProvider.java
rename to exec/java-exec/src/main/java/org/apache/drill/metastore/ParquetTableMetadataProvider.java
index 53374ff..fe9450f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ParquetTableMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/metastore/ParquetTableMetadataProvider.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.base;
+package org.apache.drill.metastore;
 
 import org.apache.hadoop.fs.Path;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SimpleFileTableMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/metastore/SimpleFileTableMetadataProvider.java
similarity index 76%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SimpleFileTableMetadataProvider.java
rename to exec/java-exec/src/main/java/org/apache/drill/metastore/SimpleFileTableMetadataProvider.java
index 15ecdd9..fc5eb83 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SimpleFileTableMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/metastore/SimpleFileTableMetadataProvider.java
@@ -15,27 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.base;
+package org.apache.drill.metastore;
 
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.metadata.schema.SchemaProvider;
 import org.apache.drill.exec.store.dfs.easy.SimpleFileTableMetadataProviderBuilder;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsImpl;
-import org.apache.drill.metastore.FileMetadata;
-import org.apache.drill.metastore.FileTableMetadata;
-import org.apache.drill.metastore.NonInterestingColumnsMetadata;
-import org.apache.drill.metastore.PartitionMetadata;
-import org.apache.drill.metastore.TableMetadata;
+import org.apache.drill.metastore.metadata.SegmentMetadata;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.metadata.FileMetadata;
+import org.apache.drill.metastore.metadata.BaseTableMetadata;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.NonInterestingColumnsMetadata;
+import org.apache.drill.metastore.metadata.PartitionMetadata;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.apache.drill.metastore.metadata.TableMetadata;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -73,7 +76,12 @@ public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
   }
 
   @Override
-  public List<FileMetadata> getFilesMetadata() {
+  public Map<Path, FileMetadata> getFilesMetadataMap() {
+    return null;
+  }
+
+  @Override
+  public Map<Path, SegmentMetadata> getSegmentsMetadataMap() {
     return null;
   }
 
@@ -88,7 +96,7 @@ public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
   }
 
   @Override
-  public NonInterestingColumnsMetadata getNonInterestingColumnsMeta() {
+  public NonInterestingColumnsMetadata getNonInterestingColumnsMetadata() {
     return null;
   }
 
@@ -129,7 +137,6 @@ public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
     }
 
     @Override
-    @SuppressWarnings("unchecked")
     public TableMetadataProvider build() {
       SchemaProvider schemaProvider = metadataProviderManager.getSchemaProvider();
       TableMetadataProvider source = metadataProviderManager.getTableMetadataProvider();
@@ -144,9 +151,7 @@ public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
           if (statsProvider.isMaterialized()) {
             for (SchemaPath column : statsProvider.getColumns()) {
               columnsStatistics.put(column,
-                  new ColumnStatisticsImpl(
-                      DrillStatsTable.getEstimatedColumnStats(statsProvider, column),
-                      Comparator.nullsFirst(Comparator.naturalOrder())));
+                  new ColumnStatistics(DrillStatsTable.getEstimatedColumnStats(statsProvider, column)));
             }
           }
         }
@@ -162,10 +167,17 @@ 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);
         }
-        TableMetadata tableMetadata = new FileTableMetadata(tableName,
-            location, schema,
-            columnsStatistics, DrillStatsTable.getEstimatedTableStats(statsProvider),
-            lastModifiedTime, "", Collections.emptySet());
+        MetadataInfo metadataInfo = new MetadataInfo(MetadataType.TABLE, MetadataInfo.GENERAL_INFO_KEY, null);
+        TableMetadata tableMetadata = BaseTableMetadata.builder()
+            .tableInfo(TableInfo.UNKNOWN_TABLE_INFO)
+            .metadataInfo(metadataInfo)
+            .location(location)
+            .schema(schema)
+            .columnsStatistics(columnsStatistics)
+            .metadataStatistics(DrillStatsTable.getEstimatedTableStats(statsProvider))
+            .lastModifiedTime(lastModifiedTime)
+            .partitionKeys(Collections.emptyMap())
+            .build();
 
         source = new SimpleFileTableMetadataProvider(tableMetadata);
         metadataProviderManager.setTableMetadataProvider(source);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
index 112cfec..a782ed4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
@@ -30,8 +30,8 @@ import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader;
 import org.apache.drill.exec.store.parquet.metadata.Metadata;
 import org.apache.drill.exec.store.parquet.metadata.MetadataBase;
-import org.apache.drill.metastore.ColumnStatistics;
-import org.apache.drill.metastore.ColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
 import org.apache.drill.exec.expr.IsPredicate;
 import org.apache.drill.exec.expr.StatisticsProvider;
 import org.apache.drill.test.BaseDirTestWatcher;
@@ -602,12 +602,10 @@ public class TestParquetFilterPushDown extends PlanTestBase {
     Mockito.doReturn(booleanStatistics).when(le).accept(ArgumentMatchers.any(), ArgumentMatchers.any());
     StatisticsProvider<Boolean> re = Mockito.mock(StatisticsProvider.class);
     Mockito.when(re.getRowCount()).thenReturn(2L); // 2 rows
-    Mockito.when(booleanStatistics.containsStatistic(ArgumentMatchers.any())).thenReturn(true); // stat is not empty
-    Mockito.when(booleanStatistics.getStatistic(ColumnStatisticsKind.NULLS_COUNT)).thenReturn(0L); // no nulls
-    Mockito.when(booleanStatistics.getStatistic(ColumnStatisticsKind.MIN_VALUE)).thenReturn(false); // min false
-    Mockito.when(booleanStatistics.getValueStatistic(ColumnStatisticsKind.MIN_VALUE)).thenReturn(false); // min false
-    Mockito.when(booleanStatistics.getStatistic(ColumnStatisticsKind.MAX_VALUE)).thenReturn(true); // max true
-    Mockito.when(booleanStatistics.getValueStatistic(ColumnStatisticsKind.MAX_VALUE)).thenReturn(true); // max false
+    Mockito.when(booleanStatistics.contains(ArgumentMatchers.any())).thenReturn(true); // stat is not empty
+    Mockito.when(booleanStatistics.get(ColumnStatisticsKind.NULLS_COUNT)).thenReturn(0L); // no nulls
+    Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MIN_VALUE)).thenReturn(false); // min false
+    Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MAX_VALUE)).thenReturn(true); // max true
     Mockito.when(booleanStatistics.getValueComparator()).thenReturn(Comparator.nullsFirst(Comparator.naturalOrder())); // comparator
     IsPredicate isTrue = (IsPredicate) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, le);
     assertEquals(RowsMatch.SOME, isTrue.matches(re));
@@ -627,12 +625,10 @@ public class TestParquetFilterPushDown extends PlanTestBase {
     Mockito.doReturn(booleanStatistics).when(le).accept(ArgumentMatchers.any(), ArgumentMatchers.any());
     StatisticsProvider<Boolean> re = Mockito.mock(StatisticsProvider.class);
     Mockito.when(re.getRowCount()).thenReturn(2L); // 2 rows
-    Mockito.when(booleanStatistics.containsStatistic(ArgumentMatchers.any())).thenReturn(true); // stat is not empty
-    Mockito.when(booleanStatistics.getStatistic(ColumnStatisticsKind.NULLS_COUNT)).thenReturn(0L); // no nulls
-    Mockito.when(booleanStatistics.getStatistic(ColumnStatisticsKind.MIN_VALUE)).thenReturn(false); // min false
-    Mockito.when(booleanStatistics.getValueStatistic(ColumnStatisticsKind.MIN_VALUE)).thenReturn(false); // min false
-    Mockito.when(booleanStatistics.getStatistic(ColumnStatisticsKind.MAX_VALUE)).thenReturn(false); // max false
-    Mockito.when(booleanStatistics.getValueStatistic(ColumnStatisticsKind.MAX_VALUE)).thenReturn(false); // max false
+    Mockito.when(booleanStatistics.contains(ArgumentMatchers.any())).thenReturn(true); // stat is not empty
+    Mockito.when(booleanStatistics.get(ColumnStatisticsKind.NULLS_COUNT)).thenReturn(0L); // no nulls
+    Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MIN_VALUE)).thenReturn(false); // min false
+    Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MAX_VALUE)).thenReturn(false); // max false
     Mockito.when(booleanStatistics.getValueComparator()).thenReturn(Comparator.nullsFirst(Comparator.naturalOrder())); // comparator
     IsPredicate isTrue = (IsPredicate) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, le);
     assertEquals(RowsMatch.NONE, isTrue.matches(re));
@@ -652,12 +648,10 @@ public class TestParquetFilterPushDown extends PlanTestBase {
     Mockito.doReturn(booleanStatistics).when(le).accept(ArgumentMatchers.any(), ArgumentMatchers.any());
     StatisticsProvider<Boolean> re = Mockito.mock(StatisticsProvider.class);
     Mockito.when(re.getRowCount()).thenReturn(Long.valueOf(2)); // 2 rows
-    Mockito.when(booleanStatistics.containsStatistic(ArgumentMatchers.any())).thenReturn(true); // stat is not empty
-    Mockito.when(booleanStatistics.getStatistic(ColumnStatisticsKind.NULLS_COUNT)).thenReturn(0L); // no nulls
-    Mockito.when(booleanStatistics.getStatistic(ColumnStatisticsKind.MIN_VALUE)).thenReturn(true); // min false
-    Mockito.when(booleanStatistics.getValueStatistic(ColumnStatisticsKind.MIN_VALUE)).thenReturn(true); // min false
-    Mockito.when(booleanStatistics.getStatistic(ColumnStatisticsKind.MAX_VALUE)).thenReturn(true); // max false
-    Mockito.when(booleanStatistics.getValueStatistic(ColumnStatisticsKind.MAX_VALUE)).thenReturn(true); // max false
+    Mockito.when(booleanStatistics.contains(ArgumentMatchers.any())).thenReturn(true); // stat is not empty
+    Mockito.when(booleanStatistics.get(ColumnStatisticsKind.NULLS_COUNT)).thenReturn(0L); // no nulls
+    Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MIN_VALUE)).thenReturn(true); // min false
+    Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MAX_VALUE)).thenReturn(true); // max false
     IsPredicate isTrue = (IsPredicate) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, le);
     assertEquals(RowsMatch.ALL, isTrue.matches(re));
     IsPredicate isFalse = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_FALSE, le);
diff --git a/metastore/file-metadata/pom.xml b/metastore/file-metadata/pom.xml
deleted file mode 100644
index d0bd794..0000000
--- a/metastore/file-metadata/pom.xml
+++ /dev/null
@@ -1,41 +0,0 @@
-<?xml version="1.0"?>
-<!--
-
-    Licensed to the Apache Software Foundation (ASF) under one
-    or more contributor license agreements.  See the NOTICE file
-    distributed with this work for additional information
-    regarding copyright ownership.  The ASF licenses this file
-    to you under the Apache License, Version 2.0 (the
-    "License"); you may not use this file except in compliance
-    with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-    Unless required by applicable law or agreed to in writing, software
-    distributed under the License is distributed on an "AS IS" BASIS,
-    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-    See the License for the specific language governing permissions and
-    limitations under the License.
-
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.drill.metastore</groupId>
-    <artifactId>metastore-parent</artifactId>
-    <version>1.17.0-SNAPSHOT</version>
-  </parent>
-
-  <artifactId>drill-file-metastore-plugin</artifactId>
-  <name>Drill File Metadata Metastore Plugin</name>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.drill.metastore</groupId>
-      <artifactId>drill-metastore-api</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-  </dependencies>
-
-</project>
diff --git a/metastore/file-metadata/src/main/java/org/apache/drill/metastore/FileMetadata.java b/metastore/file-metadata/src/main/java/org/apache/drill/metastore/FileMetadata.java
deleted file mode 100644
index 2e75886..0000000
--- a/metastore/file-metadata/src/main/java/org/apache/drill/metastore/FileMetadata.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.metastore;
-
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.SchemaPathUtils;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.hadoop.fs.Path;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Metadata which corresponds to the file level of table.
- */
-public class FileMetadata implements BaseMetadata, LocationProvider {
-
-  private final Path location;
-  private final TupleMetadata schema;
-  private final Map<SchemaPath, ColumnStatistics> columnsStatistics;
-  private final Map<String, Object> fileStatistics;
-  private final Map<String, StatisticsKind> statisticsKinds;
-  private final String tableName;
-  private final long lastModifiedTime;
-
-  public FileMetadata(Path location, TupleMetadata schema, Map<SchemaPath, ColumnStatistics> columnsStatistics,
-                      Map<StatisticsKind, Object> fileStatistics, String tableName, long lastModifiedTime) {
-    this.schema = schema;
-    this.columnsStatistics = columnsStatistics;
-    this.fileStatistics = new HashMap<>();
-    this.statisticsKinds = new HashMap<>();
-    fileStatistics.forEach((statisticsKind, value) -> {
-      this.fileStatistics.put(statisticsKind.getName(), value);
-      this.statisticsKinds.put(statisticsKind.getName(), statisticsKind);
-    });
-    this.location = location;
-    this.tableName = tableName;
-    this.lastModifiedTime = lastModifiedTime;
-  }
-
-  @Override
-  public Object getStatisticsForColumn(SchemaPath columnName, StatisticsKind statisticsKind) {
-    return columnsStatistics.get(columnName).getStatistic(statisticsKind);
-  }
-
-  @Override
-  public Object getStatistic(StatisticsKind statisticsKind) {
-    return fileStatistics.get(statisticsKind.getName());
-  }
-
-  @Override
-  public boolean containsExactStatistics(StatisticsKind statisticsKind) {
-    return statisticsKinds.get(statisticsKind.getName()).isExact();
-  }
-
-  @Override
-  public ColumnStatistics getColumnStatistics(SchemaPath columnName) {
-    return columnsStatistics.get(columnName);
-  }
-
-  @Override
-  public Path getLocation() {
-    return location;
-  }
-
-  @Override
-  public ColumnMetadata getColumn(SchemaPath name) {
-    return SchemaPathUtils.getColumnMetadata(name, schema);
-  }
-
-  @Override
-  public TupleMetadata getSchema() {
-    return schema;
-  }
-
-  @Override
-  public Map<SchemaPath, ColumnStatistics> getColumnsStatistics() {
-    return columnsStatistics;
-  }
-
-  /**
-   * Returns name of the table which contain this file.
-   *
-   * @return name of the table for this file
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * Returns last modified time of the file.
-   *
-   * @return last modified time of the file.
-   */
-  public long getLastModifiedTime() {
-    return lastModifiedTime;
-  }
-}
diff --git a/metastore/file-metadata/src/main/java/org/apache/drill/metastore/FileTableMetadata.java b/metastore/file-metadata/src/main/java/org/apache/drill/metastore/FileTableMetadata.java
deleted file mode 100644
index ef82b50..0000000
--- a/metastore/file-metadata/src/main/java/org/apache/drill/metastore/FileTableMetadata.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.metastore;
-
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.SchemaPathUtils;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.hadoop.fs.Path;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Base implementation of {@link TableMetadata} interface which corresponds to file system tables.
- */
-public class FileTableMetadata implements TableMetadata {
-  private final String tableName;
-  private final Path location;
-  private final TupleMetadata schema;
-  private final Map<SchemaPath, ColumnStatistics> columnsStatistics;
-  private final Map<String, Object> tableStatistics;
-  private final Map<String, StatisticsKind> statisticsKinds;
-  private final long lastModifiedTime;
-  private final String owner;
-  private final Set<String> partitionKeys;
-
-  public FileTableMetadata(String tableName,
-                           Path location,
-                           TupleMetadata schema,
-                           Map<SchemaPath, ColumnStatistics> columnsStatistics,
-                           Map<StatisticsKind, Object> tableStatistics,
-                           long lastModifiedTime,
-                           String owner,
-                           Set<String> partitionKeys) {
-    this.tableName = tableName;
-    this.location = location;
-    this.schema = schema;
-    this.columnsStatistics = columnsStatistics;
-    this.tableStatistics = new HashMap<>();
-    this.statisticsKinds = new HashMap<>();
-    tableStatistics.forEach((statisticsKind, value) -> {
-      this.tableStatistics.put(statisticsKind.getName(), value);
-      this.statisticsKinds.put(statisticsKind.getName(), statisticsKind);
-    });
-    this.lastModifiedTime = lastModifiedTime;
-    this.owner = owner;
-    this.partitionKeys = partitionKeys;
-  }
-
-  @Override
-  public Object getStatisticsForColumn(SchemaPath columnName, StatisticsKind statisticsKind) {
-    return columnsStatistics.get(columnName).getStatistic(statisticsKind);
-  }
-
-  @Override
-  public ColumnStatistics getColumnStatistics(SchemaPath columnName) {
-    return columnsStatistics.get(columnName);
-  }
-
-  @Override
-  public Object getStatistic(StatisticsKind statisticsKind) {
-    return tableStatistics.get(statisticsKind.getName());
-  }
-
-  @Override
-  public boolean containsExactStatistics(StatisticsKind statisticsKind) {
-    return statisticsKinds.get(statisticsKind.getName()).isExact();
-  }
-
-  @Override
-  public ColumnMetadata getColumn(SchemaPath name) {
-    return SchemaPathUtils.getColumnMetadata(name, schema);
-  }
-
-  @Override
-  public TupleMetadata getSchema() {
-    return schema;
-  }
-
-  public boolean isPartitionColumn(String fieldName) {
-    return partitionKeys.contains(fieldName);
-  }
-
-  boolean isPartitioned() {
-    return !partitionKeys.isEmpty();
-  }
-
-  @Override
-  public String getTableName() {
-    return tableName;
-  }
-
-  @Override
-  public Path getLocation() {
-    return location;
-  }
-
-  @Override
-  public long getLastModifiedTime() {
-    return lastModifiedTime;
-  }
-
-  @Override
-  public String getOwner() {
-    return owner;
-  }
-
-  @Override
-  public Map<SchemaPath, ColumnStatistics> getColumnsStatistics() {
-    return columnsStatistics;
-  }
-
-  @Override
-  public FileTableMetadata cloneWithStats(Map<SchemaPath, ColumnStatistics> columnStatistics, Map<StatisticsKind, Object> tableStatistics) {
-    Map<StatisticsKind, Object> mergedTableStatistics = new HashMap<>();
-    this.tableStatistics.forEach((statisticsName, value) -> {
-      StatisticsKind statisticsKind = statisticsKinds.get(statisticsName);
-      Object statisticsValue = this.tableStatistics.get(statisticsName);
-      mergedTableStatistics.put(statisticsKind, statisticsValue);
-    });
-
-    tableStatistics.forEach((statisticsKind, statisticsValue) -> {
-      if (statisticsValue != null &&
-        (statisticsKind.isExact() || !statisticsKinds.get(statisticsKind.getName()).isExact())) {
-        // overrides statistics value for the case when new statistics is exact or existing was estimated one
-        mergedTableStatistics.put(statisticsKind, statisticsValue);
-      }
-    });
-
-    Map<SchemaPath, ColumnStatistics> newColumnsStatistics = new HashMap<>(this.columnsStatistics);
-    for (Map.Entry<SchemaPath, ColumnStatistics> columnStatisticEntry : this.columnsStatistics.entrySet()) {
-      SchemaPath columnName = columnStatisticEntry.getKey();
-      newColumnsStatistics.put(columnName, columnStatisticEntry.getValue().cloneWithStats(columnStatistics.get(columnName)));
-    }
-
-    return new FileTableMetadata(tableName, location, schema, newColumnsStatistics, mergedTableStatistics, lastModifiedTime, owner, partitionKeys);
-  }
-}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/ColumnStatistics.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/ColumnStatistics.java
deleted file mode 100644
index 0b7aab6..0000000
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/ColumnStatistics.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.metastore;
-
-import java.util.Comparator;
-
-/**
- * Represents collection of statistics values for specific column.
- *
- * @param <T> type of column values
- */
-public interface ColumnStatistics<T> {
-
-  /**
-   * Returns statistics value which corresponds to specified {@link StatisticsKind}.
-   *
-   * @param statisticsKind kind of statistics which value should be returned
-   * @return statistics value
-   */
-  Object getStatistic(StatisticsKind statisticsKind);
-
-  /**
-   * Checks whether specified statistics kind is set in this column statistics.
-   *
-   * @param statisticsKind statistics kind to check
-   * @return true if specified statistics kind is set
-   */
-  boolean containsStatistic(StatisticsKind statisticsKind);
-
-  /**
-   * Checks whether specified statistics kind is set in this column statistics
-   * and it corresponds to the exact statistics value.
-   *
-   * @param statisticsKind statistics kind to check
-   * @return true if value which corresponds to the specified statistics kind is exact
-   */
-  boolean containsExactStatistics(StatisticsKind statisticsKind);
-
-  /**
-   * Returns {@link Comparator} for comparing values with the same type as column values.
-   *
-   * @return {@link Comparator}
-   */
-  Comparator<T> getValueComparator();
-
-  /**
-   * Returns statistics value associated with value type, like a min or max value etc.
-   *
-   * @param statisticsKind kind of statistics
-   * @return statistics value for specified statistics kind
-   */
-  @SuppressWarnings("unchecked")
-  default T getValueStatistic(StatisticsKind statisticsKind) {
-    if (statisticsKind.isValueStatistic()) {
-      return (T) getStatistic(statisticsKind);
-    }
-    return null;
-  }
-
-  /**
-   * Returns new {@link ColumnStatistics} instance with overridden statistics taken from specified {@link ColumnStatistics}.
-   *
-   * @param statistics source of statistics to override
-   * @return new {@link ColumnStatistics} instance with overridden statistics
-   */
-  ColumnStatistics<T> cloneWithStats(ColumnStatistics statistics);
-}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/ColumnStatisticsImpl.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/ColumnStatisticsImpl.java
deleted file mode 100644
index 160ee98..0000000
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/ColumnStatisticsImpl.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.metastore;
-
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Base implementation of {@link ColumnStatistics} which is not bound
- * to the specific list of column statistic kinds.
- *
- * @param <T> type of column values
- */
-public class ColumnStatisticsImpl<T> implements ColumnStatistics<T> {
-
-  private final Map<String, Object> statistics;
-  private final Map<String, StatisticsKind> statisticsKinds;
-  private final Comparator<T> valueComparator;
-
-  public ColumnStatisticsImpl(Map<StatisticsKind, Object> statistics, Comparator<T> valueComparator) {
-    this.statistics = new HashMap<>();
-    this.statisticsKinds = new HashMap<>();
-    statistics.forEach((statisticsKind, value) -> {
-      this.statistics.put(statisticsKind.getName(), value);
-      this.statisticsKinds.put(statisticsKind.getName(), statisticsKind);
-    });
-    this.valueComparator = valueComparator;
-  }
-
-  @Override
-  public Object getStatistic(StatisticsKind statisticsKind) {
-    return statistics.get(statisticsKind.getName());
-  }
-
-  @Override
-  public boolean containsStatistic(StatisticsKind statisticsKind) {
-    return statistics.containsKey(statisticsKind.getName());
-  }
-
-  @Override
-  public boolean containsExactStatistics(StatisticsKind statisticsKind) {
-    return statisticsKinds.get(statisticsKind.getName()).isExact();
-  }
-
-  @Override
-  public Comparator<T> getValueComparator() {
-    return valueComparator;
-  }
-
-  @Override
-  public ColumnStatistics<T> cloneWithStats(ColumnStatistics statistics) {
-    Map<StatisticsKind, Object> newStats = new HashMap<>();
-    this.statistics.forEach((statisticsName, value) -> {
-      StatisticsKind statisticsKind = statisticsKinds.get(statisticsName);
-      Object statisticsValue = statistics.getStatistic(statisticsKind);
-      if (statisticsValue != null &&
-          (statistics.containsExactStatistics(statisticsKind) || !statisticsKind.isExact())) {
-        // overrides statistics value for the case when new statistics is exact or existing was estimated one
-        newStats.put(statisticsKind, statisticsValue);
-      } else {
-        newStats.put(statisticsKind, value);
-      }
-    });
-
-    return new ColumnStatisticsImpl<>(newStats, valueComparator);
-  }
-}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/ColumnStatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/ColumnStatisticsKind.java
deleted file mode 100644
index 51195f4..0000000
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/ColumnStatisticsKind.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.metastore;
-
-import org.apache.drill.exec.expr.ExactStatisticsConstants;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
-
-import java.util.List;
-
-/**
- * Implementation of {@link CollectableColumnStatisticsKind} which contain base
- * column statistics kinds with implemented {@code mergeStatistics()} method.
- */
-public enum ColumnStatisticsKind implements CollectableColumnStatisticsKind {
-
-  /**
-   * Column statistics kind which represents nulls count for the specific column.
-   */
-  NULLS_COUNT(ExactStatisticsConstants.NULLS_COUNT) {
-    @Override
-    public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
-      long nullsCount = 0;
-      for (ColumnStatistics statistics : statisticsList) {
-        Long statNullsCount = (Long) statistics.getStatistic(this);
-        if (statNullsCount == null || statNullsCount == Statistic.NO_COLUMN_STATS) {
-          return Statistic.NO_COLUMN_STATS;
-        } else {
-          nullsCount += statNullsCount;
-        }
-      }
-      return nullsCount;
-    }
-
-    @Override
-    public boolean isExact() {
-      return true;
-    }
-  },
-
-  /**
-   * Column statistics kind which represents min value of the specific column.
-   */
-  MIN_VALUE(ExactStatisticsConstants.MIN_VALUE) {
-    @Override
-    @SuppressWarnings("unchecked")
-    public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
-      Object minValue = null;
-      for (ColumnStatistics statistics : statisticsList) {
-        Object statMinValue = statistics.getValueStatistic(this);
-        if (statMinValue != null && (statistics.getValueComparator().compare(minValue, statMinValue) > 0 || minValue == null)) {
-          minValue = statMinValue;
-        }
-      }
-      return minValue;
-    }
-
-    @Override
-    public boolean isValueStatistic() {
-      return true;
-    }
-
-    @Override
-    public boolean isExact() {
-      return true;
-    }
-  },
-
-  /**
-   * Column statistics kind which represents max value of the specific column.
-   */
-  MAX_VALUE(ExactStatisticsConstants.MAX_VALUE) {
-    @Override
-    @SuppressWarnings("unchecked")
-    public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
-      Object maxValue = null;
-      for (ColumnStatistics statistics : statisticsList) {
-        Object statMaxValue = statistics.getValueStatistic(this);
-        if (statMaxValue != null && statistics.getValueComparator().compare(maxValue, statMaxValue) < 0) {
-          maxValue = statMaxValue;
-        }
-      }
-      return maxValue;
-    }
-
-    @Override
-    public boolean isValueStatistic() {
-      return true;
-    }
-
-    @Override
-    public boolean isExact() {
-      return true;
-    }
-  },
-
-  /**
-   * Column statistics kind which represents number of non-null values for the specific column.
-   */
-  NON_NULL_COUNT(Statistic.NNROWCOUNT) {
-    @Override
-    public Double mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
-      double nonNullRowCount = 0;
-      for (ColumnStatistics statistics : statisticsList) {
-        Double nnRowCount = (Double) statistics.getStatistic(this);
-        if (nnRowCount != null) {
-          nonNullRowCount += nnRowCount;
-        }
-      }
-      return nonNullRowCount;
-    }
-  },
-
-  /**
-   * Column statistics kind which represents total row count for the specific column.
-   */
-  ROWCOUNT(Statistic.ROWCOUNT) {
-    @Override
-    public Double mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
-      double rowCount = 0;
-      for (ColumnStatistics statistics : statisticsList) {
-        Double count = (Double) statistics.getStatistic(this);
-        if (count != null) {
-          rowCount += count;
-        }
-      }
-      return rowCount;
-    }
-  },
-
-  /**
-   * Column statistics kind which represents number of distinct values for the specific column.
-   */
-  NDV(Statistic.NDV) {
-    @Override
-    public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
-      throw new UnsupportedOperationException("Cannot merge statistics for NDV");
-    }
-  },
-
-  /**
-   * Column statistics kind which is the width of the specific column.
-   */
-  AVG_WIDTH(Statistic.AVG_WIDTH) {
-    @Override
-    public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
-      throw new UnsupportedOperationException("Cannot merge statistics for avg_width");
-    }
-  },
-
-  /**
-   * Column statistics kind which is the histogram of the specific column.
-   */
-  HISTOGRAM("histogram") {
-    @Override
-    public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
-      throw new UnsupportedOperationException("Cannot merge statistics for histogram");
-    }
-  };
-
-  private final String statisticKey;
-
-  ColumnStatisticsKind(String statisticKey) {
-    this.statisticKey = statisticKey;
-  }
-
-  public String getName() {
-    return statisticKey;
-  }
-}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/PartitionMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/PartitionMetadata.java
deleted file mode 100644
index 23dec04..0000000
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/PartitionMetadata.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.metastore;
-
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.SchemaPathUtils;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.hadoop.fs.Path;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Represents a metadata for the table part, which corresponds to the specific partition key.
- */
-public class PartitionMetadata implements BaseMetadata {
-  private final SchemaPath column;
-  private final TupleMetadata schema;
-  private final Map<SchemaPath, ColumnStatistics> columnsStatistics;
-  private final Map<String, Object> partitionStatistics;
-  private final Map<String, StatisticsKind> statisticsKinds;
-  private final Set<Path> location;
-  private final String tableName;
-  private final long lastModifiedTime;
-
-  public PartitionMetadata(SchemaPath column,
-                           TupleMetadata schema,
-                           Map<SchemaPath, ColumnStatistics> columnsStatistics,
-                           Map<StatisticsKind, Object> partitionStatistics,
-                           Set<Path> location,
-                           String tableName,
-                           long lastModifiedTime) {
-    this.column = column;
-    this.schema = schema;
-    this.columnsStatistics = columnsStatistics;
-    this.partitionStatistics = new HashMap<>();
-    this.statisticsKinds = new HashMap<>();
-    partitionStatistics.forEach((statisticsKind, value) -> {
-      this.partitionStatistics.put(statisticsKind.getName(), value);
-      this.statisticsKinds.put(statisticsKind.getName(), statisticsKind);
-    });
-    this.location = location;
-    this.tableName = tableName;
-    this.lastModifiedTime = lastModifiedTime;
-  }
-
-  @Override
-  public ColumnMetadata getColumn(SchemaPath name) {
-    return SchemaPathUtils.getColumnMetadata(name, schema);
-  }
-
-  @Override
-  public TupleMetadata getSchema() {
-    return schema;
-  }
-
-  @Override
-  public Map<SchemaPath, ColumnStatistics> getColumnsStatistics() {
-    return columnsStatistics;
-  }
-
-  @Override
-  public ColumnStatistics getColumnStatistics(SchemaPath columnName) {
-    return columnsStatistics.get(columnName);
-  }
-
-  @Override
-  public Object getStatistic(StatisticsKind statisticsKind) {
-    return partitionStatistics.get(statisticsKind.getName());
-  }
-
-  @Override
-  public boolean containsExactStatistics(StatisticsKind statisticsKind) {
-    return statisticsKinds.get(statisticsKind.getName()).isExact();
-  }
-
-  @Override
-  public Object getStatisticsForColumn(SchemaPath columnName, StatisticsKind statisticsKind) {
-    return columnsStatistics.get(columnName).getStatistic(statisticsKind);
-  }
-
-  /**
-   * It allows to obtain the column path for this partition
-   *
-   * @return column path
-   */
-  public SchemaPath getColumn() {
-    return column;
-  }
-
-  /**
-   * File locations for this partition
-   *
-   * @return file locations
-   */
-  public Set<Path> getLocations() {
-    return location;
-  }
-
-  /**
-   * Table name of this partition
-   *
-   * @return table name
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * It allows to check the time, when any files were modified. It is in Unix Timestamp
-   *
-   * @return last modified time of files
-   */
-  public long getLastModifiedTime() {
-    return lastModifiedTime;
-  }
-
-}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/RowGroupMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/RowGroupMetadata.java
deleted file mode 100644
index af39516..0000000
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/RowGroupMetadata.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.metastore;
-
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.SchemaPathUtils;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.hadoop.fs.Path;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Metadata which corresponds to the row group level of table.
- */
-public class RowGroupMetadata implements BaseMetadata, LocationProvider {
-
-  private final TupleMetadata schema;
-  private final Map<SchemaPath, ColumnStatistics> columnsStatistics;
-  private final Map<String, Object> rowGroupStatistics;
-  private final Map<String, StatisticsKind> statisticsKinds;
-  private Map<String, Float> hostAffinity;
-  private int rowGroupIndex;
-  private Path location;
-
-  public RowGroupMetadata(TupleMetadata schema,
-                          Map<SchemaPath, ColumnStatistics> columnsStatistics,
-                          Map<StatisticsKind, Object> rowGroupStatistics,
-                          Map<String, Float> hostAffinity,
-                          int rowGroupIndex,
-                          Path location) {
-    this.schema = schema;
-    this.columnsStatistics = columnsStatistics;
-    this.rowGroupStatistics = new HashMap<>();
-    this.statisticsKinds = new HashMap<>();
-    rowGroupStatistics.forEach((statisticsKind, value) -> {
-      this.rowGroupStatistics.put(statisticsKind.getName(), value);
-      this.statisticsKinds.put(statisticsKind.getName(), statisticsKind);
-    });
-    this.hostAffinity = hostAffinity;
-    this.rowGroupIndex = rowGroupIndex;
-    this.location = location;
-  }
-
-  @Override
-  public Map<SchemaPath, ColumnStatistics> getColumnsStatistics() {
-    return columnsStatistics;
-  }
-
-  @Override
-  public ColumnStatistics getColumnStatistics(SchemaPath columnName) {
-    return columnsStatistics.get(columnName);
-  }
-
-  @Override
-  public TupleMetadata getSchema() {
-    return schema;
-  }
-
-  @Override
-  public ColumnMetadata getColumn(SchemaPath name) {
-    return SchemaPathUtils.getColumnMetadata(name, schema);
-  }
-
-  @Override
-  public Object getStatistic(StatisticsKind statisticsKind) {
-    return rowGroupStatistics.get(statisticsKind.getName());
-  }
-
-  @Override
-  public boolean containsExactStatistics(StatisticsKind statisticsKind) {
-    return statisticsKinds.get(statisticsKind.getName()).isExact();
-  }
-
-  @Override
-  public Path getLocation() {
-    return location;
-  }
-
-  @Override
-  public Object getStatisticsForColumn(SchemaPath columnName, StatisticsKind statisticsKind) {
-    return columnsStatistics.get(columnName).getStatistic(statisticsKind);
-  }
-
-  /**
-   * Returns index of current row group within its file.
-   *
-   * @return row group index
-   */
-  public int getRowGroupIndex() {
-    return rowGroupIndex;
-  }
-
-  /**
-   * Returns the host affinity for a row group.
-   *
-   * @return host affinity for the row group
-   */
-  public Map<String, Float> getHostAffinity() {
-    return hostAffinity;
-  }
-}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/TableStatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/TableStatisticsKind.java
deleted file mode 100644
index 1c10938..0000000
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/TableStatisticsKind.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.metastore;
-
-import org.apache.drill.exec.expr.ExactStatisticsConstants;
-import org.apache.drill.exec.physical.impl.statistics.Statistic;
-
-import java.util.Collection;
-
-/**
- * Implementation of {@link CollectableColumnStatisticsKind} which contain base
- * table statistics kinds with implemented {@code mergeStatistics()} method.
- */
-public enum TableStatisticsKind implements CollectableTableStatisticsKind {
-  /**
-   * Table statistics kind which represents row count for the specific table.
-   */
-  ROW_COUNT(ExactStatisticsConstants.ROW_COUNT) {
-    @Override
-    public Long mergeStatistics(Collection<? extends BaseMetadata> statistics) {
-      long rowCount = 0;
-      for (BaseMetadata statistic : statistics) {
-        Long statRowCount = getValue(statistic);
-        if (statRowCount == null || statRowCount == Statistic.NO_COLUMN_STATS) {
-          rowCount = Statistic.NO_COLUMN_STATS;
-          break;
-        } else {
-          rowCount += statRowCount;
-        }
-      }
-      return rowCount;
-    }
-
-    @Override
-    public Long getValue(BaseMetadata metadata) {
-      Long rowCount = (Long) metadata.getStatistic(this);
-      return rowCount != null ? rowCount : Statistic.NO_COLUMN_STATS;
-    }
-
-    @Override
-    public boolean isExact() {
-      return true;
-    }
-  },
-
-  /**
-   * Table statistics kind which represents estimated row count for the specific table.
-   */
-  EST_ROW_COUNT(Statistic.ROWCOUNT) {
-    @Override
-    public Double mergeStatistics(Collection<? extends BaseMetadata> statisticsList) {
-      double rowCount = 0;
-      for (BaseMetadata statistics : statisticsList) {
-        Double statRowCount = (Double) statistics.getStatistic(this);
-        if (statRowCount != null) {
-          rowCount += statRowCount;
-        }
-      }
-      return rowCount;
-    }
-
-    @Override
-    public Double getValue(BaseMetadata metadata) {
-      return (Double) metadata.getStatistic(this);
-    }
-  },
-
-  /**
-   * Table statistics kind which represents estimated row count for the specific table.
-   */
-  HAS_STATISTICS("has_statistics") {
-    @Override
-    public Boolean mergeStatistics(Collection<? extends BaseMetadata> statisticsList) {
-      for (BaseMetadata statistics : statisticsList) {
-        Boolean hasStatistics = (Boolean) statistics.getStatistic(this);
-        if (hasStatistics == null || !hasStatistics) {
-          return false;
-        }
-      }
-      return Boolean.TRUE;
-    }
-
-    @Override
-    public Boolean getValue(BaseMetadata metadata) {
-      return Boolean.TRUE.equals(metadata.getStatistic(this));
-    }
-  };
-
-  private final String statisticKey;
-
-  TableStatisticsKind(String statisticKey) {
-    this.statisticKey = statisticKey;
-  }
-
-  public String getName() {
-    return statisticKey;
-  }
-
-  /**
-   * Returns value which corresponds to this statistic kind,
-   * obtained from specified {@link BaseMetadata}.
-   *
-   * @param metadata the source of statistic value
-   * @return value which corresponds to this statistic kind
-   */
-  public abstract Object getValue(BaseMetadata metadata);
-}
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
new file mode 100644
index 0000000..e66d75e
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseMetadata.java
@@ -0,0 +1,148 @@
+/*
+ * 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.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.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
+import org.apache.drill.metastore.statistics.StatisticsKind;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+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 {
+  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 <T extends BaseMetadataBuilder<T>> BaseMetadata(BaseMetadataBuilder<T> builder) {
+    this.tableInfo = builder.tableInfo;
+    this.metadataInfo = builder.metadataInfo;
+    this.schema = builder.schema;
+    this.columnsStatistics = builder.columnsStatistics;
+    this.metadataStatistics = builder.metadataStatistics.stream()
+        .collect(Collectors.toMap(
+            statistic -> statistic.getStatisticsKind().getName(),
+            Function.identity(),
+            (a, b) -> a.getStatisticsKind().isExact() ? a : b));
+  }
+
+  @Override
+  public Map<SchemaPath, ColumnStatistics> getColumnsStatistics() {
+    return columnsStatistics;
+  }
+
+  @Override
+  public ColumnStatistics getColumnStatistics(SchemaPath columnName) {
+    return columnsStatistics.get(columnName);
+  }
+
+  @Override
+  public TupleMetadata getSchema() {
+    return schema;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public <V> V getStatistic(StatisticsKind<V> statisticsKind) {
+    StatisticsHolder<V> statisticsHolder = metadataStatistics.get(statisticsKind.getName());
+    return statisticsHolder != null ? statisticsHolder.getStatisticsValue() : null;
+  }
+
+  @Override
+  public boolean containsExactStatistics(StatisticsKind statisticsKind) {
+    StatisticsHolder statisticsHolder = metadataStatistics.get(statisticsKind.getName());
+    return statisticsHolder != null && statisticsHolder.getStatisticsKind().isExact();
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public <V> V getStatisticsForColumn(SchemaPath columnName, StatisticsKind<V> statisticsKind) {
+    return (V) columnsStatistics.get(columnName).get(statisticsKind);
+  }
+
+  @Override
+  public ColumnMetadata getColumn(SchemaPath name) {
+    return SchemaPathUtils.getColumnMetadata(name, schema);
+  }
+
+  @Override
+  public TableInfo getTableInfo() {
+    return tableInfo;
+  }
+
+  @Override
+  public MetadataInfo getMetadataInfo() {
+    return metadataInfo;
+  }
+
+  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;
+
+    public T tableInfo(TableInfo tableInfo) {
+      this.tableInfo = tableInfo;
+      return self();
+    }
+
+    public T metadataInfo(MetadataInfo metadataInfo) {
+      this.metadataInfo = metadataInfo;
+      return self();
+    }
+
+    public T schema(TupleMetadata schema) {
+      this.schema = schema;
+      return self();
+    }
+
+    public T columnsStatistics(Map<SchemaPath, ColumnStatistics> columnsStatistics) {
+      this.columnsStatistics = columnsStatistics;
+      return self();
+    }
+
+    public T metadataStatistics(Collection<StatisticsHolder> metadataStatistics) {
+      this.metadataStatistics = metadataStatistics;
+      return self();
+    }
+
+    protected void checkRequiredValues() {
+      Objects.requireNonNull(tableInfo, "tableInfo was not set");
+      Objects.requireNonNull(metadataInfo, "metadataInfo was not set");
+      Objects.requireNonNull(columnsStatistics, "columnsStatistics were not set");
+      Objects.requireNonNull(metadataStatistics, "metadataStatistics were not set");
+    }
+
+    public abstract BaseMetadata build();
+
+    protected abstract T self();
+  }
+}
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
new file mode 100644
index 0000000..690b44a
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseTableMetadata.java
@@ -0,0 +1,148 @@
+/*
+ * 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.metadata;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
+import org.apache.hadoop.fs.Path;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * 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;
+
+  private BaseTableMetadata(BaseTableMetadataBuilder builder) {
+    super(builder);
+    this.location = builder.location;
+    this.partitionKeys = builder.partitionKeys;
+    this.interestingColumns = builder.interestingColumns;
+    this.lastModifiedTime = builder.lastModifiedTime;
+  }
+
+  public boolean isPartitionColumn(String fieldName) {
+    return partitionKeys.containsKey(fieldName);
+  }
+
+  boolean isPartitioned() {
+    return !partitionKeys.isEmpty();
+  }
+
+  @Override
+  public Path getLocation() {
+    return location;
+  }
+
+  @Override
+  public long getLastModifiedTime() {
+    return lastModifiedTime;
+  }
+
+  @Override
+  public List<SchemaPath> getInterestingColumns() {
+    return interestingColumns;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public BaseTableMetadata cloneWithStats(Map<SchemaPath, ColumnStatistics> columnStatistics, List<StatisticsHolder> tableStatistics) {
+    Map<String, StatisticsHolder> mergedTableStatistics = new HashMap<>(this.metadataStatistics);
+
+    // overrides statistics value for the case when new statistics is exact or existing one was estimated
+    tableStatistics.stream()
+        .filter(statisticsHolder -> statisticsHolder.getStatisticsKind().isExact()
+              || !this.metadataStatistics.get(statisticsHolder.getStatisticsKind().getName()).getStatisticsKind().isExact())
+        .forEach(statisticsHolder -> mergedTableStatistics.put(statisticsHolder.getStatisticsKind().getName(), statisticsHolder));
+
+    Map<SchemaPath, ColumnStatistics> newColumnsStatistics = new HashMap<>(this.columnsStatistics);
+    this.columnsStatistics.forEach(
+        (columnName, value) -> newColumnsStatistics.put(columnName, value.cloneWith(columnStatistics.get(columnName))));
+
+    return BaseTableMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataInfo(metadataInfo)
+        .location(location)
+        .schema(schema)
+        .columnsStatistics(newColumnsStatistics)
+        .metadataStatistics(mergedTableStatistics.values())
+        .lastModifiedTime(lastModifiedTime)
+        .partitionKeys(partitionKeys)
+        .interestingColumns(interestingColumns)
+        .build();
+  }
+
+  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;
+
+    public BaseTableMetadataBuilder location(Path location) {
+      this.location = location;
+      return self();
+    }
+
+    public BaseTableMetadataBuilder lastModifiedTime(long lastModifiedTime) {
+      this.lastModifiedTime = lastModifiedTime;
+      return self();
+    }
+
+    public BaseTableMetadataBuilder partitionKeys(Map<String, String> partitionKeys) {
+      this.partitionKeys = partitionKeys;
+      return self();
+    }
+
+    public BaseTableMetadataBuilder interestingColumns(List<SchemaPath> interestingColumns) {
+      this.interestingColumns = interestingColumns;
+      return self();
+    }
+
+    @Override
+    protected void checkRequiredValues() {
+      super.checkRequiredValues();
+      Objects.requireNonNull(partitionKeys, "partitionKeys were not set");
+    }
+
+    @Override
+    public BaseTableMetadata build() {
+      checkRequiredValues();
+      return new BaseTableMetadata(this);
+    }
+
+    @Override
+    protected BaseTableMetadataBuilder self() {
+      return this;
+    }
+  }
+}
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
new file mode 100644
index 0000000..f99b7eb
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/FileMetadata.java
@@ -0,0 +1,91 @@
+/*
+ * 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.metadata;
+
+import org.apache.hadoop.fs.Path;
+
+import java.util.Objects;
+
+/**
+ * Metadata which corresponds to the file level of table.
+ */
+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
+  public Path getPath() {
+    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;
+  }
+
+  public Path getLocation() {
+    return path.getParent();
+  }
+
+  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();
+      Objects.requireNonNull(path, "path was not set");
+    }
+
+    @Override
+    public FileMetadata build() {
+      checkRequiredValues();
+      return new FileMetadata(this);
+    }
+
+    @Override
+    protected FileMetadataBuilder self() {
+      return this;
+    }
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/LocationProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
similarity index 94%
rename from metastore/metastore-api/src/main/java/org/apache/drill/metastore/LocationProvider.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
index e29f323..7db95fb 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/LocationProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/LocationProvider.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore;
+package org.apache.drill.metastore.metadata;
 
 import org.apache.hadoop.fs.Path;
 
@@ -29,5 +29,5 @@ public interface LocationProvider {
    *
    * @return metadata location
    */
-  Path getLocation();
+  Path getPath();
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/BaseMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java
similarity index 85%
rename from metastore/metastore-api/src/main/java/org/apache/drill/metastore/BaseMetadata.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java
index 330ec29..079ec42 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/BaseMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java
@@ -15,18 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore;
+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.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.StatisticsKind;
 
 import java.util.Map;
 
 /**
- * Common provider of tuple schema, column metadata, and statistics for table, partition, file or row group.
+ * Provider of tuple schema, column metadata, and statistics for table, partition, file or row group.
  */
-public interface BaseMetadata {
+public interface Metadata {
 
   /**
    * Returns statistics stored in current metadata represented
@@ -58,7 +60,7 @@ public interface BaseMetadata {
    * @param statisticsKind statistics kind whose value should be returned
    * @return value of non-column statistics
    */
-  Object getStatistic(StatisticsKind statisticsKind);
+  <V> V getStatistic(StatisticsKind<V> statisticsKind);
 
   /**
    * Checks whether specified statistics kind is set in this non-column statistics
@@ -77,7 +79,7 @@ public interface BaseMetadata {
    * @param statisticsKind statistics kind whose value should be returned
    * @return value of column statistics
    */
-  Object getStatisticsForColumn(SchemaPath columnName, StatisticsKind statisticsKind);
+  <V> V getStatisticsForColumn(SchemaPath columnName, StatisticsKind<V> statisticsKind);
 
   /**
    * Returns metadata description for the specified column
@@ -86,4 +88,8 @@ public interface BaseMetadata {
    * @return {@link ColumnMetadata} schema description of the column
    */
   ColumnMetadata getColumn(SchemaPath name);
+
+  TableInfo getTableInfo();
+
+  MetadataInfo getMetadataInfo();
 }
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
new file mode 100644
index 0000000..f59c99b
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataInfo.java
@@ -0,0 +1,54 @@
+/*
+ * 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.metadata;
+
+/**
+ * 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)}.
+ */
+public class MetadataInfo {
+
+  public static final String GENERAL_INFO_KEY = "GENERAL_INFO";
+  public static final String DEFAULT_SEGMENT_KEY = "DEFAULT_SEGMENT";
+  public static final String DEFAULT_COLUMN_PREFIX = "_$SEGMENT_";
+
+  private final MetadataType type;
+  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;
+  }
+
+  public MetadataType getType() {
+    return type;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  public String getIdentifier() {
+    return identifier;
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/StatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java
similarity index 50%
copy from metastore/metastore-api/src/main/java/org/apache/drill/metastore/StatisticsKind.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java
index 3be627a..da813d7 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/StatisticsKind.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/MetadataType.java
@@ -15,32 +15,47 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore;
+package org.apache.drill.metastore.metadata;
 
 /**
- * Class represents kind of statistics or metadata, for example it may be min value for column,
- * or row count for table.
+ * Enum with possible types of metadata.
  */
-public interface StatisticsKind {
+public enum MetadataType {
 
   /**
-   * Returns name which corresponds to the statistics.
-   *
-   * @return statistic kind name
+   * Metadata that can be applicable to any type.
    */
-  String getName();
+  ALL,
 
   /**
-   * Returns true if current statistics value has the same type as column values,
-   * for example it may be min value etc.
-   *
-   * @return true if current statistics value has the same type as column values
+   * Table level metadata type.
    */
-  default boolean isValueStatistic() {
-    return false;
-  }
+  TABLE,
 
-  default boolean isExact() {
-    return false;
-  }
+  /**
+   * Segment level metadata type. It corresponds to the metadata
+   * within specific directory for FS tables, or may correspond to partition for hive tables.
+   */
+  SEGMENT,
+
+  /**
+   * 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.
+   */
+  PARTITION,
+
+  /**
+   * File level metadata type.
+   */
+  FILE,
+
+  /**
+   * Row group level metadata type. Used for parquet tables.
+   */
+  ROW_GROUP,
+
+  /**
+   * Metadata type which helps to indicate that there is no overflow of metadata.
+   */
+  NONE
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/NonInterestingColumnsMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java
similarity index 73%
rename from metastore/metastore-api/src/main/java/org/apache/drill/metastore/NonInterestingColumnsMetadata.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java
index 56e6bd1..83fb5a2 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/NonInterestingColumnsMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java
@@ -15,25 +15,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore;
+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.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.StatisticsKind;
+
 import java.util.Map;
 
 /**
  * Represents a metadata for the non-interesting columns. Since the refresh command doesn't store the non-interesting
  * columns stats in the cache file, there is a need to mark column statistics of non-interesting as unknown to
  * differentiate the non-interesting columns from non-existent columns. Since the sole purpose of this class is to store
- * column statistics for non-interesting columns, some methods like getSchema, getStatistic, getColumn are not applicable
+ * column statistics for non-interesting columns, some methods like getSchema, get, getColumn are not applicable
  * to NonInterestingColumnsMetadata.
  */
-public class NonInterestingColumnsMetadata implements BaseMetadata {
+public class NonInterestingColumnsMetadata implements Metadata {
   private final Map<SchemaPath, ColumnStatistics> columnsStatistics;
 
-  public NonInterestingColumnsMetadata(
-                           Map<SchemaPath, ColumnStatistics> columnsStatistics) {
+  public NonInterestingColumnsMetadata(Map<SchemaPath, ColumnStatistics> columnsStatistics) {
     this.columnsStatistics = columnsStatistics;
   }
 
@@ -53,7 +55,7 @@ public class NonInterestingColumnsMetadata implements BaseMetadata {
   }
 
   @Override
-  public Object getStatistic(StatisticsKind statisticsKind) {
+  public <V> V getStatistic(StatisticsKind<V> statisticsKind) {
     return null;
   }
 
@@ -63,12 +65,23 @@ public class NonInterestingColumnsMetadata implements BaseMetadata {
   }
 
   @Override
-  public Object getStatisticsForColumn(SchemaPath columnName, StatisticsKind statisticsKind) {
-    return columnsStatistics.get(columnName).getStatistic(statisticsKind);
+  @SuppressWarnings("unchecked")
+  public <V> V getStatisticsForColumn(SchemaPath columnName, StatisticsKind<V> statisticsKind) {
+    return (V) columnsStatistics.get(columnName).get(statisticsKind);
   }
 
   @Override
   public ColumnMetadata getColumn(SchemaPath name) {
     return null;
   }
+
+  @Override
+  public TableInfo getTableInfo() {
+    return null;
+  }
+
+  @Override
+  public MetadataInfo getMetadataInfo() {
+    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
new file mode 100644
index 0000000..938e5fd
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/PartitionMetadata.java
@@ -0,0 +1,125 @@
+/*
+ * 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.metadata;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Represents a metadata for the table part, which corresponds to the specific partition key.
+ */
+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;
+  }
+
+  /**
+   * It allows to obtain the column path for this partition
+   *
+   * @return column path
+   */
+  public SchemaPath getColumn() {
+    return column;
+  }
+
+  /**
+   * File locations for this partition
+   *
+   * @return file locations
+   */
+  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;
+  }
+
+  public List<String> getPartitionValues() {
+    return partitionValues;
+  }
+
+  public static PartitionMetadataBuilder builder() {
+    return new PartitionMetadataBuilder();
+  }
+
+  public static class PartitionMetadataBuilder extends BaseMetadataBuilder<PartitionMetadataBuilder> {
+    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();
+    }
+
+    public PartitionMetadataBuilder column(SchemaPath column) {
+      this.column = column;
+      return self();
+    }
+
+    @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");
+    }
+
+    @Override
+    public PartitionMetadata build() {
+      checkRequiredValues();
+      return new PartitionMetadata(this);
+    }
+
+    @Override
+    protected PartitionMetadataBuilder self() {
+      return this;
+    }
+  }
+}
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
new file mode 100644
index 0000000..af37d5e
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/RowGroupMetadata.java
@@ -0,0 +1,108 @@
+/*
+ * 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.metadata;
+
+import org.apache.hadoop.fs.Path;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Metadata which corresponds to the row group level of table.
+ */
+public class RowGroupMetadata extends BaseMetadata implements LocationProvider {
+  private Map<String, Float> hostAffinity;
+  private int rowGroupIndex;
+  private Path path;
+
+  private RowGroupMetadata(RowGroupMetadataBuilder builder) {
+    super(builder);
+    this.hostAffinity = builder.hostAffinity;
+    this.rowGroupIndex = builder.rowGroupIndex;
+    this.path = builder.path;
+  }
+
+  @Override
+  public Path getPath() {
+    return path;
+  }
+
+  public Path getLocation() {
+    return path.getParent();
+  }
+
+  /**
+   * Returns index of current row group within its file.
+   *
+   * @return row group index
+   */
+  public int getRowGroupIndex() {
+    return rowGroupIndex;
+  }
+
+  /**
+   * Returns the host affinity for a row group.
+   *
+   * @return host affinity for the row group
+   */
+  public Map<String, Float> getHostAffinity() {
+    return hostAffinity;
+  }
+
+  public static RowGroupMetadataBuilder builder() {
+    return new RowGroupMetadataBuilder();
+  }
+
+  public static class RowGroupMetadataBuilder extends BaseMetadataBuilder<RowGroupMetadataBuilder> {
+    private Map<String, Float> hostAffinity;
+    private int rowGroupIndex;
+    private Path path;
+
+    public RowGroupMetadataBuilder hostAffinity(Map<String, Float> hostAffinity) {
+      this.hostAffinity = hostAffinity;
+      return self();
+    }
+
+    public RowGroupMetadataBuilder rowGroupIndex(int rowGroupIndex) {
+      this.rowGroupIndex = rowGroupIndex;
+      return self();
+    }
+
+    public RowGroupMetadataBuilder path(Path path) {
+      this.path = path;
+      return self();
+    }
+
+    @Override
+    protected void checkRequiredValues() {
+      super.checkRequiredValues();
+      Objects.requireNonNull(path, "path was not set");
+    }
+
+    @Override
+    public RowGroupMetadata build() {
+      checkRequiredValues();
+      return new RowGroupMetadata(this);
+    }
+
+    @Override
+    protected RowGroupMetadataBuilder self() {
+      return this;
+    }
+  }
+}
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
new file mode 100644
index 0000000..9a85b0e
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/SegmentMetadata.java
@@ -0,0 +1,127 @@
+/*
+ * 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.metadata;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Metadata which corresponds to the segment level of table.
+ */
+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 SegmentMetadata(SegmentMetadataBuilder builder) {
+    super(builder);
+    this.column = builder.column;
+    this.path = builder.path;
+    this.partitionValues = builder.partitionValues;
+    this.locations = builder.locations;
+    this.lastModifiedTime = builder.lastModifiedTime;
+  }
+
+  public SchemaPath getSegmentColumn() {
+    return column;
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  public List<String> getPartitionValues() {
+    return partitionValues;
+  }
+
+  public List<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;
+  }
+
+  public static SegmentMetadataBuilder builder() {
+    return new SegmentMetadataBuilder();
+  }
+
+  public static class SegmentMetadataBuilder extends BaseMetadataBuilder<SegmentMetadataBuilder> {
+    private SchemaPath column;
+    private List<String> partitionValues;
+    private Path path;
+    private List<Path> locations;
+    private long lastModifiedTime = BaseTableMetadata.NON_DEFINED_LAST_MODIFIED_TIME;
+
+    public SegmentMetadataBuilder locations(List<Path> locations) {
+      this.locations = locations;
+      return self();
+    }
+
+    public SegmentMetadataBuilder path(Path path) {
+      this.path = path;
+      return self();
+    }
+
+    public SegmentMetadataBuilder lastModifiedTime(long lastModifiedTime) {
+      this.lastModifiedTime = lastModifiedTime;
+      return self();
+    }
+
+    public SegmentMetadataBuilder partitionValues(List<String> partitionValues) {
+      this.partitionValues = partitionValues;
+      return self();
+    }
+
+    public SegmentMetadataBuilder column(SchemaPath column) {
+      this.column = column;
+      return self();
+    }
+
+    @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");
+    }
+
+    @Override
+    public SegmentMetadata build() {
+      checkRequiredValues();
+      return new SegmentMetadata(this);
+    }
+
+    @Override
+    protected SegmentMetadataBuilder self() {
+      return this;
+    }
+  }
+}
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
new file mode 100644
index 0000000..3e73851
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableInfo.java
@@ -0,0 +1,108 @@
+/*
+ * 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.metadata;
+
+/**
+ * General table information.
+ */
+public class TableInfo {
+  public static final String UNKNOWN = "UNKNOWN";
+  public static final TableInfo UNKNOWN_TABLE_INFO = TableInfo.builder()
+      .storagePlugin(UNKNOWN)
+      .workspace(UNKNOWN)
+      .name(UNKNOWN)
+      .type(UNKNOWN)
+      .owner(UNKNOWN)
+      .build();
+
+  private final String storagePlugin;
+  private final String workspace;
+  private final String name;
+  private final String type;
+  private final String owner;
+
+  private TableInfo(TableInfoBuilder builder) {
+    this.storagePlugin = builder.storagePlugin;
+    this.workspace = builder.workspace;
+    this.name = builder.name;
+    this.type = builder.type;
+    this.owner = builder.owner;
+  }
+
+  public String getStoragePlugin() {
+    return storagePlugin;
+  }
+
+  public String getWorkspace() {
+    return workspace;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public String getOwner() {
+    return owner;
+  }
+
+  public static TableInfoBuilder builder() {
+    return new TableInfoBuilder();
+  }
+
+  public static class TableInfoBuilder {
+    private String storagePlugin;
+    private String workspace;
+    private String name;
+    private String type;
+    private String owner;
+
+    public TableInfoBuilder storagePlugin(String storagePlugin) {
+      this.storagePlugin = storagePlugin;
+      return this;
+    }
+
+    public TableInfoBuilder workspace(String workspace) {
+      this.workspace = workspace;
+      return this;
+    }
+
+    public TableInfoBuilder name(String name) {
+      this.name = name;
+      return this;
+    }
+
+    public TableInfoBuilder type(String type) {
+      this.type = type;
+      return this;
+    }
+
+    public TableInfoBuilder owner(String owner) {
+      this.owner = owner;
+      return this;
+    }
+
+    public TableInfo build() {
+      return new TableInfo(this);
+    }
+
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/TableMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadata.java
similarity index 75%
rename from metastore/metastore-api/src/main/java/org/apache/drill/metastore/TableMetadata.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadata.java
index 17a1767..517d232 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/TableMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadata.java
@@ -15,21 +15,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore;
+package org.apache.drill.metastore.metadata;
 
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
 import org.apache.hadoop.fs.Path;
 
+import java.util.List;
 import java.util.Map;
 
 /**
  * Metadata which corresponds to the table level.
  */
-public interface TableMetadata extends BaseMetadata {
+public interface TableMetadata extends Metadata {
 
-  String getTableName();
   Path getLocation();
-  String getOwner();
   long getLastModifiedTime();
-  TableMetadata cloneWithStats(Map<SchemaPath, ColumnStatistics> columnStatistics, Map<StatisticsKind, Object> tableStatistics);
+  TableMetadata cloneWithStats(Map<SchemaPath, ColumnStatistics> columnStatistics, List<StatisticsHolder> tableStatistics);
+  List<SchemaPath> getInterestingColumns();
 }
diff --git a/metastore/file-metadata/src/main/java/org/apache/drill/exec/physical/base/TableMetadataProvider.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadataProvider.java
similarity index 84%
rename from metastore/file-metadata/src/main/java/org/apache/drill/exec/physical/base/TableMetadataProvider.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadataProvider.java
index f0228d4..2cea808 100644
--- a/metastore/file-metadata/src/main/java/org/apache/drill/exec/physical/base/TableMetadataProvider.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadataProvider.java
@@ -15,16 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.base;
+package org.apache.drill.metastore.metadata;
 
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.metastore.FileMetadata;
-import org.apache.drill.metastore.NonInterestingColumnsMetadata;
-import org.apache.drill.metastore.PartitionMetadata;
-import org.apache.drill.metastore.TableMetadata;
 import org.apache.hadoop.fs.Path;
 
 import java.util.List;
+import java.util.Map;
 
 /**
  * Base interface for providing table, partition, file etc. metadata for specific table.
@@ -65,11 +62,18 @@ public interface TableMetadataProvider {
   List<PartitionMetadata> getPartitionMetadata(SchemaPath columnName);
 
   /**
-   * Returns list of {@link FileMetadata} instances which provides metadata for specific file and its columns.
+   * Returns map of {@link FileMetadata} instances which provides metadata for specific file and its columns.
    *
-   * @return list of {@link FileMetadata} instances
+   * @return map of {@link FileMetadata} instances
    */
-  List<FileMetadata> getFilesMetadata();
+  Map<Path, FileMetadata> getFilesMetadataMap();
+
+  /**
+   * Returns map of {@link SegmentMetadata} instances which provides metadata for segment and its columns.
+   *
+   * @return map of {@link SegmentMetadata} instances
+   */
+  Map<Path, SegmentMetadata> getSegmentsMetadataMap();
 
   /**
    * Returns {@link FileMetadata} instance which corresponds to metadata of file for specified location.
@@ -91,5 +95,5 @@ public interface TableMetadataProvider {
    * Returns {@link NonInterestingColumnsMetadata} instance which provides metadata for non-interesting columns.
    * @return {@link NonInterestingColumnsMetadata} instance
    */
-  NonInterestingColumnsMetadata getNonInterestingColumnsMeta();
+  NonInterestingColumnsMetadata getNonInterestingColumnsMetadata();
 }
diff --git a/metastore/file-metadata/src/main/java/org/apache/drill/exec/physical/base/TableMetadataProviderBuilder.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadataProviderBuilder.java
similarity index 96%
rename from metastore/file-metadata/src/main/java/org/apache/drill/exec/physical/base/TableMetadataProviderBuilder.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadataProviderBuilder.java
index 42f4544..47bce72 100644
--- a/metastore/file-metadata/src/main/java/org/apache/drill/exec/physical/base/TableMetadataProviderBuilder.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadataProviderBuilder.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.base;
+package org.apache.drill.metastore.metadata;
 
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/BaseStatisticsKind.java
similarity index 52%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
copy to metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/BaseStatisticsKind.java
index 4e9a762..e825e88 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/BaseStatisticsKind.java
@@ -15,34 +15,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.impl.statistics;
+package org.apache.drill.metastore.statistics;
 
-import org.apache.drill.exec.vector.complex.MapVector;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
 
-public abstract class AbstractMergedStatistic implements MergedStatistic, Statistic {
-  protected String name;
-  protected String inputName;
-  protected double samplePercent;
-  protected State state;
-
-  public void initialize(String name, String inputName, double samplePercent) {
-    this.name = name;
-    this.inputName = inputName;
-    this.samplePercent = samplePercent;
-  }
-
-  @Override
-  public abstract void initialize(String inputName, double samplePercent);
+/**
+ * Implementation of {@link StatisticsKind} which contain base
+ * table statistics kinds with implemented {@code mergeStatistics()} method.
+ */
+public class BaseStatisticsKind<T> implements StatisticsKind<T> {
 
-  @Override
-  public abstract String getName();
+  protected final String statisticKey;
+  protected final boolean exact;
 
-  @Override
-  public abstract String getInput();
+  @JsonCreator
+  public BaseStatisticsKind(@JsonProperty("name") String statisticKey,
+                            @JsonProperty("exact") boolean exact) {
+    this.statisticKey = statisticKey;
+    this.exact = exact;
+  }
 
   @Override
-  public abstract void merge(MapVector input);
+  public String getName() {
+    return statisticKey;
+  }
 
   @Override
-  public abstract void setOutput(MapVector output);
+  public boolean isExact() {
+    return exact;
+  }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/CollectableColumnStatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableColumnStatisticsKind.java
similarity index 91%
rename from metastore/metastore-api/src/main/java/org/apache/drill/metastore/CollectableColumnStatisticsKind.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableColumnStatisticsKind.java
index 754c05d..00b1f1d 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/CollectableColumnStatisticsKind.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableColumnStatisticsKind.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore;
+package org.apache.drill.metastore.statistics;
 
 import java.util.List;
 
@@ -24,7 +24,7 @@ import java.util.List;
  * of other statistics, for example column nulls count may be received as a sum of nulls counts
  * of underlying metadata parts.
  */
-public interface CollectableColumnStatisticsKind extends StatisticsKind {
+public interface CollectableColumnStatisticsKind<V> extends StatisticsKind<V> {
 
   /**
    * Returns column statistics value received by collecting specified {@link ColumnStatistics}.
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/CollectableTableStatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableTableStatisticsKind.java
similarity index 84%
rename from metastore/metastore-api/src/main/java/org/apache/drill/metastore/CollectableTableStatisticsKind.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableTableStatisticsKind.java
index 2291517..345ea5e 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/CollectableTableStatisticsKind.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableTableStatisticsKind.java
@@ -15,7 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore;
+package org.apache.drill.metastore.statistics;
+
+import org.apache.drill.metastore.metadata.Metadata;
 
 import java.util.Collection;
 
@@ -24,7 +26,7 @@ import java.util.Collection;
  * of other statistics, for example row count may be received as a sum of row counts
  * of underlying metadata parts.
  */
-public interface CollectableTableStatisticsKind extends StatisticsKind {
+public interface CollectableTableStatisticsKind<T> extends StatisticsKind<T> {
 
   /**
    * Returns table statistics value received by collecting specified {@link ColumnStatistics}.
@@ -32,5 +34,5 @@ public interface CollectableTableStatisticsKind extends StatisticsKind {
    * @param statistics list of {@link ColumnStatistics} instances to be collected
    * @return column statistics value received by collecting specified {@link ColumnStatistics}
    */
-  Object mergeStatistics(Collection<? extends BaseMetadata> statistics);
+  T mergeStatistics(Collection<? extends Metadata> statistics);
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java
new file mode 100644
index 0000000..3bba216
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java
@@ -0,0 +1,179 @@
+/*
+ * 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.statistics;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.metastore.util.TableMetadataUtils;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.metastore.statistics.StatisticsHolder.OBJECT_WRITER;
+
+/**
+ * Represents collection of statistics values for specific column.
+ * For example, text representation of {@link ColumnStatistics} for varchar column
+ * is the following:
+ * <pre>
+ * {
+ *     "statistics":[
+ *         {"statisticsValue":2.1,"statisticsKind":{"name":"approx_count_distinct"}},
+ *         {"statisticsValue":"aaa","statisticsKind":{"exact":true,"name":"minValue"}},
+ *         {"statisticsValue":3,"statisticsKind":{"exact":true,"name":"nullsCount"}},
+ *         {"statisticsValue":"zzz","statisticsKind":{"exact":true,"name":"maxValue"}}],
+ *     "type":"VARCHAR"
+ * }
+ * </pre>
+ *
+ * @param <T> type of column values
+ */
+@JsonAutoDetect(
+  fieldVisibility = JsonAutoDetect.Visibility.NONE,
+  getterVisibility = JsonAutoDetect.Visibility.NONE,
+  isGetterVisibility = JsonAutoDetect.Visibility.NONE,
+  setterVisibility = JsonAutoDetect.Visibility.NONE)
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+@JsonPropertyOrder({"statistics", "comparator"})
+public class ColumnStatistics<T> {
+
+  private static final ObjectReader OBJECT_READER = new ObjectMapper().readerFor(ColumnStatistics.class);
+
+  private final Map<String, StatisticsHolder> statistics;
+  private final Comparator<T> valueComparator;
+  private final TypeProtos.MinorType type;
+
+  @JsonCreator
+  @SuppressWarnings("unchecked")
+  public ColumnStatistics(@JsonProperty("statistics") Collection<StatisticsHolder> statistics,
+                          @JsonProperty("type") TypeProtos.MinorType type) {
+    this.type = type;
+    this.valueComparator = type != null
+        ? TableMetadataUtils.getComparator(type)
+        : (Comparator<T>) TableMetadataUtils.getNaturalNullsFirstComparator();
+    this.statistics = statistics.stream()
+        .collect(Collectors.toMap(
+            statistic -> statistic.getStatisticsKind().getName(),
+            Function.identity(),
+            (a, b) -> a.getStatisticsKind().isExact() ? a : b));
+  }
+
+  public ColumnStatistics(Collection<StatisticsHolder> statistics) {
+    this(statistics, TypeProtos.MinorType.INT);
+  }
+
+  /**
+   * Returns statistics value which corresponds to specified {@link StatisticsKind}.
+   *
+   * @param statisticsKind kind of statistics which value should be returned
+   * @return statistics value
+   */
+  @SuppressWarnings("unchecked")
+  public <V> V get(StatisticsKind<V> statisticsKind) {
+    StatisticsHolder<V> statisticsHolder = statistics.get(statisticsKind.getName());
+    if (statisticsHolder != null) {
+      return statisticsHolder.getStatisticsValue();
+    }
+    return null;
+  }
+
+  /**
+   * Checks whether specified statistics kind is set in this column statistics.
+   *
+   * @param statisticsKind statistics kind to check
+   * @return true if specified statistics kind is set
+   */
+  public boolean contains(StatisticsKind statisticsKind) {
+    return statistics.containsKey(statisticsKind.getName());
+  }
+
+  /**
+   * Checks whether specified statistics kind is set in this column statistics
+   * and it corresponds to the exact statistics value.
+   *
+   * @param statisticsKind statistics kind to check
+   * @return true if value which corresponds to the specified statistics kind is exact
+   */
+  public boolean containsExact(StatisticsKind statisticsKind) {
+    StatisticsHolder statisticsHolder = statistics.get(statisticsKind.getName());
+    if (statisticsHolder != null) {
+      return statisticsHolder.getStatisticsKind().isExact();
+    }
+    return false;
+  }
+
+  /**
+   * Returns {@link Comparator} for comparing values with the same type as column values.
+   *
+   * @return {@link Comparator}
+   */
+  public Comparator<T> getValueComparator() {
+    return valueComparator;
+  }
+
+  /**
+   * Returns new {@link ColumnStatistics} instance with overridden statistics taken from specified {@link ColumnStatistics}.
+   *
+   * @param sourceStatistics source of statistics to override
+   * @return new {@link ColumnStatistics} instance with overridden statistics
+   */
+  public ColumnStatistics<T> cloneWith(ColumnStatistics<T> sourceStatistics) {
+    Map<String, StatisticsHolder> newStats = new HashMap<>(this.statistics);
+    statistics.values().forEach(statisticsHolder -> {
+      StatisticsKind currentStatisticsKind = statisticsHolder.getStatisticsKind();
+      StatisticsHolder statisticsToMerge = sourceStatistics.statistics.get(currentStatisticsKind.getName());
+      if (statisticsToMerge != null &&
+          (statisticsToMerge.getStatisticsKind().isExact() || !currentStatisticsKind.isExact())) {
+        newStats.put(currentStatisticsKind.getName(), statisticsToMerge);
+      }
+    });
+
+    return new ColumnStatistics<>(newStats.values(), type);
+  }
+
+  @JsonProperty("statistics")
+  @SuppressWarnings("unused") // used for serialization
+  private Collection<StatisticsHolder> getAll() {
+    return statistics.values();
+  }
+
+  @JsonProperty("type")
+  public TypeProtos.MinorType getComparatorType() {
+    return type;
+  }
+
+  public String jsonString() throws JsonProcessingException {
+    return OBJECT_WRITER.writeValueAsString(this);
+  }
+
+  public static ColumnStatistics of(String columnStatistics) throws IOException {
+    return OBJECT_READER.readValue(columnStatistics);
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatisticsKind.java
new file mode 100644
index 0000000..1b40b7e
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatisticsKind.java
@@ -0,0 +1,172 @@
+/*
+ * 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.statistics;
+
+import org.apache.drill.metastore.metadata.BaseMetadata;
+
+import java.util.List;
+
+/**
+ * Implementation of {@link CollectableColumnStatisticsKind} which contain base
+ * column statistics kinds with implemented {@code mergeStatistics()} method.
+ */
+public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements CollectableColumnStatisticsKind<T> {
+
+  /**
+   * Column statistics kind which represents nulls count for the specific column.
+   */
+  public static final ColumnStatisticsKind<Long> NULLS_COUNT =
+      new ColumnStatisticsKind<Long>(ExactStatisticsConstants.NULLS_COUNT, true) {
+        @Override
+        public Long mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+          long nullsCount = 0;
+          for (ColumnStatistics<?> statistics : statisticsList) {
+            Long statNullsCount = statistics.get(this);
+            if (statNullsCount == null || statNullsCount == Statistic.NO_COLUMN_STATS) {
+              return Statistic.NO_COLUMN_STATS;
+            } else {
+              nullsCount += statNullsCount;
+            }
+          }
+          return nullsCount;
+        }
+
+        @Override
+        public Long getFrom(ColumnStatistics metadata) {
+          Long rowCount = super.getFrom(metadata);
+          return rowCount != null ? rowCount : Statistic.NO_COLUMN_STATS;
+        }
+      };
+
+  /**
+   * Column statistics kind which represents min value of the specific column.
+   */
+  public static final ColumnStatisticsKind<Object> MIN_VALUE =
+      new ColumnStatisticsKind<Object>(ExactStatisticsConstants.MIN_VALUE, true) {
+        @Override
+        @SuppressWarnings("unchecked")
+        public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+          Object minValue = null;
+          for (ColumnStatistics statistics : statisticsList) {
+            Object statMinValue = getValueStatistic(statistics);
+            if (statMinValue != null && (statistics.getValueComparator().compare(minValue, statMinValue) > 0 || minValue == null)) {
+              minValue = statMinValue;
+            }
+          }
+          return minValue;
+        }
+      };
+
+  /**
+   * Column statistics kind which represents max value of the specific column.
+   */
+  public static final ColumnStatisticsKind<Object> MAX_VALUE =
+      new ColumnStatisticsKind<Object>(ExactStatisticsConstants.MAX_VALUE, true) {
+        @Override
+        @SuppressWarnings("unchecked")
+        public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+          Object maxValue = null;
+          for (ColumnStatistics statistics : statisticsList) {
+            Object statMaxValue = getValueStatistic(statistics);
+            if (statMaxValue != null && statistics.getValueComparator().compare(maxValue, statMaxValue) < 0) {
+              maxValue = statMaxValue;
+            }
+          }
+          return maxValue;
+        }
+      };
+
+  /**
+   * Column statistics kind which represents number of non-null values for the specific column.
+   */
+  public static final ColumnStatisticsKind<Double> NON_NULL_COUNT =
+      new ColumnStatisticsKind<Double>(Statistic.NNROWCOUNT, false) {
+        @Override
+        public Double mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+          double nonNullRowCount = 0;
+          for (ColumnStatistics<?> statistics : statisticsList) {
+            Double nnRowCount = statistics.get(this);
+            if (nnRowCount != null) {
+              nonNullRowCount += nnRowCount;
+            }
+          }
+          return nonNullRowCount;
+        }
+      };
+
+  /**
+   * Column statistics kind which represents total row count for the specific column.
+   */
+  public static final ColumnStatisticsKind<Double> ROWCOUNT =
+      new ColumnStatisticsKind<Double>(Statistic.ROWCOUNT, false) {
+        @Override
+        public Double mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+          double rowCount = 0;
+          for (ColumnStatistics<?> statistics : statisticsList) {
+            Double count = getFrom(statistics);
+            if (count != null) {
+              rowCount += count;
+            }
+          }
+          return rowCount;
+        }
+      };
+
+  /**
+   * Column statistics kind which represents number of distinct values for the specific column.
+   */
+  public static final ColumnStatisticsKind<Double> NDV =
+    new ColumnStatisticsKind<>(Statistic.NDV, false);
+
+  /**
+   * Column statistics kind which is the width of the specific column.
+   */
+  public static final ColumnStatisticsKind AVG_WIDTH =
+      new ColumnStatisticsKind(Statistic.AVG_WIDTH, false);
+
+  /**
+   * Column statistics kind which is the histogram of the specific column.
+   */
+  public static final ColumnStatisticsKind<Histogram> HISTOGRAM =
+    new ColumnStatisticsKind<>("histogram", false);
+
+  ColumnStatisticsKind(String statisticKey, boolean exact) {
+    super(statisticKey, exact);
+  }
+
+  /**
+   * Returns value which corresponds to this statistic kind,
+   * obtained from specified {@link BaseMetadata}.
+   *
+   * @param metadata the source of statistic value
+   * @return value which corresponds to this statistic kind
+   */
+  public T getFrom(ColumnStatistics<?> metadata) {
+    return metadata.get(this);
+  }
+
+  @SuppressWarnings("unchecked")
+  public <V> V getValueStatistic(ColumnStatistics<V> metadata) {
+    return (V) metadata.get(this);
+  }
+
+  @Override
+  public T mergeStatistics(List<? extends ColumnStatistics> statistics) {
+    throw new UnsupportedOperationException("Cannot merge statistics for " + statisticKey);
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/exec/expr/ExactStatisticsConstants.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ExactStatisticsConstants.java
similarity index 95%
rename from metastore/metastore-api/src/main/java/org/apache/drill/exec/expr/ExactStatisticsConstants.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ExactStatisticsConstants.java
index 7800ff1..38174a9 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/exec/expr/ExactStatisticsConstants.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ExactStatisticsConstants.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.expr;
+package org.apache.drill.metastore.statistics;
 
 public interface ExactStatisticsConstants {
   String MIN_VALUE = "minValue";
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/Histogram.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/Histogram.java
similarity index 77%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/Histogram.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/Histogram.java
index 062a6b7..c703da1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/Histogram.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/Histogram.java
@@ -15,27 +15,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.planner.common;
+package org.apache.drill.metastore.statistics;
 
-
-import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import org.apache.calcite.rex.RexNode;
 
 /**
  * A column specific histogram
  */
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY,
+@JsonTypeInfo(
+  use = JsonTypeInfo.Id.NAME,
   property = "category")
-@JsonSubTypes({
-  @JsonSubTypes.Type(value = NumericEquiDepthHistogram.class, name="numeric-equi-depth")
-})
 public interface Histogram {
 
   /**
-   * For a filter condition, estimate the selectivity (matching rows/total rows) for this histogram
-   * @param filter
-   * @param totalRowCount
+   * For a filter condition, estimate the selectivity (matching rows/total rows) for this histogram.
+   *
    * @return estimated selectivity or NULL if it could not be estimated for any reason
    */
   Double estimatedSelectivity(final RexNode filter, final long totalRowCount, final long ndv);
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/Statistic.java
similarity index 96%
rename from metastore/metastore-api/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/Statistic.java
index 5794a13..2be6ac0 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/Statistic.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.impl.statistics;
+package org.apache.drill.metastore.statistics;
 
 /*
  * Base Statistics class - all statistics classes should extend this class
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java
new file mode 100644
index 0000000..a1c2734
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java
@@ -0,0 +1,76 @@
+/*
+ * 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.statistics;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.util.DefaultPrettyPrinter;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+
+import java.io.IOException;
+
+/**
+ * Class-holder for statistics kind and its value.
+ *
+ * @param <T> Type of statistics value
+ */
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class StatisticsHolder<T> {
+
+  static final ObjectWriter OBJECT_WRITER = new ObjectMapper().setDefaultPrettyPrinter(new DefaultPrettyPrinter()).writer();
+  private static final ObjectReader OBJECT_READER = new ObjectMapper().readerFor(StatisticsHolder.class);
+
+  private final T statisticsValue;
+  private final BaseStatisticsKind statisticsKind;
+
+  @JsonCreator
+  public StatisticsHolder(@JsonProperty("statisticsValue") T statisticsValue,
+                          @JsonProperty("statisticsKind") BaseStatisticsKind statisticsKind) {
+    this.statisticsValue = statisticsValue;
+    this.statisticsKind = statisticsKind;
+  }
+
+  public StatisticsHolder(T statisticsValue,
+                          StatisticsKind statisticsKind) {
+    this.statisticsValue = statisticsValue;
+    this.statisticsKind = (BaseStatisticsKind) statisticsKind;
+  }
+
+  @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS,
+                include = JsonTypeInfo.As.WRAPPER_OBJECT)
+  public T getStatisticsValue() {
+    return statisticsValue;
+  }
+
+  public StatisticsKind getStatisticsKind() {
+    return statisticsKind;
+  }
+
+  public static StatisticsHolder of(String serialized) throws IOException {
+    return OBJECT_READER.readValue(serialized);
+  }
+
+  public String jsonString() throws JsonProcessingException {
+    return OBJECT_WRITER.writeValueAsString(this);
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/StatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsKind.java
similarity index 73%
rename from metastore/metastore-api/src/main/java/org/apache/drill/metastore/StatisticsKind.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsKind.java
index 3be627a..b2c651b 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/StatisticsKind.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsKind.java
@@ -15,31 +15,29 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.metastore;
+package org.apache.drill.metastore.statistics;
+
+import com.fasterxml.jackson.annotation.JsonFormat;
+import com.fasterxml.jackson.annotation.JsonGetter;
+import com.fasterxml.jackson.annotation.JsonInclude;
 
 /**
  * Class represents kind of statistics or metadata, for example it may be min value for column,
  * or row count for table.
  */
-public interface StatisticsKind {
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+@JsonFormat(shape = JsonFormat.Shape.OBJECT)
+public interface StatisticsKind<T> {
 
   /**
    * Returns name which corresponds to the statistics.
    *
    * @return statistic kind name
    */
+  @JsonGetter(value = "name")
   String getName();
 
-  /**
-   * Returns true if current statistics value has the same type as column values,
-   * for example it may be min value etc.
-   *
-   * @return true if current statistics value has the same type as column values
-   */
-  default boolean isValueStatistic() {
-    return false;
-  }
-
+  @JsonGetter(value = "exact")
   default boolean isExact() {
     return false;
   }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/TableStatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/TableStatisticsKind.java
new file mode 100644
index 0000000..34effa8
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/TableStatisticsKind.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.statistics;
+
+import org.apache.drill.metastore.metadata.Metadata;
+
+import java.util.Collection;
+
+/**
+ * Implementation of {@link CollectableColumnStatisticsKind} which contain base
+ * table statistics kinds with implemented {@code mergeStatistics()} method.
+ */
+public class TableStatisticsKind<T> extends BaseStatisticsKind<T> implements CollectableTableStatisticsKind<T> {
+
+  /**
+   * Table statistics kind which represents row count for the specific table.
+   */
+  public static final TableStatisticsKind<Long> ROW_COUNT =
+      new TableStatisticsKind<Long>(ExactStatisticsConstants.ROW_COUNT, true) {
+        @Override
+        public Long mergeStatistics(Collection<? extends Metadata> statistics) {
+          long rowCount = 0;
+          for (Metadata statistic : statistics) {
+            Long statRowCount = getValue(statistic);
+            if (statRowCount == Statistic.NO_COLUMN_STATS) {
+              rowCount = Statistic.NO_COLUMN_STATS;
+              break;
+            } else {
+              rowCount += statRowCount;
+            }
+          }
+          return rowCount;
+        }
+
+        @Override
+        public Long getValue(Metadata metadata) {
+          Long rowCount = super.getValue(metadata);
+          return rowCount != null ? rowCount : Statistic.NO_COLUMN_STATS;
+        }
+      };
+
+  /**
+   * Table statistics kind which represents estimated row count for the specific table.
+   */
+  public static final TableStatisticsKind<Double> EST_ROW_COUNT =
+      new TableStatisticsKind<Double>(Statistic.ROWCOUNT, false) {
+        @Override
+        public Double mergeStatistics(Collection<? extends Metadata> statisticsList) {
+          double rowCount = 0;
+          for (Metadata statistics : statisticsList) {
+            Double statRowCount = getValue(statistics);
+            if (statRowCount != null) {
+              rowCount += statRowCount;
+            }
+          }
+          return rowCount;
+        }
+      };
+
+  /**
+   * Table statistics kind which represents estimated row count for the specific table.
+   */
+  public static final TableStatisticsKind<Boolean> HAS_DESCRIPTIVE_STATISTICS =
+      new TableStatisticsKind<Boolean>("hasDescriptiveStatistics", false) {
+        @Override
+        public Boolean mergeStatistics(Collection<? extends Metadata> statisticsList) {
+          for (Metadata statistics : statisticsList) {
+            Boolean hasDescriptiveStatistics = statistics.getStatistic(this);
+            if (hasDescriptiveStatistics == null || !hasDescriptiveStatistics) {
+              return false;
+            }
+          }
+          return Boolean.TRUE;
+        }
+
+        @Override
+        public Boolean getValue(Metadata metadata) {
+          return Boolean.TRUE.equals(metadata.getStatistic(this));
+        }
+      };
+
+  public TableStatisticsKind(String statisticKey, boolean exact) {
+    super(statisticKey, exact);
+  }
+
+  @Override
+  public T mergeStatistics(Collection<? extends Metadata> statistics) {
+    throw new UnsupportedOperationException("Cannot merge statistics for " + statisticKey);
+  }
+
+  /**
+   * Returns value which corresponds to this statistic kind,
+   * obtained from specified {@link Metadata}.
+   *
+   * @param metadata the source of statistic value
+   * @return value which corresponds to this statistic kind
+   */
+  public T getValue(Metadata metadata) {
+    return metadata.getStatistic(this);
+  }
+}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/exec/record/metadata/SchemaPathUtils.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/SchemaPathUtils.java
similarity index 92%
rename from metastore/metastore-api/src/main/java/org/apache/drill/exec/record/metadata/SchemaPathUtils.java
rename to metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/SchemaPathUtils.java
index 5c39701..5d66035 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/exec/record/metadata/SchemaPathUtils.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/SchemaPathUtils.java
@@ -15,10 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.record.metadata;
+package org.apache.drill.metastore.util;
 
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 public class SchemaPathUtils {
 
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/TableMetadataUtils.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/TableMetadataUtils.java
new file mode 100644
index 0000000..1aac570
--- /dev/null
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/TableMetadataUtils.java
@@ -0,0 +1,135 @@
+/*
+ * 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.util;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.metastore.metadata.BaseMetadata;
+import org.apache.drill.metastore.metadata.TableMetadata;
+import org.apache.drill.metastore.statistics.CollectableColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.shaded.guava.com.google.common.primitives.UnsignedBytes;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class TableMetadataUtils {
+
+  /**
+   * Returns {@link Comparator} instance considering specified {@code type}.
+   *
+   * @param type type of the column
+   * @return {@link Comparator} instance
+   */
+  public static Comparator getComparator(TypeProtos.MinorType type) {
+    switch (type) {
+      case INTERVALDAY:
+      case INTERVAL:
+      case INTERVALYEAR:
+        return Comparator.nullsFirst(UnsignedBytes.lexicographicalComparator());
+      case UINT1:
+        return Comparator.nullsFirst(UnsignedBytes::compare);
+      case UINT2:
+      case UINT4:
+        return Comparator.nullsFirst(Integer::compareUnsigned);
+      case UINT8:
+        return Comparator.nullsFirst(Long::compareUnsigned);
+      default:
+        return getNaturalNullsFirstComparator();
+    }
+  }
+
+  /**
+   * Returns "natural order" comparator which threads nulls as min values.
+   *
+   * @param <T> type to compare
+   * @return "natural order" comparator
+   */
+  public static <T extends Comparable<T>> Comparator<T> getNaturalNullsFirstComparator() {
+    return Comparator.nullsFirst(Comparator.naturalOrder());
+  }
+
+  /**
+   * Merges list of specified metadata into the map of {@link ColumnStatistics} with columns as keys.
+   *
+   * @param <T>                 type of metadata to collect
+   * @param metadataList        list of metadata to be merged
+   * @param columns             set of columns whose statistics should be merged
+   * @param statisticsToCollect kinds of statistics that should be collected
+   * @return list of merged metadata
+   */
+  public static <T extends BaseMetadata> Map<SchemaPath, ColumnStatistics> mergeColumnsStatistics(
+      Collection<T> metadataList, Set<SchemaPath> columns, List<CollectableColumnStatisticsKind> statisticsToCollect) {
+    Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
+
+    for (SchemaPath column : columns) {
+      List<ColumnStatistics> statisticsList = new ArrayList<>();
+      for (T metadata : metadataList) {
+        ColumnStatistics statistics = metadata.getColumnsStatistics().get(column);
+        if (statistics == null) {
+          // schema change happened, set statistics which represents all nulls
+          statistics = new ColumnStatistics(
+              Collections.singletonList(
+                  new StatisticsHolder<>(TableStatisticsKind.ROW_COUNT.getValue(metadata), ColumnStatisticsKind.NULLS_COUNT)));
+        }
+        statisticsList.add(statistics);
+      }
+      List<StatisticsHolder> statisticsHolders = new ArrayList<>();
+      for (CollectableColumnStatisticsKind statisticsKind : statisticsToCollect) {
+        Object mergedStatistic = statisticsKind.mergeStatistics(statisticsList);
+        statisticsHolders.add(new StatisticsHolder<>(mergedStatistic, statisticsKind));
+      }
+      Iterator<ColumnStatistics> iterator = statisticsList.iterator();
+      // Use INT if statistics wasn't provided
+      TypeProtos.MinorType comparatorType = iterator.hasNext() ? iterator.next().getComparatorType() : TypeProtos.MinorType.INT;
+      columnsStatistics.put(column, new ColumnStatistics<>(statisticsHolders, comparatorType));
+    }
+    return columnsStatistics;
+  }
+
+  /**
+   * Updates row count and column nulls count for specified table metadata and returns new {@link TableMetadata} instance with updated statistics.
+   *
+   * @param tableMetadata table statistics to update
+   * @param statistics    list of statistics whose row count should be considered
+   * @return new {@link TableMetadata} instance with updated statistics
+   */
+  public static TableMetadata updateRowCount(TableMetadata tableMetadata, Collection<? extends BaseMetadata> statistics) {
+    List<StatisticsHolder> newStats = new ArrayList<>();
+
+    newStats.add(new StatisticsHolder<>(TableStatisticsKind.ROW_COUNT.mergeStatistics(statistics), TableStatisticsKind.ROW_COUNT));
+
+    Set<SchemaPath> columns = tableMetadata.getColumnsStatistics().keySet();
+
+    Map<SchemaPath, ColumnStatistics> columnsStatistics =
+        mergeColumnsStatistics(statistics, columns,
+            Collections.singletonList(ColumnStatisticsKind.NULLS_COUNT));
+
+    return tableMetadata.cloneWithStats(columnsStatistics, newStats);
+  }
+}
diff --git a/metastore/metastore-api/src/test/java/org/apache/drill/metastore/metadata/MetadataSerDeTest.java b/metastore/metastore-api/src/test/java/org/apache/drill/metastore/metadata/MetadataSerDeTest.java
new file mode 100644
index 0000000..34852d2
--- /dev/null
+++ b/metastore/metastore-api/src/test/java/org/apache/drill/metastore/metadata/MetadataSerDeTest.java
@@ -0,0 +1,152 @@
+/*
+ * 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.metadata;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.metastore.statistics.BaseStatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class MetadataSerDeTest {
+
+  @Test
+  public void testStatisticsHolderSerialization() throws Exception {
+    checkStatisticsHolderSerialization(1, TableStatisticsKind.ROW_COUNT,
+        "{\"statisticsValue\":1,\"statisticsKind\":{\"exact\":true,\"name\":\"rowCount\"}}");
+    checkStatisticsHolderSerialization(1.234, TableStatisticsKind.EST_ROW_COUNT,
+        "{\"statisticsValue\":1.234,\"statisticsKind\":{\"name\":\"rowcount\"}}");
+    checkStatisticsHolderSerialization(true, TableStatisticsKind.HAS_DESCRIPTIVE_STATISTICS,
+        "{\"statisticsValue\":true,\"statisticsKind\":{\"name\":\"hasDescriptiveStatistics\"}}");
+    checkStatisticsHolderSerialization(null, TableStatisticsKind.EST_ROW_COUNT,
+        "{\"statisticsKind\":{\"name\":\"rowcount\"}}");
+    checkStatisticsHolderSerialization("aAaAaAa", ColumnStatisticsKind.MIN_VALUE,
+        "{\"statisticsValue\":\"aAaAaAa\",\"statisticsKind\":{\"exact\":true,\"name\":\"minValue\"}}");
+    checkStatisticsHolderSerialization(new BigDecimal("123.321"), ColumnStatisticsKind.MIN_VALUE,
+        "{\"statisticsValue\":{\"java.math.BigDecimal\":123.321},\"statisticsKind\":{\"exact\":true,\"name\":\"minValue\"}}");
+    checkStatisticsHolderSerialization(new byte[]{1, 1, 2, 3, 5, 8}, ColumnStatisticsKind.MAX_VALUE,
+        "{\"statisticsValue\":{\"[B\":\"AQECAwUI\"},\"statisticsKind\":{\"exact\":true,\"name\":\"maxValue\"}}");
+  }
+
+  @Test
+  public void testStatisticsHolderDeserialization() throws Exception {
+    checkStatisticsHolderDeserialization(1, TableStatisticsKind.ROW_COUNT);
+    checkStatisticsHolderDeserialization(1.234, TableStatisticsKind.EST_ROW_COUNT);
+    checkStatisticsHolderDeserialization(true, TableStatisticsKind.HAS_DESCRIPTIVE_STATISTICS);
+    checkStatisticsHolderDeserialization(null, TableStatisticsKind.EST_ROW_COUNT);
+    checkStatisticsHolderDeserialization("aAaAaAa", ColumnStatisticsKind.MIN_VALUE);
+    checkStatisticsHolderDeserialization(new BigDecimal("123.321"), ColumnStatisticsKind.MIN_VALUE);
+    checkStatisticsHolderDeserialization(new byte[]{1, 1, 2, 3, 5, 8}, ColumnStatisticsKind.MAX_VALUE);
+  }
+
+  @Test
+  public void testColumnStatisticsSerialization() throws Exception {
+    List<StatisticsHolder> statistics = Arrays.asList(
+        new StatisticsHolder<>("aaa", ColumnStatisticsKind.MIN_VALUE),
+        new StatisticsHolder<>("zzz", ColumnStatisticsKind.MAX_VALUE),
+        new StatisticsHolder<>(3, ColumnStatisticsKind.NULLS_COUNT),
+        new StatisticsHolder<>(2.1, ColumnStatisticsKind.NDV));
+    ColumnStatistics<String> columnStatistics = new ColumnStatistics<>(statistics, TypeProtos.MinorType.VARCHAR);
+    String serializedColumnStatistics = columnStatistics.jsonString();
+
+    String expected =
+        "{" +
+            "\"statistics\":[" +
+                "{\"statisticsValue\":2.1,\"statisticsKind\":{\"name\":\"approx_count_distinct\"}}," +
+                "{\"statisticsValue\":\"aaa\",\"statisticsKind\":{\"exact\":true,\"name\":\"minValue\"}}," +
+                "{\"statisticsValue\":3,\"statisticsKind\":{\"exact\":true,\"name\":\"nullsCount\"}}," +
+                "{\"statisticsValue\":\"zzz\",\"statisticsKind\":{\"exact\":true,\"name\":\"maxValue\"}}]," +
+            "\"type\":\"VARCHAR\"" +
+        "}";
+
+    assertEquals("StatisticsHolder was incorrectly serialized",
+        expected,
+        serializedColumnStatistics);
+  }
+
+  @Test
+  public void testColumnStatisticsDeserialization() throws Exception {
+    List<StatisticsHolder> statistics = Arrays.asList(
+        new StatisticsHolder<>("aaa", ColumnStatisticsKind.MIN_VALUE),
+        new StatisticsHolder<>("zzz", ColumnStatisticsKind.MAX_VALUE),
+        new StatisticsHolder<>(3, ColumnStatisticsKind.NULLS_COUNT),
+        new StatisticsHolder<>(2.1, ColumnStatisticsKind.NDV));
+    ColumnStatistics<String> columnStatistics = new ColumnStatistics<>(statistics, TypeProtos.MinorType.VARCHAR);
+    String serializedColumnStatistics = columnStatistics.jsonString();
+
+    ColumnStatistics deserialized = ColumnStatistics.of(serializedColumnStatistics);
+
+    assertEquals("Type was incorrectly deserialized",
+        columnStatistics.getComparatorType(),
+        deserialized.getComparatorType());
+
+    for (StatisticsHolder statistic : statistics) {
+      assertEquals("Statistics kind was incorrectly deserialized",
+          statistic.getStatisticsKind().isExact(),
+          deserialized.containsExact(statistic.getStatisticsKind()));
+      assertEquals("Statistics value was incorrectly deserialized",
+          statistic.getStatisticsValue(),
+          deserialized.get(statistic.getStatisticsKind()));
+    }
+
+  }
+
+  private <T> void checkStatisticsHolderSerialization(T statisticsValue,
+      BaseStatisticsKind statisticsKind, String expectedString) throws JsonProcessingException {
+    StatisticsHolder<T> statisticsHolder =
+        new StatisticsHolder<>(statisticsValue, statisticsKind);
+    String serializedStatisticsHolder = statisticsHolder.jsonString();
+
+    assertEquals("StatisticsHolder was incorrectly serialized",
+        expectedString,
+        serializedStatisticsHolder);
+  }
+
+  private <T> void checkStatisticsHolderDeserialization(T statisticsValue,
+      BaseStatisticsKind statisticsKind) throws Exception {
+    StatisticsHolder<T> rowCount =
+        new StatisticsHolder<>(statisticsValue, statisticsKind);
+    StatisticsHolder deserializedRowCount = StatisticsHolder.of(rowCount.jsonString());
+
+    assertTrue("Statistics value was incorrectly deserialized",
+        Objects.deepEquals(rowCount.getStatisticsValue(), deserializedRowCount.getStatisticsValue()));
+
+    assertStatisticsKindsEquals(rowCount, deserializedRowCount);
+  }
+
+  private <T> void assertStatisticsKindsEquals(StatisticsHolder<T> expected, StatisticsHolder actual) {
+    assertEquals("isExact statistics kind was incorrectly deserialized",
+        expected.getStatisticsKind().isExact(),
+        actual.getStatisticsKind().isExact());
+
+    assertEquals("getName statistics kind was incorrectly deserialized",
+        expected.getStatisticsKind().getName(),
+        actual.getStatisticsKind().getName());
+  }
+}
diff --git a/metastore/pom.xml b/metastore/pom.xml
index 2558e11..3e5b2da 100644
--- a/metastore/pom.xml
+++ b/metastore/pom.xml
@@ -70,6 +70,5 @@
 
   <modules>
     <module>metastore-api</module>
-    <module>file-metadata</module>
   </modules>
 </project>