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/11/14 17:18:58 UTC

[drill] 05/05: DRILL-7273: Introduce operators for handling metadata

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

commit 7ab4c3739f34e9e96cbdfa883325a10311a8ef02
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Wed Jun 26 18:11:59 2019 +0300

    DRILL-7273: Introduce operators for handling metadata
    
    closes #1886
---
 .../java/org/apache/drill/common/types/Types.java  |    2 +-
 .../common/util/function/CheckedSupplier.java      |   26 +-
 .../native/client/src/protobuf/UserBitShared.pb.cc |   17 +-
 .../native/client/src/protobuf/UserBitShared.pb.h  |    6 +-
 docs/dev/MetastoreAnalyze.md                       |  119 +
 .../java-exec/src/main/codegen/data/AggrTypes1.tdd |    2 +
 .../src/main/codegen/includes/parserImpls.ftl      |  131 +-
 .../main/codegen/templates/AggrTypeFunctions1.java |    4 +
 .../templates/JsonBaseStatisticsRecordWriter.java  |    2 +-
 ...dWriter.java => StatisticsRecordCollector.java} |   62 +-
 .../codegen/templates/StatisticsRecordWriter.java  |   24 +-
 .../templates/StatisticsRecordWriterImpl.java      |   10 +-
 .../java/org/apache/drill/exec/ExecConstants.java  |   41 +-
 .../drill/exec/exception/MetadataException.java    |   80 +
 .../expr/fn/DrillComplexWriterAggFuncHolder.java   |    2 +-
 .../drill/exec/expr/fn/FunctionConverter.java      |   12 +-
 .../exec/expr/fn/impl/AggregateErrorFunctions.java |    4 +-
 .../expr/fn/impl/CollectListMapsAggFunction.java   |   77 +
 .../fn/impl/CollectToListVarcharAggFunction.java   |   68 +
 .../exec/expr/fn/impl/ParentPathFunction.java      |   55 +
 .../drill/exec/expr/fn/impl/SchemaFunctions.java   |  251 ++
 .../MetastoreMetadataProviderManager.java          |  123 +
 .../MetastoreParquetTableMetadataProvider.java     |  414 +++
 .../metastore/SimpleFileTableMetadataProvider.java |    5 +
 .../exec/metastore/analyze/AnalyzeColumnUtils.java |  130 +
 .../metastore/analyze/AnalyzeFileInfoProvider.java |   91 +
 .../metastore/analyze/AnalyzeInfoProvider.java     |  118 +
 .../analyze/AnalyzeParquetInfoProvider.java        |   73 +
 .../analyze/FileMetadataInfoCollector.java         |  411 +++
 .../analyze/MetadataAggregateContext.java          |  115 +
 .../analyze/MetadataControllerContext.java         |  176 ++
 .../metastore/analyze/MetadataHandlerContext.java  |  132 +
 .../metastore/analyze/MetadataIdentifierUtils.java |   88 +
 .../metastore/analyze/MetadataInfoCollector.java   |   82 +
 .../analyze/MetastoreAnalyzeConstants.java         |   20 +-
 .../org/apache/drill/exec/ops/QueryContext.java    |   12 +-
 .../org/apache/drill/exec/opt/BasicOptimizer.java  |    2 +-
 .../exec/physical/base/AbstractGroupScan.java      |   11 +
 .../base/AbstractGroupScanWithMetadata.java        |   95 +-
 .../apache/drill/exec/physical/base/GroupScan.java |   17 +
 .../{UnpivotMaps.java => MetadataAggPOP.java}      |   41 +-
 .../physical/config/MetadataControllerPOP.java     |   83 +
 .../{UnpivotMaps.java => MetadataHandlerPOP.java}  |   39 +-
 .../exec/physical/config/StatisticsAggregate.java  |    2 +-
 .../exec/physical/config/StreamingAggregate.java   |    9 +-
 .../drill/exec/physical/config/UnpivotMaps.java    |   11 +-
 .../physical/impl/aggregate/StreamingAggBatch.java |   30 +-
 .../physical/impl/join/HashJoinProbeTemplate.java  |    3 +-
 .../exec/physical/impl/join/JoinTemplate.java      |    3 +-
 .../physical/impl/metadata/MetadataAggBatch.java   |  288 ++
 .../impl/metadata/MetadataAggBatchCreator.java     |   38 +
 .../impl/metadata/MetadataControllerBatch.java     |  760 ++++++
 .../metadata/MetadataControllerBatchCreator.java   |   39 +
 .../impl/metadata/MetadataHandlerBatch.java        |  498 ++++
 .../impl/metadata/MetadataHandlerBatchCreator.java |   38 +
 .../exec/planner/DFSDirPartitionLocation.java      |    1 -
 .../apache/drill/exec/planner/PlannerPhase.java    |    7 +
 .../drill/exec/planner/common/DrillStatsTable.java |   49 +-
 .../logical/ConvertCountToDirectScanRule.java      |    5 +-
 .../drill/exec/planner/logical/DrillScanRel.java   |    4 +-
 .../drill/exec/planner/logical/MetadataAggRel.java |   74 +
 .../planner/logical/MetadataControllerRel.java     |   85 +
 .../exec/planner/logical/MetadataHandlerRel.java   |   73 +
 .../exec/planner/logical/ScanFieldDeterminer.java  |   44 +-
 .../physical/ConvertCountToDirectScanPrule.java    |    4 +-
 .../exec/planner/physical/MetadataAggPrel.java     |   82 +
 .../exec/planner/physical/MetadataAggPrule.java    |   60 +
 .../planner/physical/MetadataControllerPrel.java   |   95 +
 .../planner/physical/MetadataControllerPrule.java  |   46 +
 .../exec/planner/physical/MetadataHandlerPrel.java |   82 +
 .../planner/physical/MetadataHandlerPrule.java     |   45 +
 .../drill/exec/planner/physical/PrelUtil.java      |    2 +-
 .../drill/exec/planner/physical/StreamAggPrel.java |    2 +-
 .../physical/visitor/StarColumnConverter.java      |    9 +-
 .../drill/exec/planner/sql/DrillSqlWorker.java     |   78 +-
 .../drill/exec/planner/sql/SqlConverter.java       |   43 +-
 .../planner/sql/handlers/AnalyzeTableHandler.java  |   44 +-
 .../sql/handlers/MetastoreAnalyzeTableHandler.java |  536 ++++
 .../MetastoreDropTableMetadataHandler.java         |   97 +
 .../sql/handlers/RefreshMetadataHandler.java       |    8 +-
 .../sql/parser/CompoundIdentifierConverter.java    |    2 +
 .../exec/planner/sql/parser/SqlAnalyzeTable.java   |    2 +-
 .../exec/planner/sql/parser/SqlDropTable.java      |    4 +-
 ...SqlDropTable.java => SqlDropTableMetadata.java} |   88 +-
 ...yzeTable.java => SqlMetastoreAnalyzeTable.java} |  125 +-
 .../exec/planner/types/DrillRelDataTypeSystem.java |   21 +
 .../exec/record/AbstractBinaryRecordBatch.java     |    2 -
 .../drill/exec/record/AbstractRecordBatch.java     |    8 +-
 .../exec/record/AbstractSingleRecordBatch.java     |    4 +-
 .../exec/record/AbstractUnaryRecordBatch.java      |    9 -
 .../drill/exec/record/RecordBatchLoader.java       |    1 +
 .../org/apache/drill/exec/record/SchemaUtil.java   |   32 +
 .../exec/server/options/SystemOptionManager.java   |    6 +-
 .../apache/drill/exec/store/AbstractSchema.java    |    4 +-
 .../apache/drill/exec/store/ColumnExplorer.java    |  122 +-
 .../exec/store/dfs/WorkspaceSchemaFactory.java     |   36 +-
 .../exec/store/direct/MetadataDirectGroupScan.java |   19 +-
 .../easy/json/JsonStatisticsRecordWriter.java      |  341 +--
 .../store/easy/json/StatisticsCollectorImpl.java   |  387 +++
 .../store/parquet/AbstractParquetGroupScan.java    |   25 +-
 .../parquet/AbstractParquetScanBatchCreator.java   |   54 +-
 .../store/parquet/BaseParquetMetadataProvider.java |    6 +-
 .../exec/store/parquet/FilterEvaluatorUtils.java   |    9 +-
 .../ParquetFileTableMetadataProviderBuilder.java   |    4 +
 .../drill/exec/store/parquet/ParquetGroupScan.java |   58 +-
 .../parquet/ParquetTableMetadataProviderImpl.java  |    2 +-
 .../store/parquet/ParquetTableMetadataUtils.java   |  107 +-
 .../parquet/metadata/FileMetadataCollector.java    |    7 +-
 .../exec/store/parquet/metadata/Metadata.java      |   47 +-
 .../exec/store/parquet/metadata/Metadata_V4.java   |    5 +-
 .../drill/exec/work/fragment/FragmentExecutor.java |    2 +-
 .../java-exec/src/main/resources/drill-module.conf |   12 +-
 .../java/org/apache/drill/TestFunctionsQuery.java  |   12 +
 .../drill/exec/fn/impl/TestAggregateFunctions.java |  261 +-
 .../physical/impl/agg/TestAggWithAnyValue.java     |    2 +-
 .../impl/agg/TestStreamingAggEmitOutcome.java      |   85 +-
 .../physical/unit/BasicPhysicalOpUnitTest.java     |    2 +-
 .../exec/physical/unit/TestNullInputMiniPlan.java  |    2 +-
 .../apache/drill/exec/record/vector/TestLoad.java  |   57 +
 .../drill/exec/sql/TestMetastoreCommands.java      | 2824 ++++++++++++++++++++
 .../store/parquet/TestParquetFilterPushDown.java   |    3 +-
 .../TestPushDownAndPruningWithItemStar.java        |   68 +-
 .../org/apache/drill/test/OperatorFixture.java     |    3 +-
 .../apache/drill/test/PhysicalOpUnitTestBase.java  |    3 +-
 .../java/org/apache/drill/test/PrintingUtils.java  |    9 +-
 .../java/org/apache/drill/test/QueryBuilder.java   |   26 +
 .../record/metadata/AbstractColumnMetadata.java    |   24 +-
 .../record/metadata/PrimitiveColumnMetadata.java   |   32 +-
 .../drill/exec/vector/UntypedNullVector.java       |    2 +-
 .../drill/exec/vector/accessor/ColumnWriter.java   |    2 +-
 .../accessor/writer/OffsetVectorWriterImpl.java    |    4 +-
 .../vector/complex/impl/RepeatedMapReaderImpl.java |    6 +
 .../vector/complex/impl/SingleMapReaderImpl.java   |    8 +-
 .../drill/common/expression/FieldReference.java    |   37 +-
 .../common/logical/data/MetadataAggregate.java     |   28 +-
 .../common/logical/data/MetadataController.java    |   34 +-
 .../drill/common/logical/data/MetadataHandler.java |   28 +-
 .../components/tables/TestBasicRequests.java       |  165 ++
 metastore/metastore-api/pom.xml                    |    5 +
 .../components/tables/BasicTablesRequests.java     |  179 ++
 .../components/tables/MetastoreTableInfo.java      |   12 +-
 .../drill/metastore/metadata/BaseMetadata.java     |   26 +
 .../metastore/metadata/BaseTableMetadata.java      |   59 +-
 .../drill/metastore/metadata/FileMetadata.java     |   45 +
 .../drill/metastore/metadata/MetadataInfo.java     |   11 +
 .../drill/metastore/metadata/MetadataType.java     |   44 +-
 .../metastore/metadata/PartitionMetadata.java      |   51 +
 .../drill/metastore/metadata/RowGroupMetadata.java |   51 +
 .../drill/metastore/metadata/SegmentMetadata.java  |   54 +
 .../apache/drill/metastore/metadata/TableInfo.java |   13 +
 .../metastore/metadata/TableMetadataProvider.java  |    8 +
 .../metastore/statistics/BaseStatisticsKind.java   |   29 +
 .../metastore/statistics/ColumnStatistics.java     |   51 +-
 .../metastore/statistics/StatisticsHolder.java     |   44 +-
 .../metastore/statistics/TableStatisticsKind.java  |   24 +
 .../org/apache/drill/exec/proto/UserBitShared.java |   31 +-
 protocol/src/main/protobuf/UserBitShared.proto     |    2 +
 157 files changed, 11331 insertions(+), 1253 deletions(-)

diff --git a/common/src/main/java/org/apache/drill/common/types/Types.java b/common/src/main/java/org/apache/drill/common/types/Types.java
index 9179c05..57a752e 100644
--- a/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -609,7 +609,7 @@ public class Types {
   }
 
   public static MajorType overrideMode(final MajorType originalMajorType, final DataMode overrideMode) {
-    return withPrecisionAndScale(originalMajorType.getMinorType(), overrideMode, originalMajorType.getPrecision(), originalMajorType.getScale());
+    return originalMajorType.toBuilder().setMode(overrideMode).build();
   }
 
   public static MajorType getMajorTypeFromName(final String typeName) {
diff --git a/common/src/main/java/org/apache/drill/common/util/function/CheckedSupplier.java b/common/src/main/java/org/apache/drill/common/util/function/CheckedSupplier.java
index 6063383..45fecc7 100644
--- a/common/src/main/java/org/apache/drill/common/util/function/CheckedSupplier.java
+++ b/common/src/main/java/org/apache/drill/common/util/function/CheckedSupplier.java
@@ -17,6 +17,10 @@
  */
 package org.apache.drill.common.util.function;
 
+import java.util.function.Supplier;
+
+import static org.apache.drill.common.exceptions.ErrorHelper.sneakyThrow;
+
 /**
  * The java standard library does not provide a lambda function interface for functions that take no arguments,
  * but that throw an exception. So, we have to define our own here.
@@ -24,6 +28,24 @@ package org.apache.drill.common.util.function;
  * @param <E> The type of exception thrown by the lambda function.
  */
 @FunctionalInterface
-public interface CheckedSupplier<T, E extends Exception> {
-  T get() throws E;
+public interface CheckedSupplier<T, E extends Exception> extends Supplier<T> {
+
+  @Override
+  default T get() {
+    try {
+      return getAndThrow();
+    } catch (Throwable e) {
+      sneakyThrow(e);
+    }
+    // should never happen
+    throw new RuntimeException();
+  }
+
+  /**
+   * Gets a result.
+   *
+   * @return a result
+   * @throws E exception in case of errors
+   */
+  T getAndThrow() throws E;
 }
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
index 342c7f7..5a51bb8 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
@@ -1034,7 +1034,7 @@ void AddDescriptorsImpl() {
       "ATEMENT\020\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000"
       "\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014"
       "\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022"
-      "\032\n\026CANCELLATION_REQUESTED\020\006*\242\n\n\020CoreOper"
+      "\032\n\026CANCELLATION_REQUESTED\020\006*\321\n\n\020CoreOper"
       "atorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST"
       "_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020"
       "\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH"
@@ -1067,14 +1067,15 @@ void AddDescriptorsImpl() {
       "\022\023\n\017SYSLOG_SUB_SCAN\020:\022\030\n\024STATISTICS_AGGR"
       "EGATE\020;\022\020\n\014UNPIVOT_MAPS\020<\022\024\n\020STATISTICS_"
       "MERGE\020=\022\021\n\rLTSV_SUB_SCAN\020>\022\022\n\016EXCEL_SUB_"
-      "SCAN\020@\022\020\n\014SHP_SUB_SCAN\020A*g\n\nSaslStatus\022\020"
-      "\n\014SASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL"
-      "_IN_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL"
-      "_FAILED\020\004B.\n\033org.apache.drill.exec.proto"
-      "B\rUserBitSharedH\001"
+      "SCAN\020@\022\020\n\014SHP_SUB_SCAN\020A\022\024\n\020METADATA_HAN"
+      "DLER\020B\022\027\n\023METADATA_CONTROLLER\020C*g\n\nSaslS"
+      "tatus\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022"
+      "\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022"
+      "\017\n\013SASL_FAILED\020\004B.\n\033org.apache.drill.exe"
+      "c.protoB\rUserBitSharedH\001"
   };
   ::google::protobuf::DescriptorPool::InternalAddGeneratedFile(
-      descriptor, 5697);
+      descriptor, 5744);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "UserBitShared.proto", &protobuf_RegisterTypes);
   ::protobuf_Types_2eproto::AddDescriptors();
@@ -1318,6 +1319,8 @@ bool CoreOperatorType_IsValid(int value) {
     case 62:
     case 64:
     case 65:
+    case 66:
+    case 67:
       return true;
     default:
       return false;
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h
index 1878d5f..aba2e75 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.h
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h
@@ -355,11 +355,13 @@ enum CoreOperatorType {
   STATISTICS_MERGE = 61,
   LTSV_SUB_SCAN = 62,
   EXCEL_SUB_SCAN = 64,
-  SHP_SUB_SCAN = 65
+  SHP_SUB_SCAN = 65,
+  METADATA_HANDLER = 66,
+  METADATA_CONTROLLER = 67
 };
 bool CoreOperatorType_IsValid(int value);
 const CoreOperatorType CoreOperatorType_MIN = SINGLE_SENDER;
-const CoreOperatorType CoreOperatorType_MAX = SHP_SUB_SCAN;
+const CoreOperatorType CoreOperatorType_MAX = METADATA_CONTROLLER;
 const int CoreOperatorType_ARRAYSIZE = CoreOperatorType_MAX + 1;
 
 const ::google::protobuf::EnumDescriptor* CoreOperatorType_descriptor();
diff --git a/docs/dev/MetastoreAnalyze.md b/docs/dev/MetastoreAnalyze.md
new file mode 100644
index 0000000..ec27c50
--- /dev/null
+++ b/docs/dev/MetastoreAnalyze.md
@@ -0,0 +1,119 @@
+# Metastore ANALYZE commands
+
+Drill provides the functionality to collect, use and store table metadata into Drill Metastore.
+
+Set `metastore.enabled` option to true to enable Metastore usage.
+
+To collect table metadata, the following command should be used:
+
+```
+ANALYZE TABLE [table_name] [COLUMNS (col1, col2, ...)]
+REFRESH METADATA [partition LEVEL]
+{COMPUTE | ESTIMATE} | STATISTICS [(column1, column2, ...)]
+[ SAMPLE numeric PERCENT ]
+```
+
+For the case when this command is executed for the first time, whole table metadata will be collected and stored into
+ Metastore.
+If analyze was already executed for the table, and table data wasn't changed, all further analyze commands wouldn't
+ trigger table analyzing and message that table metadata is up to date will be returned.
+
+# Incremental analyze
+
+For the case when some table data was updated, Drill will try to execute incremental analyze - calculate metadata only
+ for updated data and reuse required metadata from the Metastore.
+
+Incremental analyze wouldn't be produced for the following cases:
+ - list of interesting columns specified in analyze is not a subset of interesting columns from the previous analyze;
+ - specified metadata level differs from the metadata level in previous analyze.
+
+# Metadata usage
+
+Drill provides the ability to use metadata obtained from the Metastore at the planning stage to prune segments, files
+ and row groups.
+
+Tables metadata from the Metastore is exposed to `INFORMATION_SCHEMA` tables (if Metastore usage is enabled).
+
+The following tables are populated with table metadata from the Metastore:
+
+`TABLES` table has the following additional columns populated from the Metastore:
+ - `TABLE_SOURCE` - table data type: `PARQUET`, `CSV`, `JSON`
+ - `LOCATION` - table location: `/tmp/nation`
+ - `NUM_ROWS` - number of rows in a table if known, `null` if not known
+ - `LAST_MODIFIED_TIME` - table's last modification time
+
+`COLUMNS` table has the following additional columns populated from the Metastore:
+ - `COLUMN_DEFAULT` - column default value
+ - `COLUMN_FORMAT` - usually applicable for date time columns: `yyyy-MM-dd`
+ - `NUM_NULLS` - number of nulls in column values
+ - `MIN_VAL` - column min value in String representation: `aaa`
+ - `MAX_VAL` - column max value in String representation: `zzz`
+ - `NDV` - number of distinct values in column, expressed in Double
+ - `EST_NUM_NON_NULLS` - estimated number of non null values, expressed in Double
+ - `IS_NESTED` - if column is nested. Nested columns are extracted from columns with struct type.
+
+`PARTITIONS` table has the following additional columns populated from the Metastore:
+ - `TABLE_CATALOG` - table catalog (currently we have only one catalog): `DRILL`
+ - `TABLE_SCHEMA` - table schema: `dfs.tmp`
+ - `TABLE_NAME` - table name: `nation`
+ - `METADATA_KEY` - top level segment key, the same for all nested segments and partitions: `part_int=3`
+ - `METADATA_TYPE` - `SEGMENT` or `PARTITION`
+ - `METADATA_IDENTIFIER` - current metadata identifier: `part_int=3/part_varchar=g`
+ - `PARTITION_COLUMN` - partition column name: `part_varchar`
+ - `PARTITION_VALUE` - partition column value: `g`
+ - `LOCATION` - segment location, `null` for partitions: `/tmp/nation/part_int=3`
+ - `LAST_MODIFIED_TIME` - last modification time
+
+# Metastore related options
+
+ - `metastore.enabled` - enables Drill Metastore usage to be able to store table metadata during `ANALYZE TABLE` commands 
+execution and to be able to read table metadata during regular queries execution or when querying some `INFORMATION_SCHEMA` tables.
+ - `metastore.metadata.store.depth_level` - specifies maximum level depth for collecting metadata.
+ Possible values : `TABLE`, `SEGMENT`, `PARTITION`, `FILE`, `ROW_GROUP`, `ALL`.
+ - `metastore.metadata.use_schema` - enables schema usage, stored to the Metastore.
+ - `metastore.metadata.use_statistics` - enables statistics usage, stored in the Metastore, at the planning stage.
+ - `metastore.metadata.fallback_to_file_metadata` - allows using file metadata cache for the case when required metadata is absent in the Metastore.
+ - `metastore.retrieval.retry_attempts` - specifies the number of attempts for retrying query planning after detecting that query metadata is changed. 
+ If the number of retries was exceeded, query will be planned without metadata information from the Metastore.
+ 
+# Analyze operators description
+
+Entry point for `ANALYZE` command is `MetastoreAnalyzeTableHandler` class. It creates plan which includes some
+Metastore specific operators for collecting metadata.
+
+`MetastoreAnalyzeTableHandler` uses `AnalyzeInfoProvider` for providing the information
+required for building a suitable plan for collecting metadata.
+Each group scan should provide corresponding `AnalyzeInfoProvider` implementation class.
+
+Analyze command specific operators:
+ - `MetadataAggBatch` - operator which adds aggregate calls for all incoming table columns to calculate required
+  metadata and produces aggregations. If aggregation is performed on top of another aggregation,
+  required aggregate calls for merging metadata will be added.
+ - `MetadataHandlerBatch` - operator responsible for handling metadata returned by incoming aggregate operators and
+  fetching required metadata form the Metastore to produce further aggregations.
+ - `MetadataControllerBatch` - responsible for converting obtained metadata, fetching absent metadata from the Metastore
+  and storing resulting metadata into the Metastore.
+
+`MetastoreAnalyzeTableHandler` forms plan  depending on segments count in the following form:
+
+```
+MetadataControllerBatch
+  ...
+    MetadataHandlerBatch
+      MetadataAggBatch(dir0, ...)
+        MetadataHandlerBatch
+          MetadataAggBatch(dir0, dir1, ...)
+            MetadataHandlerBatch
+              MetadataAggBatch(dir0, dir1, fqn, ...)
+                Scan(DYNAMIC_STAR **, ANY fqn, ...)
+```
+
+The lowest `MetadataAggBatch` creates required aggregate calls for every (or interesting only) table columns
+and produces aggregations with grouping by segment columns that correspond to specific table level.
+`MetadataHandlerBatch` above it populates batch with additional information about metadata type and other info.
+`MetadataAggBatch` above merges metadata calculated before to obtain metadata for parent metadata levels and also stores incoming data to populate it to the Metastore later.
+
+`MetadataControllerBatch` obtains all calculated metadata, converts it to the suitable form and sends it to the Metastore.
+
+For the case of incremental analyze, `MetastoreAnalyzeTableHandler` creates Scan with updated files only
+and provides `MetadataHandlerBatch` with information about metadata which should be fetched from the Metastore, so existing actual metadata wouldn't be recalculated.
diff --git a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
index 7512129..d8b1cae 100644
--- a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
@@ -46,8 +46,10 @@
      ]
     },
     {className: "Max", funcName: "max", types: [
+      {inputType: "Bit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
       {inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
       {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
+      {inputType: "NullableBit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
       {inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
       {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
       {inputType: "Float4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
diff --git a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
index afea80b..ae52f1f 100644
--- a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
+++ b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
@@ -698,46 +698,129 @@ Pair<SqlNodeList, SqlNodeList> ParenthesizedCompoundIdentifierList() :
 </#if>
 
 /**
- * Parses a analyze statement.
- * ANALYZE TABLE table_name {COMPUTE | ESTIMATE} | STATISTICS
- *      [(column1, column2, ...)] [ SAMPLE numeric PERCENT ]
+ * Parses a analyze statements:
+ * <ul>
+ * <li>ANALYZE TABLE [table_name] [COLUMNS (col1, col2, ...)] REFRESH METADATA [partition LEVEL] {COMPUTE | ESTIMATE} | STATISTICS [(column1, column2, ...)] [ SAMPLE numeric PERCENT ]
+ * <li>ANALYZE TABLE [table_name] DROP [METADATA|STATISTICS] [IF EXISTS]
+ * <li>ANALYZE TABLE [table_name] {COMPUTE | ESTIMATE} | STATISTICS [(column1, column2, ...)] [ SAMPLE numeric PERCENT ]
+ * </ul>
  */
 SqlNode SqlAnalyzeTable() :
 {
     SqlParserPos pos;
     SqlIdentifier tblName;
-    SqlLiteral estimate = null;
     SqlNodeList fieldList = null;
+    SqlNode level = null;
+    SqlLiteral estimate = null;
+    SqlLiteral dropMetadata = null;
+    SqlLiteral checkMetadataExistence = null;
     SqlNumericLiteral percent = null;
 }
 {
     <ANALYZE> { pos = getPos(); }
     <TABLE>
     tblName = CompoundIdentifier()
-    (
-        <COMPUTE> { estimate = SqlLiteral.createBoolean(false, pos); }
-        |
-        <ESTIMATE> { estimate = SqlLiteral.createBoolean(true, pos); }
-    )
-    <STATISTICS>
-    [
-        (fieldList = ParseRequiredFieldList("Table"))
-    ]
     [
-        <SAMPLE> percent = UnsignedNumericLiteral() <PERCENT>
-        {
-            BigDecimal rate = percent.bigDecimalValue();
-            if (rate.compareTo(BigDecimal.ZERO) <= 0 ||
-                rate.compareTo(BigDecimal.valueOf(100L)) > 0)
+        (
+            (
+                <COMPUTE> { estimate = SqlLiteral.createBoolean(false, pos); }
+                |
+                <ESTIMATE> {
+                  if (true) {
+                    throw new ParseException("ESTIMATE statistics collecting is not supported. See DRILL-7438.");
+                  }
+                  estimate = SqlLiteral.createBoolean(true, pos);
+                }
+            )
+            <STATISTICS>
+            [
+                (fieldList = ParseRequiredFieldList("Table"))
+            ]
+            [
+                <SAMPLE> percent = UnsignedNumericLiteral() <PERCENT>
+                {
+                    BigDecimal rate = percent.bigDecimalValue();
+                    if (rate.compareTo(BigDecimal.ZERO) <= 0 ||
+                        rate.compareTo(BigDecimal.valueOf(100L)) > 0)
+                    {
+                        throw new ParseException("Invalid percentage for ANALYZE TABLE");
+                    }
+                }
+            ]
             {
-                throw new ParseException("Invalid percentage for ANALYZE TABLE");
+                if (percent == null) { percent = SqlLiteral.createExactNumeric("100.0", pos); }
+                return new SqlAnalyzeTable(pos, tblName, estimate, fieldList, percent);
             }
-        }
+        )
+        |
+        (
+            [
+                <COLUMNS>
+                (
+                    fieldList = ParseRequiredFieldList("Table")
+                    |
+                    <NONE> {fieldList = SqlNodeList.EMPTY;}
+                )
+            ]
+            <REFRESH>
+            <METADATA>
+            [
+                level = StringLiteral()
+                <LEVEL>
+            ]
+            [
+                (
+                    <COMPUTE> { estimate = SqlLiteral.createBoolean(false, pos); }
+                    |
+                    <ESTIMATE> {
+                      if (true) {
+                        throw new ParseException("ESTIMATE statistics collecting is not supported. See DRILL-7438.");
+                      }
+                      estimate = SqlLiteral.createBoolean(true, pos);
+                    }
+                )
+                <STATISTICS>
+            ]
+            [
+                <SAMPLE> percent = UnsignedNumericLiteral() <PERCENT>
+                {
+                    BigDecimal rate = percent.bigDecimalValue();
+                    if (rate.compareTo(BigDecimal.ZERO) <= 0 ||
+                        rate.compareTo(BigDecimal.valueOf(100L)) > 0) {
+                      throw new ParseException("Invalid percentage for ANALYZE TABLE");
+                    }
+                }
+            ]
+            {
+                return new SqlMetastoreAnalyzeTable(pos, tblName, fieldList, level, estimate, percent);
+            }
+        )
+        |
+        (
+            <DROP>
+            [
+                <METADATA> { dropMetadata = SqlLiteral.createCharString("METADATA", pos); }
+                |
+                <STATISTICS> {
+                  if (true) {
+                    throw new ParseException("DROP STATISTICS is not supported.");
+                  }
+                  dropMetadata = SqlLiteral.createCharString("STATISTICS", pos);
+                }
+            ]
+            [
+                <IF>
+                <EXISTS> { checkMetadataExistence = SqlLiteral.createBoolean(false, pos); }
+            ]
+            {
+                if (checkMetadataExistence == null) {
+                  checkMetadataExistence = SqlLiteral.createBoolean(true, pos);
+                }
+                return new SqlDropTableMetadata(pos, tblName, dropMetadata, checkMetadataExistence);
+            }
+        )
     ]
-    {
-        if (percent == null) { percent = SqlLiteral.createExactNumeric("100.0", pos); }
-        return new SqlAnalyzeTable(pos, tblName, estimate, fieldList, percent);
-    }
+    { throw generateParseException(); }
 }
 
 
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
index 1b5068a..877a461 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
@@ -161,6 +161,8 @@ public class ${aggrtype.className}Functions {
       value.value = Float.NaN;
       <#elseif type.runningType?starts_with("Float8")>
       value.value = Double.NaN;
+      <#elseif type.runningType?starts_with("Bit")>
+      value.value = 1;
       </#if>
     <#elseif aggrtype.funcName == "max">
       <#if type.runningType?starts_with("Int")>
@@ -171,6 +173,8 @@ public class ${aggrtype.className}Functions {
       value.value = -Float.MAX_VALUE;
       <#elseif type.runningType?starts_with("Float8")>
       value.value = -Double.MAX_VALUE;
+      <#elseif type.runningType?starts_with("Bit")>
+      value.value = 0;
       </#if>
     </#if>
     }
diff --git a/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java b/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java
index 1485127..972cb14 100644
--- a/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java
@@ -40,7 +40,7 @@ import java.util.List;
  *
  * NB: Source code generated using FreeMarker template ${.template_name}
  */
-public abstract class JSONBaseStatisticsRecordWriter implements StatisticsRecordWriter {
+public abstract class JSONBaseStatisticsRecordWriter implements StatisticsRecordCollector {
 
   protected JsonOutput gen;
   protected boolean skipNullFields = true;
diff --git a/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java b/exec/java-exec/src/main/codegen/templates/StatisticsRecordCollector.java
similarity index 62%
copy from exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
copy to exec/java-exec/src/main/codegen/templates/StatisticsRecordCollector.java
index 85c5bde..d492894 100644
--- a/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/StatisticsRecordCollector.java
@@ -16,11 +16,12 @@
  * limitations under the License.
  */
 <@pp.dropOutputFile />
-<@pp.changeOutputFile name="org/apache/drill/exec/store/StatisticsRecordWriter.java" />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/StatisticsRecordCollector.java" />
 <#include "/@includes/license.ftl" />
 
 package org.apache.drill.exec.store;
 
+import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
@@ -32,62 +33,47 @@ import java.util.Map;
  * This class is generated using freemarker and the ${.template_name} template.
  */
 
-/** StatisticsRecordWriter interface. */
-public interface StatisticsRecordWriter {
+/**
+ * Interface for collecting and obtaining statistics.
+ */
+public interface StatisticsRecordCollector {
 
   /**
-   * Initialize the writer.
-   *
-   * @param writerOptions Contains key, value pair of settings.
+   * Called before starting writing fields in a record.
    * @throws IOException
    */
-  void init(Map<String, String> writerOptions) throws IOException;
+  void startStatisticsRecord() throws IOException;
 
   /**
-   * Update the schema in RecordWriter. Called at least once before starting writing the records.
-   * @param batch
+   * Called after adding all fields in a particular statistics record are added using
+   * add{TypeHolder}(fieldId, TypeHolder) methods.
    * @throws IOException
    */
-  void updateSchema(VectorAccessible batch) throws IOException;
-
-  /**
-   * Check if the writer should start a new partition, and if so, start a new partition
-   */
-  public void checkForNewPartition(int index);
+  void endStatisticsRecord() throws IOException;
 
   /**
-   * Returns if the writer is a blocking writer i.e. consumes all input before writing it out
-   * @return TRUE, if writer is blocking. FALSE, otherwise
+   * Returns true if this {@link StatisticsRecordCollector} has non-empty statistics.
+   *
+   * @return {@ode true} if this {@link StatisticsRecordCollector} has non-empty statistics
    */
-  boolean isBlockingWriter();
+  boolean hasStatistics();
 
   /**
-   * Called before starting writing fields in a record.
-   * @throws IOException
+   * Returns {@link DrillStatsTable.TableStatistics} instance with statistics collected using this {@link StatisticsRecordCollector}.
+   *
+   * @return {@link DrillStatsTable.TableStatistics} instance
    */
-  void startStatisticsRecord() throws IOException;
+  DrillStatsTable.TableStatistics getStatistics();
 
   <#list vv.types as type>
   <#list type.minor as minor>
   <#list vv.modes as mode>
-  /** Add the field value given in <code>valueHolder</code> at the given column number <code>fieldId</code>. */
-  public FieldConverter getNew${mode.prefix}${minor.class}Converter(int fieldId, String fieldName, FieldReader reader);
 
+  /**
+   * Add the field value given in <code>valueHolder</code> at the given column number <code>fieldId</code>.
+   */
+  public FieldConverter getNew${mode.prefix}${minor.class}Converter(int fieldId, String fieldName, FieldReader reader);
   </#list>
   </#list>
   </#list>
-
-  /**
-   * Called after adding all fields in a particular statistics record are added using
-   * add{TypeHolder}(fieldId, TypeHolder) methods.
-   * @throws IOException
-   */
-  void endStatisticsRecord() throws IOException;
-  /**
-   * For a blocking writer, called after processing all the records to flush out the writes
-   * @throws IOException
-   */
-  void flushBlockingWriter() throws IOException;
-  void abort() throws IOException;
-  void cleanup() throws IOException;
-}
\ No newline at end of file
+}
diff --git a/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
index 85c5bde..9c285d4 100644
--- a/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
@@ -33,7 +33,7 @@ import java.util.Map;
  */
 
 /** StatisticsRecordWriter interface. */
-public interface StatisticsRecordWriter {
+public interface StatisticsRecordWriter extends StatisticsRecordCollector {
 
   /**
    * Initialize the writer.
@@ -62,28 +62,6 @@ public interface StatisticsRecordWriter {
   boolean isBlockingWriter();
 
   /**
-   * Called before starting writing fields in a record.
-   * @throws IOException
-   */
-  void startStatisticsRecord() throws IOException;
-
-  <#list vv.types as type>
-  <#list type.minor as minor>
-  <#list vv.modes as mode>
-  /** Add the field value given in <code>valueHolder</code> at the given column number <code>fieldId</code>. */
-  public FieldConverter getNew${mode.prefix}${minor.class}Converter(int fieldId, String fieldName, FieldReader reader);
-
-  </#list>
-  </#list>
-  </#list>
-
-  /**
-   * Called after adding all fields in a particular statistics record are added using
-   * add{TypeHolder}(fieldId, TypeHolder) methods.
-   * @throws IOException
-   */
-  void endStatisticsRecord() throws IOException;
-  /**
    * For a blocking writer, called after processing all the records to flush out the writes
    * @throws IOException
    */
diff --git a/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java
index dd3ab7a..f1c6962 100644
--- a/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java
+++ b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java
@@ -31,6 +31,8 @@ import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.List;
@@ -39,9 +41,11 @@ import java.util.List;
  * This class is generated using freemarker and the ${.template_name} template.
  */
 
-/** Reads records from the RecordValueAccessor and writes into StatisticsRecordWriter. */
+/**
+ * Reads records from the RecordValueAccessor and writes into StatisticsRecordCollector.
+ */
 public class StatisticsRecordWriterImpl {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsRecordWriterImpl.class);
+  private static final Logger logger = LoggerFactory.getLogger(StatisticsRecordWriterImpl.class);
 
   private VectorAccessible batch;
   private StatisticsRecordWriter recordWriter;
@@ -97,7 +101,7 @@ public class StatisticsRecordWriterImpl {
     }
   }
 
-  public static FieldConverter getConverter(StatisticsRecordWriter recordWriter, int fieldId, String fieldName,
+  public static FieldConverter getConverter(StatisticsRecordCollector recordWriter, int fieldId, String fieldName,
       FieldReader reader) {
     switch (reader.getType().getMinorType()) {
       <#list vv.types as type>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 20668b3..91f8803 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -489,6 +489,25 @@ public final class ExecConstants {
   public static final String IMPLICIT_FILEPATH_COLUMN_LABEL = "drill.exec.storage.implicit.filepath.column.label";
   public static final OptionValidator IMPLICIT_FILEPATH_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_FILEPATH_COLUMN_LABEL,
       new OptionDescription("Available as of Drill 1.10. Sets the implicit column name for the filepath column."));
+  public static final String IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL = "drill.exec.storage.implicit.row_group_index.column.label";
+  public static final OptionValidator IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL,
+      new OptionDescription("Available as of Drill 1.17. Sets the implicit column name for the row group index (rgi) column. " +
+          "For internal usage when producing Metastore analyze."));
+
+  public static final String IMPLICIT_ROW_GROUP_START_COLUMN_LABEL = "drill.exec.storage.implicit.row_group_start.column.label";
+  public static final OptionValidator IMPLICIT_ROW_GROUP_START_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_ROW_GROUP_START_COLUMN_LABEL,
+      new OptionDescription("Available as of Drill 1.17. Sets the implicit column name for the row group start (rgs) column. " +
+          "For internal usage when producing Metastore analyze."));
+
+  public static final String IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL = "drill.exec.storage.implicit.row_group_length.column.label";
+  public static final OptionValidator IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL,
+      new OptionDescription("Available as of Drill 1.17. Sets the implicit column name for the row group length (rgl) column. " +
+          "For internal usage when producing Metastore analyze."));
+
+  public static final String IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL = "drill.exec.storage.implicit.last_modified_time.column.label";
+  public static final OptionValidator IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL,
+      new OptionDescription("Available as of Drill 1.17. Sets the implicit column name for the lastModifiedTime column. " +
+          "For internal usage when producing Metastore analyze."));
 
   public static final String JSON_READ_NUMBERS_AS_DOUBLE = "store.json.read_numbers_as_double";
   public static final BooleanValidator JSON_READ_NUMBERS_AS_DOUBLE_VALIDATOR = new BooleanValidator(JSON_READ_NUMBERS_AS_DOUBLE,
@@ -1060,8 +1079,7 @@ public final class ExecConstants {
   public static final BooleanValidator METASTORE_ENABLED_VALIDATOR = new BooleanValidator(METASTORE_ENABLED,
       new OptionDescription("Enables Drill Metastore usage to be able to store table metadata " +
           "during ANALYZE TABLE commands execution and to be able to read table metadata during regular " +
-          "queries execution or when querying some INFORMATION_SCHEMA tables. " +
-          "This option is not active for now. Default is false. (Drill 1.17+)"));
+          "queries execution or when querying some INFORMATION_SCHEMA tables. Default is false. (Drill 1.17+)"));
 
   /**
    * Option for specifying maximum level depth for collecting metadata
@@ -1070,25 +1088,22 @@ public final class ExecConstants {
    */
   public static final String METASTORE_METADATA_STORE_DEPTH_LEVEL = "metastore.metadata.store.depth_level";
   public static final EnumeratedStringValidator METASTORE_METADATA_STORE_DEPTH_LEVEL_VALIDATOR = new EnumeratedStringValidator(METASTORE_METADATA_STORE_DEPTH_LEVEL,
-      new OptionDescription("Specifies maximum level depth for collecting metadata. " +
-          "This option is not active for now. Default is 'ROW_GROUP'. (Drill 1.17+)"),
-      "TABLE", "SEGMENT", "PARTITION", "FILE", "ROW_GROUP");
+      new OptionDescription("Specifies maximum level depth for collecting metadata. Default is 'ALL'. (Drill 1.17+)"),
+      "TABLE", "SEGMENT", "PARTITION", "FILE", "ROW_GROUP", "ALL");
 
   /**
    * Option for enabling schema usage, stored to the Metastore.
    */
   public static final String METASTORE_USE_SCHEMA_METADATA = "metastore.metadata.use_schema";
   public static final BooleanValidator METASTORE_USE_SCHEMA_METADATA_VALIDATOR = new BooleanValidator(METASTORE_USE_SCHEMA_METADATA,
-      new OptionDescription("Enables schema usage, stored to the Metastore. " +
-          "This option is not active for now. Default is false. (Drill 1.17+)"));
+      new OptionDescription("Enables schema usage, stored to the Metastore. Default is false. (Drill 1.17+)"));
 
   /**
    * Option for enabling statistics usage, stored in the Metastore, at the planning stage.
    */
   public static final String METASTORE_USE_STATISTICS_METADATA = "metastore.metadata.use_statistics";
   public static final BooleanValidator METASTORE_USE_STATISTICS_METADATA_VALIDATOR = new BooleanValidator(METASTORE_USE_STATISTICS_METADATA,
-      new OptionDescription("Enables statistics usage, stored in the Metastore, at the planning stage. " +
-          "This option is not active for now. Default is false. (Drill 1.17+)"));
+      new OptionDescription("Enables statistics usage, stored in the Metastore, at the planning stage. Default is false. (Drill 1.17+)"));
 
   /**
    * Option for collecting schema and / or column statistics for every table after CTAS and CTTAS execution.
@@ -1106,16 +1121,16 @@ public final class ExecConstants {
   public static final String METASTORE_FALLBACK_TO_FILE_METADATA = "metastore.metadata.fallback_to_file_metadata";
   public static final BooleanValidator METASTORE_FALLBACK_TO_FILE_METADATA_VALIDATOR = new BooleanValidator(METASTORE_FALLBACK_TO_FILE_METADATA,
       new OptionDescription("Allows using file metadata cache for the case when required metadata is absent in the Metastore. " +
-          "This option is not active for now. Default is true. (Drill 1.17+)"));
+          "Default is true. (Drill 1.17+)"));
 
   /**
    * Option for specifying the number of attempts for retrying query planning after detecting that query metadata is changed.
    */
-  public static final String METASTORE_RETRIVAL_RETRY_ATTEMPTS = "metastore.retrival.retry_attempts";
-  public static final IntegerValidator METASTORE_RETRIVAL_RETRY_ATTEMPTS_VALIDATOR = new IntegerValidator(METASTORE_RETRIVAL_RETRY_ATTEMPTS,
+  public static final String METASTORE_RETRIEVAL_RETRY_ATTEMPTS = "metastore.retrieval.retry_attempts";
+  public static final IntegerValidator METASTORE_RETRIEVAL_RETRY_ATTEMPTS_VALIDATOR = new IntegerValidator(METASTORE_RETRIEVAL_RETRY_ATTEMPTS,
       new OptionDescription("Specifies the number of attempts for retrying query planning after detecting that query metadata is changed. " +
           "If the number of retries was exceeded, query will be planned without metadata information from the Metastore. " +
-          "This option is not active for now. Default is 5. (Drill 1.17+)"));
+          "Default is 5. (Drill 1.17+)"));
 
   public static final String PARQUET_READER_ENABLE_MAP_SUPPORT = "store.parquet.reader.enable_map_support";
   public static final BooleanValidator PARQUET_READER_ENABLE_MAP_SUPPORT_VALIDATOR = new BooleanValidator(
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/exception/MetadataException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/exception/MetadataException.java
new file mode 100644
index 0000000..d4505b2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/exception/MetadataException.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.exception;
+
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+
+/**
+ * Metadata runtime exception to indicate issues connected with table metadata.
+ */
+public class MetadataException extends DrillRuntimeException {
+  private final MetadataExceptionType exceptionType;
+
+  private MetadataException(MetadataExceptionType exceptionType) {
+    super(exceptionType.message);
+    this.exceptionType = exceptionType;
+  }
+
+  private MetadataException(MetadataExceptionType exceptionType, Throwable cause) {
+    super(exceptionType.message, cause);
+    this.exceptionType = exceptionType;
+  }
+
+  public MetadataExceptionType getExceptionType() {
+    return exceptionType;
+  }
+
+  public static MetadataException of(MetadataExceptionType exceptionType) {
+    return new MetadataException(exceptionType);
+  }
+
+  public static MetadataException of(MetadataExceptionType exceptionType, Throwable cause) {
+    return new MetadataException(exceptionType, cause);
+  }
+
+  public enum MetadataExceptionType {
+    OUTDATED_METADATA("Metastore metadata is outdated."),
+
+    INCONSISTENT_METADATA("Inconsistent Metastore metadata. " +
+        "Metadata was refreshed after it was fetched from the Metastore."),
+
+    INCOMPLETE_METADATA("Metastore does not have metadata for row groups " +
+        "and `metastore.metadata.fallback_to_file_metadata` is disabled. " +
+        "Please either execute ANALYZE with 'ROW_GROUP' level " +
+        "for the querying table or enable `metastore.metadata.fallback_to_file_metadata` to allow " +
+        "using metadata taken from the file metadata cache or table files."),
+
+    ABSENT_SCHEMA("Table schema wasn't provided " +
+        "and `metastore.metadata.use_schema` is disabled. " +
+        "Please either provide table schema for [%s] table " +
+        "(using table function or creating schema file) or enable " +
+        "`metastore.metadata.use_schema`."),
+
+    FALLBACK_EXCEPTION("Exception happened when was attempting to use fallback metadata.");
+
+    private final String message;
+
+    MetadataExceptionType(String message) {
+      this.message = message;
+    }
+
+    public String getMessage() {
+      return message;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterAggFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterAggFuncHolder.java
index ab54222..8439983 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterAggFuncHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillComplexWriterAggFuncHolder.java
@@ -90,7 +90,7 @@ public class DrillComplexWriterAggFuncHolder extends DrillAggFuncHolder {
     if (classGenerator.getCodeGenerator().getDefinition() == StreamingAggTemplate.TEMPLATE_DEFINITION) {
       aggBatchClass = classGenerator.getModel().ref(StreamingAggBatch.class);
     }
-    assert aggBatchClass != null : "ComplexWriterAggFuncHolder should only be used with an Aggregate Operator";
+    assert aggBatchClass != null : "ComplexWriterAggFuncHolder should only be used with Streaming Aggregate Operator";
 
     JExpression aggBatch = JExpr.cast(aggBatchClass, classGenerator.getMappingSet().getOutgoing());
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
index bbeea18..865f196 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
@@ -118,7 +118,8 @@ public class FunctionConverter {
         // Special processing for @Output ComplexWriter
         if (output != null && ComplexWriter.class.isAssignableFrom(fieldClass)) {
           if (outputField != null) {
-            return failure("You've declared more than one @Output field.  You must declare one and only @Output field per Function class.", func, field);
+            return failure("You've declared more than one @Output field.\n" +
+                "You must declare one and only @Output field per Function class.", func, field);
           } else {
             outputField = ValueReference.createComplexWriterRef(field.getName());
           }
@@ -127,7 +128,8 @@ public class FunctionConverter {
 
         // check that param and output are value holders.
         if (!ValueHolder.class.isAssignableFrom(fieldClass)) {
-          return failure(String.format("The field doesn't holds value of type %s which does not implement the ValueHolder interface.  All fields of type @Param or @Output must extend this interface..", fieldClass), func, field);
+          return failure(String.format("The field doesn't holds value of type %s which does not implement the ValueHolder or ComplexWriter interfaces.\n" +
+              "All fields of type @Param or @Output must extend this interface.", fieldClass), func, field);
         }
 
         // get the type field from the value holder.
@@ -171,7 +173,7 @@ public class FunctionConverter {
 
         //If the workspace var is of Holder type, get its MajorType and assign to WorkspaceReference.
         if (ValueHolder.class.isAssignableFrom(fieldClass)) {
-          MajorType majorType = null;
+          MajorType majorType;
           try {
             majorType = getStaticFieldValue("TYPE", fieldClass, MajorType.class);
           } catch (Exception e) {
@@ -190,8 +192,8 @@ public class FunctionConverter {
     FunctionInitializer initializer = new FunctionInitializer(func.getClassName(), classLoader);
     try {
       // return holder
-      ValueReference[] ps = params.toArray(new ValueReference[params.size()]);
-      WorkspaceReference[] works = workspaceFields.toArray(new WorkspaceReference[workspaceFields.size()]);
+      ValueReference[] ps = params.toArray(new ValueReference[0]);
+      WorkspaceReference[] works = workspaceFields.toArray(new WorkspaceReference[0]);
 
       FunctionAttributes functionAttributes = new FunctionAttributes(template, ps, outputField, works);
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/AggregateErrorFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/AggregateErrorFunctions.java
index 2456a84..ec6292d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/AggregateErrorFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/AggregateErrorFunctions.java
@@ -35,7 +35,7 @@ import org.apache.drill.exec.expr.holders.VarCharHolder;
  */
 public class AggregateErrorFunctions {
 
-  @FunctionTemplate(names = {"sum", "max", "avg", "stddev_pop", "stddev_samp", "stddev", "var_pop",
+  @FunctionTemplate(names = {"sum", "avg", "stddev_pop", "stddev_samp", "stddev", "var_pop",
       "var_samp", "variance"}, scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
   public static class BitAggregateErrorFunctions implements DrillAggFunc {
 
@@ -65,7 +65,7 @@ public class AggregateErrorFunctions {
 
   }
 
-  @FunctionTemplate(names = {"sum", "max", "avg", "stddev_pop", "stddev_samp", "stddev", "var_pop",
+  @FunctionTemplate(names = {"sum", "avg", "stddev_pop", "stddev_samp", "stddev", "var_pop",
       "var_samp", "variance"}, scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
   public static class NullableBitAggregateErrorFunctions implements DrillAggFunc {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CollectListMapsAggFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CollectListMapsAggFunction.java
new file mode 100644
index 0000000..c232edc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CollectListMapsAggFunction.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.ObjectHolder;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+
+/**
+ * Aggregate function which stores incoming fields into the map.
+ * This function accepts a variable number of arguments, where one argument is a field name
+ * within the resulting map and another argument is actual field to store into the map.
+ */
+@FunctionTemplate(name = "collect_list",
+                  isVarArg = true,
+                  isInternal = true,
+                  scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+public class CollectListMapsAggFunction implements DrillAggFunc {
+
+  @Param FieldReader[] inputs;
+  @Output BaseWriter.ComplexWriter writer;
+  @Workspace ObjectHolder writerHolder;
+
+  @Override
+  public void setup() {
+    writerHolder = new ObjectHolder();
+  }
+
+  @Override
+  public void add() {
+    org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter listWriter;
+    if (writerHolder.obj == null) {
+      writerHolder.obj = writer.rootAsList();
+    }
+
+    listWriter = (org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter) writerHolder.obj;
+    org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter mapWriter = listWriter.map();
+
+    mapWriter.start();
+
+    for (int i = 0; i < inputs.length; i += 2) {
+      org.apache.drill.exec.vector.complex.MapUtility.writeToMapFromReader(
+          inputs[i + 1], mapWriter, inputs[i].readText().toString(), "CollectListMapsAggFunction");
+    }
+    mapWriter.end();
+  }
+
+  @Override
+  public void output() {
+    ((org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter) writerHolder.obj).endList();
+  }
+
+  @Override
+  public void reset() {
+    writerHolder.obj = null;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CollectToListVarcharAggFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CollectToListVarcharAggFunction.java
new file mode 100644
index 0000000..7a42aee
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CollectToListVarcharAggFunction.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
+import org.apache.drill.exec.expr.holders.ObjectHolder;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+
+/**
+ * Aggregate function which collects incoming VarChar column values into the list.
+ */
+@FunctionTemplate(name = "collect_to_list_varchar",
+                  scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
+                  isInternal = true)
+public class CollectToListVarcharAggFunction implements DrillAggFunc {
+
+  @Param NullableVarCharHolder input;
+  @Output BaseWriter.ComplexWriter writer;
+  @Workspace ObjectHolder writerHolder;
+
+  @Override
+  public void setup() {
+    writerHolder = new ObjectHolder();
+  }
+
+  @Override
+  public void add() {
+    org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter listWriter;
+    if (writerHolder.obj == null) {
+      writerHolder.obj = writer.rootAsList();
+    }
+
+    listWriter = (org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter) writerHolder.obj;
+
+    if (input.isSet > 0) {
+      listWriter.varChar().writeVarChar(input.start, input.end, input.buffer);
+    }
+  }
+
+  @Override
+  public void output() {
+  }
+
+  @Override
+  public void reset() {
+    writerHolder.obj = null;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParentPathFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParentPathFunction.java
new file mode 100644
index 0000000..7ee8b47
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParentPathFunction.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+import javax.inject.Inject;
+
+@FunctionTemplate(name = "parentPath",
+                  scope = FunctionTemplate.FunctionScope.SIMPLE,
+                  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL,
+                  isInternal = true)
+public class ParentPathFunction implements DrillSimpleFunc {
+
+  @Param VarCharHolder input;
+  @Output VarCharHolder out;
+  @Inject DrillBuf buf;
+
+  @Override
+  public void setup() {
+  }
+
+  @Override
+  public void eval() {
+    org.apache.hadoop.fs.Path path =
+        new org.apache.hadoop.fs.Path(org.apache.drill.common.util.DrillStringUtils.toBinaryString(input.buffer, input.start, input.end));
+    byte[] bytes = path.getParent().toUri().getPath().getBytes();
+
+    buf = buf.reallocIfNeeded(bytes.length);
+    buf.setBytes(0, bytes);
+    out.buffer = buf;
+    out.start = 0;
+    out.end = bytes.length;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SchemaFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SchemaFunctions.java
new file mode 100644
index 0000000..24f0bc4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SchemaFunctions.java
@@ -0,0 +1,251 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
+import org.apache.drill.exec.expr.holders.ObjectHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+
+import javax.inject.Inject;
+
+public class SchemaFunctions {
+
+  /**
+   * Aggregate function which infers schema from incoming data and returns string representation of {@link TupleMetadata}
+   * with incoming schema.
+   */
+  @FunctionTemplate(name = "schema",
+                    scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
+                    isInternal = true,
+                    isVarArg = true)
+  public static class SchemaFunction implements DrillAggFunc {
+
+    @Param FieldReader[] inputs;
+    @Output NullableVarCharHolder out;
+    @Inject DrillBuf buf;
+    @Workspace ObjectHolder columnsHolder;
+
+    @Override
+    public void setup() {
+      columnsHolder = new ObjectHolder();
+    }
+
+    @Override
+    public void add() {
+      java.util.Map<String, org.apache.drill.exec.record.MaterializedField> columns;
+      if (columnsHolder.obj == null) {
+        // Janino does not support diamond operator for this case :(
+        columnsHolder.obj = new java.util.LinkedHashMap<String, org.apache.drill.exec.record.MaterializedField>();
+      }
+
+      columns = (java.util.Map<String, org.apache.drill.exec.record.MaterializedField>) columnsHolder.obj;
+
+      for (int i = 0; i < inputs.length; i += 2) {
+        String columnName = inputs[i].readObject().toString();
+        // Janino cannot infer type
+        org.apache.drill.exec.record.MaterializedField materializedField =
+            (org.apache.drill.exec.record.MaterializedField) columns.get(columnName);
+        org.apache.drill.common.types.TypeProtos.MajorType type = inputs[i + 1].getType();
+        if (materializedField != null && !materializedField.getType().equals(type)) {
+          org.apache.drill.common.types.TypeProtos.MinorType leastRestrictiveType =
+              org.apache.drill.exec.resolver.TypeCastRules.getLeastRestrictiveType(
+                  java.util.Arrays.asList(materializedField.getType().getMinorType(), type.getMinorType()));
+          org.apache.drill.common.types.TypeProtos.DataMode leastRestrictiveMode =
+              org.apache.drill.exec.resolver.TypeCastRules.getLeastRestrictiveDataMode(
+                  java.util.Arrays.asList(materializedField.getType().getMode(), type.getMode()));
+
+          org.apache.drill.exec.record.MaterializedField clone = materializedField.clone();
+          clone.replaceType(materializedField.getType().toBuilder()
+              .setMinorType(leastRestrictiveType)
+              .setMode(leastRestrictiveMode)
+              .build());
+          columns.put(columnName, clone);
+        } else {
+          if (type.getMinorType() == org.apache.drill.common.types.TypeProtos.MinorType.MAP) {
+            columns.put(columnName, inputs[i + 1].getField());
+          } else {
+            columns.put(columnName, org.apache.drill.exec.record.MaterializedField.create(columnName, type));
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      org.apache.drill.exec.record.metadata.SchemaBuilder schemaBuilder =
+          new org.apache.drill.exec.record.metadata.SchemaBuilder();
+
+      java.util.Map<String, org.apache.drill.exec.record.MaterializedField> columns =
+          (java.util.Map<String, org.apache.drill.exec.record.MaterializedField>) columnsHolder.obj;
+
+      if (columns == null) {
+        return;
+      }
+
+      for (org.apache.drill.exec.record.MaterializedField materializedField : columns.values()) {
+        // Janino compiler cannot infer types from generics :(
+        schemaBuilder.add((org.apache.drill.exec.record.MaterializedField) materializedField);
+      }
+
+      byte[] type = schemaBuilder.build().jsonString().getBytes();
+      buf = buf.reallocIfNeeded(type.length);
+      buf.setBytes(0, type);
+      out.buffer = buf;
+      out.start = 0;
+      out.end = type.length;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      columnsHolder.obj = null;
+    }
+  }
+
+  /**
+   * Aggregate function which accepts VarChar column with string representations of {@link TupleMetadata}
+   * and returns string representation of {@link TupleMetadata} with merged schema.
+   */
+  @FunctionTemplate(name = "merge_schema",
+                    scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
+                    isInternal = true)
+  public static class MergeNullableSchemaFunction implements DrillAggFunc {
+
+    @Param NullableVarCharHolder input;
+    @Output NullableVarCharHolder out;
+    @Inject DrillBuf buf;
+    @Workspace ObjectHolder schemaHolder;
+
+    @Override
+    public void setup() {
+      schemaHolder = new ObjectHolder();
+    }
+
+    @Override
+    public void add() {
+      if (input.isSet == 0) {
+        return;
+      }
+
+      org.apache.drill.exec.record.metadata.TupleMetadata currentSchema =
+          org.apache.drill.exec.expr.fn.impl.SchemaFunctions.getTupleMetadata(
+              org.apache.drill.common.util.DrillStringUtils.toBinaryString(input.buffer, input.start, input.end));
+      if (schemaHolder.obj == null) {
+        schemaHolder.obj = currentSchema;
+        return;
+      }
+
+      org.apache.drill.exec.record.metadata.TupleMetadata resolvedSchema =
+          (org.apache.drill.exec.record.metadata.TupleMetadata) schemaHolder.obj;
+
+      if (!resolvedSchema.isEquivalent(currentSchema)) {
+        throw new UnsupportedOperationException("merge_schema function does not support schema changes.");
+      }
+    }
+
+    @Override
+    public void output() {
+      org.apache.drill.exec.record.metadata.TupleMetadata resolvedSchema =
+          (org.apache.drill.exec.record.metadata.TupleMetadata) schemaHolder.obj;
+
+      byte[] type = resolvedSchema.jsonString().getBytes();
+      buf = buf.reallocIfNeeded(type.length);
+      buf.setBytes(0, type);
+      out.buffer = buf;
+      out.start = 0;
+      out.end = type.length;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      schemaHolder.obj = null;
+    }
+
+  }
+
+  @FunctionTemplate(name = "merge_schema",
+                    scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
+                    isInternal = true)
+  public static class MergeSchemaFunction implements DrillAggFunc {
+
+    @Param VarCharHolder input;
+    @Output VarCharHolder out;
+    @Inject DrillBuf buf;
+    @Workspace ObjectHolder schemaHolder;
+
+    @Override
+    public void setup() {
+      schemaHolder = new ObjectHolder();
+    }
+
+    @Override
+    public void add() {
+      org.apache.drill.exec.record.metadata.TupleMetadata currentSchema = org.apache.drill.exec.expr.fn.impl.SchemaFunctions.getTupleMetadata(
+          org.apache.drill.common.util.DrillStringUtils.toBinaryString(input.buffer, input.start, input.end));
+      if (schemaHolder.obj == null) {
+        schemaHolder.obj = currentSchema;
+        return;
+      }
+
+      org.apache.drill.exec.record.metadata.TupleMetadata resolvedSchema =
+          (org.apache.drill.exec.record.metadata.TupleMetadata) schemaHolder.obj;
+
+      if (!resolvedSchema.isEquivalent(currentSchema)) {
+        throw new UnsupportedOperationException("merge_schema function does not support schema changes.");
+      }
+    }
+
+    @Override
+    public void output() {
+      org.apache.drill.exec.record.metadata.TupleMetadata resolvedSchema =
+          (org.apache.drill.exec.record.metadata.TupleMetadata) schemaHolder.obj;
+
+      byte[] type = resolvedSchema.jsonString().getBytes();
+      buf = buf.reallocIfNeeded(type.length);
+      buf.setBytes(0, type);
+      out.buffer = buf;
+      out.start = 0;
+      out.end = type.length;
+    }
+
+    @Override
+    public void reset() {
+      schemaHolder.obj = null;
+    }
+  }
+
+  /**
+   * Wraps static method from TupleMetadata to avoid {@link IncompatibleClassChangeError} for JDK 9+.
+   * {@see JDK-8147755}.
+   *
+   * @param serialized tuple metadata in JSON string representation
+   * @return {@link TupleMetadata} instance
+   */
+  public static TupleMetadata getTupleMetadata(String serialized) {
+    return TupleMetadata.of(serialized);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreMetadataProviderManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreMetadataProviderManager.java
new file mode 100644
index 0000000..409308a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreMetadataProviderManager.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore;
+
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.record.metadata.schema.SchemaProvider;
+import org.apache.drill.metastore.MetastoreRegistry;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.apache.drill.metastore.metadata.TableMetadataProvider;
+import org.apache.drill.metastore.metadata.TableMetadataProviderBuilder;
+
+/**
+ * Implementation of {@link MetadataProviderManager} which uses Drill Metastore providers and returns
+ * builders for metastore-based {@link TableMetadataProvider} instances.
+ */
+public class MetastoreMetadataProviderManager implements MetadataProviderManager {
+
+  private final MetastoreRegistry metastoreRegistry;
+  private final TableInfo tableInfo;
+  private final MetastoreMetadataProviderConfig config;
+
+  private TableMetadataProvider tableMetadataProvider;
+
+  private SchemaProvider schemaProvider;
+  private DrillStatsTable statsProvider;
+
+  public MetastoreMetadataProviderManager(MetastoreRegistry metastoreRegistry,
+      TableInfo tableInfo, MetastoreMetadataProviderConfig config) {
+    this.metastoreRegistry = metastoreRegistry;
+    this.tableInfo = tableInfo;
+    this.config = config;
+  }
+
+  @Override
+  public void setSchemaProvider(SchemaProvider schemaProvider) {
+    this.schemaProvider = schemaProvider;
+  }
+
+  @Override
+  public SchemaProvider getSchemaProvider() {
+    return schemaProvider;
+  }
+
+  @Override
+  public void setStatsProvider(DrillStatsTable statsProvider) {
+    this.statsProvider = statsProvider;
+  }
+
+  @Override
+  public DrillStatsTable getStatsProvider() {
+    return statsProvider;
+  }
+
+  @Override
+  public void setTableMetadataProvider(TableMetadataProvider tableMetadataProvider) {
+    this.tableMetadataProvider = tableMetadataProvider;
+  }
+
+  @Override
+  public TableMetadataProvider getTableMetadataProvider() {
+    return tableMetadataProvider;
+  }
+
+  public MetastoreRegistry getMetastoreRegistry() {
+    return metastoreRegistry;
+  }
+
+  public TableInfo getTableInfo() {
+    return tableInfo;
+  }
+
+  public MetastoreMetadataProviderConfig getConfig() {
+    return config;
+  }
+
+  @Override
+  public TableMetadataProviderBuilder builder(MetadataProviderKind kind) {
+    switch (kind) {
+      case PARQUET_TABLE:
+        return new MetastoreParquetTableMetadataProvider.Builder(this);
+    }
+    return null;
+  }
+
+  public static class MetastoreMetadataProviderConfig {
+    private final boolean useSchema;
+    private final boolean useStatistics;
+    private final boolean fallbackToFileMetadata;
+
+    public MetastoreMetadataProviderConfig(boolean useSchema, boolean useStatistics, boolean fallbackToFileMetadata) {
+      this.useSchema = useSchema;
+      this.useStatistics = useStatistics;
+      this.fallbackToFileMetadata = fallbackToFileMetadata;
+    }
+
+    public boolean useSchema() {
+      return useSchema;
+    }
+
+    public boolean useStatistics() {
+      return useStatistics;
+    }
+
+    public boolean fallbackToFileMetadata() {
+      return fallbackToFileMetadata;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreParquetTableMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreParquetTableMetadataProvider.java
new file mode 100644
index 0000000..24736bf
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreParquetTableMetadataProvider.java
@@ -0,0 +1,414 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.MetadataException;
+import org.apache.drill.exec.metastore.MetastoreMetadataProviderManager.MetastoreMetadataProviderConfig;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.record.SchemaUtil;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.schema.SchemaProvider;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.FileSelection;
+import org.apache.drill.exec.store.dfs.ReadEntryWithPath;
+import org.apache.drill.exec.store.parquet.ParquetFileTableMetadataProviderBuilder;
+import org.apache.drill.exec.store.parquet.ParquetReaderConfig;
+import org.apache.drill.exec.store.parquet.ParquetTableMetadataProviderImpl;
+import org.apache.drill.exec.store.parquet.ParquetTableMetadataUtils;
+import org.apache.drill.exec.util.DrillFileSystemUtil;
+import org.apache.drill.metastore.MetastoreRegistry;
+import org.apache.drill.metastore.components.tables.BasicTablesRequests;
+import org.apache.drill.metastore.components.tables.MetastoreTableInfo;
+import org.apache.drill.metastore.metadata.BaseTableMetadata;
+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.metadata.SegmentMetadata;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.apache.drill.metastore.metadata.TableMetadata;
+import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.Statistic;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
+import org.apache.drill.metastore.util.SchemaPathUtils;
+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.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class MetastoreParquetTableMetadataProvider implements ParquetTableMetadataProvider {
+  private static final Logger logger = LoggerFactory.getLogger(MetastoreParquetTableMetadataProvider.class);
+
+  private final BasicTablesRequests basicTablesRequests;
+  private final TableInfo tableInfo;
+  private final MetastoreTableInfo metastoreTableInfo;
+  private final TupleMetadata schema;
+  private final List<ReadEntryWithPath> entries;
+  private final List<String> paths;
+  private final DrillStatsTable statsProvider;
+
+  private final boolean useSchema;
+  private final boolean useStatistics;
+  private final boolean fallbackToFileMetadata;
+
+  private BaseTableMetadata tableMetadata;
+  private Map<Path, SegmentMetadata> segmentsMetadata;
+  private List<PartitionMetadata> partitions;
+  private Map<Path, FileMetadata> files;
+  private Multimap<Path, RowGroupMetadata> rowGroups;
+  private NonInterestingColumnsMetadata nonInterestingColumnsMetadata;
+  // stores builder to provide lazy init for fallback ParquetTableMetadataProvider
+  private ParquetFileTableMetadataProviderBuilder fallbackBuilder;
+  private ParquetTableMetadataProvider fallback;
+
+  private MetastoreParquetTableMetadataProvider(List<ReadEntryWithPath> entries,
+      MetastoreRegistry metastoreRegistry, TableInfo tableInfo, TupleMetadata schema,
+      ParquetFileTableMetadataProviderBuilder fallbackBuilder, MetastoreMetadataProviderConfig config, DrillStatsTable statsProvider) {
+    this.basicTablesRequests = metastoreRegistry.get().tables().basicRequests();
+    this.tableInfo = tableInfo;
+    this.metastoreTableInfo = basicTablesRequests.metastoreTableInfo(tableInfo);
+    this.useSchema = config.useSchema();
+    this.useStatistics = config.useStatistics();
+    this.fallbackToFileMetadata = config.fallbackToFileMetadata();
+    this.schema = schema;
+    this.entries = entries == null ? new ArrayList<>() : entries;
+    this.fallbackBuilder = fallbackBuilder;
+    this.statsProvider = statsProvider;
+    this.paths = this.entries.stream()
+        .map(readEntryWithPath -> readEntryWithPath.getPath().toUri().getPath())
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public boolean isUsedMetadataCache() {
+    return false;
+  }
+
+  @Override
+  public Path getSelectionRoot() {
+    return getTableMetadata().getLocation();
+  }
+
+  @Override
+  public List<ReadEntryWithPath> getEntries() {
+    return entries;
+  }
+
+  @Override
+  public List<RowGroupMetadata> getRowGroupsMeta() {
+    return new ArrayList<>(getRowGroupsMetadataMap().values());
+  }
+
+  @Override
+  public List<Path> getLocations() {
+    return new ArrayList<>(getFilesMetadataMap().keySet());
+  }
+
+  @Override
+  public Multimap<Path, RowGroupMetadata> getRowGroupsMetadataMap() {
+    throwIfChanged();
+    if (rowGroups == null) {
+      rowGroups = LinkedListMultimap.create();
+      basicTablesRequests.rowGroupsMetadata(tableInfo, null, paths).stream()
+          .collect(Collectors.groupingBy(RowGroupMetadata::getPath, Collectors.toList()))
+          .forEach((path, rowGroupMetadata) -> rowGroups.putAll(path, rowGroupMetadata));
+      if (rowGroups.isEmpty()) {
+        if (fallbackToFileMetadata) {
+          try {
+            rowGroups = getFallbackTableMetadataProvider().getRowGroupsMetadataMap();
+          } catch (IOException e) {
+            throw MetadataException.of(MetadataException.MetadataExceptionType.FALLBACK_EXCEPTION, e);
+          }
+        } else {
+          throw MetadataException.of(MetadataException.MetadataExceptionType.INCOMPLETE_METADATA);
+        }
+      }
+    }
+    return rowGroups;
+  }
+
+  @Override
+  public Set<Path> getFileSet() {
+    throwIfChanged();
+    return getFilesMetadataMap().keySet();
+  }
+
+  @Override
+  public TableMetadata getTableMetadata() {
+    throwIfChanged();
+    if (tableMetadata == null) {
+      if (schema == null) {
+        if (useSchema) {
+          tableMetadata = basicTablesRequests.tableMetadata(tableInfo);
+        } else {
+          throw MetadataException.of(MetadataException.MetadataExceptionType.ABSENT_SCHEMA);
+        }
+      } else {
+        tableMetadata = basicTablesRequests.tableMetadata(tableInfo).toBuilder()
+            .schema(schema)
+            .build();
+      }
+
+      if (!useStatistics) {
+        // removes statistics to prevent its usage later
+        tableMetadata = tableMetadata.toBuilder()
+            .columnsStatistics(Collections.emptyMap())
+            .build();
+      }
+
+      if (statsProvider != null) {
+        if (!statsProvider.isMaterialized()) {
+          statsProvider.materialize();
+        }
+        tableMetadata = tableMetadata.cloneWithStats(
+            ParquetTableMetadataUtils.getColumnStatistics(tableMetadata.getSchema(), statsProvider),
+            DrillStatsTable.getEstimatedTableStats(statsProvider));
+      }
+    }
+    return tableMetadata;
+  }
+
+  @Override
+  public List<SchemaPath> getPartitionColumns() {
+    throwIfChanged();
+    return basicTablesRequests.interestingColumnsAndPartitionKeys(tableInfo).partitionKeys().values().stream()
+        .map(SchemaPath::getSimplePath)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public List<PartitionMetadata> getPartitionsMetadata() {
+    throwIfChanged();
+    if (partitions == null) {
+      partitions = basicTablesRequests.partitionsMetadata(tableInfo, null, null);
+    }
+    return partitions;
+  }
+
+  @Override
+  public List<PartitionMetadata> getPartitionMetadata(SchemaPath columnName) {
+    throwIfChanged();
+    return basicTablesRequests.partitionsMetadata(tableInfo, null, columnName.getRootSegmentPath());
+  }
+
+  @Override
+  public Map<Path, FileMetadata> getFilesMetadataMap() {
+    throwIfChanged();
+    if (files == null) {
+      files = basicTablesRequests.filesMetadata(tableInfo, null, paths).stream()
+          .collect(Collectors.toMap(FileMetadata::getPath, Function.identity()));
+    }
+    return files;
+  }
+
+  @Override
+  public Map<Path, SegmentMetadata> getSegmentsMetadataMap() {
+    throwIfChanged();
+    if (segmentsMetadata == null) {
+      segmentsMetadata = basicTablesRequests.segmentsMetadataByColumn(tableInfo, null, null).stream()
+          .collect(Collectors.toMap(SegmentMetadata::getPath, Function.identity()));
+    }
+    return segmentsMetadata;
+  }
+
+  @Override
+  public FileMetadata getFileMetadata(Path location) {
+    throwIfChanged();
+    return basicTablesRequests.fileMetadata(tableInfo, null, location.toUri().getPath());
+  }
+
+  @Override
+  public List<FileMetadata> getFilesForPartition(PartitionMetadata partition) {
+    throwIfChanged();
+    List<String> paths = partition.getLocations().stream()
+        .map(path -> path.toUri().getPath())
+        .collect(Collectors.toList());
+    return basicTablesRequests.filesMetadata(tableInfo, null, paths);
+  }
+
+  @Override
+  public NonInterestingColumnsMetadata getNonInterestingColumnsMetadata() {
+    throwIfChanged();
+    if (nonInterestingColumnsMetadata == null) {
+      TupleMetadata schema = getTableMetadata().getSchema();
+
+      List<StatisticsHolder> statistics = Collections.singletonList(new StatisticsHolder<>(Statistic.NO_COLUMN_STATS, ColumnStatisticsKind.NULLS_COUNT));
+
+      List<SchemaPath> columnPaths = SchemaUtil.getSchemaPaths(schema);
+      List<SchemaPath> interestingColumns = getInterestingColumns(columnPaths);
+      // populates statistics for non-interesting columns and columns for which statistics wasn't collected
+      Map<SchemaPath, ColumnStatistics> columnsStatistics = columnPaths.stream()
+          .filter(schemaPath -> !interestingColumns.contains(schemaPath)
+              || SchemaPathUtils.getColumnMetadata(schemaPath, schema).isArray())
+          .collect(Collectors.toMap(
+              Function.identity(),
+              schemaPath -> new ColumnStatistics<>(statistics, SchemaPathUtils.getColumnMetadata(schemaPath, schema).type())));
+      nonInterestingColumnsMetadata = new NonInterestingColumnsMetadata(columnsStatistics);
+    }
+    return nonInterestingColumnsMetadata;
+  }
+
+  @Override
+  public boolean checkMetadataVersion() {
+    return true;
+  }
+
+  private List<SchemaPath> getInterestingColumns(List<SchemaPath> columnPaths) {
+    if (useStatistics) {
+      return getTableMetadata().getInterestingColumns() == null
+          ? columnPaths
+          : getTableMetadata().getInterestingColumns();
+    } else {
+      // if `metastore.metadata.use_statistics` is false, all columns are treated as non-interesting
+      return Collections.emptyList();
+    }
+  }
+
+  private ParquetTableMetadataProvider getFallbackTableMetadataProvider() throws IOException {
+    if (fallback == null) {
+      fallback = fallbackBuilder == null ? null : fallbackBuilder.build();
+    }
+    return fallback;
+  }
+
+  private void throwIfChanged() {
+    if (basicTablesRequests.hasMetastoreTableInfoChanged(metastoreTableInfo)) {
+      throw MetadataException.of(MetadataException.MetadataExceptionType.INCONSISTENT_METADATA);
+    }
+  }
+
+  public static class Builder implements ParquetFileTableMetadataProviderBuilder {
+    private final MetastoreMetadataProviderManager metadataProviderManager;
+
+    private List<ReadEntryWithPath> entries;
+    private DrillFileSystem fs;
+    private TupleMetadata schema;
+
+    private FileSelection selection;
+
+    // builder for fallback ParquetFileTableMetadataProvider
+    // for the case when required metadata is absent in Metastore
+    private ParquetFileTableMetadataProviderBuilder fallback;
+
+    public Builder(MetastoreMetadataProviderManager source) {
+      this.metadataProviderManager = source;
+      this.fallback = new ParquetTableMetadataProviderImpl.Builder(FileSystemMetadataProviderManager.init());
+    }
+
+    @Override
+    public ParquetFileTableMetadataProviderBuilder withEntries(List<ReadEntryWithPath> entries) {
+      this.entries = entries;
+      fallback.withEntries(entries);
+      return this;
+    }
+
+    @Override
+    public ParquetFileTableMetadataProviderBuilder withSelectionRoot(Path selectionRoot) {
+      fallback.withSelectionRoot(selectionRoot);
+      return this;
+    }
+
+    @Override
+    public ParquetFileTableMetadataProviderBuilder withCacheFileRoot(Path cacheFileRoot) {
+      fallback.withCacheFileRoot(cacheFileRoot);
+      return this;
+    }
+
+    @Override
+    public ParquetFileTableMetadataProviderBuilder withReaderConfig(ParquetReaderConfig readerConfig) {
+      fallback.withReaderConfig(readerConfig);
+      return this;
+    }
+
+    @Override
+    public ParquetFileTableMetadataProviderBuilder withFileSystem(DrillFileSystem fs) {
+      fallback.withFileSystem(fs);
+      this.fs = fs;
+      return this;
+    }
+
+    @Override
+    public ParquetFileTableMetadataProviderBuilder withCorrectCorruptedDates(boolean autoCorrectCorruptedDates) {
+      fallback.withCorrectCorruptedDates(autoCorrectCorruptedDates);
+      return this;
+    }
+
+    @Override
+    public ParquetFileTableMetadataProviderBuilder withSelection(FileSelection selection) {
+      fallback.withSelection(selection);
+      this.selection = selection;
+      return this;
+    }
+
+    @Override
+    public ParquetFileTableMetadataProviderBuilder withSchema(TupleMetadata schema) {
+      fallback.withSchema(schema);
+      this.schema = schema;
+      return this;
+    }
+
+    @Override
+    public ParquetTableMetadataProvider build() throws IOException {
+      MetastoreParquetTableMetadataProvider provider;
+      SchemaProvider schemaProvider = metadataProviderManager.getSchemaProvider();
+      ParquetMetadataProvider source = (ParquetTableMetadataProvider) metadataProviderManager.getTableMetadataProvider();
+
+      DrillStatsTable statsProvider = metadataProviderManager.getStatsProvider();
+      // schema passed into the builder has greater priority
+      try {
+        if (this.schema == null) {
+          schema = schemaProvider != null ? schemaProvider.read().getSchema() : null;
+        }
+      } catch (IOException e) {
+        logger.debug("Unable to deserialize schema from schema file for table: {}", metadataProviderManager.getTableInfo().name(), e);
+      }
+      if (entries == null) {
+        if (!selection.isExpandedFully()) {
+          entries = DrillFileSystemUtil.listFiles(fs, selection.getSelectionRoot(), true).stream()
+              .map(fileStatus -> new ReadEntryWithPath(Path.getPathWithoutSchemeAndAuthority(fileStatus.getPath())))
+              .collect(Collectors.toList());
+        } else {
+          entries = selection.getFiles().stream()
+              .map(Path::getPathWithoutSchemeAndAuthority)
+              .map(ReadEntryWithPath::new)
+              .collect(Collectors.toList());
+        }
+      }
+      provider = new MetastoreParquetTableMetadataProvider(entries, metadataProviderManager.getMetastoreRegistry(),
+          metadataProviderManager.getTableInfo(), schema, fallback, metadataProviderManager.getConfig(), statsProvider);
+      // store results into metadataProviderManager to be able to use them when creating new instances
+      // for the case when source wasn't provided or it contains less row group metadata than the provider
+      if (source == null || source.getRowGroupsMeta().size() < provider.getRowGroupsMeta().size()) {
+        metadataProviderManager.setTableMetadataProvider(provider);
+      }
+      return provider;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java
index 64ef4e8..b3a34658 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java
@@ -100,6 +100,11 @@ public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
     return null;
   }
 
+  @Override
+  public boolean checkMetadataVersion() {
+    return false;
+  }
+
   public static class Builder implements SimpleFileTableMetadataProviderBuilder {
     private String tableName;
     private Path location;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeColumnUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeColumnUtils.java
new file mode 100644
index 0000000..296d43d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeColumnUtils.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore.analyze;
+
+import org.apache.calcite.sql.SqlKind;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.metastore.statistics.BaseStatisticsKind;
+import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
+import org.apache.drill.metastore.statistics.StatisticsKind;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+
+public class AnalyzeColumnUtils {
+  private static final String COLUMN_SEPARATOR = "$";
+
+  public static final Map<StatisticsKind, SqlKind> COLUMN_STATISTICS_FUNCTIONS = ImmutableMap.<StatisticsKind, SqlKind>builder()
+      .put(ColumnStatisticsKind.MAX_VALUE, SqlKind.MAX)
+      .put(ColumnStatisticsKind.MIN_VALUE, SqlKind.MIN)
+      .put(ColumnStatisticsKind.NON_NULL_COUNT, SqlKind.COUNT)
+      .put(TableStatisticsKind.ROW_COUNT, SqlKind.COUNT)
+      .build();
+
+  public static final Map<StatisticsKind, TypeProtos.MinorType> COLUMN_STATISTICS_TYPES = ImmutableMap.<StatisticsKind, TypeProtos.MinorType>builder()
+      .put(ColumnStatisticsKind.NON_NULL_COUNT, TypeProtos.MinorType.BIGINT)
+      .put(TableStatisticsKind.ROW_COUNT, TypeProtos.MinorType.BIGINT)
+      .build();
+
+  public static final Map<StatisticsKind, SqlKind> META_STATISTICS_FUNCTIONS = ImmutableMap.<StatisticsKind, SqlKind>builder()
+      .put(TableStatisticsKind.ROW_COUNT, SqlKind.COUNT)
+      .build();
+
+  /**
+   * Returns actual column name obtained form intermediate name which includes statistics kind and other analyze-specific info.
+   * <p>
+   * Example: column which corresponds to max statistics value for {@code `o_shippriority`} column is {@code column$maxValue$`o_shippriority`}.
+   * This method will return actual column name: {@code `o_shippriority`}.
+   *
+   * @param fullName the source of actual column name
+   * @return actual column name
+   */
+  public static String getColumnName(String fullName) {
+    return fullName.substring(fullName.indexOf(COLUMN_SEPARATOR, fullName.indexOf(COLUMN_SEPARATOR) + 1) + 1);
+  }
+
+  /**
+   * Returns {@link StatisticsKind} instance obtained form intermediate field name.
+   *
+   * @param fullName the source of {@link StatisticsKind} to obtain
+   * @return {@link StatisticsKind} instance
+   */
+  public static StatisticsKind getStatisticsKind(String fullName) {
+    String statisticsIdentifier = fullName.split("\\" + COLUMN_SEPARATOR)[1];
+    switch (statisticsIdentifier) {
+      case "minValue":
+        return ColumnStatisticsKind.MIN_VALUE;
+      case "maxValue":
+        return ColumnStatisticsKind.MAX_VALUE;
+      case "nullsCount":
+        return ColumnStatisticsKind.NULLS_COUNT;
+      case "nonnullrowcount":
+        return ColumnStatisticsKind.NON_NULL_COUNT;
+      case "rowCount":
+        return TableStatisticsKind.ROW_COUNT;
+    }
+    return new BaseStatisticsKind(statisticsIdentifier, false);
+  }
+
+  /**
+   * Returns analyze-specific field name for column statistics which includes
+   * actual column name and statistics kind information.
+   * <p>
+   * Example: analyze-specific field name for column {@code `o_shippriority`}
+   * and statistics {@code MAX_VALUE} is the following: {@code column$maxValue$`o_shippriority`}.
+   *
+   * @param columnName     name of the column
+   * @param statisticsKind statistics kind
+   * @return analyze-specific field name which includes actual column name and statistics kind information
+   */
+  public static String getColumnStatisticsFieldName(String columnName, StatisticsKind statisticsKind) {
+    return String.format("column%1$s%2$s%1$s%3$s", COLUMN_SEPARATOR, statisticsKind.getName(), columnName);
+  }
+
+  /**
+   * Returns analyze-specific field name for metadata statistics which includes statistics kind information.
+   * <p>
+   * Example: analyze-specific field name for statistics {@code ROW_COUNT} is the following: {@code metadata$rowCount}.
+   *
+   * @param statisticsKind statistics kind
+   * @return analyze-specific field name for metadata statistics
+   */
+  public static String getMetadataStatisticsFieldName(StatisticsKind statisticsKind) {
+    return String.format("metadata%s%s", COLUMN_SEPARATOR, statisticsKind.getName());
+  }
+
+  /**
+   * Checks whether specified field name is analyze-specific field for column statistics.
+   *
+   * @param fieldName name of the field to check
+   * @return {@code true} if specified field name is analyze-specific field for column statistics
+   */
+  public static boolean isColumnStatisticsField(String fieldName) {
+    return fieldName.startsWith("column" + COLUMN_SEPARATOR);
+  }
+
+  /**
+   * Checks whether specified field name is analyze-specific field for metadata statistics.
+   * @param fieldName name of the field to check
+   * @return {@code true} if specified field name is analyze-specific field for metadata statistics
+   */
+  public static boolean isMetadataStatisticsField(String fieldName) {
+    return fieldName.startsWith("metadata" + COLUMN_SEPARATOR);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeFileInfoProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeFileInfoProvider.java
new file mode 100644
index 0000000..0371cc2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeFileInfoProvider.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.exec.metastore.analyze;
+
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.store.ColumnExplorer;
+import org.apache.drill.exec.store.dfs.FileSelection;
+import org.apache.drill.exec.store.dfs.FormatSelection;
+import org.apache.drill.metastore.components.tables.BasicTablesRequests;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.TableInfo;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link AnalyzeInfoProvider} for file-based tables.
+ */
+public abstract class AnalyzeFileInfoProvider implements AnalyzeInfoProvider {
+
+  @Override
+  public List<SchemaPath> getSegmentColumns(DrillTable table, OptionManager options) throws IOException {
+    FormatSelection selection = (FormatSelection) table.getSelection();
+
+    FileSelection fileSelection = selection.getSelection();
+    if (!fileSelection.isExpandedFully()) {
+      fileSelection = FileMetadataInfoCollector.getExpandedFileSelection(fileSelection);
+    }
+
+    return ColumnExplorer.getPartitionColumnNames(fileSelection, options).stream()
+        .map(SchemaPath::getSimplePath)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public List<SqlIdentifier> getProjectionFields(MetadataType metadataLevel, OptionManager options) {
+    return Arrays.asList(
+        new SqlIdentifier(options.getString(ExecConstants.IMPLICIT_FQN_COLUMN_LABEL), SqlParserPos.ZERO),
+        new SqlIdentifier(options.getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL), SqlParserPos.ZERO));
+  }
+
+  @Override
+  public MetadataInfoCollector getMetadataInfoCollector(BasicTablesRequests basicRequests, TableInfo tableInfo,
+      FormatSelection selection, PlannerSettings settings, Supplier<TableScan> tableScanSupplier,
+      List<SchemaPath> interestingColumns, MetadataType metadataLevel, int segmentColumnsCount) throws IOException {
+    return new FileMetadataInfoCollector(basicRequests, tableInfo, selection,
+        settings, tableScanSupplier, interestingColumns, metadataLevel, segmentColumnsCount);
+  }
+
+  @Override
+  public SchemaPath getLocationField(OptionManager optionManager) {
+    return SchemaPath.getSimplePath(optionManager.getString(ExecConstants.IMPLICIT_FQN_COLUMN_LABEL));
+  }
+
+  @Override
+  public NamedExpression getParentLocationExpression(SchemaPath locationField) {
+    return new NamedExpression(new FunctionCall("parentPath",
+        Collections.singletonList(locationField), ExpressionPosition.UNKNOWN),
+        FieldReference.getWithQuotedRef(MetastoreAnalyzeConstants.LOCATION_FIELD));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeInfoProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeInfoProvider.java
new file mode 100644
index 0000000..49b8430
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeInfoProvider.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore.analyze;
+
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.store.dfs.FormatSelection;
+import org.apache.drill.metastore.components.tables.BasicTablesRequests;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.TableInfo;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.function.Supplier;
+
+/**
+ * Interface for obtaining information required for analyzing tables such as table segment columns, etc.
+ */
+public interface AnalyzeInfoProvider {
+
+  /**
+   * Returns list of segment column names for specified {@link DrillTable} table.
+   *
+   * @param table   table for which should be returned segment column names
+   * @param options option manager
+   * @return list of segment column names
+   */
+  List<SchemaPath> getSegmentColumns(DrillTable table, OptionManager options) throws IOException;
+
+  /**
+   * Returns list of fields required for ANALYZE.
+   *
+   * @param metadataLevel metadata level for analyze
+   * @param options       option manager
+   * @return list of fields required for ANALYZE
+   */
+  List<SqlIdentifier> getProjectionFields(MetadataType metadataLevel, OptionManager options);
+
+  /**
+   * Returns {@link MetadataInfoCollector} instance for obtaining information about segments, files, etc.
+   * which should be handled in metastore.
+   *
+   * @param basicRequests       Metastore tables data provider helper
+   * @param tableInfo           table info
+   * @param selection           format selection
+   * @param settings            planner settings
+   * @param tableScanSupplier   supplier for table scan
+   * @param interestingColumns  list of interesting columns
+   * @param metadataLevel       metadata level
+   * @param segmentColumnsCount number of segment columns
+   * @return {@link MetadataInfoCollector} instance
+   */
+  MetadataInfoCollector getMetadataInfoCollector(BasicTablesRequests basicRequests, TableInfo tableInfo,
+      FormatSelection selection, PlannerSettings settings, Supplier<TableScan> tableScanSupplier,
+      List<SchemaPath> interestingColumns, MetadataType metadataLevel, int segmentColumnsCount) throws IOException;
+
+  /**
+   * Provides schema path to field which will be used as a location for specific table data,
+   * for example, for file-based tables, it may be `fqn`.
+   *
+   * @param optionManager option manager
+   * @return location field
+   */
+  SchemaPath getLocationField(OptionManager optionManager);
+
+  /**
+   * Returns expression which may be used to determine parent location for specific table data,
+   * i.e. segment location. For example, for file-based tables, such expression will be `parentPath` function call.
+   *
+   * @param locationField location field
+   * @return expression to determine parent location
+   */
+  NamedExpression getParentLocationExpression(SchemaPath locationField);
+
+  /**
+   * Checks whether this {@link AnalyzeInfoProvider} supports specified {@link GroupScan} type.
+   *
+   * @param groupScan group scan
+   * @return {@code true} if this {@link AnalyzeInfoProvider} supports specified {@link GroupScan} type
+   */
+  boolean supportsGroupScan(GroupScan groupScan);
+
+  /**
+   * Returns table type name supported by this {@link AnalyzeInfoProvider}.
+   *
+   * @return table type name
+   */
+  String getTableTypeName();
+
+  /**
+   * Checks whether this {@link AnalyzeInfoProvider} supports specified {@link MetadataType}.
+   *
+   * @param metadataType metadata type
+   * @return {@code true} if this {@link AnalyzeInfoProvider} supports specified {@link MetadataType}
+   */
+  boolean supportsMetadataType(MetadataType metadataType);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeParquetInfoProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeParquetInfoProvider.java
new file mode 100644
index 0000000..4ce1424
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeParquetInfoProvider.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore.analyze;
+
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.store.parquet.ParquetGroupScan;
+import org.apache.drill.metastore.metadata.MetadataType;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Implementation of {@link AnalyzeInfoProvider} for parquet tables.
+ */
+public class AnalyzeParquetInfoProvider extends AnalyzeFileInfoProvider {
+  public static final AnalyzeParquetInfoProvider INSTANCE = new AnalyzeParquetInfoProvider();
+
+  public static final String TABLE_TYPE_NAME = "PARQUET";
+
+  @Override
+  public List<SqlIdentifier> getProjectionFields(MetadataType metadataLevel, OptionManager options) {
+    List<SqlIdentifier> columnList = new ArrayList<>(super.getProjectionFields(metadataLevel, options));
+    if (metadataLevel.includes(MetadataType.ROW_GROUP)) {
+      columnList.add(new SqlIdentifier(options.getString(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL), SqlParserPos.ZERO));
+      columnList.add(new SqlIdentifier(options.getString(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL), SqlParserPos.ZERO));
+      columnList.add(new SqlIdentifier(options.getString(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL), SqlParserPos.ZERO));
+    }
+    return Collections.unmodifiableList(columnList);
+  }
+
+  @Override
+  public boolean supportsGroupScan(GroupScan groupScan) {
+    return groupScan instanceof ParquetGroupScan;
+  }
+
+  @Override
+  public String getTableTypeName() {
+    return TABLE_TYPE_NAME;
+  }
+
+  @Override
+  public boolean supportsMetadataType(MetadataType metadataType) {
+    switch (metadataType) {
+      case ROW_GROUP:
+      case FILE:
+      case SEGMENT:
+      case TABLE:
+        return true;
+      default:
+        return false;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/FileMetadataInfoCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/FileMetadataInfoCollector.java
new file mode 100644
index 0000000..e9882d1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/FileMetadataInfoCollector.java
@@ -0,0 +1,411 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore.analyze;
+
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.SchemalessScan;
+import org.apache.drill.exec.planner.FileSystemPartitionDescriptor;
+import org.apache.drill.exec.planner.PartitionLocation;
+import org.apache.drill.exec.planner.logical.DrillRel;
+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.store.ColumnExplorer;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.FileSelection;
+import org.apache.drill.exec.store.dfs.FormatSelection;
+import org.apache.drill.exec.util.DrillFileSystemUtil;
+import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.drill.metastore.components.tables.BasicTablesRequests;
+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.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Multimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Streams;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of {@link MetadataInfoCollector} for file-based tables.
+ */
+public class FileMetadataInfoCollector implements MetadataInfoCollector {
+  private final List<MetadataInfo> allMetaToHandle;
+  private final List<MetadataInfo> metadataToRemove;
+
+  private final BasicTablesRequests basicRequests;
+  private final TableInfo tableInfo;
+  private final MetadataType metadataLevel;
+
+  private List<MetadataInfo> rowGroupsInfo = Collections.emptyList();
+  private List<MetadataInfo> filesInfo = Collections.emptyList();
+  private Multimap<Integer, MetadataInfo> segmentsInfo = ArrayListMultimap.create();
+
+  private TableScan tableScan;
+
+  private boolean outdated = true;
+
+  public FileMetadataInfoCollector(BasicTablesRequests basicRequests, TableInfo tableInfo, FormatSelection selection,
+      PlannerSettings settings, Supplier<TableScan> tableScanSupplier, List<SchemaPath> interestingColumns,
+      MetadataType metadataLevel, int segmentColumnsCount) throws IOException {
+    this.basicRequests = basicRequests;
+    this.tableInfo = tableInfo;
+    this.metadataLevel = metadataLevel;
+    this.allMetaToHandle = new ArrayList<>();
+    this.metadataToRemove = new ArrayList<>();
+    init(selection, settings, tableScanSupplier, interestingColumns, segmentColumnsCount);
+  }
+
+  @Override
+  public List<MetadataInfo> getRowGroupsInfo() {
+    return rowGroupsInfo;
+  }
+
+  @Override
+  public List<MetadataInfo> getFilesInfo() {
+    return filesInfo;
+  }
+
+  @Override
+  public Multimap<Integer, MetadataInfo> getSegmentsInfo() {
+    return segmentsInfo;
+  }
+
+  @Override
+  public List<MetadataInfo> getAllMetaToHandle() {
+    return allMetaToHandle;
+  }
+
+  @Override
+  public List<MetadataInfo> getMetadataToRemove() {
+    return metadataToRemove;
+  }
+
+  @Override
+  public TableScan getPrunedScan() {
+    return tableScan;
+  }
+
+  @Override
+  public boolean isOutdated() {
+    return outdated;
+  }
+
+  private void init(FormatSelection selection, PlannerSettings settings, Supplier<TableScan> tableScanSupplier,
+      List<SchemaPath> interestingColumns, int segmentColumnsCount) throws IOException {
+    List<SchemaPath> metastoreInterestingColumns =
+        Optional.ofNullable(basicRequests.interestingColumnsAndPartitionKeys(tableInfo).interestingColumns())
+            .map(metastoreInterestingColumnNames -> metastoreInterestingColumnNames.stream()
+                .map(SchemaPath::parseFromString)
+                .collect(Collectors.toList()))
+        .orElse(null);
+
+    Map<String, Long> filesNamesLastModifiedTime = basicRequests.filesLastModifiedTime(tableInfo, null, null);
+
+    List<String> newFiles = new ArrayList<>();
+    List<String> updatedFiles = new ArrayList<>();
+    List<String> removedFiles = new ArrayList<>(filesNamesLastModifiedTime.keySet());
+    List<String> allFiles = new ArrayList<>();
+
+    for (FileStatus fileStatus : getFileStatuses(selection)) {
+      String path = Path.getPathWithoutSchemeAndAuthority(fileStatus.getPath()).toUri().getPath();
+      Long lastModificationTime = filesNamesLastModifiedTime.get(path);
+      if (lastModificationTime == null) {
+        newFiles.add(path);
+      } else if (lastModificationTime < fileStatus.getModificationTime()) {
+        updatedFiles.add(path);
+      }
+      removedFiles.remove(path);
+      allFiles.add(path);
+    }
+
+    String selectionRoot = selection.getSelection().getSelectionRoot().toUri().getPath();
+
+    if (!Objects.equals(metastoreInterestingColumns, interestingColumns)
+        && (metastoreInterestingColumns == null || !metastoreInterestingColumns.containsAll(interestingColumns))
+        || TableStatisticsKind.ANALYZE_METADATA_LEVEL.getValue(basicRequests.tableMetadata(tableInfo)).compareTo(metadataLevel) != 0) {
+      // do not update table scan and lists of segments / files / row groups,
+      // metadata should be recalculated
+      tableScan = tableScanSupplier.get();
+      metadataToRemove.addAll(getMetadataInfoList(selectionRoot, removedFiles, MetadataType.SEGMENT, 0));
+      return;
+    }
+
+    // checks whether there are no new, updated and removed files
+    if (!newFiles.isEmpty() || !updatedFiles.isEmpty() || !removedFiles.isEmpty()) {
+      List<String> scanFiles = new ArrayList<>(newFiles);
+      scanFiles.addAll(updatedFiles);
+
+      // updates scan to read updated / new files
+      tableScan = getTableScan(settings, tableScanSupplier.get(), scanFiles);
+
+      // iterates from the end;
+      // takes deepest updated segments;
+      // finds their parents:
+      //  - fetches all segments for parent level;
+      //  - filters segments to leave parents only;
+      // obtains all child segments;
+      // filters child segments for filtered parent segments
+
+      int lastSegmentIndex = segmentColumnsCount - 1;
+
+      List<String> scanAndRemovedFiles = new ArrayList<>(scanFiles);
+      scanAndRemovedFiles.addAll(removedFiles);
+
+      // 1. Obtain files info for files from the same folder without removed files
+      // 2. Get segments for obtained files + segments for removed files
+      // 3. Get parent segments
+      // 4. Get other segments for the same parent segment
+      // 5. Remove segments which have only removed files (matched for removedFileInfo and don't match to filesInfo)
+      // 6. Do the same for parent segments
+
+      List<MetadataInfo> allFilesInfo = getMetadataInfoList(selectionRoot, allFiles, MetadataType.FILE, 0);
+
+      // first pass: collect updated segments even without files, they will be removed later
+      List<MetadataInfo> leafSegments = getMetadataInfoList(selectionRoot, scanAndRemovedFiles, MetadataType.SEGMENT, lastSegmentIndex);
+      List<MetadataInfo> removedFilesMetadata = getMetadataInfoList(selectionRoot, removedFiles, MetadataType.FILE, 0);
+
+      List<MetadataInfo> scanFilesInfo = getMetadataInfoList(selectionRoot, scanAndRemovedFiles, MetadataType.FILE, 0);
+      // files from scan + files from the same folder without removed files
+      filesInfo = leafSegments.stream()
+          .filter(parent -> scanFilesInfo.stream().anyMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier())))
+          .flatMap(parent ->
+              allFilesInfo.stream()
+                  .filter(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier())))
+          .collect(Collectors.toList());
+
+      Multimap<Integer, MetadataInfo> allSegments = populateSegments(removedFiles, allFiles, selectionRoot, lastSegmentIndex, leafSegments, removedFilesMetadata);
+
+      List<MetadataInfo> allRowGroupsInfo = getAllRowGroupsMetadataInfos(allFiles);
+
+      rowGroupsInfo = allRowGroupsInfo.stream()
+          .filter(child -> filesInfo.stream()
+              .map(MetadataInfo::identifier)
+              .anyMatch(parent -> MetadataIdentifierUtils.isMetadataKeyParent(parent, child.identifier())))
+          .collect(Collectors.toList());
+
+      List<MetadataInfo> segmentsToUpdate = getMetadataInfoList(selectionRoot, scanAndRemovedFiles, MetadataType.SEGMENT, 0);
+      Streams.concat(allSegments.values().stream(), allFilesInfo.stream(), allRowGroupsInfo.stream())
+          .filter(child -> segmentsToUpdate.stream().anyMatch(parent -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier())))
+          .filter(parent ->
+              removedFilesMetadata.stream().noneMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier()))
+                  || filesInfo.stream().anyMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier())))
+          .forEach(allMetaToHandle::add);
+
+      allMetaToHandle.addAll(segmentsToUpdate);
+
+      // removed top-level segments are handled separately since their metadata is not overridden when producing writing to the Metastore
+      List<MetadataInfo> removedTopSegments = getMetadataInfoList(selectionRoot, removedFiles, MetadataType.SEGMENT, 0).stream()
+          .filter(parent ->
+              removedFilesMetadata.stream().anyMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier()))
+                  && allFilesInfo.stream().noneMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier())))
+          .collect(Collectors.toList());
+      metadataToRemove.addAll(removedTopSegments);
+    } else {
+      // table metadata may still be actual
+      outdated = false;
+    }
+  }
+
+  private Multimap<Integer, MetadataInfo> populateSegments(List<String> removedFiles, List<String> allFiles,
+      String selectionRoot, int lastSegmentIndex, List<MetadataInfo> leafSegments, List<MetadataInfo> removedFilesMetadata) {
+    List<String> presentAndRemovedFiles = new ArrayList<>(allFiles);
+    presentAndRemovedFiles.addAll(removedFiles);
+    Multimap<Integer, MetadataInfo> allSegments = ArrayListMultimap.create();
+    if (lastSegmentIndex > 0) {
+      allSegments.putAll(lastSegmentIndex, getMetadataInfoList(selectionRoot, presentAndRemovedFiles, MetadataType.SEGMENT, lastSegmentIndex));
+    }
+
+    for (int i = lastSegmentIndex - 1; i >= 0; i--) {
+      List<MetadataInfo> currentChildSegments = leafSegments;
+      List<MetadataInfo> allParentSegments = getMetadataInfoList(selectionRoot, presentAndRemovedFiles, MetadataType.SEGMENT, i);
+      allSegments.putAll(i, allParentSegments);
+
+      // segments, parent for segments from currentChildSegments
+      List<MetadataInfo> parentSegments = allParentSegments.stream()
+          .filter(parent -> currentChildSegments.stream().anyMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier())))
+          .collect(Collectors.toList());
+
+      // all segments children for parentSegments segments except empty segments
+      List<MetadataInfo> childSegments = allSegments.get(i + 1).stream()
+          .filter(child -> parentSegments.stream().anyMatch(parent -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier())))
+          .filter(parent ->
+              removedFilesMetadata.stream().noneMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier()))
+                  || filesInfo.stream().anyMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier())))
+          .collect(Collectors.toList());
+
+      segmentsInfo.putAll(i + 1, childSegments);
+      leafSegments = childSegments;
+    }
+    segmentsInfo.putAll(0, getMetadataInfoList(selectionRoot, presentAndRemovedFiles, MetadataType.SEGMENT, 0).stream()
+        .filter(parent ->
+            removedFilesMetadata.stream().noneMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier()))
+                || filesInfo.stream().anyMatch(child -> MetadataIdentifierUtils.isMetadataKeyParent(parent.identifier(), child.identifier())))
+        .collect(Collectors.toList()));
+    return allSegments;
+  }
+
+  private List<MetadataInfo> getAllRowGroupsMetadataInfos(List<String> allFiles) {
+    List<String> metadataKeys = filesInfo.stream()
+        .map(MetadataInfo::key)
+        .distinct()
+        .collect(Collectors.toList());
+
+    BasicTablesRequests.RequestMetadata requestMetadata = BasicTablesRequests.RequestMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataKeys(metadataKeys)
+        .paths(allFiles)
+        .metadataType(MetadataType.ROW_GROUP.name())
+        .requestColumns(Arrays.asList(MetadataInfo.METADATA_KEY, MetadataInfo.METADATA_IDENTIFIER, MetadataInfo.METADATA_TYPE))
+        .build();
+
+    return basicRequests.request(requestMetadata).stream()
+        .map(unit -> MetadataInfo.builder().metadataUnit(unit).build())
+        .collect(Collectors.toList());
+  }
+
+  private List<FileStatus> getFileStatuses(FormatSelection selection) throws IOException {
+    FileSelection fileSelection = selection.getSelection();
+
+    FileSystem rawFs = fileSelection.getSelectionRoot().getFileSystem(new Configuration());
+    DrillFileSystem fs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), rawFs.getConf());
+
+    return getFileStatuses(fileSelection, fs);
+  }
+
+  private TableScan getTableScan(PlannerSettings settings, TableScan scanRel, List<String> scanFiles) {
+    FileSystemPartitionDescriptor descriptor =
+        new FileSystemPartitionDescriptor(settings, scanRel);
+
+    List<PartitionLocation> newPartitions = Lists.newArrayList(descriptor.iterator()).stream()
+        .flatMap(Collection::stream)
+        .flatMap(p -> p.getPartitionLocationRecursive().stream())
+        .filter(p -> scanFiles.contains(p.getEntirePartitionLocation().toUri().getPath()))
+        .collect(Collectors.toList());
+
+    try {
+      if (!newPartitions.isEmpty()) {
+        return descriptor.createTableScan(newPartitions, false);
+      } else {
+        DrillTable drillTable = descriptor.getTable();
+        SchemalessScan scan = new SchemalessScan(drillTable.getUserName(), ((FormatSelection) descriptor.getTable().getSelection()).getSelection().getSelectionRoot());
+
+        return new DrillScanRel(scanRel.getCluster(),
+            scanRel.getTraitSet().plus(DrillRel.DRILL_LOGICAL),
+            scanRel.getTable(),
+            scan,
+            scanRel.getRowType(),
+            DrillScanRel.getProjectedColumns(scanRel.getTable(), true),
+            true /*filter pushdown*/);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("Error happened during recreation of pruned scan", e);
+    }
+  }
+
+  private List<MetadataInfo> getMetadataInfoList(String parent, List<String> locations, MetadataType metadataType, int level) {
+    return locations.stream()
+        .map(location -> getMetadataInfo(parent, location, metadataType, level))
+        .distinct()
+        .collect(Collectors.toList());
+  }
+
+  private MetadataInfo getMetadataInfo(String parent, String location, MetadataType metadataType, int level) {
+    List<String> values = ColumnExplorer.listPartitionValues(new Path(location), new Path(parent), true);
+
+    switch (metadataType) {
+      case ROW_GROUP: {
+        throw new UnsupportedOperationException("MetadataInfo cannot be obtained for row group using file location only");
+      }
+      case FILE: {
+        String key = values.size() > 1 ? values.iterator().next() : MetadataInfo.DEFAULT_SEGMENT_KEY;
+        return MetadataInfo.builder()
+            .type(metadataType)
+            .key(key)
+            .identifier(MetadataIdentifierUtils.getMetadataIdentifierKey(values))
+            .build();
+      }
+      case SEGMENT: {
+        String key = values.size() > 1 ? values.iterator().next() : MetadataInfo.DEFAULT_SEGMENT_KEY;
+        return MetadataInfo.builder()
+            .type(metadataType)
+            .key(key)
+            .identifier(values.size() > 1 ? MetadataIdentifierUtils.getMetadataIdentifierKey(values.subList(0, level + 1)) :  MetadataInfo.DEFAULT_SEGMENT_KEY)
+            .build();
+      }
+      case TABLE: {
+        return MetadataInfo.builder()
+            .type(metadataType)
+            .key(MetadataInfo.GENERAL_INFO_KEY)
+            .build();
+      }
+      default:
+        throw new UnsupportedOperationException(metadataType.name());
+    }
+  }
+
+  /**
+   * Returns list of {@link FileStatus} file statuses obtained from specified {@link FileSelection} file selection.
+   * Specified file selection may be expanded fully if it wasn't expanded before.
+   *
+   * @param fileSelection file selection
+   * @param fs            file system
+   * @return list of {@link FileStatus} file statuses
+   */
+  public static List<FileStatus> getFileStatuses(FileSelection fileSelection, DrillFileSystem fs) throws IOException {
+    if (!fileSelection.isExpandedFully()) {
+      fileSelection = getExpandedFileSelection(fileSelection, fs);
+    }
+    return fileSelection.getStatuses(fs);
+  }
+
+  /**
+   * Returns {@link FileSelection} file selection based on specified file selection with expanded file statuses.
+   *
+   * @param fileSelection file selection
+   * @return expanded file selection
+   */
+  public static FileSelection getExpandedFileSelection(FileSelection fileSelection) throws IOException {
+    FileSystem rawFs = fileSelection.getSelectionRoot().getFileSystem(new Configuration());
+    FileSystem fs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), rawFs.getConf());
+    return getExpandedFileSelection(fileSelection, fs);
+  }
+
+  private static FileSelection getExpandedFileSelection(FileSelection fileSelection, FileSystem fs) throws IOException {
+    List<FileStatus> fileStatuses = DrillFileSystemUtil.listFiles(fs, fileSelection.getSelectionRoot(), true);
+    fileSelection = FileSelection.create(fileStatuses, null, fileSelection.getSelectionRoot());
+    return fileSelection;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataAggregateContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataAggregateContext.java
new file mode 100644
index 0000000..46f6df5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataAggregateContext.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore.analyze;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.data.NamedExpression;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.StringJoiner;
+
+/**
+ * Class which provides information required for producing metadata aggregation when performing analyze.
+ */
+@JsonDeserialize(builder = MetadataAggregateContext.MetadataAggregateContextBuilder.class)
+public class MetadataAggregateContext {
+  private final List<NamedExpression> groupByExpressions;
+  private final List<SchemaPath> interestingColumns;
+  private final List<SchemaPath> excludedColumns;
+  private final boolean createNewAggregations;
+
+  public MetadataAggregateContext(MetadataAggregateContextBuilder builder) {
+    this.groupByExpressions = builder.groupByExpressions;
+    this.interestingColumns = builder.interestingColumns;
+    this.createNewAggregations = builder.createNewAggregations;
+    this.excludedColumns = builder.excludedColumns;
+  }
+
+  @JsonProperty
+  public List<NamedExpression> groupByExpressions() {
+    return groupByExpressions;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> interestingColumns() {
+    return interestingColumns;
+  }
+
+  @JsonProperty
+  public boolean createNewAggregations() {
+    return createNewAggregations;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> excludedColumns() {
+    return excludedColumns;
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(",\n", MetadataAggregateContext.class.getSimpleName() + "[", "]")
+        .add("groupByExpressions=" + groupByExpressions)
+        .add("interestingColumns=" + interestingColumns)
+        .add("createNewAggregations=" + createNewAggregations)
+        .add("excludedColumns=" + excludedColumns)
+        .toString();
+  }
+
+  public static MetadataAggregateContextBuilder builder() {
+    return new MetadataAggregateContextBuilder();
+  }
+
+  @JsonPOJOBuilder(withPrefix = "")
+  public static class MetadataAggregateContextBuilder {
+    private List<NamedExpression> groupByExpressions;
+    private List<SchemaPath> interestingColumns;
+    private Boolean createNewAggregations;
+    private List<SchemaPath> excludedColumns;
+
+    public MetadataAggregateContextBuilder groupByExpressions(List<NamedExpression> groupByExpressions) {
+      this.groupByExpressions = groupByExpressions;
+      return this;
+    }
+
+    public MetadataAggregateContextBuilder interestingColumns(List<SchemaPath> interestingColumns) {
+      this.interestingColumns = interestingColumns;
+      return this;
+    }
+
+    public MetadataAggregateContextBuilder createNewAggregations(boolean createNewAggregations) {
+      this.createNewAggregations = createNewAggregations;
+      return this;
+    }
+
+    public MetadataAggregateContextBuilder excludedColumns(List<SchemaPath> excludedColumns) {
+      this.excludedColumns = excludedColumns;
+      return this;
+    }
+
+    public MetadataAggregateContext build() {
+      Objects.requireNonNull(groupByExpressions, "groupByExpressions were not set");
+      Objects.requireNonNull(createNewAggregations, "createNewAggregations was not set");
+      Objects.requireNonNull(excludedColumns, "excludedColumns were not set");
+      return new MetadataAggregateContext(this);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataControllerContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataControllerContext.java
new file mode 100644
index 0000000..157d3a0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataControllerContext.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore.analyze;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.metastore.components.tables.MetastoreTableInfo;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.StringJoiner;
+
+/**
+ * Class which provides information required for storing metadata to the Metastore when performing analyze.
+ */
+@JsonDeserialize(builder = MetadataControllerContext.MetadataControllerContextBuilder.class)
+public class MetadataControllerContext {
+  private final TableInfo tableInfo;
+  private final MetastoreTableInfo metastoreTableInfo;
+  private final Path location;
+  private final List<SchemaPath> interestingColumns;
+  private final List<String> segmentColumns;
+  private final List<MetadataInfo> metadataToHandle;
+  private final List<MetadataInfo> metadataToRemove;
+  private final MetadataType analyzeMetadataLevel;
+
+  private MetadataControllerContext(MetadataControllerContextBuilder builder) {
+    this.tableInfo = builder.tableInfo;
+    this.metastoreTableInfo = builder.metastoreTableInfo;
+    this.location = builder.location;
+    this.interestingColumns = builder.interestingColumns;
+    this.segmentColumns = builder.segmentColumns;
+    this.metadataToHandle = builder.metadataToHandle;
+    this.metadataToRemove = builder.metadataToRemove;
+    this.analyzeMetadataLevel = builder.analyzeMetadataLevel;
+  }
+
+  @JsonProperty
+  public TableInfo tableInfo() {
+    return tableInfo;
+  }
+
+  @JsonProperty
+  public MetastoreTableInfo metastoreTableInfo() {
+    return metastoreTableInfo;
+  }
+
+  @JsonProperty
+  public Path location() {
+    return location;
+  }
+
+  @JsonProperty
+  public List<SchemaPath> interestingColumns() {
+    return interestingColumns;
+  }
+
+  @JsonProperty
+  public List<String> segmentColumns() {
+    return segmentColumns;
+  }
+
+  @JsonProperty
+  public List<MetadataInfo> metadataToHandle() {
+    return metadataToHandle;
+  }
+
+  @JsonProperty
+  public List<MetadataInfo> metadataToRemove() {
+    return metadataToRemove;
+  }
+
+  @JsonProperty
+  public MetadataType analyzeMetadataLevel() {
+    return analyzeMetadataLevel;
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(",\n", MetadataControllerContext.class.getSimpleName() + "[", "]")
+        .add("tableInfo=" + tableInfo)
+        .add("location=" + location)
+        .add("interestingColumns=" + interestingColumns)
+        .add("segmentColumns=" + segmentColumns)
+        .add("metadataToHandle=" + metadataToHandle)
+        .add("metadataToRemove=" + metadataToRemove)
+        .add("analyzeMetadataLevel=" + analyzeMetadataLevel)
+        .toString();
+  }
+
+  public static MetadataControllerContextBuilder builder() {
+    return new MetadataControllerContextBuilder();
+  }
+
+  @JsonPOJOBuilder(withPrefix = "")
+  public static class MetadataControllerContextBuilder {
+    private TableInfo tableInfo;
+    private MetastoreTableInfo metastoreTableInfo;
+    private Path location;
+    private List<SchemaPath> interestingColumns;
+    private List<String> segmentColumns;
+    private List<MetadataInfo> metadataToHandle;
+    private List<MetadataInfo> metadataToRemove;
+    private MetadataType analyzeMetadataLevel;
+
+    public MetadataControllerContextBuilder tableInfo(TableInfo tableInfo) {
+      this.tableInfo = tableInfo;
+      return this;
+    }
+
+    public MetadataControllerContextBuilder metastoreTableInfo(MetastoreTableInfo metastoreTableInfo) {
+      this.metastoreTableInfo = metastoreTableInfo;
+      return this;
+    }
+
+    public MetadataControllerContextBuilder location(Path location) {
+      this.location = location;
+      return this;
+    }
+
+    public MetadataControllerContextBuilder interestingColumns(List<SchemaPath> interestingColumns) {
+      this.interestingColumns = interestingColumns;
+      return this;
+    }
+
+    public MetadataControllerContextBuilder segmentColumns(List<String> segmentColumns) {
+      this.segmentColumns = segmentColumns;
+      return this;
+    }
+
+    public MetadataControllerContextBuilder metadataToHandle(List<MetadataInfo> metadataToHandle) {
+      this.metadataToHandle = metadataToHandle;
+      return this;
+    }
+
+    public MetadataControllerContextBuilder metadataToRemove(List<MetadataInfo> metadataToRemove) {
+      this.metadataToRemove = metadataToRemove;
+      return this;
+    }
+
+    public MetadataControllerContextBuilder analyzeMetadataLevel(MetadataType metadataType) {
+      this.analyzeMetadataLevel = metadataType;
+      return this;
+    }
+
+    public MetadataControllerContext build() {
+      Objects.requireNonNull(tableInfo, "tableInfo was not set");
+      Objects.requireNonNull(location, "location was not set");
+      Objects.requireNonNull(segmentColumns, "segmentColumns were not set");
+      Objects.requireNonNull(metadataToHandle, "metadataToHandle was not set");
+      Objects.requireNonNull(metadataToRemove, "metadataToRemove was not set");
+      return new MetadataControllerContext(this);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataHandlerContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataHandlerContext.java
new file mode 100644
index 0000000..147deeb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataHandlerContext.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore.analyze;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.TableInfo;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.StringJoiner;
+
+/**
+ * Class which provides information required for handling results of metadata aggregation when performing analyze.
+ */
+@JsonDeserialize(builder = MetadataHandlerContext.MetadataHandlerContextBuilder.class)
+public class MetadataHandlerContext {
+  private final TableInfo tableInfo;
+  private final List<MetadataInfo> metadataToHandle;
+  private final MetadataType metadataType;
+  private final int depthLevel;
+  private final List<String> segmentColumns;
+
+  private MetadataHandlerContext(MetadataHandlerContextBuilder builder) {
+    this.tableInfo = builder.tableInfo;
+    this.metadataToHandle = builder.metadataToHandle;
+    this.metadataType = builder.metadataType;
+    this.depthLevel = builder.depthLevel;
+    this.segmentColumns = builder.segmentColumns;
+  }
+
+  @JsonProperty
+  public TableInfo tableInfo() {
+    return tableInfo;
+  }
+
+  @JsonProperty
+  public List<MetadataInfo> metadataToHandle() {
+    return metadataToHandle;
+  }
+
+  @JsonProperty
+  public MetadataType metadataType() {
+    return metadataType;
+  }
+
+  @JsonProperty
+  public int depthLevel() {
+    return depthLevel;
+  }
+
+  @JsonProperty
+  public List<String> segmentColumns() {
+    return segmentColumns;
+  }
+
+  @Override
+  public String toString() {
+    return new StringJoiner(",\n", MetadataHandlerContext.class.getSimpleName() + "[", "]")
+        .add("tableInfo=" + tableInfo)
+        .add("metadataToHandle=" + metadataToHandle)
+        .add("metadataType=" + metadataType)
+        .add("depthLevel=" + depthLevel)
+        .add("segmentColumns=" + segmentColumns)
+        .toString();
+  }
+
+  public static MetadataHandlerContextBuilder builder() {
+    return new MetadataHandlerContextBuilder();
+  }
+
+  @JsonPOJOBuilder(withPrefix = "")
+  public static class MetadataHandlerContextBuilder {
+    private TableInfo tableInfo;
+    private List<MetadataInfo> metadataToHandle;
+    private MetadataType metadataType;
+    private Integer depthLevel;
+    private List<String> segmentColumns;
+
+    public MetadataHandlerContextBuilder tableInfo(TableInfo tableInfo) {
+      this.tableInfo = tableInfo;
+      return this;
+    }
+
+    public MetadataHandlerContextBuilder metadataToHandle(List<MetadataInfo> metadataToHandle) {
+      this.metadataToHandle = metadataToHandle;
+      return this;
+    }
+
+    public MetadataHandlerContextBuilder metadataType(MetadataType metadataType) {
+      this.metadataType = metadataType;
+      return this;
+    }
+
+    public MetadataHandlerContextBuilder depthLevel(int depthLevel) {
+      this.depthLevel = depthLevel;
+      return this;
+    }
+
+    public MetadataHandlerContextBuilder segmentColumns(List<String> segmentColumns) {
+      this.segmentColumns = segmentColumns;
+      return this;
+    }
+
+    public MetadataHandlerContext build() {
+      Objects.requireNonNull(tableInfo, "tableInfo was not set");
+      Objects.requireNonNull(metadataToHandle, "metadataToHandle was not set");
+      Objects.requireNonNull(metadataType, "metadataType was not set");
+      Objects.requireNonNull(depthLevel, "depthLevel was not set");
+      Objects.requireNonNull(segmentColumns, "segmentColumns were not set");
+      return new MetadataHandlerContext(this);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataIdentifierUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataIdentifierUtils.java
new file mode 100644
index 0000000..a0eef71
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataIdentifierUtils.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore.analyze;
+
+import org.apache.drill.exec.store.ColumnExplorer;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.hadoop.fs.Path;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class MetadataIdentifierUtils {
+  private static final String METADATA_IDENTIFIER_SEPARATOR = "/";
+
+  /**
+   * Returns string representation of metadata identifier using specified metadata identifier values.
+   *
+   * @param values metadata identifier values
+   * @return string representation of metadata identifier
+   */
+  public static String getMetadataIdentifierKey(List<String> values) {
+    return String.join(METADATA_IDENTIFIER_SEPARATOR, values);
+  }
+
+  /**
+   * Checks whether the specified metadata identifier is a parent for another specified metadata identifier.
+   *
+   * @param parent parent metadata identifier
+   * @param child  child metadata identifier
+   * @return {@code true} if specified metadata identifier is a parent for another specified metadata identifier
+   */
+  public static boolean isMetadataKeyParent(String parent, String child) {
+    return child.startsWith(parent + METADATA_IDENTIFIER_SEPARATOR) || parent.equals(MetadataInfo.DEFAULT_SEGMENT_KEY);
+  }
+
+  /**
+   * Returns file metadata identifier.
+   *
+   * @param partitionValues partition values
+   * @param path            file path
+   * @return file metadata identifier
+   */
+  public static String getFileMetadataIdentifier(List<String> partitionValues, Path path) {
+    List<String> identifierValues = new ArrayList<>(partitionValues);
+    identifierValues.add(ColumnExplorer.ImplicitFileColumns.FILENAME.getValue(path));
+    return getMetadataIdentifierKey(identifierValues);
+  }
+
+  /**
+   * Returns row group metadata identifier.
+   *
+   * @param partitionValues partition values
+   * @param path            file path
+   * @param index           row group index
+   * @return row group metadata identifier
+   */
+  public static String getRowGroupMetadataIdentifier(List<String> partitionValues, Path path, int index) {
+    List<String> identifierValues = new ArrayList<>(partitionValues);
+    identifierValues.add(ColumnExplorer.ImplicitFileColumns.FILENAME.getValue(path));
+    identifierValues.add(Integer.toString(index));
+    return getMetadataIdentifierKey(identifierValues);
+  }
+
+  /**
+   * Returns array with metadata identifier values obtained from specified metadata identifier string.
+   *
+   * @param metadataIdentifier metadata identifier
+   * @return array with metadata identifier values
+   */
+  public static String[] getValuesFromMetadataIdentifier(String metadataIdentifier) {
+    return metadataIdentifier.split(METADATA_IDENTIFIER_SEPARATOR);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataInfoCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataInfoCollector.java
new file mode 100644
index 0000000..07f2237
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataInfoCollector.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.metastore.analyze;
+
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.drill.shaded.guava.com.google.common.collect.Multimap;
+
+import java.util.List;
+
+/**
+ * Interface for obtaining information about segments, files etc which should be handled in Metastore
+ * when producing incremental analyze.
+ */
+public interface MetadataInfoCollector {
+
+  /**
+   * Returns list of row groups metadata info which should be fetched from the Metastore.
+   *
+   * @return list of row groups metadata info
+   */
+  List<MetadataInfo> getRowGroupsInfo();
+
+  /**
+   * Returns list of files metadata info which should be fetched from the Metastore.
+   *
+   * @return list of files metadata info
+   */
+  List<MetadataInfo> getFilesInfo();
+
+  /**
+   * Returns list of segments metadata info which should be fetched from the Metastore.
+   *
+   * @return list of segments metadata info
+   */
+  Multimap<Integer, MetadataInfo> getSegmentsInfo();
+
+  /**
+   * Returns list of all metadata info instances which should be handled
+   * either producing analyze or when fetching from the Metastore.
+   *
+   * @return list of all metadata info
+   */
+  List<MetadataInfo> getAllMetaToHandle();
+
+  /**
+   * Returns list of all metadata info which corresponds to top-level segments and should be removed from the Metastore.
+   *
+   * @return list of all metadata info which should be removed
+   */
+  List<MetadataInfo> getMetadataToRemove();
+
+  /**
+   * Returns {@link TableScan} instance which will be used when produced incremental analyze.
+   * Table scan will contain minimal selection required for obtaining correct metadata.
+   *
+   * @return {@link TableScan} instance
+   */
+  TableScan getPrunedScan();
+
+  /**
+   * Returns true if table metadata is outdated.
+   *
+   * @return true if table metadata is outdated
+   */
+  boolean isOutdated();
+}
diff --git a/common/src/main/java/org/apache/drill/common/util/function/CheckedSupplier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetastoreAnalyzeConstants.java
similarity index 63%
copy from common/src/main/java/org/apache/drill/common/util/function/CheckedSupplier.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetastoreAnalyzeConstants.java
index 6063383..c80a685 100644
--- a/common/src/main/java/org/apache/drill/common/util/function/CheckedSupplier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetastoreAnalyzeConstants.java
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.common.util.function;
+package org.apache.drill.exec.metastore.analyze;
 
-/**
- * The java standard library does not provide a lambda function interface for functions that take no arguments,
- * but that throw an exception. So, we have to define our own here.
- * @param <T> The return type of the lambda function.
- * @param <E> The type of exception thrown by the lambda function.
- */
-@FunctionalInterface
-public interface CheckedSupplier<T, E extends Exception> {
-  T get() throws E;
+public interface MetastoreAnalyzeConstants {
+  String OK_FIELD_NAME = "ok";
+  String SUMMARY_FIELD_NAME = "summary";
+
+  String COLLECTED_MAP_FIELD = "collectedMap";
+  String LOCATIONS_FIELD = "locations";
+  String SCHEMA_FIELD = "schema";
+  String METADATA_TYPE = "metadataType";
+  String LOCATION_FIELD = "location";
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index 97db9b2..3ff7ca6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -52,6 +52,7 @@ import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.exec.util.Utilities;
 
+import org.apache.drill.metastore.MetastoreRegistry;
 import org.apache.drill.shaded.guava.com.google.common.base.Function;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
@@ -360,11 +361,10 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
     }
   }
 
-  /*
-   * Clears the type {@link SqlStatementType} of the statement. Ideally we should not clear the statement type
-   * so this should never be exposed outside the QueryContext
+  /**
+   * Clears the type {@link SqlStatementType} of the statement.
    */
-  private void clearSQLStatementType() {
+  public void clearSQLStatementType() {
     this.stmtType = null;
   }
 
@@ -389,4 +389,8 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
   public boolean isSkipProfileWrite() {
     return skipProfileWrite;
   }
+
+  public MetastoreRegistry getMetastoreRegistry() {
+    return drillbitContext.getMetastoreRegistry();
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index defdd70..2551b6d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -141,7 +141,7 @@ public class BasicOptimizer extends Optimizer {
         input = new Sort(input, orderDefs, false);
       }
 
-      return new StreamingAggregate(input, groupBy.getKeys(), groupBy.getExprs(), 1.0f);
+      return new StreamingAggregate(input, groupBy.getKeys(), groupBy.getExprs());
     }
 
     @Override
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 bc21550..600a0bd 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,6 +22,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.drill.exec.metastore.analyze.AnalyzeInfoProvider;
 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;
@@ -207,4 +208,14 @@ public abstract class AbstractGroupScan extends AbstractBase implements GroupSca
   public TableMetadata getTableMetadata() {
     return null;
   }
+
+  @Override
+  public boolean usedMetastore() {
+    return false;
+  }
+
+  @Override
+  public AnalyzeInfoProvider getAnalyzeInfoProvider() {
+    return null;
+  }
 }
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 eecf992..80d3446 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
@@ -32,6 +32,7 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.compile.sig.ConstantExpressionIdentifier;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.expr.fn.FunctionLookupContext;
 import org.apache.drill.exec.expr.stat.RowsMatch;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
 import org.apache.drill.exec.ops.UdfUtilities;
@@ -73,6 +74,7 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
@@ -107,6 +109,8 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
   // whether all files, partitions or row groups of this group scan fully match the filter
   protected boolean matchAllMetadata = false;
 
+  protected boolean usedMetastore; // false by default
+
   protected AbstractGroupScanWithMetadata(String userName, List<SchemaPath> columns, LogicalExpression filter) {
     super(userName);
     this.columns = columns;
@@ -125,6 +129,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     this.partitions = that.partitions;
     this.segments = that.segments;
     this.files = that.files;
+    this.usedMetastore = that.usedMetastore;
     this.nonInterestingColumnsMetadata = that.nonInterestingColumnsMetadata;
     this.fileSet = that.fileSet == null ? null : new HashSet<>(that.fileSet);
   }
@@ -266,10 +271,12 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
       return null;
     }
 
-    final Set<SchemaPath> schemaPathsInExpr =
-        filterExpr.accept(new FilterEvaluatorUtils.FieldReferenceFinder(), null);
-
-    GroupScanWithMetadataFilterer filteredMetadata = getFilterer().getFiltered(optionManager, filterPredicate, schemaPathsInExpr);
+    GroupScanWithMetadataFilterer filteredMetadata = getFilterer()
+        .filterExpression(filterExpr)
+        .schema(tableMetadata.getSchema())
+        .context(functionImplementationRegistry)
+        .udfUtilities(udfUtilities)
+        .getFiltered(optionManager, filterPredicate);
 
     if (isGroupScanFullyMatchesFilter(filteredMetadata)) {
       logger.debug("applyFilter() does not have any pruning since GroupScan fully matches filter");
@@ -343,10 +350,10 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
   protected abstract GroupScanWithMetadataFilterer getFilterer();
 
   public FilterPredicate getFilterPredicate(LogicalExpression filterExpr,
-                                                   UdfUtilities udfUtilities,
-                                                   FunctionImplementationRegistry functionImplementationRegistry,
-                                                   OptionManager optionManager,
-                                                   boolean omitUnsupportedExprs) {
+      UdfUtilities udfUtilities,
+      FunctionLookupContext functionImplementationRegistry,
+      OptionManager optionManager,
+      boolean omitUnsupportedExprs) {
     return getFilterPredicate(filterExpr, udfUtilities, functionImplementationRegistry, optionManager,
             omitUnsupportedExprs, supportsFileImplicitColumns(), getTableMetadata().getSchema());
   }
@@ -365,15 +372,15 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
    * @return parquet filter predicate
    */
   public static FilterPredicate getFilterPredicate(LogicalExpression filterExpr,
-                                            UdfUtilities udfUtilities,
-                                            FunctionImplementationRegistry functionImplementationRegistry,
-                                            OptionManager optionManager,
-                                            boolean omitUnsupportedExprs,
-                                            boolean supportsFileImplicitColumns,
-                                            TupleMetadata schema) {
+      UdfUtilities udfUtilities,
+      FunctionLookupContext functionImplementationRegistry,
+      OptionManager optionManager,
+      boolean omitUnsupportedExprs,
+      boolean supportsFileImplicitColumns,
+      TupleMetadata schema) {
     TupleMetadata types = schema.copy();
 
-    Set<SchemaPath> schemaPathsInExpr = filterExpr.accept(new FilterEvaluatorUtils.FieldReferenceFinder(), null);
+    Set<SchemaPath> schemaPathsInExpr = filterExpr.accept(FilterEvaluatorUtils.FieldReferenceFinder.INSTANCE, null);
 
     // adds implicit or partition columns if they weren't added before.
     if (supportsFileImplicitColumns) {
@@ -588,6 +595,11 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     return segments;
   }
 
+  @Override
+  public boolean usedMetastore() {
+    return usedMetastore;
+  }
+
   @JsonIgnore
   public NonInterestingColumnsMetadata getNonInterestingColumnsMetadata() {
     if (nonInterestingColumnsMetadata == null) {
@@ -609,6 +621,11 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
     protected Map<Path, SegmentMetadata> segments = Collections.emptyMap();
     protected Map<Path, FileMetadata> files = Collections.emptyMap();
     protected NonInterestingColumnsMetadata nonInterestingColumnsMetadata;
+    // required for rebuilding filter expression for the case of schema change
+    protected LogicalExpression filterExpression;
+    protected TupleMetadata tableSchema;
+    protected UdfUtilities udfUtilities;
+    protected FunctionLookupContext context;
 
     // 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
@@ -661,6 +678,26 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
       return self();
     }
 
+    public B filterExpression(LogicalExpression filterExpression) {
+      this.filterExpression = filterExpression;
+      return self();
+    }
+
+    public B schema(TupleMetadata tableSchema) {
+      this.tableSchema = tableSchema;
+      return self();
+    }
+
+    public B udfUtilities(UdfUtilities udfUtilities) {
+      this.udfUtilities = udfUtilities;
+      return self();
+    }
+
+    public B context(FunctionLookupContext context) {
+      this.context = context;
+      return self();
+    }
+
     public boolean isMatchAllMetadata() {
       return matchAllMetadata;
     }
@@ -691,12 +728,18 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
      *
      * @param optionManager     option manager
      * @param filterPredicate   filter expression
-     * @param schemaPathsInExpr columns used in filter expression
      * @return this instance with filtered metadata
      */
     protected B getFiltered(OptionManager optionManager,
-                                                        FilterPredicate filterPredicate,
-                                                        Set<SchemaPath> schemaPathsInExpr) {
+        FilterPredicate filterPredicate) {
+      Objects.requireNonNull(filterExpression, "filterExpression was not set");
+      Objects.requireNonNull(tableSchema, "tableSchema was not set");
+      Objects.requireNonNull(udfUtilities, "udfUtilities were not set");
+      Objects.requireNonNull(context, "context was not set");
+
+      Set<SchemaPath> schemaPathsInExpr =
+          filterExpression.accept(FilterEvaluatorUtils.FieldReferenceFinder.INSTANCE, null);
+
       if (source.getTableMetadata() != null) {
         filterTableMetadata(filterPredicate, schemaPathsInExpr);
       }
@@ -902,13 +945,19 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
      * @param <T>               type of metadata to filter
      * @return filtered metadata
      */
-    public <T extends Metadata> 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) {
+        TupleMetadata schema = metadata.getSchema();
+        if (schema != null && !tableSchema.isEquivalent(schema)) {
+          filterPredicate = getFilterPredicate(filterExpression, udfUtilities,
+              context, optionManager, true, true, schema);
+        }
         Map<SchemaPath, ColumnStatistics> columnsStatistics = metadata.getColumnsStatistics();
 
         // adds partition (dir) column statistics if it may be used during filter evaluation
@@ -924,7 +973,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
         }
         RowsMatch match = FilterEvaluatorUtils.matches(filterPredicate,
             columnsStatistics, TableStatisticsKind.ROW_COUNT.getValue(metadata),
-            metadata.getSchema(), schemaPathsInExpr);
+            schema, schemaPathsInExpr);
         if (match == RowsMatch.NONE) {
           continue; // No file comply to the filter => drop the file
         }
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 ebbb717..72c26e9 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
@@ -24,6 +24,7 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.metastore.analyze.AnalyzeInfoProvider;
 import org.apache.drill.exec.ops.UdfUtilities;
 import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
@@ -171,4 +172,20 @@ public interface GroupScan extends Scan, HasAffinity {
 
   @JsonIgnore
   TableMetadata getTableMetadata();
+
+  /**
+   * Returns {@code true} if current group scan uses metadata obtained from the Metastore.
+   *
+   * @return {@code true} if current group scan uses metadata obtained from the Metastore, {@code false} otherwise.
+   */
+  @JsonIgnore
+  boolean usedMetastore();
+
+  /**
+   * Returns {@link AnalyzeInfoProvider} instance which will be used when running ANALYZE statement.
+   *
+   * @return {@link AnalyzeInfoProvider} instance
+   */
+  @JsonIgnore
+  AnalyzeInfoProvider getAnalyzeInfoProvider();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MetadataAggPOP.java
similarity index 57%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MetadataAggPOP.java
index 4a561bf..f31735c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MetadataAggPOP.java
@@ -17,43 +17,32 @@
  */
 package org.apache.drill.exec.physical.config;
 
-import java.util.List;
-
-import org.apache.drill.exec.physical.base.AbstractSingle;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.base.PhysicalVisitor;
-import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
-
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.metastore.analyze.MetadataAggregateContext;
 
-@JsonTypeName("unpivot-maps")
-public class UnpivotMaps extends AbstractSingle {
-  private final List<String> mapFieldsNames;
+import java.util.Collections;
 
-  @JsonCreator
-  public UnpivotMaps(@JsonProperty("child") PhysicalOperator child, List<String> mapFieldsNames) {
-    super(child);
-    this.mapFieldsNames = mapFieldsNames;
-  }
+@JsonTypeName("metadataAggregate")
+public class MetadataAggPOP extends StreamingAggregate {
+  private final MetadataAggregateContext context;
 
-  public List<String> getMapFieldNames() {
-    return mapFieldsNames;
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
-    return physicalVisitor.visitUnpivot(this, value);
+  @JsonCreator
+  public MetadataAggPOP(@JsonProperty("child") PhysicalOperator child,
+      @JsonProperty("context") MetadataAggregateContext context) {
+    super(child, context.groupByExpressions(), Collections.emptyList());
+    this.context = context;
   }
 
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    return new UnpivotMaps(child, mapFieldsNames);
+    return new MetadataAggPOP(child, context);
   }
 
-  @Override
-  public int getOperatorType() {
-    return CoreOperatorType.UNPIVOT_MAPS_VALUE;
+  @JsonProperty
+  public MetadataAggregateContext getContext() {
+    return context;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MetadataControllerPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MetadataControllerPOP.java
new file mode 100644
index 0000000..6b317ff
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MetadataControllerPOP.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.config;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.metastore.analyze.MetadataControllerContext;
+import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+@JsonTypeName("metadataController")
+public class MetadataControllerPOP extends AbstractBase {
+  private final MetadataControllerContext context;
+  private final PhysicalOperator left;
+  private final PhysicalOperator right;
+
+  @JsonCreator
+  public MetadataControllerPOP(@JsonProperty("left") PhysicalOperator left,
+      @JsonProperty("right") PhysicalOperator right, @JsonProperty("context") MetadataControllerContext context) {
+    this.context = context;
+    this.left = left;
+    this.right = right;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitOp(this, value);
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.size() == 2);
+    return new MetadataControllerPOP(children.get(0), children.get(1), context);
+  }
+
+  @Override
+  public int getOperatorType() {
+    return UserBitShared.CoreOperatorType.METADATA_CONTROLLER_VALUE;
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Arrays.asList(left, right).iterator();
+  }
+
+  @JsonProperty
+  public MetadataControllerContext getContext() {
+    return context;
+  }
+
+  @JsonProperty
+  public PhysicalOperator getLeft() {
+    return left;
+  }
+
+  @JsonProperty
+  public PhysicalOperator getRight() {
+    return right;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MetadataHandlerPOP.java
similarity index 69%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MetadataHandlerPOP.java
index 4a561bf..1599d90 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MetadataHandlerPOP.java
@@ -17,43 +17,42 @@
  */
 package org.apache.drill.exec.physical.config;
 
-import java.util.List;
-
-import org.apache.drill.exec.physical.base.AbstractSingle;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.base.PhysicalVisitor;
-import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
-
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.metastore.analyze.MetadataHandlerContext;
+import org.apache.drill.exec.proto.UserBitShared;
 
-@JsonTypeName("unpivot-maps")
-public class UnpivotMaps extends AbstractSingle {
-  private final List<String> mapFieldsNames;
+@JsonTypeName("metadataHandler")
+public class MetadataHandlerPOP extends AbstractSingle {
+  private final MetadataHandlerContext context;
 
   @JsonCreator
-  public UnpivotMaps(@JsonProperty("child") PhysicalOperator child, List<String> mapFieldsNames) {
+  public MetadataHandlerPOP(@JsonProperty("child") PhysicalOperator child,
+      @JsonProperty("context") MetadataHandlerContext context) {
     super(child);
-    this.mapFieldsNames = mapFieldsNames;
+    this.context = context;
   }
 
-  public List<String> getMapFieldNames() {
-    return mapFieldsNames;
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new MetadataHandlerPOP(child, context);
   }
 
   @Override
   public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
-    return physicalVisitor.visitUnpivot(this, value);
+    return physicalVisitor.visitOp(this, value);
   }
 
   @Override
-  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    return new UnpivotMaps(child, mapFieldsNames);
+  public int getOperatorType() {
+    return UserBitShared.CoreOperatorType.METADATA_HANDLER_VALUE;
   }
 
-  @Override
-  public int getOperatorType() {
-    return CoreOperatorType.UNPIVOT_MAPS_VALUE;
+  public MetadataHandlerContext getContext() {
+    return context;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
index 1fa30ab..58fee51 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
@@ -37,7 +37,7 @@ public class StatisticsAggregate extends StreamingAggregate {
   public StatisticsAggregate(
       @JsonProperty("child") PhysicalOperator child,
       @JsonProperty("functions") List<String> functions) {
-    super(child, null, null, 0.f);
+    super(child, null, null);
     this.functions = ImmutableList.copyOf(functions);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StreamingAggregate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StreamingAggregate.java
index c928b15..91956e0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StreamingAggregate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StreamingAggregate.java
@@ -37,14 +37,13 @@ public class StreamingAggregate extends AbstractSingle {
   private final List<NamedExpression> keys;
   private final List<NamedExpression> exprs;
 
-  private final float cardinality;
-
   @JsonCreator
-  public StreamingAggregate(@JsonProperty("child") PhysicalOperator child, @JsonProperty("keys") List<NamedExpression> keys, @JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("cardinality") float cardinality) {
+  public StreamingAggregate(@JsonProperty("child") PhysicalOperator child,
+      @JsonProperty("keys") List<NamedExpression> keys,
+      @JsonProperty("exprs") List<NamedExpression> exprs) {
     super(child);
     this.keys = keys;
     this.exprs = exprs;
-    this.cardinality = cardinality;
   }
 
   public List<NamedExpression> getKeys() {
@@ -62,7 +61,7 @@ public class StreamingAggregate extends AbstractSingle {
 
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    return new StreamingAggregate(child, keys, exprs, cardinality);
+    return new StreamingAggregate(child, keys, exprs);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
index 4a561bf..2210f7a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
@@ -30,16 +30,17 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 
 @JsonTypeName("unpivot-maps")
 public class UnpivotMaps extends AbstractSingle {
-  private final List<String> mapFieldsNames;
+  private final List<String> mapFieldNames;
 
   @JsonCreator
-  public UnpivotMaps(@JsonProperty("child") PhysicalOperator child, List<String> mapFieldsNames) {
+  public UnpivotMaps(@JsonProperty("child") PhysicalOperator child, @JsonProperty("mapFieldNames") List<String> mapFieldNames) {
     super(child);
-    this.mapFieldsNames = mapFieldsNames;
+    this.mapFieldNames = mapFieldNames;
   }
 
+  @JsonProperty
   public List<String> getMapFieldNames() {
-    return mapFieldsNames;
+    return mapFieldNames;
   }
 
   @Override
@@ -49,7 +50,7 @@ public class UnpivotMaps extends AbstractSingle {
 
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    return new UnpivotMaps(child, mapFieldsNames);
+    return new UnpivotMaps(child, mapFieldNames);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index 7d02c99..8fc7118 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -231,7 +231,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
       switch (lastKnownOutcome) {
         case NONE:
 
-          if (first && popConfig.getKeys().size() == 0) {
+          if (first && getKeyExpressions().size() == 0) {
             // if we have a straight aggregate and empty input batch, we need to handle it in a different way
             // Wewant to produce the special batch only if we got a NONE as the first outcome after
             // OK_NEW_SCHEMA. If we get a NONE immediately after we see an EMIT, then we have already handled
@@ -259,7 +259,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
         case EMIT:
           // if we get an EMIT with an empty batch as the first (and therefore only) batch
           // we have to do the special handling
-          if (firstBatchForDataSet && popConfig.getKeys().size() == 0 && incoming.getRecordCount() == 0) {
+          if (firstBatchForDataSet && getKeyExpressions().size() == 0 && incoming.getRecordCount() == 0) {
             constructSpecialBatch();
             // If outcome is NONE then we send the special batch in the first iteration and the NONE
             // outcome in the next iteration. If outcome is EMIT, we can send the special
@@ -311,7 +311,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
         return returnOutcome;
       case RETURN_AND_RESET:
         //WE could have got a string of batches, all empty, until we hit an emit
-        if (firstBatchForDataSet && popConfig.getKeys().size() == 0 && recordCount == 0) {
+        if (firstBatchForDataSet && getKeyExpressions().size() == 0 && recordCount == 0) {
           // if we have a straight aggregate and empty input batch, we need to handle it in a different way
           constructSpecialBatch();
           // If outcome is NONE then we send the special batch in the first iteration and the NONE
@@ -419,7 +419,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
            * buffer
            */
           throw new DrillRuntimeException("FixedWidth vectors is the expected output vector type. " +
-              "Corresponding expression: " + popConfig.getExprs().get(exprIndex).toString());
+              "Corresponding expression: " + getValueExpressions().get(exprIndex).toString());
         }
       }
       exprIndex++;
@@ -454,21 +454,21 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     complexWriters.add(writer);
   }
 
-  protected StreamingAggregator createAggregatorInternal() throws SchemaChangeException, ClassTransformationException, IOException{
+  protected StreamingAggregator createAggregatorInternal() throws SchemaChangeException, ClassTransformationException, IOException {
     ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getOptions());
     cg.getCodeGenerator().plainJavaCapable(true);
     // Uncomment out this line to debug the generated code.
     //cg.getCodeGenerator().saveCodeForDebugging(true);
     container.clear();
 
-    LogicalExpression[] keyExprs = new LogicalExpression[popConfig.getKeys().size()];
-    LogicalExpression[] valueExprs = new LogicalExpression[popConfig.getExprs().size()];
-    TypedFieldId[] keyOutputIds = new TypedFieldId[popConfig.getKeys().size()];
+    LogicalExpression[] keyExprs = new LogicalExpression[getKeyExpressions().size()];
+    LogicalExpression[] valueExprs = new LogicalExpression[getValueExpressions().size()];
+    TypedFieldId[] keyOutputIds = new TypedFieldId[getKeyExpressions().size()];
 
     ErrorCollector collector = new ErrorCollectorImpl();
 
     for (int i = 0; i < keyExprs.length; i++) {
-      NamedExpression ne = popConfig.getKeys().get(i);
+      NamedExpression ne = getKeyExpressions().get(i);
       LogicalExpression expr = ExpressionTreeMaterializer.materialize(ne.getExpr(), incoming, collector,context.getFunctionRegistry() );
       if (expr == null) {
         continue;
@@ -481,8 +481,8 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     }
 
     for (int i = 0; i < valueExprs.length; i++) {
-      NamedExpression ne = popConfig.getExprs().get(i);
-      LogicalExpression expr = ExpressionTreeMaterializer.materialize(ne.getExpr(), incoming, collector, context.getFunctionRegistry());
+      NamedExpression ne = getValueExpressions().get(i);
+      LogicalExpression expr = ExpressionTreeMaterializer.materialize(ne.getExpr(), incoming, collector, context.getFunctionRegistry(), true, false);
       if (expr instanceof IfExpression) {
         throw UserException.unsupportedError(new UnsupportedOperationException("Union type not supported in aggregate functions")).build(logger);
       }
@@ -534,6 +534,14 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     return agg;
   }
 
+  protected List<NamedExpression> getValueExpressions() {
+    return popConfig.getExprs();
+  }
+
+  protected List<NamedExpression> getKeyExpressions() {
+    return popConfig.getKeys();
+  }
+
   private final GeneratorMapping IS_SAME = GeneratorMapping.create("setupInterior", "isSame", null, null);
   private final MappingSet IS_SAME_I1 = new MappingSet("index1", null, IS_SAME, IS_SAME);
   private final MappingSet IS_SAME_I2 = new MappingSet("index2", null, IS_SAME, IS_SAME);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
index 4699b3e..e862c0e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 
 import com.carrotsearch.hppc.IntArrayList;
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.physical.config.HashJoinPOP;
 import org.apache.drill.exec.physical.impl.common.HashPartition;
 import org.apache.drill.exec.planner.common.JoinControl;
 import org.apache.drill.exec.record.BatchSchema;
@@ -137,7 +136,7 @@ public abstract class HashJoinProbeTemplate implements HashJoinProbe {
 
     partitionMask = numPartitions - 1; // e.g. 32 --> 0x1F
     bitsInMask = Integer.bitCount(partitionMask); // e.g. 0x1F -> 5
-    joinControl = new JoinControl(((HashJoinPOP)outgoingJoinBatch.getPopConfig()).getJoinControl());
+    joinControl = new JoinControl(outgoingJoinBatch.getPopConfig().getJoinControl());
 
     probeState = ProbeState.PROBE_PROJECT;
     this.recordsToProcess = 0;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinTemplate.java
index e13fd85..6bc2b7e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinTemplate.java
@@ -21,7 +21,6 @@ import javax.inject.Named;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.config.MergeJoinPOP;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.calcite.rel.core.JoinRelType;
 
@@ -41,7 +40,7 @@ public abstract class JoinTemplate implements JoinWorker {
    * @return  true of join succeeded; false if the worker needs to be regenerated
    */
   public final boolean doJoin(final JoinStatus status) {
-    final boolean isLeftJoin = (((MergeJoinPOP)status.outputBatch.getPopConfig()).getJoinType() == JoinRelType.LEFT);
+    final boolean isLeftJoin = status.outputBatch.getPopConfig().getJoinType() == JoinRelType.LEFT;
     status.setHasMoreData(false);
     while (!status.isOutgoingBatchFull()) {
       if (status.right.finished()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggBatch.java
new file mode 100644
index 0000000..51dfb15
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggBatch.java
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.metadata;
+
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.metastore.analyze.MetastoreAnalyzeConstants;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.MetadataAggPOP;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
+import org.apache.drill.exec.metastore.analyze.AnalyzeColumnUtils;
+import org.apache.drill.exec.planner.types.DrillRelDataTypeSystem;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.StreamSupport;
+
+/**
+ * Operator which adds aggregate calls for all incoming columns to calculate required metadata and produces aggregations.
+ * If aggregation is performed on top of another aggregation, required aggregate calls for merging metadata will be added.
+ */
+public class MetadataAggBatch extends StreamingAggBatch {
+
+  private List<NamedExpression> valueExpressions;
+
+  public MetadataAggBatch(MetadataAggPOP popConfig, RecordBatch incoming, FragmentContext context) throws OutOfMemoryException {
+    super(popConfig, incoming, context);
+  }
+
+  @Override
+  protected StreamingAggregator createAggregatorInternal()
+      throws SchemaChangeException, ClassTransformationException, IOException {
+    valueExpressions = new ArrayList<>();
+    MetadataAggPOP popConfig = (MetadataAggPOP) this.popConfig;
+
+    List<SchemaPath> excludedColumns = popConfig.getContext().excludedColumns();
+
+    BatchSchema schema = incoming.getSchema();
+    // Iterates through input expressions and adds aggregate calls for table fields
+    // to collect required statistics (MIN, MAX, COUNT, etc.) or aggregate calls to merge incoming metadata
+    getUnflattenedFileds(Lists.newArrayList(schema), null)
+        .forEach((fieldName, fieldRef) -> addColumnAggregateCalls(fieldRef, fieldName));
+
+    List<LogicalExpression> fieldsList = new ArrayList<>();
+    StreamSupport.stream(schema.spliterator(), false)
+        .map(MaterializedField::getName)
+        .filter(field -> !excludedColumns.contains(FieldReference.getWithQuotedRef(field)))
+        .forEach(filedName -> {
+          // adds string literal with field name to the list
+          fieldsList.add(ValueExpressions.getChar(filedName,
+              DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getDefaultPrecision(SqlTypeName.VARCHAR)));
+          // adds field reference to the list
+          fieldsList.add(FieldReference.getWithQuotedRef(filedName));
+        });
+
+    if (popConfig.getContext().createNewAggregations()) {
+      addMetadataAggregateCalls();
+      // infer schema from incoming data
+      addSchemaCall(fieldsList);
+      addNewMetadataAggregations();
+    } else {
+      addCollectListCall(fieldsList);
+      addMergeSchemaCall();
+    }
+
+    for (SchemaPath excludedColumn : excludedColumns) {
+      if (excludedColumn.equals(SchemaPath.getSimplePath(context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL)))
+          || excludedColumn.equals(SchemaPath.getSimplePath(context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL)))) {
+        LogicalExpression lastModifiedTime = new FunctionCall("any_value",
+            Collections.singletonList(
+                FieldReference.getWithQuotedRef(excludedColumn.getRootSegmentPath())),
+            ExpressionPosition.UNKNOWN);
+
+        valueExpressions.add(new NamedExpression(lastModifiedTime,
+            FieldReference.getWithQuotedRef(excludedColumn.getRootSegmentPath())));
+      }
+    }
+
+    addMaxLastModifiedCall();
+
+    return super.createAggregatorInternal();
+  }
+
+  /**
+   * Adds {@code max(`lastModifiedTime`)} function call to the value expressions list.
+   */
+  private void addMaxLastModifiedCall() {
+    String lastModifiedColumn = context.getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL);
+    LogicalExpression lastModifiedTime = new FunctionCall("max",
+        Collections.singletonList(
+            FieldReference.getWithQuotedRef(lastModifiedColumn)),
+        ExpressionPosition.UNKNOWN);
+
+    valueExpressions.add(new NamedExpression(lastModifiedTime,
+        FieldReference.getWithQuotedRef(lastModifiedColumn)));
+  }
+
+  /**
+   * Adds {@code collect_list()}} function call with specified fields list
+   * and excluded columns as arguments of this function to collect all these fields into repeated map list.
+   *
+   * @param fieldList list of the function arguments
+   */
+  private void addCollectListCall(List<LogicalExpression> fieldList) {
+    ArrayList<LogicalExpression> collectListArguments = new ArrayList<>(fieldList);
+    MetadataAggPOP popConfig = (MetadataAggPOP) this.popConfig;
+    List<SchemaPath> excludedColumns = popConfig.getContext().excludedColumns();
+    // populate columns which weren't included in the schema, but should be collected to the COLLECTED_MAP_FIELD
+    for (SchemaPath logicalExpressions : excludedColumns) {
+      // adds string literal with field name to the list
+      collectListArguments.add(ValueExpressions.getChar(logicalExpressions.getRootSegmentPath(),
+          DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getDefaultPrecision(SqlTypeName.VARCHAR)));
+      // adds field reference to the list
+      collectListArguments.add(FieldReference.getWithQuotedRef(logicalExpressions.getRootSegmentPath()));
+    }
+
+    LogicalExpression collectList = new FunctionCall("collect_list",
+        collectListArguments, ExpressionPosition.UNKNOWN);
+
+    valueExpressions.add(
+        new NamedExpression(collectList, FieldReference.getWithQuotedRef(MetastoreAnalyzeConstants.COLLECTED_MAP_FIELD)));
+  }
+
+  /**
+   * Adds {@code merge_schema(`schema`)}} function call to obtain schema common for underlying metadata parts.
+   */
+  private void addMergeSchemaCall() {
+    LogicalExpression schemaExpr = new FunctionCall("merge_schema",
+        Collections.singletonList(FieldReference.getWithQuotedRef(MetastoreAnalyzeConstants.SCHEMA_FIELD)),
+        ExpressionPosition.UNKNOWN);
+
+    valueExpressions.add(new NamedExpression(schemaExpr, FieldReference.getWithQuotedRef(MetastoreAnalyzeConstants.SCHEMA_FIELD)));
+  }
+
+  /**
+   * Adds {@code collect_to_list_varchar(`fqn`)} call to collect file paths into the list.
+   */
+  private void addNewMetadataAggregations() {
+    LogicalExpression locationsExpr = new FunctionCall("collect_to_list_varchar",
+        Collections.singletonList(SchemaPath.getSimplePath(context.getOptions().getString(ExecConstants.IMPLICIT_FQN_COLUMN_LABEL))),
+        ExpressionPosition.UNKNOWN);
+
+    valueExpressions.add(new NamedExpression(locationsExpr, FieldReference.getWithQuotedRef(MetastoreAnalyzeConstants.LOCATIONS_FIELD)));
+  }
+
+  /**
+   * Adds a call to {@code schema()}} function with specified fields list
+   * as arguments of this function to obtain their schema.
+   *
+   * @param fieldsList list of the function arguments
+   */
+  private void addSchemaCall(List<LogicalExpression> fieldsList) {
+    LogicalExpression schemaExpr = new FunctionCall("schema",
+        fieldsList, ExpressionPosition.UNKNOWN);
+
+    valueExpressions.add(new NamedExpression(schemaExpr, FieldReference.getWithQuotedRef(MetastoreAnalyzeConstants.SCHEMA_FIELD)));
+  }
+
+  /**
+   * Adds aggregate calls to calculate non-column metadata.
+   */
+  private void addMetadataAggregateCalls() {
+    AnalyzeColumnUtils.META_STATISTICS_FUNCTIONS.forEach((statisticsKind, sqlKind) -> {
+      LogicalExpression call = new FunctionCall(sqlKind.name(),
+          Collections.singletonList(ValueExpressions.getBigInt(1)), ExpressionPosition.UNKNOWN);
+      valueExpressions.add(
+          new NamedExpression(call,
+              FieldReference.getWithQuotedRef(AnalyzeColumnUtils.getMetadataStatisticsFieldName(statisticsKind))));
+    });
+  }
+
+  /**
+   * Returns map with field names as keys and field references as values. For the case when field is map,
+   * fully qualified child names will be present in this map.
+   * For example, for (a{b, c}, d) fields list will be returned map with a.b, a.c and d keys.
+   *
+   * @param fields       list of top-level fields to unflatten if required
+   * @param parentFields list of parent name segments
+   * @return map with field names as keys and field references as values
+   */
+  private Map<String, FieldReference> getUnflattenedFileds(Collection<MaterializedField> fields, List<String> parentFields) {
+    Map<String, FieldReference> fieldNameRefMap = new HashMap<>();
+    for (MaterializedField field : fields) {
+      // statistics collecting is not supported for array types
+      if (field.getType().getMode() != TypeProtos.DataMode.REPEATED) {
+        MetadataAggPOP popConfig = (MetadataAggPOP) this.popConfig;
+        List<SchemaPath> excludedColumns = popConfig.getContext().excludedColumns();
+        // excludedColumns are applied for root fields only
+        if (parentFields != null || !excludedColumns.contains(SchemaPath.getSimplePath(field.getName()))) {
+          List<String> currentPath;
+          if (parentFields == null) {
+            currentPath = Collections.singletonList(field.getName());
+          } else {
+            currentPath = new ArrayList<>(parentFields);
+            currentPath.add(field.getName());
+          }
+          if (field.getType().getMinorType() == TypeProtos.MinorType.MAP && popConfig.getContext().createNewAggregations()) {
+            fieldNameRefMap.putAll(getUnflattenedFileds(field.getChildren(), currentPath));
+          } else {
+            SchemaPath schemaPath = SchemaPath.getCompoundPath(currentPath.toArray(new String[0]));
+            // adds backticks for popConfig.createNewAggregations() to ensure that field will be parsed correctly
+            String name = popConfig.getContext().createNewAggregations() ? schemaPath.toExpr() : schemaPath.getRootSegmentPath();
+            fieldNameRefMap.put(name, new FieldReference(schemaPath));
+          }
+        }
+      }
+    }
+
+    return fieldNameRefMap;
+  }
+
+  /**
+   * Adds aggregate calls to calculate column metadata either from column data itself
+   * or from previously calculated metadata.
+   *
+   * @param fieldRef  field reference
+   * @param fieldName field name
+   */
+  private void addColumnAggregateCalls(FieldReference fieldRef, String fieldName) {
+    MetadataAggPOP popConfig = (MetadataAggPOP) this.popConfig;
+    List<SchemaPath> interestingColumns = popConfig.getContext().interestingColumns();
+    if (popConfig.getContext().createNewAggregations()) {
+      if (interestingColumns == null || interestingColumns.contains(fieldRef)) {
+        // collect statistics for all or only interesting columns if they are specified
+        AnalyzeColumnUtils.COLUMN_STATISTICS_FUNCTIONS.forEach((statisticsKind, sqlKind) -> {
+          LogicalExpression call = new FunctionCall(sqlKind.name(),
+              Collections.singletonList(fieldRef), ExpressionPosition.UNKNOWN);
+          valueExpressions.add(
+              new NamedExpression(call,
+                  FieldReference.getWithQuotedRef(AnalyzeColumnUtils.getColumnStatisticsFieldName(fieldName, statisticsKind))));
+        });
+      }
+    } else if (AnalyzeColumnUtils.isColumnStatisticsField(fieldName)
+        || AnalyzeColumnUtils.isMetadataStatisticsField(fieldName)) {
+      SqlKind function = AnalyzeColumnUtils.COLUMN_STATISTICS_FUNCTIONS.get(
+          AnalyzeColumnUtils.getStatisticsKind(fieldName));
+      if (function == SqlKind.COUNT) {
+        // for the case when aggregation was done, call SUM function for the results of COUNT aggregate call
+        function = SqlKind.SUM;
+      }
+      LogicalExpression functionCall = new FunctionCall(function.name(),
+          Collections.singletonList(fieldRef), ExpressionPosition.UNKNOWN);
+      valueExpressions.add(new NamedExpression(functionCall, fieldRef));
+    }
+  }
+
+  @Override
+  protected List<NamedExpression> getValueExpressions() {
+    return valueExpressions;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggBatchCreator.java
new file mode 100644
index 0000000..eda40e9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggBatchCreator.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.metadata;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.config.MetadataAggPOP;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.util.List;
+
+public class MetadataAggBatchCreator implements BatchCreator<MetadataAggPOP> {
+
+  @Override
+  public CloseableRecordBatch getBatch(ExecutorFragmentContext context,
+      MetadataAggPOP config, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() == 1);
+    return new MetadataAggBatch(config, children.iterator().next(), context);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
new file mode 100644
index 0000000..0c94e3d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
@@ -0,0 +1,760 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.metadata;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.metastore.analyze.AnalyzeColumnUtils;
+import org.apache.drill.exec.metastore.analyze.MetastoreAnalyzeConstants;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.MetadataControllerPOP;
+import org.apache.drill.exec.physical.rowSet.DirectRowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetReader;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.WriterPrel;
+import org.apache.drill.exec.metastore.analyze.MetadataIdentifierUtils;
+import org.apache.drill.exec.record.AbstractBinaryRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.store.StatisticsRecordCollector;
+import org.apache.drill.exec.store.StatisticsRecordWriterImpl;
+import org.apache.drill.exec.store.easy.json.StatisticsCollectorImpl;
+import org.apache.drill.exec.store.parquet.ParquetTableMetadataUtils;
+import org.apache.drill.exec.vector.BitVector;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.exec.vector.accessor.ArrayReader;
+import org.apache.drill.exec.vector.accessor.ObjectReader;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.metastore.components.tables.MetastoreTableInfo;
+import org.apache.drill.metastore.components.tables.TableMetadataUnit;
+import org.apache.drill.metastore.components.tables.Tables;
+import org.apache.drill.metastore.expressions.FilterExpression;
+import org.apache.drill.metastore.metadata.BaseMetadata;
+import org.apache.drill.metastore.metadata.BaseTableMetadata;
+import org.apache.drill.metastore.metadata.FileMetadata;
+import org.apache.drill.metastore.metadata.MetadataInfo;
+import org.apache.drill.metastore.metadata.MetadataType;
+import org.apache.drill.metastore.metadata.PartitionMetadata;
+import org.apache.drill.metastore.metadata.RowGroupMetadata;
+import org.apache.drill.metastore.metadata.SegmentMetadata;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.apache.drill.metastore.operate.Modify;
+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.ExactStatisticsConstants;
+import org.apache.drill.metastore.statistics.StatisticsHolder;
+import org.apache.drill.metastore.statistics.StatisticsKind;
+import org.apache.drill.metastore.statistics.TableStatisticsKind;
+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.Multimap;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Terminal operator for producing ANALYZE statement. This operator is responsible for converting
+ * obtained metadata, fetching absent metadata from the Metastore and storing resulting metadata into the Metastore.
+ * <p>
+ * This operator has two inputs: left input contains metadata and right input contains statistics metadata.
+ */
+public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataControllerPOP> {
+  private static final Logger logger = LoggerFactory.getLogger(MetadataControllerBatch.class);
+
+  private final Tables tables;
+  private final TableInfo tableInfo;
+  private final Map<String, MetadataInfo> metadataToHandle;
+  private final StatisticsRecordCollector statisticsCollector;
+  private final List<TableMetadataUnit> metadataUnits;
+
+  private boolean firstLeft = true;
+  private boolean firstRight = true;
+  private boolean finished = false;
+  private boolean finishedRight = false;
+  private int recordCount = 0;
+
+  protected MetadataControllerBatch(MetadataControllerPOP popConfig,
+      FragmentContext context, RecordBatch left, RecordBatch right) throws OutOfMemoryException {
+    super(popConfig, context, false, left, right);
+    this.tables = context.getMetastoreRegistry().get().tables();
+    this.tableInfo = popConfig.getContext().tableInfo();
+    this.metadataToHandle = popConfig.getContext().metadataToHandle() == null
+        ? null
+        : popConfig.getContext().metadataToHandle().stream()
+            .collect(Collectors.toMap(MetadataInfo::identifier, Function.identity()));
+    this.metadataUnits = new ArrayList<>();
+    this.statisticsCollector = new StatisticsCollectorImpl();
+  }
+
+  protected boolean setupNewSchema() {
+    container.clear();
+
+    container.addOrGet(MetastoreAnalyzeConstants.OK_FIELD_NAME, Types.required(TypeProtos.MinorType.BIT), null);
+    container.addOrGet(MetastoreAnalyzeConstants.SUMMARY_FIELD_NAME, Types.required(TypeProtos.MinorType.VARCHAR), null);
+
+    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
+
+    return true;
+  }
+
+  @Override
+  public IterOutcome innerNext() {
+    IterOutcome outcome;
+    boolean finishedLeft;
+    if (finished) {
+      return IterOutcome.NONE;
+    }
+
+    if (!finishedRight) {
+      outcome = handleRightIncoming();
+      if (outcome != null) {
+        return outcome;
+      }
+    }
+
+    outer:
+    while (true) {
+      outcome = next(0, left);
+      switch (outcome) {
+        case NONE:
+          // all incoming data was processed when returned OK_NEW_SCHEMA
+          finishedLeft = !firstLeft;
+          break outer;
+        case OUT_OF_MEMORY:
+        case NOT_YET:
+        case STOP:
+          return outcome;
+        case OK_NEW_SCHEMA:
+          if (firstLeft) {
+            firstLeft = false;
+            if (!setupNewSchema()) {
+              outcome = IterOutcome.OK;
+            }
+            IterOutcome out = handleLeftIncoming();
+            if (out != IterOutcome.OK) {
+              return out;
+            }
+            return outcome;
+          }
+          //fall through
+        case OK:
+          assert !firstLeft : "First batch should be OK_NEW_SCHEMA";
+          IterOutcome out = handleLeftIncoming();
+          if (out != IterOutcome.OK) {
+            return out;
+          }
+          break;
+        default:
+          context.getExecutorState()
+              .fail(new UnsupportedOperationException("Unsupported upstream state " + outcome));
+          close();
+          killIncoming(false);
+          return IterOutcome.STOP;
+      }
+    }
+
+    if (finishedLeft) {
+      IterOutcome out = writeToMetastore();
+      finished = true;
+      return out;
+    }
+    return outcome;
+  }
+
+  private IterOutcome handleRightIncoming() {
+    IterOutcome outcome;
+    outer:
+    while (true) {
+      outcome = next(0, right);
+      switch (outcome) {
+        case NONE:
+          // all incoming data was processed
+          finishedRight = true;
+          break outer;
+        case OUT_OF_MEMORY:
+        case NOT_YET:
+        case STOP:
+          return outcome;
+        case OK_NEW_SCHEMA:
+          firstRight = false;
+          //fall through
+        case OK:
+          assert !firstRight : "First batch should be OK_NEW_SCHEMA";
+          try {
+            appendStatistics(statisticsCollector);
+          } catch (IOException e) {
+            context.getExecutorState().fail(e);
+            close();
+            killIncoming(false);
+            return IterOutcome.STOP;
+          }
+          break;
+        default:
+          context.getExecutorState()
+              .fail(new UnsupportedOperationException("Unsupported upstream state " + outcome));
+          close();
+          killIncoming(false);
+          return IterOutcome.STOP;
+      }
+    }
+    return null;
+  }
+
+  private IterOutcome handleLeftIncoming() {
+    try {
+      metadataUnits.addAll(getMetadataUnits(left.getContainer()));
+    } catch (Exception e) {
+      context.getExecutorState().fail(e);
+      close();
+      killIncoming(false);
+      return IterOutcome.STOP;
+    }
+    return IterOutcome.OK;
+  }
+
+  private IterOutcome writeToMetastore() {
+    FilterExpression deleteFilter = popConfig.getContext().tableInfo().toFilter();
+
+    for (MetadataInfo metadataInfo : popConfig.getContext().metadataToRemove()) {
+      deleteFilter = FilterExpression.and(deleteFilter,
+          FilterExpression.equal(MetadataInfo.METADATA_KEY, metadataInfo.key()));
+    }
+
+    Modify<TableMetadataUnit> modify = tables.modify();
+    if (!popConfig.getContext().metadataToRemove().isEmpty()) {
+      modify.delete(deleteFilter);
+    }
+
+    MetastoreTableInfo metastoreTableInfo = popConfig.getContext().metastoreTableInfo();
+
+    if (tables.basicRequests().hasMetastoreTableInfoChanged(metastoreTableInfo)) {
+      context.getExecutorState()
+          .fail(new IllegalStateException(String.format("Metadata for table [%s] was changed before analyze is finished", tableInfo.name())));
+      close();
+      killIncoming(false);
+      return IterOutcome.STOP;
+    }
+
+    modify.overwrite(metadataUnits)
+        .execute();
+
+    BitVector bitVector =
+        container.addOrGet(MetastoreAnalyzeConstants.OK_FIELD_NAME, Types.required(TypeProtos.MinorType.BIT), null);
+    VarCharVector varCharVector =
+        container.addOrGet(MetastoreAnalyzeConstants.SUMMARY_FIELD_NAME, Types.required(TypeProtos.MinorType.VARCHAR), null);
+
+    bitVector.allocateNew();
+    varCharVector.allocateNew();
+
+    bitVector.getMutator().set(0, 1);
+    varCharVector.getMutator().setSafe(0,
+        String.format("Collected / refreshed metadata for table [%s.%s.%s]",
+            popConfig.getContext().tableInfo().storagePlugin(),
+            popConfig.getContext().tableInfo().workspace(),
+            popConfig.getContext().tableInfo().name()).getBytes());
+
+    bitVector.getMutator().setValueCount(1);
+    varCharVector.getMutator().setValueCount(1);
+    container.setRecordCount(++recordCount);
+
+    return IterOutcome.OK;
+  }
+
+  private List<TableMetadataUnit> getMetadataUnits(VectorContainer container) {
+    List<TableMetadataUnit> metadataUnits = new ArrayList<>();
+    RowSetReader reader = DirectRowSet.fromContainer(container).reader();
+    while (reader.next()) {
+      metadataUnits.addAll(getMetadataUnits(reader, 0));
+    }
+
+    if (!metadataToHandle.isEmpty()) {
+      // leaves only table metadata and metadata which belongs to segments to be overridden
+      metadataUnits = metadataUnits.stream()
+          .filter(tableMetadataUnit ->
+              metadataToHandle.values().stream()
+                  .map(MetadataInfo::key)
+                  .anyMatch(s -> s.equals(tableMetadataUnit.metadataKey()))
+              || MetadataType.TABLE.name().equals(tableMetadataUnit.metadataType()))
+          .collect(Collectors.toList());
+
+      // leaves only metadata which should be fetched from the Metastore
+      metadataUnits.stream()
+          .map(TableMetadataUnit::metadataIdentifier)
+          .forEach(metadataToHandle::remove);
+
+      List<TableMetadataUnit> metadata = metadataToHandle.isEmpty()
+          ? Collections.emptyList()
+          : tables.basicRequests().metadata(popConfig.getContext().tableInfo(), metadataToHandle.values());
+
+      metadataUnits.addAll(metadata);
+    }
+
+    boolean insertDefaultSegment = metadataUnits.stream()
+        .noneMatch(metadataUnit -> metadataUnit.metadataType().equals(MetadataType.SEGMENT.name()));
+
+    if (insertDefaultSegment) {
+      TableMetadataUnit defaultSegmentMetadata = getDefaultSegment(metadataUnits);
+      metadataUnits.add(defaultSegmentMetadata);
+    }
+
+    return metadataUnits;
+  }
+
+  private TableMetadataUnit getDefaultSegment(List<TableMetadataUnit> metadataUnits) {
+    TableMetadataUnit tableMetadataUnit = metadataUnits.stream()
+        .filter(metadataUnit -> metadataUnit.metadataType().equals(MetadataType.TABLE.name()))
+        .findAny()
+        .orElseThrow(() -> new IllegalStateException("Table metadata wasn't found among collected metadata."));
+
+    List<String> paths = metadataUnits.stream()
+        .filter(metadataUnit -> metadataUnit.metadataType().equals(MetadataType.FILE.name()))
+        .map(TableMetadataUnit::path)
+        .collect(Collectors.toList());
+
+    return tableMetadataUnit.toBuilder()
+        .metadataType(MetadataType.SEGMENT.name())
+        .metadataKey(MetadataInfo.DEFAULT_SEGMENT_KEY)
+        .owner(null)
+        .tableType(null)
+        .metadataStatistics(Collections.emptyList())
+        .columnsStatistics(Collections.emptyMap())
+        .path(tableMetadataUnit.location())
+        .schema(null)
+        .locations(paths)
+        .build();
+  }
+
+  private List<TableMetadataUnit> getMetadataUnits(TupleReader reader, int nestingLevel) {
+    List<TableMetadataUnit> metadataUnits = new ArrayList<>();
+
+    TupleMetadata columnMetadata = reader.tupleSchema();
+    ObjectReader metadataColumnReader = reader.column(MetastoreAnalyzeConstants.METADATA_TYPE);
+    Preconditions.checkNotNull(metadataColumnReader, "metadataType column wasn't found");
+
+    ObjectReader underlyingMetadataReader = reader.column(MetastoreAnalyzeConstants.COLLECTED_MAP_FIELD);
+    if (underlyingMetadataReader != null) {
+      if (!underlyingMetadataReader.schema().isArray()) {
+        throw new IllegalStateException("Incoming vector with name `collected_map` should be repeated map");
+      }
+      // current row contains information about underlying metadata
+      ArrayReader array = underlyingMetadataReader.array();
+      while (array.next()) {
+        metadataUnits.addAll(getMetadataUnits(array.tuple(), nestingLevel + 1));
+      }
+    }
+
+    List<StatisticsHolder> metadataStatistics = getMetadataStatistics(reader, columnMetadata);
+
+    Long rowCount = (Long) metadataStatistics.stream()
+        .filter(statisticsHolder -> statisticsHolder.getStatisticsKind() == TableStatisticsKind.ROW_COUNT)
+        .findAny()
+        .map(StatisticsHolder::getStatisticsValue)
+        .orElse(null);
+
+    Map<SchemaPath, ColumnStatistics> columnStatistics = getColumnStatistics(reader, columnMetadata, rowCount);
+
+    MetadataType metadataType = MetadataType.valueOf(metadataColumnReader.scalar().getString());
+
+    BaseMetadata metadata;
+
+    switch (metadataType) {
+      case TABLE: {
+        metadata = getTableMetadata(reader, metadataStatistics, columnStatistics);
+        break;
+      }
+      case SEGMENT: {
+        metadata = getSegmentMetadata(reader, metadataStatistics, columnStatistics, nestingLevel);
+        break;
+      }
+      case PARTITION: {
+        metadata = getPartitionMetadata(reader, metadataStatistics, columnStatistics, nestingLevel);
+        break;
+      }
+      case FILE: {
+        metadata = getFileMetadata(reader, metadataStatistics, columnStatistics, nestingLevel);
+        break;
+      }
+      case ROW_GROUP: {
+        metadata = getRowGroupMetadata(reader, metadataStatistics, columnStatistics, nestingLevel);
+        break;
+      }
+      default:
+        throw new UnsupportedOperationException("Unsupported metadata type: " + metadataType);
+    }
+    metadataUnits.add(metadata.toMetadataUnit());
+
+    return metadataUnits;
+  }
+
+  private PartitionMetadata getPartitionMetadata(TupleReader reader, List<StatisticsHolder> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics> columnStatistics, int nestingLevel) {
+    List<String> segmentColumns = popConfig.getContext().segmentColumns();
+    String lastModifiedTimeCol = context.getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL);
+
+    String segmentKey = segmentColumns.size() > 0
+        ? reader.column(segmentColumns.iterator().next()).scalar().getString()
+        : MetadataInfo.DEFAULT_SEGMENT_KEY;
+
+    List<String> partitionValues = segmentColumns.stream()
+        .limit(nestingLevel)
+        .map(columnName -> reader.column(columnName).scalar().getString())
+        .collect(Collectors.toList());
+    String metadataIdentifier = MetadataIdentifierUtils.getMetadataIdentifierKey(partitionValues);
+    MetadataInfo metadataInfo = MetadataInfo.builder()
+        .type(MetadataType.PARTITION)
+        .key(segmentKey)
+        .identifier(StringUtils.defaultIfEmpty(metadataIdentifier, null))
+        .build();
+    return PartitionMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataInfo(metadataInfo)
+        .columnsStatistics(columnStatistics)
+        .metadataStatistics(metadataStatistics)
+        .locations(getIncomingLocations(reader))
+        .lastModifiedTime(Long.parseLong(reader.column(lastModifiedTimeCol).scalar().getString()))
+//            .column(SchemaPath.getSimplePath("dir1"))
+//            .partitionValues()
+        .schema(TupleMetadata.of(reader.column(MetastoreAnalyzeConstants.SCHEMA_FIELD).scalar().getString()))
+        .build();
+  }
+
+  @SuppressWarnings("unchecked")
+  private BaseTableMetadata getTableMetadata(TupleReader reader, List<StatisticsHolder> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics> columnStatistics) {
+    String lastModifiedTimeCol = context.getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL);
+    List<StatisticsHolder> updatedMetaStats = new ArrayList<>(metadataStatistics);
+    updatedMetaStats.add(new StatisticsHolder(popConfig.getContext().analyzeMetadataLevel(), TableStatisticsKind.ANALYZE_METADATA_LEVEL));
+
+    MetadataInfo metadataInfo = MetadataInfo.builder()
+        .type(MetadataType.TABLE)
+        .key(MetadataInfo.GENERAL_INFO_KEY)
+        .build();
+
+    BaseTableMetadata tableMetadata = BaseTableMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataInfo(metadataInfo)
+        .columnsStatistics(columnStatistics)
+        .metadataStatistics(updatedMetaStats)
+        .partitionKeys(Collections.emptyMap())
+        .interestingColumns(popConfig.getContext().interestingColumns())
+        .location(popConfig.getContext().location())
+        .lastModifiedTime(Long.parseLong(reader.column(lastModifiedTimeCol).scalar().getString()))
+        .schema(TupleMetadata.of(reader.column(MetastoreAnalyzeConstants.SCHEMA_FIELD).scalar().getString()))
+        .build();
+
+    if (context.getOptions().getOption(PlannerSettings.STATISTICS_USE)) {
+      DrillStatsTable statistics = new DrillStatsTable(statisticsCollector.getStatistics());
+      Map<SchemaPath, ColumnStatistics> tableColumnStatistics =
+          ParquetTableMetadataUtils.getColumnStatistics(tableMetadata.getSchema(), statistics);
+      tableMetadata = tableMetadata.cloneWithStats(tableColumnStatistics, DrillStatsTable.getEstimatedTableStats(statistics));
+    }
+
+    return tableMetadata;
+  }
+
+  private SegmentMetadata getSegmentMetadata(TupleReader reader, List<StatisticsHolder> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics> columnStatistics, int nestingLevel) {
+    List<String> segmentColumns = popConfig.getContext().segmentColumns();
+    String lastModifiedTimeCol = context.getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL);
+
+    String segmentKey = segmentColumns.size() > 0
+        ? reader.column(segmentColumns.iterator().next()).scalar().getString()
+        : MetadataInfo.DEFAULT_SEGMENT_KEY;
+
+    List<String> partitionValues = segmentColumns.stream()
+        .limit(nestingLevel)
+        .map(columnName -> reader.column(columnName).scalar().getString())
+        .collect(Collectors.toList());
+    String metadataIdentifier = MetadataIdentifierUtils.getMetadataIdentifierKey(partitionValues);
+
+    MetadataInfo metadataInfo = MetadataInfo.builder()
+        .type(MetadataType.SEGMENT)
+        .key(segmentKey)
+        .identifier(StringUtils.defaultIfEmpty(metadataIdentifier, null))
+        .build();
+
+    return SegmentMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataInfo(metadataInfo)
+        .columnsStatistics(columnStatistics)
+        .metadataStatistics(metadataStatistics)
+        .path(new Path(reader.column(MetastoreAnalyzeConstants.LOCATION_FIELD).scalar().getString()))
+        .locations(getIncomingLocations(reader))
+        .column(segmentColumns.size() > 0 ? SchemaPath.getSimplePath(segmentColumns.get(nestingLevel - 1)) : null)
+        .partitionValues(partitionValues)
+        .lastModifiedTime(Long.parseLong(reader.column(lastModifiedTimeCol).scalar().getString()))
+        .schema(TupleMetadata.of(reader.column(MetastoreAnalyzeConstants.SCHEMA_FIELD).scalar().getString()))
+        .build();
+  }
+
+  private FileMetadata getFileMetadata(TupleReader reader, List<StatisticsHolder> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics> columnStatistics, int nestingLevel) {
+    List<String> segmentColumns = popConfig.getContext().segmentColumns();
+    String lastModifiedTimeCol = context.getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL);
+
+    String segmentKey = segmentColumns.size() > 0
+        ? reader.column(segmentColumns.iterator().next()).scalar().getString()
+        : MetadataInfo.DEFAULT_SEGMENT_KEY;
+
+    List<String> partitionValues = segmentColumns.stream()
+        .limit(nestingLevel - 1)
+        .map(columnName -> reader.column(columnName).scalar().getString())
+        .collect(Collectors.toList());
+
+    Path path = new Path(reader.column(MetastoreAnalyzeConstants.LOCATION_FIELD).scalar().getString());
+    String metadataIdentifier = MetadataIdentifierUtils.getFileMetadataIdentifier(partitionValues, path);
+
+    MetadataInfo metadataInfo = MetadataInfo.builder()
+        .type(MetadataType.FILE)
+        .key(segmentKey)
+        .identifier(StringUtils.defaultIfEmpty(metadataIdentifier, null))
+        .build();
+
+    return FileMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataInfo(metadataInfo)
+        .columnsStatistics(columnStatistics)
+        .metadataStatistics(metadataStatistics)
+        .path(path)
+        .lastModifiedTime(Long.parseLong(reader.column(lastModifiedTimeCol).scalar().getString()))
+        .schema(TupleMetadata.of(reader.column(MetastoreAnalyzeConstants.SCHEMA_FIELD).scalar().getString()))
+        .build();
+  }
+
+  private RowGroupMetadata getRowGroupMetadata(TupleReader reader,List<StatisticsHolder> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics> columnStatistics, int nestingLevel) {
+    String lastModifiedTimeCol = context.getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL);
+    String rgi = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL);
+
+    List<String> segmentColumns = popConfig.getContext().segmentColumns();
+    String segmentKey = segmentColumns.size() > 0
+        ? reader.column(segmentColumns.iterator().next()).scalar().getString()
+        : MetadataInfo.DEFAULT_SEGMENT_KEY;
+
+    List<String> partitionValues = segmentColumns.stream()
+        .limit(nestingLevel - 2)
+        .map(columnName -> reader.column(columnName).scalar().getString())
+        .collect(Collectors.toList());
+
+    Path path = new Path(reader.column(MetastoreAnalyzeConstants.LOCATION_FIELD).scalar().getString());
+
+    int rowGroupIndex = Integer.parseInt(reader.column(rgi).scalar().getString());
+
+    String metadataIdentifier = MetadataIdentifierUtils.getRowGroupMetadataIdentifier(partitionValues, path, rowGroupIndex);
+
+    MetadataInfo metadataInfo = MetadataInfo.builder()
+        .type(MetadataType.ROW_GROUP)
+        .key(segmentKey)
+        .identifier(StringUtils.defaultIfEmpty(metadataIdentifier, null))
+        .build();
+
+    return RowGroupMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataInfo(metadataInfo)
+        .columnsStatistics(columnStatistics)
+        .metadataStatistics(metadataStatistics)
+        .hostAffinity(Collections.emptyMap())
+        .rowGroupIndex(rowGroupIndex)
+        .path(path)
+        .lastModifiedTime(Long.parseLong(reader.column(lastModifiedTimeCol).scalar().getString()))
+        .schema(TupleMetadata.of(reader.column(MetastoreAnalyzeConstants.SCHEMA_FIELD).scalar().getString()))
+        .build();
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<SchemaPath, ColumnStatistics> getColumnStatistics(TupleReader reader, TupleMetadata columnMetadata, Long rowCount) {
+    Multimap<String, StatisticsHolder> columnStatistics = ArrayListMultimap.create();
+    Map<String, TypeProtos.MinorType> columnTypes = new HashMap<>();
+    for (ColumnMetadata column : columnMetadata) {
+      String fieldName = AnalyzeColumnUtils.getColumnName(column.name());
+
+      if (AnalyzeColumnUtils.isColumnStatisticsField(column.name())) {
+        StatisticsKind statisticsKind = AnalyzeColumnUtils.getStatisticsKind(column.name());
+        columnStatistics.put(fieldName,
+            new StatisticsHolder(getConvertedColumnValue(reader.column(column.name())), statisticsKind));
+        if (statisticsKind.getName().equalsIgnoreCase(ColumnStatisticsKind.MIN_VALUE.getName())
+            || statisticsKind.getName().equalsIgnoreCase(ColumnStatisticsKind.MAX_VALUE.getName())) {
+          columnTypes.putIfAbsent(fieldName, column.type());
+        }
+      }
+    }
+
+    // adds NON_NULL_COUNT to use it during filter pushdown
+    if (rowCount != null) {
+      Map<String, StatisticsHolder> nullsCountColumnStatistics = new HashMap<>();
+      columnStatistics.asMap().forEach((key, value) ->
+          value.stream()
+              .filter(statisticsHolder -> statisticsHolder.getStatisticsKind() == ColumnStatisticsKind.NON_NULL_COUNT)
+              .findAny()
+              .map(statisticsHolder -> (Long) statisticsHolder.getStatisticsValue())
+              .ifPresent(nonNullCount ->
+                  nullsCountColumnStatistics.put(
+                      key,
+                      new StatisticsHolder(rowCount - nonNullCount, ColumnStatisticsKind.NULLS_COUNT))));
+
+      nullsCountColumnStatistics.forEach(columnStatistics::put);
+    }
+
+    Map<SchemaPath, ColumnStatistics> resultingStats = new HashMap<>();
+
+    columnStatistics.asMap().forEach((fieldName, statisticsHolders) ->
+        resultingStats.put(SchemaPath.parseFromString(fieldName), new ColumnStatistics(statisticsHolders, columnTypes.get(fieldName))));
+    return resultingStats;
+  }
+
+  @SuppressWarnings("unchecked")
+  private List<StatisticsHolder> getMetadataStatistics(TupleReader reader, TupleMetadata columnMetadata) {
+    List<StatisticsHolder> metadataStatistics = new ArrayList<>();
+    String rgs = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL);
+    String rgl = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL);
+    for (ColumnMetadata column : columnMetadata) {
+      String columnName = column.name();
+      if (AnalyzeColumnUtils.isMetadataStatisticsField(columnName)) {
+        metadataStatistics.add(new StatisticsHolder(reader.column(columnName).getObject(),
+            AnalyzeColumnUtils.getStatisticsKind(columnName)));
+      } else if (columnName.equals(rgs)) {
+        metadataStatistics.add(new StatisticsHolder(Long.parseLong(reader.column(columnName).scalar().getString()),
+            new BaseStatisticsKind(ExactStatisticsConstants.START, true)));
+      } else if (columnName.equals(rgl)) {
+        metadataStatistics.add(new StatisticsHolder(Long.parseLong(reader.column(columnName).scalar().getString()),
+            new BaseStatisticsKind(ExactStatisticsConstants.LENGTH, true)));
+      }
+    }
+    return metadataStatistics;
+  }
+
+  private void appendStatistics(StatisticsRecordCollector statisticsCollector) throws IOException {
+    if (context.getOptions().getOption(PlannerSettings.STATISTICS_USE)) {
+      List<FieldConverter> fieldConverters = new ArrayList<>();
+      int fieldId = 0;
+
+      for (VectorWrapper wrapper : right) {
+        if (wrapper.getField().getName().equalsIgnoreCase(WriterPrel.PARTITION_COMPARATOR_FIELD)) {
+          continue;
+        }
+        FieldReader reader = wrapper.getValueVector().getReader();
+        FieldConverter converter =
+            StatisticsRecordWriterImpl.getConverter(statisticsCollector, fieldId++, wrapper.getField().getName(), reader);
+        fieldConverters.add(converter);
+      }
+
+      for (int counter = 0; counter < right.getRecordCount(); counter++) {
+        statisticsCollector.startStatisticsRecord();
+        // write the current record
+        for (FieldConverter converter : fieldConverters) {
+          converter.setPosition(counter);
+          converter.startField();
+          converter.writeField();
+          converter.endField();
+        }
+        statisticsCollector.endStatisticsRecord();
+      }
+    }
+  }
+
+  private Object getConvertedColumnValue(ObjectReader objectReader) {
+    switch (objectReader.schema().type()) {
+      case VARBINARY:
+      case FIXEDBINARY:
+        return new String(objectReader.scalar().getBytes());
+      default:
+        return objectReader.getObject();
+    }
+
+  }
+
+  private Set<Path> getIncomingLocations(TupleReader reader) {
+    Set<Path> childLocations = new HashSet<>();
+
+    ObjectReader metadataColumnReader = reader.column(MetastoreAnalyzeConstants.METADATA_TYPE);
+    Preconditions.checkNotNull(metadataColumnReader, "metadataType column wasn't found");
+
+    MetadataType metadataType = MetadataType.valueOf(metadataColumnReader.scalar().getString());
+
+    switch (metadataType) {
+      case SEGMENT:
+      case PARTITION: {
+        ObjectReader locationsReader = reader.column(MetastoreAnalyzeConstants.LOCATIONS_FIELD);
+        // populate list of file locations from "locations" field if it is present in the schema
+        if (locationsReader != null && locationsReader.type() == ObjectType.ARRAY) {
+          ArrayReader array = locationsReader.array();
+          while (array.next()) {
+            childLocations.add(new Path(array.scalar().getString()));
+          }
+          break;
+        }
+        // in the opposite case, populate list of file locations using underlying metadata
+        ObjectReader underlyingMetadataReader = reader.column(MetastoreAnalyzeConstants.COLLECTED_MAP_FIELD);
+        if (underlyingMetadataReader != null) {
+          // current row contains information about underlying metadata
+          ArrayReader array = underlyingMetadataReader.array();
+          array.rewind();
+          while (array.next()) {
+            childLocations.addAll(getIncomingLocations(array.tuple()));
+          }
+        }
+        break;
+      }
+      case FILE: {
+        childLocations.add(new Path(reader.column(MetastoreAnalyzeConstants.LOCATION_FIELD).scalar().getString()));
+      }
+    }
+
+    return childLocations;
+  }
+
+  @Override
+  protected void killIncoming(boolean sendUpstream) {
+    left.kill(sendUpstream);
+    right.kill(sendUpstream);
+  }
+
+  @Override
+  public void dump() {
+    logger.error("MetadataHandlerBatch[container={}, popConfig={}]", container, popConfig);
+  }
+
+  @Override
+  public int getRecordCount() {
+    return recordCount;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatchCreator.java
new file mode 100644
index 0000000..46238bc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatchCreator.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.metadata;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.config.MetadataControllerPOP;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.util.List;
+
+@SuppressWarnings("unused")
+public class MetadataControllerBatchCreator implements BatchCreator<MetadataControllerPOP> {
+
+  @Override
+  public CloseableRecordBatch getBatch(ExecutorFragmentContext context,
+      MetadataControllerPOP config, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() == 2);
+    return new MetadataControllerBatch(config, context, children.get(0), children.get(1));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
new file mode 100644
index 0000000..7444a04
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
@@ -0,0 +1,498 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.metadata;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.metastore.analyze.MetastoreAnalyzeConstants;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.MetadataHandlerPOP;
+import org.apache.drill.exec.metastore.analyze.AnalyzeColumnUtils;
+import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
+import org.apache.drill.exec.physical.resultSet.RowSetLoader;
+import org.apache.drill.exec.physical.resultSet.impl.OptionBuilder;
+import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl;
+import org.apache.drill.exec.physical.rowSet.DirectRowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetReader;
+import org.apache.drill.exec.metastore.analyze.MetadataIdentifierUtils;
+import org.apache.drill.exec.record.AbstractSingleRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.metastore.components.tables.BasicTablesRequests;
+import org.apache.drill.metastore.components.tables.Tables;
+import org.apache.drill.metastore.metadata.BaseMetadata;
+import org.apache.drill.metastore.metadata.FileMetadata;
+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.RowGroupMetadata;
+import org.apache.drill.metastore.metadata.SegmentMetadata;
+import org.apache.drill.metastore.statistics.ExactStatisticsConstants;
+import org.apache.drill.metastore.statistics.StatisticsKind;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
+import static org.apache.drill.exec.record.RecordBatch.IterOutcome.STOP;
+
+/**
+ * Operator responsible for handling metadata returned by incoming aggregate operators and fetching
+ * required metadata form the Metastore.
+ */
+public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHandlerPOP> {
+  private static final Logger logger = LoggerFactory.getLogger(MetadataHandlerBatch.class);
+
+  private final Tables tables;
+  private final MetadataType metadataType;
+  private final Map<String, MetadataInfo> metadataToHandle;
+
+  private boolean firstBatch = true;
+
+  protected MetadataHandlerBatch(MetadataHandlerPOP popConfig,
+      FragmentContext context, RecordBatch incoming) throws OutOfMemoryException {
+    super(popConfig, context, incoming);
+    this.tables = context.getMetastoreRegistry().get().tables();
+    this.metadataType = popConfig.getContext().metadataType();
+    this.metadataToHandle = popConfig.getContext().metadataToHandle() != null
+        ? popConfig.getContext().metadataToHandle().stream()
+            .collect(Collectors.toMap(MetadataInfo::identifier, Function.identity()))
+        : null;
+  }
+
+  @Override
+  public IterOutcome doWork() {
+    // 1. Consume data from incoming operators and update metadataToHandle to remove incoming metadata
+    // 2. For the case when incoming operator returned nothing - no updated underlying metadata was found.
+    // 3. Fetches metadata which should be handled but wasn't returned by incoming batch from the Metastore
+
+    IterOutcome outcome = next(incoming);
+
+    switch (outcome) {
+      case NONE:
+        if (firstBatch) {
+          Preconditions.checkState(metadataToHandle.isEmpty(),
+              "Incoming batch didn't return the result for modified segments");
+        }
+        return outcome;
+      case OK_NEW_SCHEMA:
+        if (firstBatch) {
+          firstBatch = false;
+          if (!setupNewSchema()) {
+            outcome = IterOutcome.OK;
+          }
+        }
+        doWorkInternal();
+        return outcome;
+      case OK:
+        assert !firstBatch : "First batch should be OK_NEW_SCHEMA";
+        doWorkInternal();
+        // fall thru
+      case OUT_OF_MEMORY:
+      case NOT_YET:
+      case STOP:
+        return outcome;
+      default:
+        context.getExecutorState()
+            .fail(new UnsupportedOperationException("Unsupported upstream state " + outcome));
+        close();
+        killIncoming(false);
+        return IterOutcome.STOP;
+    }
+  }
+
+  @Override
+  public IterOutcome innerNext() {
+    IterOutcome outcome = getLastKnownOutcome();
+    if (outcome != NONE && outcome != STOP) {
+      outcome = super.innerNext();
+    }
+    // if incoming is exhausted, reads metadata which should be obtained from the Metastore
+    // and returns OK or NONE if there is no metadata to read
+    if (outcome == IterOutcome.NONE && !metadataToHandle.isEmpty()) {
+      BasicTablesRequests basicTablesRequests = tables.basicRequests();
+
+      switch (metadataType) {
+        case ROW_GROUP: {
+          List<RowGroupMetadata> rowGroups =
+              basicTablesRequests.rowGroupsMetadata(
+                  popConfig.getContext().tableInfo(),
+                  new ArrayList<>(metadataToHandle.values()));
+          return populateContainer(rowGroups);
+        }
+        case FILE: {
+          List<FileMetadata> files =
+              basicTablesRequests.filesMetadata(
+                  popConfig.getContext().tableInfo(),
+                  new ArrayList<>(metadataToHandle.values()));
+          return populateContainer(files);
+        }
+        case SEGMENT: {
+          List<SegmentMetadata> segments =
+              basicTablesRequests.segmentsMetadata(
+                  popConfig.getContext().tableInfo(),
+                  new ArrayList<>(metadataToHandle.values()));
+          return populateContainer(segments);
+        }
+      }
+    }
+    return outcome;
+  }
+
+  private <T extends BaseMetadata & LocationProvider> IterOutcome populateContainer(List<T> metadata) {
+    VectorContainer populatedContainer;
+    if (firstBatch) {
+      populatedContainer = writeMetadata(metadata);
+      setupSchemaFromContainer(populatedContainer);
+    } else {
+      populatedContainer = writeMetadataUsingBatchSchema(metadata);
+    }
+    container.transferIn(populatedContainer);
+    container.setRecordCount(populatedContainer.getRecordCount());
+
+    if (firstBatch) {
+      firstBatch = false;
+      return IterOutcome.OK_NEW_SCHEMA;
+    } else {
+      return IterOutcome.OK;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private <T extends BaseMetadata & LocationProvider> VectorContainer writeMetadata(List<T> metadataList) {
+    BaseMetadata firstElement = metadataList.iterator().next();
+
+    ResultSetLoader resultSetLoader = getResultSetLoaderForMetadata(firstElement);
+    resultSetLoader.startBatch();
+    RowSetLoader rowWriter = resultSetLoader.writer();
+    Iterator<T> segmentsIterator = metadataList.iterator();
+    while (!rowWriter.isFull() && segmentsIterator.hasNext()) {
+      T metadata = segmentsIterator.next();
+      metadataToHandle.remove(metadata.getMetadataInfo().identifier());
+
+      List<Object> arguments = new ArrayList<>();
+      // adds required segment names to the arguments
+      arguments.add(metadata.getPath().toUri().getPath());
+      Collections.addAll(
+          arguments,
+          Arrays.copyOf(
+              MetadataIdentifierUtils.getValuesFromMetadataIdentifier(metadata.getMetadataInfo().identifier()),
+              popConfig.getContext().segmentColumns().size()));
+
+      // adds column statistics values assuming that they are sorted in alphabetic order
+      // (see getResultSetLoaderForMetadata() method)
+      metadata.getColumnsStatistics().entrySet().stream()
+          .sorted(Comparator.comparing(e -> e.getKey().toExpr()))
+          .map(Map.Entry::getValue)
+          .flatMap(columnStatistics ->
+              AnalyzeColumnUtils.COLUMN_STATISTICS_FUNCTIONS.keySet().stream()
+                  .map(columnStatistics::get))
+          .forEach(arguments::add);
+
+      AnalyzeColumnUtils.META_STATISTICS_FUNCTIONS.keySet().stream()
+          .map(metadata::getStatistic)
+          .forEach(arguments::add);
+
+      // collectedMap field value
+      arguments.add(new Object[]{});
+
+      if (metadataType == MetadataType.SEGMENT) {
+        arguments.add(((SegmentMetadata) metadata).getLocations().stream()
+            .map(path -> path.toUri().getPath())
+            .toArray(String[]::new));
+      }
+
+      if (metadataType == MetadataType.ROW_GROUP) {
+        arguments.add(String.valueOf(((RowGroupMetadata) metadata).getRowGroupIndex()));
+        arguments.add(Long.toString(metadata.getStatistic(() -> ExactStatisticsConstants.START)));
+        arguments.add(Long.toString(metadata.getStatistic(() -> ExactStatisticsConstants.LENGTH)));
+      }
+
+      arguments.add(metadata.getSchema().jsonString());
+      arguments.add(String.valueOf(metadata.getLastModifiedTime()));
+      arguments.add(metadataType.name());
+      rowWriter.addRow(arguments.toArray());
+    }
+
+    return resultSetLoader.harvest();
+  }
+
+  private ResultSetLoader getResultSetLoaderForMetadata(BaseMetadata baseMetadata) {
+    SchemaBuilder schemaBuilder = new SchemaBuilder()
+        .addNullable(MetastoreAnalyzeConstants.LOCATION_FIELD, MinorType.VARCHAR);
+    for (String segmentColumn : popConfig.getContext().segmentColumns()) {
+      schemaBuilder.addNullable(segmentColumn, MinorType.VARCHAR);
+    }
+
+    baseMetadata.getColumnsStatistics().entrySet().stream()
+        .sorted(Comparator.comparing(e -> e.getKey().getRootSegmentPath()))
+        .forEach(entry -> {
+          for (StatisticsKind statisticsKind : AnalyzeColumnUtils.COLUMN_STATISTICS_FUNCTIONS.keySet()) {
+            MinorType type = AnalyzeColumnUtils.COLUMN_STATISTICS_TYPES.get(statisticsKind);
+            type = type != null ? type : entry.getValue().getComparatorType();
+            schemaBuilder.addNullable(
+                AnalyzeColumnUtils.getColumnStatisticsFieldName(entry.getKey().getRootSegmentPath(), statisticsKind),
+                type);
+          }
+        });
+
+    for (StatisticsKind statisticsKind : AnalyzeColumnUtils.META_STATISTICS_FUNCTIONS.keySet()) {
+      schemaBuilder.addNullable(
+          AnalyzeColumnUtils.getMetadataStatisticsFieldName(statisticsKind),
+          AnalyzeColumnUtils.COLUMN_STATISTICS_TYPES.get(statisticsKind));
+    }
+
+    schemaBuilder
+        .addMapArray(MetastoreAnalyzeConstants.COLLECTED_MAP_FIELD)
+        .resumeSchema();
+
+    if (metadataType == MetadataType.SEGMENT) {
+      schemaBuilder.addArray(MetastoreAnalyzeConstants.LOCATIONS_FIELD, MinorType.VARCHAR);
+    }
+
+    if (metadataType == MetadataType.ROW_GROUP) {
+      schemaBuilder.addNullable(context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL), MinorType.VARCHAR);
+      schemaBuilder.addNullable(context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL), MinorType.VARCHAR);
+      schemaBuilder.addNullable(context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL), MinorType.VARCHAR);
+    }
+
+    schemaBuilder
+        .addNullable(MetastoreAnalyzeConstants.SCHEMA_FIELD, MinorType.VARCHAR)
+        .addNullable(context.getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL), MinorType.VARCHAR)
+        .add(MetastoreAnalyzeConstants.METADATA_TYPE, MinorType.VARCHAR);
+
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schemaBuilder.buildSchema())
+        .build();
+
+    return new ResultSetLoaderImpl(container.getAllocator(), options);
+  }
+
+  @SuppressWarnings("unchecked")
+  private <T extends BaseMetadata & LocationProvider> VectorContainer writeMetadataUsingBatchSchema(List<T> metadataList) {
+    Preconditions.checkArgument(!metadataList.isEmpty(), "Metadata list shouldn't be empty.");
+
+    String lastModifiedTimeField = context.getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL);
+    String rgiField = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL);
+    String rgsField = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL);
+    String rglField = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL);
+
+    ResultSetLoader resultSetLoader = getResultSetLoaderWithBatchSchema();
+    resultSetLoader.startBatch();
+    RowSetLoader rowWriter = resultSetLoader.writer();
+    Iterator<T> segmentsIterator = metadataList.iterator();
+    while (!rowWriter.isFull() && segmentsIterator.hasNext()) {
+      T metadata = segmentsIterator.next();
+      metadataToHandle.remove(metadata.getMetadataInfo().identifier());
+
+      List<Object> arguments = new ArrayList<>();
+      for (VectorWrapper<?> vectorWrapper : container) {
+
+        String[] identifierValues = Arrays.copyOf(
+            MetadataIdentifierUtils.getValuesFromMetadataIdentifier(metadata.getMetadataInfo().identifier()),
+            popConfig.getContext().segmentColumns().size());
+
+        MaterializedField field = vectorWrapper.getField();
+        String fieldName = field.getName();
+        if (fieldName.equals(MetastoreAnalyzeConstants.LOCATION_FIELD)) {
+          arguments.add(metadata.getPath().toUri().getPath());
+        } else if (fieldName.equals(MetastoreAnalyzeConstants.LOCATIONS_FIELD)) {
+          if (metadataType == MetadataType.SEGMENT) {
+            arguments.add(((SegmentMetadata) metadata).getLocations().stream()
+                .map(path -> path.toUri().getPath())
+                .toArray(String[]::new));
+          } else {
+            arguments.add(null);
+          }
+        } else if (popConfig.getContext().segmentColumns().contains(fieldName)) {
+          arguments.add(identifierValues[popConfig.getContext().segmentColumns().indexOf(fieldName)]);
+        } else if (AnalyzeColumnUtils.isColumnStatisticsField(fieldName)) {
+          arguments.add(
+              metadata.getColumnStatistics(SchemaPath.parseFromString(AnalyzeColumnUtils.getColumnName(fieldName)))
+                  .get(AnalyzeColumnUtils.getStatisticsKind(fieldName)));
+        } else if (AnalyzeColumnUtils.isMetadataStatisticsField(fieldName)) {
+          arguments.add(metadata.getStatistic(AnalyzeColumnUtils.getStatisticsKind(fieldName)));
+        } else if (fieldName.equals(MetastoreAnalyzeConstants.COLLECTED_MAP_FIELD)) {
+          // collectedMap field value
+          arguments.add(new Object[]{});
+        } else if (fieldName.equals(MetastoreAnalyzeConstants.SCHEMA_FIELD)) {
+          arguments.add(metadata.getSchema().jsonString());
+        } else if (fieldName.equals(lastModifiedTimeField)) {
+          arguments.add(String.valueOf(metadata.getLastModifiedTime()));
+        } else if (fieldName.equals(rgiField)) {
+          arguments.add(String.valueOf(((RowGroupMetadata) metadata).getRowGroupIndex()));
+        } else if (fieldName.equals(rgsField)) {
+          arguments.add(Long.toString(metadata.getStatistic(() -> ExactStatisticsConstants.START)));
+        } else if (fieldName.equals(rglField)) {
+          arguments.add(Long.toString(metadata.getStatistic(() -> ExactStatisticsConstants.LENGTH)));
+        } else if (fieldName.equals(MetastoreAnalyzeConstants.METADATA_TYPE)) {
+          arguments.add(metadataType.name());
+        } else {
+          throw new UnsupportedOperationException(String.format("Found unexpected field [%s] in incoming batch.",  field));
+        }
+      }
+
+      rowWriter.addRow(arguments.toArray());
+    }
+
+    return resultSetLoader.harvest();
+  }
+
+  private ResultSetLoader getResultSetLoaderWithBatchSchema() {
+    String lastModifiedTimeField = context.getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL);
+    String rgiField = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL);
+    String rgsField = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL);
+    String rglField = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL);
+    SchemaBuilder schemaBuilder = new SchemaBuilder();
+    // adds fields to the schema preserving their order to avoid issues in outcoming batches
+    for (VectorWrapper<?> vectorWrapper : container) {
+      MaterializedField field = vectorWrapper.getField();
+      String fieldName = field.getName();
+      if (fieldName.equals(MetastoreAnalyzeConstants.LOCATION_FIELD)
+          || fieldName.equals(MetastoreAnalyzeConstants.SCHEMA_FIELD)
+          || fieldName.equals(lastModifiedTimeField)
+          || fieldName.equals(rgiField)
+          || fieldName.equals(rgsField)
+          || fieldName.equals(rglField)
+          || fieldName.equals(MetastoreAnalyzeConstants.METADATA_TYPE)
+          || popConfig.getContext().segmentColumns().contains(fieldName)) {
+        schemaBuilder.add(fieldName, field.getType().getMinorType(), field.getDataMode());
+      } else if (AnalyzeColumnUtils.isColumnStatisticsField(fieldName)
+          || AnalyzeColumnUtils.isMetadataStatisticsField(fieldName)) {
+        schemaBuilder.add(fieldName, field.getType().getMinorType(), field.getType().getMode());
+      } else if (fieldName.equals(MetastoreAnalyzeConstants.COLLECTED_MAP_FIELD)) {
+        schemaBuilder.addMapArray(fieldName)
+            .resumeSchema();
+      } else if (fieldName.equals(MetastoreAnalyzeConstants.LOCATIONS_FIELD)) {
+        schemaBuilder.addArray(fieldName, MinorType.VARCHAR);
+      } else {
+        throw new UnsupportedOperationException(String.format("Found unexpected field [%s] in incoming batch.",  field));
+      }
+    }
+
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schemaBuilder.buildSchema())
+        .build();
+
+    return new ResultSetLoaderImpl(container.getAllocator(), options);
+  }
+
+  private void setupSchemaFromContainer(VectorContainer populatedContainer) {
+    container.clear();
+    StreamSupport.stream(populatedContainer.spliterator(), false)
+        .map(VectorWrapper::getField)
+        .filter(field -> field.getType().getMinorType() != MinorType.NULL)
+        .forEach(container::addOrGet);
+    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
+  }
+
+  protected boolean setupNewSchema() {
+    setupSchemaFromContainer(incoming.getContainer());
+    return true;
+  }
+
+  private IterOutcome doWorkInternal() {
+    container.transferIn(incoming.getContainer());
+    VarCharVector metadataTypeVector = container.addOrGet(
+        MaterializedField.create(MetastoreAnalyzeConstants.METADATA_TYPE, Types.required(MinorType.VARCHAR)));
+    metadataTypeVector.allocateNew();
+    for (int i = 0; i < incoming.getRecordCount(); i++) {
+      metadataTypeVector.getMutator().setSafe(i, metadataType.name().getBytes());
+    }
+
+    metadataTypeVector.getMutator().setValueCount(incoming.getRecordCount());
+    container.setRecordCount(incoming.getRecordCount());
+
+    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
+    updateMetadataToHandle();
+
+    return IterOutcome.OK;
+  }
+
+  private void updateMetadataToHandle() {
+    RowSetReader reader = DirectRowSet.fromContainer(container).reader();
+    // updates metadataToHandle to be able to fetch required data which wasn't returned by incoming batch
+    if (metadataToHandle != null && !metadataToHandle.isEmpty()) {
+      switch (metadataType) {
+        case ROW_GROUP: {
+          String rgiColumnName = context.getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL);
+          while (reader.next() && !metadataToHandle.isEmpty()) {
+            List<String> partitionValues = popConfig.getContext().segmentColumns().stream()
+                .map(columnName -> reader.column(columnName).scalar().getString())
+                .collect(Collectors.toList());
+            Path location = new Path(reader.column(MetastoreAnalyzeConstants.LOCATION_FIELD).scalar().getString());
+            int rgi = Integer.parseInt(reader.column(rgiColumnName).scalar().getString());
+            metadataToHandle.remove(MetadataIdentifierUtils.getRowGroupMetadataIdentifier(partitionValues, location, rgi));
+          }
+          break;
+        }
+        case FILE: {
+          while (reader.next() && !metadataToHandle.isEmpty()) {
+            List<String> partitionValues = popConfig.getContext().segmentColumns().stream()
+                .map(columnName -> reader.column(columnName).scalar().getString())
+                .collect(Collectors.toList());
+            Path location = new Path(reader.column(MetastoreAnalyzeConstants.LOCATION_FIELD).scalar().getString());
+            // use metadata identifier for files since row group indexes are not required when file is updated
+            metadataToHandle.remove(MetadataIdentifierUtils.getFileMetadataIdentifier(partitionValues, location));
+          }
+          break;
+        }
+        case SEGMENT: {
+          while (reader.next() && !metadataToHandle.isEmpty()) {
+            List<String> partitionValues = popConfig.getContext().segmentColumns().stream()
+                .limit(popConfig.getContext().depthLevel())
+                .map(columnName -> reader.column(columnName).scalar().getString())
+                .collect(Collectors.toList());
+            metadataToHandle.remove(MetadataIdentifierUtils.getMetadataIdentifierKey(partitionValues));
+          }
+          break;
+        }
+      }
+    }
+  }
+
+  @Override
+  public void dump() {
+    logger.error("MetadataHandlerBatch[container={}, popConfig={}]", container, popConfig);
+  }
+
+  @Override
+  public int getRecordCount() {
+    return container.getRecordCount();
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatchCreator.java
new file mode 100644
index 0000000..d2d3ff5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatchCreator.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.metadata;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.config.MetadataHandlerPOP;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.util.List;
+
+public class MetadataHandlerBatchCreator implements BatchCreator<MetadataHandlerPOP> {
+
+  @Override
+  public CloseableRecordBatch getBatch(ExecutorFragmentContext context,
+      MetadataHandlerPOP config, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() == 1);
+    return new MetadataHandlerBatch(config, context, children.iterator().next());
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java
index 440178b..67278f6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java
@@ -81,5 +81,4 @@ public class DFSDirPartitionLocation implements PartitionLocation {
 
     return DrillFileSystemUtil.createPathSafe(path.toString());
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
index c19ff33..a3e907b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
@@ -17,6 +17,9 @@
  */
 package org.apache.drill.exec.planner;
 
+import org.apache.drill.exec.planner.physical.MetadataAggPrule;
+import org.apache.drill.exec.planner.physical.MetadataControllerPrule;
+import org.apache.drill.exec.planner.physical.MetadataHandlerPrule;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet.Builder;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
@@ -519,6 +522,10 @@ public enum PlannerPhase {
     ruleList.add(RowKeyJoinPrule.INSTANCE);
     ruleList.add(AnalyzePrule.INSTANCE);
 
+    ruleList.add(MetadataControllerPrule.INSTANCE);
+    ruleList.add(MetadataHandlerPrule.INSTANCE);
+    ruleList.add(MetadataAggPrule.INSTANCE);
+
     ruleList.add(UnnestPrule.INSTANCE);
     ruleList.add(LateralJoinPrule.INSTANCE);
 
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 695cff6..9e4e925 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
@@ -93,6 +93,15 @@ public class DrillStatsTable {
     this.table = table;
   }
 
+  public DrillStatsTable(TableStatistics statistics) {
+    this.statistics = statistics;
+    this.schemaName = null;
+    this.tableName = null;
+    this.tablePath = null;
+    this.fs = null;
+    materializeFromStatistics();
+  }
+
   public String getSchemaName() {
     return schemaName;
   }
@@ -201,24 +210,7 @@ public class DrillStatsTable {
       // Deserialize statistics from JSON
       this.statistics = readStatistics(table, tablePath);
       // Handle based on the statistics version read from the file
-      if (statistics instanceof Statistics_v0) {
-        // Do nothing
-      } else if (statistics instanceof Statistics_v1) {
-        for (DirectoryStatistics_v1 ds : ((Statistics_v1) statistics).getDirectoryStatistics()) {
-          for (ColumnStatistics_v1 cs : ds.getColumnStatistics()) {
-            ndv.put(cs.getName(), cs.getNdv());
-            nnRowCount.put(cs.getName(), (long) cs.getNonNullCount());
-            rowCount = Math.max(rowCount, cs.getCount());
-
-            // get the histogram for this column
-            Histogram hist = cs.getHistogram();
-            histogram.put(cs.getName(), hist);
-          }
-        }
-      }
-      if (statistics != null) { // See stats are available before setting materialized
-        materialized = true;
-      }
+      materializeFromStatistics();
     } catch (FileNotFoundException ex) {
       logger.debug(String.format("Did not find statistics file %s", tablePath.toString()), ex);
     } catch (IOException ex) {
@@ -226,6 +218,27 @@ public class DrillStatsTable {
     }
   }
 
+  private void materializeFromStatistics() {
+    if (statistics instanceof Statistics_v0) {
+      // Do nothing
+    } else if (statistics instanceof Statistics_v1) {
+      for (DirectoryStatistics_v1 ds : ((Statistics_v1) statistics).getDirectoryStatistics()) {
+        for (ColumnStatistics_v1 cs : ds.getColumnStatistics()) {
+          ndv.put(cs.getName(), cs.getNdv());
+          nnRowCount.put(cs.getName(), (long) cs.getNonNullCount());
+          rowCount = Math.max(rowCount, cs.getCount());
+
+          // get the histogram for this column
+          Histogram hist = cs.getHistogram();
+          histogram.put(cs.getName(), hist);
+        }
+      }
+    }
+    if (statistics != null) { // See stats are available before setting materialized
+      materialized = true;
+    }
+  }
+
   /* Each change to the format SHOULD increment the default and/or the max values of the option
    * exec.statistics.capability_version
    */
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 28c23f0..fb29cda 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
@@ -170,7 +170,7 @@ public class ConvertCountToDirectScanRule extends RelOptRule {
         Collections.singletonList(new ArrayList<>(result.values())));
 
     final ScanStats scanStats = new ScanStats(ScanStats.GroupScanProperty.EXACT_ROW_COUNT, 1, 1, scanRowType.getFieldCount());
-    final MetadataDirectGroupScan directScan = new MetadataDirectGroupScan(reader, summaryFileName, 1, scanStats, true);
+    final MetadataDirectGroupScan directScan = new MetadataDirectGroupScan(reader, summaryFileName, 1, scanStats, true, false);
 
     final DrillDirectScanRel newScan = new DrillDirectScanRel(scan.getCluster(), scan.getTraitSet().plus(DrillRel.DRILL_LOGICAL),
       directScan, scanRowType);
@@ -214,7 +214,8 @@ public class ConvertCountToDirectScanRule extends RelOptRule {
 
     Metadata_V4.MetadataSummary metadataSummary = Metadata.getSummary(fs, selectionRoot, false, parquetReaderConfig);
 
-    return metadataSummary != null ? new ImmutablePair<>(true, metadataSummary) : new ImmutablePair<>(false, null);
+    return metadataSummary != null ? new ImmutablePair<>(true, metadataSummary) :
+        new ImmutablePair<>(false, null);
   }
 
   /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
index 82658af..d38210d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
@@ -176,9 +176,9 @@ public class DrillScanRel extends DrillScanRelBase implements DrillRel {
     return this.partitionFilterPushdown;
   }
 
-  private static List<SchemaPath> getProjectedColumns(final RelOptTable table, boolean isSelectStar) {
+  public static List<SchemaPath> getProjectedColumns(final RelOptTable table, boolean isSelectStar) {
     List<String> columnNames = table.getRowType().getFieldNames();
-    List<SchemaPath> projectedColumns = new ArrayList<SchemaPath>(columnNames.size());
+    List<SchemaPath> projectedColumns = new ArrayList<>(columnNames.size());
 
     for (String columnName : columnNames) {
        projectedColumns.add(SchemaPath.getSimplePath(columnName));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/MetadataAggRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/MetadataAggRel.java
new file mode 100644
index 0000000..d798116
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/MetadataAggRel.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.logical;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.common.logical.data.MetadataAggregate;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
+import org.apache.drill.exec.metastore.analyze.MetadataAggregateContext;
+
+import java.util.List;
+
+public class MetadataAggRel extends SingleRel implements DrillRel {
+
+  private final MetadataAggregateContext context;
+
+  public MetadataAggRel(RelOptCluster cluster, RelTraitSet traits, RelNode input,
+      MetadataAggregateContext context) {
+    super(cluster, traits, input);
+    this.context = context;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    double dRows = mq.getRowCount(getInput());
+    double dCpu = dRows * DrillCostBase.COMPARE_CPU_COST;
+    double dIo = 0;
+    return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new MetadataAggRel(getCluster(), traitSet, sole(inputs), context);
+  }
+
+  @Override
+  public LogicalOperator implement(DrillImplementor implementor) {
+    LogicalOperator inputOp = implementor.visitChild(this, 0, getInput());
+    MetadataAggregate rel = new MetadataAggregate();
+    rel.setInput(inputOp);
+    return rel;
+  }
+
+  public MetadataAggregateContext getContext() {
+    return context;
+  }
+
+  @Override
+  public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw).item("context: ", context);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/MetadataControllerRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/MetadataControllerRel.java
new file mode 100644
index 0000000..ee1f2f1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/MetadataControllerRel.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.logical;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.common.logical.data.MetadataController;
+import org.apache.drill.exec.metastore.analyze.MetastoreAnalyzeConstants;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
+import org.apache.drill.exec.metastore.analyze.MetadataControllerContext;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.util.List;
+
+public class MetadataControllerRel extends BiRel implements DrillRel {
+  private final MetadataControllerContext context;
+
+  public MetadataControllerRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right,
+      MetadataControllerContext context) {
+    super(cluster, traits, left, right);
+    this.context = context;
+  }
+
+  public MetadataControllerContext getContext() {
+    return context;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    double dRows = Math.max(mq.getRowCount(getLeft()), mq.getRowCount(getRight()));
+    double dCpu = dRows * DrillCostBase.COMPARE_CPU_COST;
+    double dIo = 0;
+    return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    Preconditions.checkArgument(inputs.size() == 2);
+    return new MetadataControllerRel(getCluster(), traitSet, inputs.get(0), inputs.get(1), context);
+  }
+
+  @Override
+  public LogicalOperator implement(DrillImplementor implementor) {
+    LogicalOperator left = implementor.visitChild(this, 0, getLeft());
+    LogicalOperator right = implementor.visitChild(this, 1, getRight());
+    return new MetadataController(left, right);
+  }
+
+  @Override
+  protected RelDataType deriveRowType() {
+    return getCluster().getTypeFactory().builder()
+        .add(MetastoreAnalyzeConstants.OK_FIELD_NAME, SqlTypeName.BOOLEAN)
+        .add(MetastoreAnalyzeConstants.SUMMARY_FIELD_NAME, SqlTypeName.VARCHAR)
+        .build();
+  }
+
+  @Override
+  public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw).item("context: ", context);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/MetadataHandlerRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/MetadataHandlerRel.java
new file mode 100644
index 0000000..ed7ddf2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/MetadataHandlerRel.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.logical;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.common.logical.data.MetadataHandler;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
+import org.apache.drill.exec.metastore.analyze.MetadataHandlerContext;
+
+import java.util.List;
+
+public class MetadataHandlerRel extends SingleRel implements DrillRel {
+  private final MetadataHandlerContext context;
+
+  public MetadataHandlerRel(RelOptCluster cluster, RelTraitSet traits, RelNode input,
+      MetadataHandlerContext context) {
+    super(cluster, traits, input);
+    this.context = context;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    double dRows = mq.getRowCount(getInput());
+    double dCpu = dRows * DrillCostBase.COMPARE_CPU_COST;
+    double dIo = 0;
+    return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new MetadataHandlerRel(getCluster(), traitSet, sole(inputs), context);
+  }
+
+  @Override
+  public LogicalOperator implement(DrillImplementor implementor) {
+    LogicalOperator inputOp = implementor.visitChild(this, 0, getInput());
+    MetadataHandler rel = new MetadataHandler();
+    rel.setInput(inputOp);
+    return rel;
+  }
+
+  public MetadataHandlerContext getContext() {
+    return context;
+  }
+
+  @Override
+  public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw).item("context: ", context);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
index f256ad1..c3c271e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
@@ -21,9 +21,7 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.data.Values;
 import org.apache.drill.common.logical.data.Filter;
@@ -42,6 +40,7 @@ import org.apache.drill.common.logical.data.Store;
 import org.apache.drill.common.logical.data.Union;
 import org.apache.drill.common.logical.data.visitors.AbstractLogicalVisitor;
 
+import org.apache.drill.exec.store.parquet.FilterEvaluatorUtils.FieldReferenceFinder;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
@@ -51,12 +50,9 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
  * can then be used to update scan object to appear to be explicitly fielded for optimization purposes.
  */
 public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldDeterminer.FieldList, RuntimeException> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanFieldDeterminer.class);
 
-  private FieldReferenceFinder finder = new FieldReferenceFinder();
   private Map<Scan, FieldList> scanFields = Maps.newHashMap();
 
-
   public static Map<Scan, FieldList> getFieldLists(LogicalPlan plan){
     Collection<SinkOperator> ops = plan.getGraph().getRoots();
     Preconditions.checkArgument(ops.size() == 1, "Scan Field determiner currently only works with plans that have a single root.");
@@ -121,10 +117,10 @@ public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldD
   public Void visitGroupingAggregate(GroupingAggregate groupBy, FieldList value) {
     FieldList list = new FieldList();
     for (NamedExpression e : groupBy.getExprs()) {
-      list.addProjected(e.getExpr().accept(finder, null));
+      list.addProjected(e.getExpr().accept(FieldReferenceFinder.INSTANCE, null));
     }
     for (NamedExpression e : groupBy.getKeys()) {
-      list.addProjected(e.getExpr().accept(finder, null));
+      list.addProjected(e.getExpr().accept(FieldReferenceFinder.INSTANCE, null));
     }
     groupBy.getInput().accept(this, list);
     return null;
@@ -132,7 +128,7 @@ public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldD
 
   @Override
   public Void visitFilter(Filter filter, FieldList value) {
-    value.addReferenced(filter.getExpr().accept(finder, null));
+    value.addReferenced(filter.getExpr().accept(FieldReferenceFinder.INSTANCE, null));
     return null;
   }
 
@@ -140,7 +136,7 @@ public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldD
   public Void visitProject(Project project, FieldList value) {
     FieldList fl = new FieldList();
     for (NamedExpression e : project.getSelections()) {
-      fl.addProjected(e.getExpr().accept(finder, null));
+      fl.addProjected(e.getExpr().accept(FieldReferenceFinder.INSTANCE, null));
     }
     return null;
   }
@@ -153,7 +149,7 @@ public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldD
   @Override
   public Void visitOrder(Order order, FieldList fl) {
     for (Ordering o : order.getOrderings()) {
-      fl.addReferenced(o.getExpr().accept(finder, null));
+      fl.addReferenced(o.getExpr().accept(FieldReferenceFinder.INSTANCE, null));
     }
     return null;
   }
@@ -163,7 +159,7 @@ public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldD
     {
       FieldList leftList = fl.clone();
       for (JoinCondition c : join.getConditions()) {
-        leftList.addReferenced(c.getLeft().accept(finder, null));
+        leftList.addReferenced(c.getLeft().accept(FieldReferenceFinder.INSTANCE, null));
       }
       join.getLeft().accept(this, leftList);
     }
@@ -171,7 +167,7 @@ public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldD
     {
       FieldList rightList = fl.clone();
       for (JoinCondition c : join.getConditions()) {
-        rightList.addReferenced(c.getRight().accept(finder, null));
+        rightList.addReferenced(c.getRight().accept(FieldReferenceFinder.INSTANCE, null));
       }
       join.getLeft().accept(this, rightList);
     }
@@ -191,28 +187,4 @@ public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldD
     }
     return null;
   }
-
-
-  /**
-   * Search through a LogicalExpression, finding all internal schema path references and returning them in a set.
-   */
-  private class FieldReferenceFinder extends AbstractExprVisitor<Set<SchemaPath>, Void, RuntimeException> {
-
-    @Override
-    public Set<SchemaPath> visitSchemaPath(SchemaPath path, Void value) {
-      Set<SchemaPath> set = Sets.newHashSet();
-      set.add(path);
-      return set;
-    }
-
-    @Override
-    public Set<SchemaPath> visitUnknown(LogicalExpression e, Void value) {
-      Set<SchemaPath> paths = Sets.newHashSet();
-      for (LogicalExpression ex : e) {
-        paths.addAll(ex.accept(this, null));
-      }
-      return paths;
-    }
-
-  }
 }
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 0900ba7..b335b6c 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
@@ -100,7 +100,7 @@ public class ConvertCountToDirectScanPrule extends Prule {
   public void onMatch(RelOptRuleCall call) {
     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 DrillProjectRel project = call.rels.length == 3 ? call.rel(1) : null;
 
     final GroupScan oldGrpScan = scan.getGroupScan();
     final PlannerSettings settings = PrelUtil.getPlannerSettings(call.getPlanner());
@@ -130,7 +130,7 @@ public class ConvertCountToDirectScanPrule extends Prule {
 
     final ScanStats scanStats = new ScanStats(ScanStats.GroupScanProperty.EXACT_ROW_COUNT, 1, 1, scanRowType.getFieldCount());
     final int numFiles = oldGrpScan.hasFiles() ? oldGrpScan.getFiles().size() : -1;
-    final GroupScan directScan = new MetadataDirectGroupScan(reader, oldGrpScan.getSelectionRoot(), numFiles, scanStats, false);
+    final GroupScan directScan = new MetadataDirectGroupScan(reader, oldGrpScan.getSelectionRoot(), numFiles, scanStats, false, oldGrpScan.usedMetastore());
 
     final DirectScanPrel newScan = DirectScanPrel.create(scan, scan.getTraitSet().plus(Prel.DRILL_PHYSICAL)
         .plus(DrillDistributionTrait.SINGLETON), directScan, scanRowType);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataAggPrel.java
new file mode 100644
index 0000000..a50f1a8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataAggPrel.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.MetadataAggPOP;
+import org.apache.drill.exec.planner.common.DrillRelNode;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.metastore.analyze.MetadataAggregateContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+public class MetadataAggPrel extends SingleRel implements DrillRelNode, Prel {
+  private final MetadataAggregateContext context;
+
+  public MetadataAggPrel(RelOptCluster cluster, RelTraitSet traits, RelNode input,
+      MetadataAggregateContext context) {
+    super(cluster, traits, input);
+    this.context = context;
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getInput();
+    MetadataAggPOP physicalOperator = new MetadataAggPOP(child.getPhysicalOperator(creator), context);
+    return creator.addMetadata(this, physicalOperator);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitPrel(this, value);
+  }
+
+  @Override
+  public BatchSchema.SelectionVectorMode[] getSupportedEncodings() {
+    return BatchSchema.SelectionVectorMode.DEFAULT;
+  }
+
+  @Override
+  public BatchSchema.SelectionVectorMode getEncoding() {
+    return BatchSchema.SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public boolean needsFinalColumnReordering() {
+    return true;
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return PrelUtil.iter(getInput());
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    Preconditions.checkState(inputs.size() == 1);
+    return new MetadataAggPrel(getCluster(), traitSet, inputs.iterator().next(), context);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataAggPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataAggPrule.java
new file mode 100644
index 0000000..2ce6e46
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataAggPrule.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.MetadataAggRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class MetadataAggPrule extends Prule {
+  public static final MetadataAggPrule INSTANCE = new MetadataAggPrule();
+
+  public MetadataAggPrule() {
+    super(RelOptHelper.any(MetadataAggRel.class, DrillRel.DRILL_LOGICAL),
+        "MetadataAggPrule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    MetadataAggRel relNode = call.rel(0);
+    RelNode input = relNode.getInput();
+
+    int groupByExprsSize = relNode.getContext().groupByExpressions().size();
+
+    // group by expressions will be returned first
+    RelCollation collation = RelCollations.of(IntStream.range(1, groupByExprsSize)
+        .mapToObj(RelFieldCollation::new)
+        .collect(Collectors.toList()));
+
+    // TODO: update DrillDistributionTrait when implemented parallelization for metadata collecting (see DRILL-7433)
+    RelTraitSet traits = call.getPlanner().emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
+    traits = groupByExprsSize > 0 ? traits.plus(collation) : traits;
+    RelNode convertedInput = convert(input, traits);
+    call.transformTo(
+        new MetadataAggPrel(relNode.getCluster(), traits, convertedInput, relNode.getContext()));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataControllerPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataControllerPrel.java
new file mode 100644
index 0000000..fb78a60
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataControllerPrel.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.exec.metastore.analyze.MetastoreAnalyzeConstants;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.MetadataControllerPOP;
+import org.apache.drill.exec.planner.common.DrillRelNode;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.metastore.analyze.MetadataControllerContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+public class MetadataControllerPrel extends BiRel implements DrillRelNode, Prel {
+  private final MetadataControllerContext context;
+
+  protected MetadataControllerPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left,
+      RelNode right, MetadataControllerContext context) {
+    super(cluster, traits, left, right);
+    this.context = context;
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel left = (Prel) this.getLeft();
+    Prel right = (Prel) this.getRight();
+    MetadataControllerPOP physicalOperator =
+        new MetadataControllerPOP(left.getPhysicalOperator(creator), right.getPhysicalOperator(creator), context);
+    return creator.addMetadata(this, physicalOperator);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitPrel(this, value);
+  }
+
+  @Override
+  public BatchSchema.SelectionVectorMode[] getSupportedEncodings() {
+    return BatchSchema.SelectionVectorMode.ALL;
+  }
+
+  @Override
+  public BatchSchema.SelectionVectorMode getEncoding() {
+    return BatchSchema.SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    Preconditions.checkArgument(inputs.size() == 2);
+    return new MetadataControllerPrel(getCluster(), traitSet, inputs.get(0), inputs.get(1), context);
+  }
+
+  @Override
+  public boolean needsFinalColumnReordering() {
+    return true;
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return PrelUtil.iter(getLeft(), getRight());
+  }
+
+  @Override
+  protected RelDataType deriveRowType() {
+    return getCluster().getTypeFactory().builder()
+        .add(MetastoreAnalyzeConstants.OK_FIELD_NAME, SqlTypeName.BOOLEAN)
+        .add(MetastoreAnalyzeConstants.SUMMARY_FIELD_NAME, SqlTypeName.VARCHAR)
+        .build();
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataControllerPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataControllerPrule.java
new file mode 100644
index 0000000..70f2f812
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataControllerPrule.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.MetadataControllerRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+
+public class MetadataControllerPrule extends Prule {
+  public static final MetadataControllerPrule INSTANCE = new MetadataControllerPrule();
+
+  public MetadataControllerPrule() {
+    super(RelOptHelper.some(MetadataControllerRel.class, DrillRel.DRILL_LOGICAL,
+        RelOptHelper.any(RelNode.class)), "MetadataControllerPrule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    MetadataControllerRel relNode = call.rel(0);
+    RelNode left = relNode.getLeft();
+    RelNode right = relNode.getRight();
+    RelTraitSet traits = call.getPlanner().emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
+    RelNode convertedLeft = convert(left, traits);
+    RelNode convertedRight = convert(right, traits);
+    call.transformTo(new MetadataControllerPrel(relNode.getCluster(),
+        relNode.getTraitSet().plus(Prel.DRILL_PHYSICAL), convertedLeft, convertedRight, relNode.getContext()));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataHandlerPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataHandlerPrel.java
new file mode 100644
index 0000000..42cc9fb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataHandlerPrel.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.MetadataHandlerPOP;
+import org.apache.drill.exec.planner.common.DrillRelNode;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.metastore.analyze.MetadataHandlerContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+public class MetadataHandlerPrel extends SingleRel implements DrillRelNode, Prel {
+  private final MetadataHandlerContext context;
+
+  protected MetadataHandlerPrel(RelOptCluster cluster, RelTraitSet traits, RelNode input, MetadataHandlerContext context) {
+    super(cluster, traits, input);
+    this.context = context;
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getInput();
+    MetadataHandlerPOP physicalOperator = new MetadataHandlerPOP(child.getPhysicalOperator(creator), context);
+
+    return creator.addMetadata(this, physicalOperator);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitPrel(this, value);
+  }
+
+  @Override
+  public BatchSchema.SelectionVectorMode[] getSupportedEncodings() {
+    return BatchSchema.SelectionVectorMode.ALL;
+  }
+
+  @Override
+  public BatchSchema.SelectionVectorMode getEncoding() {
+    return BatchSchema.SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public boolean needsFinalColumnReordering() {
+    return true;
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return PrelUtil.iter(getInput());
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    Preconditions.checkState(inputs.size() == 1);
+    return new MetadataHandlerPrel(getCluster(), traitSet, inputs.iterator().next(), context);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataHandlerPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataHandlerPrule.java
new file mode 100644
index 0000000..8424469
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MetadataHandlerPrule.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.MetadataHandlerRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+
+public class MetadataHandlerPrule extends Prule {
+  public static final MetadataHandlerPrule INSTANCE = new MetadataHandlerPrule();
+
+  public MetadataHandlerPrule() {
+    super(RelOptHelper.any(MetadataHandlerRel.class, DrillRel.DRILL_LOGICAL),
+        "MetadataHandlerPrule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    MetadataHandlerRel relNode = call.rel(0);
+    RelNode input = relNode.getInput();
+    RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.DEFAULT);
+    RelNode convertedInput = convert(input, traits);
+    call.transformTo(new MetadataHandlerPrel(relNode.getCluster(), traits,
+        convertedInput,
+        relNode.getContext()));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
index 4381c00..e6cba6b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
@@ -49,7 +49,7 @@ public class PrelUtil {
     final List<String> childFields = rowType.getFieldNames();
 
     for (RelFieldCollation fc : collation.getFieldCollations()) {
-      FieldReference fr = new FieldReference(childFields.get(fc.getFieldIndex()), ExpressionPosition.UNKNOWN, false);
+      FieldReference fr = new FieldReference(childFields.get(fc.getFieldIndex()), ExpressionPosition.UNKNOWN);
       orderExpr.add(new Ordering(fc.getDirection(), fr, fc.nullDirection));
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
index ef41ab1..7582b20 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
@@ -82,7 +82,7 @@ public class StreamAggPrel extends AggPrelBase implements Prel{
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
 
     Prel child = (Prel) this.getInput();
-    StreamingAggregate g = new StreamingAggregate(child.getPhysicalOperator(creator), keys, aggExprs, 1.0f);
+    StreamingAggregate g = new StreamingAggregate(child.getPhysicalOperator(creator), keys, aggExprs);
 
     return creator.addMetadata(this, g);
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/StarColumnConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/StarColumnConverter.java
index 91eeffe..e4b8800 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/StarColumnConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/StarColumnConverter.java
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.drill.exec.planner.StarColumnHelper;
+import org.apache.drill.exec.planner.physical.MetadataControllerPrel;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.ProjectAllowDupPrel;
 import org.apache.drill.exec.planner.physical.ProjectPrel;
@@ -154,7 +155,7 @@ public class StarColumnConverter extends BasePrelVisitor<Prel, Void, RuntimeExce
     // when the project expression is RexInPutRef, since we may insert a PAS which will
     // rename the projected fields.
 
-    RelNode child = ((Prel) prel.getInput(0)).accept(this, null);
+    Prel child = ((Prel) prel.getInput(0)).accept(this, null);
 
     List<String> fieldNames = Lists.newArrayList();
 
@@ -176,7 +177,7 @@ public class StarColumnConverter extends BasePrelVisitor<Prel, Void, RuntimeExce
     ProjectPrel newProj = (ProjectPrel) prel.copy(prel.getTraitSet(), child, prel.getProjects(), rowType);
 
     if (ProjectRemoveRule.isTrivial(newProj)) {
-      return (Prel) child;
+      return child;
     } else {
       return newProj;
     }
@@ -184,6 +185,10 @@ public class StarColumnConverter extends BasePrelVisitor<Prel, Void, RuntimeExce
 
   @Override
   public Prel visitPrel(Prel prel, Void value) throws RuntimeException {
+    if (prel instanceof MetadataControllerPrel) {
+      // disallow renaming projections for analyze command
+      return prel;
+    }
     // Require prefix rename : there exists other expression, in addition to a star column.
     if (!prefixedForStar  // not set yet.
         && StarColumnHelper.containsStarColumn(prel.getRowType())
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index 5ce9c19..c26e39f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -31,6 +31,7 @@ import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.ValidationException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.MetadataException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.ops.QueryContext.SqlStatementType;
 import org.apache.drill.exec.physical.PhysicalPlan;
@@ -40,6 +41,7 @@ import org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler;
 import org.apache.drill.exec.planner.sql.handlers.DescribeSchemaHandler;
 import org.apache.drill.exec.planner.sql.handlers.DescribeTableHandler;
 import org.apache.drill.exec.planner.sql.handlers.ExplainHandler;
+import org.apache.drill.exec.planner.sql.handlers.MetastoreAnalyzeTableHandler;
 import org.apache.drill.exec.planner.sql.handlers.RefreshMetadataHandler;
 import org.apache.drill.exec.planner.sql.handlers.ResetOptionHandler;
 import org.apache.drill.exec.planner.sql.handlers.SchemaHandler;
@@ -49,13 +51,13 @@ import org.apache.drill.exec.planner.sql.parser.DrillSqlCall;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlDescribeTable;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlResetOption;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlSetOption;
-import org.apache.drill.exec.planner.sql.parser.SqlCreateTable;
 import org.apache.drill.exec.planner.sql.parser.SqlSchema;
 import org.apache.drill.exec.testing.ControlsInjector;
 import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
+import org.apache.drill.shaded.guava.com.google.common.base.Throwables;
 import org.apache.hadoop.security.AccessControlException;
 
 public class DrillSqlWorker {
@@ -120,17 +122,70 @@ public class DrillSqlWorker {
   private static PhysicalPlan convertPlan(QueryContext context, String sql, Pointer<String> textPlan)
       throws ForemanSetupException, RelConversionException, IOException, ValidationException {
     Pointer<String> textPlanCopy = textPlan == null ? null : new Pointer<>(textPlan.value);
+    long retryAttempts = context.getOption(ExecConstants.METASTORE_RETRIEVAL_RETRY_ATTEMPTS).num_val;
     try {
-      return getQueryPlan(context, sql, textPlan);
+      return getPhysicalPlan(context, sql, textPlan, retryAttempts);
     } catch (Exception e) {
       logger.trace("There was an error during conversion into physical plan. " +
           "Will sync remote and local function registries if needed and retry " +
           "in case if issue was due to missing function implementation.", e);
+      // it is prohibited to retry query planning for ANALYZE statement since it changes
+      // query-level option values and will fail when rerunning with updated values
       if (context.getFunctionRegistry().syncWithRemoteRegistry(
-          context.getDrillOperatorTable().getFunctionRegistryVersion())) {
+              context.getDrillOperatorTable().getFunctionRegistryVersion())
+        && context.getSQLStatementType() != SqlStatementType.ANALYZE) {
         context.reloadDrillOperatorTable();
         logger.trace("Local function registry was synchronized with remote. Trying to find function one more time.");
-        return getQueryPlan(context, sql, textPlanCopy);
+        return getPhysicalPlan(context, sql, textPlanCopy, retryAttempts);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Converts sql query string into query physical plan.
+   * For the case when {@link MetadataException} was thrown during query planning,
+   * attempts to convert sql query string again, until number of attempts
+   * exceeds {@code metastore.retrieval.retry_attempts}.
+   * If number of attempts exceeds {@code metastore.retrieval.retry_attempts},
+   * query will be converted into physical plan without metastore usage.
+   *
+   * @param context       query context
+   * @param sql           sql query
+   * @param textPlan      text plan
+   * @param retryAttempts number of attempts to convert sql query string into query physical plan
+   * @return query physical plan
+   */
+  private static PhysicalPlan getPhysicalPlan(QueryContext context, String sql, Pointer<String> textPlan,
+      long retryAttempts) throws ForemanSetupException, RelConversionException, IOException, ValidationException {
+    try {
+      return getQueryPlan(context, sql, textPlan);
+    } catch (Exception e) {
+      Throwable rootCause = Throwables.getRootCause(e);
+      // Calcite wraps exceptions thrown during planning, so checks whether original exception is OutdatedMetadataException
+      if (rootCause instanceof MetadataException) {
+        // resets SqlStatementType to avoid errors when it is set during further attempts
+        context.clearSQLStatementType();
+        switch (((MetadataException) rootCause).getExceptionType()) {
+          case OUTDATED_METADATA:
+            logger.warn("Metastore table metadata is outdated. " +
+                "Retrying to obtain query plan without Metastore usage.");
+            break;
+          case INCONSISTENT_METADATA:
+            if (retryAttempts > 0) {
+              logger.debug("Table metadata was changed during query planning. " +
+                  "Retrying to obtain query plan using updated metadata.");
+              return getPhysicalPlan(context, sql, textPlan, --retryAttempts);
+            }
+            logger.warn("Table metadata was changing during query planning for all `metastore.retrieval.retry_attempts` = {} attempts.",
+                context.getOption(ExecConstants.METASTORE_RETRIEVAL_RETRY_ATTEMPTS).num_val);
+            break;
+          default:
+            logger.error("Exception happened during query planning using Metastore: {}", rootCause.getMessage(), rootCause);
+        }
+        logger.warn("Retrying to obtain query plan without Metastore usage.");
+        context.getOptions().setLocalOption(ExecConstants.METASTORE_ENABLED, false);
+        return getQueryPlan(context, sql, textPlan);
       }
       throw e;
     }
@@ -188,24 +243,25 @@ public class DrillSqlWorker {
         }
       case CREATE_TABLE:
         handler = ((DrillSqlCall) sqlNode).getSqlHandler(config, textPlan);
+        context.setSQLStatementType(SqlStatementType.CTAS);
+        break;
+      case SELECT:
+        handler = new DefaultSqlHandler(config, textPlan);
+        context.setSQLStatementType(SqlStatementType.SELECT);
         break;
       case DROP_TABLE:
       case CREATE_VIEW:
       case DROP_VIEW:
       case OTHER_DDL:
       case OTHER:
-        if (sqlNode instanceof SqlCreateTable) {
-          handler = ((DrillSqlCall) sqlNode).getSqlHandler(config, textPlan);
-          context.setSQLStatementType(SqlStatementType.CTAS);
-          break;
-        }
-
         if (sqlNode instanceof DrillSqlCall) {
           handler = ((DrillSqlCall) sqlNode).getSqlHandler(config);
-          if (handler instanceof AnalyzeTableHandler) {
+          if (handler instanceof AnalyzeTableHandler || handler instanceof MetastoreAnalyzeTableHandler) {
             context.setSQLStatementType(SqlStatementType.ANALYZE);
           } else if (handler instanceof RefreshMetadataHandler) {
             context.setSQLStatementType(SqlStatementType.REFRESH);
+          } else {
+            context.setSQLStatementType(SqlStatementType.OTHER);
           }
           break;
         }
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 3078253..3f398a4 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
@@ -35,6 +35,7 @@ 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.metastore.MetadataProviderManager;
+import org.apache.drill.exec.planner.types.DrillRelDataTypeSystem;
 import org.apache.drill.metastore.metadata.TableMetadataProvider;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalProject;
@@ -60,7 +61,6 @@ import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
@@ -86,7 +86,6 @@ import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.Util;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.QueryContext;
@@ -103,14 +102,14 @@ import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
 import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.util.DecimalUtility;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class responsible for managing parsing, validation and toRel conversion for sql statements.
  */
 public class SqlConverter {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SqlConverter.class);
-
-  private static DrillTypeSystem DRILL_TYPE_SYSTEM = new DrillTypeSystem();
+  private static final Logger logger = LoggerFactory.getLogger(SqlConverter.class);
 
   private final JavaTypeFactory typeFactory;
   private final SqlParser.Config parserConfig;
@@ -153,7 +152,7 @@ public class SqlConverter {
     this.sqlToRelConverterConfig = new SqlToRelConverterConfig();
     this.isInnerQuery = false;
     this.isExpandedView = false;
-    this.typeFactory = new JavaTypeFactoryImpl(DRILL_TYPE_SYSTEM);
+    this.typeFactory = new JavaTypeFactoryImpl(DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM);
     this.defaultSchema = context.getNewDefaultSchema();
     this.rootSchema = rootSchema(defaultSchema);
     this.temporarySchema = context.getConfig().getString(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE);
@@ -362,38 +361,6 @@ public class SqlConverter {
     }
   }
 
-  private static class DrillTypeSystem extends RelDataTypeSystemImpl {
-
-    @Override
-    public int getDefaultPrecision(SqlTypeName typeName) {
-      switch (typeName) {
-      case CHAR:
-      case BINARY:
-      case VARCHAR:
-      case VARBINARY:
-        return Types.MAX_VARCHAR_LENGTH;
-      default:
-        return super.getDefaultPrecision(typeName);
-      }
-    }
-
-    @Override
-    public int getMaxNumericScale() {
-      return 38;
-    }
-
-    @Override
-    public int getMaxNumericPrecision() {
-      return 38;
-    }
-
-    @Override
-    public boolean isSchemaCaseSensitive() {
-      // Drill uses case-insensitive and case-preserve policy
-      return false;
-    }
-  }
-
   public RelRoot toRel(final SqlNode validatedNode) {
     if (planner == null) {
       planner = new VolcanoPlanner(costFactory, settings);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
index e751e8c..232e81a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
@@ -18,14 +18,8 @@
 package org.apache.drill.exec.planner.sql.handlers;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
@@ -42,7 +36,6 @@ import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
-import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillScanRel;
 import org.apache.drill.exec.planner.logical.DrillScreenRel;
@@ -61,7 +54,6 @@ import org.apache.drill.exec.store.parquet.ParquetFormatConfig;
 import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
@@ -213,29 +205,6 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
       throw new UnsupportedOperationException();
     }
 
-    if (convertedRelNode instanceof DrillProjectRel) {
-      DrillProjectRel projectRel = (DrillProjectRel) convertedRelNode;
-      DrillScanRel scanRel = findScan(projectRel);
-      List<RelDataTypeField> fields = Lists.newArrayList();
-      RexBuilder b = projectRel.getCluster().getRexBuilder();
-      List<RexNode> projections = Lists.newArrayList();
-      // Get the original scan column names - after projection pushdown they should refer to the full col names
-      List<String> fieldNames = new ArrayList<>();
-      List<RelDataTypeField> fieldTypes = projectRel.getRowType().getFieldList();
-      for (SchemaPath colPath : scanRel.getGroupScan().getColumns()) {
-        fieldNames.add(colPath.toString());
-      }
-      for (int i =0; i < fieldTypes.size(); i++) {
-        projections.add(b.makeInputRef(projectRel, i));
-      }
-      // Get the projection row-types
-      RelDataType newRowType = RexUtil.createStructType(projectRel.getCluster().getTypeFactory(),
-              projections, fieldNames, null);
-      DrillProjectRel renamedProject = DrillProjectRel.create(convertedRelNode.getCluster(),
-              convertedRelNode.getTraitSet(), convertedRelNode, projections, newRowType);
-      convertedRelNode = renamedProject;
-    }
-
     final RelNode analyzeRel = new DrillAnalyzeRel(
         convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode, samplePercent);
 
@@ -249,12 +218,15 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
     return new DrillScreenRel(writerRel.getCluster(), writerRel.getTraitSet(), writerRel);
   }
 
-  private DrillScanRel findScan(RelNode rel) {
-    if (rel instanceof DrillScanRel) {
-      return (DrillScanRel) rel;
-    } else {
-      return findScan(rel.getInput(0));
+  public static DrillScanRel findScan(RelNode... rels) {
+    for (RelNode rel : rels) {
+      if (rel instanceof DrillScanRel) {
+        return (DrillScanRel) rel;
+      } else {
+        return findScan(rel.getInputs().toArray(new RelNode[0]));
+      }
     }
+    return null;
   }
   // Make sure no unsupported features in ANALYZE statement are used
   private static void verifyNoUnsupportedFunctions(final SqlAnalyzeTable analyzeTable) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreAnalyzeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreAnalyzeTableHandler.java
new file mode 100644
index 0000000..c34ee99
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreAnalyzeTableHandler.java
@@ -0,0 +1,536 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.sql.handlers;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlNumericLiteral;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.common.util.function.CheckedSupplier;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.metastore.analyze.AnalyzeInfoProvider;
+import org.apache.drill.exec.metastore.analyze.MetadataAggregateContext;
+import org.apache.drill.exec.metastore.analyze.MetadataControllerContext;
+import org.apache.drill.exec.metastore.analyze.MetadataHandlerContext;
+import org.apache.drill.exec.metastore.analyze.MetadataInfoCollector;
+import org.apache.drill.exec.metastore.analyze.MetastoreAnalyzeConstants;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillScreenRel;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.logical.MetadataAggRel;
+import org.apache.drill.exec.planner.logical.MetadataControllerRel;
+import org.apache.drill.exec.planner.logical.MetadataHandlerRel;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.sql.SchemaUtilites;
+import org.apache.drill.exec.planner.sql.parser.SqlMetastoreAnalyzeTable;
+import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.dfs.FormatSelection;
+import org.apache.drill.exec.util.Pointer;
+import org.apache.drill.exec.work.foreman.ForemanSetupException;
+import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
+import org.apache.drill.metastore.components.tables.BasicTablesRequests;
+import org.apache.drill.metastore.components.tables.MetastoreTableInfo;
+import org.apache.drill.metastore.exceptions.MetastoreException;
+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.shaded.guava.com.google.common.collect.ArrayListMultimap;
+import org.apache.drill.shaded.guava.com.google.common.collect.Multimap;
+import org.apache.parquet.Strings;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.drill.exec.planner.logical.DrillRelFactories.LOGICAL_BUILDER;
+
+/**
+ * Constructs plan to be executed for collecting metadata and storing it to the Metastore.
+ */
+public class MetastoreAnalyzeTableHandler extends DefaultSqlHandler {
+  private static final Logger logger = LoggerFactory.getLogger(MetastoreAnalyzeTableHandler.class);
+
+  public MetastoreAnalyzeTableHandler(SqlHandlerConfig config, Pointer<String> textPlan) {
+    super(config, textPlan);
+  }
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode sqlNode)
+      throws ValidationException, RelConversionException, IOException, ForemanSetupException {
+    if (!context.getOptions().getOption(ExecConstants.METASTORE_ENABLED_VALIDATOR)) {
+      throw UserException.validationError()
+          .message("Running ANALYZE TABLE REFRESH METADATA command when Metastore is disabled (`metastore.enabled` is set to false)")
+          .build(logger);
+    }
+    // disables during analyze to prevent using data about locations from the Metastore
+    context.getOptions().setLocalOption(ExecConstants.METASTORE_ENABLED, false);
+    SqlMetastoreAnalyzeTable sqlAnalyzeTable = unwrap(sqlNode, SqlMetastoreAnalyzeTable.class);
+
+    AbstractSchema drillSchema = SchemaUtilites.resolveToDrillSchema(
+        config.getConverter().getDefaultSchema(), sqlAnalyzeTable.getSchemaPath());
+    DrillTable table = getDrillTable(drillSchema, sqlAnalyzeTable.getName());
+
+    AnalyzeInfoProvider analyzeInfoProvider = table.getGroupScan().getAnalyzeInfoProvider();
+
+    if (analyzeInfoProvider == null) {
+      throw UserException.validationError()
+          .message("ANALYZE is not supported for group scan [%s]", table.getGroupScan())
+          .build(logger);
+    }
+
+    SqlIdentifier tableIdentifier = sqlAnalyzeTable.getTableIdentifier();
+    // creates select with DYNAMIC_STAR column and analyze specific columns to obtain corresponding table scan
+    SqlSelect scanSql = new SqlSelect(
+        SqlParserPos.ZERO,
+        SqlNodeList.EMPTY,
+        getColumnList(sqlAnalyzeTable, analyzeInfoProvider),
+        tableIdentifier,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null,
+        null
+    );
+
+    ConvertedRelNode convertedRelNode = validateAndConvert(rewrite(scanSql));
+    RelDataType validatedRowType = convertedRelNode.getValidatedRowType();
+
+    RelNode relScan = convertedRelNode.getConvertedNode();
+
+    DrillRel drel = convertToDrel(relScan, drillSchema, table, sqlAnalyzeTable);
+
+    Prel prel = convertToPrel(drel, validatedRowType);
+    logAndSetTextPlan("Drill Physical", prel, logger);
+    PhysicalOperator pop = convertToPop(prel);
+    PhysicalPlan plan = convertToPlan(pop);
+    log("Drill Plan", plan, logger);
+    return plan;
+  }
+
+  private DrillTable getDrillTable(AbstractSchema drillSchema, String tableName) {
+    Table tableFromSchema = SqlHandlerUtil.getTableFromSchema(drillSchema, tableName);
+
+    if (tableFromSchema == null) {
+      throw UserException.validationError()
+          .message("No table with given name [%s] exists in schema [%s]", tableName, drillSchema.getFullSchemaName())
+          .build(logger);
+    }
+
+    switch (tableFromSchema.getJdbcTableType()) {
+      case TABLE:
+        if (tableFromSchema instanceof DrillTable) {
+          return  (DrillTable) tableFromSchema;
+        } else {
+          throw UserException.validationError()
+              .message("ANALYZE does not support [%s] table kind", tableFromSchema.getClass().getSimpleName())
+              .build(logger);
+        }
+      default:
+        throw UserException.validationError()
+            .message("ANALYZE does not support [%s] object type", tableFromSchema.getJdbcTableType())
+            .build(logger);
+    }
+  }
+
+  /**
+   * Generates the column list with {@link SchemaPath#DYNAMIC_STAR} and columns required for analyze.
+   */
+  private SqlNodeList getColumnList(SqlMetastoreAnalyzeTable sqlAnalyzeTable, AnalyzeInfoProvider analyzeInfoProvider) {
+    SqlNodeList columnList = new SqlNodeList(SqlParserPos.ZERO);
+    columnList.add(new SqlIdentifier(SchemaPath.DYNAMIC_STAR, SqlParserPos.ZERO));
+    MetadataType metadataLevel = getMetadataType(sqlAnalyzeTable);
+    for (SqlIdentifier field : analyzeInfoProvider.getProjectionFields(metadataLevel, context.getPlannerSettings().getOptions())) {
+      columnList.add(field);
+    }
+    return columnList;
+  }
+
+  private MetadataType getMetadataType(SqlMetastoreAnalyzeTable sqlAnalyzeTable) {
+    SqlLiteral stringLiteral = sqlAnalyzeTable.getLevel();
+    // for the case when metadata level is not specified in ANALYZE statement,
+    // value from the `metastore.metadata.store.depth_level` option is used
+    String metadataLevel;
+    if (stringLiteral == null) {
+      metadataLevel = context.getOption(ExecConstants.METASTORE_METADATA_STORE_DEPTH_LEVEL).string_val;
+    } else {
+      metadataLevel = stringLiteral.toValue();
+    }
+    return metadataLevel != null ? MetadataType.valueOf(metadataLevel.toUpperCase()) : MetadataType.ALL;
+  }
+
+  /**
+   * Converts to Drill logical plan
+   */
+  private DrillRel convertToDrel(RelNode relNode, AbstractSchema schema,
+      DrillTable table, SqlMetastoreAnalyzeTable sqlAnalyzeTable) throws ForemanSetupException, IOException {
+    RelBuilder relBuilder = LOGICAL_BUILDER.create(relNode.getCluster(), null);
+
+    AnalyzeInfoProvider analyzeInfoProvider = table.getGroupScan().getAnalyzeInfoProvider();
+
+    List<String> schemaPath = schema.getSchemaPath();
+    String pluginName = schemaPath.get(0);
+    String workspaceName = Strings.join(schemaPath.subList(1, schemaPath.size()), AbstractSchema.SCHEMA_SEPARATOR);
+
+    TableInfo tableInfo = TableInfo.builder()
+        .name(sqlAnalyzeTable.getName())
+        .owner(table.getUserName())
+        .type(analyzeInfoProvider.getTableTypeName())
+        .storagePlugin(pluginName)
+        .workspace(workspaceName)
+        .build();
+
+    List<String> segmentColumns = analyzeInfoProvider.getSegmentColumns(table, context.getPlannerSettings().getOptions()).stream()
+        .map(SchemaPath::getRootSegmentPath)
+        .collect(Collectors.toList());
+    List<NamedExpression> segmentExpressions = segmentColumns.stream()
+        .map(partitionName ->
+            new NamedExpression(SchemaPath.getSimplePath(partitionName), FieldReference.getWithQuotedRef(partitionName)))
+        .collect(Collectors.toList());
+
+    List<MetadataInfo> rowGroupsInfo = Collections.emptyList();
+    List<MetadataInfo> filesInfo = Collections.emptyList();
+    Multimap<Integer, MetadataInfo> segments = ArrayListMultimap.create();
+
+    BasicTablesRequests basicRequests;
+    try {
+      basicRequests = context.getMetastoreRegistry().get()
+          .tables()
+          .basicRequests();
+    } catch (MetastoreException e) {
+      logger.error("Error when obtaining Metastore instance for table {}", sqlAnalyzeTable.getName(), e);
+      DrillRel convertedRelNode = convertToRawDrel(
+          relBuilder.values(
+                new String[]{MetastoreAnalyzeConstants.OK_FIELD_NAME, MetastoreAnalyzeConstants.SUMMARY_FIELD_NAME},
+                false, e.getMessage())
+              .build());
+      return new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
+    }
+
+    MetadataType metadataLevel = getMetadataType(sqlAnalyzeTable);
+
+    List<SchemaPath> interestingColumns = sqlAnalyzeTable.getFieldNames();
+
+    MetastoreTableInfo metastoreTableInfo = basicRequests.metastoreTableInfo(tableInfo);
+
+    List<MetadataInfo> allMetaToHandle = new ArrayList<>();
+    List<MetadataInfo> metadataToRemove = new ArrayList<>();
+
+    // Step 1: checks whether table metadata is present in the Metastore to determine
+    // whether incremental analyze may be produced
+    if (metastoreTableInfo.isExists()) {
+      RelNode finalRelNode = relNode;
+      CheckedSupplier<TableScan, SqlUnsupportedException> tableScanSupplier =
+          () -> AnalyzeTableHandler.findScan(convertToDrel(finalRelNode.getInput(0)));
+
+      MetadataInfoCollector metadataInfoCollector = analyzeInfoProvider.getMetadataInfoCollector(basicRequests,
+          tableInfo, (FormatSelection) table.getSelection(), context.getPlannerSettings(),
+          tableScanSupplier, interestingColumns, metadataLevel, segmentColumns.size());
+
+      if (!metadataInfoCollector.isOutdated()) {
+        DrillRel convertedRelNode = convertToRawDrel(
+            relBuilder.values(new String[]{MetastoreAnalyzeConstants.OK_FIELD_NAME, MetastoreAnalyzeConstants.SUMMARY_FIELD_NAME},
+                false, "Table metadata is up to date, analyze wasn't performed.")
+                .build());
+        return new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
+      }
+
+      // updates scan to read updated / new files, pass removed files into metadata handler
+      relNode = relNode.copy(relNode.getTraitSet(), Collections.singletonList(metadataInfoCollector.getPrunedScan()));
+
+      filesInfo = metadataInfoCollector.getFilesInfo();
+      segments = metadataInfoCollector.getSegmentsInfo();
+      rowGroupsInfo = metadataInfoCollector.getRowGroupsInfo();
+
+      allMetaToHandle = metadataInfoCollector.getAllMetaToHandle();
+      metadataToRemove = metadataInfoCollector.getMetadataToRemove();
+    }
+
+    // Step 2: constructs plan for producing analyze
+    DrillRel convertedRelNode = convertToRawDrel(relNode);
+
+    boolean createNewAggregations = true;
+
+    // List of columns for which statistics should be collected: interesting columns + segment columns
+    List<SchemaPath> statisticsColumns = interestingColumns == null ? null : new ArrayList<>(interestingColumns);
+    if (statisticsColumns != null) {
+      segmentColumns.stream()
+          .map(SchemaPath::getSimplePath)
+          .forEach(statisticsColumns::add);
+    }
+
+    SchemaPath locationField = analyzeInfoProvider.getLocationField(config.getContext().getOptions());
+
+    if (analyzeInfoProvider.supportsMetadataType(MetadataType.ROW_GROUP) && metadataLevel.includes(MetadataType.ROW_GROUP)) {
+      MetadataHandlerContext handlerContext = MetadataHandlerContext.builder()
+          .tableInfo(tableInfo)
+          .metadataToHandle(rowGroupsInfo)
+          .metadataType(MetadataType.ROW_GROUP)
+          .depthLevel(segmentExpressions.size())
+          .segmentColumns(segmentColumns)
+          .build();
+
+      convertedRelNode = getRowGroupAggRelNode(segmentExpressions, convertedRelNode, createNewAggregations,
+          statisticsColumns, handlerContext);
+
+      createNewAggregations = false;
+      locationField = SchemaPath.getSimplePath(MetastoreAnalyzeConstants.LOCATION_FIELD);
+    }
+
+    if (analyzeInfoProvider.supportsMetadataType(MetadataType.FILE) && metadataLevel.includes(MetadataType.FILE)) {
+      MetadataHandlerContext handlerContext = MetadataHandlerContext.builder()
+          .tableInfo(tableInfo)
+          .metadataToHandle(filesInfo)
+          .metadataType(MetadataType.FILE)
+          .depthLevel(segmentExpressions.size())
+          .segmentColumns(segmentColumns)
+          .build();
+
+      convertedRelNode = getFileAggRelNode(segmentExpressions, convertedRelNode,
+          createNewAggregations, statisticsColumns, locationField, handlerContext);
+
+      locationField = SchemaPath.getSimplePath(MetastoreAnalyzeConstants.LOCATION_FIELD);
+
+      createNewAggregations = false;
+    }
+
+    if (analyzeInfoProvider.supportsMetadataType(MetadataType.SEGMENT) && metadataLevel.includes(MetadataType.SEGMENT)) {
+      for (int i = segmentExpressions.size(); i > 0; i--) {
+        MetadataHandlerContext handlerContext = MetadataHandlerContext.builder()
+            .tableInfo(tableInfo)
+            .metadataToHandle(new ArrayList<>(segments.get(i - 1)))
+            .metadataType(MetadataType.SEGMENT)
+            .depthLevel(i)
+            .segmentColumns(segmentColumns.subList(0, i))
+            .build();
+
+        convertedRelNode = getSegmentAggRelNode(segmentExpressions, convertedRelNode,
+            createNewAggregations, statisticsColumns, locationField, analyzeInfoProvider, i, handlerContext);
+
+        locationField = SchemaPath.getSimplePath(MetastoreAnalyzeConstants.LOCATION_FIELD);
+
+        createNewAggregations = false;
+      }
+    }
+
+    if (analyzeInfoProvider.supportsMetadataType(MetadataType.TABLE) && metadataLevel.includes(MetadataType.TABLE)) {
+      MetadataHandlerContext handlerContext = MetadataHandlerContext.builder()
+          .tableInfo(tableInfo)
+          .metadataToHandle(Collections.emptyList())
+          .metadataType(MetadataType.TABLE)
+          .depthLevel(segmentExpressions.size())
+          .segmentColumns(segmentColumns)
+          .build();
+
+      convertedRelNode = getTableAggRelNode(convertedRelNode, createNewAggregations,
+          statisticsColumns, locationField, handlerContext);
+    } else {
+      throw new IllegalStateException("Analyze table with NONE level");
+    }
+
+    boolean useStatistics = context.getOptions().getOption(PlannerSettings.STATISTICS_USE);
+    SqlNumericLiteral samplePercentLiteral = sqlAnalyzeTable.getSamplePercent();
+    double samplePercent = samplePercentLiteral == null ? 100.0 : samplePercentLiteral.intValue(true);
+
+    // Step 3: adds rel nodes for producing statistics analyze if required
+    RelNode analyzeRel = useStatistics
+        ? new DrillAnalyzeRel(
+              convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertToRawDrel(relNode), samplePercent)
+        : convertToRawDrel(relBuilder.values(new String[]{""}, "").build());
+
+    MetadataControllerContext metadataControllerContext = MetadataControllerContext.builder()
+        .tableInfo(tableInfo)
+        .metastoreTableInfo(metastoreTableInfo)
+        .location(((FormatSelection) table.getSelection()).getSelection().getSelectionRoot())
+        .interestingColumns(interestingColumns)
+        .segmentColumns(segmentColumns)
+        .metadataToHandle(allMetaToHandle)
+        .metadataToRemove(metadataToRemove)
+        .analyzeMetadataLevel(metadataLevel)
+        .build();
+
+    convertedRelNode = new MetadataControllerRel(convertedRelNode.getCluster(),
+        convertedRelNode.getTraitSet(),
+        convertedRelNode,
+        analyzeRel,
+        metadataControllerContext);
+
+    return new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
+  }
+
+  private DrillRel getTableAggRelNode(DrillRel convertedRelNode, boolean createNewAggregations,
+      List<SchemaPath> statisticsColumns, SchemaPath locationField, MetadataHandlerContext handlerContext) {
+    SchemaPath lastModifiedTimeField =
+        SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL));
+
+    List<SchemaPath> excludedColumns = Arrays.asList(locationField, lastModifiedTimeField);
+
+    MetadataAggregateContext aggregateContext = MetadataAggregateContext.builder()
+        .groupByExpressions(Collections.emptyList())
+        .interestingColumns(statisticsColumns)
+        .createNewAggregations(createNewAggregations)
+        .excludedColumns(excludedColumns)
+        .build();
+
+    convertedRelNode = new MetadataAggRel(convertedRelNode.getCluster(),
+        convertedRelNode.getTraitSet(), convertedRelNode, aggregateContext);
+
+    convertedRelNode =
+        new MetadataHandlerRel(convertedRelNode.getCluster(),
+            convertedRelNode.getTraitSet(),
+            convertedRelNode,
+            handlerContext);
+    return convertedRelNode;
+  }
+
+  private DrillRel getSegmentAggRelNode(List<NamedExpression> segmentExpressions, DrillRel convertedRelNode,
+      boolean createNewAggregations, List<SchemaPath> statisticsColumns, SchemaPath locationField,
+      AnalyzeInfoProvider analyzeInfoProvider, int segmentLevel, MetadataHandlerContext handlerContext) {
+      SchemaPath lastModifiedTimeField =
+        SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL));
+
+    List<SchemaPath> excludedColumns = Arrays.asList(lastModifiedTimeField, locationField);
+
+    List<NamedExpression> groupByExpressions = new ArrayList<>();
+    groupByExpressions.add(analyzeInfoProvider.getParentLocationExpression(locationField));
+
+    groupByExpressions.addAll(segmentExpressions);
+
+    MetadataAggregateContext aggregateContext = MetadataAggregateContext.builder()
+        .groupByExpressions(groupByExpressions.subList(0, segmentLevel + 1))
+        .interestingColumns(statisticsColumns)
+        .createNewAggregations(createNewAggregations)
+        .excludedColumns(excludedColumns)
+        .build();
+
+    convertedRelNode = new MetadataAggRel(convertedRelNode.getCluster(),
+        convertedRelNode.getTraitSet(), convertedRelNode, aggregateContext);
+
+    convertedRelNode =
+        new MetadataHandlerRel(convertedRelNode.getCluster(),
+            convertedRelNode.getTraitSet(),
+            convertedRelNode,
+            handlerContext);
+    return convertedRelNode;
+  }
+
+  private DrillRel getFileAggRelNode(List<NamedExpression> segmentExpressions, DrillRel convertedRelNode,
+      boolean createNewAggregations, List<SchemaPath> statisticsColumns, SchemaPath locationField, MetadataHandlerContext handlerContext) {
+    SchemaPath lastModifiedTimeField =
+        SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL));
+
+    List<SchemaPath> excludedColumns = Arrays.asList(lastModifiedTimeField, locationField);
+
+    NamedExpression locationExpression =
+        new NamedExpression(locationField, FieldReference.getWithQuotedRef(MetastoreAnalyzeConstants.LOCATION_FIELD));
+    List<NamedExpression> fileGroupByExpressions = new ArrayList<>(segmentExpressions);
+    fileGroupByExpressions.add(locationExpression);
+
+    MetadataAggregateContext aggregateContext = MetadataAggregateContext.builder()
+        .groupByExpressions(fileGroupByExpressions)
+        .interestingColumns(statisticsColumns)
+        .createNewAggregations(createNewAggregations)
+        .excludedColumns(excludedColumns)
+        .build();
+
+    convertedRelNode = new MetadataAggRel(convertedRelNode.getCluster(),
+        convertedRelNode.getTraitSet(), convertedRelNode, aggregateContext);
+
+    convertedRelNode =
+        new MetadataHandlerRel(convertedRelNode.getCluster(),
+            convertedRelNode.getTraitSet(),
+            convertedRelNode,
+            handlerContext);
+    return convertedRelNode;
+  }
+
+  private DrillRel getRowGroupAggRelNode(List<NamedExpression> segmentExpressions, DrillRel convertedRelNode,
+      boolean createNewAggregations, List<SchemaPath> statisticsColumns, MetadataHandlerContext handlerContext) {
+    SchemaPath locationField =
+        SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_FQN_COLUMN_LABEL));
+    SchemaPath lastModifiedTimeField =
+        SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL));
+
+    String rowGroupIndexColumn = config.getContext().getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL);
+    SchemaPath rgiField = SchemaPath.getSimplePath(rowGroupIndexColumn);
+
+    List<NamedExpression> rowGroupGroupByExpressions =
+        getRowGroupExpressions(segmentExpressions, locationField, rowGroupIndexColumn, rgiField);
+
+    SchemaPath rowGroupStartField =
+        SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL));
+    SchemaPath rowGroupLengthField =
+        SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL));
+
+    List<SchemaPath> excludedColumns = Arrays.asList(lastModifiedTimeField, locationField, rgiField, rowGroupStartField, rowGroupLengthField);
+
+    MetadataAggregateContext aggregateContext = MetadataAggregateContext.builder()
+        .groupByExpressions(rowGroupGroupByExpressions)
+        .interestingColumns(statisticsColumns)
+        .createNewAggregations(createNewAggregations)
+        .excludedColumns(excludedColumns)
+        .build();
+
+    convertedRelNode = new MetadataAggRel(convertedRelNode.getCluster(),
+        convertedRelNode.getTraitSet(), convertedRelNode, aggregateContext);
+
+    convertedRelNode =
+        new MetadataHandlerRel(convertedRelNode.getCluster(),
+            convertedRelNode.getTraitSet(),
+            convertedRelNode,
+            handlerContext);
+    return convertedRelNode;
+  }
+
+  private List<NamedExpression> getRowGroupExpressions(List<NamedExpression> segmentExpressions,
+      SchemaPath locationField, String rowGroupIndexColumn, SchemaPath rgiField) {
+    List<NamedExpression> rowGroupGroupByExpressions = new ArrayList<>(segmentExpressions);
+    rowGroupGroupByExpressions.add(
+        new NamedExpression(rgiField,
+            FieldReference.getWithQuotedRef(rowGroupIndexColumn)));
+
+    rowGroupGroupByExpressions.add(
+        new NamedExpression(locationField, FieldReference.getWithQuotedRef(MetastoreAnalyzeConstants.LOCATION_FIELD)));
+    return rowGroupGroupByExpressions;
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreDropTableMetadataHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreDropTableMetadataHandler.java
new file mode 100644
index 0000000..3f1dbea
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreDropTableMetadataHandler.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.sql.handlers;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.SchemaUtilites;
+import org.apache.drill.exec.planner.sql.parser.SqlDropTableMetadata;
+import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.util.Pointer;
+import org.apache.drill.exec.work.foreman.ForemanSetupException;
+import org.apache.drill.metastore.components.tables.MetastoreTableInfo;
+import org.apache.drill.metastore.components.tables.Tables;
+import org.apache.drill.metastore.exceptions.MetastoreException;
+import org.apache.drill.metastore.metadata.TableInfo;
+import org.apache.parquet.Strings;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class MetastoreDropTableMetadataHandler extends DefaultSqlHandler {
+  private static final Logger logger = LoggerFactory.getLogger(MetastoreDropTableMetadataHandler.class);
+
+  public MetastoreDropTableMetadataHandler(SqlHandlerConfig config, Pointer<String> textPlan) {
+    super(config, textPlan);
+  }
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException {
+    if (!context.getOptions().getOption(ExecConstants.METASTORE_ENABLED_VALIDATOR)) {
+      throw UserException.validationError()
+          .message("Running ANALYZE TABLE DROP command when Metastore is disabled (`metastore.enabled` is set to false)")
+          .build(logger);
+    }
+
+    SqlDropTableMetadata dropTableMetadata = unwrap(sqlNode, SqlDropTableMetadata.class);
+
+    AbstractSchema drillSchema = SchemaUtilites.resolveToDrillSchema(
+        config.getConverter().getDefaultSchema(), dropTableMetadata.getSchemaPath());
+
+    List<String> schemaPath = drillSchema.getSchemaPath();
+    String pluginName = schemaPath.get(0);
+    String workspaceName = Strings.join(schemaPath.subList(1, schemaPath.size()), AbstractSchema.SCHEMA_SEPARATOR);
+
+    TableInfo tableInfo = TableInfo.builder()
+        .name(dropTableMetadata.getName())
+        .storagePlugin(pluginName)
+        .workspace(workspaceName)
+        .build();
+
+    try {
+      Tables tables = context.getMetastoreRegistry().get().tables();
+
+      MetastoreTableInfo metastoreTableInfo = tables.basicRequests()
+          .metastoreTableInfo(tableInfo);
+
+      if (!metastoreTableInfo.isExists()) {
+        if (dropTableMetadata.checkMetadataExistence()) {
+          throw UserException.validationError()
+              .message("Metadata for table [%s] not found.", dropTableMetadata.getName())
+              .build(logger);
+        }
+        return DirectPlan.createDirectPlan(context, false,
+            String.format("Metadata for table [%s] does not exist.", dropTableMetadata.getName()));
+      }
+
+      tables.modify()
+          .delete(tableInfo.toFilter())
+          .execute();
+    } catch (MetastoreException e) {
+      logger.error("Error when dropping metadata for table {}", dropTableMetadata.getName(), e);
+      return DirectPlan.createDirectPlan(context, false, e.getMessage());
+    }
+
+    return DirectPlan.createDirectPlan(context, true,
+        String.format("Metadata for table [%s] dropped.", dropTableMetadata.getName()));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
index c6dac43..7b6a19a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/RefreshMetadataHandler.java
@@ -78,7 +78,7 @@ public class RefreshMetadataHandler extends DefaultSqlHandler {
 
       final String tableName = refreshTable.getName();
       final SqlNodeList columnList = getColumnList(refreshTable);
-      final Set<String> columnSet = getColumnRootSegments(columnList);
+      final Set<SchemaPath> columnSet = getColumnRootSegments(columnList);
       final SqlLiteral allColumns = refreshTable.getAllColumns();
 
       if (tableName.contains("*") || tableName.contains("?")) {
@@ -143,12 +143,12 @@ public class RefreshMetadataHandler extends DefaultSqlHandler {
     }
   }
 
-  private Set<String> getColumnRootSegments(SqlNodeList columnList) {
-    Set<String> columnSet = new HashSet<>();
+  private Set<SchemaPath> getColumnRootSegments(SqlNodeList columnList) {
+    Set<SchemaPath> columnSet = new HashSet<>();
     if (columnList != null) {
       for (SqlNode column : columnList.getList()) {
         // Add only the root segment. Collect metadata for all the columns under that root segment
-        columnSet.add(SchemaPath.parseFromString(column.toString()).getRootSegmentPath());
+        columnSet.add(SchemaPath.getSimplePath(SchemaPath.parseFromString(column.toString()).getRootSegmentPath()));
       }
     }
     return columnSet;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
index e9f586e..a7602e1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
@@ -64,6 +64,8 @@ public class CompoundIdentifierConverter extends SqlShuttle {
     // returned by getOperandList() method for concrete SqlCall implementation.
     REWRITE_RULES = ImmutableMap.<Class<? extends SqlCall>, RewriteType[]>builder()
         .put(SqlAnalyzeTable.class, arrayOf(D, D, E, D))
+        .put(SqlMetastoreAnalyzeTable.class, arrayOf(D, E, D, D, D))
+        .put(SqlDropTableMetadata.class, arrayOf(D, D, D))
         .put(SqlSelect.class, arrayOf(D, E, D, E, E, E, E, E, D, D))
         .put(SqlCreateTable.class, arrayOf(D, D, D, E, D, D))
         .put(SqlCreateView.class, arrayOf(D, E, E, D))
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
index c5bbc5e..039fe0e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
@@ -44,7 +44,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
  * SQL tree for ANALYZE statement.
  */
 public class SqlAnalyzeTable extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("ANALYZE_TABLE", SqlKind.OTHER) {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("ANALYZE_TABLE", SqlKind.OTHER_DDL) {
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
       Preconditions.checkArgument(operands.length == 4, "SqlAnalyzeTable.createCall() has to get 4 operands!");
       return new SqlAnalyzeTable(pos, (SqlIdentifier) operands[0], (SqlLiteral) operands[1],
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
index dbc6788..f5bf0b7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
@@ -42,8 +42,8 @@ public class SqlDropTable extends DrillSqlCall {
     }
   };
 
-  private SqlIdentifier tableName;
-  private boolean tableExistenceCheck;
+  private final SqlIdentifier tableName;
+  private final boolean tableExistenceCheck;
 
   public SqlDropTable(SqlParserPos pos, SqlIdentifier tableName, SqlLiteral tableExistenceCheck) {
     this(pos, tableName, tableExistenceCheck.booleanValue());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTableMetadata.java
similarity index 54%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTableMetadata.java
index dbc6788..3bec792 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTableMetadata.java
@@ -17,11 +17,6 @@
  */
 package org.apache.drill.exec.planner.sql.parser;
 
-import java.util.List;
-
-import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
-import org.apache.drill.exec.planner.sql.handlers.DropTableHandler;
-import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
@@ -31,28 +26,34 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.Util;
+import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
+import org.apache.drill.exec.planner.sql.handlers.MetastoreDropTableMetadataHandler;
+import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
+import org.apache.drill.exec.util.Pointer;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class SqlDropTableMetadata extends DrillSqlCall {
 
-public class SqlDropTable extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_TABLE", SqlKind.DROP_TABLE) {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_TABLE_METADATA", SqlKind.OTHER) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
-      return new SqlDropTable(pos, (SqlIdentifier) operands[0], (SqlLiteral) operands[1]);
+      return new SqlDropTableMetadata(pos, (SqlIdentifier) operands[0], (SqlLiteral) operands[1], (SqlLiteral) operands[2]);
     }
   };
 
-  private SqlIdentifier tableName;
-  private boolean tableExistenceCheck;
+  private final SqlIdentifier tableName;
+  private final boolean checkMetadataExistence;
+  private final DropMetadataType dropType;
 
-  public SqlDropTable(SqlParserPos pos, SqlIdentifier tableName, SqlLiteral tableExistenceCheck) {
-    this(pos, tableName, tableExistenceCheck.booleanValue());
-  }
-
-  public SqlDropTable(SqlParserPos pos, SqlIdentifier tableName, boolean tableExistenceCheck) {
+  public SqlDropTableMetadata(SqlParserPos pos, SqlIdentifier tableName, SqlLiteral dropType, SqlLiteral checkMetadataExistence) {
     super(pos);
     this.tableName = tableName;
-    this.tableExistenceCheck = tableExistenceCheck;
+    this.dropType = DropMetadataType.valueOf(dropType.getStringValue().toUpperCase());
+    this.checkMetadataExistence = checkMetadataExistence.booleanValue();
   }
 
   @Override
@@ -62,52 +63,61 @@ public class SqlDropTable extends DrillSqlCall {
 
   @Override
   public List<SqlNode> getOperandList() {
-    final List<SqlNode> ops =
-        ImmutableList.of(
-            tableName,
-            SqlLiteral.createBoolean(tableExistenceCheck, SqlParserPos.ZERO)
-        );
-    return ops;
+    return Arrays.asList(
+        tableName,
+        SqlLiteral.createCharString(dropType.name(), SqlParserPos.ZERO),
+        SqlLiteral.createBoolean(checkMetadataExistence, SqlParserPos.ZERO)
+    );
   }
 
   @Override
   public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-    writer.keyword("DROP");
+    writer.keyword("ANALYZE");
     writer.keyword("TABLE");
-    if (tableExistenceCheck) {
+    tableName.unparse(writer, leftPrec, rightPrec);
+    writer.keyword("DROP");
+    writer.keyword(dropType.name());
+    if (checkMetadataExistence) {
       writer.keyword("IF");
       writer.keyword("EXISTS");
     }
-    tableName.unparse(writer, leftPrec, rightPrec);
+  }
+
+  @Override
+  public AbstractSqlHandler getSqlHandler(SqlHandlerConfig config, Pointer<String> textPlan) {
+    return new MetastoreDropTableMetadataHandler(config, textPlan);
   }
 
   @Override
   public AbstractSqlHandler getSqlHandler(SqlHandlerConfig config) {
-    return new DropTableHandler(config);
+    return getSqlHandler(config, null);
   }
 
-  public List<String> getSchema() {
+  public List<String> getSchemaPath() {
     if (tableName.isSimple()) {
-      return ImmutableList.of();
+      return Collections.emptyList();
     }
 
-    return tableName.names.subList(0, tableName.names.size()-1);
+    return tableName.names.subList(0, tableName.names.size() - 1);
   }
 
   public String getName() {
-    if (tableName.isSimple()) {
-      return tableName.getSimple();
-    }
-
-    return tableName.names.get(tableName.names.size() - 1);
+    return Util.last(tableName.names);
   }
 
-  public SqlIdentifier getTableIdentifier() {
-    return tableName;
+  public boolean checkMetadataExistence() {
+    return checkMetadataExistence;
   }
 
-  public boolean checkTableExistence() {
-    return tableExistenceCheck;
+  public DropMetadataType getDropType() {
+    return dropType;
   }
 
+  /**
+   * Enum for metadata types to drop.
+   */
+  public enum DropMetadataType {
+    METADATA,
+    STATISTICS
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlMetastoreAnalyzeTable.java
similarity index 52%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlMetastoreAnalyzeTable.java
index c5bbc5e..2a22f90 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlMetastoreAnalyzeTable.java
@@ -17,8 +17,6 @@
  */
 package org.apache.drill.exec.planner.sql.parser;
 
-import java.util.List;
-
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
@@ -31,39 +29,39 @@ import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.Util;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
-import org.apache.drill.exec.planner.sql.handlers.AnalyzeTableHandler;
+import org.apache.drill.exec.planner.sql.handlers.MetastoreAnalyzeTableHandler;
 import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
 import org.apache.drill.exec.util.Pointer;
 
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
 
-/**
- * SQL tree for ANALYZE statement.
- */
-public class SqlAnalyzeTable extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("ANALYZE_TABLE", SqlKind.OTHER) {
+public class SqlMetastoreAnalyzeTable extends DrillSqlCall {
+
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("METASTORE_ANALYZE_TABLE", SqlKind.OTHER_DDL) {
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
-      Preconditions.checkArgument(operands.length == 4, "SqlAnalyzeTable.createCall() has to get 4 operands!");
-      return new SqlAnalyzeTable(pos, (SqlIdentifier) operands[0], (SqlLiteral) operands[1],
-          (SqlNodeList) operands[2], (SqlNumericLiteral) operands[3]
-      );
+      return new SqlMetastoreAnalyzeTable(pos, (SqlIdentifier) operands[0], (SqlNodeList) operands[1], operands[2],
+          (SqlLiteral) operands[3], (SqlNumericLiteral) operands[4]);
     }
   };
 
-  private final SqlIdentifier tblName;
-  private final SqlLiteral estimate;
+  private final SqlIdentifier tableName;
   private final SqlNodeList fieldList;
+  private final SqlLiteral level;
+  private final SqlLiteral estimate;
   private final SqlNumericLiteral samplePercent;
 
-  public SqlAnalyzeTable(SqlParserPos pos, SqlIdentifier tblName, SqlLiteral estimate,
-      SqlNodeList fieldList, SqlNumericLiteral samplePercent) {
+  public SqlMetastoreAnalyzeTable(SqlParserPos pos, SqlIdentifier tableName, SqlNodeList fieldList,
+      SqlNode level, SqlLiteral estimate, SqlNumericLiteral samplePercent) {
     super(pos);
-    this.tblName = tblName;
-    this.estimate = estimate;
+    this.tableName = tableName;
     this.fieldList = fieldList;
+    this.level = level != null ? SqlLiteral.unchain(level) : null;
+    this.estimate = estimate;
     this.samplePercent = samplePercent;
   }
 
@@ -74,39 +72,47 @@ public class SqlAnalyzeTable extends DrillSqlCall {
 
   @Override
   public List<SqlNode> getOperandList() {
-    final List<SqlNode> operands = Lists.newArrayListWithCapacity(4);
-    operands.add(tblName);
-    operands.add(estimate);
-    operands.add(fieldList);
-    operands.add(samplePercent);
-    return operands;
+    return Arrays.asList(tableName, fieldList, level, estimate, samplePercent);
   }
 
   @Override
   public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
     writer.keyword("ANALYZE");
     writer.keyword("TABLE");
-    tblName.unparse(writer, leftPrec, rightPrec);
-    writer.keyword(estimate.booleanValue() ? "ESTIMATE" : "COMPUTE");
-    writer.keyword("STATISTICS");
-
-    if (fieldList != null && fieldList.size() > 0) {
-      writer.keyword("(");
-      fieldList.get(0).unparse(writer, leftPrec, rightPrec);
-      for (int i = 1; i < fieldList.size(); i++) {
-        writer.keyword(",");
-        fieldList.get(i).unparse(writer, leftPrec, rightPrec);
+    tableName.unparse(writer, leftPrec, rightPrec);
+    if (fieldList != null) {
+      writer.keyword("COLUMNS");
+      if (fieldList.size() > 0) {
+        writer.keyword("(");
+        fieldList.get(0).unparse(writer, leftPrec, rightPrec);
+        for (int i = 1; i < fieldList.size(); i++) {
+          writer.keyword(",");
+          fieldList.get(i).unparse(writer, leftPrec, rightPrec);
+        }
+        writer.keyword(")");
+      } else {
+        writer.keyword("NONE");
       }
-      writer.keyword(")");
     }
-    writer.keyword("SAMPLE");
-    samplePercent.unparse(writer, leftPrec, rightPrec);
-    writer.keyword("PERCENT");
+    writer.keyword("REFRESH");
+    writer.keyword("METADATA");
+    if (level != null) {
+      level.unparse(writer, leftPrec, rightPrec);
+    }
+    if (estimate != null) {
+      writer.keyword(estimate.booleanValue() ? "ESTIMATE" : "COMPUTE");
+      writer.keyword("STATISTICS");
+    }
+    if (samplePercent != null) {
+      writer.keyword("SAMPLE");
+      samplePercent.unparse(writer, leftPrec, rightPrec);
+      writer.keyword("PERCENT");
+    }
   }
 
   @Override
   public AbstractSqlHandler getSqlHandler(SqlHandlerConfig config, Pointer<String> textPlan) {
-    return new AnalyzeTableHandler(config, textPlan);
+    return new MetastoreAnalyzeTableHandler(config, textPlan);
   }
 
   @Override
@@ -115,42 +121,45 @@ public class SqlAnalyzeTable extends DrillSqlCall {
   }
 
   public List<String> getSchemaPath() {
-    if (tblName.isSimple()) {
-      return ImmutableList.of();
+    if (tableName.isSimple()) {
+      return Collections.emptyList();
     }
 
-    return tblName.names.subList(0, tblName.names.size() - 1);
+    return tableName.names.subList(0, tableName.names.size() - 1);
   }
 
   public SqlIdentifier getTableIdentifier() {
-    return tblName;
+    return tableName;
   }
 
   public String getName() {
-    return Util.last(tblName.names);
+    return Util.last(tableName.names);
   }
 
-  public List<String> getFieldNames() {
+  public List<SchemaPath> getFieldNames() {
     if (fieldList == null) {
-      return ImmutableList.of();
+      return null;
     }
 
-    List<String> columnNames = Lists.newArrayList();
-    for (SqlNode node : fieldList.getList()) {
-      columnNames.add(node.toString());
-    }
-    return columnNames;
+    return fieldList.getList().stream()
+        .map(SqlNode::toString)
+        .map(SchemaPath::parseFromString)
+        .collect(Collectors.toList());
   }
 
   public SqlNodeList getFieldList() {
     return fieldList;
   }
 
-  public boolean getEstimate() {
-    return estimate.booleanValue();
+  public SqlLiteral getLevel() {
+    return level;
+  }
+
+  public SqlLiteral getEstimate() {
+    return estimate;
   }
 
-  public int getSamplePercent() {
-    return samplePercent.intValue(true);
+  public SqlNumericLiteral getSamplePercent() {
+    return samplePercent;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/DrillRelDataTypeSystem.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/DrillRelDataTypeSystem.java
index c32a28d..cdaad29 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/DrillRelDataTypeSystem.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/DrillRelDataTypeSystem.java
@@ -19,12 +19,27 @@ package org.apache.drill.exec.planner.types;
 
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.common.types.Types;
 
 public class DrillRelDataTypeSystem extends RelDataTypeSystemImpl {
 
   public static final RelDataTypeSystem DRILL_REL_DATATYPE_SYSTEM = new DrillRelDataTypeSystem();
 
   @Override
+  public int getDefaultPrecision(SqlTypeName typeName) {
+    switch (typeName) {
+      case CHAR:
+      case BINARY:
+      case VARCHAR:
+      case VARBINARY:
+        return Types.MAX_VARCHAR_LENGTH;
+      default:
+        return super.getDefaultPrecision(typeName);
+    }
+  }
+
+  @Override
   public int getMaxNumericScale() {
     return 38;
   }
@@ -34,4 +49,10 @@ public class DrillRelDataTypeSystem extends RelDataTypeSystemImpl {
     return 38;
   }
 
+  @Override
+  public boolean isSchemaCaseSensitive() {
+    // Drill uses case-insensitive policy
+    return false;
+  }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java
index 486fb1e..84afa3b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java
@@ -23,8 +23,6 @@ import org.apache.drill.exec.ops.MetricDef;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 
 public abstract class AbstractBinaryRecordBatch<T extends PhysicalOperator> extends  AbstractRecordBatch<T> {
-  private static final org.slf4j.Logger logger =
-    org.slf4j.LoggerFactory.getLogger(new Object() {}.getClass().getEnclosingClass());
 
   protected final RecordBatch left;
   protected final RecordBatch right;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
index a389303..73f03f8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
@@ -82,7 +82,7 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
     }
   }
 
-  public static enum BatchState {
+  public enum BatchState {
     /** Need to build schema and return. */
     BUILD_SCHEMA,
     /** This is still the first data batch. */
@@ -107,7 +107,7 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
     return context;
   }
 
-  public PhysicalOperator getPopConfig() {
+  public T getPopConfig() {
     return popConfig;
   }
 
@@ -118,10 +118,10 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
     return next(0, b);
   }
 
-  public final IterOutcome next(int inputIndex, RecordBatch b){
+  public final IterOutcome next(int inputIndex, RecordBatch b) {
     IterOutcome next;
     stats.stopProcessing();
-    try{
+    try {
       if (!context.getExecutorState().shouldContinue()) {
         return IterOutcome.STOP;
       }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
index c8e2bda..96b9cee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
@@ -21,13 +21,11 @@ import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 
-
 /**
- * Implements an AbstractUnaryRecordBatch where the inoming record batch is known at the time of creation
+ * Implements an AbstractUnaryRecordBatch where the incoming record batch is known at the time of creation
  * @param <T>
  */
 public abstract class AbstractSingleRecordBatch<T extends PhysicalOperator> extends AbstractUnaryRecordBatch<T> {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(new Object() {}.getClass().getEnclosingClass());
 
   protected final RecordBatch incoming;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractUnaryRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractUnaryRecordBatch.java
index 9627780..a7cb01f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractUnaryRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractUnaryRecordBatch.java
@@ -136,15 +136,6 @@ public abstract class AbstractUnaryRecordBatch<T extends PhysicalOperator> exten
     }
   }
 
-  @Override
-  public BatchSchema getSchema() {
-    if (container.hasSchema()) {
-      return container.getSchema();
-    }
-
-    return null;
-  }
-
   protected abstract boolean setupNewSchema() throws SchemaChangeException;
   protected abstract IterOutcome doWork();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index 0b36fac..8361dbe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -152,6 +152,7 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
       schema = builder.build();
       newVectors.buildSchema(BatchSchema.SelectionVectorMode.NONE);
       container = newVectors;
+      container.setRecordCount(valueCount);
     } catch (final Throwable cause) {
       // We have to clean up new vectors created here and pass over the actual cause. It is upper layer who should
       // adjudicate to call upper layer specific clean up logic.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
index 56a781e..7eceb56 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
@@ -17,9 +17,12 @@
  */
 package org.apache.drill.exec.record;
 
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.DataMode;
@@ -29,6 +32,7 @@ import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.MetadataUtils;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.metadata.TupleSchema;
@@ -187,4 +191,32 @@ public class SchemaUtil {
     }
     return tuple;
   }
+
+  /**
+   * Returns list of {@link SchemaPath} for fields taken from specified schema.
+   *
+   * @param schema the source of fields to return
+   * @return list of {@link SchemaPath}
+   */
+  public static List<SchemaPath> getSchemaPaths(TupleMetadata schema) {
+    return SchemaUtil.getColumnPaths(schema, null).stream()
+        .map(stringList -> SchemaPath.getCompoundPath(stringList.toArray(new String[0])))
+        .collect(Collectors.toList());
+  }
+
+  private static List<List<String>> getColumnPaths(TupleMetadata schema, List<String> parentNames) {
+    List<List<String>> result = new ArrayList<>();
+    for (ColumnMetadata columnMetadata : schema) {
+      if (columnMetadata.isMap()) {
+        List<String> currentNames = parentNames == null
+            ? new ArrayList<>()
+            : new ArrayList<>(parentNames);
+        currentNames.add(columnMetadata.name());
+        result.addAll(getColumnPaths(columnMetadata.mapSchema(), currentNames));
+      } else {
+        result.add(Collections.singletonList(columnMetadata.name()));
+      }
+    }
+    return result;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 897a7c1..28fa3e3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -260,6 +260,10 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       new OptionDefinition(ExecConstants.IMPLICIT_SUFFIX_COLUMN_LABEL_VALIDATOR),
       new OptionDefinition(ExecConstants.IMPLICIT_FQN_COLUMN_LABEL_VALIDATOR),
       new OptionDefinition(ExecConstants.IMPLICIT_FILEPATH_COLUMN_LABEL_VALIDATOR),
+      new OptionDefinition(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL_VALIDATOR),
+      new OptionDefinition(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL_VALIDATOR),
+      new OptionDefinition(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL_VALIDATOR),
+      new OptionDefinition(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL_VALIDATOR),
       new OptionDefinition(ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE_VALIDATOR),
       new OptionDefinition(ExecConstants.CREATE_PREPARE_STATEMENT_TIMEOUT_MILLIS_VALIDATOR),
       new OptionDefinition(ExecConstants.DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR,  new OptionMetaData(OptionValue.AccessibleScopes.SYSTEM, true, false)),
@@ -296,7 +300,7 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       new OptionDefinition(ExecConstants.METASTORE_USE_STATISTICS_METADATA_VALIDATOR),
       new OptionDefinition(ExecConstants.METASTORE_CTAS_AUTO_COLLECT_METADATA_VALIDATOR),
       new OptionDefinition(ExecConstants.METASTORE_FALLBACK_TO_FILE_METADATA_VALIDATOR),
-      new OptionDefinition(ExecConstants.METASTORE_RETRIVAL_RETRY_ATTEMPTS_VALIDATOR),
+      new OptionDefinition(ExecConstants.METASTORE_RETRIEVAL_RETRY_ATTEMPTS_VALIDATOR),
       new OptionDefinition(ExecConstants.PARQUET_READER_ENABLE_MAP_SUPPORT_VALIDATOR, new OptionMetaData(OptionValue.AccessibleScopes.SYSTEM_AND_SESSION, false, false))
     };
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
index 5b303f9..d232f5b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
@@ -82,6 +82,8 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
       drillTable.getMetadataProviderManager().setSchemaProvider(schemaProvider);
     });
 
+  public static final String SCHEMA_SEPARATOR = ".";
+
   protected final List<String> schemaPath;
   protected final String name;
 
@@ -113,7 +115,7 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
   }
 
   public String getFullSchemaName() {
-    return Joiner.on(".").join(schemaPath);
+    return Joiner.on(SCHEMA_SEPARATOR).join(schemaPath);
   }
 
   public abstract String getTypeName();
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 57341cd..2b2dd74 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
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.store;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -26,6 +27,7 @@ import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.exec.ExecConstants;
@@ -33,6 +35,7 @@ import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.util.Utilities;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
@@ -46,7 +49,9 @@ public class ColumnExplorer {
   private final List<Integer> selectedPartitionColumns;
   private final List<SchemaPath> tableColumns;
   private final Map<String, ImplicitFileColumns> allImplicitColumns;
+  private final Map<String, ImplicitInternalFileColumns> allInternalColumns;
   private final Map<String, ImplicitFileColumns> selectedImplicitColumns;
+  private final Map<String, ImplicitInternalFileColumns> selectedInternalColumns;
 
   /**
    * Helper class that encapsulates logic for sorting out columns
@@ -58,7 +63,9 @@ public class ColumnExplorer {
     this.selectedPartitionColumns = Lists.newArrayList();
     this.tableColumns = Lists.newArrayList();
     this.allImplicitColumns = initImplicitFileColumns(optionManager);
+    this.allInternalColumns = initImplicitInternalFileColumns(optionManager);
     this.selectedImplicitColumns = CaseInsensitiveMap.newHashMap();
+    this.selectedInternalColumns = CaseInsensitiveMap.newHashMap();
     if (columns == null) {
       isStarQuery = false;
       this.columns = null;
@@ -97,6 +104,22 @@ public class ColumnExplorer {
   }
 
   /**
+   * Creates case insensitive map with implicit internal file columns as keys and
+   * appropriate ImplicitFileColumns enum as values
+   */
+  public static Map<String, ImplicitInternalFileColumns> initImplicitInternalFileColumns(OptionManager optionManager) {
+    Map<String, ImplicitInternalFileColumns> map = CaseInsensitiveMap.newHashMap();
+    for (ImplicitInternalFileColumns e : ImplicitInternalFileColumns.values()) {
+      OptionValue optionValue;
+      if ((optionValue = optionManager.getOption(e.name)) != null) {
+        map.put(optionValue.string_val, e);
+      }
+    }
+
+    return map;
+  }
+
+  /**
    * Returns list with implicit column names taken from specified {@link SchemaConfig}.
    *
    * @param schemaConfig the source of session options values.
@@ -157,10 +180,40 @@ public class ColumnExplorer {
    * @return list with partition column names.
    */
   public static List<String> getPartitionColumnNames(FileSelection selection, SchemaConfig schemaConfig) {
-    int partitionsCount = getPartitionDepth(selection);
 
     String partitionColumnLabel = schemaConfig.getOption(
         ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
+
+    return getPartitionColumnNames(selection, partitionColumnLabel);
+  }
+
+  /**
+   * Returns list with partition column names.
+   * For the case when table has several levels of nesting, max level is chosen.
+   *
+   * @param selection     the source of file paths
+   * @param optionManager the source of session option value for partition column label
+   * @return list with partition column names.
+   */
+  public static List<String> getPartitionColumnNames(FileSelection selection, OptionManager optionManager) {
+
+    String partitionColumnLabel = optionManager.getString(
+        ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL);
+
+    return getPartitionColumnNames(selection, partitionColumnLabel);
+  }
+
+  /**
+   * Returns list with partition column names.
+   * For the case when table has several levels of nesting, max level is chosen.
+   *
+   * @param selection            the source of file paths
+   * @param partitionColumnLabel partition column label
+   * @return list with partition column names.
+   */
+  private static List<String> getPartitionColumnNames(FileSelection selection, String partitionColumnLabel) {
+    int partitionsCount = getPartitionDepth(selection);
+
     List<String> partitions = new ArrayList<>();
 
     // generates partition column names: dir0, dir1 etc.
@@ -219,6 +272,48 @@ public class ColumnExplorer {
   }
 
   /**
+   * Creates map with implicit and internal columns where key is column name, value is columns actual value.
+   * This map contains partition, implicit and internal file columns (if requested).
+   * Partition columns names are formed based in partition designator and value index.
+   *
+   * @param filePath                   file path, used to populate file implicit columns
+   * @param partitionValues            list of partition values
+   * @param includeFileImplicitColumns if file implicit columns should be included into the result
+   * @param fs                         file system
+   * @param index                      index of row group to populate
+   * @return implicit columns map
+   */
+  public Map<String, String> populateImplicitAndInternalColumns(Path filePath,
+      List<String> partitionValues, boolean includeFileImplicitColumns, FileSystem fs, int index, long start, long length) {
+
+    Map<String, String> implicitValues =
+        new LinkedHashMap<>(populateImplicitColumns(filePath, partitionValues, includeFileImplicitColumns));
+
+    selectedInternalColumns.forEach((key, value) -> {
+      switch (value) {
+        case ROW_GROUP_INDEX:
+          implicitValues.put(key, String.valueOf(index));
+          break;
+        case ROW_GROUP_START:
+          implicitValues.put(key, String.valueOf(start));
+          break;
+        case ROW_GROUP_LENGTH:
+          implicitValues.put(key, String.valueOf(length));
+          break;
+        case LAST_MODIFIED_TIME:
+          try {
+            implicitValues.put(key, String.valueOf(fs.getFileStatus(filePath).getModificationTime()));
+          } catch (IOException e) {
+            throw new DrillRuntimeException(e);
+          }
+          break;
+      }
+    });
+
+    return implicitValues;
+  }
+
+  /**
    * Compares root and file path to determine directories
    * that are present in the file path but absent in root.
    * Example: root - a/b/c, file - a/b/c/d/e/0_0_0.parquet, result - d/e.
@@ -314,12 +409,16 @@ public class ColumnExplorer {
       if (isStarQuery) {
         if (allImplicitColumns.get(path) != null) {
           selectedImplicitColumns.put(path, allImplicitColumns.get(path));
+        } else if (allInternalColumns.get(path) != null) {
+          selectedInternalColumns.put(path, allInternalColumns.get(path));
         }
       } else {
         if (isPartitionColumn(partitionDesignator, path)) {
           selectedPartitionColumns.add(Integer.parseInt(path.substring(partitionDesignator.length())));
         } else if (allImplicitColumns.get(path) != null) {
           selectedImplicitColumns.put(path, allImplicitColumns.get(path));
+        } else if (allInternalColumns.get(path) != null) {
+          selectedInternalColumns.put(path, allInternalColumns.get(path));
         } else {
           tableColumns.add(column);
         }
@@ -386,4 +485,25 @@ public class ColumnExplorer {
      */
     public abstract String getValue(Path path);
   }
+
+  /**
+   * Columns that give internal information about file or its parts.
+   * Columns are implicit, so should be called explicitly in query.
+   */
+  public enum ImplicitInternalFileColumns {
+
+    LAST_MODIFIED_TIME(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL),
+
+    ROW_GROUP_INDEX(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL),
+
+    ROW_GROUP_START(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL),
+
+    ROW_GROUP_LENGTH(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL);
+
+    private final String name;
+
+    ImplicitInternalFileColumns(String name) {
+      this.name = name;
+    }
+  }
 }
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 098ebdb..87b5592 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
@@ -53,6 +53,8 @@ import org.apache.drill.exec.dotdrill.DotDrillUtil;
 import org.apache.drill.exec.dotdrill.View;
 import org.apache.drill.exec.metastore.FileSystemMetadataProviderManager;
 import org.apache.drill.exec.metastore.MetadataProviderManager;
+import org.apache.drill.exec.metastore.MetastoreMetadataProviderManager;
+import org.apache.drill.exec.metastore.MetastoreMetadataProviderManager.MetastoreMetadataProviderConfig;
 import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.planner.logical.DrillTable;
@@ -71,6 +73,10 @@ import org.apache.drill.exec.util.DrillFileSystemUtil;
 import org.apache.drill.exec.store.StorageStrategy;
 import org.apache.drill.exec.store.easy.json.JSONFormatPlugin;
 import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.drill.metastore.MetastoreRegistry;
+import org.apache.drill.metastore.components.tables.MetastoreTableInfo;
+import org.apache.drill.metastore.exceptions.MetastoreException;
+import org.apache.drill.metastore.metadata.TableInfo;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
@@ -438,8 +444,34 @@ public class WorkspaceSchemaFactory {
       }
       final DrillTable table = tables.get(tableKey);
       if (table != null) {
-        MetadataProviderManager providerManager = FileSystemMetadataProviderManager.init();
-
+        MetadataProviderManager providerManager = null;
+
+        if (schemaConfig.getOption(ExecConstants.METASTORE_ENABLED).bool_val) {
+          try {
+            MetastoreRegistry metastoreRegistry = plugin.getContext().getMetastoreRegistry();
+            TableInfo tableInfo = TableInfo.builder()
+                .storagePlugin(plugin.getName())
+                .workspace(schemaName)
+                .name(tableName)
+                .build();
+
+            MetastoreTableInfo metastoreTableInfo = metastoreRegistry.get()
+                .tables()
+                .basicRequests()
+                .metastoreTableInfo(tableInfo);
+            if (metastoreTableInfo.isExists()) {
+              providerManager = new MetastoreMetadataProviderManager(metastoreRegistry, tableInfo,
+                  new MetastoreMetadataProviderConfig(schemaConfig.getOption(ExecConstants.METASTORE_USE_SCHEMA_METADATA).bool_val,
+                      schemaConfig.getOption(ExecConstants.METASTORE_USE_STATISTICS_METADATA).bool_val,
+                      schemaConfig.getOption(ExecConstants.METASTORE_FALLBACK_TO_FILE_METADATA).bool_val));
+            }
+          } catch (MetastoreException e) {
+            logger.warn("Exception happened during obtaining Metastore instance.", e);
+          }
+        }
+        if (providerManager == null) {
+          providerManager = FileSystemMetadataProviderManager.init();
+        }
         setMetadataTable(providerManager, table, tableName);
         setSchema(providerManager, tableName);
         table.setTableMetadataProviderManager(providerManager);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java
index da63f80..4c08d54 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/MetadataDirectGroupScan.java
@@ -43,18 +43,22 @@ public class MetadataDirectGroupScan extends DirectGroupScan {
   @JsonProperty
   private final int numFiles;
   @JsonProperty
-  private boolean usedMetadataSummaryFile;
+  private final boolean usedMetadataSummaryFile;
+  @JsonProperty
+  private final boolean usedMetastore;
 
   @JsonCreator
   public MetadataDirectGroupScan(@JsonProperty("reader") RecordReader reader,
-                                 @JsonProperty("selectionRoot") Path selectionRoot,
-                                 @JsonProperty("numFiles") int numFiles,
-                                 @JsonProperty("stats") ScanStats stats,
-                                 @JsonProperty("usedMetadataSummaryFile") boolean usedMetadataSummaryFile) {
+      @JsonProperty("selectionRoot") Path selectionRoot,
+      @JsonProperty("numFiles") int numFiles,
+      @JsonProperty("stats") ScanStats stats,
+      @JsonProperty("usedMetadataSummaryFile") boolean usedMetadataSummaryFile,
+      @JsonProperty("usedMetastore") boolean usedMetastore) {
     super(reader, stats);
     this.selectionRoot = selectionRoot;
     this.numFiles = numFiles;
     this.usedMetadataSummaryFile = usedMetadataSummaryFile;
+    this.usedMetastore = usedMetastore;
   }
 
   @Override
@@ -65,7 +69,7 @@ public class MetadataDirectGroupScan extends DirectGroupScan {
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
     assert children == null || children.isEmpty();
-    return new MetadataDirectGroupScan(reader, selectionRoot, numFiles, stats, usedMetadataSummaryFile);
+    return new MetadataDirectGroupScan(reader, selectionRoot, numFiles, stats, usedMetadataSummaryFile, usedMetastore);
   }
 
   @Override
@@ -81,7 +85,7 @@ public class MetadataDirectGroupScan extends DirectGroupScan {
    * </p>
    *
    * <p>
-   * Example: [selectionRoot = [/tmp/users], numFiles = 1, usedMetadataSummaryFile = false]
+   * Example: [selectionRoot = [/tmp/users], numFiles = 1, usedMetadataSummaryFile = false, usedMetastore = true]
    * </p>
    *
    * @return string representation of group scan data
@@ -94,6 +98,7 @@ public class MetadataDirectGroupScan extends DirectGroupScan {
     }
     builder.append("numFiles = ").append(numFiles).append(", ");
     builder.append("usedMetadataSummaryFile = ").append(usedMetadataSummaryFile).append(", ");
+    builder.append("usedMetastore = ").append(usedMetastore).append(", ");
     builder.append(super.getDigest());
     return builder.toString();
   }
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 3be8996..51cdbe2 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
@@ -17,20 +17,13 @@
  */
 package org.apache.drill.exec.store.easy.json;
 
-import com.fasterxml.jackson.core.JsonFactory;
 import java.io.IOException;
-import java.time.LocalDate;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Map;
 
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.metastore.statistics.Statistic;
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.store.StatisticsRecordWriter;
 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;
-import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
 import org.apache.drill.exec.store.JSONBaseStatisticsRecordWriter;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
 import org.apache.drill.exec.util.ImpersonationUtil;
@@ -39,32 +32,23 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-public class JsonStatisticsRecordWriter extends JSONBaseStatisticsRecordWriter {
+public class JsonStatisticsRecordWriter extends JSONBaseStatisticsRecordWriter implements StatisticsRecordWriter {
 
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonStatisticsRecordWriter.class);
-  private static final String LINE_FEED = String.format("%n");
+
   private String location;
   private String prefix;
-  private String fieldDelimiter;
   private String extension;
-  private boolean useExtendedOutput;
   private FileSystem fs = null;
-  private STATS_VERSION statisticsVersion;
-  private final JsonFactory factory = new JsonFactory();
-  private String lastDirectory = null;
-  private Configuration fsConf = null;
-  private FormatPlugin formatPlugin = null;
-  private String nextField = null;
-  private DrillStatsTable.TableStatistics statistics;
-  private List<DrillStatsTable.ColumnStatistics> columnStatisticsList = new ArrayList<DrillStatsTable.ColumnStatistics>();
-  private DrillStatsTable.ColumnStatistics columnStatistics;
-  private LocalDate dirComputedTime = null;
+  private Configuration fsConf;
+  private FormatPlugin formatPlugin;
   private Path fileName = null;
-  private String queryId = null;
+
   private long recordsWritten = -1;
-  private boolean errStatus = false;
 
-  public JsonStatisticsRecordWriter(Configuration fsConf, FormatPlugin formatPlugin){
+  private StatisticsCollectorImpl statisticsCollector = new StatisticsCollectorImpl();
+
+  public JsonStatisticsRecordWriter(Configuration fsConf, FormatPlugin formatPlugin) {
     this.fsConf = fsConf;
     this.formatPlugin = formatPlugin;
   }
@@ -73,12 +57,9 @@ public class JsonStatisticsRecordWriter extends JSONBaseStatisticsRecordWriter {
   public void init(Map<String, String> writerOptions) throws IOException {
     this.location = writerOptions.get("location");
     this.prefix = writerOptions.get("prefix");
-    this.fieldDelimiter = writerOptions.get("separator");
     this.extension = writerOptions.get("extension");
-    this.useExtendedOutput = Boolean.parseBoolean(writerOptions.get("extended"));
     this.skipNullFields = Boolean.parseBoolean(writerOptions.get("skipnulls"));
-    this.statisticsVersion = DrillStatsTable.CURRENT_VERSION;
-    this.queryId = writerOptions.get("queryid");
+    String queryId = writerOptions.get("queryid");
      //Write as DRILL process user
     this.fs = ImpersonationUtil.createFileSystem(ImpersonationUtil.getProcessUserName(), fsConf);
 
@@ -106,7 +87,7 @@ public class JsonStatisticsRecordWriter extends JSONBaseStatisticsRecordWriter {
   }
 
   @Override
-  public void updateSchema(VectorAccessible batch) throws IOException {
+  public void updateSchema(VectorAccessible batch) {
     // no op
   }
 
@@ -122,329 +103,73 @@ public class JsonStatisticsRecordWriter extends JSONBaseStatisticsRecordWriter {
 
   @Override
   public FieldConverter getNewBigIntConverter(int fieldId, String fieldName, FieldReader reader) {
-    return new BigIntJsonConverter(fieldId, fieldName, reader);
-  }
-
-  public class BigIntJsonConverter extends FieldConverter {
-
-    public BigIntJsonConverter(int fieldId, String fieldName, FieldReader reader) {
-      super(fieldId, fieldName, reader);
-    }
-
-    @Override
-    public void startField() throws IOException {
-      if (fieldName.equals(Statistic.SCHEMA)) {
-        nextField = fieldName;
-      } else if (fieldName.equals(Statistic.ROWCOUNT)
-            || fieldName.equals(Statistic.NNROWCOUNT)
-            || fieldName.equals(Statistic.NDV)
-            || fieldName.equals(Statistic.AVG_WIDTH)
-            || fieldName.equals(Statistic.SUM_DUPS)) {
-        nextField = fieldName;
-      }
-    }
-
-    @Override
-    public void writeField() throws IOException {
-      if (nextField == null) {
-        errStatus = true;
-        throw new IOException("Statistics writer encountered unexpected field");
-      }
-      if (nextField.equals(Statistic.SCHEMA)) {
-        ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setSchema(reader.readLong());
-      } else if (nextField.equals(Statistic.ROWCOUNT)) {
-        ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setCount(reader.readLong());
-      } else if (nextField.equals(Statistic.NNROWCOUNT)) {
-        ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNonNullCount(reader.readLong());
-      } else if (nextField.equals(Statistic.NDV)) {
-        ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNdv(reader.readLong());
-      } else if (nextField.equals(Statistic.AVG_WIDTH)) {
-        ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setAvgWidth(reader.readLong());
-      } else if (nextField.equals(Statistic.SUM_DUPS)) {
-        // Ignore Count_Approx_Dups statistic
-      }
-    }
-
-    @Override
-    public void endField() throws IOException {
-      nextField = null;
-    }
+    return statisticsCollector.getNewBigIntConverter(fieldId, fieldName, reader);
   }
 
   @Override
   public FieldConverter getNewIntConverter(int fieldId, String fieldName, FieldReader reader) {
-    return new IntJsonConverter(fieldId, fieldName, reader);
-  }
-
-  public class IntJsonConverter extends FieldConverter {
-
-    public IntJsonConverter(int fieldId, String fieldName, FieldReader reader) {
-      super(fieldId, fieldName, reader);
-    }
-
-    @Override
-    public void startField() throws IOException {
-      if (fieldName.equals(Statistic.COLTYPE)) {
-        nextField = fieldName;
-      }
-    }
-
-    @Override
-    public void writeField() throws IOException {
-      if (nextField == null) {
-        errStatus = true;
-        throw new IOException("Statistics writer encountered unexpected field");
-      }
-      if (nextField.equals(Statistic.COLTYPE)) {
-        // Do not write out the type
-      }
-    }
-
-    @Override
-    public void endField() throws IOException {
-      nextField = null;
-    }
+    return statisticsCollector.getNewIntConverter(fieldId, fieldName, reader);
   }
 
   @Override
   public FieldConverter getNewDateConverter(int fieldId, String fieldName, FieldReader reader) {
-    return new DateJsonConverter(fieldId, fieldName, reader);
-  }
-
-  public class DateJsonConverter extends FieldConverter {
-
-    public DateJsonConverter(int fieldId, String fieldName, FieldReader reader) {
-      super(fieldId, fieldName, reader);
-    }
-
-    @Override
-    public void startField() throws IOException {
-      if (fieldName.equals(Statistic.COMPUTED)) {
-        nextField = fieldName;
-      }
-    }
-
-    @Override
-    public void writeField() throws IOException {
-      if (nextField == null) {
-        errStatus = true;
-        throw new IOException("Statistics writer encountered unexpected field");
-      }
-      if (nextField.equals((Statistic.COMPUTED))) {
-        LocalDate computedTime = reader.readLocalDate();
-        if (dirComputedTime == null
-               || computedTime.compareTo(dirComputedTime) > 0) {
-          dirComputedTime = computedTime;
-        }
-      }
-    }
-
-    @Override
-    public void endField() throws IOException {
-      nextField = null;
-    }
+    return statisticsCollector.getNewDateConverter(fieldId, fieldName, reader);
   }
 
   @Override
   public FieldConverter getNewVarCharConverter(int fieldId, String fieldName, FieldReader reader) {
-    return new VarCharJsonConverter(fieldId, fieldName, reader);
-  }
-
-  public class VarCharJsonConverter extends FieldConverter {
-
-    public VarCharJsonConverter(int fieldId, String fieldName, FieldReader reader) {
-      super(fieldId, fieldName, reader);
-    }
-
-    @Override
-    public void startField() throws IOException {
-      if (fieldName.equals(Statistic.COLNAME)) {
-        nextField = fieldName;
-      } else if (fieldName.equals(Statistic.COLTYPE)) {
-        nextField = fieldName;
-      }
-    }
-
-    @Override
-    public void writeField() throws IOException {
-      if (nextField == null) {
-        errStatus = true;
-        throw new IOException("Statistics writer encountered unexpected field");
-      }
-      if (nextField.equals(Statistic.COLNAME)) {
-        ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setName(SchemaPath.parseFromString(reader.readText().toString()));
-      } else if (nextField.equals(Statistic.COLTYPE)) {
-        MajorType fieldType = DrillStatsTable.getMapper().readValue(reader.readText().toString(), MajorType.class);
-        ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setType(fieldType);
-      }
-    }
-
-    @Override
-    public void endField() {
-      nextField = null;
-    }
+    return statisticsCollector.getNewVarCharConverter(fieldId, fieldName, reader);
   }
 
   @Override
   public FieldConverter getNewNullableBigIntConverter(int fieldId, String fieldName, FieldReader reader) {
-    return new NullableBigIntJsonConverter(fieldId, fieldName, reader);
-  }
-
-  public class NullableBigIntJsonConverter extends FieldConverter {
-
-    public NullableBigIntJsonConverter(int fieldId, String fieldName, FieldReader reader) {
-      super(fieldId, fieldName, reader);
-    }
-
-    @Override
-    public void startField() throws IOException {
-      if (!skipNullFields || this.reader.isSet()) {
-        if (fieldName.equals(Statistic.ROWCOUNT)
-            || fieldName.equals(Statistic.NNROWCOUNT)
-            || fieldName.equals(Statistic.NDV)
-            || fieldName.equals(Statistic.SUM_DUPS)) {
-          nextField = fieldName;
-        }
-      }
-    }
-
-    @Override
-    public void writeField() throws IOException {
-      if (!skipNullFields || this.reader.isSet()) {
-        if (nextField == null) {
-          errStatus = true;
-          throw new IOException("Statistics writer encountered unexpected field");
-        }
-        if (nextField.equals(Statistic.ROWCOUNT)) {
-          ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setCount(reader.readLong());
-        } else if (nextField.equals(Statistic.NNROWCOUNT)) {
-          ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNonNullCount(reader.readLong());
-        } else if (nextField.equals(Statistic.NDV)) {
-          ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNdv(reader.readLong());
-        } else if (nextField.equals(Statistic.SUM_DUPS)) {
-          // Ignore Count_Approx_Dups statistic
-        }
-      }
-    }
-
-    @Override
-    public void endField() throws IOException {
-      nextField = null;
-    }
+    return statisticsCollector.getNewNullableBigIntConverter(fieldId, fieldName, reader);
   }
 
   @Override
   public FieldConverter getNewNullableVarBinaryConverter(int fieldId, String fieldName, FieldReader reader) {
-    return new NullableVarBinaryJsonConverter(fieldId, fieldName, reader);
-  }
-
-  public class NullableVarBinaryJsonConverter extends FieldConverter {
-
-    public NullableVarBinaryJsonConverter(int fieldId, String fieldName, FieldReader reader) {
-      super(fieldId, fieldName, reader);
-    }
-
-    @Override
-    public void startField() throws IOException {
-      if (!skipNullFields || this.reader.isSet()) {
-        if (fieldName.equals(Statistic.HLL)
-            || fieldName.equals(Statistic.HLL_MERGE)
-            || fieldName.equals(Statistic.TDIGEST_MERGE)) {
-          nextField = fieldName;
-        }
-      }
-    }
-
-    @Override
-    public void writeField() throws IOException {
-      if (!skipNullFields || this.reader.isSet()) {
-        if (nextField == null) {
-          errStatus = true;
-          throw new IOException("Statistics writer encountered unexpected field");
-        }
-        if (nextField.equals(Statistic.HLL)
-            || nextField.equals(Statistic.HLL_MERGE)) {
-          // Do NOT write out the HLL output, since it is not used yet for computing statistics for a
-          // subset of partitions in the query OR for computing NDV with incremental statistics.
-        }  else if (nextField.equals(Statistic.TDIGEST_MERGE)) {
-          byte[] tdigest_bytearray = reader.readByteArray();
-          ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).buildHistogram(tdigest_bytearray);
-        }
-      }
-    }
-
-    @Override
-    public void endField() throws IOException {
-      nextField = null;
-    }
+    return statisticsCollector.getNewNullableVarBinaryConverter(fieldId, fieldName, reader);
   }
 
   @Override
   public FieldConverter getNewNullableFloat8Converter(int fieldId, String fieldName, FieldReader reader) {
-    return new NullableFloat8JsonConverter(fieldId, fieldName, reader);
+    return statisticsCollector.getNewNullableFloat8Converter(fieldId, fieldName, reader);
   }
 
-  public class NullableFloat8JsonConverter extends FieldConverter {
-
-    public NullableFloat8JsonConverter(int fieldId, String fieldName, FieldReader reader) {
-      super(fieldId, fieldName, reader);
-    }
-
-    @Override
-    public void startField() throws IOException {
-      if (!skipNullFields || this.reader.isSet()) {
-        if (fieldName.equals(Statistic.AVG_WIDTH)) {
-          nextField = fieldName;
-        }
-      }
-    }
-
-    @Override
-    public void writeField() throws IOException {
-      if (!skipNullFields || this.reader.isSet()) {
-        if (nextField == null) {
-          errStatus = true;
-          throw new IOException("Statistics writer encountered unexpected field");
-        }
-        if (nextField.equals(Statistic.AVG_WIDTH)) {
-          ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setAvgWidth(reader.readDouble());
-        }
-      }
-    }
+  @Override
+  public void startStatisticsRecord() {
+    statisticsCollector.startStatisticsRecord();
+  }
 
-    @Override
-    public void endField() throws IOException {
-      nextField = null;
-    }
+  @Override
+  public void endStatisticsRecord() {
+    statisticsCollector.endStatisticsRecord();
+    ++recordsWritten;
   }
 
   @Override
-  public void startStatisticsRecord() throws IOException {
-    columnStatistics = new DrillStatsTable.ColumnStatistics_v1();
+  public boolean hasStatistics() {
+    return recordsWritten > 0;
   }
 
   @Override
-  public void endStatisticsRecord() throws IOException {
-    columnStatisticsList.add(columnStatistics);
-    ++recordsWritten;
+  public DrillStatsTable.TableStatistics getStatistics() {
+    return statisticsCollector.getStatistics();
   }
 
   @Override
   public void flushBlockingWriter() throws IOException {
     Path permFileName = new Path(location, prefix + "." + extension);
     try {
-      if (errStatus) {
+      if (statisticsCollector.hasErrors()) {
         // Encountered some error
         throw new IOException("Statistics writer encountered unexpected field");
       } else if (recordsWritten < 0) {
         throw new IOException("Statistics writer did not have data");
       }
-      // Generated the statistics data structure to be serialized
-      statistics = DrillStatsTable.generateDirectoryStructure(dirComputedTime.toString(),
-          columnStatisticsList);
       if (formatPlugin.supportsStatistics()) {
         // Invoke the format plugin stats API to write out the stats
-        formatPlugin.writeStatistics(statistics, fs, fileName);
+        formatPlugin.writeStatistics(statisticsCollector.getStatistics(), fs, fileName);
         // Delete existing permanent file and rename .tmp file to permanent file
         // If failed to do so then delete the .tmp file
         fs.delete(permFileName, false);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/StatisticsCollectorImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/StatisticsCollectorImpl.java
new file mode 100644
index 0000000..f2dcac9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/StatisticsCollectorImpl.java
@@ -0,0 +1,387 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.easy.json;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.store.JSONBaseStatisticsRecordWriter;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.metastore.statistics.Statistic;
+
+import java.io.IOException;
+import java.time.LocalDate;
+import java.util.ArrayList;
+import java.util.List;
+
+public class StatisticsCollectorImpl extends JSONBaseStatisticsRecordWriter {
+
+  private final List<DrillStatsTable.ColumnStatistics> columnStatisticsList = new ArrayList<>();
+
+  private String nextField = null;
+  private DrillStatsTable.ColumnStatistics columnStatistics;
+  private LocalDate dirComputedTime = null;
+  private boolean errStatus = false;
+
+  @Override
+  public void startStatisticsRecord() {
+    columnStatistics = new DrillStatsTable.ColumnStatistics_v1();
+  }
+
+  @Override
+  public void endStatisticsRecord() {
+    columnStatisticsList.add(columnStatistics);
+  }
+
+  @Override
+  public boolean hasStatistics() {
+    return !columnStatisticsList.isEmpty();
+  }
+
+  public DrillStatsTable.TableStatistics getStatistics() {
+    return DrillStatsTable.generateDirectoryStructure(dirComputedTime.toString(),
+        columnStatisticsList);
+  }
+
+  public boolean hasErrors() {
+    return errStatus;
+  }
+
+  @Override
+  public FieldConverter getNewBigIntConverter(int fieldId, String fieldName, FieldReader reader) {
+    return new BigIntJsonConverter(fieldId, fieldName, reader);
+  }
+
+  @Override
+  public FieldConverter getNewIntConverter(int fieldId, String fieldName, FieldReader reader) {
+    return new IntJsonConverter(fieldId, fieldName, reader);
+  }
+
+  @Override
+  public FieldConverter getNewDateConverter(int fieldId, String fieldName, FieldReader reader) {
+    return new DateJsonConverter(fieldId, fieldName, reader);
+  }
+
+  @Override
+  public FieldConverter getNewVarCharConverter(int fieldId, String fieldName, FieldReader reader) {
+    return new VarCharJsonConverter(fieldId, fieldName, reader);
+  }
+
+  @Override
+  public FieldConverter getNewNullableBigIntConverter(int fieldId, String fieldName, FieldReader reader) {
+    return new NullableBigIntJsonConverter(fieldId, fieldName, reader);
+  }
+
+  @Override
+  public FieldConverter getNewNullableVarBinaryConverter(int fieldId, String fieldName, FieldReader reader) {
+    return new NullableVarBinaryJsonConverter(fieldId, fieldName, reader);
+  }
+
+  @Override
+  public FieldConverter getNewNullableFloat8Converter(int fieldId, String fieldName, FieldReader reader) {
+    return new NullableFloat8JsonConverter(fieldId, fieldName, reader);
+  }
+
+  public class BigIntJsonConverter extends FieldConverter {
+
+    public BigIntJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+      super(fieldId, fieldName, reader);
+    }
+
+    @Override
+    public void startField() throws IOException {
+      switch (fieldName) {
+        case Statistic.SCHEMA:
+        case Statistic.ROWCOUNT:
+        case Statistic.NNROWCOUNT:
+        case Statistic.NDV:
+        case Statistic.AVG_WIDTH:
+        case Statistic.SUM_DUPS:
+          nextField = fieldName;
+      }
+    }
+
+    @Override
+    public void writeField() throws IOException {
+      if (nextField == null) {
+        errStatus = true;
+        throw new IOException("Statistics writer encountered unexpected field");
+      }
+      DrillStatsTable.ColumnStatistics_v1 columnStatistics =
+          (DrillStatsTable.ColumnStatistics_v1) StatisticsCollectorImpl.this.columnStatistics;
+      switch (nextField) {
+        case Statistic.SCHEMA:
+          columnStatistics.setSchema(reader.readLong());
+          break;
+        case Statistic.ROWCOUNT:
+          columnStatistics.setCount(reader.readLong());
+          break;
+        case Statistic.NNROWCOUNT:
+          columnStatistics.setNonNullCount(reader.readLong());
+          break;
+        case Statistic.NDV:
+          columnStatistics.setNdv(reader.readLong());
+          break;
+        case Statistic.AVG_WIDTH:
+          columnStatistics.setAvgWidth(reader.readLong());
+          break;
+        case Statistic.SUM_DUPS:
+          // Ignore Count_Approx_Dups statistic
+          break;
+      }
+    }
+
+    @Override
+    public void endField() {
+      nextField = null;
+    }
+  }
+
+  public class IntJsonConverter extends FieldConverter {
+
+    public IntJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+      super(fieldId, fieldName, reader);
+    }
+
+    @Override
+    public void startField() throws IOException {
+      if (fieldName.equals(Statistic.COLTYPE)) {
+        nextField = fieldName;
+      }
+    }
+
+    @Override
+    public void writeField() throws IOException {
+      if (nextField == null) {
+        errStatus = true;
+        throw new IOException("Statistics writer encountered unexpected field");
+      }
+      if (nextField.equals(Statistic.COLTYPE)) {
+        // Do not write out the type
+      }
+    }
+
+    @Override
+    public void endField() {
+      nextField = null;
+    }
+  }
+
+  public class DateJsonConverter extends FieldConverter {
+
+    public DateJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+      super(fieldId, fieldName, reader);
+    }
+
+    @Override
+    public void startField() throws IOException {
+      if (fieldName.equals(Statistic.COMPUTED)) {
+        nextField = fieldName;
+      }
+    }
+
+    @Override
+    public void writeField() throws IOException {
+      if (nextField == null) {
+        errStatus = true;
+        throw new IOException("Statistics writer encountered unexpected field");
+      }
+      if (nextField.equals((Statistic.COMPUTED))) {
+        LocalDate computedTime = reader.readLocalDate();
+        if (dirComputedTime == null
+            || computedTime.compareTo(dirComputedTime) > 0) {
+          dirComputedTime = computedTime;
+        }
+      }
+    }
+
+    @Override
+    public void endField() {
+      nextField = null;
+    }
+  }
+
+  public class VarCharJsonConverter extends FieldConverter {
+
+    public VarCharJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+      super(fieldId, fieldName, reader);
+    }
+
+    @Override
+    public void startField() throws IOException {
+      switch (fieldName) {
+        case Statistic.COLNAME:
+        case Statistic.COLTYPE:
+          nextField = fieldName;
+      }
+    }
+
+    @Override
+    public void writeField() throws IOException {
+      if (nextField == null) {
+        errStatus = true;
+        throw new IOException("Statistics writer encountered unexpected field");
+      }
+      switch (nextField) {
+        case Statistic.COLNAME:
+          ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setName(SchemaPath.parseFromString(reader.readText().toString()));
+          break;
+        case Statistic.COLTYPE:
+          TypeProtos.MajorType fieldType = DrillStatsTable.getMapper().readValue(reader.readText().toString(), TypeProtos.MajorType.class);
+          ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setType(fieldType);
+          break;
+      }
+    }
+
+    @Override
+    public void endField() {
+      nextField = null;
+    }
+  }
+
+  public class NullableBigIntJsonConverter extends FieldConverter {
+
+    public NullableBigIntJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+      super(fieldId, fieldName, reader);
+    }
+
+    @Override
+    public void startField() throws IOException {
+      if (!skipNullFields || this.reader.isSet()) {
+        switch (fieldName) {
+          case Statistic.ROWCOUNT:
+          case Statistic.NNROWCOUNT:
+          case Statistic.NDV:
+          case Statistic.SUM_DUPS:
+            nextField = fieldName;
+            break;
+        }
+      }
+    }
+
+    @Override
+    public void writeField() throws IOException {
+      if (!skipNullFields || this.reader.isSet()) {
+        if (nextField == null) {
+          errStatus = true;
+          throw new IOException("Statistics writer encountered unexpected field");
+        }
+        switch (nextField) {
+          case Statistic.ROWCOUNT:
+            ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setCount(reader.readLong());
+            break;
+          case Statistic.NNROWCOUNT:
+            ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNonNullCount(reader.readLong());
+            break;
+          case Statistic.NDV:
+            ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setNdv(reader.readLong());
+            break;
+          case Statistic.SUM_DUPS:
+            // Ignore Count_Approx_Dups statistic
+            break;
+        }
+      }
+    }
+
+    @Override
+    public void endField() {
+      nextField = null;
+    }
+  }
+
+  public class NullableVarBinaryJsonConverter extends FieldConverter {
+
+    public NullableVarBinaryJsonConverter(int fieldId, String fieldName, FieldReader reader) {
+      super(fieldId, fieldName, reader);
+    }
+
+    @Override
+    public void startField() throws IOException {
+      if (!skipNullFields || this.reader.isSet()) {
+        switch (fieldName) {
+          case Statistic.HLL:
+          case Statistic.HLL_MERGE:
+          case Statistic.TDIGEST_MERGE:
+            nextField = fieldName;
+            break;
+        }
+      }
+    }
+
+    @Override
+    public void writeField() throws IOException {
+      if (!skipNullFields || this.reader.isSet()) {
+        if (nextField == null) {
+          errStatus = true;
+          throw new IOException("Statistics writer encountered unexpected field");
+        }
+        switch (nextField) {
+          case Statistic.HLL:
+          case Statistic.HLL_MERGE:
+            // Do NOT write out the HLL output, since it is not used yet for computing statistics for a
+            // subset of partitions in the query OR for computing NDV with incremental statistics.
+            break;
+          case Statistic.TDIGEST_MERGE:
+            byte[] tdigest_bytearray = reader.readByteArray();
+            ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).buildHistogram(tdigest_bytearray);
+            break;
+        }
+      }
+    }
+
+    @Override
+    public void endField() {
+      nextField = null;
+    }
+  }
+
+  public class NullableFloat8JsonConverter extends FieldConverter {
+
+    public NullableFloat8JsonConverter(int fieldId, String fieldName, FieldReader reader) {
+      super(fieldId, fieldName, reader);
+    }
+
+    @Override
+    public void startField() throws IOException {
+      if (!skipNullFields || this.reader.isSet()) {
+        if (fieldName.equals(Statistic.AVG_WIDTH)) {
+          nextField = fieldName;
+        }
+      }
+    }
+
+    @Override
+    public void writeField() throws IOException {
+      if (!skipNullFields || this.reader.isSet()) {
+        if (nextField == null) {
+          errStatus = true;
+          throw new IOException("Statistics writer encountered unexpected field");
+        }
+        if (nextField.equals(Statistic.AVG_WIDTH)) {
+          ((DrillStatsTable.ColumnStatistics_v1) columnStatistics).setAvgWidth(reader.readDouble());
+        }
+      }
+    }
+
+    @Override
+    public void endField() {
+      nextField = null;
+    }
+  }
+}
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 2e802b2..5539ba4 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
@@ -234,10 +234,12 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
       return null;
     }
 
-    Set<SchemaPath> schemaPathsInExpr =
-        filterExpr.accept(new FilterEvaluatorUtils.FieldReferenceFinder(), null);
-
-    RowGroupScanFilterer filteredMetadata = getFilterer().getFiltered(optionManager, filterPredicate, schemaPathsInExpr);
+    RowGroupScanFilterer filteredMetadata = getFilterer()
+        .filterExpression(filterExpr)
+        .schema(tableMetadata.getSchema())
+        .context(functionImplementationRegistry)
+        .udfUtilities(udfUtilities)
+        .getFiltered(optionManager, filterPredicate);
 
     // checks whether metadata for specific level was available and there was no reduction of metadata
     if (isGroupScanFullyMatchesFilter(filteredMetadata)) {
@@ -316,7 +318,7 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
   }
 
   // narrows the return type
-  protected abstract RowGroupScanFilterer getFilterer();
+  protected abstract RowGroupScanFilterer<? extends RowGroupScanFilterer> getFilterer();
 
   protected Multimap<Path, RowGroupMetadata> pruneRowGroupsForFiles(Map<Path, FileMetadata> filteredFileMetadata) {
     Multimap<Path, RowGroupMetadata> prunedRowGroups = LinkedListMultimap.create();
@@ -538,11 +540,11 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
     }
 
     @Override
-    protected B getFiltered(OptionManager optionManager, FilterPredicate filterPredicate, Set<SchemaPath> schemaPathsInExpr) {
-      super.getFiltered(optionManager, filterPredicate, schemaPathsInExpr);
+    protected B getFiltered(OptionManager optionManager, FilterPredicate filterPredicate) {
+      super.getFiltered(optionManager, filterPredicate);
 
       if (!((AbstractParquetGroupScan) source).getRowGroupsMetadata().isEmpty()) {
-        filterRowGroupMetadata(optionManager, filterPredicate, schemaPathsInExpr);
+        filterRowGroupMetadata(optionManager, filterPredicate);
       }
       return self();
     }
@@ -552,11 +554,12 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
      *
      * @param optionManager     option manager
      * @param filterPredicate   filter expression
-     * @param schemaPathsInExpr columns used in filter expression
      */
     protected void filterRowGroupMetadata(OptionManager optionManager,
-                                          FilterPredicate filterPredicate,
-                                          Set<SchemaPath> schemaPathsInExpr) {
+                                          FilterPredicate filterPredicate) {
+      Set<SchemaPath> schemaPathsInExpr =
+          filterExpression.accept(FilterEvaluatorUtils.FieldReferenceFinder.INSTANCE, null);
+
       AbstractParquetGroupScan abstractParquetGroupScan = (AbstractParquetGroupScan) source;
       Multimap<Path, RowGroupMetadata> prunedRowGroups;
       if (!abstractParquetGroupScan.getFilesMetadata().isEmpty()
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 838180d..de555c2 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
@@ -21,9 +21,9 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.expr.FilterPredicate;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.stat.RowsMatch;
 import org.apache.drill.exec.ops.ExecutorFragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
@@ -31,6 +31,7 @@ import org.apache.drill.exec.physical.base.AbstractGroupScanWithMetadata;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.store.CommonParquetRecordReader;
@@ -41,6 +42,7 @@ import org.apache.drill.exec.store.parquet.metadata.MetadataBase;
 import org.apache.drill.exec.store.parquet.metadata.Metadata_V4;
 import org.apache.drill.exec.store.parquet2.DrillParquetReader;
 import org.apache.drill.metastore.statistics.ColumnStatistics;
+import org.apache.drill.metastore.util.SchemaPathUtils;
 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;
@@ -51,6 +53,8 @@ import org.apache.parquet.hadoop.CodecFactory;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
... 6987 lines suppressed ...