Return-Path: X-Original-To: apmail-tajo-commits-archive@minotaur.apache.org Delivered-To: apmail-tajo-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id CEF3EC553 for ; Tue, 2 Dec 2014 03:56:34 +0000 (UTC) Received: (qmail 63808 invoked by uid 500); 2 Dec 2014 03:56:34 -0000 Delivered-To: apmail-tajo-commits-archive@tajo.apache.org Received: (qmail 63735 invoked by uid 500); 2 Dec 2014 03:56:34 -0000 Mailing-List: contact commits-help@tajo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@tajo.apache.org Delivered-To: mailing list commits@tajo.apache.org Received: (qmail 63712 invoked by uid 99); 2 Dec 2014 03:56:34 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 02 Dec 2014 03:56:34 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 48B8D9B55FE; Tue, 2 Dec 2014 03:56:34 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jihoonson@apache.org To: commits@tajo.apache.org Date: Tue, 02 Dec 2014 03:56:34 -0000 Message-Id: <2ffdcd4accbe47e0bde9a6e5f56b57ba@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [01/14] tajo git commit: TAJO-1163: TableDesc should use URI instead of Path. Repository: tajo Updated Branches: refs/heads/index_support 1e9c38ded -> 708453cd8 http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java index 42338a8..d507b97 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java @@ -45,7 +45,10 @@ import org.apache.tajo.engine.planner.global.DataChannel; import org.apache.tajo.engine.planner.global.MasterPlan; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.master.session.Session; -import org.apache.tajo.plan.*; +import org.apache.tajo.plan.LogicalOptimizer; +import org.apache.tajo.plan.LogicalPlan; +import org.apache.tajo.plan.LogicalPlanner; +import org.apache.tajo.plan.PlanningException; import org.apache.tajo.plan.expr.AggregationFunctionCallEval; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.util.PlannerUtil; @@ -140,7 +143,7 @@ public class TestPhysicalPlanner { employee = new TableDesc( CatalogUtil.buildFQName(DEFAULT_DATABASE_NAME, "employee"), employeeSchema, employeeMeta, - employeePath); + employeePath.toUri()); catalog.createTable(employee); Path scorePath = new Path(testDir, "score"); @@ -149,7 +152,7 @@ public class TestPhysicalPlanner { appender.init(); score = new TableDesc( CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "score"), scoreSchema, scoreMeta, - scorePath); + scorePath.toUri()); tuple = new VTuple(scoreSchema.size()); int m = 0; for (int i = 1; i <= 5; i++) { @@ -192,7 +195,7 @@ public class TestPhysicalPlanner { appender.init(); largeScore = new TableDesc( CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "score_large"), scoreSchmea, scoreLargeMeta, - scoreLargePath); + scoreLargePath.toUri()); Tuple tuple = new VTuple(scoreSchmea.size()); int m = 0; @@ -244,7 +247,7 @@ public class TestPhysicalPlanner { @Test public final void testCreateScanPlan() throws IOException, PlanningException { FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), - employee.getPath(), Integer.MAX_VALUE); + new Path(employee.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanPlan"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan), @@ -275,7 +278,7 @@ public class TestPhysicalPlanner { @Test public final void testCreateScanWithFilterPlan() throws IOException, PlanningException { FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), - employee.getPath(), Integer.MAX_VALUE); + new Path(employee.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanWithFilterPlan"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan), @@ -303,7 +306,7 @@ public class TestPhysicalPlanner { @Test public final void testGroupByPlan() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByPlan"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -334,7 +337,7 @@ public class TestPhysicalPlanner { @Test public final void testHashGroupByPlanWithALLField() throws IOException, PlanningException { // TODO - currently, this query does not use hash-based group operator. - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir( "target/test-data/testHashGroupByPlanWithALLField"); @@ -364,7 +367,7 @@ public class TestPhysicalPlanner { @Test public final void testSortGroupByPlan() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testSortGroupByPlan"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -427,7 +430,7 @@ public class TestPhysicalPlanner { @Test public final void testStorePlan() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -475,7 +478,7 @@ public class TestPhysicalPlanner { assertTrue("Checking meaningfulness of test", stats.getNumBytes() > StorageUnit.MB); FileFragment[] frags = StorageManager.splitNG(conf, "default.score_large", largeScore.getMeta(), - largeScore.getPath(), Integer.MAX_VALUE); + new Path(largeScore.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlanWithMaxOutputFileSize"); QueryContext queryContext = new QueryContext(conf, session); @@ -525,7 +528,7 @@ public class TestPhysicalPlanner { @Test public final void testStorePlanWithRCFile() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlanWithRCFile"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -566,7 +569,7 @@ public class TestPhysicalPlanner { @Test public final void testEnforceForDefaultColumnPartitionStorePlan() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -593,7 +596,7 @@ public class TestPhysicalPlanner { Enforcer enforcer = new Enforcer(); enforcer.enforceColumnPartitionAlgorithm(createTableNode.getPID(), ColumnPartitionAlgorithm.HASH_PARTITION); - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -617,7 +620,7 @@ public class TestPhysicalPlanner { Enforcer enforcer = new Enforcer(); enforcer.enforceColumnPartitionAlgorithm(createTableNode.getPID(), ColumnPartitionAlgorithm.SORT_PARTITION); - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -633,7 +636,7 @@ public class TestPhysicalPlanner { @Test public final void testPartitionedStorePlan() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), id, new FileFragment[] { frags[0] }, @@ -703,7 +706,7 @@ public class TestPhysicalPlanner { // Preparing working dir and input fragments FileFragment[] frags = StorageManager.splitNG(conf, "default.score_large", largeScore.getMeta(), - largeScore.getPath(), Integer.MAX_VALUE); + new Path(largeScore.getPath()), Integer.MAX_VALUE); QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testPartitionedStorePlanWithMaxFileSize"); @@ -766,7 +769,7 @@ public class TestPhysicalPlanner { @Test public final void testPartitionedStorePlanWithEmptyGroupingSet() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan); @@ -833,7 +836,7 @@ public class TestPhysicalPlanner { @Test public final void testAggregationFunction() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testAggregationFunction"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -864,7 +867,7 @@ public class TestPhysicalPlanner { @Test public final void testCountFunction() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCountFunction"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -892,7 +895,7 @@ public class TestPhysicalPlanner { @Test public final void testGroupByWithNullValue() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByWithNullValue"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -917,8 +920,8 @@ public class TestPhysicalPlanner { @Test public final void testUnionPlan() throws IOException, PlanningException, CloneNotSupportedException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(), - Integer.MAX_VALUE); + FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), + new Path(employee.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testUnionPlan"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan), @@ -982,8 +985,8 @@ public class TestPhysicalPlanner { //@Test public final void testCreateIndex() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(), - Integer.MAX_VALUE); + FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), + new Path(employee.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateIndex"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan), @@ -1009,8 +1012,8 @@ public class TestPhysicalPlanner { @Test public final void testDuplicateEliminate() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), - Integer.MAX_VALUE); + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), + new Path(score.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testDuplicateEliminate"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -1044,7 +1047,7 @@ public class TestPhysicalPlanner { @Test public final void testIndexedStoreExec() throws IOException, PlanningException { FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), - employee.getPath(), Integer.MAX_VALUE); + new Path(employee.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testIndexedStoreExec"); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), @@ -1137,7 +1140,7 @@ public class TestPhysicalPlanner { @Test public final void testSortEnforcer() throws IOException, PlanningException { FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), - employee.getPath(), Integer.MAX_VALUE); + new Path(employee.getPath()), Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testSortEnforcer"); Expr context = analyzer.parse(SORT_QUERY[0]); @@ -1187,8 +1190,8 @@ public class TestPhysicalPlanner { @Test public final void testGroupByEnforcer() throws IOException, PlanningException { - FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), score.getPath(), Integer.MAX_VALUE); - + FileFragment[] frags = StorageManager.splitNG(conf, "default.score", score.getMeta(), new Path(score.getPath()), + Integer.MAX_VALUE); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByEnforcer"); Expr context = analyzer.parse(QUERIES[7]); LogicalPlan plan = planner.createPlan(defaultContext, context); http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java index 9da9dee..a23a2d1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestProgressExternalSortExec.java @@ -109,7 +109,7 @@ public class TestProgressExternalSortExec { testDataStats = appender.getStats(); employee = new TableDesc( CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, employeeMeta, - employeePath); + employeePath.toUri()); catalog.createTable(employee); analyzer = new SQLAnalyzer(); planner = new LogicalPlanner(catalog); @@ -136,8 +136,8 @@ public class TestProgressExternalSortExec { } private void testProgress(int sortBufferBytesNum) throws Exception { - FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), employee.getPath(), - Integer.MAX_VALUE); + FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employee.getMeta(), + new Path(employee.getPath()), Integer.MAX_VALUE); Path workDir = new Path(testDir, TestExternalSortExec.class.getName()); TaskAttemptContext ctx = new TaskAttemptContext(new QueryContext(conf), LocalTajoTestingUtility.newQueryUnitAttemptId(), new FileFragment[] { frags[0] }, workDir); http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java index df1732c..9ebe871 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterHashJoinExec.java @@ -29,14 +29,17 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.engine.parser.SQLAnalyzer; -import org.apache.tajo.plan.LogicalPlanner; import org.apache.tajo.engine.planner.PhysicalPlanner; import org.apache.tajo.engine.planner.PhysicalPlannerImpl; -import org.apache.tajo.plan.PlanningException; import org.apache.tajo.engine.planner.enforce.Enforcer; -import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.engine.query.QueryContext; -import org.apache.tajo.storage.*; +import org.apache.tajo.plan.LogicalPlanner; +import org.apache.tajo.plan.PlanningException; +import org.apache.tajo.plan.logical.LogicalNode; +import org.apache.tajo.storage.Appender; +import org.apache.tajo.storage.StorageManager; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.TUtil; @@ -229,9 +232,9 @@ public class TestRightOuterHashJoinExec { @Test public final void testRightOuter_HashJoinExec0() throws IOException, PlanningException { - FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), + FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE); - FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), dep3.getPath(), + FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags); @@ -270,9 +273,9 @@ public class TestRightOuterHashJoinExec { @Test public final void testRightOuter_HashJoinExec1() throws IOException, PlanningException { - FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), + FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE); - FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), + FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(emp3Frags, job3Frags); @@ -311,9 +314,9 @@ public class TestRightOuterHashJoinExec { @Test public final void testRightOuter_HashJoinExec2() throws IOException, PlanningException { - FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), + FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE); - FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), + FileFragment[] job3Frags = StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags); http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java index 77e4e4e..4956b7f 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestRightOuterMergeJoinExec.java @@ -313,8 +313,10 @@ public class TestRightOuterMergeJoinExec { Enforcer enforcer = new Enforcer(); enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN); - FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE); - FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), dep3.getPath(), Integer.MAX_VALUE); + FileFragment[] emp3Frags = StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), + Integer.MAX_VALUE); + FileFragment[] dep3Frags = StorageManager.splitNG(conf, DEP3_NAME, dep3.getMeta(), new Path(dep3.getPath()), + Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(emp3Frags, dep3Frags); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin0"); @@ -349,9 +351,9 @@ public class TestRightOuterMergeJoinExec { enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN); FileFragment[] emp3Frags = - StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE); + StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE); FileFragment[] job3Frags = - StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), Integer.MAX_VALUE); + StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin1"); @@ -385,9 +387,9 @@ public class TestRightOuterMergeJoinExec { enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN); FileFragment[] emp3Frags = - StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE); + StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE); FileFragment[] job3Frags = - StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), job3.getPath(), Integer.MAX_VALUE); + StorageManager.splitNG(conf, JOB3_NAME, job3.getMeta(), new Path(job3.getPath()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(job3Frags, emp3Frags); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuterMergeJoin2"); @@ -421,9 +423,9 @@ public class TestRightOuterMergeJoinExec { enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN); FileFragment[] emp3Frags = - StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE); + StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE); FileFragment[] dep4Frags = - StorageManager.splitNG(conf, DEP4_NAME, dep4.getMeta(), dep4.getPath(), Integer.MAX_VALUE); + StorageManager.splitNG(conf, DEP4_NAME, dep4.getMeta(), new Path(dep4.getPath()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(emp3Frags, dep4Frags); Path workDir = CommonTestingUtil.getTestDir("target/test-data/testRightOuter_MergeJoin3"); @@ -458,9 +460,9 @@ public class TestRightOuterMergeJoinExec { enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN); FileFragment[] emp3Frags = - StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE); + StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE); FileFragment[] phone3Frags = - StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(), + StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(emp3Frags, phone3Frags); @@ -495,8 +497,8 @@ public class TestRightOuterMergeJoinExec { enforcer.enforceJoinAlgorithm(joinNode.getPID(), JoinAlgorithm.MERGE_JOIN); FileFragment[] emp3Frags = - StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), emp3.getPath(), Integer.MAX_VALUE); - FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), phone3.getPath(), + StorageManager.splitNG(conf, EMP3_NAME, emp3.getMeta(), new Path(emp3.getPath()), Integer.MAX_VALUE); + FileFragment[] phone3Frags = StorageManager.splitNG(conf, PHONE3_NAME, phone3.getMeta(), new Path(phone3.getPath()), Integer.MAX_VALUE); FileFragment[] merged = TUtil.concat(phone3Frags,emp3Frags); http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java index 3ed5997..afa7430 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java @@ -97,7 +97,7 @@ public class TestSortExec { TableDesc desc = new TableDesc( CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, employeeMeta, - tablePath); + tablePath.toUri()); catalog.createTable(desc); analyzer = new SQLAnalyzer(); http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java index 27d59a9..0e02079 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java @@ -71,7 +71,7 @@ public class TestCTASQuery extends QueryTestCaseBase { assertEquals("key", partitionDesc.getExpressionSchema().getColumns().get(0).getSimpleName()); FileSystem fs = FileSystem.get(testBase.getTestingCluster().getConfiguration()); - Path path = desc.getPath(); + Path path = new Path(desc.getPath()); assertTrue(fs.isDirectory(path)); assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=17.0"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=36.0"))); @@ -114,7 +114,7 @@ public class TestCTASQuery extends QueryTestCaseBase { assertEquals("key", partitionDesc.getExpressionSchema().getColumns().get(0).getSimpleName()); FileSystem fs = FileSystem.get(cluster.getConfiguration()); - Path path = desc.getPath(); + Path path = new Path(desc.getPath()); assertTrue(fs.isDirectory(path)); assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=17.0"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=36.0"))); @@ -225,7 +225,7 @@ public class TestCTASQuery extends QueryTestCaseBase { TableDesc desc = client.getTableDesc("managed_table1"); assertNotNull(desc); - assertEquals("managed_table1", desc.getPath().getName()); + assertEquals("managed_table1", new Path(desc.getPath()).getName()); } else { assertFalse(client.existTable("managed_Table1")); assertTrue(client.existTable("MANAGED_TABLE1")); @@ -233,7 +233,7 @@ public class TestCTASQuery extends QueryTestCaseBase { TableDesc desc = client.getTableDesc("MANAGED_TABLE1"); assertNotNull(desc); - assertEquals("MANAGED_TABLE1", desc.getPath().getName()); + assertEquals("MANAGED_TABLE1", new Path(desc.getPath()).getName()); } } } http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java index cb77936..18f55ff 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java @@ -107,8 +107,9 @@ public class TestCreateTable extends QueryTestCaseBase { // checking the existence of the table directory and validating the path FileSystem fs = testingCluster.getMaster().getStorageManager().getFileSystem(); Path warehouseDir = TajoConf.getWarehouseDir(testingCluster.getConfiguration()); - assertTrue(fs.exists(oldTableDesc.getPath())); - assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, originalTableName), oldTableDesc.getPath()); + assertTrue(fs.exists(new Path(oldTableDesc.getPath()))); + assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, originalTableName), + new Path(oldTableDesc.getPath())); // Rename client.executeQuery("ALTER TABLE " + CatalogUtil.denormalizeIdentifier(oldFQTableName) @@ -117,8 +118,8 @@ public class TestCreateTable extends QueryTestCaseBase { // checking the existence of the new table directory and validating the path final String newFQTableName = CatalogUtil.buildFQName(databaseName, newTableName); TableDesc newTableDesc = client.getTableDesc(newFQTableName); - assertTrue(fs.exists(newTableDesc.getPath())); - assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, newTableName), newTableDesc.getPath()); + assertTrue(fs.exists(new Path(newTableDesc.getPath()))); + assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, newTableName), new Path(newTableDesc.getPath())); } @Test http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java index 0d309c7..9c97a55 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java @@ -266,7 +266,7 @@ public class TestInsertQuery extends QueryTestCaseBase { TableDesc tableDesc = testingCluster.getMaster().getCatalog().getTableDesc(getCurrentDatabase(), tableName); assertNotNull(tableDesc); - Path path = tableDesc.getPath(); + Path path = new Path(tableDesc.getPath()); FileSystem fs = path.getFileSystem(testingCluster.getConfiguration()); FileStatus[] files = fs.listStatus(path); @@ -451,10 +451,10 @@ public class TestInsertQuery extends QueryTestCaseBase { } FileSystem fs = FileSystem.get(testingCluster.getConfiguration()); - assertTrue(fs.exists(desc.getPath())); + assertTrue(fs.exists(new Path(desc.getPath()))); CompressionCodecFactory factory = new CompressionCodecFactory(testingCluster.getConfiguration()); - for (FileStatus file : fs.listStatus(desc.getPath())) { + for (FileStatus file : fs.listStatus(new Path(desc.getPath()))) { CompressionCodec codec = factory.getCodec(file.getPath()); assertTrue(codec instanceof DeflateCodec); } http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java index 3d413b7..c1cec2b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java @@ -563,7 +563,7 @@ public class TestJoinBroadcast extends QueryTestCaseBase { appender.flush(); appender.close(); } - Path dataPath = new Path(table.getPath(), fileIndex + ".csv"); + Path dataPath = new Path(table.getPath().toString(), fileIndex + ".csv"); fileIndex++; appender = StorageManager.getStorageManager(conf).getAppender(tableMeta, schema, dataPath); @@ -811,16 +811,17 @@ public class TestJoinBroadcast extends QueryTestCaseBase { private void addEmptyDataFile(String tableName, boolean isPartitioned) throws Exception { TableDesc table = client.getTableDesc(tableName); - FileSystem fs = table.getPath().getFileSystem(conf); + Path path = new Path(table.getPath()); + FileSystem fs = path.getFileSystem(conf); if (isPartitioned) { - List partitionPathList = getPartitionPathList(fs, table.getPath()); + List partitionPathList = getPartitionPathList(fs, path); for (Path eachPath: partitionPathList) { Path dataPath = new Path(eachPath, 0 + "_empty.csv"); OutputStream out = fs.create(dataPath); out.close(); } } else { - Path dataPath = new Path(table.getPath(), 0 + "_empty.csv"); + Path dataPath = new Path(path, 0 + "_empty.csv"); OutputStream out = fs.create(dataPath); out.close(); } http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java index 6378814..0e9ec7d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java @@ -202,7 +202,7 @@ public class TestTablePartitions extends QueryTestCaseBase { private void assertPartitionDirectories(TableDesc desc) throws IOException { FileSystem fs = FileSystem.get(conf); - Path path = desc.getPath(); + Path path = new Path(desc.getPath()); assertTrue(fs.isDirectory(path)); assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=17.0"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/key=36.0"))); @@ -309,7 +309,7 @@ public class TestTablePartitions extends QueryTestCaseBase { res.close(); TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName); - Path path = desc.getPath(); + Path path = new Path(desc.getPath()); FileSystem fs = FileSystem.get(conf); assertTrue(fs.isDirectory(path)); @@ -373,7 +373,7 @@ public class TestTablePartitions extends QueryTestCaseBase { res.close(); TableDesc desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName); - Path path = desc.getPath(); + Path path = new Path(desc.getPath()); FileSystem fs = FileSystem.get(conf); assertTrue(fs.isDirectory(path)); @@ -426,7 +426,7 @@ public class TestTablePartitions extends QueryTestCaseBase { res.close(); desc = catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName); - path = desc.getPath(); + path = new Path(desc.getPath()); assertTrue(fs.isDirectory(path)); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1"))); @@ -455,7 +455,7 @@ public class TestTablePartitions extends QueryTestCaseBase { "R\n" + "R\n"; - String tableData = getTableFileContents(desc.getPath()); + String tableData = getTableFileContents(new Path(desc.getPath())); assertEquals(expected, tableData); res = executeString("select * from " + tableName + " where col2 = 2"); @@ -541,10 +541,10 @@ public class TestTablePartitions extends QueryTestCaseBase { } FileSystem fs = FileSystem.get(conf); - assertTrue(fs.exists(desc.getPath())); + assertTrue(fs.exists(new Path(desc.getPath()))); CompressionCodecFactory factory = new CompressionCodecFactory(conf); - Path path = desc.getPath(); + Path path = new Path(desc.getPath()); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=2"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=3"))); @@ -578,10 +578,10 @@ public class TestTablePartitions extends QueryTestCaseBase { } FileSystem fs = FileSystem.get(conf); - assertTrue(fs.exists(desc.getPath())); + assertTrue(fs.exists(new Path(desc.getPath()))); CompressionCodecFactory factory = new CompressionCodecFactory(conf); - Path path = desc.getPath(); + Path path = new Path(desc.getPath()); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=2"))); @@ -623,10 +623,10 @@ public class TestTablePartitions extends QueryTestCaseBase { } FileSystem fs = FileSystem.get(conf); - assertTrue(fs.exists(desc.getPath())); + assertTrue(fs.exists(new Path(desc.getPath()))); CompressionCodecFactory factory = new CompressionCodecFactory(conf); - Path path = desc.getPath(); + Path path = new Path(desc.getPath()); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1/col3=17.0"))); @@ -706,10 +706,10 @@ public class TestTablePartitions extends QueryTestCaseBase { } FileSystem fs = FileSystem.get(conf); - assertTrue(fs.exists(desc.getPath())); + assertTrue(fs.exists(new Path(desc.getPath()))); CompressionCodecFactory factory = new CompressionCodecFactory(conf); - Path path = desc.getPath(); + Path path = new Path(desc.getPath()); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1"))); assertTrue(fs.isDirectory(new Path(path.toUri() + "/col1=1/col2=1/col3=17.0"))); http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java index 0c92ac1..f491c63 100644 --- a/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java +++ b/tajo-core/src/test/java/org/apache/tajo/jdbc/TestResultSet.java @@ -95,7 +95,7 @@ public class TestResultSet { stats.setNumBlocks(1000); stats.setNumShuffleOutputs(100); desc = new TableDesc(CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "score"), - scoreSchema, scoreMeta, p); + scoreSchema, scoreMeta, p.toUri()); desc.setStats(stats); } http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java index e35b80c..186a7f5 100644 --- a/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java +++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java @@ -142,7 +142,7 @@ public class TestRangeRetrieverHandler { TableDesc employee = new TableDesc( CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, employeeMeta, - tableDir); + tableDir.toUri()); catalog.createTable(employee); FileFragment[] frags = StorageManager.splitNG(conf, "default.employee", employeeMeta, tableDir, Integer.MAX_VALUE); @@ -265,7 +265,7 @@ public class TestRangeRetrieverHandler { appender.close(); TableDesc employee = new TableDesc( - CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, meta, tablePath); + CatalogUtil.buildFQName(TajoConstants.DEFAULT_DATABASE_NAME, "employee"), schema, meta, tablePath.toUri()); catalog.createTable(employee); FileFragment[] frags = sm.splitNG(conf, "default.employee", meta, tablePath, Integer.MAX_VALUE); http://git-wip-us.apache.org/repos/asf/tajo/blob/f6e09a5a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java ---------------------------------------------------------------------- diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java index e953ccf..3b1b4e3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java @@ -1289,10 +1289,11 @@ public class LogicalPlanner extends BaseAlgebraVisitor 0) { // There are at least one indexable predicates return findFilteredPaths(paritionValuesSchema, - indexablePredicateSet.toArray(new EvalNode[indexablePredicateSet.size()]), table.getPath()); + indexablePredicateSet.toArray(new EvalNode[indexablePredicateSet.size()]), new Path(table.getPath())); } else { // otherwise, we will get all partition paths. - return findFilteredPaths(paritionValuesSchema, null, table.getPath()); + return findFilteredPaths(paritionValuesSchema, null, new Path(table.getPath())); } }