This is an automated email from the ASF dual-hosted git repository. qiangcai pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/carbondata.git
The following commit(s) were added to refs/heads/master by this push: new 16763bd [CARBONDATA-3786] Presto carbon reader should use tablePath from hive catalog 16763bd is described below commit 16763bdf46ec3d5ed4ba008dc819bf9da212247a Author: ajantha-bhat <ajanthab...@gmail.com> AuthorDate: Wed Apr 29 21:53:03 2020 +0530 [CARBONDATA-3786] Presto carbon reader should use tablePath from hive catalog Why is this PR needed? In upgrade scenarios of 1.6 to 2.0, when sparl.sql.warehouse is not configured. Hive storage location is not proper. so presto carbon integration should use tablePath from hive storage instead of location. What changes were proposed in this PR? use tablePath instead of location from hive metatstroe table. This closes #3731 --- .../org/apache/carbondata/presto/CarbondataSplitManager.java | 9 +++++++-- .../org/apache/carbondata/presto/CarbondataSplitManager.java | 9 +++++++-- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/integration/presto/src/main/prestodb/org/apache/carbondata/presto/CarbondataSplitManager.java b/integration/presto/src/main/prestodb/org/apache/carbondata/presto/CarbondataSplitManager.java index d6caa4f..5bcadd9 100755 --- a/integration/presto/src/main/prestodb/org/apache/carbondata/presto/CarbondataSplitManager.java +++ b/integration/presto/src/main/prestodb/org/apache/carbondata/presto/CarbondataSplitManager.java @@ -66,6 +66,7 @@ import com.facebook.presto.spi.TableNotFoundException; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; import com.facebook.presto.spi.predicate.TupleDomain; import com.google.common.collect.ImmutableList; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -109,8 +110,12 @@ public class CarbondataSplitManager extends HiveSplitManager { if (!table.getStorage().getStorageFormat().getInputFormat().contains("carbon")) { return super.getSplits(transactionHandle, session, layoutHandle, splitSchedulingStrategy); } - String location = table.getStorage().getLocation(); - + // for hive metastore, get table location from catalog table's tablePath + String location = table.getStorage().getSerdeParameters().get("tablePath"); + if (StringUtils.isEmpty(location)) { + // file metastore case tablePath can be null, so get from location + location = table.getStorage().getLocation(); + } String queryId = System.nanoTime() + ""; QueryStatistic statistic = new QueryStatistic(); QueryStatisticsRecorder statisticRecorder = CarbonTimeStatisticsFactory.createDriverRecorder(); diff --git a/integration/presto/src/main/prestosql/org/apache/carbondata/presto/CarbondataSplitManager.java b/integration/presto/src/main/prestosql/org/apache/carbondata/presto/CarbondataSplitManager.java index 93ebe42..fe1dd9a 100755 --- a/integration/presto/src/main/prestosql/org/apache/carbondata/presto/CarbondataSplitManager.java +++ b/integration/presto/src/main/prestosql/org/apache/carbondata/presto/CarbondataSplitManager.java @@ -68,6 +68,7 @@ import io.prestosql.spi.connector.FixedSplitSource; import io.prestosql.spi.connector.SchemaTableName; import io.prestosql.spi.connector.TableNotFoundException; import io.prestosql.spi.predicate.TupleDomain; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -118,8 +119,12 @@ public class CarbondataSplitManager extends HiveSplitManager { if (!table.getStorage().getStorageFormat().getInputFormat().contains("carbon")) { return super.getSplits(transactionHandle, session, tableHandle, splitSchedulingStrategy); } - String location = table.getStorage().getLocation(); - + // for hive metastore, get table location from catalog table's tablePath + String location = table.getStorage().getSerdeParameters().get("tablePath"); + if (StringUtils.isEmpty(location)) { + // file metastore case tablePath can be null, so get from location + location = table.getStorage().getLocation(); + } String queryId = System.nanoTime() + ""; QueryStatistic statistic = new QueryStatistic(); QueryStatisticsRecorder statisticRecorder = CarbonTimeStatisticsFactory.createDriverRecorder();