You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2018/03/20 15:44:34 UTC
[01/10] carbondata git commit: [HOTFIX] Fix CI random failure [Forced
Update!]
Repository: carbondata
Updated Branches:
refs/heads/carbonfile 99766b8af -> b384b6e1f (forced update)
[HOTFIX] Fix CI random failure
This closes #2068
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/04ff3676
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/04ff3676
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/04ff3676
Branch: refs/heads/carbonfile
Commit: 04ff36764c797264f5396fa3cbf1d6fe883737e0
Parents: a386f1f
Author: Jacky Li <ja...@qq.com>
Authored: Thu Mar 15 19:49:07 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Fri Mar 16 21:12:25 2018 +0800
----------------------------------------------------------------------
.../statusmanager/SegmentStatusManager.java | 66 ++++++++++++--------
.../carbondata/spark/util/CommonUtil.scala | 2 +-
.../preaaggregate/PreAggregateTableHelper.scala | 13 +++-
3 files changed, 53 insertions(+), 28 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/04ff3676/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index 820a5a4..f466018 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -30,7 +30,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
-import org.apache.carbondata.common.exceptions.TableStatusLockException;
import org.apache.carbondata.common.logging.LogService;
import org.apache.carbondata.common.logging.LogServiceFactory;
import org.apache.carbondata.core.constants.CarbonCommonConstants;
@@ -834,10 +833,30 @@ public class SegmentStatusManager {
}
}
- public static void deleteLoadsAndUpdateMetadata(
+ private static class ReturnTuple {
+ LoadMetadataDetails[] details;
+ boolean isUpdateRequired;
+ ReturnTuple(LoadMetadataDetails[] details, boolean isUpdateRequired) {
+ this.details = details;
+ this.isUpdateRequired = isUpdateRequired;
+ }
+ }
+
+ private static ReturnTuple isUpdationRequired(
+ boolean isForceDeletion,
CarbonTable carbonTable,
- boolean isForceDeletion) throws IOException {
- deleteLoadsAndUpdateMetadata(carbonTable, isForceDeletion, null);
+ AbsoluteTableIdentifier absoluteTableIdentifier) {
+ LoadMetadataDetails[] details =
+ SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
+ // Delete marked loads
+ boolean isUpdationRequired =
+ DeleteLoadFolders.deleteLoadFoldersFromFileSystem(
+ absoluteTableIdentifier,
+ isForceDeletion,
+ details,
+ carbonTable.getMetadataPath()
+ );
+ return new ReturnTuple(details, isUpdationRequired);
}
public static void deleteLoadsAndUpdateMetadata(
@@ -845,31 +864,29 @@ public class SegmentStatusManager {
boolean isForceDeletion,
List<PartitionSpec> partitionSpecs) throws IOException {
if (isLoadDeletionRequired(carbonTable.getMetadataPath())) {
- LoadMetadataDetails[] details =
- SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
- ICarbonLock carbonTableStatusLock = CarbonLockFactory.getCarbonLockObj(
- identifier, LockUsage.TABLE_STATUS_LOCK);
-
- // Delete marked loads
- boolean isUpdationRequired = DeleteLoadFolders.deleteLoadFoldersFromFileSystem(
- identifier, isForceDeletion, details, carbonTable.getMetadataPath());
-
- boolean updationCompletionStatus = false;
-
- if (isUpdationRequired) {
+ ReturnTuple tuple = isUpdationRequired(isForceDeletion, carbonTable, identifier);
+ if (tuple.isUpdateRequired) {
+ ICarbonLock carbonTableStatusLock =
+ CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.TABLE_STATUS_LOCK);
+ boolean locked = false;
try {
// Update load metadate file after cleaning deleted nodes
- if (carbonTableStatusLock.lockWithRetries()) {
+ locked = carbonTableStatusLock.lockWithRetries();
+ if (locked) {
LOG.info("Table status lock has been successfully acquired.");
-
+ // Again read status and check to verify updation required or not.
+ ReturnTuple tuple2 = isUpdationRequired(isForceDeletion, carbonTable, identifier);
+ if (!tuple2.isUpdateRequired) {
+ return;
+ }
// read latest table status again.
LoadMetadataDetails[] latestMetadata =
SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
// update the metadata details from old to new status.
List<LoadMetadataDetails> latestStatus =
- updateLoadMetadataFromOldToNew(details, latestMetadata);
+ updateLoadMetadataFromOldToNew(tuple2.details, latestMetadata);
writeLoadMetadata(identifier, latestStatus);
} else {
@@ -881,14 +898,13 @@ public class SegmentStatusManager {
"running in the background.";
LOG.audit(errorMsg);
LOG.error(errorMsg);
- throw new TableStatusLockException(errorMsg + " Please try after some time.");
+ throw new IOException(errorMsg + " Please try after some time.");
}
- updationCompletionStatus = true;
+ DeleteLoadFolders.physicalFactAndMeasureMetadataDeletion(
+ identifier, carbonTable.getMetadataPath(), isForceDeletion, partitionSpecs);
} finally {
- CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.TABLE_STATUS_LOCK);
- if (updationCompletionStatus) {
- DeleteLoadFolders.physicalFactAndMeasureMetadataDeletion(
- identifier, carbonTable.getMetadataPath(), isForceDeletion, partitionSpecs);
+ if (locked) {
+ CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.TABLE_STATUS_LOCK);
}
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/04ff3676/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index d3093fb..77ff139 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -815,7 +815,7 @@ object CommonUtil {
try {
val carbonTable = CarbonMetadata.getInstance
.getCarbonTable(identifier.getCarbonTableIdentifier.getTableUniqueName)
- SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable, true)
+ SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable, true, null)
} catch {
case _: Exception =>
LOGGER.warn(s"Error while cleaning table " +
http://git-wip-us.apache.org/repos/asf/carbondata/blob/04ff3676/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
index b64c91e..d89aa5b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
@@ -26,12 +26,13 @@ import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.command.management.CarbonLoadDataCommand
import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil
+import org.apache.spark.sql.optimizer.CarbonFilters
import org.apache.spark.sql.parser.CarbonSpark2SqlParser
import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
import org.apache.carbondata.core.constants.CarbonCommonConstants
import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
/**
* Below helper class will be used to create pre-aggregate table
@@ -164,7 +165,15 @@ case class PreAggregateTableHelper(
// This will be used to check if the parent table has any segments or not. If not then no
// need to fire load for pre-aggregate table. Therefore reading the load details for PARENT
// table.
- SegmentStatusManager.deleteLoadsAndUpdateMetadata(parentTable, false)
+ SegmentStatusManager.deleteLoadsAndUpdateMetadata(
+ parentTable,
+ false,
+ CarbonFilters.getCurrentPartitions(
+ sparkSession,
+ TableIdentifier(parentTable.getTableName,
+ Some(parentTable.getDatabaseName))
+ ).map(_.asJava).orNull)
+
if (SegmentStatusManager.isLoadInProgressInTable(parentTable)) {
throw new UnsupportedOperationException(
"Cannot create pre-aggregate table when insert is in progress on main table")
[10/10] carbondata git commit: [CARBONDATA-2224][File Level Reader
Support] Refactoring of #2055
Posted by ja...@apache.org.
[CARBONDATA-2224][File Level Reader Support] Refactoring of #2055
Review comment fixes and refactoring of #2055
This closes #2069
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/b384b6e1
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/b384b6e1
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/b384b6e1
Branch: refs/heads/carbonfile
Commit: b384b6e1fe54fe6e48bae40948e3e41721413c0f
Parents: 223c25d
Author: Ajantha-Bhat <aj...@gmail.com>
Authored: Fri Mar 16 16:06:04 2018 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Mar 20 23:44:00 2018 +0800
----------------------------------------------------------------------
.../core/metadata/schema/table/CarbonTable.java | 4 +-
.../apache/carbondata/core/util/CarbonUtil.java | 2 +-
.../carbondata/examples/HadoopFileExample.scala | 8 +-
.../hadoop/api/CarbonFileInputFormat.java | 535 ++-----------------
.../hadoop/api/CarbonInputFormat.java | 534 ++++++++++++++++++
.../hadoop/api/CarbonTableInputFormat.java | 463 +---------------
.../carbondata/hadoop/util/SchemaReader.java | 8 +-
...FileInputFormatWithExternalCarbonTable.scala | 16 +-
...tCreateTableUsingSparkCarbonFileFormat.scala | 25 +-
...tSparkCarbonFileFormatWithSparkSession.scala | 4 +-
.../spark/rdd/CarbonIUDMergerRDD.scala | 6 +-
.../carbondata/spark/rdd/CarbonMergerRDD.scala | 6 +-
.../spark/rdd/CarbonScanPartitionRDD.scala | 4 +-
.../carbondata/spark/rdd/CarbonScanRDD.scala | 60 +--
.../org/apache/spark/util/PartitionUtils.scala | 4 +-
.../org/apache/spark/sql/CarbonCountStar.scala | 4 +-
.../command/mutation/DeleteExecution.scala | 4 +-
.../command/table/CarbonDropTableCommand.scala | 2 +-
.../datasources/SparkCarbonFileFormat.scala | 15 +-
.../sql/execution/strategy/DDLStrategy.scala | 6 +-
.../spark/sql/parser/CarbonSparkSqlParser.scala | 8 +-
.../spark/sql/hive/CarbonSessionState.scala | 2 +-
.../spark/sql/hive/CarbonSessionState.scala | 2 +-
.../carbondata/streaming/StreamHandoffRDD.scala | 12 +-
24 files changed, 669 insertions(+), 1065 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 278dc96..9e0d80a 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -826,8 +826,8 @@ public class CarbonTable implements Serializable {
return external != null && external.equalsIgnoreCase("true");
}
- public boolean isFileLevelExternalTable() {
- String external = tableInfo.getFactTable().getTableProperties().get("_filelevelexternal");
+ public boolean isFileLevelFormat() {
+ String external = tableInfo.getFactTable().getTableProperties().get("_filelevelformat");
return external != null && external.equalsIgnoreCase("true");
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 5a5f65d..c52680a 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -2213,7 +2213,7 @@ public final class CarbonUtil {
* @param schemaFilePath
* @return
*/
- public static org.apache.carbondata.format.TableInfo inferSchemaFileExternalTable(
+ public static org.apache.carbondata.format.TableInfo inferSchema(
String carbonDataFilePath, AbsoluteTableIdentifier absoluteTableIdentifier,
boolean schemaExists) throws IOException {
TBaseCreator createTBase = new ThriftReader.TBaseCreator() {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/examples/spark2/src/main/scala/org/apache/carbondata/examples/HadoopFileExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/HadoopFileExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/HadoopFileExample.scala
index d75abc2..465e660 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/HadoopFileExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/HadoopFileExample.scala
@@ -19,7 +19,7 @@ package org.apache.carbondata.examples
import org.apache.hadoop.conf.Configuration
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
import org.apache.carbondata.hadoop.CarbonProjection
// scalastyle:off println
@@ -34,9 +34,9 @@ object HadoopFileExample {
projection.addColumn("c1") // column c1
projection.addColumn("c3") // column c3
val conf = new Configuration()
- CarbonTableInputFormat.setColumnProjection(conf, projection)
- CarbonTableInputFormat.setDatabaseName(conf, "default")
- CarbonTableInputFormat.setTableName(conf, "carbon1")
+ CarbonInputFormat.setColumnProjection(conf, projection)
+ CarbonInputFormat.setDatabaseName(conf, "default")
+ CarbonInputFormat.setTableName(conf, "carbon1")
val sc = spark.sparkContext
val input = sc.newAPIHadoopFile(s"${ExampleUtils.storeLocation}/default/carbon1",
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
index b86b1cc..ff532b7 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
@@ -17,11 +17,8 @@
package org.apache.carbondata.hadoop.api;
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
import java.io.IOException;
import java.io.Serializable;
-import java.lang.reflect.Constructor;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.LinkedList;
@@ -30,21 +27,11 @@ import java.util.Map;
import org.apache.carbondata.common.annotations.InterfaceAudience;
import org.apache.carbondata.common.annotations.InterfaceStability;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapChooser;
-import org.apache.carbondata.core.datamap.DataMapLevel;
import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.exception.InvalidConfigurationException;
-import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
import org.apache.carbondata.core.metadata.schema.table.TableInfo;
import org.apache.carbondata.core.mutate.UpdateVO;
@@ -52,241 +39,58 @@ import org.apache.carbondata.core.scan.expression.Expression;
import org.apache.carbondata.core.scan.filter.SingleTableProvider;
import org.apache.carbondata.core.scan.filter.TableProvider;
import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.stats.QueryStatistic;
-import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.stats.QueryStatisticsRecorder;
import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeConverter;
-import org.apache.carbondata.core.util.DataTypeConverterImpl;
import org.apache.carbondata.core.util.path.CarbonTablePath;
import org.apache.carbondata.hadoop.CarbonInputSplit;
-import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
-import org.apache.carbondata.hadoop.CarbonProjection;
-import org.apache.carbondata.hadoop.CarbonRecordReader;
-import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
-import org.apache.carbondata.hadoop.readsupport.impl.DictionaryDecodeReadSupport;
import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
-import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
import org.apache.carbondata.hadoop.util.SchemaReader;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.security.TokenCache;
/**
- * Input format of CarbonData file.
+ * InputFormat for reading carbondata files without table level metadata support,
+ * schema is inferred as following steps:
+ * 1. read from schema file is exists
+ * 2. read from data file footer
*
* @param <T>
*/
@InterfaceAudience.User
@InterfaceStability.Evolving
-public class CarbonFileInputFormat<T> extends FileInputFormat<Void, T> implements Serializable {
-
- public static final String READ_SUPPORT_CLASS = "carbon.read.support.class";
- // comma separated list of input segment numbers
- public static final String INPUT_SEGMENT_NUMBERS =
- "mapreduce.input.carboninputformat.segmentnumbers";
- private static final String VALIDATE_INPUT_SEGMENT_IDs =
- "mapreduce.input.carboninputformat.validsegments";
- // comma separated list of input files
- public static final String INPUT_FILES = "mapreduce.input.carboninputformat.files";
- private static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
- private static final Log LOG = LogFactory.getLog(CarbonFileInputFormat.class);
- private static final String FILTER_PREDICATE =
- "mapreduce.input.carboninputformat.filter.predicate";
- private static final String COLUMN_PROJECTION = "mapreduce.input.carboninputformat.projection";
- private static final String TABLE_INFO = "mapreduce.input.carboninputformat.tableinfo";
- private static final String CARBON_READ_SUPPORT = "mapreduce.input.carboninputformat.readsupport";
- private static final String CARBON_CONVERTER = "mapreduce.input.carboninputformat.converter";
- private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr";
- public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
- public static final String TABLE_NAME = "mapreduce.input.carboninputformat.tableName";
- private static final String PARTITIONS_TO_PRUNE =
- "mapreduce.input.carboninputformat.partitions.to.prune";
- public static final String UPADTE_T =
- "mapreduce.input.carboninputformat.partitions.to.prune";
+public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Serializable {
// a cache for carbon table, it will be used in task side
private CarbonTable carbonTable;
- /**
- * Set the `tableInfo` in `configuration`
- */
- public static void setTableInfo(Configuration configuration, TableInfo tableInfo)
- throws IOException {
- if (null != tableInfo) {
- configuration.set(TABLE_INFO, CarbonUtil.encodeToString(tableInfo.serialize()));
- }
- }
-
- /**
- * Get TableInfo object from `configuration`
- */
- private static TableInfo getTableInfo(Configuration configuration) throws IOException {
- String tableInfoStr = configuration.get(TABLE_INFO);
- if (tableInfoStr == null) {
- return null;
+ protected CarbonTable getOrCreateCarbonTable(Configuration configuration) throws IOException {
+ CarbonTable carbonTableTemp;
+ if (carbonTable == null) {
+ // carbon table should be created either from deserialized table info (schema saved in
+ // hive metastore) or by reading schema in HDFS (schema saved in HDFS)
+ TableInfo tableInfo = getTableInfo(configuration);
+ CarbonTable localCarbonTable;
+ if (tableInfo != null) {
+ localCarbonTable = CarbonTable.buildFromTableInfo(tableInfo);
+ } else {
+ String schemaPath = CarbonTablePath
+ .getSchemaFilePath(getAbsoluteTableIdentifier(configuration).getTablePath());
+ if (!FileFactory.isFileExist(schemaPath, FileFactory.getFileType(schemaPath))) {
+ TableInfo tableInfoInfer =
+ SchemaReader.inferSchema(getAbsoluteTableIdentifier(configuration));
+ localCarbonTable = CarbonTable.buildFromTableInfo(tableInfoInfer);
+ } else {
+ localCarbonTable =
+ SchemaReader.readCarbonTableFromStore(getAbsoluteTableIdentifier(configuration));
+ }
+ }
+ this.carbonTable = localCarbonTable;
+ return localCarbonTable;
} else {
- TableInfo output = new TableInfo();
- output.readFields(
- new DataInputStream(
- new ByteArrayInputStream(CarbonUtil.decodeStringToBytes(tableInfoStr))));
- return output;
- }
- }
-
-
- public static void setTablePath(Configuration configuration, String tablePath) {
- configuration.set(FileInputFormat.INPUT_DIR, tablePath);
- }
-
- public static void setPartitionIdList(Configuration configuration, List<String> partitionIds) {
- configuration.set(ALTER_PARTITION_ID, partitionIds.toString());
- }
-
-
- public static void setDataMapJob(Configuration configuration, DataMapJob dataMapJob)
- throws IOException {
- if (dataMapJob != null) {
- String toString = ObjectSerializationUtil.convertObjectToString(dataMapJob);
- configuration.set(DATA_MAP_DSTR, toString);
- }
- }
-
- public static DataMapJob getDataMapJob(Configuration configuration) throws IOException {
- String jobString = configuration.get(DATA_MAP_DSTR);
- if (jobString != null) {
- return (DataMapJob) ObjectSerializationUtil.convertStringToObject(jobString);
- }
- return null;
- }
-
- /**
- * It sets unresolved filter expression.
- *
- * @param configuration
- * @param filterExpression
- */
- public static void setFilterPredicates(Configuration configuration, Expression filterExpression) {
- if (filterExpression == null) {
- return;
- }
- try {
- String filterString = ObjectSerializationUtil.convertObjectToString(filterExpression);
- configuration.set(FILTER_PREDICATE, filterString);
- } catch (Exception e) {
- throw new RuntimeException("Error while setting filter expression to Job", e);
- }
- }
-
- public static void setColumnProjection(Configuration configuration, CarbonProjection projection) {
- if (projection == null || projection.isEmpty()) {
- return;
- }
- String[] allColumns = projection.getAllColumns();
- StringBuilder builder = new StringBuilder();
- for (String column : allColumns) {
- builder.append(column).append(",");
- }
- String columnString = builder.toString();
- columnString = columnString.substring(0, columnString.length() - 1);
- configuration.set(COLUMN_PROJECTION, columnString);
- }
-
- public static String getColumnProjection(Configuration configuration) {
- return configuration.get(COLUMN_PROJECTION);
- }
-
-
- /**
- * Set list of segments to access
- */
- public static void setSegmentsToAccess(Configuration configuration, List<Segment> validSegments) {
- configuration.set(INPUT_SEGMENT_NUMBERS, CarbonUtil.convertToString(validSegments));
- }
-
- /**
- * Set `CARBON_INPUT_SEGMENTS` from property to configuration
- */
- public static void setQuerySegment(Configuration conf, AbsoluteTableIdentifier identifier) {
- String dbName = identifier.getCarbonTableIdentifier().getDatabaseName().toLowerCase();
- String tbName = identifier.getCarbonTableIdentifier().getTableName().toLowerCase();
- String segmentNumbersFromProperty = CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS + dbName + "." + tbName, "*");
- if (!segmentNumbersFromProperty.trim().equals("*")) {
- CarbonFileInputFormat
- .setSegmentsToAccess(conf, Segment.toSegmentList(segmentNumbersFromProperty.split(",")));
- }
- }
-
- /**
- * set list of segment to access
- */
- public static void setValidateSegmentsToAccess(Configuration configuration, Boolean validate) {
- configuration.set(CarbonFileInputFormat.VALIDATE_INPUT_SEGMENT_IDs, validate.toString());
- }
-
- /**
- * get list of segment to access
- */
- public static boolean getValidateSegmentsToAccess(Configuration configuration) {
- return configuration.get(CarbonFileInputFormat.VALIDATE_INPUT_SEGMENT_IDs, "true")
- .equalsIgnoreCase("true");
- }
-
- /**
- * set list of partitions to prune
- */
- public static void setPartitionsToPrune(Configuration configuration,
- List<PartitionSpec> partitions) {
- if (partitions == null) {
- return;
- }
- try {
- String partitionString =
- ObjectSerializationUtil.convertObjectToString(new ArrayList<>(partitions));
- configuration.set(PARTITIONS_TO_PRUNE, partitionString);
- } catch (Exception e) {
- throw new RuntimeException("Error while setting patition information to Job", e);
- }
- }
-
- /**
- * get list of partitions to prune
- */
- private static List<PartitionSpec> getPartitionsToPrune(Configuration configuration)
- throws IOException {
- String partitionString = configuration.get(PARTITIONS_TO_PRUNE);
- if (partitionString != null) {
- return (List<PartitionSpec>) ObjectSerializationUtil.convertStringToObject(partitionString);
- }
- return null;
- }
-
- public AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration)
- throws IOException {
- String tablePath = configuration.get(INPUT_DIR, "");
- try {
- return AbsoluteTableIdentifier
- .from(tablePath, getDatabaseName(configuration), getTableName(configuration));
- } catch (InvalidConfigurationException e) {
- throw new IOException(e);
+ carbonTableTemp = this.carbonTable;
+ return carbonTableTemp;
}
}
@@ -306,8 +110,6 @@ public class CarbonFileInputFormat<T> extends FileInputFormat<Void, T> implement
if (null == carbonTable) {
throw new IOException("Missing/Corrupt schema file for table.");
}
- // TableDataMap blockletMap = DataMapStoreManager.getInstance()
- // .getDataMap(identifier, BlockletDataMap.NAME, BlockletDataMapFactory.class.getName());
if (getValidateSegmentsToAccess(job.getConfiguration())) {
// get all valid segments and set them into the configuration
@@ -346,8 +148,6 @@ public class CarbonFileInputFormat<T> extends FileInputFormat<Void, T> implement
return null;
}
-
-
/**
* {@inheritDoc}
* Configurations FileInputFormat.INPUT_DIR, CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS
@@ -404,279 +204,4 @@ public class CarbonFileInputFormat<T> extends FileInputFormat<Void, T> implement
}
return result;
}
-
- protected Expression getFilterPredicates(Configuration configuration) {
- try {
- String filterExprString = configuration.get(FILTER_PREDICATE);
- if (filterExprString == null) {
- return null;
- }
- Object filter = ObjectSerializationUtil.convertStringToObject(filterExprString);
- return (Expression) filter;
- } catch (IOException e) {
- throw new RuntimeException("Error while reading filter expression", e);
- }
- }
-
- /**
- * get data blocks of given segment
- */
- private List<CarbonInputSplit> getDataBlocksOfSegment(JobContext job,
- AbsoluteTableIdentifier absoluteTableIdentifier, FilterResolverIntf resolver,
- BitSet matchedPartitions, List<Segment> segmentIds, PartitionInfo partitionInfo,
- List<Integer> oldPartitionIdList) throws IOException {
-
- QueryStatisticsRecorder recorder = CarbonTimeStatisticsFactory.createDriverRecorder();
- QueryStatistic statistic = new QueryStatistic();
-
- // get tokens for all the required FileSystem for table path
- TokenCache.obtainTokensForNamenodes(job.getCredentials(),
- new Path[] { new Path(absoluteTableIdentifier.getTablePath()) }, job.getConfiguration());
- boolean distributedCG = Boolean.parseBoolean(CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
- CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT));
- DataMapExprWrapper dataMapExprWrapper =
- DataMapChooser.get().choose(getOrCreateCarbonTable(job.getConfiguration()), resolver);
- DataMapJob dataMapJob = getDataMapJob(job.getConfiguration());
- List<PartitionSpec> partitionsToPrune = getPartitionsToPrune(job.getConfiguration());
- List<ExtendedBlocklet> prunedBlocklets;
- if (distributedCG || dataMapExprWrapper.getDataMapType() == DataMapLevel.FG) {
- DistributableDataMapFormat datamapDstr =
- new DistributableDataMapFormat(absoluteTableIdentifier, dataMapExprWrapper,
- segmentIds, partitionsToPrune,
- BlockletDataMapFactory.class.getName());
- prunedBlocklets = dataMapJob.execute(datamapDstr, resolver);
- // Apply expression on the blocklets.
- prunedBlocklets = dataMapExprWrapper.pruneBlocklets(prunedBlocklets);
- } else {
- prunedBlocklets = dataMapExprWrapper.prune(segmentIds, partitionsToPrune);
- }
-
- List<CarbonInputSplit> resultFilterredBlocks = new ArrayList<>();
- int partitionIndex = 0;
- List<Integer> partitionIdList = new ArrayList<>();
- if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
- partitionIdList = partitionInfo.getPartitionIds();
- }
- for (ExtendedBlocklet blocklet : prunedBlocklets) {
- long partitionId = CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(
- CarbonTablePath.DataFileUtil.getTaskNo(blocklet.getPath()));
-
- // OldPartitionIdList is only used in alter table partition command because it change
- // partition info first and then read data.
- // For other normal query should use newest partitionIdList
- if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
- if (oldPartitionIdList != null) {
- partitionIndex = oldPartitionIdList.indexOf((int)partitionId);
- } else {
- partitionIndex = partitionIdList.indexOf((int)partitionId);
- }
- }
- if (partitionIndex != -1) {
- // matchedPartitions variable will be null in two cases as follows
- // 1. the table is not a partition table
- // 2. the table is a partition table, and all partitions are matched by query
- // for partition table, the task id of carbaondata file name is the partition id.
- // if this partition is not required, here will skip it.
- if (matchedPartitions == null || matchedPartitions.get(partitionIndex)) {
- CarbonInputSplit inputSplit = convertToCarbonInputSplit(blocklet);
- if (inputSplit != null) {
- resultFilterredBlocks.add(inputSplit);
- }
- }
- }
- }
- statistic
- .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER, System.currentTimeMillis());
- recorder.recordStatisticsForDriver(statistic, job.getConfiguration().get("query.id"));
- return resultFilterredBlocks;
- }
-
- private CarbonInputSplit convertToCarbonInputSplit(ExtendedBlocklet blocklet) throws IOException {
- CarbonInputSplit split =
- CarbonInputSplit.from(blocklet.getSegmentId(),
- blocklet.getBlockletId(), new FileSplit(new Path(blocklet.getPath()), 0,
- blocklet.getLength(), blocklet.getLocations()),
- ColumnarFormatVersion.valueOf((short) blocklet.getDetailInfo().getVersionNumber()),
- blocklet.getDataMapWriterPath());
- split.setDetailInfo(blocklet.getDetailInfo());
- return split;
- }
-
- @Override
- public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
- TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
- Configuration configuration = taskAttemptContext.getConfiguration();
- QueryModel queryModel = createQueryModel(inputSplit, taskAttemptContext);
- CarbonReadSupport<T> readSupport = getReadSupportClass(configuration);
- return new CarbonRecordReader<T>(queryModel, readSupport);
- }
-
- public QueryModel createQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
- throws IOException {
- Configuration configuration = taskAttemptContext.getConfiguration();
- CarbonTable carbonTable = getOrCreateCarbonTable(configuration);
- TableProvider tableProvider = new SingleTableProvider(carbonTable);
-
- // query plan includes projection column
- String projectionString = getColumnProjection(configuration);
- String[] projectionColumnNames = null;
- if (projectionString != null) {
- projectionColumnNames = projectionString.split(",");
- }
- QueryModel queryModel = carbonTable.createQueryWithProjection(
- projectionColumnNames, getDataTypeConverter(configuration));
-
- // set the filter to the query model in order to filter blocklet before scan
- Expression filter = getFilterPredicates(configuration);
- boolean[] isFilterDimensions = new boolean[carbonTable.getDimensionOrdinalMax()];
- // getAllMeasures returns list of visible and invisible columns
- boolean[] isFilterMeasures =
- new boolean[carbonTable.getAllMeasures().size()];
- CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, isFilterDimensions,
- isFilterMeasures);
- queryModel.setIsFilterDimensions(isFilterDimensions);
- queryModel.setIsFilterMeasures(isFilterMeasures);
- FilterResolverIntf filterIntf = CarbonInputFormatUtil
- .resolveFilter(filter, carbonTable.getAbsoluteTableIdentifier(), tableProvider);
- queryModel.setFilterExpressionResolverTree(filterIntf);
-
- // update the file level index store if there are invalid segment
- if (inputSplit instanceof CarbonMultiBlockSplit) {
- CarbonMultiBlockSplit split = (CarbonMultiBlockSplit) inputSplit;
- List<String> invalidSegments = split.getAllSplits().get(0).getInvalidSegments();
- if (invalidSegments.size() > 0) {
- queryModel.setInvalidSegmentIds(invalidSegments);
- }
- List<UpdateVO> invalidTimestampRangeList =
- split.getAllSplits().get(0).getInvalidTimestampRange();
- if ((null != invalidTimestampRangeList) && (invalidTimestampRangeList.size() > 0)) {
- queryModel.setInvalidBlockForSegmentId(invalidTimestampRangeList);
- }
- }
- return queryModel;
- }
-
- private CarbonTable getOrCreateCarbonTable(Configuration configuration) throws IOException {
- CarbonTable carbonTableTemp;
- if (carbonTable == null) {
- // carbon table should be created either from deserialized table info (schema saved in
- // hive metastore) or by reading schema in HDFS (schema saved in HDFS)
- TableInfo tableInfo = getTableInfo(configuration);
- CarbonTable localCarbonTable;
- if (tableInfo != null) {
- localCarbonTable = CarbonTable.buildFromTableInfo(tableInfo);
- } else {
- String schemaPath = CarbonTablePath
- .getSchemaFilePath(getAbsoluteTableIdentifier(configuration).getTablePath());
- if (!FileFactory.isFileExist(schemaPath, FileFactory.getFileType(schemaPath))) {
- TableInfo tableInfoInfer =
- SchemaReader.inferSchema(getAbsoluteTableIdentifier(configuration));
- localCarbonTable = CarbonTable.buildFromTableInfo(tableInfoInfer);
- } else {
- localCarbonTable =
- SchemaReader.readCarbonTableFromStore(getAbsoluteTableIdentifier(configuration));
- }
- }
- this.carbonTable = localCarbonTable;
- return localCarbonTable;
- } else {
- carbonTableTemp = this.carbonTable;
- return carbonTableTemp;
- }
- }
-
-
- public CarbonReadSupport<T> getReadSupportClass(Configuration configuration) {
- String readSupportClass = configuration.get(CARBON_READ_SUPPORT);
- //By default it uses dictionary decoder read class
- CarbonReadSupport<T> readSupport = null;
- if (readSupportClass != null) {
- try {
- Class<?> myClass = Class.forName(readSupportClass);
- Constructor<?> constructor = myClass.getConstructors()[0];
- Object object = constructor.newInstance();
- if (object instanceof CarbonReadSupport) {
- readSupport = (CarbonReadSupport) object;
- }
- } catch (ClassNotFoundException ex) {
- LOG.error("Class " + readSupportClass + "not found", ex);
- } catch (Exception ex) {
- LOG.error("Error while creating " + readSupportClass, ex);
- }
- } else {
- readSupport = new DictionaryDecodeReadSupport<>();
- }
- return readSupport;
- }
-
- @Override
- protected boolean isSplitable(JobContext context, Path filename) {
- try {
- // Don't split the file if it is local file system
- FileSystem fileSystem = filename.getFileSystem(context.getConfiguration());
- if (fileSystem instanceof LocalFileSystem) {
- return false;
- }
- } catch (Exception e) {
- return true;
- }
- return true;
- }
-
- /**
- * return valid segment to access
- */
- private String[] getSegmentsToAccess(JobContext job) {
- String segmentString = job.getConfiguration().get(INPUT_SEGMENT_NUMBERS, "");
- if (segmentString.trim().isEmpty()) {
- return new String[0];
- }
- return segmentString.split(",");
- }
-
- public static DataTypeConverter getDataTypeConverter(Configuration configuration)
- throws IOException {
- String converter = configuration.get(CARBON_CONVERTER);
- if (converter == null) {
- return new DataTypeConverterImpl();
- }
- return (DataTypeConverter) ObjectSerializationUtil.convertStringToObject(converter);
- }
-
- public static void setDatabaseName(Configuration configuration, String databaseName) {
- if (null != databaseName) {
- configuration.set(DATABASE_NAME, databaseName);
- }
- }
-
- public static String getDatabaseName(Configuration configuration)
- throws InvalidConfigurationException {
- String databseName = configuration.get(DATABASE_NAME);
- if (null == databseName) {
- throw new InvalidConfigurationException("Database name is not set.");
- }
- return databseName;
- }
-
- public static void setTableName(Configuration configuration, String tableName) {
- if (null != tableName) {
- configuration.set(TABLE_NAME, tableName);
- }
- }
-
- public static String getTableName(Configuration configuration)
- throws InvalidConfigurationException {
- String tableName = configuration.get(TABLE_NAME);
- if (tableName == null) {
- throw new InvalidConfigurationException("Table name is not set");
- }
- return tableName;
- }
-
- public org.apache.hadoop.mapred.RecordReader<Void, T> getRecordReader(
- org.apache.hadoop.mapred.InputSplit split, JobConf job, Reporter reporter)
- throws IOException {
- return null;
- }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
new file mode 100644
index 0000000..3cc9c5f
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
@@ -0,0 +1,534 @@
+/*
+ * 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.carbondata.hadoop.api;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datamap.DataMapChooser;
+import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
+import org.apache.carbondata.core.exception.InvalidConfigurationException;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.metadata.schema.PartitionInfo;
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.mutate.UpdateVO;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.filter.SingleTableProvider;
+import org.apache.carbondata.core.scan.filter.TableProvider;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.stats.QueryStatistic;
+import org.apache.carbondata.core.stats.QueryStatisticsConstants;
+import org.apache.carbondata.core.stats.QueryStatisticsRecorder;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeConverter;
+import org.apache.carbondata.core.util.DataTypeConverterImpl;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+import org.apache.carbondata.hadoop.CarbonProjection;
+import org.apache.carbondata.hadoop.CarbonRecordReader;
+import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
+import org.apache.carbondata.hadoop.readsupport.impl.DictionaryDecodeReadSupport;
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
+import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.security.TokenCache;
+
+/**
+ * Base class for carbondata input format, there are two input format implementations:
+ * 1. CarbonFileInputFormat: for reading carbondata files without table level metadata support.
+ *
+ * 2. CarbonTableInputFormat: for reading carbondata files with table level metadata support,
+ * such as segment and explicit schema metadata.
+ *
+ * @param <T>
+ */
+public abstract class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
+ // comma separated list of input segment numbers
+ public static final String INPUT_SEGMENT_NUMBERS =
+ "mapreduce.input.carboninputformat.segmentnumbers";
+ private static final String VALIDATE_INPUT_SEGMENT_IDs =
+ "mapreduce.input.carboninputformat.validsegments";
+ // comma separated list of input files
+ private static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
+ private static final Log LOG = LogFactory.getLog(CarbonInputFormat.class);
+ private static final String FILTER_PREDICATE =
+ "mapreduce.input.carboninputformat.filter.predicate";
+ private static final String COLUMN_PROJECTION = "mapreduce.input.carboninputformat.projection";
+ private static final String TABLE_INFO = "mapreduce.input.carboninputformat.tableinfo";
+ private static final String CARBON_READ_SUPPORT = "mapreduce.input.carboninputformat.readsupport";
+ private static final String CARBON_CONVERTER = "mapreduce.input.carboninputformat.converter";
+ private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr";
+ public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
+ public static final String TABLE_NAME = "mapreduce.input.carboninputformat.tableName";
+ private static final String PARTITIONS_TO_PRUNE =
+ "mapreduce.input.carboninputformat.partitions.to.prune";
+
+ /**
+ * Set the `tableInfo` in `configuration`
+ */
+ public static void setTableInfo(Configuration configuration, TableInfo tableInfo)
+ throws IOException {
+ if (null != tableInfo) {
+ configuration.set(TABLE_INFO, CarbonUtil.encodeToString(tableInfo.serialize()));
+ }
+ }
+
+ /**
+ * Get TableInfo object from `configuration`
+ */
+ protected static TableInfo getTableInfo(Configuration configuration) throws IOException {
+ String tableInfoStr = configuration.get(TABLE_INFO);
+ if (tableInfoStr == null) {
+ return null;
+ } else {
+ TableInfo output = new TableInfo();
+ output.readFields(new DataInputStream(
+ new ByteArrayInputStream(CarbonUtil.decodeStringToBytes(tableInfoStr))));
+ return output;
+ }
+ }
+
+ /**
+ * Get the cached CarbonTable or create it by TableInfo in `configuration`
+ */
+ protected abstract CarbonTable getOrCreateCarbonTable(Configuration configuration)
+ throws IOException;
+
+ public static void setTablePath(Configuration configuration, String tablePath) {
+ configuration.set(FileInputFormat.INPUT_DIR, tablePath);
+ }
+
+ public static void setPartitionIdList(Configuration configuration, List<String> partitionIds) {
+ configuration.set(ALTER_PARTITION_ID, partitionIds.toString());
+ }
+
+ public static void setDataMapJob(Configuration configuration, DataMapJob dataMapJob)
+ throws IOException {
+ if (dataMapJob != null) {
+ String toString = ObjectSerializationUtil.convertObjectToString(dataMapJob);
+ configuration.set(DATA_MAP_DSTR, toString);
+ }
+ }
+
+ public static DataMapJob getDataMapJob(Configuration configuration) throws IOException {
+ String jobString = configuration.get(DATA_MAP_DSTR);
+ if (jobString != null) {
+ return (DataMapJob) ObjectSerializationUtil.convertStringToObject(jobString);
+ }
+ return null;
+ }
+
+ /**
+ * It sets unresolved filter expression.
+ *
+ * @param configuration
+ * @param filterExpression
+ */
+ public static void setFilterPredicates(Configuration configuration, Expression filterExpression) {
+ if (filterExpression == null) {
+ return;
+ }
+ try {
+ String filterString = ObjectSerializationUtil.convertObjectToString(filterExpression);
+ configuration.set(FILTER_PREDICATE, filterString);
+ } catch (Exception e) {
+ throw new RuntimeException("Error while setting filter expression to Job", e);
+ }
+ }
+
+ public static void setColumnProjection(Configuration configuration, CarbonProjection projection) {
+ if (projection == null || projection.isEmpty()) {
+ return;
+ }
+ String[] allColumns = projection.getAllColumns();
+ StringBuilder builder = new StringBuilder();
+ for (String column : allColumns) {
+ builder.append(column).append(",");
+ }
+ String columnString = builder.toString();
+ columnString = columnString.substring(0, columnString.length() - 1);
+ configuration.set(COLUMN_PROJECTION, columnString);
+ }
+
+ public static String getColumnProjection(Configuration configuration) {
+ return configuration.get(COLUMN_PROJECTION);
+ }
+
+ /**
+ * Set list of segments to access
+ */
+ public static void setSegmentsToAccess(Configuration configuration, List<Segment> validSegments) {
+ configuration.set(INPUT_SEGMENT_NUMBERS, CarbonUtil.convertToString(validSegments));
+ }
+
+ /**
+ * Set `CARBON_INPUT_SEGMENTS` from property to configuration
+ */
+ public static void setQuerySegment(Configuration conf, AbsoluteTableIdentifier identifier) {
+ String dbName = identifier.getCarbonTableIdentifier().getDatabaseName().toLowerCase();
+ String tbName = identifier.getCarbonTableIdentifier().getTableName().toLowerCase();
+ String segmentNumbersFromProperty = CarbonProperties.getInstance()
+ .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS + dbName + "." + tbName, "*");
+ if (!segmentNumbersFromProperty.trim().equals("*")) {
+ CarbonInputFormat
+ .setSegmentsToAccess(conf, Segment.toSegmentList(segmentNumbersFromProperty.split(",")));
+ }
+ }
+
+ /**
+ * set list of segment to access
+ */
+ public static void setValidateSegmentsToAccess(Configuration configuration, Boolean validate) {
+ configuration.set(CarbonInputFormat.VALIDATE_INPUT_SEGMENT_IDs, validate.toString());
+ }
+
+ /**
+ * get list of segment to access
+ */
+ public static boolean getValidateSegmentsToAccess(Configuration configuration) {
+ return configuration.get(CarbonInputFormat.VALIDATE_INPUT_SEGMENT_IDs, "true")
+ .equalsIgnoreCase("true");
+ }
+
+ /**
+ * set list of partitions to prune
+ */
+ public static void setPartitionsToPrune(Configuration configuration,
+ List<PartitionSpec> partitions) {
+ if (partitions == null) {
+ return;
+ }
+ try {
+ String partitionString =
+ ObjectSerializationUtil.convertObjectToString(new ArrayList<>(partitions));
+ configuration.set(PARTITIONS_TO_PRUNE, partitionString);
+ } catch (Exception e) {
+ throw new RuntimeException("Error while setting patition information to Job" + partitions, e);
+ }
+ }
+
+ /**
+ * get list of partitions to prune
+ */
+ public static List<PartitionSpec> getPartitionsToPrune(Configuration configuration)
+ throws IOException {
+ String partitionString = configuration.get(PARTITIONS_TO_PRUNE);
+ if (partitionString != null) {
+ return (List<PartitionSpec>) ObjectSerializationUtil.convertStringToObject(partitionString);
+ }
+ return null;
+ }
+
+ public AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration)
+ throws IOException {
+ String tablePath = configuration.get(INPUT_DIR, "");
+ try {
+ return AbsoluteTableIdentifier
+ .from(tablePath, getDatabaseName(configuration), getTableName(configuration));
+ } catch (InvalidConfigurationException e) {
+ throw new IOException(e);
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ * Configurations FileInputFormat.INPUT_DIR
+ * are used to get table path to read.
+ *
+ * @param job
+ * @return List<InputSplit> list of CarbonInputSplit
+ * @throws IOException
+ */
+ @Override public abstract List<InputSplit> getSplits(JobContext job) throws IOException;
+
+ protected Expression getFilterPredicates(Configuration configuration) {
+ try {
+ String filterExprString = configuration.get(FILTER_PREDICATE);
+ if (filterExprString == null) {
+ return null;
+ }
+ Object filter = ObjectSerializationUtil.convertStringToObject(filterExprString);
+ return (Expression) filter;
+ } catch (IOException e) {
+ throw new RuntimeException("Error while reading filter expression", e);
+ }
+ }
+
+ /**
+ * get data blocks of given segment
+ */
+ protected List<CarbonInputSplit> getDataBlocksOfSegment(JobContext job,
+ AbsoluteTableIdentifier absoluteTableIdentifier, FilterResolverIntf resolver,
+ BitSet matchedPartitions, List<Segment> segmentIds, PartitionInfo partitionInfo,
+ List<Integer> oldPartitionIdList) throws IOException {
+
+ QueryStatisticsRecorder recorder = CarbonTimeStatisticsFactory.createDriverRecorder();
+ QueryStatistic statistic = new QueryStatistic();
+
+ // get tokens for all the required FileSystem for table path
+ TokenCache.obtainTokensForNamenodes(job.getCredentials(),
+ new Path[] { new Path(absoluteTableIdentifier.getTablePath()) }, job.getConfiguration());
+ boolean distributedCG = Boolean.parseBoolean(CarbonProperties.getInstance()
+ .getProperty(CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
+ CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT));
+ DataMapExprWrapper dataMapExprWrapper =
+ DataMapChooser.get().choose(getOrCreateCarbonTable(job.getConfiguration()), resolver);
+ DataMapJob dataMapJob = getDataMapJob(job.getConfiguration());
+ List<PartitionSpec> partitionsToPrune = getPartitionsToPrune(job.getConfiguration());
+ List<ExtendedBlocklet> prunedBlocklets;
+ if (distributedCG || dataMapExprWrapper.getDataMapType() == DataMapLevel.FG) {
+ DistributableDataMapFormat datamapDstr =
+ new DistributableDataMapFormat(absoluteTableIdentifier, dataMapExprWrapper, segmentIds,
+ partitionsToPrune, BlockletDataMapFactory.class.getName());
+ prunedBlocklets = dataMapJob.execute(datamapDstr, resolver);
+ // Apply expression on the blocklets.
+ prunedBlocklets = dataMapExprWrapper.pruneBlocklets(prunedBlocklets);
+ } else {
+ prunedBlocklets = dataMapExprWrapper.prune(segmentIds, partitionsToPrune);
+ }
+
+ List<CarbonInputSplit> resultFilterredBlocks = new ArrayList<>();
+ int partitionIndex = 0;
+ List<Integer> partitionIdList = new ArrayList<>();
+ if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
+ partitionIdList = partitionInfo.getPartitionIds();
+ }
+ for (ExtendedBlocklet blocklet : prunedBlocklets) {
+ long partitionId = CarbonTablePath.DataFileUtil
+ .getTaskIdFromTaskNo(CarbonTablePath.DataFileUtil.getTaskNo(blocklet.getPath()));
+
+ // OldPartitionIdList is only used in alter table partition command because it change
+ // partition info first and then read data.
+ // For other normal query should use newest partitionIdList
+ if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
+ if (oldPartitionIdList != null) {
+ partitionIndex = oldPartitionIdList.indexOf((int) partitionId);
+ } else {
+ partitionIndex = partitionIdList.indexOf((int) partitionId);
+ }
+ }
+ if (partitionIndex != -1) {
+ // matchedPartitions variable will be null in two cases as follows
+ // 1. the table is not a partition table
+ // 2. the table is a partition table, and all partitions are matched by query
+ // for partition table, the task id of carbaondata file name is the partition id.
+ // if this partition is not required, here will skip it.
+ if (matchedPartitions == null || matchedPartitions.get(partitionIndex)) {
+ CarbonInputSplit inputSplit = convertToCarbonInputSplit(blocklet);
+ if (inputSplit != null) {
+ resultFilterredBlocks.add(inputSplit);
+ }
+ }
+ }
+ }
+ statistic
+ .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER, System.currentTimeMillis());
+ recorder.recordStatisticsForDriver(statistic, job.getConfiguration().get("query.id"));
+ return resultFilterredBlocks;
+ }
+
+ private CarbonInputSplit convertToCarbonInputSplit(ExtendedBlocklet blocklet) throws IOException {
+ CarbonInputSplit split = CarbonInputSplit
+ .from(blocklet.getSegmentId(), blocklet.getBlockletId(),
+ new FileSplit(new Path(blocklet.getPath()), 0, blocklet.getLength(),
+ blocklet.getLocations()),
+ ColumnarFormatVersion.valueOf((short) blocklet.getDetailInfo().getVersionNumber()),
+ blocklet.getDataMapWriterPath());
+ split.setDetailInfo(blocklet.getDetailInfo());
+ return split;
+ }
+
+ @Override public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
+ TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
+ Configuration configuration = taskAttemptContext.getConfiguration();
+ QueryModel queryModel = createQueryModel(inputSplit, taskAttemptContext);
+ CarbonReadSupport<T> readSupport = getReadSupportClass(configuration);
+ return new CarbonRecordReader<T>(queryModel, readSupport);
+ }
+
+ public QueryModel createQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+ throws IOException {
+ Configuration configuration = taskAttemptContext.getConfiguration();
+ CarbonTable carbonTable = getOrCreateCarbonTable(configuration);
+ TableProvider tableProvider = new SingleTableProvider(carbonTable);
+
+ // query plan includes projection column
+ String projectionString = getColumnProjection(configuration);
+ String[] projectionColumnNames = null;
+ if (projectionString != null) {
+ projectionColumnNames = projectionString.split(",");
+ }
+ QueryModel queryModel = carbonTable
+ .createQueryWithProjection(projectionColumnNames, getDataTypeConverter(configuration));
+
+ // set the filter to the query model in order to filter blocklet before scan
+ Expression filter = getFilterPredicates(configuration);
+ boolean[] isFilterDimensions = new boolean[carbonTable.getDimensionOrdinalMax()];
+ // getAllMeasures returns list of visible and invisible columns
+ boolean[] isFilterMeasures = new boolean[carbonTable.getAllMeasures().size()];
+ CarbonInputFormatUtil
+ .processFilterExpression(filter, carbonTable, isFilterDimensions, isFilterMeasures);
+ queryModel.setIsFilterDimensions(isFilterDimensions);
+ queryModel.setIsFilterMeasures(isFilterMeasures);
+ FilterResolverIntf filterIntf = CarbonInputFormatUtil
+ .resolveFilter(filter, carbonTable.getAbsoluteTableIdentifier(), tableProvider);
+ queryModel.setFilterExpressionResolverTree(filterIntf);
+
+ // update the file level index store if there are invalid segment
+ if (inputSplit instanceof CarbonMultiBlockSplit) {
+ CarbonMultiBlockSplit split = (CarbonMultiBlockSplit) inputSplit;
+ List<String> invalidSegments = split.getAllSplits().get(0).getInvalidSegments();
+ if (invalidSegments.size() > 0) {
+ queryModel.setInvalidSegmentIds(invalidSegments);
+ }
+ List<UpdateVO> invalidTimestampRangeList =
+ split.getAllSplits().get(0).getInvalidTimestampRange();
+ if ((null != invalidTimestampRangeList) && (invalidTimestampRangeList.size() > 0)) {
+ queryModel.setInvalidBlockForSegmentId(invalidTimestampRangeList);
+ }
+ }
+ return queryModel;
+ }
+
+ public CarbonReadSupport<T> getReadSupportClass(Configuration configuration) {
+ String readSupportClass = configuration.get(CARBON_READ_SUPPORT);
+ //By default it uses dictionary decoder read class
+ CarbonReadSupport<T> readSupport = null;
+ if (readSupportClass != null) {
+ try {
+ Class<?> myClass = Class.forName(readSupportClass);
+ Constructor<?> constructor = myClass.getConstructors()[0];
+ Object object = constructor.newInstance();
+ if (object instanceof CarbonReadSupport) {
+ readSupport = (CarbonReadSupport) object;
+ }
+ } catch (ClassNotFoundException ex) {
+ LOG.error("Class " + readSupportClass + "not found", ex);
+ } catch (Exception ex) {
+ LOG.error("Error while creating " + readSupportClass, ex);
+ }
+ } else {
+ readSupport = new DictionaryDecodeReadSupport<>();
+ }
+ return readSupport;
+ }
+
+ @Override protected boolean isSplitable(JobContext context, Path filename) {
+ try {
+ // Don't split the file if it is local file system
+ FileSystem fileSystem = filename.getFileSystem(context.getConfiguration());
+ if (fileSystem instanceof LocalFileSystem) {
+ return false;
+ }
+ } catch (Exception e) {
+ return true;
+ }
+ return true;
+ }
+
+ public static void setCarbonReadSupport(Configuration configuration,
+ Class<? extends CarbonReadSupport> readSupportClass) {
+ if (readSupportClass != null) {
+ configuration.set(CARBON_READ_SUPPORT, readSupportClass.getName());
+ }
+ }
+
+ /**
+ * It is optional, if user does not set then it reads from store
+ *
+ * @param configuration
+ * @param converter is the Data type converter for different computing engine
+ * @throws IOException
+ */
+ public static void setDataTypeConverter(Configuration configuration, DataTypeConverter converter)
+ throws IOException {
+ if (null != converter) {
+ configuration.set(CARBON_CONVERTER,
+ ObjectSerializationUtil.convertObjectToString(converter));
+ }
+ }
+
+ public static DataTypeConverter getDataTypeConverter(Configuration configuration)
+ throws IOException {
+ String converter = configuration.get(CARBON_CONVERTER);
+ if (converter == null) {
+ return new DataTypeConverterImpl();
+ }
+ return (DataTypeConverter) ObjectSerializationUtil.convertStringToObject(converter);
+ }
+
+ public static void setDatabaseName(Configuration configuration, String databaseName) {
+ if (null != databaseName) {
+ configuration.set(DATABASE_NAME, databaseName);
+ }
+ }
+
+ public static String getDatabaseName(Configuration configuration)
+ throws InvalidConfigurationException {
+ String databseName = configuration.get(DATABASE_NAME);
+ if (null == databseName) {
+ throw new InvalidConfigurationException("Database name is not set.");
+ }
+ return databseName;
+ }
+
+ public static void setTableName(Configuration configuration, String tableName) {
+ if (null != tableName) {
+ configuration.set(TABLE_NAME, tableName);
+ }
+ }
+
+ public static String getTableName(Configuration configuration)
+ throws InvalidConfigurationException {
+ String tableName = configuration.get(TABLE_NAME);
+ if (tableName == null) {
+ throw new InvalidConfigurationException("Table name is not set");
+ }
+ return tableName;
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index bcc487e..efe962d 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -17,11 +17,8 @@
package org.apache.carbondata.hadoop.api;
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
import java.io.File;
import java.io.IOException;
-import java.lang.reflect.Constructor;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.BitSet;
@@ -31,21 +28,14 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Map;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapChooser;
-import org.apache.carbondata.core.datamap.DataMapLevel;
import org.apache.carbondata.core.datamap.DataMapStoreManager;
import org.apache.carbondata.core.datamap.Segment;
import org.apache.carbondata.core.datamap.TableDataMap;
-import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.exception.InvalidConfigurationException;
import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
import org.apache.carbondata.core.indexstore.PartitionSpec;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
import org.apache.carbondata.core.metadata.schema.PartitionInfo;
import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -60,29 +50,15 @@ import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
import org.apache.carbondata.core.scan.filter.SingleTableProvider;
import org.apache.carbondata.core.scan.filter.TableProvider;
import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.stats.QueryStatistic;
-import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.stats.QueryStatisticsRecorder;
import org.apache.carbondata.core.statusmanager.FileFormat;
import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeConverter;
-import org.apache.carbondata.core.util.DataTypeConverterImpl;
import org.apache.carbondata.core.util.path.CarbonTablePath;
import org.apache.carbondata.format.BlockIndex;
import org.apache.carbondata.hadoop.CarbonInputSplit;
-import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
-import org.apache.carbondata.hadoop.CarbonProjection;
-import org.apache.carbondata.hadoop.CarbonRecordReader;
-import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
-import org.apache.carbondata.hadoop.readsupport.impl.DictionaryDecodeReadSupport;
import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
-import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
import org.apache.carbondata.hadoop.util.SchemaReader;
import org.apache.commons.logging.Log;
@@ -91,80 +67,38 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.security.TokenCache;
/**
- * Input format of CarbonData file.
+ * InputFormat for reading carbondata files with table level metadata support,
+ * such as segment and explicit schema metadata.
*
* @param <T>
*/
-public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
+public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
// comma separated list of input segment numbers
public static final String INPUT_SEGMENT_NUMBERS =
"mapreduce.input.carboninputformat.segmentnumbers";
- private static final String VALIDATE_INPUT_SEGMENT_IDs =
- "mapreduce.input.carboninputformat.validsegments";
// comma separated list of input files
public static final String INPUT_FILES = "mapreduce.input.carboninputformat.files";
private static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
private static final Log LOG = LogFactory.getLog(CarbonTableInputFormat.class);
- private static final String FILTER_PREDICATE =
- "mapreduce.input.carboninputformat.filter.predicate";
- private static final String COLUMN_PROJECTION = "mapreduce.input.carboninputformat.projection";
- private static final String TABLE_INFO = "mapreduce.input.carboninputformat.tableinfo";
private static final String CARBON_READ_SUPPORT = "mapreduce.input.carboninputformat.readsupport";
private static final String CARBON_CONVERTER = "mapreduce.input.carboninputformat.converter";
- private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr";
public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
public static final String TABLE_NAME = "mapreduce.input.carboninputformat.tableName";
- private static final String PARTITIONS_TO_PRUNE =
- "mapreduce.input.carboninputformat.partitions.to.prune";
- public static final String UPADTE_T =
- "mapreduce.input.carboninputformat.partitions.to.prune";
-
// a cache for carbon table, it will be used in task side
private CarbonTable carbonTable;
/**
- * Set the `tableInfo` in `configuration`
- */
- public static void setTableInfo(Configuration configuration, TableInfo tableInfo)
- throws IOException {
- if (null != tableInfo) {
- configuration.set(TABLE_INFO, CarbonUtil.encodeToString(tableInfo.serialize()));
- }
- }
-
- /**
- * Get TableInfo object from `configuration`
- */
- private static TableInfo getTableInfo(Configuration configuration) throws IOException {
- String tableInfoStr = configuration.get(TABLE_INFO);
- if (tableInfoStr == null) {
- return null;
- } else {
- TableInfo output = new TableInfo();
- output.readFields(
- new DataInputStream(
- new ByteArrayInputStream(CarbonUtil.decodeStringToBytes(tableInfoStr))));
- return output;
- }
- }
-
- /**
* Get the cached CarbonTable or create it by TableInfo in `configuration`
*/
- private CarbonTable getOrCreateCarbonTable(Configuration configuration) throws IOException {
+ protected CarbonTable getOrCreateCarbonTable(Configuration configuration) throws IOException {
if (carbonTable == null) {
// carbon table should be created either from deserialized table info (schema saved in
// hive metastore) or by reading schema in HDFS (schema saved in HDFS)
@@ -183,150 +117,6 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
}
}
- public static void setTablePath(Configuration configuration, String tablePath) {
- configuration.set(FileInputFormat.INPUT_DIR, tablePath);
- }
-
- public static void setPartitionIdList(Configuration configuration, List<String> partitionIds) {
- configuration.set(ALTER_PARTITION_ID, partitionIds.toString());
- }
-
-
- public static void setDataMapJob(Configuration configuration, DataMapJob dataMapJob)
- throws IOException {
- if (dataMapJob != null) {
- String toString = ObjectSerializationUtil.convertObjectToString(dataMapJob);
- configuration.set(DATA_MAP_DSTR, toString);
- }
- }
-
- private static DataMapJob getDataMapJob(Configuration configuration) throws IOException {
- String jobString = configuration.get(DATA_MAP_DSTR);
- if (jobString != null) {
- return (DataMapJob) ObjectSerializationUtil.convertStringToObject(jobString);
- }
- return null;
- }
-
- /**
- * It sets unresolved filter expression.
- *
- * @param configuration
- * @param filterExpression
- */
- public static void setFilterPredicates(Configuration configuration, Expression filterExpression) {
- if (filterExpression == null) {
- return;
- }
- try {
- String filterString = ObjectSerializationUtil.convertObjectToString(filterExpression);
- configuration.set(FILTER_PREDICATE, filterString);
- } catch (Exception e) {
- throw new RuntimeException("Error while setting filter expression to Job", e);
- }
- }
-
- public static void setColumnProjection(Configuration configuration, CarbonProjection projection) {
- if (projection == null || projection.isEmpty()) {
- return;
- }
- String[] allColumns = projection.getAllColumns();
- StringBuilder builder = new StringBuilder();
- for (String column : allColumns) {
- builder.append(column).append(",");
- }
- String columnString = builder.toString();
- columnString = columnString.substring(0, columnString.length() - 1);
- configuration.set(COLUMN_PROJECTION, columnString);
- }
-
- public static String getColumnProjection(Configuration configuration) {
- return configuration.get(COLUMN_PROJECTION);
- }
-
- public static void setCarbonReadSupport(Configuration configuration,
- Class<? extends CarbonReadSupport> readSupportClass) {
- if (readSupportClass != null) {
- configuration.set(CARBON_READ_SUPPORT, readSupportClass.getName());
- }
- }
-
- /**
- * Set list of segments to access
- */
- public static void setSegmentsToAccess(Configuration configuration, List<Segment> validSegments) {
- configuration.set(INPUT_SEGMENT_NUMBERS, CarbonUtil.convertToString(validSegments));
- }
-
- /**
- * Set `CARBON_INPUT_SEGMENTS` from property to configuration
- */
- public static void setQuerySegment(Configuration conf, AbsoluteTableIdentifier identifier) {
- String dbName = identifier.getCarbonTableIdentifier().getDatabaseName().toLowerCase();
- String tbName = identifier.getCarbonTableIdentifier().getTableName().toLowerCase();
- String segmentNumbersFromProperty = CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS + dbName + "." + tbName, "*");
- if (!segmentNumbersFromProperty.trim().equals("*")) {
- CarbonTableInputFormat
- .setSegmentsToAccess(conf, Segment.toSegmentList(segmentNumbersFromProperty.split(",")));
- }
- }
-
- /**
- * set list of segment to access
- */
- public static void setValidateSegmentsToAccess(Configuration configuration, Boolean validate) {
- configuration.set(CarbonTableInputFormat.VALIDATE_INPUT_SEGMENT_IDs, validate.toString());
- }
-
- /**
- * get list of segment to access
- */
- public static boolean getValidateSegmentsToAccess(Configuration configuration) {
- return configuration.get(CarbonTableInputFormat.VALIDATE_INPUT_SEGMENT_IDs, "true")
- .equalsIgnoreCase("true");
- }
-
- /**
- * set list of partitions to prune
- */
- public static void setPartitionsToPrune(Configuration configuration,
- List<PartitionSpec> partitions) {
- if (partitions == null) {
- return;
- }
- try {
- String partitionString =
- ObjectSerializationUtil.convertObjectToString(new ArrayList<>(partitions));
- configuration.set(PARTITIONS_TO_PRUNE, partitionString);
- } catch (Exception e) {
- throw new RuntimeException("Error while setting patition information to Job" + partitions, e);
- }
- }
-
- /**
- * get list of partitions to prune
- */
- public static List<PartitionSpec> getPartitionsToPrune(Configuration configuration)
- throws IOException {
- String partitionString = configuration.get(PARTITIONS_TO_PRUNE);
- if (partitionString != null) {
- return (List<PartitionSpec>) ObjectSerializationUtil.convertStringToObject(partitionString);
- }
- return null;
- }
-
- public AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration)
- throws IOException {
- String tablePath = configuration.get(INPUT_DIR, "");
- try {
- return AbsoluteTableIdentifier
- .from(tablePath, getDatabaseName(configuration), getTableName(configuration));
- } catch (InvalidConfigurationException e) {
- throw new IOException(e);
- }
- }
-
/**
* {@inheritDoc}
* Configurations FileInputFormat.INPUT_DIR
@@ -362,8 +152,6 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
return getSplitsOfStreaming(job, identifier, streamSegments);
}
-
-
List<Segment> filteredSegmentToAccess = getFilteredSegment(job, segments.getValidSegments());
if (filteredSegmentToAccess.size() == 0) {
return new ArrayList<>(0);
@@ -716,195 +504,6 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
return result;
}
- protected Expression getFilterPredicates(Configuration configuration) {
- try {
- String filterExprString = configuration.get(FILTER_PREDICATE);
- if (filterExprString == null) {
- return null;
- }
- Object filter = ObjectSerializationUtil.convertStringToObject(filterExprString);
- return (Expression) filter;
- } catch (IOException e) {
- throw new RuntimeException("Error while reading filter expression", e);
- }
- }
-
- /**
- * get data blocks of given segment
- */
- private List<org.apache.carbondata.hadoop.CarbonInputSplit> getDataBlocksOfSegment(JobContext job,
- AbsoluteTableIdentifier absoluteTableIdentifier, FilterResolverIntf resolver,
- BitSet matchedPartitions, List<Segment> segmentIds, PartitionInfo partitionInfo,
- List<Integer> oldPartitionIdList) throws IOException {
-
- QueryStatisticsRecorder recorder = CarbonTimeStatisticsFactory.createDriverRecorder();
- QueryStatistic statistic = new QueryStatistic();
-
- // get tokens for all the required FileSystem for table path
- TokenCache.obtainTokensForNamenodes(job.getCredentials(),
- new Path[] { new Path(absoluteTableIdentifier.getTablePath()) }, job.getConfiguration());
- boolean distributedCG = Boolean.parseBoolean(CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
- CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT));
- DataMapExprWrapper dataMapExprWrapper =
- DataMapChooser.get().choose(getOrCreateCarbonTable(job.getConfiguration()), resolver);
- DataMapJob dataMapJob = getDataMapJob(job.getConfiguration());
- List<PartitionSpec> partitionsToPrune = getPartitionsToPrune(job.getConfiguration());
- List<ExtendedBlocklet> prunedBlocklets;
- if (distributedCG || dataMapExprWrapper.getDataMapType() == DataMapLevel.FG) {
- DistributableDataMapFormat datamapDstr =
- new DistributableDataMapFormat(absoluteTableIdentifier, dataMapExprWrapper,
- segmentIds, partitionsToPrune,
- BlockletDataMapFactory.class.getName());
- prunedBlocklets = dataMapJob.execute(datamapDstr, resolver);
- // Apply expression on the blocklets.
- prunedBlocklets = dataMapExprWrapper.pruneBlocklets(prunedBlocklets);
- } else {
- prunedBlocklets = dataMapExprWrapper.prune(segmentIds, partitionsToPrune);
- }
-
- List<org.apache.carbondata.hadoop.CarbonInputSplit> resultFilterredBlocks = new ArrayList<>();
- int partitionIndex = 0;
- List<Integer> partitionIdList = new ArrayList<>();
- if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
- partitionIdList = partitionInfo.getPartitionIds();
- }
- for (ExtendedBlocklet blocklet : prunedBlocklets) {
- long partitionId = CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(
- CarbonTablePath.DataFileUtil.getTaskNo(blocklet.getPath()));
-
- // OldPartitionIdList is only used in alter table partition command because it change
- // partition info first and then read data.
- // For other normal query should use newest partitionIdList
- if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
- if (oldPartitionIdList != null) {
- partitionIndex = oldPartitionIdList.indexOf((int)partitionId);
- } else {
- partitionIndex = partitionIdList.indexOf((int)partitionId);
- }
- }
- if (partitionIndex != -1) {
- // matchedPartitions variable will be null in two cases as follows
- // 1. the table is not a partition table
- // 2. the table is a partition table, and all partitions are matched by query
- // for partition table, the task id of carbaondata file name is the partition id.
- // if this partition is not required, here will skip it.
- if (matchedPartitions == null || matchedPartitions.get(partitionIndex)) {
- CarbonInputSplit inputSplit = convertToCarbonInputSplit(blocklet);
- if (inputSplit != null) {
- resultFilterredBlocks.add(inputSplit);
- }
- }
- }
- }
- statistic
- .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER, System.currentTimeMillis());
- recorder.recordStatisticsForDriver(statistic, job.getConfiguration().get("query.id"));
- return resultFilterredBlocks;
- }
-
- private CarbonInputSplit convertToCarbonInputSplit(ExtendedBlocklet blocklet) throws IOException {
- org.apache.carbondata.hadoop.CarbonInputSplit split =
- org.apache.carbondata.hadoop.CarbonInputSplit.from(blocklet.getSegmentId(),
- blocklet.getBlockletId(), new FileSplit(new Path(blocklet.getPath()), 0,
- blocklet.getLength(), blocklet.getLocations()),
- ColumnarFormatVersion.valueOf((short) blocklet.getDetailInfo().getVersionNumber()),
- blocklet.getDataMapWriterPath());
- split.setDetailInfo(blocklet.getDetailInfo());
- return split;
- }
-
- @Override
- public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
- TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
- Configuration configuration = taskAttemptContext.getConfiguration();
- QueryModel queryModel = createQueryModel(inputSplit, taskAttemptContext);
- CarbonReadSupport<T> readSupport = getReadSupportClass(configuration);
- return new CarbonRecordReader<T>(queryModel, readSupport);
- }
-
- public QueryModel createQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
- throws IOException {
- Configuration configuration = taskAttemptContext.getConfiguration();
- CarbonTable carbonTable = getOrCreateCarbonTable(configuration);
- TableProvider tableProvider = new SingleTableProvider(carbonTable);
-
- // query plan includes projection column
- String projectionString = getColumnProjection(configuration);
- String[] projectionColumnNames = null;
- if (projectionString != null) {
- projectionColumnNames = projectionString.split(",");
- }
- QueryModel queryModel = carbonTable.createQueryWithProjection(
- projectionColumnNames, getDataTypeConverter(configuration));
-
- // set the filter to the query model in order to filter blocklet before scan
- Expression filter = getFilterPredicates(configuration);
- boolean[] isFilterDimensions = new boolean[carbonTable.getDimensionOrdinalMax()];
- // getAllMeasures returns list of visible and invisible columns
- boolean[] isFilterMeasures =
- new boolean[carbonTable.getAllMeasures().size()];
- CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, isFilterDimensions,
- isFilterMeasures);
- queryModel.setIsFilterDimensions(isFilterDimensions);
- queryModel.setIsFilterMeasures(isFilterMeasures);
- FilterResolverIntf filterIntf = CarbonInputFormatUtil
- .resolveFilter(filter, carbonTable.getAbsoluteTableIdentifier(), tableProvider);
- queryModel.setFilterExpressionResolverTree(filterIntf);
-
- // update the file level index store if there are invalid segment
- if (inputSplit instanceof CarbonMultiBlockSplit) {
- CarbonMultiBlockSplit split = (CarbonMultiBlockSplit) inputSplit;
- List<String> invalidSegments = split.getAllSplits().get(0).getInvalidSegments();
- if (invalidSegments.size() > 0) {
- queryModel.setInvalidSegmentIds(invalidSegments);
- }
- List<UpdateVO> invalidTimestampRangeList =
- split.getAllSplits().get(0).getInvalidTimestampRange();
- if ((null != invalidTimestampRangeList) && (invalidTimestampRangeList.size() > 0)) {
- queryModel.setInvalidBlockForSegmentId(invalidTimestampRangeList);
- }
- }
- return queryModel;
- }
-
- public CarbonReadSupport<T> getReadSupportClass(Configuration configuration) {
- String readSupportClass = configuration.get(CARBON_READ_SUPPORT);
- //By default it uses dictionary decoder read class
- CarbonReadSupport<T> readSupport = null;
- if (readSupportClass != null) {
- try {
- Class<?> myClass = Class.forName(readSupportClass);
- Constructor<?> constructor = myClass.getConstructors()[0];
- Object object = constructor.newInstance();
- if (object instanceof CarbonReadSupport) {
- readSupport = (CarbonReadSupport) object;
- }
- } catch (ClassNotFoundException ex) {
- LOG.error("Class " + readSupportClass + "not found", ex);
- } catch (Exception ex) {
- LOG.error("Error while creating " + readSupportClass, ex);
- }
- } else {
- readSupport = new DictionaryDecodeReadSupport<>();
- }
- return readSupport;
- }
-
- @Override
- protected boolean isSplitable(JobContext context, Path filename) {
- try {
- // Don't split the file if it is local file system
- FileSystem fileSystem = filename.getFileSystem(context.getConfiguration());
- if (fileSystem instanceof LocalFileSystem) {
- return false;
- }
- } catch (Exception e) {
- return true;
- }
- return true;
- }
-
/**
* return valid segment to access
*/
@@ -969,58 +568,4 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
return new BlockMappingVO(blockRowCountMapping, segmentAndBlockCountMapping);
}
-
- /**
- * It is optional, if user does not set then it reads from store
- *
- * @param configuration
- * @param converter is the Data type converter for different computing engine
- * @throws IOException
- */
- public static void setDataTypeConverter(Configuration configuration, DataTypeConverter converter)
- throws IOException {
- if (null != converter) {
- configuration.set(CARBON_CONVERTER,
- ObjectSerializationUtil.convertObjectToString(converter));
- }
- }
-
- public static DataTypeConverter getDataTypeConverter(Configuration configuration)
- throws IOException {
- String converter = configuration.get(CARBON_CONVERTER);
- if (converter == null) {
- return new DataTypeConverterImpl();
- }
- return (DataTypeConverter) ObjectSerializationUtil.convertStringToObject(converter);
- }
-
- public static void setDatabaseName(Configuration configuration, String databaseName) {
- if (null != databaseName) {
- configuration.set(DATABASE_NAME, databaseName);
- }
- }
-
- public static String getDatabaseName(Configuration configuration)
- throws InvalidConfigurationException {
- String databseName = configuration.get(DATABASE_NAME);
- if (null == databseName) {
- throw new InvalidConfigurationException("Database name is not set.");
- }
- return databseName;
- }
-
- public static void setTableName(Configuration configuration, String tableName) {
- if (null != tableName) {
- configuration.set(TABLE_NAME, tableName);
- }
- }
-
- public static String getTableName(Configuration configuration)
- throws InvalidConfigurationException {
- String tableName = configuration.get(TABLE_NAME);
- if (tableName == null) {
- throw new InvalidConfigurationException("Table name is not set");
- }
- return tableName;
- }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
index ab7c333..9df59e6 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
@@ -87,11 +87,11 @@ public class SchemaReader {
// Convert the ColumnSchema -> TableSchema -> TableInfo.
// Return the TableInfo.
org.apache.carbondata.format.TableInfo tableInfo =
- CarbonUtil.inferSchemaFileExternalTable(identifier.getTablePath(), identifier, false);
+ CarbonUtil.inferSchema(identifier.getTablePath(), identifier, false);
SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
- TableInfo wrapperTableInfo = schemaConverter
- .fromExternalToWrapperTableInfo(tableInfo, identifier.getDatabaseName(),
- identifier.getTableName(), identifier.getTablePath());
+ TableInfo wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
+ tableInfo, identifier.getDatabaseName(), identifier.getTableName(),
+ identifier.getTablePath());
return wrapperTableInfo;
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
index 8b1f63f..7841a23 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
@@ -105,14 +105,14 @@ class TestCarbonFileInputFormatWithExternalCarbonTable extends QueryTest with Be
}
//TO DO, need to remove segment dependency and tableIdentifier Dependency
- test("read carbondata files (sdk Writer Output) using the Carbonfile ") {
+ test("read carbondata files (sdk Writer Output) using the carbonfile ") {
buildTestData(false)
assert(new File(writerPath).exists())
sql("DROP TABLE IF EXISTS sdkOutputTable")
- //new provider Carbonfile
+ //new provider carbonfile
sql(
- s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbonfile' LOCATION
|'$writerPath' """.stripMargin)
sql("Describe formatted sdkOutputTable").show(false)
@@ -152,7 +152,7 @@ class TestCarbonFileInputFormatWithExternalCarbonTable extends QueryTest with Be
//data source file format
sql(
- s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbonfile' LOCATION
|'$writerPath' """.stripMargin)
val exception = intercept[MalformedCarbonCommandException]
@@ -176,7 +176,7 @@ class TestCarbonFileInputFormatWithExternalCarbonTable extends QueryTest with Be
//data source file format
sql(
- s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbonfile' LOCATION
|'$writerPath' """.stripMargin)
//org.apache.spark.SparkException: Index file not present to read the carbondata file
@@ -192,7 +192,7 @@ class TestCarbonFileInputFormatWithExternalCarbonTable extends QueryTest with Be
cleanTestData()
}
-
+ // TODO: Make the sparkCarbonFileFormat to work without index file
test("Read sdk writer output file without Carbondata file should fail") {
buildTestData(false)
deleteIndexFile(writerPath, CarbonCommonConstants.FACT_FILE_EXT)
@@ -202,7 +202,7 @@ class TestCarbonFileInputFormatWithExternalCarbonTable extends QueryTest with Be
val exception = intercept[Exception] {
// data source file format
sql(
- s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbonfile' LOCATION
|'$writerPath' """.stripMargin)
}
assert(exception.getMessage()
@@ -225,7 +225,7 @@ class TestCarbonFileInputFormatWithExternalCarbonTable extends QueryTest with Be
val exception = intercept[Exception] {
//data source file format
sql(
- s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbonfile' LOCATION
|'$writerPath' """.stripMargin)
sql("select * from sdkOutputTable").show(false)
[05/10] carbondata git commit: [CARBONDATA-2236]added sdv test cases
for standard partition
Posted by ja...@apache.org.
[CARBONDATA-2236]added sdv test cases for standard partition
added sdv test cases for standard partition
This closes #2042
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/98b85501
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/98b85501
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/98b85501
Branch: refs/heads/carbonfile
Commit: 98b855014e3a68d108a99865d84634deb8ac7425
Parents: 0609fc5
Author: praveenmeenakshi56 <pr...@gmail.com>
Authored: Wed Mar 7 20:32:39 2018 +0530
Committer: kunal642 <ku...@gmail.com>
Committed: Mon Mar 19 15:46:39 2018 +0530
----------------------------------------------------------------------
.../generated/StandardPartitionTestCase.scala | 380 +++++++++++++++++++
.../cluster/sdv/suite/SDVSuites.scala | 7 +-
2 files changed, 384 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/98b85501/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/StandardPartitionTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/StandardPartitionTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/StandardPartitionTestCase.scala
new file mode 100644
index 0000000..c941363
--- /dev/null
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/StandardPartitionTestCase.scala
@@ -0,0 +1,380 @@
+/*
+ * 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.carbondata.cluster.sdv.generated
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util._
+import org.scalatest.BeforeAndAfterAll
+
+/**
+ * Test Class for partitionTestCase to verify all scenerios
+ */
+
+class StandardPartitionTestCase extends QueryTest with BeforeAndAfterAll {
+
+ override def beforeAll = {
+ CarbonProperties.getInstance()
+ .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd HH:mm:ss")
+ .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
+ }
+
+ //Verify exception if column in partitioned by is already specified in table schema
+ test("Standard-Partition_TC001", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ intercept[Exception] {
+ sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (INTEGER_COLUMN1 int)STORED BY 'carbondata'""")
+ }
+ }
+
+ //Verify table is created with Partition
+ test("Standard-Partition_TC002", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double) PARTITIONED BY (INTEGER_COLUMN1 int)STORED BY 'carbondata'""")
+ val df = sql(s"""DESC uniqdata""")
+ assert(df.collect().reverse.head.get(0).toString.toUpperCase.contains("INTEGER_COLUMN1"))
+ }
+
+ //Verify table is created with Partition with table comment
+ test("Standard-Partition_TC003",Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, decimalField DECIMAL(18,2), charField CHAR(5), floatField FLOAT ) COMMENT 'partition_table' PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ val df = sql(s"""DESC formatted partition_table""")
+ checkExistence(df, true, "partition_table")
+ }
+
+ //Verify Exception while creating a partition table, with ARRAY type partitioned column
+ test("Standard-Partition_TC004", Include) {
+ sql(s"""drop table if exists partition_table_array""")
+ intercept[Exception] {
+ sql(s"""CREATE TABLE partition_table_array(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, stringField STRING, timestampField TIMESTAMP, decimalField DECIMAL(18,2), dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (complexData ARRAY<STRING>) STORED BY 'carbondata'""")
+ }
+ sql(s"""drop table if exists partition_table_array""")
+
+ }
+
+ //Verify exception while Creating a partition table with DICTIONARY_INCLUDE
+ test("Standard-Partition_TC005", Include) {
+ sql(s"""drop table if exists partition_table_string""")
+ intercept[Exception] {
+ sql(s"""CREATE TABLE partition_table_string(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestampField TIMESTAMP, decimalField DECIMAL(18,2), dateField DATE, charField CHAR(5), floatField FLOAT, complexData ARRAY<STRING> ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_INCLUDE'='stringField')""")
+ }
+ sql(s"""drop table if exists partition_table_string""")
+ }
+
+ //Creating a partition table with DICTIONARY_EXCLUDE
+ test("Standard-Partition_TC006", Include) {
+ sql(s"""drop table if exists partition_table_string""")
+ sql(s"""CREATE TABLE partition_table_string(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestampField TIMESTAMP, decimalField DECIMAL(18,2), dateField DATE, charField CHAR(5), floatField FLOAT, complexData ARRAY<STRING> ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata' TBLPROPERTIES('DICTIONARY_EXCLUDE'='stringField')""")
+ sql(s"""drop table if exists partition_table_string""")
+ }
+
+ //Verify exception if datatype is not provided with partition column
+ test("Standard-Partition_TC007", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ intercept[Exception] {
+ sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double) PARTITIONED BY (DOJ)STORED BY 'carbondata'""")
+ }
+ }
+
+ //Verify exception if non existent file header is provided in partition
+ test("Standard-Partition_TC008", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double) PARTITIONED BY (DOJ timestamp)STORED BY 'carbondata'""")
+ intercept[Exception] {
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOJ,,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ }
+ }
+
+ //Verify exception if PARTITION BY is empty
+ test("Standard-Partition_TC009", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ intercept[Exception] {
+ sql(s"""CREATE TABLE uniqdata (CUST_ID int,CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double) PARTITIONED BY() STORED BY 'carbondata'""")
+ }
+ }
+
+ //Loading data into partitioned table with SORT_SCOPE=LOCAL_SORT
+ test("Standard-Partition_TC010", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata' TBLPROPERTIES('SORT_SCOPE'='LOCAL_SORT')""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(10)))
+ }
+
+ //Loading data into partitioned table with SORT_SCOPE=GLOBAL_SORT
+ test("Standard-Partition_TC011", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata' TBLPROPERTIES('SORT_SCOPE'='GLOBAL_SORT')""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(10)))
+ }
+
+ //Loading data into partitioned table with SORT_SCOPE=BATCH_SORT
+ test("Standard-Partition_TC012", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata' TBLPROPERTIES('SORT_SCOPE'='BATCH_SORT')""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(10)))
+ }
+
+ //Loading data into partitioned table with SORT_SCOPE=NO_SORT
+ test("Standard-Partition_TC013", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata' TBLPROPERTIES('SORT_SCOPE'='NO_SORT')""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(10)))
+ }
+
+ //Loading data into a partitioned table with Bad Records Action = FORCE
+ test("Standard-Partition_TC014", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table options('BAD_RECORDS_ACTION'='FORCE')""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(10)))
+ }
+
+ //Verify Exception when Loading data into a partitioned table with Bad Records Action = FAIL
+ test("Standard-Partition_TC015", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ intercept[Exception] {
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table options('FILEHEADER'='shortfield,intfield,bigintfield,doublefield,stringfield,timestamp,decimalfield,datefield,charfield,floatfield','BAD_RECORDS_ACTION'='FAIL')""")
+ }
+ }
+
+ //Loading data into a partitioned table with Bad Records Action = IGNORE
+ test("Standard-Partition_TC016", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table options('FILEHEADER'='shortfield,intfield,bigintfield,doublefield,stringfield,timestamp,decimalfield,datefield,charfield,floatfield','BAD_RECORDS_ACTION'='IGNORE')""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(0)))
+ }
+
+ //Loading data into a partitioned table with Bad Records Action = REDIRECT
+ test("Standard-Partition_TC017", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table options('FILEHEADER'='shortfield,intfield,bigintfield,doublefield,stringfield,timestamp,decimalfield,datefield,charfield,floatfield','BAD_RECORDS_ACTION'='REDIRECT')""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(0)))
+ }
+
+ //Loading data into a partitioned table with SINGLE_PASS=TRUE
+ test("Standard-Partition_TC018", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table options('SINGLE_PASS'='TRUE')""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(10)))
+ }
+
+ //Loading data into a partitioned table with SINGLE_PASS=FALSE
+ test("Standard-Partition_TC019", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table options('SINGLE_PASS'='FALSE')""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""), Seq(Row(10)))
+ }
+
+ //Verify load with Standard Partition
+ test("Standard-Partition_TC020", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ checkAnswer(sql(s"""select count(*) from uniqdata"""), Seq(Row(28)))
+ sql(s"""drop table if exists uniqdata""")
+ }
+
+ //Verify load with Standard Partition with limit 1
+ test("Standard-Partition_TC021", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ checkAnswer(sql(s"""select * from uniqdata limit 1"""),Seq(Row("CUST_NAME_00000","ACTIVE_EMUI_VERSION_00000",null,null,null,12345678901.0000000000,22345678901.0000000000,1.123456749E10,-1.123456749E10,1,null,1)))
+ }
+
+ //Verify load with Standard Partition with select partition column
+ test("Standard-Partition_TC022", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ checkAnswer(sql(s"""select CUST_ID from uniqdata limit 1"""), Seq(Row(1)))
+ }
+
+ //Verify table creation if 2 partition columns are provided
+ test("Standard-Partition_TC023", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int) PARTITIONED BY (CUST_ID int,DOJ timestamp) STORED BY 'carbondata'""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='1',doj) OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ val df = sql(s"""DESC uniqdata""").collect()
+ assert(df(df.indexWhere(_.get(0).toString.contains("# Partition")) + 3).get(0).toString.contains("doj"))
+ assert(df(df.indexWhere(_.get(0).toString.contains("# Partition")) + 2).get(0).toString.contains("cust_id"))
+ sql(s"""drop table if exists uniqdata""")
+ }
+
+ //Verify load with Standard Partition after compaction
+ test("Standard-Partition_TC024", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ sql(s"""alter table uniqdata compact 'minor'""")
+ checkAnswer(sql(s"""select count(*) from uniqdata"""), Seq(Row(84)))
+ }
+
+ //Verify join operation on Standard Partition
+ test("Standard-Partition_TC025", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""drop table if exists uniqdata1""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ sql(s"""CREATE TABLE uniqdata1 (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata1 partition(CUST_ID='1') OPTIONS('DELIMITER'=',','BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ checkAnswer(sql(s"""select a.cust_id, b.cust_id from uniqdata a, uniqdata1 b where a.cust_id >= b.cust_id limit 1"""),Seq(Row(1,1)))
+ sql(s"""drop table if exists uniqdata1""")
+ }
+
+ //Verify exception if partition column is dropped
+ test("Standard-Partition_TC026", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ intercept[Exception] {
+ sql(s"""alter table uniqdata drop columns(CUST_ID)""")
+ }
+ }
+
+ //Verify INSERT operation on standard partition
+ test("Standard-Partition_TC027", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ sql(s"""insert into table uniqdata values ('a', '1','2015-07-01 00:00:00', 5678,7654,23.4, 55.6, 7654, 8765,33,'2015-07-01 00:00:00', 1)""")
+ sql(s"""insert into table uniqdata values ('a', '1','2015-07-01 00:00:00', 5678,7654,23.4, 55.6, 7654, 8765,33,'2015-07-01 00:00:00', 0)""")
+ checkAnswer(sql(s"""select count(*) from uniqdata"""), Seq(Row(2)))
+ }
+
+ //Verify INSERT INTO SELECT operation on standard partition
+ test("Standard-Partition_TC028", Include) {
+ sql(s"""DROP TABLE IF EXISTS PARTITION_TABLE""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2), dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""DROP TABLE IF EXISTS PARTITION_TABLE_load""")
+ sql(s"""CREATE TABLE partition_table_load (shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2), dateField DATE, charField CHAR(5), floatField FLOAT ) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table_load options ('BAD_RECORDS_ACTION'='FORCE')""")
+ sql(s"""INSERT into TABLE partition_table PARTITION (stringfield = 'Hello') SELECT * FROM partition_table_load au WHERE au.intfield = 25""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""),Seq(Row(2)))
+ sql(s"""drop table if exists PARTITION_TABLE""")
+ sql(s"""drop table if exists PARTITION_TABLE_load""")
+ }
+
+ //Verify INSERT overwrite operation on standard partition
+ test("Standard-Partition_TC029", Include) {
+ sql(s"""DROP TABLE IF EXISTS PARTITION_TABLE""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2), dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""DROP TABLE IF EXISTS PARTITION_TABLE_load""")
+ sql(s"""CREATE TABLE partition_table_load (shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2), dateField DATE, charField CHAR(5), floatField FLOAT ) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table_load options ('BAD_RECORDS_ACTION'='FORCE')""")
+ sql(s"""INSERT OVERWRITE TABLE partition_table PARTITION (stringfield = 'Hello') SELECT * FROM partition_table_load au WHERE au.intField = 25""")
+ sql(s"""INSERT OVERWRITE TABLE partition_table PARTITION (stringfield = 'Hello') SELECT * FROM partition_table_load au WHERE au.intField = 25""")
+ checkAnswer(sql(s"""select count(*) from partition_table"""),Seq(Row(2)))
+ sql(s"""drop table if exists PARTITION_TABLE""")
+ sql(s"""drop table if exists PARTITION_TABLE_load""")
+ }
+
+ //Verify date with > filter condition and standard partition
+ test("Standard-Partition_TC030", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='4') OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ checkAnswer(sql(s"""select count(*) from uniqdata where CUST_ID>3"""),Seq(Row(28)))
+ }
+
+ //Verify date with = filter condition and standard partition
+ test("Standard-Partition_TC031", Include) {
+ sql(s"""drop table if exists uniqdata""")
+ sql(s"""CREATE TABLE uniqdata (CUST_NAME String,ACTIVE_EMUI_VERSION string, DOB timestamp, BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 decimal(30,10), DECIMAL_COLUMN2 decimal(36,10),Double_COLUMN1 double, Double_COLUMN2 double,INTEGER_COLUMN1 int, DOJ timestamp) PARTITIONED BY (CUST_ID int) STORED BY 'carbondata'""")
+ sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata partition(CUST_ID='4')OPTIONS('DELIMITER'=',' , 'BAD_RECORDS_ACTION'='FORCE','QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""")
+ checkAnswer(sql(s"""select count(*) from uniqdata where CUST_ID=3"""),Seq(Row(0)))
+ }
+
+ //Verify update partition_table on standard Partition
+ test("Standard-Partition_TC032", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table PARTITION (stringField = "Hello")""")
+ sql("""update partition_table set (stringfield)=('China') where stringfield = 'Hello'""").collect
+ checkAnswer(sql(s"""select stringfield from partition_table where charfield='c' limit 1"""),Seq(Row("China")))
+ }
+
+ //Verify update partition_table on standard Partition
+ test("Standard-Partition_TC033", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table PARTITION (stringField = 'Hello')""")
+ sql("""update partition_table set (stringfield)=('China') where charfield = 'c'""").collect
+ sql("""update partition_table set (stringfield)=('China123') where stringfield != 'China'""").collect
+ checkAnswer(sql(s"""select stringfield from partition_table where charfield='c' limit 1"""),Seq(Row("China")))
+ }
+
+ //Verify update partition_table on standard Partition
+ test("Standard-Partition_TC034", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table PARTITION (stringField = 'Hello')""")
+ sql("""update partition_table set (stringfield)=('China') where stringfield = 'Hello'""").collect
+ sql("""update partition_table set (stringfield)=('China123') where stringfield != 'China'""").collect
+ sql("""update partition_table set (stringfield)=('Japan') where stringfield > 'China'""").collect
+ checkAnswer(sql(s"""select stringfield from partition_table where charfield='c' limit 1"""), Seq(Row("China")))
+ }
+
+ //Verify update partition_table on standard Partition
+ test("Standard-Partition_TC035", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table PARTITION (stringField = 'Hello')""")
+ sql("""update partition_table set (stringfield)=('Asia') where stringfield < 'Hello'""").collect
+ checkAnswer(sql(s"""select stringfield from partition_table where charfield='c' limit 1"""), Seq(Row("Hello")))
+ }
+
+ //Verify update partition_table on standard Partition
+ test("Standard-Partition_TC036", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table PARTITION (stringField = 'Hello')""")
+ sql("""update partition_table set (stringfield)=('China') where stringfield = 'Hello'""").collect
+ sql("""update partition_table set (stringfield)=('Europe') where stringfield LIKE 'C%'""").collect
+ checkAnswer(sql(s"""select stringfield from partition_table where charfield='c' limit 1"""), Seq(Row("Europe")))
+ }
+
+ //Verify update partition_table on standard Partition
+ test("Standard-Partition_TC037", Include) {
+ sql(s"""drop table if exists partition_table""")
+ sql(s"""CREATE TABLE partition_table(shortField SHORT, intField INT, bigintField LONG, doubleField DOUBLE, timestamp TIMESTAMP, decimalField DECIMAL(18,2),dateField DATE, charField CHAR(5), floatField FLOAT ) PARTITIONED BY (stringField STRING) STORED BY 'carbondata'""")
+ sql(s"""load data inpath '$resourcesPath/Data/partition/list_partition_table.csv' into table partition_table PARTITION (stringField = 'Hello')""")
+ sql("""update partition_table set (stringfield)=('China') where stringfield = 'Hello'""").collect
+ sql("""update partition_table set (stringfield)=('Europe') where stringfield LIKE 'C%'""").collect
+ checkAnswer(sql(s"""select stringfield from partition_table where charfield='c' limit 1"""), Seq(Row("Europe")))
+ }
+
+ override def afterAll {
+ CarbonProperties.getInstance()
+ .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+ .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
+ sql("drop table if exists uniqdata")
+ sql("drop table if exists partition_table")
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/carbondata/blob/98b85501/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
index 32f74fb..da6e829 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/suite/SDVSuites.scala
@@ -57,7 +57,7 @@ class SDVSuites extends Suites with BeforeAndAfterAll {
new TimestamptypesTestCase ::
new V3offheapvectorTestCase ::
new Vector1TestCase ::
- new Vector2TestCase ::Nil
+ new Vector2TestCase :: Nil
override val nestedSuites = suites.toIndexedSeq
@@ -139,7 +139,8 @@ class SDVSuites3 extends Suites with BeforeAndAfterAll {
new DataLoadingV3TestCase ::
new QueriesIncludeDictionaryTestCase ::
new TestRegisterCarbonTable ::
- new TableCommentAlterTableTestCase :: Nil
+ new TableCommentAlterTableTestCase ::
+ new StandardPartitionTestCase :: Nil
override val nestedSuites = suites.toIndexedSeq
@@ -155,7 +156,7 @@ class SDVSuites3 extends Suites with BeforeAndAfterAll {
*/
class SDVSuites4 extends Suites with BeforeAndAfterAll {
- val suites = new CarbonV1toV3CompatabilityTestCase :: Nil
+ val suites = new CarbonV1toV3CompatabilityTestCase :: Nil
override val nestedSuites = suites.toIndexedSeq
[02/10] carbondata git commit: [CARBONDATA-2253][SDK] Support write
JSON/Avro data to carbon files
Posted by ja...@apache.org.
[CARBONDATA-2253][SDK] Support write JSON/Avro data to carbon files
This PR adds AvroCarbonWriter in SDK, it can be used to write JSON or Avro data to carbon files
This closes #2061
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/e39b0a14
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/e39b0a14
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/e39b0a14
Branch: refs/heads/carbonfile
Commit: e39b0a14a196224dbe6fdce2ff53e09b3b76b876
Parents: 04ff367
Author: Jacky Li <ja...@qq.com>
Authored: Wed Mar 14 00:06:37 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Fri Mar 16 23:02:37 2018 +0800
----------------------------------------------------------------------
pom.xml | 6 -
store/sdk/pom.xml | 95 +------
.../carbondata/sdk/file/AvroCarbonWriter.java | 125 +++++++++
.../sdk/file/CarbonWriterBuilder.java | 29 +-
.../sdk/file/AvroCarbonWriterTest.java | 104 ++++++++
.../sdk/file/CSVCarbonWriterSuite.java | 267 -------------------
.../sdk/file/CSVCarbonWriterTest.java | 267 +++++++++++++++++++
7 files changed, 519 insertions(+), 374 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e39b0a14/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 972be1e..287e052 100644
--- a/pom.xml
+++ b/pom.xml
@@ -579,12 +579,6 @@
<id>include-all</id>
</profile>
<profile>
- <id>store-sdk</id>
- <modules>
- <module>store/sdk</module>
- </modules>
- </profile>
- <profile>
<id>sdvtest</id>
<modules>
<module>integration/spark-common-cluster-test</module>
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e39b0a14/store/sdk/pom.xml
----------------------------------------------------------------------
diff --git a/store/sdk/pom.xml b/store/sdk/pom.xml
index 9f7038a..1d1735e 100644
--- a/store/sdk/pom.xml
+++ b/store/sdk/pom.xml
@@ -25,55 +25,27 @@
<version>${project.version}</version>
</dependency>
<dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
+ <groupId>tech.allegro.schema.json2avro</groupId>
+ <artifactId>converter</artifactId>
+ <version>0.2.5</version>
<scope>test</scope>
</dependency>
<dependency>
- <groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.binary.version}</artifactId>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
+ <sourceDirectory>src/main/java</sourceDirectory>
<resources>
<resource>
- <directory>src/resources</directory>
- </resource>
- <resource>
<directory>.</directory>
</resource>
</resources>
<plugins>
<plugin>
- <groupId>org.scala-tools</groupId>
- <artifactId>maven-scala-plugin</artifactId>
- <version>2.15.2</version>
- <executions>
- <execution>
- <id>compile</id>
- <goals>
- <goal>compile</goal>
- </goals>
- <phase>compile</phase>
- </execution>
- <execution>
- <id>testCompile</id>
- <goals>
- <goal>testCompile</goal>
- </goals>
- <phase>test</phase>
- </execution>
- <execution>
- <phase>process-resources</phase>
- <goals>
- <goal>compile</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- <plugin>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.7</source>
@@ -82,61 +54,6 @@
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-surefire-plugin</artifactId>
- <version>2.18</version>
- <!-- Note config is repeated in scalatest config -->
- <configuration>
- <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
- <argLine>-Xmx3g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m</argLine>
- <systemProperties>
- <java.awt.headless>true</java.awt.headless>
- </systemProperties>
- <failIfNoTests>false</failIfNoTests>
- </configuration>
- </plugin>
- <plugin>
- <groupId>org.scalatest</groupId>
- <artifactId>scalatest-maven-plugin</artifactId>
- <version>1.0</version>
- <!-- Note config is repeated in surefire config -->
- <configuration>
- <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
- <junitxml>.</junitxml>
- <filereports>CarbonTestSuite.txt</filereports>
- <argLine> ${argLine} -ea -Xmx3g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m
- </argLine>
- <stderr />
- <environmentVariables>
- </environmentVariables>
- <systemProperties>
- <java.awt.headless>true</java.awt.headless>
- </systemProperties>
- </configuration>
- <executions>
- <execution>
- <id>test</id>
- <goals>
- <goal>test</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-deploy-plugin</artifactId>
- <configuration>
- <skip>true</skip>
- </configuration>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-install-plugin</artifactId>
- <configuration>
- <skip>true</skip>
- </configuration>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<configuration>
<shadedArtifactAttached>false</shadedArtifactAttached>
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e39b0a14/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
new file mode 100644
index 0000000..e88164c
--- /dev/null
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.sdk.file;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat;
+import org.apache.carbondata.hadoop.internal.ObjectArrayWritable;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+/**
+ * Writer Implementation to write Avro Record to carbondata file.
+ */
+@InterfaceAudience.Internal
+class AvroCarbonWriter extends CarbonWriter {
+
+ private RecordWriter<NullWritable, ObjectArrayWritable> recordWriter;
+ private TaskAttemptContext context;
+ private ObjectArrayWritable writable;
+ private Schema avroSchema;
+
+ AvroCarbonWriter(CarbonLoadModel loadModel) throws IOException {
+ Configuration hadoopConf = new Configuration();
+ CarbonTableOutputFormat.setLoadModel(hadoopConf, loadModel);
+ CarbonTableOutputFormat format = new CarbonTableOutputFormat();
+ JobID jobId = new JobID(UUID.randomUUID().toString(), 0);
+ Random random = new Random();
+ TaskID task = new TaskID(jobId, TaskType.MAP, random.nextInt());
+ TaskAttemptID attemptID = new TaskAttemptID(task, random.nextInt());
+ TaskAttemptContextImpl context = new TaskAttemptContextImpl(hadoopConf, attemptID);
+ this.recordWriter = format.getRecordWriter(context);
+ this.context = context;
+ this.writable = new ObjectArrayWritable();
+ }
+
+ private String[] avroToCsv(GenericData.Record avroRecord) {
+ if (avroSchema == null) {
+ avroSchema = avroRecord.getSchema();
+ }
+ List<Schema.Field> fields = avroSchema.getFields();
+ String[] csvField = new String[fields.size()];
+ for (int i = 0; i < fields.size(); i++) {
+ csvField[i] = avroFieldToString(fields.get(i), avroRecord.get(i));
+ }
+ return csvField;
+ }
+
+ private String avroFieldToString(Schema.Field fieldType, Object fieldValue) {
+ StringBuilder out = new StringBuilder();
+ Schema.Type type = fieldType.schema().getType();
+ switch (type) {
+ case BOOLEAN:
+ case INT:
+ case LONG:
+ case DOUBLE:
+ case STRING:
+ out.append(fieldValue.toString());
+ break;
+ default:
+ throw new UnsupportedOperationException();
+ // TODO: convert complex type
+ }
+ return out.toString();
+ }
+
+ /**
+ * Write single row data, input row is Avro Record
+ */
+ @Override
+ public void write(Object object) throws IOException {
+ GenericData.Record record = (GenericData.Record) object;
+
+ // convert Avro record to CSV String[]
+ String[] csvRecord = avroToCsv(record);
+ writable.set(csvRecord);
+ try {
+ recordWriter.write(NullWritable.get(), writable);
+ } catch (InterruptedException e) {
+ throw new IOException(e);
+ }
+ }
+
+ /**
+ * Flush and close the writer
+ */
+ @Override
+ public void close() throws IOException {
+ try {
+ recordWriter.close(context);
+ } catch (InterruptedException e) {
+ throw new IOException(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e39b0a14/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
index 8734341..5be60c4 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
@@ -104,7 +104,23 @@ public class CarbonWriterBuilder {
public CarbonWriter buildWriterForCSVInput() throws IOException, InvalidLoadOptionException {
Objects.requireNonNull(schema, "schema should not be null");
Objects.requireNonNull(path, "path should not be null");
+ CarbonLoadModel loadModel = createLoadModel();
+ return new CSVCarbonWriter(loadModel);
+ }
+
+ /**
+ * Build a {@link CarbonWriter}, which accepts Avro object
+ * @return
+ * @throws IOException
+ */
+ public CarbonWriter buildWriterForAvroInput() throws IOException, InvalidLoadOptionException {
+ Objects.requireNonNull(schema, "schema should not be null");
+ Objects.requireNonNull(path, "path should not be null");
+ CarbonLoadModel loadModel = createLoadModel();
+ return new AvroCarbonWriter(loadModel);
+ }
+ private CarbonLoadModel createLoadModel() throws IOException, InvalidLoadOptionException {
// build CarbonTable using schema
CarbonTable table = buildCarbonTable();
if (persistSchemaFile) {
@@ -113,18 +129,7 @@ public class CarbonWriterBuilder {
}
// build LoadModel
- CarbonLoadModel loadModel = buildLoadModel(table);
- return new CSVCarbonWriter(loadModel);
- }
-
- /**
- * Build a {@link CarbonWriter}, which accepts Avro object
- * @return
- * @throws IOException
- */
- public CarbonWriter buildWriterForAvroInput() throws IOException {
- // TODO
- throw new UnsupportedOperationException();
+ return buildLoadModel(table);
}
/**
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e39b0a14/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java
new file mode 100644
index 0000000..25c34e0
--- /dev/null
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.carbondata.sdk.file;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.CharEncoding;
+import org.junit.Assert;
+import org.junit.Test;
+
+import tech.allegro.schema.json2avro.converter.JsonAvroConverter;
+import org.apache.avro.Schema;
+
+public class AvroCarbonWriterTest {
+ private String path = "./AvroCarbonWriterSuiteWriteFiles";
+
+ @Test
+ public void testWriteBasic() throws IOException {
+ FileUtils.deleteDirectory(new File(path));
+
+ // Avro schema
+ String avroSchema =
+ "{" +
+ " \"type\" : \"record\"," +
+ " \"name\" : \"Acme\"," +
+ " \"fields\" : ["
+ + "{ \"name\" : \"name\", \"type\" : \"string\" },"
+ + "{ \"name\" : \"age\", \"type\" : \"int\" }]" +
+ "}";
+
+ String json = "{\"name\":\"bob\", \"age\":10}";
+
+ // conversion to GenericData.Record
+ JsonAvroConverter converter = new JsonAvroConverter();
+ GenericData.Record record = converter.convertToGenericDataRecord(
+ json.getBytes(CharEncoding.UTF_8), new Schema.Parser().parse(avroSchema));
+
+ Field[] fields = new Field[2];
+ fields[0] = new Field("name", DataTypes.STRING);
+ fields[1] = new Field("age", DataTypes.STRING);
+
+ try {
+ CarbonWriter writer = CarbonWriter.builder()
+ .withSchema(new org.apache.carbondata.sdk.file.Schema(fields))
+ .outputPath(path)
+ .buildWriterForAvroInput();
+
+ for (int i = 0; i < 100; i++) {
+ writer.write(record);
+ }
+ writer.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+
+ File segmentFolder = new File(CarbonTablePath.getSegmentPath(path, "null"));
+ Assert.assertTrue(segmentFolder.exists());
+
+ File[] dataFiles = segmentFolder.listFiles(new FileFilter() {
+ @Override public boolean accept(File pathname) {
+ return pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT);
+ }
+ });
+ Assert.assertNotNull(dataFiles);
+ Assert.assertEquals(1, dataFiles.length);
+
+ FileUtils.deleteDirectory(new File(path));
+ }
+
+ @Test
+ public void testWriteAllPrimitive() throws IOException {
+ // TODO
+ }
+
+ @Test
+ public void testWriteNestedRecord() throws IOException {
+ // TODO
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e39b0a14/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
deleted file mode 100644
index 0ac6f38..0000000
--- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
+++ /dev/null
@@ -1,267 +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.carbondata.sdk.file;
-
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-import org.apache.commons.io.FileUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test suite for {@link CSVCarbonWriter}
- */
-public class CSVCarbonWriterSuite {
-
- @Test
- public void testWriteFiles() throws IOException {
- String path = "./testWriteFiles";
- FileUtils.deleteDirectory(new File(path));
-
- Field[] fields = new Field[2];
- fields[0] = new Field("name", DataTypes.STRING);
- fields[1] = new Field("age", DataTypes.INT);
-
- writeFilesAndVerify(new Schema(fields), path);
-
- FileUtils.deleteDirectory(new File(path));
- }
-
- @Test
- public void testWriteFilesJsonSchema() throws IOException {
- String path = "./testWriteFilesJsonSchema";
- FileUtils.deleteDirectory(new File(path));
-
- String schema = new StringBuilder()
- .append("[ \n")
- .append(" {\"name\":\"string\"},\n")
- .append(" {\"age\":\"int\"},\n")
- .append(" {\"height\":\"double\"}\n")
- .append("]")
- .toString();
-
- writeFilesAndVerify(Schema.parseJson(schema), path);
-
- FileUtils.deleteDirectory(new File(path));
- }
-
- private void writeFilesAndVerify(Schema schema, String path) {
- writeFilesAndVerify(schema, path, null);
- }
-
- private void writeFilesAndVerify(Schema schema, String path, String[] sortColumns) {
- writeFilesAndVerify(100, schema, path, sortColumns, false, -1, -1);
- }
-
- private void writeFilesAndVerify(Schema schema, String path, boolean persistSchema) {
- writeFilesAndVerify(100, schema, path, null, persistSchema, -1, -1);
- }
-
- /**
- * Invoke CarbonWriter API to write carbon files and assert the file is rewritten
- * @param rows number of rows to write
- * @param schema schema of the file
- * @param path local write path
- * @param sortColumns sort columns
- * @param persistSchema true if want to persist schema file
- * @param blockletSize blockletSize in the file, -1 for default size
- * @param blockSize blockSize in the file, -1 for default size
- */
- private void writeFilesAndVerify(int rows, Schema schema, String path, String[] sortColumns,
- boolean persistSchema, int blockletSize, int blockSize) {
- try {
- CarbonWriterBuilder builder = CarbonWriter.builder()
- .withSchema(schema)
- .outputPath(path);
- if (sortColumns != null) {
- builder = builder.sortBy(sortColumns);
- }
- if (persistSchema) {
- builder = builder.persistSchemaFile(true);
- }
- if (blockletSize != -1) {
- builder = builder.withBlockletSize(blockletSize);
- }
- if (blockSize != -1) {
- builder = builder.withBlockSize(blockSize);
- }
-
- CarbonWriter writer = builder.buildWriterForCSVInput();
-
- for (int i = 0; i < rows; i++) {
- writer.write(new String[]{"robot" + (i % 10), String.valueOf(i), String.valueOf((double) i / 2)});
- }
- writer.close();
- } catch (Exception e) {
- e.printStackTrace();
- Assert.fail(e.getMessage());
- }
-
- File segmentFolder = new File(CarbonTablePath.getSegmentPath(path, "null"));
- Assert.assertTrue(segmentFolder.exists());
-
- File[] dataFiles = segmentFolder.listFiles(new FileFilter() {
- @Override public boolean accept(File pathname) {
- return pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT);
- }
- });
- Assert.assertNotNull(dataFiles);
- Assert.assertTrue(dataFiles.length > 0);
- }
-
- @Test
- public void testAllPrimitiveDataType() throws IOException {
- // TODO: write all data type and read by CarbonRecordReader to verify the content
- String path = "./testWriteFiles";
- FileUtils.deleteDirectory(new File(path));
-
- Field[] fields = new Field[9];
- fields[0] = new Field("stringField", DataTypes.STRING);
- fields[1] = new Field("intField", DataTypes.INT);
- fields[2] = new Field("shortField", DataTypes.SHORT);
- fields[3] = new Field("longField", DataTypes.LONG);
- fields[4] = new Field("doubleField", DataTypes.DOUBLE);
- fields[5] = new Field("boolField", DataTypes.BOOLEAN);
- fields[6] = new Field("dateField", DataTypes.DATE);
- fields[7] = new Field("timeField", DataTypes.TIMESTAMP);
- fields[8] = new Field("decimalField", DataTypes.createDecimalType(8, 2));
-
- try {
- CarbonWriterBuilder builder = CarbonWriter.builder()
- .withSchema(new Schema(fields))
- .outputPath(path);
-
- CarbonWriter writer = builder.buildWriterForCSVInput();
-
- for (int i = 0; i < 100; i++) {
- String[] row = new String[]{
- "robot" + (i % 10),
- String.valueOf(i),
- String.valueOf(i),
- String.valueOf(Long.MAX_VALUE - i),
- String.valueOf((double) i / 2),
- String.valueOf(true),
- "2019-03-02",
- "2019-02-12 03:03:34"
- };
- writer.write(row);
- }
- writer.close();
- } catch (Exception e) {
- e.printStackTrace();
- Assert.fail(e.getMessage());
- }
-
- File segmentFolder = new File(CarbonTablePath.getSegmentPath(path, "null"));
- Assert.assertTrue(segmentFolder.exists());
-
- File[] dataFiles = segmentFolder.listFiles(new FileFilter() {
- @Override public boolean accept(File pathname) {
- return pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT);
- }
- });
- Assert.assertNotNull(dataFiles);
- Assert.assertTrue(dataFiles.length > 0);
-
- FileUtils.deleteDirectory(new File(path));
- }
-
- @Test
- public void test2Blocklet() throws IOException {
- String path = "./testWriteFiles";
- FileUtils.deleteDirectory(new File(path));
-
- Field[] fields = new Field[2];
- fields[0] = new Field("name", DataTypes.STRING);
- fields[1] = new Field("age", DataTypes.INT);
-
- writeFilesAndVerify(1000 * 1000, new Schema(fields), path, null, false, 1, 100);
-
- // TODO: implement reader to verify the number of blocklet in the file
-
- FileUtils.deleteDirectory(new File(path));
- }
-
- @Test
- public void test2Block() throws IOException {
- String path = "./testWriteFiles";
- FileUtils.deleteDirectory(new File(path));
-
- Field[] fields = new Field[2];
- fields[0] = new Field("name", DataTypes.STRING);
- fields[1] = new Field("age", DataTypes.INT);
-
- writeFilesAndVerify(1000 * 1000, new Schema(fields), path, null, false, 2, 2);
-
- File segmentFolder = new File(CarbonTablePath.getSegmentPath(path, "null"));
- File[] dataFiles = segmentFolder.listFiles(new FileFilter() {
- @Override public boolean accept(File pathname) {
- return pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT);
- }
- });
- Assert.assertNotNull(dataFiles);
- Assert.assertEquals(2, dataFiles.length);
-
- FileUtils.deleteDirectory(new File(path));
- }
-
- @Test
- public void testSortColumns() throws IOException {
- String path = "./testWriteFiles";
- FileUtils.deleteDirectory(new File(path));
-
- Field[] fields = new Field[2];
- fields[0] = new Field("name", DataTypes.STRING);
- fields[1] = new Field("age", DataTypes.INT);
-
- writeFilesAndVerify(new Schema(fields), path, new String[]{"name"});
-
- // TODO: implement reader and verify the data is sorted
-
- FileUtils.deleteDirectory(new File(path));
- }
-
- @Test
- public void testPartitionOutput() {
- // TODO: test write data with partition
- }
-
- @Test
- public void testSchemaPersistence() throws IOException {
- String path = "./testWriteFiles";
- FileUtils.deleteDirectory(new File(path));
-
- Field[] fields = new Field[2];
- fields[0] = new Field("name", DataTypes.STRING);
- fields[1] = new Field("age", DataTypes.INT);
-
- writeFilesAndVerify(new Schema(fields), path, true);
-
- String schemaFile = CarbonTablePath.getSchemaFilePath(path);
- Assert.assertTrue(new File(schemaFile).exists());
-
- FileUtils.deleteDirectory(new File(path));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e39b0a14/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterTest.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterTest.java
new file mode 100644
index 0000000..2281fe6
--- /dev/null
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterTest.java
@@ -0,0 +1,267 @@
+/*
+ * 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.carbondata.sdk.file;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test suite for {@link CSVCarbonWriter}
+ */
+public class CSVCarbonWriterTest {
+
+ @Test
+ public void testWriteFiles() throws IOException {
+ String path = "./testWriteFiles";
+ FileUtils.deleteDirectory(new File(path));
+
+ Field[] fields = new Field[2];
+ fields[0] = new Field("name", DataTypes.STRING);
+ fields[1] = new Field("age", DataTypes.INT);
+
+ writeFilesAndVerify(new Schema(fields), path);
+
+ FileUtils.deleteDirectory(new File(path));
+ }
+
+ @Test
+ public void testWriteFilesJsonSchema() throws IOException {
+ String path = "./testWriteFilesJsonSchema";
+ FileUtils.deleteDirectory(new File(path));
+
+ String schema = new StringBuilder()
+ .append("[ \n")
+ .append(" {\"name\":\"string\"},\n")
+ .append(" {\"age\":\"int\"},\n")
+ .append(" {\"height\":\"double\"}\n")
+ .append("]")
+ .toString();
+
+ writeFilesAndVerify(Schema.parseJson(schema), path);
+
+ FileUtils.deleteDirectory(new File(path));
+ }
+
+ private void writeFilesAndVerify(Schema schema, String path) {
+ writeFilesAndVerify(schema, path, null);
+ }
+
+ private void writeFilesAndVerify(Schema schema, String path, String[] sortColumns) {
+ writeFilesAndVerify(100, schema, path, sortColumns, false, -1, -1);
+ }
+
+ private void writeFilesAndVerify(Schema schema, String path, boolean persistSchema) {
+ writeFilesAndVerify(100, schema, path, null, persistSchema, -1, -1);
+ }
+
+ /**
+ * Invoke CarbonWriter API to write carbon files and assert the file is rewritten
+ * @param rows number of rows to write
+ * @param schema schema of the file
+ * @param path local write path
+ * @param sortColumns sort columns
+ * @param persistSchema true if want to persist schema file
+ * @param blockletSize blockletSize in the file, -1 for default size
+ * @param blockSize blockSize in the file, -1 for default size
+ */
+ private void writeFilesAndVerify(int rows, Schema schema, String path, String[] sortColumns,
+ boolean persistSchema, int blockletSize, int blockSize) {
+ try {
+ CarbonWriterBuilder builder = CarbonWriter.builder()
+ .withSchema(schema)
+ .outputPath(path);
+ if (sortColumns != null) {
+ builder = builder.sortBy(sortColumns);
+ }
+ if (persistSchema) {
+ builder = builder.persistSchemaFile(true);
+ }
+ if (blockletSize != -1) {
+ builder = builder.withBlockletSize(blockletSize);
+ }
+ if (blockSize != -1) {
+ builder = builder.withBlockSize(blockSize);
+ }
+
+ CarbonWriter writer = builder.buildWriterForCSVInput();
+
+ for (int i = 0; i < rows; i++) {
+ writer.write(new String[]{"robot" + (i % 10), String.valueOf(i), String.valueOf((double) i / 2)});
+ }
+ writer.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+
+ File segmentFolder = new File(CarbonTablePath.getSegmentPath(path, "null"));
+ Assert.assertTrue(segmentFolder.exists());
+
+ File[] dataFiles = segmentFolder.listFiles(new FileFilter() {
+ @Override public boolean accept(File pathname) {
+ return pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT);
+ }
+ });
+ Assert.assertNotNull(dataFiles);
+ Assert.assertTrue(dataFiles.length > 0);
+ }
+
+ @Test
+ public void testAllPrimitiveDataType() throws IOException {
+ // TODO: write all data type and read by CarbonRecordReader to verify the content
+ String path = "./testWriteFiles";
+ FileUtils.deleteDirectory(new File(path));
+
+ Field[] fields = new Field[9];
+ fields[0] = new Field("stringField", DataTypes.STRING);
+ fields[1] = new Field("intField", DataTypes.INT);
+ fields[2] = new Field("shortField", DataTypes.SHORT);
+ fields[3] = new Field("longField", DataTypes.LONG);
+ fields[4] = new Field("doubleField", DataTypes.DOUBLE);
+ fields[5] = new Field("boolField", DataTypes.BOOLEAN);
+ fields[6] = new Field("dateField", DataTypes.DATE);
+ fields[7] = new Field("timeField", DataTypes.TIMESTAMP);
+ fields[8] = new Field("decimalField", DataTypes.createDecimalType(8, 2));
+
+ try {
+ CarbonWriterBuilder builder = CarbonWriter.builder()
+ .withSchema(new Schema(fields))
+ .outputPath(path);
+
+ CarbonWriter writer = builder.buildWriterForCSVInput();
+
+ for (int i = 0; i < 100; i++) {
+ String[] row = new String[]{
+ "robot" + (i % 10),
+ String.valueOf(i),
+ String.valueOf(i),
+ String.valueOf(Long.MAX_VALUE - i),
+ String.valueOf((double) i / 2),
+ String.valueOf(true),
+ "2019-03-02",
+ "2019-02-12 03:03:34"
+ };
+ writer.write(row);
+ }
+ writer.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+
+ File segmentFolder = new File(CarbonTablePath.getSegmentPath(path, "null"));
+ Assert.assertTrue(segmentFolder.exists());
+
+ File[] dataFiles = segmentFolder.listFiles(new FileFilter() {
+ @Override public boolean accept(File pathname) {
+ return pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT);
+ }
+ });
+ Assert.assertNotNull(dataFiles);
+ Assert.assertTrue(dataFiles.length > 0);
+
+ FileUtils.deleteDirectory(new File(path));
+ }
+
+ @Test
+ public void test2Blocklet() throws IOException {
+ String path = "./testWriteFiles";
+ FileUtils.deleteDirectory(new File(path));
+
+ Field[] fields = new Field[2];
+ fields[0] = new Field("name", DataTypes.STRING);
+ fields[1] = new Field("age", DataTypes.INT);
+
+ writeFilesAndVerify(1000 * 1000, new Schema(fields), path, null, false, 1, 100);
+
+ // TODO: implement reader to verify the number of blocklet in the file
+
+ FileUtils.deleteDirectory(new File(path));
+ }
+
+ @Test
+ public void test2Block() throws IOException {
+ String path = "./testWriteFiles";
+ FileUtils.deleteDirectory(new File(path));
+
+ Field[] fields = new Field[2];
+ fields[0] = new Field("name", DataTypes.STRING);
+ fields[1] = new Field("age", DataTypes.INT);
+
+ writeFilesAndVerify(1000 * 1000, new Schema(fields), path, null, false, 2, 2);
+
+ File segmentFolder = new File(CarbonTablePath.getSegmentPath(path, "null"));
+ File[] dataFiles = segmentFolder.listFiles(new FileFilter() {
+ @Override public boolean accept(File pathname) {
+ return pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT);
+ }
+ });
+ Assert.assertNotNull(dataFiles);
+ Assert.assertEquals(2, dataFiles.length);
+
+ FileUtils.deleteDirectory(new File(path));
+ }
+
+ @Test
+ public void testSortColumns() throws IOException {
+ String path = "./testWriteFiles";
+ FileUtils.deleteDirectory(new File(path));
+
+ Field[] fields = new Field[2];
+ fields[0] = new Field("name", DataTypes.STRING);
+ fields[1] = new Field("age", DataTypes.INT);
+
+ writeFilesAndVerify(new Schema(fields), path, new String[]{"name"});
+
+ // TODO: implement reader and verify the data is sorted
+
+ FileUtils.deleteDirectory(new File(path));
+ }
+
+ @Test
+ public void testPartitionOutput() {
+ // TODO: test write data with partition
+ }
+
+ @Test
+ public void testSchemaPersistence() throws IOException {
+ String path = "./testWriteFiles";
+ FileUtils.deleteDirectory(new File(path));
+
+ Field[] fields = new Field[2];
+ fields[0] = new Field("name", DataTypes.STRING);
+ fields[1] = new Field("age", DataTypes.INT);
+
+ writeFilesAndVerify(new Schema(fields), path, true);
+
+ String schemaFile = CarbonTablePath.getSchemaFilePath(path);
+ Assert.assertTrue(new File(schemaFile).exists());
+
+ FileUtils.deleteDirectory(new File(path));
+ }
+
+}
[09/10] carbondata git commit: [CARBONDATA-2224][File Level Reader
Support] Refactoring of #2055
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableUsingSparkCarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableUsingSparkCarbonFileFormat.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableUsingSparkCarbonFileFormat.scala
index d284e50..f421d44 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableUsingSparkCarbonFileFormat.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableUsingSparkCarbonFileFormat.scala
@@ -113,11 +113,11 @@ class TestCreateTableUsingSparkCarbonFileFormat extends QueryTest with BeforeAnd
//data source file format
if (sqlContext.sparkContext.version.startsWith("2.1")) {
//data source file format
- sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ sql(s"""CREATE TABLE sdkOutputTable USING carbonfile OPTIONS (PATH '$filePath') """)
} else if (sqlContext.sparkContext.version.startsWith("2.2")) {
//data source file format
sql(
- s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ s"""CREATE TABLE sdkOutputTable USING carbonfile LOCATION
|'$filePath' """.stripMargin)
} else{
// TO DO
@@ -162,11 +162,11 @@ class TestCreateTableUsingSparkCarbonFileFormat extends QueryTest with BeforeAnd
if (sqlContext.sparkContext.version.startsWith("2.1")) {
//data source file format
- sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ sql(s"""CREATE TABLE sdkOutputTable USING carbonfile OPTIONS (PATH '$filePath') """)
} else if (sqlContext.sparkContext.version.startsWith("2.2")) {
//data source file format
sql(
- s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ s"""CREATE TABLE sdkOutputTable USING carbonfile LOCATION
|'$filePath' """.stripMargin)
} else{
// TO DO
@@ -184,6 +184,7 @@ class TestCreateTableUsingSparkCarbonFileFormat extends QueryTest with BeforeAnd
cleanTestData()
}
+ // TODO: Make the sparkCarbonFileFormat to work without index file
test("Read sdk writer output file without Carbondata file should fail") {
buildTestData(false)
deleteIndexFile(writerPath, CarbonCommonConstants.FACT_FILE_EXT)
@@ -194,11 +195,11 @@ class TestCreateTableUsingSparkCarbonFileFormat extends QueryTest with BeforeAnd
// data source file format
if (sqlContext.sparkContext.version.startsWith("2.1")) {
//data source file format
- sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ sql(s"""CREATE TABLE sdkOutputTable USING carbonfile OPTIONS (PATH '$filePath') """)
} else if (sqlContext.sparkContext.version.startsWith("2.2")) {
//data source file format
sql(
- s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ s"""CREATE TABLE sdkOutputTable USING carbonfile LOCATION
|'$filePath' """.stripMargin)
} else{
// TO DO
@@ -224,11 +225,11 @@ class TestCreateTableUsingSparkCarbonFileFormat extends QueryTest with BeforeAnd
//data source file format
if (sqlContext.sparkContext.version.startsWith("2.1")) {
//data source file format
- sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ sql(s"""CREATE TABLE sdkOutputTable USING carbonfile OPTIONS (PATH '$filePath') """)
} else if (sqlContext.sparkContext.version.startsWith("2.2")) {
//data source file format
sql(
- s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ s"""CREATE TABLE sdkOutputTable USING carbonfile LOCATION
|'$filePath' """.stripMargin)
} else{
// TO DO
@@ -254,11 +255,11 @@ class TestCreateTableUsingSparkCarbonFileFormat extends QueryTest with BeforeAnd
if (sqlContext.sparkContext.version.startsWith("2.1")) {
//data source file format
- sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ sql(s"""CREATE TABLE sdkOutputTable USING carbonfile OPTIONS (PATH '$filePath') """)
} else if (sqlContext.sparkContext.version.startsWith("2.2")) {
//data source file format
sql(
- s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ s"""CREATE TABLE sdkOutputTable USING carbonfile LOCATION
|'$filePath' """.stripMargin)
} else{
// TO DO
@@ -303,11 +304,11 @@ class TestCreateTableUsingSparkCarbonFileFormat extends QueryTest with BeforeAnd
if (sqlContext.sparkContext.version.startsWith("2.1")) {
//data source file format
- sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ sql(s"""CREATE TABLE sdkOutputTable USING carbonfile OPTIONS (PATH '$filePath') """)
} else if (sqlContext.sparkContext.version.startsWith("2.2")) {
//data source file format
sql(
- s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ s"""CREATE TABLE sdkOutputTable USING carbonfile LOCATION
|'$filePath' """.stripMargin)
} else{
// TO DO
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
index 9a46676..de91f2a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
@@ -132,11 +132,11 @@ object TestSparkCarbonFileFormatWithSparkSession {
//data source file format
if (spark.sparkContext.version.startsWith("2.1")) {
//data source file format
- spark.sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ spark.sql(s"""CREATE TABLE sdkOutputTable USING carbonfile OPTIONS (PATH '$filePath') """)
} else if (spark.sparkContext.version.startsWith("2.2")) {
//data source file format
spark.sql(
- s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ s"""CREATE TABLE sdkOutputTable USING carbonfile LOCATION
|'$filePath' """.stripMargin)
} else{
// TO DO
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
index 4378c15..2ba6e5e 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.command.CarbonMergerMapping
import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonMultiBlockSplit}
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
import org.apache.carbondata.processing.loading.model.CarbonLoadModel
import org.apache.carbondata.processing.merger.CarbonDataMergerUtil
@@ -63,9 +63,9 @@ class CarbonIUDMergerRDD[K, V](
val defaultParallelism = sparkContext.defaultParallelism
val noOfBlocks = 0
- CarbonTableInputFormat.setSegmentsToAccess(
+ CarbonInputFormat.setSegmentsToAccess(
job.getConfiguration, carbonMergerMapping.validSegments.toList.asJava)
- CarbonTableInputFormat.setTableInfo(
+ CarbonInputFormat.setTableInfo(
job.getConfiguration,
carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getTableInfo)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index da268c1..7ae2d14 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -48,7 +48,7 @@ import org.apache.carbondata.core.statusmanager.{FileFormat, SegmentUpdateStatus
import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonMultiBlockSplit}
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
import org.apache.carbondata.hadoop.util.{CarbonInputFormatUtil, CarbonInputSplitTaskInfo}
import org.apache.carbondata.processing.loading.TableProcessingOperations
import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -263,10 +263,10 @@ class CarbonMergerRDD[K, V](
SparkHadoopUtil.get.addCredentials(jobConf)
val job: Job = new Job(jobConf)
val format = CarbonInputFormatUtil.createCarbonInputFormat(absoluteTableIdentifier, job)
- CarbonTableInputFormat.setPartitionsToPrune(
+ CarbonInputFormat.setPartitionsToPrune(
job.getConfiguration,
carbonMergerMapping.currentPartitions.map(_.asJava).orNull)
- CarbonTableInputFormat.setTableInfo(job.getConfiguration,
+ CarbonInputFormat.setTableInfo(job.getConfiguration,
carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getTableInfo)
var updateDetails: UpdateVO = null
// initialise query_id for job
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
index 5647427..84f6659 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
@@ -45,7 +45,7 @@ import org.apache.carbondata.core.scan.result.iterator.PartitionSpliterRawResult
import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper
import org.apache.carbondata.core.util.{ByteUtil, DataTypeUtil}
import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonMultiBlockSplit}
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
import org.apache.carbondata.processing.merger.CarbonCompactionUtil
import org.apache.carbondata.processing.partition.spliter.CarbonSplitExecutor
@@ -96,7 +96,7 @@ class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
val job = new Job(jobConf)
val format = CarbonInputFormatUtil.createCarbonTableInputFormat(absoluteTableIdentifier,
partitionIds.toList.asJava, job)
- CarbonTableInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
+ CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
job.getConfiguration.set("query.id", queryId)
val splits = format.getSplitsOfOneSegment(job, segmentId,
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 6afd2c0..a9b8353 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -50,7 +50,7 @@ import org.apache.carbondata.core.stats.{QueryStatistic, QueryStatisticsConstant
import org.apache.carbondata.core.statusmanager.FileFormat
import org.apache.carbondata.core.util._
import org.apache.carbondata.hadoop._
-import org.apache.carbondata.hadoop.api.{CarbonFileInputFormat, CarbonTableInputFormat}
+import org.apache.carbondata.hadoop.api.{CarbonFileInputFormat, CarbonInputFormat, CarbonTableInputFormat}
import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
import org.apache.carbondata.processing.util.CarbonLoaderUtil
import org.apache.carbondata.spark.InitInputMetrics
@@ -90,7 +90,7 @@ class CarbonScanRDD(
val jobConf = new JobConf(conf)
SparkHadoopUtil.get.addCredentials(jobConf)
val job = Job.getInstance(jobConf)
- val fileLevelExternal = tableInfo.getFactTable().getTableProperties().get("_filelevelexternal")
+ val fileLevelExternal = tableInfo.getFactTable().getTableProperties().get("_filelevelformat")
val format = if (fileLevelExternal != null && fileLevelExternal.equalsIgnoreCase("true")) {
prepareFileInputFormatForDriver(job.getConfiguration)
} else {
@@ -432,53 +432,53 @@ class CarbonScanRDD(
}
def prepareInputFormatForDriver(conf: Configuration): CarbonTableInputFormat[Object] = {
- CarbonTableInputFormat.setTableInfo(conf, tableInfo)
- CarbonTableInputFormat.setDatabaseName(conf, tableInfo.getDatabaseName)
- CarbonTableInputFormat.setTableName(conf, tableInfo.getFactTable.getTableName)
+ CarbonInputFormat.setTableInfo(conf, tableInfo)
+ CarbonInputFormat.setDatabaseName(conf, tableInfo.getDatabaseName)
+ CarbonInputFormat.setTableName(conf, tableInfo.getFactTable.getTableName)
if (partitionNames != null) {
- CarbonTableInputFormat.setPartitionsToPrune(conf, partitionNames.asJava)
+ CarbonInputFormat.setPartitionsToPrune(conf, partitionNames.asJava)
}
createInputFormat(conf)
}
def prepareFileInputFormatForDriver(conf: Configuration): CarbonFileInputFormat[Object] = {
- CarbonFileInputFormat.setTableInfo(conf, tableInfo)
- CarbonFileInputFormat.setDatabaseName(conf, tableInfo.getDatabaseName)
- CarbonFileInputFormat.setTableName(conf, tableInfo.getFactTable.getTableName)
+ CarbonInputFormat.setTableInfo(conf, tableInfo)
+ CarbonInputFormat.setDatabaseName(conf, tableInfo.getDatabaseName)
+ CarbonInputFormat.setTableName(conf, tableInfo.getFactTable.getTableName)
if (partitionNames != null) {
- CarbonFileInputFormat.setPartitionsToPrune(conf, partitionNames.asJava)
+ CarbonInputFormat.setPartitionsToPrune(conf, partitionNames.asJava)
}
createFileInputFormat(conf)
}
- private def prepareInputFormatForExecutor(conf: Configuration): CarbonTableInputFormat[Object] = {
- CarbonTableInputFormat.setCarbonReadSupport(conf, readSupport)
+ private def prepareInputFormatForExecutor(conf: Configuration): CarbonInputFormat[Object] = {
+ CarbonInputFormat.setCarbonReadSupport(conf, readSupport)
val tableInfo1 = getTableInfo
- CarbonTableInputFormat.setTableInfo(conf, tableInfo1)
- CarbonTableInputFormat.setDatabaseName(conf, tableInfo1.getDatabaseName)
- CarbonTableInputFormat.setTableName(conf, tableInfo1.getFactTable.getTableName)
- CarbonTableInputFormat.setDataTypeConverter(conf, new SparkDataTypeConverterImpl)
+ CarbonInputFormat.setTableInfo(conf, tableInfo1)
+ CarbonInputFormat.setDatabaseName(conf, tableInfo1.getDatabaseName)
+ CarbonInputFormat.setTableName(conf, tableInfo1.getFactTable.getTableName)
+ CarbonInputFormat.setDataTypeConverter(conf, new SparkDataTypeConverterImpl)
createInputFormat(conf)
}
private def createFileInputFormat(conf: Configuration): CarbonFileInputFormat[Object] = {
val format = new CarbonFileInputFormat[Object]
- CarbonFileInputFormat.setTablePath(conf,
+ CarbonInputFormat.setTablePath(conf,
identifier.appendWithLocalPrefix(identifier.getTablePath))
- CarbonFileInputFormat.setQuerySegment(conf, identifier)
- CarbonFileInputFormat.setFilterPredicates(conf, filterExpression)
- CarbonFileInputFormat.setColumnProjection(conf, columnProjection)
- CarbonFileInputFormat.setDataMapJob(conf, new SparkDataMapJob)
+ CarbonInputFormat.setQuerySegment(conf, identifier)
+ CarbonInputFormat.setFilterPredicates(conf, filterExpression)
+ CarbonInputFormat.setColumnProjection(conf, columnProjection)
+ CarbonInputFormat.setDataMapJob(conf, new SparkDataMapJob)
if (CarbonProperties.getInstance().getProperty(
CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT).toBoolean) {
- CarbonTableInputFormat.setDataMapJob(conf, new SparkDataMapJob)
+ CarbonInputFormat.setDataMapJob(conf, new SparkDataMapJob)
}
// when validate segments is disabled in thread local update it to CarbonTableInputFormat
val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
if (carbonSessionInfo != null) {
- CarbonTableInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getSessionParams
+ CarbonInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getSessionParams
.getProperty(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
identifier.getCarbonTableIdentifier.getDatabaseName + "." +
identifier.getCarbonTableIdentifier.getTableName, "true").toBoolean)
@@ -489,22 +489,22 @@ class CarbonScanRDD(
private def createInputFormat(conf: Configuration): CarbonTableInputFormat[Object] = {
val format = new CarbonTableInputFormat[Object]
- CarbonTableInputFormat.setTablePath(conf,
+ CarbonInputFormat.setTablePath(conf,
identifier.appendWithLocalPrefix(identifier.getTablePath))
- CarbonTableInputFormat.setQuerySegment(conf, identifier)
- CarbonTableInputFormat.setFilterPredicates(conf, filterExpression)
- CarbonTableInputFormat.setColumnProjection(conf, columnProjection)
- CarbonTableInputFormat.setDataMapJob(conf, new SparkDataMapJob)
+ CarbonInputFormat.setQuerySegment(conf, identifier)
+ CarbonInputFormat.setFilterPredicates(conf, filterExpression)
+ CarbonInputFormat.setColumnProjection(conf, columnProjection)
+ CarbonInputFormat.setDataMapJob(conf, new SparkDataMapJob)
if (CarbonProperties.getInstance().getProperty(
CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT).toBoolean) {
- CarbonTableInputFormat.setDataMapJob(conf, new SparkDataMapJob)
+ CarbonInputFormat.setDataMapJob(conf, new SparkDataMapJob)
}
// when validate segments is disabled in thread local update it to CarbonTableInputFormat
val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
if (carbonSessionInfo != null) {
- CarbonTableInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getSessionParams
+ CarbonInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getSessionParams
.getProperty(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
identifier.getCarbonTableIdentifier.getDatabaseName + "." +
identifier.getCarbonTableIdentifier.getTableName, "true").toBoolean)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
index 1656efa..48ebdb4 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
@@ -36,7 +36,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
import org.apache.carbondata.core.util.CarbonUtil
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.hadoop.CarbonInputSplit
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
import org.apache.carbondata.processing.loading.model.CarbonLoadModel
import org.apache.carbondata.spark.util.CommonUtil
@@ -154,7 +154,7 @@ object PartitionUtils {
val job = new Job(jobConf)
val format = CarbonInputFormatUtil
.createCarbonTableInputFormat(identifier, partitionIds.asJava, job)
- CarbonTableInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
+ CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
val splits = format.getSplitsOfOneSegment(job, segmentId,
oldPartitionIdList.map(_.asInstanceOf[Integer]).asJava, partitionInfo)
val blockList = splits.asScala.map(_.asInstanceOf[CarbonInputSplit])
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
index c286c50..2d19fd4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
@@ -34,7 +34,7 @@ import org.apache.spark.sql.optimizer.CarbonFilters
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
import org.apache.carbondata.core.metadata.schema.table.CarbonTable
import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
case class CarbonCountStar(
attributesRaw: Seq[Attribute],
@@ -45,7 +45,7 @@ case class CarbonCountStar(
override def doExecute(): RDD[InternalRow] = {
val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
val (job, tableInputFormat) = createCarbonInputFormat(absoluteTableIdentifier)
- CarbonTableInputFormat.setQuerySegment(job.getConfiguration, absoluteTableIdentifier)
+ CarbonInputFormat.setQuerySegment(job.getConfiguration, absoluteTableIdentifier)
// get row count
val rowCount = CarbonUpdateUtil.getRowCount(
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
index 10d55ef..8eaeab1 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
@@ -43,7 +43,7 @@ import org.apache.carbondata.core.mutate.data.RowCountDetailsVO
import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentUpdateStatusManager}
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
import org.apache.carbondata.processing.exception.MultipleMatchingException
import org.apache.carbondata.processing.loading.FailureCauses
import org.apache.carbondata.spark.DeleteDelataResultImpl
@@ -90,7 +90,7 @@ object DeleteExecution {
}
val (carbonInputFormat, job) = createCarbonInputFormat(absoluteTableIdentifier)
- CarbonTableInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
+ CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
val keyRdd = deleteRdd.map({ row =>
val tupleId: String = row
.getString(row.fieldIndex(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
index cf22569..74da11a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
@@ -150,7 +150,7 @@ case class CarbonDropTableCommand(
// delete table data only if it is not external table
if (FileFactory.isFileExist(tablePath, fileType) &&
- !(carbonTable.isExternalTable || carbonTable.isFileLevelExternalTable)) {
+ !(carbonTable.isExternalTable || carbonTable.isFileLevelFormat)) {
val file = FileFactory.getCarbonFile(tablePath, fileType)
CarbonUtil.deleteFoldersAndFilesSilent(file)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonFileFormat.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonFileFormat.scala
index fa54e0d..2daece3 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonFileFormat.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonFileFormat.scala
@@ -51,7 +51,7 @@ import org.apache.carbondata.core.scan.model.QueryModel
import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonProjection, CarbonRecordReader, InputMetricsStats}
-import org.apache.carbondata.hadoop.api.{CarbonFileInputFormat, DataMapJob}
+import org.apache.carbondata.hadoop.api.{CarbonFileInputFormat, CarbonInputFormat, DataMapJob}
import org.apache.carbondata.spark.util.CarbonScalaUtil
@InterfaceAudience.User
@@ -105,9 +105,9 @@ class SparkCarbonFileFormat extends FileFormat
}
}
- override def shortName(): String = "Carbonfile"
+ override def shortName(): String = "carbonfile"
- override def toString: String = "Carbonfile"
+ override def toString: String = "carbonfile"
override def hashCode(): Int = getClass.hashCode()
@@ -179,10 +179,9 @@ class SparkCarbonFileFormat extends FileFormat
supportBatchValue = supportBatch(sparkSession, dataSchema)
}
- CarbonFileInputFormat.setTableName(job.getConfiguration, "externaldummy")
- CarbonFileInputFormat.setDatabaseName(job.getConfiguration, "default")
+ CarbonInputFormat.setTableName(job.getConfiguration, "externaldummy")
+ CarbonInputFormat.setDatabaseName(job.getConfiguration, "default")
CarbonMetadata.getInstance.removeTable("default_externaldummy")
- val dataMapJob: DataMapJob = CarbonFileInputFormat.getDataMapJob(job.getConfiguration)
val format: CarbonFileInputFormat[Object] = new CarbonFileInputFormat[Object]
(file: PartitionedFile) => {
@@ -207,9 +206,9 @@ class SparkCarbonFileFormat extends FileFormat
conf1.set("mapreduce.input.carboninputformat.tableName", "externaldummy")
conf1.set("mapreduce.input.carboninputformat.databaseName", "default")
conf1.set("mapreduce.input.fileinputformat.inputdir", tablePath)
- CarbonFileInputFormat.setColumnProjection(conf1, carbonProjection)
+ CarbonInputFormat.setColumnProjection(conf1, carbonProjection)
filter match {
- case Some(c) => CarbonFileInputFormat.setFilterPredicates(conf1, c)
+ case Some(c) => CarbonInputFormat.setFilterPredicates(conf1, c)
case None => None
}
val attemptContext = new TaskAttemptContextImpl(conf1, attemptId)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index d85ef68..b20349c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -112,7 +112,7 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
if (isCarbonTable) {
val carbonTable = CarbonEnv.getCarbonTable(alterTableChangeDataTypeModel.databaseName,
alterTableChangeDataTypeModel.tableName)(sparkSession)
- if (carbonTable != null && carbonTable.isFileLevelExternalTable) {
+ if (carbonTable != null && carbonTable.isFileLevelFormat) {
throw new MalformedCarbonCommandException(
"Unsupported alter operation on Carbon external fileformat table")
} else {
@@ -128,7 +128,7 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
if (isCarbonTable) {
val carbonTable = CarbonEnv.getCarbonTable(alterTableAddColumnsModel.databaseName,
alterTableAddColumnsModel.tableName)(sparkSession)
- if (carbonTable != null && carbonTable.isFileLevelExternalTable) {
+ if (carbonTable != null && carbonTable.isFileLevelFormat) {
throw new MalformedCarbonCommandException(
"Unsupported alter operation on Carbon external fileformat table")
} else {
@@ -144,7 +144,7 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
if (isCarbonTable) {
val carbonTable = CarbonEnv.getCarbonTable(alterTableDropColumnModel.databaseName,
alterTableDropColumnModel.tableName)(sparkSession)
- if (carbonTable != null && carbonTable.isFileLevelExternalTable) {
+ if (carbonTable != null && carbonTable.isFileLevelFormat) {
throw new MalformedCarbonCommandException(
"Unsupported alter operation on Carbon external fileformat table")
} else {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index 69fd366..55eb5ac 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -265,7 +265,7 @@ class CarbonHelperSqlAstBuilder(conf: SQLConf,
val table = try {
val schemaPath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath)
if (!FileFactory.isFileExist(schemaPath, FileFactory.getFileType(schemaPath)) &&
- provider.equalsIgnoreCase("'Carbonfile'")) {
+ provider.equalsIgnoreCase("'carbonfile'")) {
SchemaReader.inferSchema(identifier)
}
else {
@@ -277,12 +277,12 @@ class CarbonHelperSqlAstBuilder(conf: SQLConf,
operationNotAllowed(s"Invalid table path provided: ${tablePath.get} ", tableHeader)
}
// set "_external" property, so that DROP TABLE will not delete the data
- if (provider.equalsIgnoreCase("'Carbonfile'")) {
- table.getFactTable.getTableProperties.put("_filelevelexternal", "true")
+ if (provider.equalsIgnoreCase("'carbonfile'")) {
+ table.getFactTable.getTableProperties.put("_filelevelformat", "true")
table.getFactTable.getTableProperties.put("_external", "false")
} else {
table.getFactTable.getTableProperties.put("_external", "true")
- table.getFactTable.getTableProperties.put("_filelevelexternal", "false")
+ table.getFactTable.getTableProperties.put("_filelevelformat", "false")
}
table
} else {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
index 1f5808a..e8ae67b 100644
--- a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -326,7 +326,7 @@ class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSes
val fileStorage = helper.getFileStorage(ctx.createFileFormat)
if (fileStorage.equalsIgnoreCase("'carbondata'") ||
- fileStorage.equalsIgnoreCase("'Carbonfile'") ||
+ fileStorage.equalsIgnoreCase("'carbonfile'") ||
fileStorage.equalsIgnoreCase("'org.apache.carbondata.format'")) {
val createTableTuple = (ctx.createTableHeader, ctx.skewSpec, ctx.bucketSpec,
ctx.partitionColumns, ctx.columns, ctx.tablePropertyList, ctx.locationSpec(),
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
index c28e4ba..3cb9bd6 100644
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -325,7 +325,7 @@ class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSes
val fileStorage = helper.getFileStorage(ctx.createFileFormat)
if (fileStorage.equalsIgnoreCase("'carbondata'") ||
- fileStorage.equalsIgnoreCase("'Carbonfile'") ||
+ fileStorage.equalsIgnoreCase("'carbonfile'") ||
fileStorage.equalsIgnoreCase("'org.apache.carbondata.format'")) {
val createTableTuple = (ctx.createTableHeader, ctx.skewSpec,
ctx.bucketSpec, ctx.partitionColumns, ctx.columns, ctx.tablePropertyList,ctx.locationSpec(),
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b384b6e1/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
index 47ef5f2..8ab658d 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
@@ -41,7 +41,7 @@ import org.apache.carbondata.core.util.CarbonUtil
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonProjection}
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostStatusUpdateEvent, LoadTablePreStatusUpdateEvent}
import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -145,16 +145,16 @@ class StreamHandoffRDD[K, V](
val inputSplit = split.asInstanceOf[HandoffPartition].split.value
val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, split.index, 0)
val hadoopConf = new Configuration()
- CarbonTableInputFormat.setDatabaseName(hadoopConf, carbonTable.getDatabaseName)
- CarbonTableInputFormat.setTableName(hadoopConf, carbonTable.getTableName)
- CarbonTableInputFormat.setTablePath(hadoopConf, carbonTable.getTablePath)
+ CarbonInputFormat.setDatabaseName(hadoopConf, carbonTable.getDatabaseName)
+ CarbonInputFormat.setTableName(hadoopConf, carbonTable.getTableName)
+ CarbonInputFormat.setTablePath(hadoopConf, carbonTable.getTablePath)
val projection = new CarbonProjection
val dataFields = carbonTable.getStreamStorageOrderColumn(carbonTable.getTableName)
(0 until dataFields.size()).foreach { index =>
projection.addColumn(dataFields.get(index).getColName)
}
- CarbonTableInputFormat.setColumnProjection(hadoopConf, projection)
- CarbonTableInputFormat.setTableInfo(hadoopConf, carbonTable.getTableInfo)
+ CarbonInputFormat.setColumnProjection(hadoopConf, projection)
+ CarbonInputFormat.setTableInfo(hadoopConf, carbonTable.getTableInfo)
val attemptContext = new TaskAttemptContextImpl(hadoopConf, attemptId)
val format = new CarbonTableInputFormat[Array[Object]]()
val model = format.createQueryModel(inputSplit, attemptContext)
[07/10] carbondata git commit: [CARBONDATA-2224][File Level Reader
Support] External File level reader support
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
index 0298eea..cf22569 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
@@ -150,7 +150,7 @@ case class CarbonDropTableCommand(
// delete table data only if it is not external table
if (FileFactory.isFileExist(tablePath, fileType) &&
- !carbonTable.isExternalTable) {
+ !(carbonTable.isExternalTable || carbonTable.isFileLevelExternalTable)) {
val file = FileFactory.getCarbonFile(tablePath, fileType)
CarbonUtil.deleteFoldersAndFilesSilent(file)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
deleted file mode 100644
index 2eed988..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
+++ /dev/null
@@ -1,443 +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.spark.sql.execution.datasources
-
-import java.io.File
-import java.util
-import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.atomic.AtomicLong
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.util.Random
-
-import org.apache.hadoop.fs.{FileStatus, Path}
-import org.apache.hadoop.io.NullWritable
-import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}
-import org.apache.spark.SparkEnv
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
-import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.sources.DataSourceRegister
-import org.apache.spark.sql.types._
-
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.indexstore
-import org.apache.carbondata.core.metadata.SegmentFileStore
-import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
-import org.apache.carbondata.core.util.{CarbonProperties, DataTypeConverterImpl, DataTypeUtil}
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.hadoop.api.{CarbonOutputCommitter, CarbonTableOutputFormat}
-import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat.CarbonRecordWriter
-import org.apache.carbondata.hadoop.internal.ObjectArrayWritable
-import org.apache.carbondata.hadoop.util.ObjectSerializationUtil
-import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, Util}
-
-class CarbonFileFormat
- extends FileFormat
- with DataSourceRegister
- with Logging
-with Serializable {
-
- override def shortName(): String = "carbondata"
-
- override def inferSchema(sparkSession: SparkSession,
- options: Map[String, String],
- files: Seq[FileStatus]): Option[StructType] = {
- None
- }
-
- SparkSession.getActiveSession.get.sessionState.conf.setConfString(
- "spark.sql.sources.commitProtocolClass",
- "org.apache.spark.sql.execution.datasources.CarbonSQLHadoopMapReduceCommitProtocol")
-
- override def prepareWrite(
- sparkSession: SparkSession,
- job: Job,
- options: Map[String, String],
- dataSchema: StructType): OutputWriterFactory = {
- val conf = job.getConfiguration
- conf.setClass(
- SQLConf.OUTPUT_COMMITTER_CLASS.key,
- classOf[CarbonOutputCommitter],
- classOf[CarbonOutputCommitter])
- conf.set("carbon.commit.protocol", "carbon.commit.protocol")
- job.setOutputFormatClass(classOf[CarbonTableOutputFormat])
- val table = CarbonEnv.getCarbonTable(
- TableIdentifier(options("tableName"), options.get("dbName")))(sparkSession)
- val model = new CarbonLoadModel
- val carbonProperty = CarbonProperties.getInstance()
- val optionsFinal = LoadOption.fillOptionWithDefaultValue(options.asJava)
- val tableProperties = table.getTableInfo.getFactTable.getTableProperties
- optionsFinal.put("sort_scope", tableProperties.asScala.getOrElse("sort_scope",
- carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
- carbonProperty.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
- CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))))
- val partitionStr =
- table.getTableInfo.getFactTable.getPartitionInfo.getColumnSchemaList.asScala.map(
- _.getColumnName.toLowerCase).mkString(",")
- optionsFinal.put(
- "fileheader",
- dataSchema.fields.map(_.name.toLowerCase).mkString(",") + "," + partitionStr)
- val optionsLocal = new mutable.HashMap[String, String]()
- optionsLocal ++= options
- optionsLocal += (("header", "false"))
- new CarbonLoadModelBuilder(table).build(
- optionsLocal.toMap.asJava,
- optionsFinal,
- model,
- conf)
- model.setUseOnePass(options.getOrElse("onepass", "false").toBoolean)
- model.setDictionaryServerHost(options.getOrElse("dicthost", null))
- model.setDictionaryServerPort(options.getOrElse("dictport", "-1").toInt)
- CarbonTableOutputFormat.setOverwrite(conf, options("overwrite").toBoolean)
- model.setPartitionLoad(true)
-
- val staticPartition = options.getOrElse("staticpartition", null)
- if (staticPartition != null) {
- conf.set("carbon.staticpartition", staticPartition)
- }
- // In case of update query there is chance to remove the older segments, so here we can set
- // the to be deleted segments to mark as delete while updating tablestatus
- val segemntsTobeDeleted = options.get("segmentsToBeDeleted")
- if (segemntsTobeDeleted.isDefined) {
- conf.set(CarbonTableOutputFormat.SEGMENTS_TO_BE_DELETED, segemntsTobeDeleted.get)
- }
-
- val currPartition = options.getOrElse("currentpartition", null)
- if (currPartition != null) {
- conf.set("carbon.currentpartition", currPartition)
- }
- // Update with the current in progress load.
- val currEntry = options.getOrElse("currentloadentry", null)
- if (currEntry != null) {
- val loadEntry =
- ObjectSerializationUtil.convertStringToObject(currEntry).asInstanceOf[LoadMetadataDetails]
- val details =
- SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(model.getTablePath))
- model.setSegmentId(loadEntry.getLoadName)
- model.setFactTimeStamp(loadEntry.getLoadStartTime)
- val list = new util.ArrayList[LoadMetadataDetails](details.toList.asJava)
- list.add(loadEntry)
- model.setLoadMetadataDetails(list)
- }
- // Set the update timestamp if user sets in case of update query. It needs to be updated
- // in load status update time
- val updateTimeStamp = options.get("updatetimestamp")
- if (updateTimeStamp.isDefined) {
- conf.set(CarbonTableOutputFormat.UPADTE_TIMESTAMP, updateTimeStamp.get)
- }
- CarbonTableOutputFormat.setLoadModel(conf, model)
-
- new OutputWriterFactory {
-
- /**
- * counter used for generating task numbers. This is used to generate unique partition numbers
- * in case of partitioning
- */
- val counter = new AtomicLong()
- val taskIdMap = new ConcurrentHashMap[String, java.lang.Long]()
-
- override def newInstance(
- path: String,
- dataSchema: StructType,
- context: TaskAttemptContext): OutputWriter = {
- val model = CarbonTableOutputFormat.getLoadModel(context.getConfiguration)
- val isCarbonUseMultiDir = CarbonProperties.getInstance().isUseMultiTempDir
- var storeLocation: Array[String] = Array[String]()
- val isCarbonUseLocalDir = CarbonProperties.getInstance()
- .getProperty("carbon.use.local.dir", "false").equalsIgnoreCase("true")
-
-
- val taskNumber = generateTaskNumber(path, context, model.getSegmentId)
- val tmpLocationSuffix =
- File.separator + "carbon" + System.nanoTime() + File.separator + taskNumber
- if (isCarbonUseLocalDir) {
- val yarnStoreLocations = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
- if (!isCarbonUseMultiDir && null != yarnStoreLocations && yarnStoreLocations.nonEmpty) {
- // use single dir
- storeLocation = storeLocation :+
- (yarnStoreLocations(Random.nextInt(yarnStoreLocations.length)) + tmpLocationSuffix)
- if (storeLocation == null || storeLocation.isEmpty) {
- storeLocation = storeLocation :+
- (System.getProperty("java.io.tmpdir") + tmpLocationSuffix)
- }
- } else {
- // use all the yarn dirs
- storeLocation = yarnStoreLocations.map(_ + tmpLocationSuffix)
- }
- } else {
- storeLocation =
- storeLocation :+ (System.getProperty("java.io.tmpdir") + tmpLocationSuffix)
- }
- CarbonTableOutputFormat.setTempStoreLocations(context.getConfiguration, storeLocation)
- new CarbonOutputWriter(path, context, dataSchema.map(_.dataType), taskNumber, model)
- }
-
- /**
- * Generate taskid using the taskid of taskcontext and the path. It should be unique in case
- * of partition tables.
- */
- private def generateTaskNumber(path: String,
- context: TaskAttemptContext, segmentId: String): String = {
- var partitionNumber: java.lang.Long = taskIdMap.get(path)
- if (partitionNumber == null) {
- partitionNumber = counter.incrementAndGet()
- // Generate taskid using the combination of taskid and partition number to make it unique.
- taskIdMap.put(path, partitionNumber)
- }
- val taskID = context.getTaskAttemptID.getTaskID.getId
- CarbonScalaUtil.generateUniqueNumber(taskID, segmentId, partitionNumber)
- }
-
- override def getFileExtension(context: TaskAttemptContext): String = {
- CarbonTablePath.CARBON_DATA_EXT
- }
-
- }
- }
-}
-
-case class CarbonSQLHadoopMapReduceCommitProtocol(jobId: String, path: String, isAppend: Boolean)
- extends SQLHadoopMapReduceCommitProtocol(jobId, path, isAppend) {
- override def newTaskTempFileAbsPath(taskContext: TaskAttemptContext,
- absoluteDir: String,
- ext: String): String = {
- val carbonFlow = taskContext.getConfiguration.get("carbon.commit.protocol")
- if (carbonFlow != null) {
- super.newTaskTempFile(taskContext, Some(absoluteDir), ext)
- } else {
- super.newTaskTempFileAbsPath(taskContext, absoluteDir, ext)
- }
- }
-}
-
-/**
- * It is a just class to make compile between spark 2.1 and 2.2
- */
-private trait AbstractCarbonOutputWriter {
- def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal")
- def writeInternal(row: InternalRow): Unit = {
- writeCarbon(row)
- }
- def write(row: InternalRow): Unit = {
- writeCarbon(row)
- }
- def writeCarbon(row: InternalRow): Unit
-}
-
-private class CarbonOutputWriter(path: String,
- context: TaskAttemptContext,
- fieldTypes: Seq[DataType],
- taskNo : String,
- model: CarbonLoadModel)
- extends OutputWriter with AbstractCarbonOutputWriter {
-
- val converter = new DataTypeConverterImpl
-
- val partitions =
- getPartitionsFromPath(path, context, model).map(ExternalCatalogUtils.unescapePathName)
- val staticPartition: util.HashMap[String, Boolean] = {
- val staticPart = context.getConfiguration.get("carbon.staticpartition")
- if (staticPart != null) {
- ObjectSerializationUtil.convertStringToObject(
- staticPart).asInstanceOf[util.HashMap[String, Boolean]]
- } else {
- null
- }
- }
- lazy val currPartitions: util.List[indexstore.PartitionSpec] = {
- val currParts = context.getConfiguration.get("carbon.currentpartition")
- if (currParts != null) {
- ObjectSerializationUtil.convertStringToObject(
- currParts).asInstanceOf[util.List[indexstore.PartitionSpec]]
- } else {
- new util.ArrayList[indexstore.PartitionSpec]()
- }
- }
- var (updatedPartitions, partitionData) = if (partitions.nonEmpty) {
- val updatedPartitions = partitions.map(splitPartition)
- (updatedPartitions, updatePartitions(updatedPartitions.map(_._2)))
- } else {
- (Map.empty[String, String].toArray, Array.empty)
- }
-
- private def splitPartition(p: String) = {
- val value = p.substring(p.indexOf("=") + 1, p.length)
- val col = p.substring(0, p.indexOf("="))
- // NUll handling case. For null hive creates with this special name
- if (value.equals("__HIVE_DEFAULT_PARTITION__")) {
- (col, null)
- // we should replace back the special string with empty value.
- } else if (value.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
- (col, "")
- } else {
- (col, value)
- }
- }
-
- lazy val writePath = {
- val updatedPath = getPartitionPath(path, context, model)
- // in case of partition location specified by user then search the partitions from the current
- // partitions to get the corresponding partitions.
- if (partitions.isEmpty) {
- val writeSpec = new indexstore.PartitionSpec(null, updatedPath)
- val index = currPartitions.indexOf(writeSpec)
- if (index > -1) {
- val spec = currPartitions.get(index)
- updatedPartitions = spec.getPartitions.asScala.map(splitPartition).toArray
- partitionData = updatePartitions(updatedPartitions.map(_._2))
- }
- }
- updatedPath
- }
-
- val writable = new ObjectArrayWritable
-
- private def updatePartitions(partitionData: Seq[String]): Array[AnyRef] = {
- model.getCarbonDataLoadSchema.getCarbonTable.getTableInfo.getFactTable.getPartitionInfo
- .getColumnSchemaList.asScala.zipWithIndex.map { case (col, index) =>
-
- val dataType = if (col.hasEncoding(Encoding.DICTIONARY)) {
- DataTypes.INT
- } else if (col.getDataType.equals(DataTypes.TIMESTAMP) ||
- col.getDataType.equals(DataTypes.DATE)) {
- DataTypes.LONG
- } else {
- col.getDataType
- }
- if (staticPartition != null && staticPartition.get(col.getColumnName.toLowerCase)) {
- val converetedVal =
- CarbonScalaUtil.convertStaticPartitions(
- partitionData(index),
- col,
- model.getCarbonDataLoadSchema.getCarbonTable)
- if (col.hasEncoding(Encoding.DICTIONARY)) {
- converetedVal.toInt.asInstanceOf[AnyRef]
- } else {
- DataTypeUtil.getDataBasedOnDataType(
- converetedVal,
- dataType,
- converter)
- }
- } else {
- DataTypeUtil.getDataBasedOnDataType(partitionData(index), dataType, converter)
- }
- }.toArray
- }
-
- private val recordWriter: CarbonRecordWriter = {
- context.getConfiguration.set("carbon.outputformat.taskno", taskNo)
- context.getConfiguration.set("carbon.outputformat.writepath",
- writePath + "/" + model.getSegmentId + "_" + model.getFactTimeStamp + ".tmp")
- new CarbonTableOutputFormat() {
- override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = {
- new Path(path)
- }
- }.getRecordWriter(context).asInstanceOf[CarbonRecordWriter]
- }
-
- // TODO Implement writesupport interface to support writing Row directly to recordwriter
- def writeCarbon(row: InternalRow): Unit = {
- val data = new Array[AnyRef](fieldTypes.length + partitionData.length)
- var i = 0
- while (i < fieldTypes.length) {
- if (!row.isNullAt(i)) {
- fieldTypes(i) match {
- case StringType =>
- data(i) = row.getString(i)
- case d: DecimalType =>
- data(i) = row.getDecimal(i, d.precision, d.scale).toJavaBigDecimal
- case other =>
- data(i) = row.get(i, other)
- }
- }
- i += 1
- }
- if (partitionData.length > 0) {
- System.arraycopy(partitionData, 0, data, fieldTypes.length, partitionData.length)
- }
- writable.set(data)
- recordWriter.write(NullWritable.get(), writable)
- }
-
-
- override def writeInternal(row: InternalRow): Unit = {
- writeCarbon(row)
- }
-
- override def close(): Unit = {
- recordWriter.close(context)
- // write partition info to new file.
- val partitonList = new util.ArrayList[String]()
- val formattedPartitions =
- // All dynamic partitions need to be converted to proper format
- CarbonScalaUtil.updatePartitions(
- updatedPartitions.toMap,
- model.getCarbonDataLoadSchema.getCarbonTable)
- formattedPartitions.foreach(p => partitonList.add(p._1 + "=" + p._2))
- SegmentFileStore.writeSegmentFile(
- model.getTablePath,
- taskNo,
- writePath,
- model.getSegmentId + "_" + model.getFactTimeStamp + "",
- partitonList)
- }
-
- def getPartitionPath(path: String,
- attemptContext: TaskAttemptContext,
- model: CarbonLoadModel): String = {
- if (updatedPartitions.nonEmpty) {
- val formattedPartitions =
- // All dynamic partitions need to be converted to proper format
- CarbonScalaUtil.updatePartitions(
- updatedPartitions.toMap,
- model.getCarbonDataLoadSchema.getCarbonTable)
- val partitionstr = formattedPartitions.map{p =>
- ExternalCatalogUtils.escapePathName(p._1) + "=" + ExternalCatalogUtils.escapePathName(p._2)
- }.mkString(CarbonCommonConstants.FILE_SEPARATOR)
- model.getCarbonDataLoadSchema.getCarbonTable.getTablePath +
- CarbonCommonConstants.FILE_SEPARATOR + partitionstr
- } else {
- var updatedPath = FileFactory.getUpdatedFilePath(path)
- updatedPath.substring(0, updatedPath.lastIndexOf("/"))
- }
- }
-
- def getPartitionsFromPath(
- path: String,
- attemptContext: TaskAttemptContext,
- model: CarbonLoadModel): Array[String] = {
- var attemptId = attemptContext.getTaskAttemptID.toString + "/"
- if (path.indexOf(attemptId) > -1) {
- val str = path.substring(path.indexOf(attemptId) + attemptId.length, path.lastIndexOf("/"))
- if (str.length > 0) {
- str.split("/")
- } else {
- Array.empty
- }
- } else {
- Array.empty
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonFileFormat.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonFileFormat.scala
new file mode 100644
index 0000000..fa54e0d
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonFileFormat.scala
@@ -0,0 +1,269 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.mapreduce._
+import org.apache.hadoop.mapreduce.lib.input.FileSplit
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.spark.{SparkException, TaskContext}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.execution.datasources.text.TextOutputWriter
+import org.apache.spark.sql.optimizer.CarbonFilters
+import org.apache.spark.sql.sources.{DataSourceRegister, Filter}
+import org.apache.spark.sql.types.{AtomicType, StructField, StructType}
+
+import org.apache.carbondata.common.annotations.{InterfaceAudience, InterfaceStability}
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.{DataMapChooser, DataMapStoreManager, Segment}
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, ColumnarFormatVersion}
+import org.apache.carbondata.core.reader.CarbonHeaderReader
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.logical.AndExpression
+import org.apache.carbondata.core.scan.model.QueryModel
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonProjection, CarbonRecordReader, InputMetricsStats}
+import org.apache.carbondata.hadoop.api.{CarbonFileInputFormat, DataMapJob}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+@InterfaceAudience.User
+@InterfaceStability.Evolving
+class SparkCarbonFileFormat extends FileFormat
+ with DataSourceRegister
+ with Logging
+ with Serializable {
+
+ @transient val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+
+ override def inferSchema(sparkSession: SparkSession,
+ options: Map[String, String],
+ files: Seq[FileStatus]): Option[StructType] = {
+ val filePaths = CarbonUtil.getFilePathExternalFilePath(
+ options.get("path").get)
+ if (filePaths.size() == 0) {
+ throw new SparkException("CarbonData file is not present in the location mentioned in DDL")
+ }
+ val carbonHeaderReader: CarbonHeaderReader = new CarbonHeaderReader(filePaths.get(0))
+ val fileHeader = carbonHeaderReader.readHeader
+ val table_columns: java.util.List[org.apache.carbondata.format.ColumnSchema] = fileHeader
+ .getColumn_schema
+ var colArray = ArrayBuffer[StructField]()
+ for (i <- 0 to table_columns.size() - 1) {
+ val col = CarbonUtil.thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i))
+ colArray += (new StructField(col.getColumnName,
+ CarbonScalaUtil.convertCarbonToSparkDataType(col.getDataType), false))
+ }
+ colArray.+:(Nil)
+
+ Some(StructType(colArray))
+ }
+
+ override def prepareWrite(sparkSession: SparkSession,
+ job: Job,
+ options: Map[String, String],
+ dataSchema: StructType): OutputWriterFactory = {
+
+ new OutputWriterFactory {
+ override def newInstance(
+ path: String,
+ dataSchema: StructType,
+ context: TaskAttemptContext): OutputWriter = {
+ new TextOutputWriter(path, dataSchema, context)
+ }
+
+ override def getFileExtension(context: TaskAttemptContext): String = {
+ CarbonTablePath.CARBON_DATA_EXT
+ }
+ }
+ }
+
+ override def shortName(): String = "Carbonfile"
+
+ override def toString: String = "Carbonfile"
+
+ override def hashCode(): Int = getClass.hashCode()
+
+ override def equals(other: Any): Boolean = other.isInstanceOf[SparkCarbonFileFormat]
+
+ def supportVector(sparkSession: SparkSession, schema: StructType): Boolean = {
+ val vectorizedReader = {
+ if (sparkSession.sqlContext.sparkSession.conf
+ .contains(CarbonCommonConstants.ENABLE_VECTOR_READER)) {
+ sparkSession.sqlContext.sparkSession.conf.get(CarbonCommonConstants.ENABLE_VECTOR_READER)
+ } else if (System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER) != null) {
+ System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER)
+ } else {
+ CarbonProperties.getInstance().getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER,
+ CarbonCommonConstants.ENABLE_VECTOR_READER_DEFAULT)
+ }
+ }
+ vectorizedReader.toBoolean
+ }
+
+
+ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = {
+ val conf = sparkSession.sessionState.conf
+ conf.wholeStageEnabled &&
+ schema.length <= conf.wholeStageMaxNumFields &&
+ schema.forall(_.dataType.isInstanceOf[AtomicType])
+ }
+
+
+ def createVectorizedCarbonRecordReader(queryModel: QueryModel,
+ inputMetricsStats: InputMetricsStats, enableBatch: String): RecordReader[Void, Object] = {
+ val name = "org.apache.carbondata.spark.vectorreader.VectorizedCarbonRecordReader"
+ try {
+ val cons = Class.forName(name).getDeclaredConstructors
+ cons.head.setAccessible(true)
+ cons.head.newInstance(queryModel, inputMetricsStats, enableBatch)
+ .asInstanceOf[RecordReader[Void, Object]]
+ } catch {
+ case e: Exception =>
+ LOGGER.error(e)
+ null
+ }
+ }
+
+ override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+ dataSchema: StructType,
+ partitionSchema: StructType,
+ requiredSchema: StructType,
+ filters: Seq[Filter],
+ options: Map[String, String],
+ hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
+
+ val filter : Option[Expression] = filters.flatMap { filter =>
+ CarbonFilters.createCarbonFilter(dataSchema, filter)
+ }.reduceOption(new AndExpression(_, _))
+
+ val projection = requiredSchema.map(_.name).toArray
+ val carbonProjection = new CarbonProjection
+ projection.foreach(carbonProjection.addColumn)
+
+ val conf = new Configuration()
+ val jobConf = new JobConf(conf)
+ SparkHadoopUtil.get.addCredentials(jobConf)
+ val job = Job.getInstance(jobConf)
+ var supportBatchValue: Boolean = false
+
+ val readVector = supportVector(sparkSession, dataSchema)
+ if (readVector) {
+ supportBatchValue = supportBatch(sparkSession, dataSchema)
+ }
+
+ CarbonFileInputFormat.setTableName(job.getConfiguration, "externaldummy")
+ CarbonFileInputFormat.setDatabaseName(job.getConfiguration, "default")
+ CarbonMetadata.getInstance.removeTable("default_externaldummy")
+ val dataMapJob: DataMapJob = CarbonFileInputFormat.getDataMapJob(job.getConfiguration)
+ val format: CarbonFileInputFormat[Object] = new CarbonFileInputFormat[Object]
+
+ (file: PartitionedFile) => {
+ assert(file.partitionValues.numFields == partitionSchema.size)
+
+ if (file.filePath.endsWith(CarbonCommonConstants.FACT_FILE_EXT)) {
+ val fileSplit =
+ new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, Array.empty)
+
+ val path: String = options.get("path").get
+ val endindex: Int = path.indexOf("Fact") - 1
+ val tablePath = path.substring(0, endindex)
+ lazy val identifier: AbsoluteTableIdentifier = AbsoluteTableIdentifier.from(
+ tablePath,
+ "default",
+ "externaldummy")
+ val split = CarbonInputSplit.from("null", "0", fileSplit, ColumnarFormatVersion.V3, null)
+
+
+ val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
+ val conf1 = new Configuration()
+ conf1.set("mapreduce.input.carboninputformat.tableName", "externaldummy")
+ conf1.set("mapreduce.input.carboninputformat.databaseName", "default")
+ conf1.set("mapreduce.input.fileinputformat.inputdir", tablePath)
+ CarbonFileInputFormat.setColumnProjection(conf1, carbonProjection)
+ filter match {
+ case Some(c) => CarbonFileInputFormat.setFilterPredicates(conf1, c)
+ case None => None
+ }
+ val attemptContext = new TaskAttemptContextImpl(conf1, attemptId)
+
+ val model = format.createQueryModel(split, attemptContext)
+
+ var segments = new java.util.ArrayList[Segment]()
+ val seg = new Segment("null", null)
+ segments.add(seg)
+ var partition : java.util.List[PartitionSpec] = new java.util.ArrayList[PartitionSpec]()
+
+
+ val segmentPath = CarbonTablePath.getSegmentPath(identifier.getTablePath(), "null")
+ val indexFiles = new SegmentIndexFileStore().getIndexFilesFromSegment(segmentPath)
+ if (indexFiles.size() == 0) {
+ throw new SparkException("Index file not present to read the carbondata file")
+ }
+
+ val tab = model.getTable
+ DataMapStoreManager.getInstance().clearDataMaps(identifier)
+ val dataMapExprWrapper = DataMapChooser.get
+ .choose(tab, model.getFilterExpressionResolverTree)
+
+ // TODO : handle the partition for CarbonFileLevelFormat
+ val prunedBlocklets = dataMapExprWrapper.prune(segments, null)
+
+ val detailInfo = prunedBlocklets.get(0).getDetailInfo
+ detailInfo.readColumnSchema(detailInfo.getColumnSchemaBinary)
+ split.setDetailInfo(detailInfo)
+
+ val carbonReader = if (readVector) {
+ val vectorizedReader = createVectorizedCarbonRecordReader(model,
+ null,
+ supportBatchValue.toString)
+ vectorizedReader.initialize(split, attemptContext)
+ logDebug(s"Appending $partitionSchema ${ file.partitionValues }")
+ vectorizedReader
+ } else {
+ val reader = new CarbonRecordReader(model,
+ format.getReadSupportClass(attemptContext.getConfiguration), null)
+ reader.initialize(split, attemptContext)
+ reader
+ }
+
+ val iter = new RecordReaderIterator(carbonReader)
+ Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close()))
+
+ iter.asInstanceOf[Iterator[InternalRow]]
+ }
+ else {
+ Iterator.empty
+ }
+ }
+ }
+}
+
+
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala
new file mode 100644
index 0000000..d34b201
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/SparkCarbonTableFormat.scala
@@ -0,0 +1,443 @@
+/*
+ * 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.spark.sql.execution.datasources
+
+import java.io.File
+import java.util
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicLong
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.util.Random
+
+import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.hadoop.io.NullWritable
+import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.DataSourceRegister
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.indexstore
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.core.util.{CarbonProperties, DataTypeConverterImpl, DataTypeUtil}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.hadoop.api.{CarbonOutputCommitter, CarbonTableOutputFormat}
+import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat.CarbonRecordWriter
+import org.apache.carbondata.hadoop.internal.ObjectArrayWritable
+import org.apache.carbondata.hadoop.util.ObjectSerializationUtil
+import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, Util}
+
+class SparkCarbonTableFormat
+ extends FileFormat
+ with DataSourceRegister
+ with Logging
+with Serializable {
+
+ override def shortName(): String = "carbondata"
+
+ override def inferSchema(sparkSession: SparkSession,
+ options: Map[String, String],
+ files: Seq[FileStatus]): Option[StructType] = {
+ None
+ }
+
+ SparkSession.getActiveSession.get.sessionState.conf.setConfString(
+ "spark.sql.sources.commitProtocolClass",
+ "org.apache.spark.sql.execution.datasources.CarbonSQLHadoopMapReduceCommitProtocol")
+
+ override def prepareWrite(
+ sparkSession: SparkSession,
+ job: Job,
+ options: Map[String, String],
+ dataSchema: StructType): OutputWriterFactory = {
+ val conf = job.getConfiguration
+ conf.setClass(
+ SQLConf.OUTPUT_COMMITTER_CLASS.key,
+ classOf[CarbonOutputCommitter],
+ classOf[CarbonOutputCommitter])
+ conf.set("carbon.commit.protocol", "carbon.commit.protocol")
+ job.setOutputFormatClass(classOf[CarbonTableOutputFormat])
+ val table = CarbonEnv.getCarbonTable(
+ TableIdentifier(options("tableName"), options.get("dbName")))(sparkSession)
+ val model = new CarbonLoadModel
+ val carbonProperty = CarbonProperties.getInstance()
+ val optionsFinal = LoadOption.fillOptionWithDefaultValue(options.asJava)
+ val tableProperties = table.getTableInfo.getFactTable.getTableProperties
+ optionsFinal.put("sort_scope", tableProperties.asScala.getOrElse("sort_scope",
+ carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
+ carbonProperty.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
+ CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))))
+ val partitionStr =
+ table.getTableInfo.getFactTable.getPartitionInfo.getColumnSchemaList.asScala.map(
+ _.getColumnName.toLowerCase).mkString(",")
+ optionsFinal.put(
+ "fileheader",
+ dataSchema.fields.map(_.name.toLowerCase).mkString(",") + "," + partitionStr)
+ val optionsLocal = new mutable.HashMap[String, String]()
+ optionsLocal ++= options
+ optionsLocal += (("header", "false"))
+ new CarbonLoadModelBuilder(table).build(
+ optionsLocal.toMap.asJava,
+ optionsFinal,
+ model,
+ conf)
+ model.setUseOnePass(options.getOrElse("onepass", "false").toBoolean)
+ model.setDictionaryServerHost(options.getOrElse("dicthost", null))
+ model.setDictionaryServerPort(options.getOrElse("dictport", "-1").toInt)
+ CarbonTableOutputFormat.setOverwrite(conf, options("overwrite").toBoolean)
+ model.setPartitionLoad(true)
+
+ val staticPartition = options.getOrElse("staticpartition", null)
+ if (staticPartition != null) {
+ conf.set("carbon.staticpartition", staticPartition)
+ }
+ // In case of update query there is chance to remove the older segments, so here we can set
+ // the to be deleted segments to mark as delete while updating tablestatus
+ val segemntsTobeDeleted = options.get("segmentsToBeDeleted")
+ if (segemntsTobeDeleted.isDefined) {
+ conf.set(CarbonTableOutputFormat.SEGMENTS_TO_BE_DELETED, segemntsTobeDeleted.get)
+ }
+
+ val currPartition = options.getOrElse("currentpartition", null)
+ if (currPartition != null) {
+ conf.set("carbon.currentpartition", currPartition)
+ }
+ // Update with the current in progress load.
+ val currEntry = options.getOrElse("currentloadentry", null)
+ if (currEntry != null) {
+ val loadEntry =
+ ObjectSerializationUtil.convertStringToObject(currEntry).asInstanceOf[LoadMetadataDetails]
+ val details =
+ SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(model.getTablePath))
+ model.setSegmentId(loadEntry.getLoadName)
+ model.setFactTimeStamp(loadEntry.getLoadStartTime)
+ val list = new util.ArrayList[LoadMetadataDetails](details.toList.asJava)
+ list.add(loadEntry)
+ model.setLoadMetadataDetails(list)
+ }
+ // Set the update timestamp if user sets in case of update query. It needs to be updated
+ // in load status update time
+ val updateTimeStamp = options.get("updatetimestamp")
+ if (updateTimeStamp.isDefined) {
+ conf.set(CarbonTableOutputFormat.UPADTE_TIMESTAMP, updateTimeStamp.get)
+ }
+ CarbonTableOutputFormat.setLoadModel(conf, model)
+
+ new OutputWriterFactory {
+
+ /**
+ * counter used for generating task numbers. This is used to generate unique partition numbers
+ * in case of partitioning
+ */
+ val counter = new AtomicLong()
+ val taskIdMap = new ConcurrentHashMap[String, java.lang.Long]()
+
+ override def newInstance(
+ path: String,
+ dataSchema: StructType,
+ context: TaskAttemptContext): OutputWriter = {
+ val model = CarbonTableOutputFormat.getLoadModel(context.getConfiguration)
+ val isCarbonUseMultiDir = CarbonProperties.getInstance().isUseMultiTempDir
+ var storeLocation: Array[String] = Array[String]()
+ val isCarbonUseLocalDir = CarbonProperties.getInstance()
+ .getProperty("carbon.use.local.dir", "false").equalsIgnoreCase("true")
+
+
+ val taskNumber = generateTaskNumber(path, context, model.getSegmentId)
+ val tmpLocationSuffix =
+ File.separator + "carbon" + System.nanoTime() + File.separator + taskNumber
+ if (isCarbonUseLocalDir) {
+ val yarnStoreLocations = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
+ if (!isCarbonUseMultiDir && null != yarnStoreLocations && yarnStoreLocations.nonEmpty) {
+ // use single dir
+ storeLocation = storeLocation :+
+ (yarnStoreLocations(Random.nextInt(yarnStoreLocations.length)) + tmpLocationSuffix)
+ if (storeLocation == null || storeLocation.isEmpty) {
+ storeLocation = storeLocation :+
+ (System.getProperty("java.io.tmpdir") + tmpLocationSuffix)
+ }
+ } else {
+ // use all the yarn dirs
+ storeLocation = yarnStoreLocations.map(_ + tmpLocationSuffix)
+ }
+ } else {
+ storeLocation =
+ storeLocation :+ (System.getProperty("java.io.tmpdir") + tmpLocationSuffix)
+ }
+ CarbonTableOutputFormat.setTempStoreLocations(context.getConfiguration, storeLocation)
+ new CarbonOutputWriter(path, context, dataSchema.map(_.dataType), taskNumber, model)
+ }
+
+ /**
+ * Generate taskid using the taskid of taskcontext and the path. It should be unique in case
+ * of partition tables.
+ */
+ private def generateTaskNumber(path: String,
+ context: TaskAttemptContext, segmentId: String): String = {
+ var partitionNumber: java.lang.Long = taskIdMap.get(path)
+ if (partitionNumber == null) {
+ partitionNumber = counter.incrementAndGet()
+ // Generate taskid using the combination of taskid and partition number to make it unique.
+ taskIdMap.put(path, partitionNumber)
+ }
+ val taskID = context.getTaskAttemptID.getTaskID.getId
+ CarbonScalaUtil.generateUniqueNumber(taskID, segmentId, partitionNumber)
+ }
+
+ override def getFileExtension(context: TaskAttemptContext): String = {
+ CarbonTablePath.CARBON_DATA_EXT
+ }
+
+ }
+ }
+}
+
+case class CarbonSQLHadoopMapReduceCommitProtocol(jobId: String, path: String, isAppend: Boolean)
+ extends SQLHadoopMapReduceCommitProtocol(jobId, path, isAppend) {
+ override def newTaskTempFileAbsPath(taskContext: TaskAttemptContext,
+ absoluteDir: String,
+ ext: String): String = {
+ val carbonFlow = taskContext.getConfiguration.get("carbon.commit.protocol")
+ if (carbonFlow != null) {
+ super.newTaskTempFile(taskContext, Some(absoluteDir), ext)
+ } else {
+ super.newTaskTempFileAbsPath(taskContext, absoluteDir, ext)
+ }
+ }
+}
+
+/**
+ * It is a just class to make compile between spark 2.1 and 2.2
+ */
+private trait AbstractCarbonOutputWriter {
+ def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal")
+ def writeInternal(row: InternalRow): Unit = {
+ writeCarbon(row)
+ }
+ def write(row: InternalRow): Unit = {
+ writeCarbon(row)
+ }
+ def writeCarbon(row: InternalRow): Unit
+}
+
+private class CarbonOutputWriter(path: String,
+ context: TaskAttemptContext,
+ fieldTypes: Seq[DataType],
+ taskNo : String,
+ model: CarbonLoadModel)
+ extends OutputWriter with AbstractCarbonOutputWriter {
+
+ val converter = new DataTypeConverterImpl
+
+ val partitions =
+ getPartitionsFromPath(path, context, model).map(ExternalCatalogUtils.unescapePathName)
+ val staticPartition: util.HashMap[String, Boolean] = {
+ val staticPart = context.getConfiguration.get("carbon.staticpartition")
+ if (staticPart != null) {
+ ObjectSerializationUtil.convertStringToObject(
+ staticPart).asInstanceOf[util.HashMap[String, Boolean]]
+ } else {
+ null
+ }
+ }
+ lazy val currPartitions: util.List[indexstore.PartitionSpec] = {
+ val currParts = context.getConfiguration.get("carbon.currentpartition")
+ if (currParts != null) {
+ ObjectSerializationUtil.convertStringToObject(
+ currParts).asInstanceOf[util.List[indexstore.PartitionSpec]]
+ } else {
+ new util.ArrayList[indexstore.PartitionSpec]()
+ }
+ }
+ var (updatedPartitions, partitionData) = if (partitions.nonEmpty) {
+ val updatedPartitions = partitions.map(splitPartition)
+ (updatedPartitions, updatePartitions(updatedPartitions.map(_._2)))
+ } else {
+ (Map.empty[String, String].toArray, Array.empty)
+ }
+
+ private def splitPartition(p: String) = {
+ val value = p.substring(p.indexOf("=") + 1, p.length)
+ val col = p.substring(0, p.indexOf("="))
+ // NUll handling case. For null hive creates with this special name
+ if (value.equals("__HIVE_DEFAULT_PARTITION__")) {
+ (col, null)
+ // we should replace back the special string with empty value.
+ } else if (value.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
+ (col, "")
+ } else {
+ (col, value)
+ }
+ }
+
+ lazy val writePath = {
+ val updatedPath = getPartitionPath(path, context, model)
+ // in case of partition location specified by user then search the partitions from the current
+ // partitions to get the corresponding partitions.
+ if (partitions.isEmpty) {
+ val writeSpec = new indexstore.PartitionSpec(null, updatedPath)
+ val index = currPartitions.indexOf(writeSpec)
+ if (index > -1) {
+ val spec = currPartitions.get(index)
+ updatedPartitions = spec.getPartitions.asScala.map(splitPartition).toArray
+ partitionData = updatePartitions(updatedPartitions.map(_._2))
+ }
+ }
+ updatedPath
+ }
+
+ val writable = new ObjectArrayWritable
+
+ private def updatePartitions(partitionData: Seq[String]): Array[AnyRef] = {
+ model.getCarbonDataLoadSchema.getCarbonTable.getTableInfo.getFactTable.getPartitionInfo
+ .getColumnSchemaList.asScala.zipWithIndex.map { case (col, index) =>
+
+ val dataType = if (col.hasEncoding(Encoding.DICTIONARY)) {
+ DataTypes.INT
+ } else if (col.getDataType.equals(DataTypes.TIMESTAMP) ||
+ col.getDataType.equals(DataTypes.DATE)) {
+ DataTypes.LONG
+ } else {
+ col.getDataType
+ }
+ if (staticPartition != null && staticPartition.get(col.getColumnName.toLowerCase)) {
+ val converetedVal =
+ CarbonScalaUtil.convertStaticPartitions(
+ partitionData(index),
+ col,
+ model.getCarbonDataLoadSchema.getCarbonTable)
+ if (col.hasEncoding(Encoding.DICTIONARY)) {
+ converetedVal.toInt.asInstanceOf[AnyRef]
+ } else {
+ DataTypeUtil.getDataBasedOnDataType(
+ converetedVal,
+ dataType,
+ converter)
+ }
+ } else {
+ DataTypeUtil.getDataBasedOnDataType(partitionData(index), dataType, converter)
+ }
+ }.toArray
+ }
+
+ private val recordWriter: CarbonRecordWriter = {
+ context.getConfiguration.set("carbon.outputformat.taskno", taskNo)
+ context.getConfiguration.set("carbon.outputformat.writepath",
+ writePath + "/" + model.getSegmentId + "_" + model.getFactTimeStamp + ".tmp")
+ new CarbonTableOutputFormat() {
+ override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = {
+ new Path(path)
+ }
+ }.getRecordWriter(context).asInstanceOf[CarbonRecordWriter]
+ }
+
+ // TODO Implement writesupport interface to support writing Row directly to recordwriter
+ def writeCarbon(row: InternalRow): Unit = {
+ val data = new Array[AnyRef](fieldTypes.length + partitionData.length)
+ var i = 0
+ while (i < fieldTypes.length) {
+ if (!row.isNullAt(i)) {
+ fieldTypes(i) match {
+ case StringType =>
+ data(i) = row.getString(i)
+ case d: DecimalType =>
+ data(i) = row.getDecimal(i, d.precision, d.scale).toJavaBigDecimal
+ case other =>
+ data(i) = row.get(i, other)
+ }
+ }
+ i += 1
+ }
+ if (partitionData.length > 0) {
+ System.arraycopy(partitionData, 0, data, fieldTypes.length, partitionData.length)
+ }
+ writable.set(data)
+ recordWriter.write(NullWritable.get(), writable)
+ }
+
+
+ override def writeInternal(row: InternalRow): Unit = {
+ writeCarbon(row)
+ }
+
+ override def close(): Unit = {
+ recordWriter.close(context)
+ // write partition info to new file.
+ val partitonList = new util.ArrayList[String]()
+ val formattedPartitions =
+ // All dynamic partitions need to be converted to proper format
+ CarbonScalaUtil.updatePartitions(
+ updatedPartitions.toMap,
+ model.getCarbonDataLoadSchema.getCarbonTable)
+ formattedPartitions.foreach(p => partitonList.add(p._1 + "=" + p._2))
+ SegmentFileStore.writeSegmentFile(
+ model.getTablePath,
+ taskNo,
+ writePath,
+ model.getSegmentId + "_" + model.getFactTimeStamp + "",
+ partitonList)
+ }
+
+ def getPartitionPath(path: String,
+ attemptContext: TaskAttemptContext,
+ model: CarbonLoadModel): String = {
+ if (updatedPartitions.nonEmpty) {
+ val formattedPartitions =
+ // All dynamic partitions need to be converted to proper format
+ CarbonScalaUtil.updatePartitions(
+ updatedPartitions.toMap,
+ model.getCarbonDataLoadSchema.getCarbonTable)
+ val partitionstr = formattedPartitions.map{p =>
+ ExternalCatalogUtils.escapePathName(p._1) + "=" + ExternalCatalogUtils.escapePathName(p._2)
+ }.mkString(CarbonCommonConstants.FILE_SEPARATOR)
+ model.getCarbonDataLoadSchema.getCarbonTable.getTablePath +
+ CarbonCommonConstants.FILE_SEPARATOR + partitionstr
+ } else {
+ var updatedPath = FileFactory.getUpdatedFilePath(path)
+ updatedPath.substring(0, updatedPath.lastIndexOf("/"))
+ }
+ }
+
+ def getPartitionsFromPath(
+ path: String,
+ attemptContext: TaskAttemptContext,
+ model: CarbonLoadModel): Array[String] = {
+ var attemptId = attemptContext.getTaskAttemptID.toString + "/"
+ if (path.indexOf(attemptId) > -1) {
+ val str = path.substring(path.indexOf(attemptId) + attemptId.length, path.lastIndexOf("/"))
+ if (str.length > 0) {
+ str.split("/")
+ } else {
+ Array.empty
+ }
+ } else {
+ Array.empty
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index ec20ec2..d85ef68 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -110,7 +110,14 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
.tableExists(TableIdentifier(alterTableChangeDataTypeModel.tableName,
alterTableChangeDataTypeModel.databaseName))(sparkSession)
if (isCarbonTable) {
- ExecutedCommandExec(dataTypeChange) :: Nil
+ val carbonTable = CarbonEnv.getCarbonTable(alterTableChangeDataTypeModel.databaseName,
+ alterTableChangeDataTypeModel.tableName)(sparkSession)
+ if (carbonTable != null && carbonTable.isFileLevelExternalTable) {
+ throw new MalformedCarbonCommandException(
+ "Unsupported alter operation on Carbon external fileformat table")
+ } else {
+ ExecutedCommandExec(dataTypeChange) :: Nil
+ }
} else {
throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
}
@@ -119,7 +126,14 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
.tableExists(TableIdentifier(alterTableAddColumnsModel.tableName,
alterTableAddColumnsModel.databaseName))(sparkSession)
if (isCarbonTable) {
- ExecutedCommandExec(addColumn) :: Nil
+ val carbonTable = CarbonEnv.getCarbonTable(alterTableAddColumnsModel.databaseName,
+ alterTableAddColumnsModel.tableName)(sparkSession)
+ if (carbonTable != null && carbonTable.isFileLevelExternalTable) {
+ throw new MalformedCarbonCommandException(
+ "Unsupported alter operation on Carbon external fileformat table")
+ } else {
+ ExecutedCommandExec(addColumn) :: Nil
+ }
} else {
throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
}
@@ -128,7 +142,14 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
.tableExists(TableIdentifier(alterTableDropColumnModel.tableName,
alterTableDropColumnModel.databaseName))(sparkSession)
if (isCarbonTable) {
- ExecutedCommandExec(dropColumn) :: Nil
+ val carbonTable = CarbonEnv.getCarbonTable(alterTableDropColumnModel.databaseName,
+ alterTableDropColumnModel.tableName)(sparkSession)
+ if (carbonTable != null && carbonTable.isFileLevelExternalTable) {
+ throw new MalformedCarbonCommandException(
+ "Unsupported alter operation on Carbon external fileformat table")
+ } else {
+ ExecutedCommandExec(dropColumn) :: Nil
+ }
} else {
throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index 4996bec..b2f4505 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.Inner
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.execution.command.mutation.CarbonProjectForDeleteCommand
-import org.apache.spark.sql.execution.datasources.{CarbonFileFormat, CatalogFileIndex, FileFormat, HadoopFsRelation, LogicalRelation}
+import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FileFormat, HadoopFsRelation, LogicalRelation, SparkCarbonTableFormat}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CarbonException
import org.apache.spark.util.CarbonReflectionUtils
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index e0fff08..69fd366 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -33,7 +33,9 @@ import org.apache.spark.sql.util.CarbonException
import org.apache.spark.util.CarbonReflectionUtils
import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.datastore.impl.FileFactory
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.hadoop.util.SchemaReader
import org.apache.carbondata.spark.CarbonOption
import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil}
@@ -144,19 +146,24 @@ class CarbonHelperSqlAstBuilder(conf: SQLConf,
.getOrElse(Map.empty)
}
- def createCarbonTable(tableHeader: CreateTableHeaderContext,
- skewSpecContext: SkewSpecContext,
- bucketSpecContext: BucketSpecContext,
- partitionColumns: ColTypeListContext,
- columns : ColTypeListContext,
- tablePropertyList : TablePropertyListContext,
- locationSpecContext: SqlBaseParser.LocationSpecContext,
- tableComment : Option[String],
- ctas: TerminalNode,
- query: QueryContext) : LogicalPlan = {
+ def createCarbonTable(createTableTuple: (CreateTableHeaderContext, SkewSpecContext,
+ BucketSpecContext, ColTypeListContext, ColTypeListContext, TablePropertyListContext,
+ LocationSpecContext, Option[String], TerminalNode, QueryContext, String)): LogicalPlan = {
// val parser = new CarbonSpark2SqlParser
+ val (tableHeader, skewSpecContext,
+ bucketSpecContext,
+ partitionColumns,
+ columns,
+ tablePropertyList,
+ locationSpecContext,
+ tableComment,
+ ctas,
+ query,
+ provider) = createTableTuple
+
val (tableIdentifier, temp, ifNotExists, external) = visitCreateTableHeader(tableHeader)
+
// TODO: implement temporary tables
if (temp) {
throw new ParseException(
@@ -256,13 +263,27 @@ class CarbonHelperSqlAstBuilder(conf: SQLConf,
CarbonEnv.getDatabaseName(tableIdentifier.database)(sparkSession),
tableIdentifier.table)
val table = try {
- SchemaReader.getTableInfo(identifier)
- } catch {
+ val schemaPath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath)
+ if (!FileFactory.isFileExist(schemaPath, FileFactory.getFileType(schemaPath)) &&
+ provider.equalsIgnoreCase("'Carbonfile'")) {
+ SchemaReader.inferSchema(identifier)
+ }
+ else {
+ SchemaReader.getTableInfo(identifier)
+ }
+ }
+ catch {
case e: Throwable =>
operationNotAllowed(s"Invalid table path provided: ${tablePath.get} ", tableHeader)
}
// set "_external" property, so that DROP TABLE will not delete the data
- table.getFactTable.getTableProperties.put("_external", "true")
+ if (provider.equalsIgnoreCase("'Carbonfile'")) {
+ table.getFactTable.getTableProperties.put("_filelevelexternal", "true")
+ table.getFactTable.getTableProperties.put("_external", "false")
+ } else {
+ table.getFactTable.getTableProperties.put("_external", "true")
+ table.getFactTable.getTableProperties.put("_filelevelexternal", "false")
+ }
table
} else {
// prepare table model of the collected tokens
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
index d381144..1f5808a 100644
--- a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -326,18 +326,13 @@ class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSes
val fileStorage = helper.getFileStorage(ctx.createFileFormat)
if (fileStorage.equalsIgnoreCase("'carbondata'") ||
+ fileStorage.equalsIgnoreCase("'Carbonfile'") ||
fileStorage.equalsIgnoreCase("'org.apache.carbondata.format'")) {
- helper.createCarbonTable(
- tableHeader = ctx.createTableHeader,
- skewSpecContext = ctx.skewSpec,
- bucketSpecContext = ctx.bucketSpec,
- partitionColumns = ctx.partitionColumns,
- columns = ctx.columns,
- tablePropertyList = ctx.tablePropertyList,
- locationSpecContext = ctx.locationSpec(),
- tableComment = Option(ctx.STRING()).map(string),
- ctas = ctx.AS,
- query = ctx.query)
+ val createTableTuple = (ctx.createTableHeader, ctx.skewSpec, ctx.bucketSpec,
+ ctx.partitionColumns, ctx.columns, ctx.tablePropertyList, ctx.locationSpec(),
+ Option(ctx.STRING()).map(string),
+ ctx.AS, ctx.query, fileStorage)
+ helper.createCarbonTable(createTableTuple)
} else {
super.visitCreateTable(ctx)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
index f033a8e..c28e4ba 100644
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -325,18 +325,12 @@ class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSes
val fileStorage = helper.getFileStorage(ctx.createFileFormat)
if (fileStorage.equalsIgnoreCase("'carbondata'") ||
+ fileStorage.equalsIgnoreCase("'Carbonfile'") ||
fileStorage.equalsIgnoreCase("'org.apache.carbondata.format'")) {
- helper.createCarbonTable(
- tableHeader = ctx.createTableHeader,
- skewSpecContext = ctx.skewSpec,
- bucketSpecContext = ctx.bucketSpec,
- partitionColumns = ctx.partitionColumns,
- columns = ctx.columns,
- tablePropertyList = ctx.tablePropertyList,
- locationSpecContext = ctx.locationSpec(),
- tableComment = Option(ctx.STRING()).map(string),
- ctas = ctx.AS,
- query = ctx.query)
+ val createTableTuple = (ctx.createTableHeader, ctx.skewSpec,
+ ctx.bucketSpec, ctx.partitionColumns, ctx.columns, ctx.tablePropertyList,ctx.locationSpec(),
+ Option(ctx.STRING()).map(string), ctx.AS, ctx.query, fileStorage)
+ helper.createCarbonTable(createTableTuple)
} else {
super.visitCreateHiveTable(ctx)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
----------------------------------------------------------------------
diff --git a/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index d09c9b5..5831f3e 100644
--- a/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ b/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -14,4 +14,5 @@
## See the License for the specific language governing permissions and
## limitations under the License.
## ------------------------------------------------------------------------
-org.apache.spark.sql.CarbonSource
\ No newline at end of file
+org.apache.spark.sql.CarbonSource
+org.apache.spark.sql.SparkCarbonFileFormat
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterTest.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterTest.java
index 2281fe6..9c07065 100644
--- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterTest.java
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterTest.java
@@ -21,6 +21,7 @@ import java.io.File;
import java.io.FileFilter;
import java.io.IOException;
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -113,9 +114,12 @@ public class CSVCarbonWriterTest {
writer.write(new String[]{"robot" + (i % 10), String.valueOf(i), String.valueOf((double) i / 2)});
}
writer.close();
- } catch (Exception e) {
+ } catch (IOException e) {
e.printStackTrace();
Assert.fail(e.getMessage());
+ } catch (InvalidLoadOptionException l) {
+ l.printStackTrace();
+ Assert.fail(l.getMessage());
}
File segmentFolder = new File(CarbonTablePath.getSegmentPath(path, "null"));
[04/10] carbondata git commit: [CARBONDATA-2230]Add a path into table
path to store lock files and delete useless segment lock files before loading
Posted by ja...@apache.org.
[CARBONDATA-2230]Add a path into table path to store lock files and delete useless segment lock files before loading
This closes #2045
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/0609fc52
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/0609fc52
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/0609fc52
Branch: refs/heads/carbonfile
Commit: 0609fc52cdb472121aa93350925674cec194b068
Parents: dbec6f9
Author: Zhang Zhichao <44...@qq.com>
Authored: Thu Mar 8 15:18:09 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Mon Mar 19 00:16:13 2018 +0800
----------------------------------------------------------------------
.../core/constants/CarbonCommonConstants.java | 11 +++++
.../core/datastore/impl/FileFactory.java | 12 +++++-
.../carbondata/core/locks/CarbonLockUtil.java | 34 ++++++++++++++++
.../carbondata/core/locks/HdfsFileLock.java | 35 +++++++++-------
.../carbondata/core/locks/LocalFileLock.java | 34 +++++++---------
.../carbondata/core/locks/S3FileLock.java | 42 +++++++++-----------
.../carbondata/core/locks/ZooKeeperLocking.java | 7 ++--
.../statusmanager/SegmentStatusManager.java | 2 +
.../carbondata/core/util/CarbonProperties.java | 20 ++++++++++
.../core/util/path/CarbonTablePath.java | 32 ++++++++++++++-
.../org/apache/spark/util/AlterTableUtil.scala | 3 +-
11 files changed, 166 insertions(+), 66 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index b0e3536..1b135dc 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -1588,6 +1588,17 @@ public final class CarbonCommonConstants {
public static final String CARBON_SHOW_DATAMAPS_DEFAULT = "true";
+ /**
+ * Currently the segment lock files are not deleted immediately when unlock,
+ * this value indicates the number of hours the segment lock files will be preserved.
+ */
+ @CarbonProperty
+ public static final String CARBON_SEGMENT_LOCK_FILES_PRESERVE_HOURS =
+ "carbon.segment.lock.files.preserve.hours";
+
+ // default value is 2 days
+ public static final String CARBON_SEGMENT_LOCK_FILES_PRESERVE_HOURS_DEFAULT = "48";
+
private CarbonCommonConstants() {
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
index ef84fb3..1529649 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
@@ -192,8 +192,7 @@ public final class FileFactory {
}
/**
- * This method checks the given path exists or not and also is it file or
- * not if the performFileCheck is true
+ * This method checks the given path exists or not.
*
* @param filePath - Path
* @param fileType - FileType Local/HDFS
@@ -202,6 +201,15 @@ public final class FileFactory {
return getCarbonFile(filePath).isFileExist(filePath, fileType);
}
+ /**
+ * This method checks the given path exists or not.
+ *
+ * @param filePath - Path
+ */
+ public static boolean isFileExist(String filePath) throws IOException {
+ return isFileExist(filePath, getFileType(filePath));
+ }
+
public static boolean createNewFile(String filePath, FileType fileType) throws IOException {
return createNewFile(filePath, fileType, true, null);
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
index c399ef4..5ac2bc9 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
@@ -19,8 +19,13 @@ package org.apache.carbondata.core.locks;
import org.apache.carbondata.common.logging.LogService;
import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
/**
* This class contains all carbon lock utilities
@@ -107,4 +112,33 @@ public class CarbonLockUtil {
}
}
+ /**
+ * Currently the segment lock files are not deleted immediately when unlock,
+ * so it needs to delete expired lock files before delete loads.
+ */
+ public static void deleteExpiredSegmentLockFiles(CarbonTable carbonTable) {
+ final long currTime = System.currentTimeMillis();
+ final long segmentLockFilesPreservTime =
+ CarbonProperties.getInstance().getSegmentLockFilesPreserveHours();
+ AbsoluteTableIdentifier absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
+ String lockFilesDir = CarbonTablePath
+ .getLockFilesDirPath(absoluteTableIdentifier.getTablePath());
+ CarbonFile[] files = FileFactory.getCarbonFile(lockFilesDir)
+ .listFiles(new CarbonFileFilter() {
+
+ @Override public boolean accept(CarbonFile pathName) {
+ if (CarbonTablePath.isSegmentLockFilePath(pathName.getName())) {
+ if ((currTime - pathName.getLastModifiedTime()) > segmentLockFilesPreservTime) {
+ return true;
+ }
+ }
+ return false;
+ }
+ }
+ );
+
+ for (CarbonFile file : files) {
+ file.delete();
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java b/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java
index be98f7d..3c28f9d 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/HdfsFileLock.java
@@ -22,35 +22,38 @@ import java.io.IOException;
import org.apache.carbondata.common.logging.LogService;
import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.datastore.impl.FileFactory;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
/**
* This class is used to handle the HDFS File locking.
- * This is acheived using the concept of acquiring the data out stream using Append option.
+ * This is achieved using the concept of acquiring the data out stream using Append option.
*/
public class HdfsFileLock extends AbstractCarbonLock {
private static final LogService LOGGER =
LogServiceFactory.getLogService(HdfsFileLock.class.getName());
/**
- * location hdfs file location
+ * lockFilePath is the location of the lock file.
*/
- private String location;
+ private String lockFilePath;
- private DataOutputStream dataOutputStream;
+ /**
+ * lockFileDir is the directory of the lock file.
+ */
+ private String lockFileDir;
- private static String tmpPath;
+ private DataOutputStream dataOutputStream;
/**
* @param lockFileLocation
* @param lockFile
*/
public HdfsFileLock(String lockFileLocation, String lockFile) {
- this.location = lockFileLocation
- + CarbonCommonConstants.FILE_SEPARATOR + lockFile;
- LOGGER.info("HDFS lock path:" + this.location);
+ this.lockFileDir = CarbonTablePath.getLockFilesDirPath(lockFileLocation);
+ this.lockFilePath = CarbonTablePath.getLockFilePath(lockFileLocation, lockFile);
+ LOGGER.info("HDFS lock path:" + this.lockFilePath);
initRetry();
}
@@ -58,7 +61,7 @@ public class HdfsFileLock extends AbstractCarbonLock {
* @param lockFilePath
*/
public HdfsFileLock(String lockFilePath) {
- this.location = lockFilePath;
+ this.lockFilePath = lockFilePath;
initRetry();
}
@@ -75,11 +78,15 @@ public class HdfsFileLock extends AbstractCarbonLock {
*/
@Override public boolean lock() {
try {
- if (!FileFactory.isFileExist(location, FileFactory.getFileType(location))) {
- FileFactory.createNewLockFile(location, FileFactory.getFileType(location));
+ if (null != this.lockFileDir &&
+ !FileFactory.isFileExist(lockFileDir)) {
+ FileFactory.mkdirs(lockFileDir, FileFactory.getFileType(lockFileDir));
+ }
+ if (!FileFactory.isFileExist(lockFilePath)) {
+ FileFactory.createNewLockFile(lockFilePath, FileFactory.getFileType(lockFilePath));
}
- dataOutputStream =
- FileFactory.getDataOutputStreamUsingAppend(location, FileFactory.getFileType(location));
+ dataOutputStream = FileFactory.getDataOutputStreamUsingAppend(lockFilePath,
+ FileFactory.getFileType(lockFilePath));
return true;
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
index e3b3126..6983562 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
@@ -26,10 +26,10 @@ import java.nio.file.StandardOpenOption;
import org.apache.carbondata.common.logging.LogService;
import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.datastore.impl.FileFactory;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
/**
* This class handles the file locking in the local file system.
@@ -37,9 +37,14 @@ import org.apache.carbondata.core.util.CarbonUtil;
*/
public class LocalFileLock extends AbstractCarbonLock {
/**
- * location is the location of the lock file.
+ * lockFilePath is the location of the lock file.
*/
- private String location;
+ private String lockFilePath;
+
+ /**
+ * lockFileDir is the directory of the lock file.
+ */
+ private String lockFileDir;
/**
* channel is the FileChannel of the lock file.
@@ -52,27 +57,18 @@ public class LocalFileLock extends AbstractCarbonLock {
private FileLock fileLock;
/**
- * lock file
- */
- private String lockFile;
-
- private String lockFilePath;
-
- /**
* LOGGER for logging the messages.
*/
private static final LogService LOGGER =
LogServiceFactory.getLogService(LocalFileLock.class.getName());
-
-
/**
* @param lockFileLocation
* @param lockFile
*/
public LocalFileLock(String lockFileLocation, String lockFile) {
- this.location = lockFileLocation;
- this.lockFile = lockFile;
+ this.lockFileDir = CarbonTablePath.getLockFilesDirPath(lockFileLocation);
+ this.lockFilePath = CarbonTablePath.getLockFilePath(lockFileLocation, lockFile);
initRetry();
}
@@ -92,13 +88,11 @@ public class LocalFileLock extends AbstractCarbonLock {
*/
@Override public boolean lock() {
try {
- if (!FileFactory.isFileExist(location, FileFactory.getFileType(location))) {
- FileFactory.mkdirs(location, FileFactory.getFileType(location));
+ if (!FileFactory.isFileExist(lockFileDir)) {
+ FileFactory.mkdirs(lockFileDir, FileFactory.getFileType(lockFileDir));
}
- lockFilePath = location + CarbonCommonConstants.FILE_SEPARATOR +
- lockFile;
- if (!FileFactory.isFileExist(lockFilePath, FileFactory.getFileType(location))) {
- FileFactory.createNewLockFile(lockFilePath, FileFactory.getFileType(location));
+ if (!FileFactory.isFileExist(lockFilePath)) {
+ FileFactory.createNewLockFile(lockFilePath, FileFactory.getFileType(lockFilePath));
}
channel = FileChannel.open(Paths.get(lockFilePath), StandardOpenOption.WRITE,
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/locks/S3FileLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/S3FileLock.java b/core/src/main/java/org/apache/carbondata/core/locks/S3FileLock.java
index 8836960..464becb 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/S3FileLock.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/S3FileLock.java
@@ -22,10 +22,9 @@ import java.io.IOException;
import org.apache.carbondata.common.logging.LogService;
import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
import org.apache.carbondata.core.datastore.impl.FileFactory;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
/**
* This class is used to handle the S3 File locking.
@@ -36,9 +35,14 @@ public class S3FileLock extends AbstractCarbonLock {
private static final LogService LOGGER =
LogServiceFactory.getLogService(S3FileLock.class.getName());
/**
- * location s3 file location
+ * lockFilePath is the location of the lock file.
*/
- private String location;
+ private String lockFilePath;
+
+ /**
+ * lockFileDir is the directory of the lock file.
+ */
+ private String lockFileDir;
private DataOutputStream dataOutputStream;
@@ -55,8 +59,9 @@ public class S3FileLock extends AbstractCarbonLock {
* @param lockFile
*/
public S3FileLock(String lockFileLocation, String lockFile) {
- this.location = lockFileLocation + CarbonCommonConstants.FILE_SEPARATOR + lockFile;
- LOGGER.info("S3 lock path:" + this.location);
+ this.lockFileDir = CarbonTablePath.getLockFilesDirPath(lockFileLocation);
+ this.lockFilePath = CarbonTablePath.getLockFilePath(lockFileLocation, lockFile);
+ LOGGER.info("S3 lock path:" + this.lockFilePath);
initRetry();
}
@@ -71,21 +76,6 @@ public class S3FileLock extends AbstractCarbonLock {
status = true;
} catch (IOException e) {
status = false;
- } finally {
- CarbonFile carbonFile =
- FileFactory.getCarbonFile(location, FileFactory.getFileType(location));
- if (carbonFile.exists()) {
- if (carbonFile.delete()) {
- LOGGER.info("Deleted the lock file " + location);
- } else {
- LOGGER.error("Not able to delete the lock file " + location);
- status = false;
- }
- } else {
- LOGGER.error(
- "Not able to delete the lock file because it is not existed in location " + location);
- status = false;
- }
}
}
return status;
@@ -96,11 +86,15 @@ public class S3FileLock extends AbstractCarbonLock {
*/
@Override public boolean lock() {
try {
- if (!FileFactory.isFileExist(location, FileFactory.getFileType(location))) {
- FileFactory.createNewLockFile(location, FileFactory.getFileType(location));
+ if (!FileFactory.isFileExist(lockFileDir)) {
+ FileFactory.mkdirs(lockFileDir, FileFactory.getFileType(lockFileDir));
+ }
+ if (!FileFactory.isFileExist(lockFilePath)) {
+ FileFactory.createNewLockFile(lockFilePath, FileFactory.getFileType(lockFilePath));
}
dataOutputStream =
- FileFactory.getDataOutputStreamUsingAppend(location, FileFactory.getFileType(location));
+ FileFactory.getDataOutputStreamUsingAppend(lockFilePath,
+ FileFactory.getFileType(lockFilePath));
return true;
} catch (IOException e) {
LOGGER.error(e, e.getMessage());
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java b/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java
index 1de5004..5a055ab 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/ZooKeeperLocking.java
@@ -25,6 +25,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -88,12 +89,12 @@ public class ZooKeeperLocking extends AbstractCarbonLock {
*/
public ZooKeeperLocking(String lockLocation, String lockFile) {
this.lockName = lockFile;
- this.tableIdFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + lockLocation;
+ this.tableIdFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR
+ + CarbonTablePath.getLockFilesDirPath(lockLocation);
initialize();
- this.lockTypeFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + lockLocation
- + CarbonCommonConstants.FILE_SEPARATOR + lockFile;
+ this.lockTypeFolder = tableIdFolder + CarbonCommonConstants.FILE_SEPARATOR + lockFile;
try {
createBaseNode();
// if exists returns null then path doesnt exist. so creating.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index f466018..aa73fee 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -909,6 +909,8 @@ public class SegmentStatusManager {
}
}
}
+ // delete the expired segment lock files
+ CarbonLockUtil.deleteExpiredSegmentLockFiles(carbonTable);
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index 6fd087b..acc266c 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -1382,4 +1382,24 @@ public final class CarbonProperties {
return thresholdSize;
}
+ /**
+ * Get the number of hours the segment lock files will be preserved.
+ * It will be converted to microseconds to return.
+ */
+ public long getSegmentLockFilesPreserveHours() {
+ long preserveSeconds;
+ try {
+ int preserveHours = Integer.parseInt(CarbonProperties.getInstance()
+ .getProperty(CarbonCommonConstants.CARBON_SEGMENT_LOCK_FILES_PRESERVE_HOURS,
+ CarbonCommonConstants.CARBON_SEGMENT_LOCK_FILES_PRESERVE_HOURS_DEFAULT));
+ preserveSeconds = preserveHours * 3600 * 1000L;
+ } catch (NumberFormatException exc) {
+ LOGGER.error(
+ "The segment lock files preserv hours is invalid. Using the default value "
+ + CarbonCommonConstants.CARBON_SEGMENT_LOCK_FILES_PRESERVE_HOURS_DEFAULT);
+ preserveSeconds = Integer.parseInt(
+ CarbonCommonConstants.CARBON_SEGMENT_LOCK_FILES_PRESERVE_HOURS_DEFAULT) * 3600 * 1000L;
+ }
+ return preserveSeconds;
+ }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index 50c5a31..a37d5cd 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -22,6 +22,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.locks.LockUsage;
import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
import org.apache.hadoop.fs.Path;
@@ -41,6 +42,7 @@ public class CarbonTablePath {
private static final String PARTITION_PREFIX = "Part";
private static final String DATA_PART_PREFIX = "part-";
private static final String BATCH_PREFIX = "_batchno";
+ private static final String LOCK_DIR = "LockFiles";
public static final String TABLE_STATUS_FILE = "tablestatus";
public static final String CARBON_DATA_EXT = ".carbondata";
@@ -633,4 +635,32 @@ public class CarbonTablePath {
return getMetadataPath(tablePath) + CarbonCommonConstants.FILE_SEPARATOR + "segments";
}
-}
\ No newline at end of file
+ /**
+ * Get the lock files directory
+ */
+ public static String getLockFilesDirPath(String tablePath) {
+ return tablePath + CarbonCommonConstants.FILE_SEPARATOR + LOCK_DIR;
+ }
+
+ /**
+ * Get the lock file
+ */
+ public static String getLockFilePath(String tablePath, String lockType) {
+ return getLockFilesDirPath(tablePath) + CarbonCommonConstants.FILE_SEPARATOR + lockType;
+ }
+
+ /**
+ * Get the segment lock file according to table path and segment load name
+ */
+ public static String getSegmentLockFilePath(String tablePath, String loadName) {
+ return getLockFilesDirPath(tablePath) + CarbonCommonConstants.FILE_SEPARATOR +
+ addSegmentPrefix(loadName) + LockUsage.LOCK;
+ }
+
+ /**
+ * return true if this lock file is a segment lock file otherwise false.
+ */
+ public static boolean isSegmentLockFilePath(String lockFileName) {
+ return lockFileName.startsWith(SEGMENT_PREFIX) && lockFileName.endsWith(LockUsage.LOCK);
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/0609fc52/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index 9c2c7e7..45e956a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -113,8 +113,7 @@ object AlterTableUtil {
tablePath: String): Unit = {
val lockLocation = tablePath
locks.zip(locksAcquired).foreach { case (carbonLock, lockType) =>
- val lockFilePath = lockLocation + CarbonCommonConstants.FILE_SEPARATOR +
- lockType
+ val lockFilePath = CarbonTablePath.getLockFilePath(lockLocation, lockType)
if (carbonLock.releaseLockManually(lockFilePath)) {
LOGGER.info(s"Alter table lock released successfully: ${ lockType }")
} else {
[06/10] carbondata git commit: [CARBONDATA-2223] Adding Listener
Support for Partition
Posted by ja...@apache.org.
[CARBONDATA-2223] Adding Listener Support for Partition
Adding Listener Support for Partition
This closes #2031
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/f5cdd5ca
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/f5cdd5ca
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/f5cdd5ca
Branch: refs/heads/carbonfile
Commit: f5cdd5ca9dcf22984ed300fe1d2d36939755e947
Parents: 98b8550
Author: dhatchayani <dh...@gmail.com>
Authored: Mon Mar 5 15:17:13 2018 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Tue Mar 20 19:24:18 2018 +0530
----------------------------------------------------------------------
.../core/constants/CarbonCommonConstants.java | 3 +++
.../indexstore/BlockletDataMapIndexStore.java | 14 +++++++++-----
.../core/metadata/SegmentFileStore.java | 2 +-
.../apache/carbondata/core/util/CarbonUtil.java | 18 +++++++++++++++++-
.../hadoop/api/CarbonOutputCommitter.java | 4 ----
.../carbondata/events/AlterTableEvents.scala | 2 +-
.../spark/rdd/CarbonTableCompactor.scala | 4 +++-
.../scala/org/apache/spark/sql/CarbonEnv.scala | 20 ++++++++++++++++++++
.../management/CarbonLoadDataCommand.scala | 7 ++++++-
.../sql/test/Spark2TestQueryExecutor.scala | 1 +
.../spark/sql/hive/CarbonSessionState.scala | 2 +-
.../processing/loading/events/LoadEvents.java | 11 -----------
.../processing/util/CarbonLoaderUtil.java | 6 +++---
13 files changed, 65 insertions(+), 29 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 1b135dc..33a1884 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -1297,6 +1297,9 @@ public final class CarbonCommonConstants {
public static final String CARBON_SESSIONSTATE_CLASSNAME = "spark.carbon.sessionstate.classname";
+ public static final String CARBON_COMMON_LISTENER_REGISTER_CLASSNAME =
+ "spark.carbon.common.listener.register.classname";
+
@CarbonProperty
public static final String CARBON_LEASE_RECOVERY_RETRY_COUNT =
"carbon.lease.recovery.retry.count";
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
index 53ef496..befa121 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
@@ -19,8 +19,10 @@ package org.apache.carbondata.core.indexstore;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.carbondata.common.logging.LogService;
@@ -81,8 +83,9 @@ public class BlockletDataMapIndexStore
if (dataMap == null) {
try {
SegmentIndexFileStore indexFileStore = new SegmentIndexFileStore();
+ Set<String> filesRead = new HashSet<>();
Map<String, BlockMetaInfo> blockMetaInfoMap =
- getBlockMetaInfoMap(identifier, indexFileStore);
+ getBlockMetaInfoMap(identifier, indexFileStore, filesRead);
dataMap = loadAndGetDataMap(identifier, indexFileStore, blockMetaInfoMap);
} catch (MemoryException e) {
LOGGER.error("memory exception when loading datamap: " + e.getMessage());
@@ -93,13 +96,14 @@ public class BlockletDataMapIndexStore
}
private Map<String, BlockMetaInfo> getBlockMetaInfoMap(TableBlockIndexUniqueIdentifier identifier,
- SegmentIndexFileStore indexFileStore) throws IOException {
+ SegmentIndexFileStore indexFileStore, Set<String> filesRead) throws IOException {
if (identifier.getMergeIndexFileName() != null) {
CarbonFile indexMergeFile = FileFactory.getCarbonFile(
identifier.getIndexFilePath() + CarbonCommonConstants.FILE_SEPARATOR + identifier
.getMergeIndexFileName());
- if (indexMergeFile.exists()) {
+ if (indexMergeFile.exists() && !filesRead.contains(indexMergeFile.getPath())) {
indexFileStore.readAllIIndexOfSegment(new CarbonFile[] { indexMergeFile });
+ filesRead.add(indexMergeFile.getPath());
}
}
if (indexFileStore.getFileData(identifier.getIndexFileName()) == null) {
@@ -151,10 +155,10 @@ public class BlockletDataMapIndexStore
}
if (missedIdentifiers.size() > 0) {
SegmentIndexFileStore indexFileStore = new SegmentIndexFileStore();
-
+ Set<String> filesRead = new HashSet<>();
for (TableBlockIndexUniqueIdentifier identifier: missedIdentifiers) {
Map<String, BlockMetaInfo> blockMetaInfoMap =
- getBlockMetaInfoMap(identifier, indexFileStore);
+ getBlockMetaInfoMap(identifier, indexFileStore, filesRead);
blockletDataMaps.add(
loadAndGetDataMap(identifier, indexFileStore, blockMetaInfoMap));
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
index 3fc8ad6..4adc977 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
@@ -370,7 +370,7 @@ public class SegmentFileStore {
for (Map.Entry<String, FolderDetails> entry : getLocationMap().entrySet()) {
String location = entry.getKey();
if (entry.getValue().isRelative) {
- location = tablePath + CarbonCommonConstants.FILE_SEPARATOR + location;
+ location = tablePath + location;
}
if (entry.getValue().status.equals(SegmentStatus.SUCCESS.getMessage())) {
for (String indexFile : entry.getValue().getFiles()) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index b961b60..06511f8 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -2326,12 +2326,28 @@ public final class CarbonUtil {
throws IOException {
long carbonDataSize = 0L;
long carbonIndexSize = 0L;
+ List<String> listOfFilesRead = new ArrayList<>();
HashMap<String, Long> dataAndIndexSize = new HashMap<String, Long>();
if (fileStore.getLocationMap() != null) {
fileStore.readIndexFiles();
+ Map<String, String> indexFiles = fileStore.getIndexFiles();
Map<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
for (Map.Entry<String, List<String>> entry : indexFilesMap.entrySet()) {
- carbonIndexSize += FileFactory.getCarbonFile(entry.getKey()).getSize();
+ // get the size of carbonindex file
+ String indexFile = entry.getKey();
+ String mergeIndexFile = indexFiles.get(indexFile);
+ if (null != mergeIndexFile) {
+ String mergeIndexPath = indexFile
+ .substring(0, indexFile.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR) + 1)
+ + mergeIndexFile;
+ if (!listOfFilesRead.contains(mergeIndexPath)) {
+ carbonIndexSize += FileFactory.getCarbonFile(mergeIndexPath).getSize();
+ listOfFilesRead.add(mergeIndexPath);
+ }
+ } else {
+ carbonIndexSize += FileFactory.getCarbonFile(indexFile).getSize();
+ }
+ // get the size of carbondata files
for (String blockFile : entry.getValue()) {
carbonDataSize += FileFactory.getCarbonFile(blockFile).getSize();
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
index 7ea11bd..4634b06 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonOutputCommitter.java
@@ -160,13 +160,9 @@ public class CarbonOutputCommitter extends FileOutputCommitter {
if (operationContext != null) {
LoadEvents.LoadTablePostStatusUpdateEvent postStatusUpdateEvent =
new LoadEvents.LoadTablePostStatusUpdateEvent(loadModel);
- LoadEvents.LoadTableMergePartitionEvent loadTableMergePartitionEvent =
- new LoadEvents.LoadTableMergePartitionEvent(readPath);
try {
OperationListenerBus.getInstance()
.fireEvent(postStatusUpdateEvent, (OperationContext) operationContext);
- OperationListenerBus.getInstance()
- .fireEvent(loadTableMergePartitionEvent, (OperationContext) operationContext);
} catch (Exception e) {
throw new IOException(e);
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
index 671e132..538df4a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
@@ -185,7 +185,7 @@ case class AlterTableCompactionPreStatusUpdateEvent(sparkSession: SparkSession,
* Compaction Event for handling post update status file operations, like committing child
* datamaps in one transaction
*/
-case class AlterTableCompactionPostStatusUpdateEvent(
+case class AlterTableCompactionPostStatusUpdateEvent(sparkSession: SparkSession,
carbonTable: CarbonTable,
carbonMergerMapping: CarbonMergerMapping,
carbonLoadModel: CarbonLoadModel,
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
index 231b748..a987127 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
@@ -246,7 +246,9 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
carbonLoadModel,
compactionType,
segmentFileName)
- val compactionLoadStatusPostEvent = AlterTableCompactionPostStatusUpdateEvent(carbonTable,
+
+ val compactionLoadStatusPostEvent = AlterTableCompactionPostStatusUpdateEvent(sc.sparkSession,
+ carbonTable,
carbonMergerMapping,
carbonLoadModel,
mergedLoadName)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
index 8c3ca0f..95bbd29 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
@@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
import org.apache.spark.sql.execution.command.preaaggregate._
import org.apache.spark.sql.execution.command.timeseries.TimeSeriesFunction
import org.apache.spark.sql.hive.{HiveSessionCatalog, _}
+import org.apache.spark.util.CarbonReflectionUtils
import org.apache.carbondata.common.logging.LogServiceFactory
import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -135,6 +136,17 @@ object CarbonEnv {
}
/**
+ * Method
+ * 1. To initialize Listeners to their respective events in the OperationListenerBus
+ * 2. To register common listeners
+ *
+ */
+ def init(sparkSession: SparkSession): Unit = {
+ initListeners
+ registerCommonListener(sparkSession)
+ }
+
+ /**
* Method to initialize Listeners to their respective events in the OperationListenerBus.
*/
def initListeners(): Unit = {
@@ -158,6 +170,14 @@ object CarbonEnv {
.addListener(classOf[AlterTableCompactionPostStatusUpdateEvent], CommitPreAggregateListener)
}
+ def registerCommonListener(sparkSession: SparkSession): Unit = {
+ val clsName = sparkSession.sparkContext.conf
+ .get(CarbonCommonConstants.CARBON_COMMON_LISTENER_REGISTER_CLASSNAME)
+ if (null != clsName && !clsName.isEmpty) {
+ CarbonReflectionUtils.createObject(clsName)
+ }
+ }
+
/**
* Return carbon table instance from cache or by looking up table in `sparkSession`
*/
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index eb00ebf..18c268c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -68,7 +68,7 @@ import org.apache.carbondata.events.exception.PreEventException
import org.apache.carbondata.hadoop.util.ObjectSerializationUtil
import org.apache.carbondata.processing.exception.DataLoadingException
import org.apache.carbondata.processing.loading.TableProcessingOperations
-import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
+import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, LoadTablePostExecutionEvent, LoadTablePreExecutionEvent, LoadTablePreStatusUpdateEvent}
import org.apache.carbondata.processing.loading.exception.NoRetryException
import org.apache.carbondata.processing.loading.model.{CarbonLoadModelBuilder, LoadOption}
import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -767,6 +767,11 @@ case class CarbonLoadDataCommand(
carbonLoadModel,
table,
operationContext)
+
+ val loadTablePreStatusUpdateEvent: LoadTablePreStatusUpdateEvent =
+ new LoadTablePreStatusUpdateEvent(table.getCarbonTableIdentifier, carbonLoadModel)
+ OperationListenerBus.getInstance().fireEvent(loadTablePreStatusUpdateEvent, operationContext)
+
} catch {
case e: Exception =>
throw new Exception(
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
index b341d6a..d30e96d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
@@ -67,6 +67,7 @@ object Spark2TestQueryExecutor {
.enableHiveSupport()
.config("spark.sql.warehouse.dir", warehouse)
.config("spark.sql.crossJoin.enabled", "true")
+ .config(CarbonCommonConstants.CARBON_COMMON_LISTENER_REGISTER_CLASSNAME, "")
.getOrCreateCarbonSession(null, TestQueryExecutor.metastoredb)
if (warehouse.startsWith("hdfs://")) {
System.setProperty(CarbonCommonConstants.HDFS_TEMP_LOCATION, warehouse)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
index ba2fe947..d381144 100644
--- a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -79,7 +79,7 @@ class CarbonSessionCatalog(
}
// Initialize all listeners to the Operation bus.
- CarbonEnv.initListeners()
+ CarbonEnv.init(sparkSession)
/**
* This method will invalidate carbonrelation from cache if carbon table is updated in
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/processing/src/main/java/org/apache/carbondata/processing/loading/events/LoadEvents.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/events/LoadEvents.java b/processing/src/main/java/org/apache/carbondata/processing/loading/events/LoadEvents.java
index a3fa292..50ebc34 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/events/LoadEvents.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/events/LoadEvents.java
@@ -182,15 +182,4 @@ public class LoadEvents {
}
}
- public static class LoadTableMergePartitionEvent extends Event {
- private String segmentPath;
-
- public LoadTableMergePartitionEvent(String segmentPath) {
- this.segmentPath = segmentPath;
- }
-
- public String getSegmentPath() {
- return segmentPath;
- }
- }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f5cdd5ca/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index 922a7ee..65827b0 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -1103,14 +1103,14 @@ public final class CarbonLoaderUtil {
* Merge index files with in the segment of partitioned table
* @param segmentId
* @param tablePath
- * @param uniqueId
* @return
* @throws IOException
*/
- public static String mergeIndexFilesinPartitionedSegment(String segmentId, String tablePath,
- String uniqueId) throws IOException {
+ public static String mergeIndexFilesinPartitionedSegment(String segmentId, String tablePath)
+ throws IOException {
CarbonIndexFileMergeWriter.SegmentIndexFIleMergeStatus segmentIndexFIleMergeStatus =
new CarbonIndexFileMergeWriter().mergeCarbonIndexFilesOfSegment(segmentId, tablePath);
+ String uniqueId = "";
if (segmentIndexFIleMergeStatus != null) {
uniqueId = System.currentTimeMillis() + "";
String newSegmentFileName = segmentId + "_" + uniqueId + CarbonTablePath.SEGMENT_EXT;
[03/10] carbondata git commit: [CARBONDATA-2254][DOC] Optimize
CarbonData documentation
Posted by ja...@apache.org.
[CARBONDATA-2254][DOC] Optimize CarbonData documentation
Optimize CarbonData documentation
This closes #2062
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/dbec6f9f
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/dbec6f9f
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/dbec6f9f
Branch: refs/heads/carbonfile
Commit: dbec6f9f28ecd607b846b5ba0edbf2e0c99d1dab
Parents: e39b0a1
Author: root <60...@qq.com>
Authored: Wed Mar 14 12:19:35 2018 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sun Mar 18 22:59:16 2018 +0800
----------------------------------------------------------------------
README.md | 4 ++--
docs/installation-guide.md | 4 ++--
.../scala/org/apache/carbondata/examples/AlterTableExample.scala | 2 +-
3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/dbec6f9f/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 3c9fe6a..3f45917 100644
--- a/README.md
+++ b/README.md
@@ -50,8 +50,8 @@ CarbonData is built using Apache Maven, to [build CarbonData](https://github.com
* [Useful Tips](https://github.com/apache/carbondata/blob/master/docs/useful-tips-on-carbondata.md)
## Other Technical Material
-[Apache CarbonData meetup material](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609)
-[Use Case Articles](https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Articles)
+* [Apache CarbonData meetup material](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609)
+* [Use Case Articles](https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Articles)
## Fork and Contribute
This is an active open source project for everyone, and we are always open to people who want to use this system or contribute to it.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/dbec6f9f/docs/installation-guide.md
----------------------------------------------------------------------
diff --git a/docs/installation-guide.md b/docs/installation-guide.md
index 37f1fdb..f679338 100644
--- a/docs/installation-guide.md
+++ b/docs/installation-guide.md
@@ -73,7 +73,7 @@ followed by :
**NOTE**: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.
-To get started with CarbonData : [Quick Start](quick-start-guide.md), [DDL Operations on CarbonData](ddl-operation-on-carbondata.md)
+To get started with CarbonData : [Quick Start](quick-start-guide.md), [Data Management on CarbonData](data-management-on-carbondata.md)
## Installing and Configuring CarbonData on Spark on YARN Cluster
@@ -129,7 +129,7 @@ mv carbondata.tar.gz carbonlib/
```
**NOTE**: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.
- Getting started with CarbonData : [Quick Start](quick-start-guide.md), [DDL Operations on CarbonData](ddl-operation-on-carbondata.md)
+ Getting started with CarbonData : [Quick Start](quick-start-guide.md), [Data Management on CarbonData](data-management-on-carbondata.md)
## Query Execution Using CarbonData Thrift Server
http://git-wip-us.apache.org/repos/asf/carbondata/blob/dbec6f9f/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
index 472dc44..6fffd30 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
@@ -26,7 +26,7 @@ import org.apache.carbondata.core.util.CarbonProperties
/**
* For alter table relative syntax, you can refer to DDL operation
- * document (ddl-operation-on-carbondata.md)
+ * document (data-management-on-carbondata.md)
*/
object AlterTableExample {
[08/10] carbondata git commit: [CARBONDATA-2224][File Level Reader
Support] External File level reader support
Posted by ja...@apache.org.
[CARBONDATA-2224][File Level Reader Support] External File level reader support
File level reader reads any carbondata file placed in any external file path.
This closes #2055
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/223c25de
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/223c25de
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/223c25de
Branch: refs/heads/carbonfile
Commit: 223c25de091fa5410875695492398dc5bfafa257
Parents: f5cdd5c
Author: sounakr <so...@gmail.com>
Authored: Sat Feb 24 07:55:14 2018 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Mar 20 23:44:00 2018 +0800
----------------------------------------------------------------------
.../core/metadata/schema/table/CarbonTable.java | 6 +
.../apache/carbondata/core/util/CarbonUtil.java | 209 +++++-
.../hadoop/api/CarbonFileInputFormat.java | 682 +++++++++++++++++++
.../carbondata/hadoop/util/SchemaReader.java | 17 +-
integration/spark-common-test/pom.xml | 6 +
...FileInputFormatWithExternalCarbonTable.scala | 240 +++++++
...tCreateTableUsingSparkCarbonFileFormat.scala | 327 +++++++++
...tSparkCarbonFileFormatWithSparkSession.scala | 176 +++++
.../carbondata/spark/rdd/CarbonScanRDD.scala | 64 +-
.../VectorizedCarbonRecordReader.java | 22 +-
.../management/CarbonLoadDataCommand.scala | 4 +-
.../command/table/CarbonDropTableCommand.scala | 2 +-
.../datasources/CarbonFileFormat.scala | 443 ------------
.../datasources/SparkCarbonFileFormat.scala | 269 ++++++++
.../datasources/SparkCarbonTableFormat.scala | 443 ++++++++++++
.../sql/execution/strategy/DDLStrategy.scala | 27 +-
.../spark/sql/hive/CarbonAnalysisRules.scala | 2 +-
.../spark/sql/parser/CarbonSparkSqlParser.scala | 47 +-
.../spark/sql/hive/CarbonSessionState.scala | 17 +-
.../spark/sql/hive/CarbonSessionState.scala | 16 +-
....apache.spark.sql.sources.DataSourceRegister | 3 +-
.../sdk/file/CSVCarbonWriterTest.java | 6 +-
22 files changed, 2522 insertions(+), 506 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index f14672f..278dc96 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -826,6 +826,12 @@ public class CarbonTable implements Serializable {
return external != null && external.equalsIgnoreCase("true");
}
+ public boolean isFileLevelExternalTable() {
+ String external = tableInfo.getFactTable().getTableProperties().get("_filelevelexternal");
+ return external != null && external.equalsIgnoreCase("true");
+ }
+
+
public long size() throws IOException {
Map<String, Long> dataIndexSize = CarbonUtil.calculateDataIndexSize(this);
Long dataSize = dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE);
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 06511f8..5a5f65d 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -41,6 +41,7 @@ import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
import org.apache.carbondata.core.datastore.impl.FileFactory;
import org.apache.carbondata.core.exception.InvalidConfigurationException;
import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
@@ -52,18 +53,26 @@ import org.apache.carbondata.core.metadata.SegmentFileStore;
import org.apache.carbondata.core.metadata.ValueEncoderMeta;
import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
import org.apache.carbondata.core.metadata.blocklet.SegmentInfo;
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypeAdapter;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.DecimalType;
import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
+import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
import org.apache.carbondata.core.metadata.schema.table.AggregationDataMapSchema;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.TableSchema;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
import org.apache.carbondata.core.mutate.UpdateVO;
+import org.apache.carbondata.core.reader.CarbonHeaderReader;
import org.apache.carbondata.core.reader.ThriftReader;
import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator;
import org.apache.carbondata.core.scan.model.ProjectionDimension;
@@ -77,6 +86,8 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
import org.apache.carbondata.format.BlockletHeader;
import org.apache.carbondata.format.DataChunk2;
import org.apache.carbondata.format.DataChunk3;
+import org.apache.carbondata.format.FileHeader;
+
import com.google.gson.Gson;
import com.google.gson.GsonBuilder;
@@ -1279,7 +1290,7 @@ public final class CarbonUtil {
int counter = 0;
for (int i = 0; i < wrapperColumnSchemaList.size(); i++) {
if (CarbonUtil.hasEncoding(wrapperColumnSchemaList.get(i).getEncodingList(),
- org.apache.carbondata.core.metadata.encoder.Encoding.DICTIONARY)) {
+ Encoding.DICTIONARY)) {
cardinality.add(dictionaryColumnCardinality[counter]);
counter++;
} else if (!wrapperColumnSchemaList.get(i).isDimensionColumn()) {
@@ -2068,6 +2079,202 @@ public final class CarbonUtil {
return tableInfo;
}
+ public static ColumnSchema thriftColumnSchmeaToWrapperColumnSchema(
+ org.apache.carbondata.format.ColumnSchema externalColumnSchema) {
+ ColumnSchema wrapperColumnSchema = new ColumnSchema();
+ wrapperColumnSchema.setColumnUniqueId(externalColumnSchema.getColumn_id());
+ wrapperColumnSchema.setColumnName(externalColumnSchema.getColumn_name());
+ wrapperColumnSchema.setColumnar(externalColumnSchema.isColumnar());
+ DataType dataType = thriftDataTyopeToWrapperDataType(externalColumnSchema.data_type);
+ if (DataTypes.isDecimal(dataType)) {
+ DecimalType decimalType = (DecimalType) dataType;
+ decimalType.setPrecision(externalColumnSchema.getPrecision());
+ decimalType.setScale(externalColumnSchema.getScale());
+ }
+ wrapperColumnSchema.setDataType(dataType);
+ wrapperColumnSchema.setDimensionColumn(externalColumnSchema.isDimension());
+ List<Encoding> encoders = new ArrayList<Encoding>();
+ for (org.apache.carbondata.format.Encoding encoder : externalColumnSchema.getEncoders()) {
+ encoders.add(fromExternalToWrapperEncoding(encoder));
+ }
+ wrapperColumnSchema.setEncodingList(encoders);
+ wrapperColumnSchema.setNumberOfChild(externalColumnSchema.getNum_child());
+ wrapperColumnSchema.setPrecision(externalColumnSchema.getPrecision());
+ wrapperColumnSchema.setColumnGroup(externalColumnSchema.getColumn_group_id());
+ wrapperColumnSchema.setScale(externalColumnSchema.getScale());
+ wrapperColumnSchema.setDefaultValue(externalColumnSchema.getDefault_value());
+ wrapperColumnSchema.setSchemaOrdinal(externalColumnSchema.getSchemaOrdinal());
+ Map<String, String> properties = externalColumnSchema.getColumnProperties();
+ if (properties != null) {
+ if (properties.get(CarbonCommonConstants.SORT_COLUMNS) != null) {
+ wrapperColumnSchema.setSortColumn(true);
+ }
+ }
+ wrapperColumnSchema.setFunction(externalColumnSchema.getAggregate_function());
+ List<org.apache.carbondata.format.ParentColumnTableRelation> parentColumnTableRelation =
+ externalColumnSchema.getParentColumnTableRelations();
+ if (null != parentColumnTableRelation) {
+ wrapperColumnSchema.setParentColumnTableRelations(
+ fromThriftToWrapperParentTableColumnRelations(parentColumnTableRelation));
+ }
+ return wrapperColumnSchema;
+ }
+
+ static List<ParentColumnTableRelation> fromThriftToWrapperParentTableColumnRelations(
+ List<org.apache.carbondata.format.ParentColumnTableRelation> thirftParentColumnRelation) {
+ List<ParentColumnTableRelation> parentColumnTableRelationList = new ArrayList<>();
+ for (org.apache.carbondata.format.ParentColumnTableRelation carbonTableRelation :
+ thirftParentColumnRelation) {
+ RelationIdentifier relationIdentifier =
+ new RelationIdentifier(carbonTableRelation.getRelationIdentifier().getDatabaseName(),
+ carbonTableRelation.getRelationIdentifier().getTableName(),
+ carbonTableRelation.getRelationIdentifier().getTableId());
+ ParentColumnTableRelation parentColumnTableRelation =
+ new ParentColumnTableRelation(relationIdentifier, carbonTableRelation.getColumnId(),
+ carbonTableRelation.getColumnName());
+ parentColumnTableRelationList.add(parentColumnTableRelation);
+ }
+ return parentColumnTableRelationList;
+ }
+
+ static Encoding fromExternalToWrapperEncoding(
+ org.apache.carbondata.format.Encoding encoderThrift) {
+ switch (encoderThrift) {
+ case DICTIONARY:
+ return Encoding.DICTIONARY;
+ case DELTA:
+ return Encoding.DELTA;
+ case RLE:
+ return Encoding.RLE;
+ case INVERTED_INDEX:
+ return Encoding.INVERTED_INDEX;
+ case BIT_PACKED:
+ return Encoding.BIT_PACKED;
+ case DIRECT_DICTIONARY:
+ return Encoding.DIRECT_DICTIONARY;
+ default:
+ throw new IllegalArgumentException(encoderThrift.toString() + " is not supported");
+ }
+ }
+
+ static DataType thriftDataTyopeToWrapperDataType(
+ org.apache.carbondata.format.DataType dataTypeThrift) {
+ switch (dataTypeThrift) {
+ case BOOLEAN:
+ return DataTypes.BOOLEAN;
+ case STRING:
+ return DataTypes.STRING;
+ case SHORT:
+ return DataTypes.SHORT;
+ case INT:
+ return DataTypes.INT;
+ case LONG:
+ return DataTypes.LONG;
+ case DOUBLE:
+ return DataTypes.DOUBLE;
+ case DECIMAL:
+ return DataTypes.createDefaultDecimalType();
+ case DATE:
+ return DataTypes.DATE;
+ case TIMESTAMP:
+ return DataTypes.TIMESTAMP;
+ case ARRAY:
+ return DataTypes.createDefaultArrayType();
+ case STRUCT:
+ return DataTypes.createDefaultStructType();
+ default:
+ return DataTypes.STRING;
+ }
+ }
+
+ public static List<String> getFilePathExternalFilePath(String path) {
+
+ // return the list of carbondata files in the given path.
+ CarbonFile segment = FileFactory.getCarbonFile(path, FileFactory.getFileType(path));
+ CarbonFile[] dataFiles = segment.listFiles(new CarbonFileFilter() {
+ @Override public boolean accept(CarbonFile file) {
+
+ if (file.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT)) {
+ return true;
+ }
+ return false;
+ }
+ });
+ List<String> filePaths = new ArrayList<>(dataFiles.length);
+ for (CarbonFile dfiles : dataFiles) {
+ filePaths.add(dfiles.getAbsolutePath());
+ }
+ return filePaths;
+ }
+
+ /**
+ * This method will read the schema file from a given path
+ *
+ * @param schemaFilePath
+ * @return
+ */
+ public static org.apache.carbondata.format.TableInfo inferSchemaFileExternalTable(
+ String carbonDataFilePath, AbsoluteTableIdentifier absoluteTableIdentifier,
+ boolean schemaExists) throws IOException {
+ TBaseCreator createTBase = new ThriftReader.TBaseCreator() {
+ public org.apache.thrift.TBase<org.apache.carbondata.format.TableInfo,
+ org.apache.carbondata.format.TableInfo._Fields> create() {
+ return new org.apache.carbondata.format.TableInfo();
+ }
+ };
+ if (schemaExists == false) {
+ List<String> filePaths =
+ getFilePathExternalFilePath(carbonDataFilePath + "/Fact/Part0/Segment_null");
+ String fistFilePath = null;
+ try {
+ fistFilePath = filePaths.get(0);
+ } catch (Exception e) {
+ LOGGER.error("CarbonData file is not present in the table location");
+ }
+ CarbonHeaderReader carbonHeaderReader = new CarbonHeaderReader(fistFilePath);
+ FileHeader fileHeader = carbonHeaderReader.readHeader();
+ List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
+ List<org.apache.carbondata.format.ColumnSchema> table_columns = fileHeader.getColumn_schema();
+ for (int i = 0; i < table_columns.size(); i++) {
+ ColumnSchema col = thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i));
+ col.setColumnReferenceId(col.getColumnUniqueId());
+ columnSchemaList.add(col);
+ }
+ TableSchema tableSchema = new TableSchema();
+ tableSchema.setTableName(absoluteTableIdentifier.getTableName());
+ tableSchema.setBucketingInfo(null);
+ tableSchema.setSchemaEvalution(null);
+ tableSchema.setTableId(UUID.randomUUID().toString());
+ tableSchema.setListOfColumns(columnSchemaList);
+
+ ThriftWrapperSchemaConverterImpl thriftWrapperSchemaConverter =
+ new ThriftWrapperSchemaConverterImpl();
+ SchemaEvolutionEntry schemaEvolutionEntry = new SchemaEvolutionEntry();
+ schemaEvolutionEntry.setTimeStamp(System.currentTimeMillis());
+ SchemaEvolution schemaEvol = new SchemaEvolution();
+ List<SchemaEvolutionEntry> schEntryList = new ArrayList<>();
+ schEntryList.add(schemaEvolutionEntry);
+ schemaEvol.setSchemaEvolutionEntryList(schEntryList);
+ tableSchema.setSchemaEvalution(schemaEvol);
+
+ org.apache.carbondata.format.TableSchema thriftFactTable =
+ thriftWrapperSchemaConverter.fromWrapperToExternalTableSchema(tableSchema);
+ org.apache.carbondata.format.TableInfo tableInfo =
+ new org.apache.carbondata.format.TableInfo(thriftFactTable,
+ new ArrayList<org.apache.carbondata.format.TableSchema>());
+
+ tableInfo.setDataMapSchemas(null);
+ return tableInfo;
+ } else {
+ ThriftReader thriftReader = new ThriftReader(carbonDataFilePath, createTBase);
+ thriftReader.open();
+ org.apache.carbondata.format.TableInfo tableInfo =
+ (org.apache.carbondata.format.TableInfo) thriftReader.read();
+ thriftReader.close();
+ return tableInfo;
+ }
+ }
+
public static void dropDatabaseDirectory(String databasePath)
throws IOException, InterruptedException {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
new file mode 100644
index 0000000..b86b1cc
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
@@ -0,0 +1,682 @@
+/*
+ * 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.carbondata.hadoop.api;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datamap.DataMapChooser;
+import org.apache.carbondata.core.datamap.DataMapLevel;
+import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.exception.InvalidConfigurationException;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
+import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.metadata.schema.PartitionInfo;
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.mutate.UpdateVO;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.filter.SingleTableProvider;
+import org.apache.carbondata.core.scan.filter.TableProvider;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.stats.QueryStatistic;
+import org.apache.carbondata.core.stats.QueryStatisticsConstants;
+import org.apache.carbondata.core.stats.QueryStatisticsRecorder;
+import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeConverter;
+import org.apache.carbondata.core.util.DataTypeConverterImpl;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+import org.apache.carbondata.hadoop.CarbonProjection;
+import org.apache.carbondata.hadoop.CarbonRecordReader;
+import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
+import org.apache.carbondata.hadoop.readsupport.impl.DictionaryDecodeReadSupport;
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
+import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
+import org.apache.carbondata.hadoop.util.SchemaReader;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.security.TokenCache;
+
+/**
+ * Input format of CarbonData file.
+ *
+ * @param <T>
+ */
+@InterfaceAudience.User
+@InterfaceStability.Evolving
+public class CarbonFileInputFormat<T> extends FileInputFormat<Void, T> implements Serializable {
+
+ public static final String READ_SUPPORT_CLASS = "carbon.read.support.class";
+ // comma separated list of input segment numbers
+ public static final String INPUT_SEGMENT_NUMBERS =
+ "mapreduce.input.carboninputformat.segmentnumbers";
+ private static final String VALIDATE_INPUT_SEGMENT_IDs =
+ "mapreduce.input.carboninputformat.validsegments";
+ // comma separated list of input files
+ public static final String INPUT_FILES = "mapreduce.input.carboninputformat.files";
+ private static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
+ private static final Log LOG = LogFactory.getLog(CarbonFileInputFormat.class);
+ private static final String FILTER_PREDICATE =
+ "mapreduce.input.carboninputformat.filter.predicate";
+ private static final String COLUMN_PROJECTION = "mapreduce.input.carboninputformat.projection";
+ private static final String TABLE_INFO = "mapreduce.input.carboninputformat.tableinfo";
+ private static final String CARBON_READ_SUPPORT = "mapreduce.input.carboninputformat.readsupport";
+ private static final String CARBON_CONVERTER = "mapreduce.input.carboninputformat.converter";
+ private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr";
+ public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
+ public static final String TABLE_NAME = "mapreduce.input.carboninputformat.tableName";
+ private static final String PARTITIONS_TO_PRUNE =
+ "mapreduce.input.carboninputformat.partitions.to.prune";
+ public static final String UPADTE_T =
+ "mapreduce.input.carboninputformat.partitions.to.prune";
+
+ // a cache for carbon table, it will be used in task side
+ private CarbonTable carbonTable;
+
+ /**
+ * Set the `tableInfo` in `configuration`
+ */
+ public static void setTableInfo(Configuration configuration, TableInfo tableInfo)
+ throws IOException {
+ if (null != tableInfo) {
+ configuration.set(TABLE_INFO, CarbonUtil.encodeToString(tableInfo.serialize()));
+ }
+ }
+
+ /**
+ * Get TableInfo object from `configuration`
+ */
+ private static TableInfo getTableInfo(Configuration configuration) throws IOException {
+ String tableInfoStr = configuration.get(TABLE_INFO);
+ if (tableInfoStr == null) {
+ return null;
+ } else {
+ TableInfo output = new TableInfo();
+ output.readFields(
+ new DataInputStream(
+ new ByteArrayInputStream(CarbonUtil.decodeStringToBytes(tableInfoStr))));
+ return output;
+ }
+ }
+
+
+ public static void setTablePath(Configuration configuration, String tablePath) {
+ configuration.set(FileInputFormat.INPUT_DIR, tablePath);
+ }
+
+ public static void setPartitionIdList(Configuration configuration, List<String> partitionIds) {
+ configuration.set(ALTER_PARTITION_ID, partitionIds.toString());
+ }
+
+
+ public static void setDataMapJob(Configuration configuration, DataMapJob dataMapJob)
+ throws IOException {
+ if (dataMapJob != null) {
+ String toString = ObjectSerializationUtil.convertObjectToString(dataMapJob);
+ configuration.set(DATA_MAP_DSTR, toString);
+ }
+ }
+
+ public static DataMapJob getDataMapJob(Configuration configuration) throws IOException {
+ String jobString = configuration.get(DATA_MAP_DSTR);
+ if (jobString != null) {
+ return (DataMapJob) ObjectSerializationUtil.convertStringToObject(jobString);
+ }
+ return null;
+ }
+
+ /**
+ * It sets unresolved filter expression.
+ *
+ * @param configuration
+ * @param filterExpression
+ */
+ public static void setFilterPredicates(Configuration configuration, Expression filterExpression) {
+ if (filterExpression == null) {
+ return;
+ }
+ try {
+ String filterString = ObjectSerializationUtil.convertObjectToString(filterExpression);
+ configuration.set(FILTER_PREDICATE, filterString);
+ } catch (Exception e) {
+ throw new RuntimeException("Error while setting filter expression to Job", e);
+ }
+ }
+
+ public static void setColumnProjection(Configuration configuration, CarbonProjection projection) {
+ if (projection == null || projection.isEmpty()) {
+ return;
+ }
+ String[] allColumns = projection.getAllColumns();
+ StringBuilder builder = new StringBuilder();
+ for (String column : allColumns) {
+ builder.append(column).append(",");
+ }
+ String columnString = builder.toString();
+ columnString = columnString.substring(0, columnString.length() - 1);
+ configuration.set(COLUMN_PROJECTION, columnString);
+ }
+
+ public static String getColumnProjection(Configuration configuration) {
+ return configuration.get(COLUMN_PROJECTION);
+ }
+
+
+ /**
+ * Set list of segments to access
+ */
+ public static void setSegmentsToAccess(Configuration configuration, List<Segment> validSegments) {
+ configuration.set(INPUT_SEGMENT_NUMBERS, CarbonUtil.convertToString(validSegments));
+ }
+
+ /**
+ * Set `CARBON_INPUT_SEGMENTS` from property to configuration
+ */
+ public static void setQuerySegment(Configuration conf, AbsoluteTableIdentifier identifier) {
+ String dbName = identifier.getCarbonTableIdentifier().getDatabaseName().toLowerCase();
+ String tbName = identifier.getCarbonTableIdentifier().getTableName().toLowerCase();
+ String segmentNumbersFromProperty = CarbonProperties.getInstance()
+ .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS + dbName + "." + tbName, "*");
+ if (!segmentNumbersFromProperty.trim().equals("*")) {
+ CarbonFileInputFormat
+ .setSegmentsToAccess(conf, Segment.toSegmentList(segmentNumbersFromProperty.split(",")));
+ }
+ }
+
+ /**
+ * set list of segment to access
+ */
+ public static void setValidateSegmentsToAccess(Configuration configuration, Boolean validate) {
+ configuration.set(CarbonFileInputFormat.VALIDATE_INPUT_SEGMENT_IDs, validate.toString());
+ }
+
+ /**
+ * get list of segment to access
+ */
+ public static boolean getValidateSegmentsToAccess(Configuration configuration) {
+ return configuration.get(CarbonFileInputFormat.VALIDATE_INPUT_SEGMENT_IDs, "true")
+ .equalsIgnoreCase("true");
+ }
+
+ /**
+ * set list of partitions to prune
+ */
+ public static void setPartitionsToPrune(Configuration configuration,
+ List<PartitionSpec> partitions) {
+ if (partitions == null) {
+ return;
+ }
+ try {
+ String partitionString =
+ ObjectSerializationUtil.convertObjectToString(new ArrayList<>(partitions));
+ configuration.set(PARTITIONS_TO_PRUNE, partitionString);
+ } catch (Exception e) {
+ throw new RuntimeException("Error while setting patition information to Job", e);
+ }
+ }
+
+ /**
+ * get list of partitions to prune
+ */
+ private static List<PartitionSpec> getPartitionsToPrune(Configuration configuration)
+ throws IOException {
+ String partitionString = configuration.get(PARTITIONS_TO_PRUNE);
+ if (partitionString != null) {
+ return (List<PartitionSpec>) ObjectSerializationUtil.convertStringToObject(partitionString);
+ }
+ return null;
+ }
+
+ public AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configuration)
+ throws IOException {
+ String tablePath = configuration.get(INPUT_DIR, "");
+ try {
+ return AbsoluteTableIdentifier
+ .from(tablePath, getDatabaseName(configuration), getTableName(configuration));
+ } catch (InvalidConfigurationException e) {
+ throw new IOException(e);
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ * Configurations FileInputFormat.INPUT_DIR
+ * are used to get table path to read.
+ *
+ * @param job
+ * @return List<InputSplit> list of CarbonInputSplit
+ * @throws IOException
+ */
+ @Override
+ public List<InputSplit> getSplits(JobContext job) throws IOException {
+ AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
+ CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
+ if (null == carbonTable) {
+ throw new IOException("Missing/Corrupt schema file for table.");
+ }
+ // TableDataMap blockletMap = DataMapStoreManager.getInstance()
+ // .getDataMap(identifier, BlockletDataMap.NAME, BlockletDataMapFactory.class.getName());
+
+ if (getValidateSegmentsToAccess(job.getConfiguration())) {
+ // get all valid segments and set them into the configuration
+ // check for externalTable segment (Segment_null)
+ // process and resolve the expression
+ Expression filter = getFilterPredicates(job.getConfiguration());
+ TableProvider tableProvider = new SingleTableProvider(carbonTable);
+ // this will be null in case of corrupt schema file.
+ PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName());
+ CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);
+
+ FilterResolverIntf filterInterface = CarbonInputFormatUtil
+ .resolveFilter(filter, carbonTable.getAbsoluteTableIdentifier(), tableProvider);
+
+ String segmentDir = CarbonTablePath.getSegmentPath(identifier.getTablePath(), "null");
+ FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
+ if (FileFactory.isFileExist(segmentDir, fileType)) {
+ // if external table Segments are found, add it to the List
+ List<Segment> externalTableSegments = new ArrayList<Segment>();
+ Segment seg = new Segment("null", null);
+ externalTableSegments.add(seg);
+
+ Map<String, String> indexFiles =
+ new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir);
+
+ if (indexFiles.size() == 0) {
+ throw new RuntimeException("Index file not present to read the carbondata file");
+ }
+ // do block filtering and get split
+ List<InputSplit> splits =
+ getSplits(job, filterInterface, externalTableSegments, null, partitionInfo, null);
+
+ return splits;
+ }
+ }
+ return null;
+ }
+
+
+
+ /**
+ * {@inheritDoc}
+ * Configurations FileInputFormat.INPUT_DIR, CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS
+ * are used to get table path to read.
+ *
+ * @return
+ * @throws IOException
+ */
+ private List<InputSplit> getSplits(JobContext job, FilterResolverIntf filterResolver,
+ List<Segment> validSegments, BitSet matchedPartitions, PartitionInfo partitionInfo,
+ List<Integer> oldPartitionIdList) throws IOException {
+
+ List<InputSplit> result = new LinkedList<InputSplit>();
+ UpdateVO invalidBlockVOForSegmentId = null;
+ Boolean isIUDTable = false;
+
+ AbsoluteTableIdentifier absoluteTableIdentifier =
+ getOrCreateCarbonTable(job.getConfiguration()).getAbsoluteTableIdentifier();
+ SegmentUpdateStatusManager updateStatusManager =
+ new SegmentUpdateStatusManager(absoluteTableIdentifier);
+
+ isIUDTable = (updateStatusManager.getUpdateStatusDetails().length != 0);
+
+ // for each segment fetch blocks matching filter in Driver BTree
+ List<CarbonInputSplit> dataBlocksOfSegment =
+ getDataBlocksOfSegment(job, absoluteTableIdentifier, filterResolver, matchedPartitions,
+ validSegments, partitionInfo, oldPartitionIdList);
+ for (CarbonInputSplit inputSplit : dataBlocksOfSegment) {
+
+ // Get the UpdateVO for those tables on which IUD operations being performed.
+ if (isIUDTable) {
+ invalidBlockVOForSegmentId =
+ updateStatusManager.getInvalidTimestampRange(inputSplit.getSegmentId());
+ }
+ String[] deleteDeltaFilePath = null;
+ if (isIUDTable) {
+ // In case IUD is not performed in this table avoid searching for
+ // invalidated blocks.
+ if (CarbonUtil
+ .isInvalidTableBlock(inputSplit.getSegmentId(), inputSplit.getPath().toString(),
+ invalidBlockVOForSegmentId, updateStatusManager)) {
+ continue;
+ }
+ // When iud is done then only get delete delta files for a block
+ try {
+ deleteDeltaFilePath = updateStatusManager
+ .getDeleteDeltaFilePath(inputSplit.getPath().toString(), inputSplit.getSegmentId());
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ }
+ inputSplit.setDeleteDeltaFiles(deleteDeltaFilePath);
+ result.add(inputSplit);
+ }
+ return result;
+ }
+
+ protected Expression getFilterPredicates(Configuration configuration) {
+ try {
+ String filterExprString = configuration.get(FILTER_PREDICATE);
+ if (filterExprString == null) {
+ return null;
+ }
+ Object filter = ObjectSerializationUtil.convertStringToObject(filterExprString);
+ return (Expression) filter;
+ } catch (IOException e) {
+ throw new RuntimeException("Error while reading filter expression", e);
+ }
+ }
+
+ /**
+ * get data blocks of given segment
+ */
+ private List<CarbonInputSplit> getDataBlocksOfSegment(JobContext job,
+ AbsoluteTableIdentifier absoluteTableIdentifier, FilterResolverIntf resolver,
+ BitSet matchedPartitions, List<Segment> segmentIds, PartitionInfo partitionInfo,
+ List<Integer> oldPartitionIdList) throws IOException {
+
+ QueryStatisticsRecorder recorder = CarbonTimeStatisticsFactory.createDriverRecorder();
+ QueryStatistic statistic = new QueryStatistic();
+
+ // get tokens for all the required FileSystem for table path
+ TokenCache.obtainTokensForNamenodes(job.getCredentials(),
+ new Path[] { new Path(absoluteTableIdentifier.getTablePath()) }, job.getConfiguration());
+ boolean distributedCG = Boolean.parseBoolean(CarbonProperties.getInstance()
+ .getProperty(CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
+ CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT));
+ DataMapExprWrapper dataMapExprWrapper =
+ DataMapChooser.get().choose(getOrCreateCarbonTable(job.getConfiguration()), resolver);
+ DataMapJob dataMapJob = getDataMapJob(job.getConfiguration());
+ List<PartitionSpec> partitionsToPrune = getPartitionsToPrune(job.getConfiguration());
+ List<ExtendedBlocklet> prunedBlocklets;
+ if (distributedCG || dataMapExprWrapper.getDataMapType() == DataMapLevel.FG) {
+ DistributableDataMapFormat datamapDstr =
+ new DistributableDataMapFormat(absoluteTableIdentifier, dataMapExprWrapper,
+ segmentIds, partitionsToPrune,
+ BlockletDataMapFactory.class.getName());
+ prunedBlocklets = dataMapJob.execute(datamapDstr, resolver);
+ // Apply expression on the blocklets.
+ prunedBlocklets = dataMapExprWrapper.pruneBlocklets(prunedBlocklets);
+ } else {
+ prunedBlocklets = dataMapExprWrapper.prune(segmentIds, partitionsToPrune);
+ }
+
+ List<CarbonInputSplit> resultFilterredBlocks = new ArrayList<>();
+ int partitionIndex = 0;
+ List<Integer> partitionIdList = new ArrayList<>();
+ if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
+ partitionIdList = partitionInfo.getPartitionIds();
+ }
+ for (ExtendedBlocklet blocklet : prunedBlocklets) {
+ long partitionId = CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(
+ CarbonTablePath.DataFileUtil.getTaskNo(blocklet.getPath()));
+
+ // OldPartitionIdList is only used in alter table partition command because it change
+ // partition info first and then read data.
+ // For other normal query should use newest partitionIdList
+ if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
+ if (oldPartitionIdList != null) {
+ partitionIndex = oldPartitionIdList.indexOf((int)partitionId);
+ } else {
+ partitionIndex = partitionIdList.indexOf((int)partitionId);
+ }
+ }
+ if (partitionIndex != -1) {
+ // matchedPartitions variable will be null in two cases as follows
+ // 1. the table is not a partition table
+ // 2. the table is a partition table, and all partitions are matched by query
+ // for partition table, the task id of carbaondata file name is the partition id.
+ // if this partition is not required, here will skip it.
+ if (matchedPartitions == null || matchedPartitions.get(partitionIndex)) {
+ CarbonInputSplit inputSplit = convertToCarbonInputSplit(blocklet);
+ if (inputSplit != null) {
+ resultFilterredBlocks.add(inputSplit);
+ }
+ }
+ }
+ }
+ statistic
+ .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER, System.currentTimeMillis());
+ recorder.recordStatisticsForDriver(statistic, job.getConfiguration().get("query.id"));
+ return resultFilterredBlocks;
+ }
+
+ private CarbonInputSplit convertToCarbonInputSplit(ExtendedBlocklet blocklet) throws IOException {
+ CarbonInputSplit split =
+ CarbonInputSplit.from(blocklet.getSegmentId(),
+ blocklet.getBlockletId(), new FileSplit(new Path(blocklet.getPath()), 0,
+ blocklet.getLength(), blocklet.getLocations()),
+ ColumnarFormatVersion.valueOf((short) blocklet.getDetailInfo().getVersionNumber()),
+ blocklet.getDataMapWriterPath());
+ split.setDetailInfo(blocklet.getDetailInfo());
+ return split;
+ }
+
+ @Override
+ public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
+ TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
+ Configuration configuration = taskAttemptContext.getConfiguration();
+ QueryModel queryModel = createQueryModel(inputSplit, taskAttemptContext);
+ CarbonReadSupport<T> readSupport = getReadSupportClass(configuration);
+ return new CarbonRecordReader<T>(queryModel, readSupport);
+ }
+
+ public QueryModel createQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+ throws IOException {
+ Configuration configuration = taskAttemptContext.getConfiguration();
+ CarbonTable carbonTable = getOrCreateCarbonTable(configuration);
+ TableProvider tableProvider = new SingleTableProvider(carbonTable);
+
+ // query plan includes projection column
+ String projectionString = getColumnProjection(configuration);
+ String[] projectionColumnNames = null;
+ if (projectionString != null) {
+ projectionColumnNames = projectionString.split(",");
+ }
+ QueryModel queryModel = carbonTable.createQueryWithProjection(
+ projectionColumnNames, getDataTypeConverter(configuration));
+
+ // set the filter to the query model in order to filter blocklet before scan
+ Expression filter = getFilterPredicates(configuration);
+ boolean[] isFilterDimensions = new boolean[carbonTable.getDimensionOrdinalMax()];
+ // getAllMeasures returns list of visible and invisible columns
+ boolean[] isFilterMeasures =
+ new boolean[carbonTable.getAllMeasures().size()];
+ CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, isFilterDimensions,
+ isFilterMeasures);
+ queryModel.setIsFilterDimensions(isFilterDimensions);
+ queryModel.setIsFilterMeasures(isFilterMeasures);
+ FilterResolverIntf filterIntf = CarbonInputFormatUtil
+ .resolveFilter(filter, carbonTable.getAbsoluteTableIdentifier(), tableProvider);
+ queryModel.setFilterExpressionResolverTree(filterIntf);
+
+ // update the file level index store if there are invalid segment
+ if (inputSplit instanceof CarbonMultiBlockSplit) {
+ CarbonMultiBlockSplit split = (CarbonMultiBlockSplit) inputSplit;
+ List<String> invalidSegments = split.getAllSplits().get(0).getInvalidSegments();
+ if (invalidSegments.size() > 0) {
+ queryModel.setInvalidSegmentIds(invalidSegments);
+ }
+ List<UpdateVO> invalidTimestampRangeList =
+ split.getAllSplits().get(0).getInvalidTimestampRange();
+ if ((null != invalidTimestampRangeList) && (invalidTimestampRangeList.size() > 0)) {
+ queryModel.setInvalidBlockForSegmentId(invalidTimestampRangeList);
+ }
+ }
+ return queryModel;
+ }
+
+ private CarbonTable getOrCreateCarbonTable(Configuration configuration) throws IOException {
+ CarbonTable carbonTableTemp;
+ if (carbonTable == null) {
+ // carbon table should be created either from deserialized table info (schema saved in
+ // hive metastore) or by reading schema in HDFS (schema saved in HDFS)
+ TableInfo tableInfo = getTableInfo(configuration);
+ CarbonTable localCarbonTable;
+ if (tableInfo != null) {
+ localCarbonTable = CarbonTable.buildFromTableInfo(tableInfo);
+ } else {
+ String schemaPath = CarbonTablePath
+ .getSchemaFilePath(getAbsoluteTableIdentifier(configuration).getTablePath());
+ if (!FileFactory.isFileExist(schemaPath, FileFactory.getFileType(schemaPath))) {
+ TableInfo tableInfoInfer =
+ SchemaReader.inferSchema(getAbsoluteTableIdentifier(configuration));
+ localCarbonTable = CarbonTable.buildFromTableInfo(tableInfoInfer);
+ } else {
+ localCarbonTable =
+ SchemaReader.readCarbonTableFromStore(getAbsoluteTableIdentifier(configuration));
+ }
+ }
+ this.carbonTable = localCarbonTable;
+ return localCarbonTable;
+ } else {
+ carbonTableTemp = this.carbonTable;
+ return carbonTableTemp;
+ }
+ }
+
+
+ public CarbonReadSupport<T> getReadSupportClass(Configuration configuration) {
+ String readSupportClass = configuration.get(CARBON_READ_SUPPORT);
+ //By default it uses dictionary decoder read class
+ CarbonReadSupport<T> readSupport = null;
+ if (readSupportClass != null) {
+ try {
+ Class<?> myClass = Class.forName(readSupportClass);
+ Constructor<?> constructor = myClass.getConstructors()[0];
+ Object object = constructor.newInstance();
+ if (object instanceof CarbonReadSupport) {
+ readSupport = (CarbonReadSupport) object;
+ }
+ } catch (ClassNotFoundException ex) {
+ LOG.error("Class " + readSupportClass + "not found", ex);
+ } catch (Exception ex) {
+ LOG.error("Error while creating " + readSupportClass, ex);
+ }
+ } else {
+ readSupport = new DictionaryDecodeReadSupport<>();
+ }
+ return readSupport;
+ }
+
+ @Override
+ protected boolean isSplitable(JobContext context, Path filename) {
+ try {
+ // Don't split the file if it is local file system
+ FileSystem fileSystem = filename.getFileSystem(context.getConfiguration());
+ if (fileSystem instanceof LocalFileSystem) {
+ return false;
+ }
+ } catch (Exception e) {
+ return true;
+ }
+ return true;
+ }
+
+ /**
+ * return valid segment to access
+ */
+ private String[] getSegmentsToAccess(JobContext job) {
+ String segmentString = job.getConfiguration().get(INPUT_SEGMENT_NUMBERS, "");
+ if (segmentString.trim().isEmpty()) {
+ return new String[0];
+ }
+ return segmentString.split(",");
+ }
+
+ public static DataTypeConverter getDataTypeConverter(Configuration configuration)
+ throws IOException {
+ String converter = configuration.get(CARBON_CONVERTER);
+ if (converter == null) {
+ return new DataTypeConverterImpl();
+ }
+ return (DataTypeConverter) ObjectSerializationUtil.convertStringToObject(converter);
+ }
+
+ public static void setDatabaseName(Configuration configuration, String databaseName) {
+ if (null != databaseName) {
+ configuration.set(DATABASE_NAME, databaseName);
+ }
+ }
+
+ public static String getDatabaseName(Configuration configuration)
+ throws InvalidConfigurationException {
+ String databseName = configuration.get(DATABASE_NAME);
+ if (null == databseName) {
+ throw new InvalidConfigurationException("Database name is not set.");
+ }
+ return databseName;
+ }
+
+ public static void setTableName(Configuration configuration, String tableName) {
+ if (null != tableName) {
+ configuration.set(TABLE_NAME, tableName);
+ }
+ }
+
+ public static String getTableName(Configuration configuration)
+ throws InvalidConfigurationException {
+ String tableName = configuration.get(TABLE_NAME);
+ if (tableName == null) {
+ throw new InvalidConfigurationException("Table name is not set");
+ }
+ return tableName;
+ }
+
+ public org.apache.hadoop.mapred.RecordReader<Void, T> getRecordReader(
+ org.apache.hadoop.mapred.InputSplit split, JobConf job, Reporter reporter)
+ throws IOException {
+ return null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
index dfa8dd1..ab7c333 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
@@ -28,7 +28,6 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
import org.apache.carbondata.core.metadata.schema.table.TableInfo;
import org.apache.carbondata.core.util.CarbonUtil;
import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
/**
* TODO: It should be removed after store manager implementation.
@@ -59,6 +58,7 @@ public class SchemaReader {
throw new IOException("File does not exist: " + schemaFilePath);
}
}
+
/**
* the method returns the Wrapper TableInfo
*
@@ -79,4 +79,19 @@ public class SchemaReader {
carbonTableIdentifier.getTableName(),
identifier.getTablePath());
}
+
+
+ public static TableInfo inferSchema(AbsoluteTableIdentifier identifier)
+ throws IOException {
+ // This routine is going to infer schema from the carbondata file footer
+ // Convert the ColumnSchema -> TableSchema -> TableInfo.
+ // Return the TableInfo.
+ org.apache.carbondata.format.TableInfo tableInfo =
+ CarbonUtil.inferSchemaFileExternalTable(identifier.getTablePath(), identifier, false);
+ SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
+ TableInfo wrapperTableInfo = schemaConverter
+ .fromExternalToWrapperTableInfo(tableInfo, identifier.getDatabaseName(),
+ identifier.getTableName(), identifier.getTablePath());
+ return wrapperTableInfo;
+ }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark-common-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/pom.xml b/integration/spark-common-test/pom.xml
index b7f19fd..1c6cee9 100644
--- a/integration/spark-common-test/pom.xml
+++ b/integration/spark-common-test/pom.xml
@@ -105,6 +105,12 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>org.apache.carbondata</groupId>
+ <artifactId>carbondata-store-sdk</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-hive-thriftserver_${scala.binary.version}</artifactId>
<scope>test</scope>
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
new file mode 100644
index 0000000..8b1f63f
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCarbonFileInputFormatWithExternalCarbonTable.scala
@@ -0,0 +1,240 @@
+/*
+ * 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.carbondata.spark.testsuite.createTable
+
+import java.io.File
+
+import org.apache.commons.io.FileUtils
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.sdk.file.{CarbonWriter, Schema}
+
+
+class TestCarbonFileInputFormatWithExternalCarbonTable extends QueryTest with BeforeAndAfterAll {
+
+ var writerPath = new File(this.getClass.getResource("/").getPath
+ +
+ "../." +
+ "./src/test/resources/SparkCarbonFileFormat/WriterOutput/")
+ .getCanonicalPath
+ //getCanonicalPath gives path with \, so code expects /. Need to handle in code ?
+ writerPath = writerPath.replace("\\", "/");
+
+
+ def buildTestData(persistSchema:Boolean) = {
+
+ FileUtils.deleteDirectory(new File(writerPath))
+
+ val schema = new StringBuilder()
+ .append("[ \n")
+ .append(" {\"name\":\"string\"},\n")
+ .append(" {\"age\":\"int\"},\n")
+ .append(" {\"height\":\"double\"}\n")
+ .append("]")
+ .toString()
+
+ try {
+ val builder = CarbonWriter.builder()
+ val writer =
+ if (persistSchema) {
+ builder.persistSchemaFile(true)
+ builder.withSchema(Schema.parseJson(schema)).outputPath(writerPath).buildWriterForCSVInput()
+ } else {
+ builder.withSchema(Schema.parseJson(schema)).outputPath(writerPath).buildWriterForCSVInput()
+ }
+
+ var i = 0
+ while (i < 100) {
+ writer.write(Array[String]("robot" + i, String.valueOf(i), String.valueOf(i.toDouble / 2)))
+ i += 1
+ }
+ writer.close()
+ } catch {
+ case ex: Exception => None
+ case _ => None
+ }
+ }
+
+ def cleanTestData() = {
+ FileUtils.deleteDirectory(new File(writerPath))
+ }
+
+ def deleteIndexFile(path: String, extension: String) : Unit = {
+ val file: CarbonFile = FileFactory
+ .getCarbonFile(path, FileFactory.getFileType(path))
+
+ for (eachDir <- file.listFiles) {
+ if (!eachDir.isDirectory) {
+ if (eachDir.getName.endsWith(extension)) {
+ CarbonUtil.deleteFoldersAndFilesSilent(eachDir)
+ }
+ } else {
+ deleteIndexFile(eachDir.getPath, extension)
+ }
+ }
+ }
+
+ override def beforeAll(): Unit = {
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+ // create carbon table and insert data
+ }
+
+ override def afterAll(): Unit = {
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+ }
+
+ //TO DO, need to remove segment dependency and tableIdentifier Dependency
+ test("read carbondata files (sdk Writer Output) using the Carbonfile ") {
+ buildTestData(false)
+ assert(new File(writerPath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ //new provider Carbonfile
+ sql(
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ |'$writerPath' """.stripMargin)
+
+ sql("Describe formatted sdkOutputTable").show(false)
+
+ sql("select * from sdkOutputTable").show(false)
+
+ sql("select * from sdkOutputTable limit 3").show(false)
+
+ sql("select name from sdkOutputTable").show(false)
+
+ sql("select age from sdkOutputTable").show(false)
+
+ sql("select * from sdkOutputTable where age > 2 and age < 8").show(200, false)
+
+ sql("select * from sdkOutputTable where name = 'robot3'").show(200, false)
+
+ sql("select * from sdkOutputTable where name like 'robo%' limit 5").show(200, false)
+
+ sql("select * from sdkOutputTable where name like '%obot%' limit 2").show(200, false)
+
+ sql("select sum(age) from sdkOutputTable where name like 'robot1%' ").show(200, false)
+
+ sql("select count(*) from sdkOutputTable where name like 'robot%' ").show(200, false)
+
+ sql("select count(*) from sdkOutputTable").show(200, false)
+
+ sql("DROP TABLE sdkOutputTable")
+ // drop table should not delete the files
+ assert(new File(writerPath).exists())
+ cleanTestData()
+ }
+
+ test("should not allow to alter datasource carbontable ") {
+ buildTestData(false)
+ assert(new File(writerPath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ //data source file format
+ sql(
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ |'$writerPath' """.stripMargin)
+
+ val exception = intercept[MalformedCarbonCommandException]
+ {
+ sql("Alter table sdkOutputTable change age age BIGINT")
+ }
+ assert(exception.getMessage()
+ .contains("Unsupported alter operation on Carbon external fileformat table"))
+
+ sql("DROP TABLE sdkOutputTable")
+ // drop table should not delete the files
+ assert(new File(writerPath).exists())
+ cleanTestData()
+ }
+
+ test("Read sdk writer output file without index file should fail") {
+ buildTestData(false)
+ deleteIndexFile(writerPath, CarbonCommonConstants.UPDATE_INDEX_FILE_EXT)
+ assert(new File(writerPath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ //data source file format
+ sql(
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ |'$writerPath' """.stripMargin)
+
+ //org.apache.spark.SparkException: Index file not present to read the carbondata file
+ val exception = intercept[java.lang.RuntimeException]
+ {
+ sql("select * from sdkOutputTable").show(false)
+ }
+ assert(exception.getMessage().contains("Index file not present to read the carbondata file"))
+
+ sql("DROP TABLE sdkOutputTable")
+ // drop table should not delete the files
+ assert(new File(writerPath).exists())
+ cleanTestData()
+ }
+
+
+ test("Read sdk writer output file without Carbondata file should fail") {
+ buildTestData(false)
+ deleteIndexFile(writerPath, CarbonCommonConstants.FACT_FILE_EXT)
+ assert(new File(writerPath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ val exception = intercept[Exception] {
+ // data source file format
+ sql(
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ |'$writerPath' """.stripMargin)
+ }
+ assert(exception.getMessage()
+ .contains("Operation not allowed: Invalid table path provided:"))
+
+
+ // drop table should not delete the files
+ assert(new File(writerPath).exists())
+ cleanTestData()
+ }
+
+
+ test("Read sdk writer output file without any file should fail") {
+ buildTestData(false)
+ deleteIndexFile(writerPath, CarbonCommonConstants.FACT_FILE_EXT)
+ deleteIndexFile(writerPath, CarbonCommonConstants.UPDATE_INDEX_FILE_EXT)
+ assert(new File(writerPath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ val exception = intercept[Exception] {
+ //data source file format
+ sql(
+ s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'Carbonfile' LOCATION
+ |'$writerPath' """.stripMargin)
+
+ sql("select * from sdkOutputTable").show(false)
+ }
+ assert(exception.getMessage()
+ .contains("Operation not allowed: Invalid table path provided:"))
+
+ // drop table should not delete the files
+ assert(new File(writerPath).exists())
+ cleanTestData()
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableUsingSparkCarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableUsingSparkCarbonFileFormat.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableUsingSparkCarbonFileFormat.scala
new file mode 100644
index 0000000..d284e50
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableUsingSparkCarbonFileFormat.scala
@@ -0,0 +1,327 @@
+/*
+ * 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.carbondata.spark.testsuite.createTable
+
+import java.io.File
+
+import org.apache.commons.io.FileUtils
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.sdk.file.{CarbonWriter, Schema}
+
+class TestCreateTableUsingSparkCarbonFileFormat extends QueryTest with BeforeAndAfterAll {
+
+
+ override def beforeAll(): Unit = {
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+ }
+
+ override def afterAll(): Unit = {
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+ }
+
+ var writerPath = new File(this.getClass.getResource("/").getPath
+ +
+ "../." +
+ "./src/test/resources/SparkCarbonFileFormat/WriterOutput/")
+ .getCanonicalPath
+ //getCanonicalPath gives path with \, so code expects /. Need to handle in code ?
+ writerPath = writerPath.replace("\\", "/");
+
+ val filePath = writerPath + "/Fact/Part0/Segment_null/"
+
+ def buildTestData(persistSchema:Boolean) = {
+
+ FileUtils.deleteDirectory(new File(writerPath))
+
+ val schema = new StringBuilder()
+ .append("[ \n")
+ .append(" {\"name\":\"string\"},\n")
+ .append(" {\"age\":\"int\"},\n")
+ .append(" {\"height\":\"double\"}\n")
+ .append("]")
+ .toString()
+
+ try {
+ val builder = CarbonWriter.builder()
+ val writer =
+ if (persistSchema) {
+ builder.persistSchemaFile(true)
+ builder.withSchema(Schema.parseJson(schema)).outputPath(writerPath).buildWriterForCSVInput()
+ } else {
+ builder.withSchema(Schema.parseJson(schema)).outputPath(writerPath).buildWriterForCSVInput()
+ }
+
+ var i = 0
+ while (i < 100) {
+ writer.write(Array[String]("robot" + i, String.valueOf(i), String.valueOf(i.toDouble / 2)))
+ i += 1
+ }
+ writer.close()
+ } catch {
+ case ex: Exception => None
+ case _ => None
+ }
+ }
+
+ def cleanTestData() = {
+ FileUtils.deleteDirectory(new File(writerPath))
+ }
+
+ def deleteIndexFile(path: String, extension: String) : Unit = {
+ val file: CarbonFile = FileFactory
+ .getCarbonFile(path, FileFactory.getFileType(path))
+
+ for (eachDir <- file.listFiles) {
+ if (!eachDir.isDirectory) {
+ if (eachDir.getName.endsWith(extension)) {
+ CarbonUtil.deleteFoldersAndFilesSilent(eachDir)
+ }
+ } else {
+ deleteIndexFile(eachDir.getPath, extension)
+ }
+ }
+ }
+
+ //TO DO, need to remove segment dependency and tableIdentifier Dependency
+ test("read carbondata files (sdk Writer Output) using the SparkCarbonFileFormat ") {
+ buildTestData(false)
+ assert(new File(filePath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ //data source file format
+ if (sqlContext.sparkContext.version.startsWith("2.1")) {
+ //data source file format
+ sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ } else if (sqlContext.sparkContext.version.startsWith("2.2")) {
+ //data source file format
+ sql(
+ s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ |'$filePath' """.stripMargin)
+ } else{
+ // TO DO
+ }
+
+ sql("Describe formatted sdkOutputTable").show(false)
+
+ sql("select * from sdkOutputTable").show(false)
+
+ sql("select * from sdkOutputTable limit 3").show(false)
+
+ sql("select name from sdkOutputTable").show(false)
+
+ sql("select age from sdkOutputTable").show(false)
+
+ sql("select * from sdkOutputTable where age > 2 and age < 8").show(200,false)
+
+ sql("select * from sdkOutputTable where name = 'robot3'").show(200,false)
+
+ sql("select * from sdkOutputTable where name like 'robo%' limit 5").show(200,false)
+
+ sql("select * from sdkOutputTable where name like '%obot%' limit 2").show(200,false)
+
+ sql("select sum(age) from sdkOutputTable where name like 'robot1%' ").show(200,false)
+
+ sql("select count(*) from sdkOutputTable where name like 'robot%' ").show(200,false)
+
+ sql("select count(*) from sdkOutputTable").show(200,false)
+
+ sql("DROP TABLE sdkOutputTable")
+ // drop table should not delete the files
+ assert(new File(filePath).exists())
+ cleanTestData()
+ }
+
+
+ test("should not allow to alter datasource carbontable ") {
+ buildTestData(false)
+ assert(new File(filePath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+
+ if (sqlContext.sparkContext.version.startsWith("2.1")) {
+ //data source file format
+ sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ } else if (sqlContext.sparkContext.version.startsWith("2.2")) {
+ //data source file format
+ sql(
+ s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ |'$filePath' """.stripMargin)
+ } else{
+ // TO DO
+ }
+
+ val exception = intercept[MalformedCarbonCommandException]
+ {
+ sql("Alter table sdkOutputTable change age age BIGINT")
+ }
+ assert(exception.getMessage().contains("Unsupported alter operation on hive table"))
+
+ sql("DROP TABLE sdkOutputTable")
+ // drop table should not delete the files
+ assert(new File(filePath).exists())
+ cleanTestData()
+ }
+
+ test("Read sdk writer output file without Carbondata file should fail") {
+ buildTestData(false)
+ deleteIndexFile(writerPath, CarbonCommonConstants.FACT_FILE_EXT)
+ assert(new File(filePath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ val exception = intercept[org.apache.spark.SparkException] {
+ // data source file format
+ if (sqlContext.sparkContext.version.startsWith("2.1")) {
+ //data source file format
+ sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ } else if (sqlContext.sparkContext.version.startsWith("2.2")) {
+ //data source file format
+ sql(
+ s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ |'$filePath' """.stripMargin)
+ } else{
+ // TO DO
+ }
+ }
+ assert(exception.getMessage()
+ .contains("CarbonData file is not present in the location mentioned in DDL"))
+
+ // drop table should not delete the files
+ assert(new File(filePath).exists())
+ cleanTestData()
+ }
+
+
+ test("Read sdk writer output file without any file should fail") {
+ buildTestData(false)
+ deleteIndexFile(writerPath, CarbonCommonConstants.UPDATE_INDEX_FILE_EXT)
+ deleteIndexFile(writerPath, CarbonCommonConstants.FACT_FILE_EXT)
+ assert(new File(filePath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ val exception = intercept[org.apache.spark.SparkException] {
+ //data source file format
+ if (sqlContext.sparkContext.version.startsWith("2.1")) {
+ //data source file format
+ sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ } else if (sqlContext.sparkContext.version.startsWith("2.2")) {
+ //data source file format
+ sql(
+ s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ |'$filePath' """.stripMargin)
+ } else{
+ // TO DO
+ }
+
+ sql("select * from sdkOutputTable").show(false)
+ }
+ assert(exception.getMessage()
+ .contains("CarbonData file is not present in the location mentioned in DDL"))
+
+ // drop table should not delete the files
+ assert(new File(filePath).exists())
+ cleanTestData()
+ }
+
+ test("Read sdk writer output file withSchema") {
+ buildTestData(true)
+ assert(new File(filePath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ //data source file format
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ if (sqlContext.sparkContext.version.startsWith("2.1")) {
+ //data source file format
+ sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ } else if (sqlContext.sparkContext.version.startsWith("2.2")) {
+ //data source file format
+ sql(
+ s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ |'$filePath' """.stripMargin)
+ } else{
+ // TO DO
+ }
+
+ sql("Describe formatted sdkOutputTable").show(false)
+
+ sql("select * from sdkOutputTable").show(false)
+
+ sql("select * from sdkOutputTable limit 3").show(false)
+
+ sql("select name from sdkOutputTable").show(false)
+
+ sql("select age from sdkOutputTable").show(false)
+
+ sql("select * from sdkOutputTable where age > 2 and age < 8").show(200, false)
+
+ sql("select * from sdkOutputTable where name = 'robot3'").show(200, false)
+
+ sql("select * from sdkOutputTable where name like 'robo%' limit 5").show(200, false)
+
+ sql("select * from sdkOutputTable where name like '%obot%' limit 2").show(200, false)
+
+ sql("select sum(age) from sdkOutputTable where name like 'robot1%' ").show(200, false)
+
+ sql("select count(*) from sdkOutputTable where name like 'robot%' ").show(200, false)
+
+ sql("select count(*) from sdkOutputTable").show(200, false)
+
+ sql("DROP TABLE sdkOutputTable")
+
+ // drop table should not delete the files
+ assert(new File(filePath).exists())
+ cleanTestData()
+ }
+
+ test("Read sdk writer output file without index file should fail") {
+ buildTestData(false)
+ deleteIndexFile(writerPath, CarbonCommonConstants.UPDATE_INDEX_FILE_EXT)
+ assert(new File(filePath).exists())
+ sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+ if (sqlContext.sparkContext.version.startsWith("2.1")) {
+ //data source file format
+ sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ } else if (sqlContext.sparkContext.version.startsWith("2.2")) {
+ //data source file format
+ sql(
+ s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ |'$filePath' """.stripMargin)
+ } else{
+ // TO DO
+ }
+ //org.apache.spark.SparkException: Index file not present to read the carbondata file
+ val exception = intercept[org.apache.spark.SparkException]
+ {
+ sql("select * from sdkOutputTable").show(false)
+ }
+ assert(exception.getMessage().contains("Index file not present to read the carbondata file"))
+
+ sql("DROP TABLE sdkOutputTable")
+ // drop table should not delete the files
+ assert(new File(filePath).exists())
+ cleanTestData()
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
new file mode 100644
index 0000000..9a46676
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestSparkCarbonFileFormatWithSparkSession.scala
@@ -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.carbondata.spark.testsuite.createTable
+
+import java.io.File
+
+import org.apache.commons.io.FileUtils
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.sdk.file.{CarbonWriter, Schema}
+
+
+object TestSparkCarbonFileFormatWithSparkSession {
+
+ var writerPath = new File(this.getClass.getResource("/").getPath
+ +
+ "../." +
+ "./src/test/resources/SparkCarbonFileFormat/WriterOutput/")
+ .getCanonicalPath
+ //getCanonicalPath gives path with \, so code expects /. Need to handle in code ?
+ writerPath = writerPath.replace("\\", "/");
+
+ val filePath = writerPath + "/Fact/Part0/Segment_null/"
+
+ def buildTestData(persistSchema:Boolean) = {
+
+ FileUtils.deleteDirectory(new File(writerPath))
+
+ val schema = new StringBuilder()
+ .append("[ \n")
+ .append(" {\"name\":\"string\"},\n")
+ .append(" {\"age\":\"int\"},\n")
+ .append(" {\"height\":\"double\"}\n")
+ .append("]")
+ .toString()
+
+ try {
+ val builder = CarbonWriter.builder()
+ val writer =
+ if (persistSchema) {
+ builder.persistSchemaFile(true)
+ builder.withSchema(Schema.parseJson(schema)).outputPath(writerPath).buildWriterForCSVInput()
+ } else {
+ builder.withSchema(Schema.parseJson(schema)).outputPath(writerPath).buildWriterForCSVInput()
+ }
+
+ var i = 0
+ while (i < 100) {
+ writer.write(Array[String]("robot" + i, String.valueOf(i), String.valueOf(i.toDouble / 2)))
+ i += 1
+ }
+ writer.close()
+ } catch {
+ case ex: Exception => None
+ case _ => None
+ }
+ }
+
+ def cleanTestData() = {
+ FileUtils.deleteDirectory(new File(writerPath))
+ }
+
+ def deleteIndexFile(path: String, extension: String) : Unit = {
+ val file: CarbonFile = FileFactory
+ .getCarbonFile(path, FileFactory.getFileType(path))
+
+ for (eachDir <- file.listFiles) {
+ if (!eachDir.isDirectory) {
+ if (eachDir.getName.endsWith(extension)) {
+ CarbonUtil.deleteFoldersAndFilesSilent(eachDir)
+ }
+ } else {
+ deleteIndexFile(eachDir.getPath, extension)
+ }
+ }
+ }
+
+ def main(args: Array[String]): Unit = {
+ val rootPath = new File(this.getClass.getResource("/").getPath
+ + "../../../..").getCanonicalPath
+ val storeLocation = s"$rootPath/examples/spark2/target/store"
+ val warehouse = s"$rootPath/examples/spark2/target/warehouse"
+ val metastoredb = s"$rootPath/examples/spark2/target/metastore_db"
+
+ // clean data folder
+ if (true) {
+ val clean = (path: String) => FileUtils.deleteDirectory(new File(path))
+ clean(storeLocation)
+ clean(warehouse)
+ clean(metastoredb)
+ }
+
+ val spark = SparkSession
+ .builder()
+ .master("local")
+ .appName("TestSparkCarbonFileFormatWithSparkSession")
+ .enableHiveSupport()
+ .config("spark.sql.warehouse.dir", warehouse)
+ .config("javax.jdo.option.ConnectionURL",
+ s"jdbc:derby:;databaseName=$metastoredb;create=true")
+ .getOrCreate()
+
+ CarbonProperties.getInstance()
+ .addProperty("carbon.storelocation", storeLocation)
+
+ spark.sparkContext.setLogLevel("WARN")
+
+ CarbonProperties.getInstance()
+ .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd HH:mm:ss")
+ .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy/MM/dd")
+ buildTestData(false)
+ assert(new File(filePath).exists())
+ //data source file format
+ if (spark.sparkContext.version.startsWith("2.1")) {
+ //data source file format
+ spark.sql(s"""CREATE TABLE sdkOutputTable USING Carbonfile OPTIONS (PATH '$filePath') """)
+ } else if (spark.sparkContext.version.startsWith("2.2")) {
+ //data source file format
+ spark.sql(
+ s"""CREATE TABLE sdkOutputTable USING Carbonfile LOCATION
+ |'$filePath' """.stripMargin)
+ } else{
+ // TO DO
+ }
+
+ spark.sql("Describe formatted sdkOutputTable").show(false)
+
+ spark.sql("select * from sdkOutputTable").show(false)
+
+ spark.sql("select * from sdkOutputTable limit 3").show(false)
+
+ spark.sql("select name from sdkOutputTable").show(false)
+
+ spark.sql("select age from sdkOutputTable").show(false)
+
+ spark.sql("select * from sdkOutputTable where age > 2 and age < 8").show(200,false)
+
+ spark.sql("select * from sdkOutputTable where name = 'robot3'").show(200,false)
+
+ spark.sql("select * from sdkOutputTable where name like 'robo%' limit 5").show(200,false)
+
+ spark.sql("select * from sdkOutputTable where name like '%obot%' limit 2").show(200,false)
+
+ spark.sql("select sum(age) from sdkOutputTable where name like 'robot1%' ").show(200,false)
+
+ spark.sql("select count(*) from sdkOutputTable where name like 'robot%' ").show(200,false)
+
+ spark.sql("select count(*) from sdkOutputTable").show(200,false)
+
+ spark.sql("DROP TABLE sdkOutputTable")
+ // drop table should not delete the files
+ assert(new File(filePath).exists())
+ cleanTestData()
+
+ spark.stop()
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 49a8023..6afd2c0 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -42,7 +42,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
import org.apache.carbondata.core.datastore.block.Distributable
import org.apache.carbondata.core.indexstore.PartitionSpec
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.schema.table.TableInfo
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
import org.apache.carbondata.core.scan.expression.Expression
import org.apache.carbondata.core.scan.filter.FilterUtil
import org.apache.carbondata.core.scan.model.QueryModel
@@ -50,7 +50,7 @@ import org.apache.carbondata.core.stats.{QueryStatistic, QueryStatisticsConstant
import org.apache.carbondata.core.statusmanager.FileFormat
import org.apache.carbondata.core.util._
import org.apache.carbondata.hadoop._
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.hadoop.api.{CarbonFileInputFormat, CarbonTableInputFormat}
import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
import org.apache.carbondata.processing.util.CarbonLoaderUtil
import org.apache.carbondata.spark.InitInputMetrics
@@ -90,13 +90,21 @@ class CarbonScanRDD(
val jobConf = new JobConf(conf)
SparkHadoopUtil.get.addCredentials(jobConf)
val job = Job.getInstance(jobConf)
- val format = prepareInputFormatForDriver(job.getConfiguration)
-
+ val fileLevelExternal = tableInfo.getFactTable().getTableProperties().get("_filelevelexternal")
+ val format = if (fileLevelExternal != null && fileLevelExternal.equalsIgnoreCase("true")) {
+ prepareFileInputFormatForDriver(job.getConfiguration)
+ } else {
+ prepareInputFormatForDriver(job.getConfiguration)
+ }
// initialise query_id for job
job.getConfiguration.set("query.id", queryId)
// get splits
val splits = format.getSplits(job)
+ if ((splits == null) && format.isInstanceOf[CarbonFileInputFormat[Object]]) {
+ throw new SparkException(
+ "CarbonData file not exist in the segment_null (SDK writer Output) path")
+ }
// separate split
// 1. for batch splits, invoke distributeSplits method to create partitions
@@ -113,7 +121,7 @@ class CarbonScanRDD(
}
val batchPartitions = distributeColumnarSplits(columnarSplits)
// check and remove InExpression from filterExpression
- checkAndRemoveInExpressinFromFilterExpression(format, batchPartitions)
+ checkAndRemoveInExpressinFromFilterExpression(batchPartitions)
if (streamSplits.isEmpty) {
batchPartitions.toArray
} else {
@@ -354,7 +362,9 @@ class CarbonScanRDD(
case _ =>
// create record reader for CarbonData file format
if (vectorReader) {
- val carbonRecordReader = createVectorizedCarbonRecordReader(model, inputMetricsStats)
+ val carbonRecordReader = createVectorizedCarbonRecordReader(model,
+ inputMetricsStats,
+ "true")
if (carbonRecordReader == null) {
new CarbonRecordReader(model,
format.getReadSupportClass(attemptContext.getConfiguration), inputMetricsStats)
@@ -431,6 +441,16 @@ class CarbonScanRDD(
createInputFormat(conf)
}
+ def prepareFileInputFormatForDriver(conf: Configuration): CarbonFileInputFormat[Object] = {
+ CarbonFileInputFormat.setTableInfo(conf, tableInfo)
+ CarbonFileInputFormat.setDatabaseName(conf, tableInfo.getDatabaseName)
+ CarbonFileInputFormat.setTableName(conf, tableInfo.getFactTable.getTableName)
+ if (partitionNames != null) {
+ CarbonFileInputFormat.setPartitionsToPrune(conf, partitionNames.asJava)
+ }
+ createFileInputFormat(conf)
+ }
+
private def prepareInputFormatForExecutor(conf: Configuration): CarbonTableInputFormat[Object] = {
CarbonTableInputFormat.setCarbonReadSupport(conf, readSupport)
val tableInfo1 = getTableInfo
@@ -441,6 +461,32 @@ class CarbonScanRDD(
createInputFormat(conf)
}
+ private def createFileInputFormat(conf: Configuration): CarbonFileInputFormat[Object] = {
+ val format = new CarbonFileInputFormat[Object]
+ CarbonFileInputFormat.setTablePath(conf,
+ identifier.appendWithLocalPrefix(identifier.getTablePath))
+ CarbonFileInputFormat.setQuerySegment(conf, identifier)
+ CarbonFileInputFormat.setFilterPredicates(conf, filterExpression)
+ CarbonFileInputFormat.setColumnProjection(conf, columnProjection)
+ CarbonFileInputFormat.setDataMapJob(conf, new SparkDataMapJob)
+ if (CarbonProperties.getInstance().getProperty(
+ CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
+ CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT).toBoolean) {
+ CarbonTableInputFormat.setDataMapJob(conf, new SparkDataMapJob)
+ }
+
+ // when validate segments is disabled in thread local update it to CarbonTableInputFormat
+ val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
+ if (carbonSessionInfo != null) {
+ CarbonTableInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getSessionParams
+ .getProperty(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
+ identifier.getCarbonTableIdentifier.getDatabaseName + "." +
+ identifier.getCarbonTableIdentifier.getTableName, "true").toBoolean)
+ }
+ format
+ }
+
+
private def createInputFormat(conf: Configuration): CarbonTableInputFormat[Object] = {
val format = new CarbonTableInputFormat[Object]
CarbonTableInputFormat.setTablePath(conf,
@@ -485,7 +531,6 @@ class CarbonScanRDD(
* @param identifiedPartitions
*/
private def checkAndRemoveInExpressinFromFilterExpression(
- format: CarbonTableInputFormat[Object],
identifiedPartitions: mutable.Buffer[Partition]) = {
if (null != filterExpression) {
if (identifiedPartitions.nonEmpty &&
@@ -533,12 +578,13 @@ class CarbonScanRDD(
}
def createVectorizedCarbonRecordReader(queryModel: QueryModel,
- inputMetricsStats: InputMetricsStats): RecordReader[Void, Object] = {
+ inputMetricsStats: InputMetricsStats, enableBatch: String): RecordReader[Void, Object] = {
val name = "org.apache.carbondata.spark.vectorreader.VectorizedCarbonRecordReader"
try {
val cons = Class.forName(name).getDeclaredConstructors
cons.head.setAccessible(true)
- cons.head.newInstance(queryModel, inputMetricsStats).asInstanceOf[RecordReader[Void, Object]]
+ cons.head.newInstance(queryModel, inputMetricsStats, enableBatch)
+ .asInstanceOf[RecordReader[Void, Object]]
} catch {
case e: Exception =>
LOGGER.error(e)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index 73da878..903bf44 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -91,10 +91,21 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
private InputMetricsStats inputMetricsStats;
- public VectorizedCarbonRecordReader(QueryModel queryModel, InputMetricsStats inputMetricsStats) {
+ public VectorizedCarbonRecordReader(QueryModel queryModel, InputMetricsStats inputMetricsStats,
+ String enableBatch) {
this.queryModel = queryModel;
this.inputMetricsStats = inputMetricsStats;
- enableReturningBatches();
+ if (enableBatch.equals("true")) {
+ enableReturningBatches();
+ }
+ }
+
+
+ /*
+ * Can be called before any rows are returned to enable returning columnar batches directly.
+ */
+ public void enableReturningBatches() {
+ returnColumnarBatch = true;
}
/**
@@ -273,12 +284,7 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
if (columnarBatch == null) initBatch();
}
- /*
- * Can be called before any rows are returned to enable returning columnar batches directly.
- */
- private void enableReturningBatches() {
- returnColumnarBatch = true;
- }
+
/**
* Advances to the next batch of rows. Returns false if there are no more.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/223c25de/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 18c268c..7a6aa53 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Sort}
import org.apache.spark.sql.execution.LogicalRDD
import org.apache.spark.sql.execution.SQLExecution.EXECUTION_ID_KEY
import org.apache.spark.sql.execution.command.{AtomicRunnableCommand, DataLoadTableFileMapping, UpdateTableModel}
-import org.apache.spark.sql.execution.datasources.{CarbonFileFormat, CatalogFileIndex, FindDataSourceTable, HadoopFsRelation, LogicalRelation}
+import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FindDataSourceTable, HadoopFsRelation, LogicalRelation, SparkCarbonTableFormat}
import org.apache.spark.sql.hive.CarbonRelation
import org.apache.spark.sql.optimizer.CarbonFilters
import org.apache.spark.sql.types._
@@ -1020,7 +1020,7 @@ case class CarbonLoadDataCommand(
partitionSchema = partitionSchema,
dataSchema = dataSchema,
bucketSpec = catalogTable.bucketSpec,
- fileFormat = new CarbonFileFormat,
+ fileFormat = new SparkCarbonTableFormat,
options = options.toMap)(sparkSession = sparkSession)
CarbonReflectionUtils.getLogicalRelation(hdfsRelation,