DRILL-8214: Replace EnumerableTableScan usage with LogicalTableScan (#2538)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
index 865ea04..650c220 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
@@ -27,6 +27,7 @@
 import java.util.Map;
 
 import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.util.GuavaUtils;
 import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
@@ -34,7 +35,6 @@
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
-import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.prepare.RelOptTableImpl;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.util.BitSets;
@@ -42,7 +42,7 @@
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.physical.base.FileGroupScan;
-import org.apache.drill.exec.planner.logical.DirPrunedEnumerableTableScan;
+import org.apache.drill.exec.planner.logical.DirPrunedTableScan;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillScanRel;
 import org.apache.drill.exec.planner.logical.DrillTable;
@@ -71,7 +71,8 @@
   private final DrillTable table;
 
   public FileSystemPartitionDescriptor(PlannerSettings settings, TableScan scanRel) {
-    Preconditions.checkArgument(scanRel instanceof DrillScanRel || scanRel instanceof EnumerableTableScan);
+    Preconditions.checkArgument(scanRel instanceof DrillScanRel
+      || supportsScan(scanRel));
     this.partitionLabel = settings.getFsPartitionColumnLabel();
     this.partitionLabelLength = partitionLabel.length();
     this.scanRel = scanRel;
@@ -203,7 +204,7 @@
         fileLocations = selection.getFiles();
         isExpandedPartial = selection.isExpandedPartial();
       }
-    } else if (scanRel instanceof EnumerableTableScan) {
+    } else if (supportsScan(scanRel)) {
       FileSelection selection = ((FormatSelection) table.getSelection()).getSelection();
       fileLocations = selection.getFiles();
       isExpandedPartial = selection.isExpandedPartial();
@@ -242,7 +243,7 @@
                       scanRel.getRowType(),
                       ((DrillScanRel) scanRel).getColumns(),
                       true /*filter pushdown*/);
-    } else if (scanRel instanceof EnumerableTableScan) {
+    } else if (supportsScan(scanRel)) {
       FormatSelection newFormatSelection = new FormatSelection(formatSelection.getFormat(), newFileSelection);
 
       DynamicDrillTable dynamicDrillTable = new DynamicDrillTable(table.getPlugin(), table.getStorageEngineName(),
@@ -253,10 +254,10 @@
       RelOptTableImpl newOptTableImpl = RelOptTableImpl.create(relOptTable.getRelOptSchema(), relOptTable.getRowType(),
           newTable, GuavaUtils.convertToUnshadedImmutableList(ImmutableList.of()));
 
-      // return an EnumerableTableScan with fileSelection being part of digest of TableScan node.
-      return DirPrunedEnumerableTableScan.create(scanRel.getCluster(), newOptTableImpl, newFileSelection.toString());
+      // return an DirPrunedTableScan with fileSelection being part of digest of TableScan node.
+      return DirPrunedTableScan.create(scanRel.getCluster(), newOptTableImpl, newFileSelection.toString());
     } else {
-      throw new UnsupportedOperationException("Only DrillScanRel and EnumerableTableScan is allowed!");
+      throw new UnsupportedOperationException("Only DrillScanRel and DirPrunedTableScan is allowed!");
     }
   }
 
@@ -272,4 +273,9 @@
     return selection instanceof FormatSelection
         && ((FormatSelection)selection).getSelection().getCacheFileRoot() != null;
   }
+
+  private static boolean supportsScan(TableScan scanRel) {
+    return scanRel instanceof DirPrunedTableScan
+      || scanRel instanceof LogicalTableScan;
+  }
 }
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 97b34e1..aa61c34 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
@@ -346,6 +346,7 @@
       // Due to infinite loop in planning (DRILL-3257/CALCITE-1271), temporarily use this rule in Hep planner
       // RuleInstance.PROJECT_SET_OP_TRANSPOSE_RULE,
       RuleInstance.PROJECT_WINDOW_TRANSPOSE_RULE,
+      DrillPushProjectIntoScanRule.LOGICAL_INSTANCE,
       DrillPushProjectIntoScanRule.INSTANCE,
       DrillPushProjectIntoScanRule.DRILL_LOGICAL_INSTANCE,
 
@@ -353,7 +354,8 @@
        Convert from Calcite Logical to Drill Logical Rules.
        */
       RuleInstance.EXPAND_CONVERSION_RULE,
-      DrillScanRule.INSTANCE,
+      DrillScanRule.LOGICAL_TABLE_SCAN_TO_DRILL,
+      DrillScanRule.DIR_PRUNED_TABLE_SCAN_TO_DRILL,
       DrillFilterRule.INSTANCE,
       DrillProjectRule.INSTANCE,
       DrillWindowRule.INSTANCE,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java
deleted file mode 100644
index 3287a80..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.logical;
-
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import org.apache.calcite.adapter.enumerable.EnumerableConvention;
-import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.schema.Table;
-
-import java.util.List;
-
-/**
- * This class extends from EnumerableTableScan. It puts the file selection string into it's digest.
- * When directory-based partition pruning applied, file selection could be different for the same
- * table.
- */
-public class DirPrunedEnumerableTableScan extends EnumerableTableScan {
-  private final String digestFromSelection;
-
-  public DirPrunedEnumerableTableScan(RelOptCluster cluster, RelTraitSet traitSet,
-      RelOptTable table, Class elementType, String digestFromSelection) {
-    super(cluster, traitSet, table, elementType);
-    this.digestFromSelection = digestFromSelection;
-  }
-
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    final Table tbl = this.table.unwrap(Table.class);
-    Class elementType = EnumerableTableScan.deduceElementType(tbl);
-
-    return new DirPrunedEnumerableTableScan(getCluster(), traitSet, table, elementType, digestFromSelection);
-  }
-
-  /** Creates an DirPrunedEnumerableTableScan. */
-  public static EnumerableTableScan create(RelOptCluster cluster,
-      RelOptTable relOptTable, String digestFromSelection) {
-    final Table table = relOptTable.unwrap(Table.class);
-    Class elementType = EnumerableTableScan.deduceElementType(table);
-    final RelTraitSet traitSet =
-        cluster.traitSetOf(EnumerableConvention.INSTANCE)
-            .replaceIfs(RelCollationTraitDef.INSTANCE,
-                () -> {
-                  if (table != null) {
-                    return table.getStatistic().getCollations();
-                  }
-                  return ImmutableList.of();
-                });
-    return new DirPrunedEnumerableTableScan(cluster, traitSet, relOptTable, elementType, digestFromSelection);
-  }
-
-  @Override
-  public RelWriter explainTerms(RelWriter pw) {
-    return super.explainTerms(pw).item("selection", this.digestFromSelection);
-  }
-
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedTableScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedTableScan.java
new file mode 100644
index 0000000..91973d7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedTableScan.java
@@ -0,0 +1,70 @@
+/*
+ * 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.Convention;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.schema.Table;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * This class extends from {@link TableScan}. It puts the file selection string into it's digest.
+ * When directory-based partition pruning applied, file selection could be different for the same
+ * table.
+ */
+public class DirPrunedTableScan extends TableScan {
+  private final String digestFromSelection;
+
+  public DirPrunedTableScan(RelOptCluster cluster, RelTraitSet traitSet,
+    RelOptTable table, String digestFromSelection) {
+    super(cluster, traitSet, table);
+    this.digestFromSelection = digestFromSelection;
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new DirPrunedTableScan(getCluster(), traitSet, table, digestFromSelection);
+  }
+
+  /** Creates an DirPrunedTableScan. */
+  public static TableScan create(RelOptCluster cluster,
+    RelOptTable relOptTable, String digestFromSelection) {
+    Table table = relOptTable.unwrap(Table.class);
+    RelTraitSet traitSet =
+      cluster.traitSetOf(Convention.NONE)
+        .replaceIfs(RelCollationTraitDef.INSTANCE,
+          () -> table != null
+            ? table.getStatistic().getCollations()
+            : Collections.emptyList());
+    return new DirPrunedTableScan(cluster, traitSet, relOptTable, digestFromSelection);
+  }
+
+  @Override
+  public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw).item("selection", this.digestFromSelection);
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectIntoScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectIntoScanRule.java
index 9da72c4..46dd07c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectIntoScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectIntoScanRule.java
@@ -17,13 +17,13 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
@@ -46,16 +46,28 @@
 public class DrillPushProjectIntoScanRule extends RelOptRule {
   public static final RelOptRule INSTANCE =
       new DrillPushProjectIntoScanRule(LogicalProject.class,
-          EnumerableTableScan.class,
+          DirPrunedTableScan.class,
           "DrillPushProjectIntoScanRule:enumerable") {
 
         @Override
         protected boolean skipScanConversion(RelDataType projectRelDataType, TableScan scan) {
-          // do not allow skipping conversion of EnumerableTableScan to DrillScanRel if rule is applicable
+          // do not allow skipping conversion of DirPrunedTableScan to DrillScanRel if rule is applicable
           return false;
         }
       };
 
+  public static final RelOptRule LOGICAL_INSTANCE =
+    new DrillPushProjectIntoScanRule(LogicalProject.class,
+      LogicalTableScan.class,
+      "DrillPushProjectIntoScanRule:none") {
+
+      @Override
+      protected boolean skipScanConversion(RelDataType projectRelDataType, TableScan scan) {
+        // do not allow skipping conversion of LogicalTableScan to DrillScanRel if rule is applicable
+        return false;
+      }
+    };
+
   public static final RelOptRule DRILL_LOGICAL_INSTANCE =
       new DrillPushProjectIntoScanRule(LogicalProject.class,
           DrillScanRel.class,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
index da9d707..d94a87b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
@@ -17,24 +17,25 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
-
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalTableScan;
 
 public class DrillScanRule  extends RelOptRule {
-  public static final RelOptRule INSTANCE = new DrillScanRule();
+  public static final RelOptRule LOGICAL_TABLE_SCAN_TO_DRILL = new DrillScanRule(LogicalTableScan.class);
+  public static final RelOptRule DIR_PRUNED_TABLE_SCAN_TO_DRILL = new DrillScanRule(DirPrunedTableScan.class);
 
-  private DrillScanRule() {
-    super(RelOptHelper.any(EnumerableTableScan.class),
-        DrillRelFactories.LOGICAL_BUILDER, "DrillScanRule");
+  private DrillScanRule(Class<? extends TableScan> scan) {
+    super(RelOptHelper.any(scan),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillScanRule:" + scan.getSimpleName());
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final EnumerableTableScan access = call.rel(0);
-    final RelTraitSet traits = access.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
+    TableScan access = call.rel(0);
+    RelTraitSet traits = access.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     call.transformTo(new DrillScanRel(access.getCluster(), traits, access.getTable()));
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
index 7a9dc0c..1df5eec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
@@ -20,14 +20,15 @@
 import java.io.IOException;
 import java.util.Objects;
 
-import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.schema.Schema.TableType;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Statistics;
 import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.drill.common.JSONOptions;
@@ -42,7 +43,7 @@
 import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.util.ImpersonationUtil;
 
-public abstract class DrillTable implements Table {
+public abstract class DrillTable implements Table, TranslatableTable {
 
   private final String storageEngineName;
   private final StoragePluginConfig storageEngineConfig;
@@ -165,10 +166,11 @@
     return Statistics.UNKNOWN;
   }
 
+  @Override
   public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable table) {
     // returns non-drill table scan to allow directory-based partition pruning
     // before table group scan is created
-    return EnumerableTableScan.create(context.getCluster(), table);
+    return LogicalTableScan.create(context.getCluster(), table);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
index b72ecee..11e85fc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
@@ -26,10 +26,11 @@
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
+import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.logical.DirPrunedTableScan;
 import org.apache.drill.exec.util.DrillFileSystemUtil;
 import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
-import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Filter;
@@ -541,7 +542,7 @@
   public abstract PartitionDescriptor getPartitionDescriptor(PlannerSettings settings, TableScan scanRel);
 
   private static boolean isQualifiedDirPruning(final TableScan scan) {
-    if (scan instanceof EnumerableTableScan) {
+    if (supportsScan(scan)) {
       final Object selection = DrillRelOptUtil.getDrillTable(scan).getSelection();
       if (selection instanceof FormatSelection
           && ((FormatSelection)selection).supportsDirPruning()) {
@@ -767,7 +768,7 @@
     }
 
     private static boolean isQualifiedFilePruning(final TableScan scan) {
-      if (scan instanceof EnumerableTableScan) {
+      if (supportsScan(scan)) {
         Object selection = DrillRelOptUtil.getDrillTable(scan).getSelection();
         return selection instanceof FormatSelection;
       } else if (scan instanceof DrillScanRel) {
@@ -778,4 +779,9 @@
       return false;
     }
   }
+
+  private static boolean supportsScan(TableScan scan) {
+    return scan instanceof DirPrunedTableScan
+      || scan instanceof LogicalTableScan;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DrillTableInfo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DrillTableInfo.java
index 0b45973..09b8f02 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DrillTableInfo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DrillTableInfo.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.planner.sql.handlers;
 
 import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlIdentifier;
@@ -31,7 +30,6 @@
 import org.apache.calcite.util.Util;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.planner.logical.DrillTable;
-import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
 import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
@@ -92,8 +90,7 @@
         AbstractSchema drillSchema = SchemaUtilites.resolveToDrillSchema(
             config.getConverter().getDefaultSchema(), SchemaUtilites.getSchemaPath(tableIdentifier));
 
-        TranslatableTable translatableTable = tableMacro.getTable(config.getConverter().getTypeFactory(), prepareTableMacroOperands(call.operand(0)));
-        DrillTable table = ((DrillTranslatableTable) translatableTable).getDrillTable();
+        DrillTable table = (DrillTable) tableMacro.getTable(config.getConverter().getTypeFactory(), prepareTableMacroOperands(call.operand(0)));
         return new DrillTableInfo(table, drillSchema.getSchemaPath(), Util.last(tableIdentifier.names));
       }
       case IDENTIFIER: {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/table/function/WithOptionsTableMacro.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/table/function/WithOptionsTableMacro.java
index 2a6bebc..6531ba1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/table/function/WithOptionsTableMacro.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/table/function/WithOptionsTableMacro.java
@@ -24,7 +24,6 @@
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.planner.logical.DrillTable;
-import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -54,9 +53,7 @@
         .message("Unable to find table [%s]", sig.getName())
         .build(logger);
     }
-    return drillTable instanceof TranslatableTable
-        ? (TranslatableTable) drillTable :
-        new DrillTranslatableTable(drillTable);
+    return drillTable;
   }
 
   @Override