You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by zh...@apache.org on 2019/12/09 14:51:33 UTC
[carbondata] branch master updated: [CARBONDATA-3606] Remove custom
partition feature #3498
This is an automated email from the ASF dual-hosted git repository.
zhangzc pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git
The following commit(s) were added to refs/heads/master by this push:
new 1a181d1 [CARBONDATA-3606] Remove custom partition feature #3498
1a181d1 is described below
commit 1a181d19fea55eb128f8fbe104dadb772b53a601
Author: Jacky Li <ja...@qq.com>
AuthorDate: Fri Dec 6 14:45:35 2019 +0800
[CARBONDATA-3606] Remove custom partition feature #3498
Custom partition feature is deprecated, use Hive standard partition instead.
This closes #3498
---
.../exceptions/DeprecatedFeatureException.java | 12 +-
.../core/constants/CarbonCommonConstants.java | 3 -
.../ThriftWrapperSchemaConverterImpl.java | 36 +-
.../core/metadata/schema/PartitionInfo.java | 89 +-
.../scan/filter/FilterExpressionProcessor.java | 152 ---
.../core/scan/filter/FilterProcessor.java | 7 -
.../core/scan/filter/partition/AndFilterImpl.java | 44 -
.../scan/filter/partition/EqualToFilterImpl.java | 60 -
.../core/scan/filter/partition/InFilterImpl.java | 61 -
.../partition/KeepAllPartitionFilterImpl.java | 34 -
.../core/scan/filter/partition/OrFilterImpl.java | 44 -
.../scan/filter/partition/PartitionFilterIntf.java | 36 -
.../scan/filter/partition/PartitionFilterUtil.java | 209 ----
.../partition/PruneAllPartitionFilterImpl.java | 34 -
.../scan/filter/partition/RangeFilterImpl.java | 88 --
.../core/scan/partition/HashPartitioner.java | 43 -
.../core/scan/partition/ListPartitioner.java | 79 --
.../core/scan/partition/PartitionUtil.java | 129 --
.../core/scan/partition/RangePartitioner.java | 91 --
.../apache/carbondata/core/util/CarbonUtil.java | 10 -
docs/ddl-of-carbondata.md | 93 --
.../examples/CarbonPartitionExample.scala | 215 ----
.../apache/carbondata/examplesCI/RunExamples.scala | 4 -
format/src/main/thrift/schema.thrift | 1 +
.../hadoop/api/CarbonFileInputFormat.java | 16 +-
.../carbondata/hadoop/api/CarbonInputFormat.java | 40 +-
.../hadoop/api/CarbonTableInputFormat.java | 138 +--
.../impl/DictionaryDecodeReadSupport.java | 5 -
.../sdv/generated/BatchSortLoad3TestCase.scala | 9 -
.../cluster/sdv/generated/PartitionTestCase.scala | 520 --------
.../testsuite/binary/TestBinaryDataType.scala | 63 +-
.../dataload/TestLoadDataWithCompression.scala | 4 +-
.../TestCreateDDLForComplexMapType.scala | 2 +-
.../createTable/TestCreateTableLike.scala | 14 -
.../createTable/TestCreateTableWithSortScope.scala | 8 +-
.../TestNonTransactionalCarbonTable.scala | 6 +-
.../CompactionSupportGlobalSortBigFileTest.scala | 2 +-
.../CompactionSupportGlobalSortFunctionTest.scala | 2 +-
.../CompactionSupportGlobalSortParameterTest.scala | 2 +-
.../dataload/TestGlobalSortDataLoad.scala | 14 -
...adDataWithMalformedCarbonCommandException.scala | 4 +-
.../TestAllDataTypeForPartitionTable.scala | 1238 --------------------
.../TestCompactionForPartitionTable.scala | 82 --
.../partition/TestDDLForPartitionTable.scala | 430 -------
...DDLForPartitionTableWithDefaultProperties.scala | 206 ----
.../TestDataLoadingForPartitionTable.scala | 409 -------
.../partition/TestQueryForPartitionTable.scala | 287 -----
.../testsuite/partition/TestShowPartitions.scala | 133 +--
.../partition/TestUpdateForPartitionTable.scala | 27 +-
.../spark/partition/DropPartitionCallable.java | 40 -
.../spark/partition/SplitPartitionCallable.java | 42 -
.../apache/carbondata/spark/PartitionFactory.scala | 65 -
.../spark/rdd/AlterTableLoadPartitionRDD.scala | 106 --
.../carbondata/spark/rdd/CarbonMergerRDD.scala | 13 +-
.../spark/rdd/CarbonScanPartitionRDD.scala | 271 -----
.../spark/rdd/CarbonSparkPartition.scala | 1 -
.../spark/rdd/NewCarbonDataLoadRDD.scala | 3 +-
.../carbondata/spark/rdd/PartitionDropper.scala | 118 --
.../carbondata/spark/rdd/PartitionSplitter.scala | 93 --
.../carbondata/spark/util/CarbonScalaUtil.scala | 4 +-
.../apache/carbondata/spark/util/CommonUtil.scala | 186 +--
.../spark/sql/catalyst/CarbonDDLSqlParser.scala | 74 +-
.../org/apache/spark/util/PartitionUtils.scala | 265 +----
.../spark/rdd/CarbonDataRDDFactory.scala | 168 +--
.../mutation/CarbonProjectForUpdateCommand.scala | 8 -
.../CarbonAlterTableDropPartitionCommand.scala | 273 -----
.../CarbonAlterTableSplitPartitionCommand.scala | 323 -----
.../CarbonShowCarbonPartitionsCommand.scala | 56 -
.../table/CarbonDescribeFormattedCommand.scala | 11 +-
.../spark/sql/execution/strategy/DDLStrategy.scala | 21 +-
.../spark/sql/parser/CarbonSpark2SqlParser.scala | 40 +-
.../sql/parser/CarbonSparkSqlParserUtil.scala | 22 +-
.../partition/TestAlterPartitionTable.scala | 926 ---------------
.../loading/model/CarbonLoadModelBuilder.java | 22 +-
.../CarbonRowDataWriterProcessorStepImpl.java | 7 -
.../processing/partition/DataPartitioner.java | 35 -
.../carbondata/processing/partition/Partition.java | 35 -
.../spliter/AbstractCarbonQueryExecutor.java | 90 --
.../partition/spliter/CarbonSplitExecutor.java | 73 --
.../partition/spliter/RowResultProcessor.java | 113 --
.../store/writer/AbstractFactDataWriter.java | 11 -
.../writer/v3/CarbonFactDataWriterImplV3.java | 8 -
.../processing/util/CarbonDataProcessorUtil.java | 44 -
83 files changed, 103 insertions(+), 8699 deletions(-)
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/Partitioner.java b/common/src/main/java/org/apache/carbondata/common/exceptions/DeprecatedFeatureException.java
similarity index 77%
rename from core/src/main/java/org/apache/carbondata/core/scan/partition/Partitioner.java
rename to common/src/main/java/org/apache/carbondata/common/exceptions/DeprecatedFeatureException.java
index 772a98e..d38a131 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/Partitioner.java
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/DeprecatedFeatureException.java
@@ -15,13 +15,11 @@
* limitations under the License.
*/
-package org.apache.carbondata.core.scan.partition;
+package org.apache.carbondata.common.exceptions;
-import java.io.Serializable;
+public class DeprecatedFeatureException extends RuntimeException {
-public interface Partitioner extends Serializable {
-
- int numPartitions();
-
- int getPartition(Object key);
+ public DeprecatedFeatureException(String featureName) {
+ super(featureName + " is deprecated in CarbonData 2.0");
+ }
}
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 9b12251..7a4fa77 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
@@ -441,9 +441,6 @@ public final class CarbonCommonConstants {
public static final String SORT_SCOPE = "sort_scope";
public static final String RANGE_COLUMN = "range_column";
public static final String PARTITION_TYPE = "partition_type";
- public static final String NUM_PARTITIONS = "num_partitions";
- public static final String RANGE_INFO = "range_info";
- public static final String LIST_INFO = "list_info";
public static final String COLUMN_PROPERTIES = "columnproperties";
// table block size in MB
public static final String TABLE_BLOCKSIZE = "table_blocksize";
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index c64ef09..9fd25d6 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -22,6 +22,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException;
import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -239,15 +240,10 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
}
switch (wrapperPartitionType) {
case HASH:
- return org.apache.carbondata.format.PartitionType.HASH;
case LIST:
- return org.apache.carbondata.format.PartitionType.LIST;
case RANGE:
- return org.apache.carbondata.format.PartitionType.RANGE;
case RANGE_INTERVAL:
- return org.apache.carbondata.format.PartitionType.RANGE_INTERVAL;
- case NATIVE_HIVE:
- return org.apache.carbondata.format.PartitionType.NATIVE_HIVE;
+ throw new DeprecatedFeatureException("Custom Partition");
default:
return org.apache.carbondata.format.PartitionType.NATIVE_HIVE;
}
@@ -260,16 +256,8 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
for (ColumnSchema wrapperColumnSchema : wrapperPartitionInfo.getColumnSchemaList()) {
thriftColumnSchema.add(fromWrapperToExternalColumnSchema(wrapperColumnSchema));
}
- org.apache.carbondata.format.PartitionInfo externalPartitionInfo =
- new org.apache.carbondata.format.PartitionInfo(thriftColumnSchema,
- fromWrapperToExternalPartitionType(wrapperPartitionInfo.getPartitionType()));
- externalPartitionInfo.setList_info(wrapperPartitionInfo.getListInfo());
- externalPartitionInfo.setRange_info(wrapperPartitionInfo.getRangeInfo());
- externalPartitionInfo.setNum_partitions(wrapperPartitionInfo.getNumPartitions());
- externalPartitionInfo.setMax_partition(wrapperPartitionInfo.getMaxPartitionId());
- externalPartitionInfo.setPartition_ids(wrapperPartitionInfo
- .getPartitionIds());
- return externalPartitionInfo;
+ return new org.apache.carbondata.format.PartitionInfo(thriftColumnSchema,
+ fromWrapperToExternalPartitionType(wrapperPartitionInfo.getPartitionType()));
}
/* (non-Javadoc)
@@ -578,15 +566,10 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
}
switch (externalPartitionType) {
case HASH:
- return PartitionType.HASH;
case LIST:
- return PartitionType.LIST;
case RANGE:
- return PartitionType.RANGE;
case RANGE_INTERVAL:
- return PartitionType.RANGE_INTERVAL;
- case NATIVE_HIVE:
- return PartitionType.NATIVE_HIVE;
+ throw new DeprecatedFeatureException("Custom Partition");
default:
return PartitionType.NATIVE_HIVE;
}
@@ -599,15 +582,8 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
externalPartitionInfo.getPartition_columns()) {
wrapperColumnSchema.add(fromExternalToWrapperColumnSchema(columnSchema));
}
- PartitionInfo wrapperPartitionInfo = new PartitionInfo(wrapperColumnSchema,
+ return new PartitionInfo(wrapperColumnSchema,
fromExternalToWrapperPartitionType(externalPartitionInfo.getPartition_type()));
- wrapperPartitionInfo.setListInfo(externalPartitionInfo.getList_info());
- wrapperPartitionInfo.setRangeInfo(externalPartitionInfo.getRange_info());
- wrapperPartitionInfo.setNumPartitions(externalPartitionInfo.getNum_partitions());
- wrapperPartitionInfo.setPartitionIds(externalPartitionInfo
- .getPartition_ids());
- wrapperPartitionInfo.setMaxPartitionId(externalPartitionInfo.getMax_partition());
- return wrapperPartitionInfo;
}
/* (non-Javadoc)
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
index cd69b83..7854267 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
@@ -42,27 +42,32 @@ public class PartitionInfo implements Serializable, Writable {
/**
* range information defined for range partition table
*/
+ @Deprecated
private List<String> rangeInfo;
/**
- * value list defined for list partition table
+ * list information defined for range partition table
*/
+ @Deprecated
private List<List<String>> listInfo;
/**
* total count of partitions
*/
+ @Deprecated
private int numPartitions;
/**
* current max partition id, increase only, will be used in alter table partition operation
*/
+ @Deprecated
private int maxPartitionId;
/**
* record the partitionId in the logical ascending order
* initiate when table created and changed when alter table
*/
+ @Deprecated
private List<Integer> partitionIds;
public PartitionInfo() {
@@ -75,36 +80,6 @@ public class PartitionInfo implements Serializable, Writable {
this.partitionIds = new ArrayList<>();
}
- /**
- * add partition means split default partition, add in last directly
- */
- public void addPartition(int addPartitionCount) {
- for (int i = 0; i < addPartitionCount; i++) {
- partitionIds.add(++maxPartitionId);
- numPartitions++;
- }
- }
-
- /**
- * e.g. original partition[0,1,2,3,4,5]
- * split partition 2 to partition 6,7,8 (will not reuse 2)
- * then sourcePartitionId is 2, newPartitionNumbers is 3
- * @param sourcePartitionIndex
- * @param newPartitionNumbers
- */
- public void splitPartition(int sourcePartitionIndex, int newPartitionNumbers) {
- partitionIds.remove(sourcePartitionIndex);
- for (int i = 0; i < newPartitionNumbers; i++) {
- partitionIds.add(sourcePartitionIndex + i, ++maxPartitionId);
- }
- numPartitions = numPartitions - 1 + newPartitionNumbers;
- }
-
- public void dropPartition(int index) {
- partitionIds.remove(index);
- numPartitions--;
- }
-
public List<ColumnSchema> getColumnSchemaList() {
return columnSchemaList;
}
@@ -117,58 +92,6 @@ public class PartitionInfo implements Serializable, Writable {
return partitionType;
}
- public void setRangeInfo(List<String> rangeInfo) {
- this.rangeInfo = rangeInfo;
- }
-
- public List<String> getRangeInfo() {
- return rangeInfo;
- }
-
- public void setListInfo(List<List<String>> listInfo) {
- this.listInfo = listInfo;
- }
-
- public List<List<String>> getListInfo() {
- return listInfo;
- }
-
- public void initialize(int partitionNum) {
- for (int i = 0; i < partitionNum; i++) {
- partitionIds.add(i);
- }
- maxPartitionId = partitionNum - 1;
- numPartitions = partitionNum;
- }
-
- public void setNumPartitions(int numPartitions) {
- this.numPartitions = numPartitions;
- }
-
- public int getNumPartitions() {
- return numPartitions;
- }
-
- public int getMaxPartitionId() {
- return maxPartitionId;
- }
-
- public void setMaxPartitionId(int maxPartitionId) {
- this.maxPartitionId = maxPartitionId;
- }
-
- public List<Integer> getPartitionIds() {
- return partitionIds;
- }
-
- public void setPartitionIds(List<Integer> partitionIdList) {
- this.partitionIds = partitionIdList;
- }
-
- public int getPartitionId(int index) {
- return partitionIds.get(index);
- }
-
@Override
public void write(DataOutput output) throws IOException {
output.writeInt(columnSchemaList.size());
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
index 1c2f51c..7b6b5a8 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
@@ -19,27 +19,19 @@ package org.apache.carbondata.core.scan.filter;
import java.io.IOException;
import java.util.BitSet;
-import java.util.List;
import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datastore.DataRefNode;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
import org.apache.carbondata.core.scan.expression.BinaryExpression;
-import org.apache.carbondata.core.scan.expression.ColumnExpression;
import org.apache.carbondata.core.scan.expression.Expression;
import org.apache.carbondata.core.scan.expression.LiteralExpression;
import org.apache.carbondata.core.scan.expression.conditional.BinaryConditionalExpression;
import org.apache.carbondata.core.scan.expression.conditional.ConditionalExpression;
-import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
import org.apache.carbondata.core.scan.expression.conditional.GreaterThanEqualToExpression;
-import org.apache.carbondata.core.scan.expression.conditional.GreaterThanExpression;
import org.apache.carbondata.core.scan.expression.conditional.InExpression;
-import org.apache.carbondata.core.scan.expression.conditional.LessThanEqualToExpression;
import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression;
-import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
import org.apache.carbondata.core.scan.expression.conditional.StartsWithExpression;
import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
import org.apache.carbondata.core.scan.expression.logical.AndExpression;
@@ -48,14 +40,6 @@ import org.apache.carbondata.core.scan.expression.logical.TrueExpression;
import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
import org.apache.carbondata.core.scan.filter.executer.ImplicitColumnFilterExecutor;
import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
-import org.apache.carbondata.core.scan.filter.partition.AndFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.EqualToFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.InFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.KeepAllPartitionFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.OrFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.PartitionFilterIntf;
-import org.apache.carbondata.core.scan.filter.partition.PruneAllPartitionFilterImpl;
-import org.apache.carbondata.core.scan.filter.partition.RangeFilterImpl;
import org.apache.carbondata.core.scan.filter.resolver.ConditionalFilterResolverImpl;
import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
import org.apache.carbondata.core.scan.filter.resolver.LogicalFilterResolverImpl;
@@ -63,8 +47,6 @@ import org.apache.carbondata.core.scan.filter.resolver.RowLevelFilterResolverImp
import org.apache.carbondata.core.scan.filter.resolver.RowLevelRangeFilterResolverImpl;
import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.FalseConditionalResolverImpl;
import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.TrueConditionalResolverImpl;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
import org.apache.log4j.Logger;
@@ -91,140 +73,6 @@ public class FilterExpressionProcessor implements FilterProcessor {
}
/**
- * Get the map of required partitions
- * The value of "1" in BitSet represent the required partition
- * @param expressionTree
- * @param partitionInfo
- * @return
- */
- @Override
- public BitSet getFilteredPartitions(Expression expressionTree,
- PartitionInfo partitionInfo) {
- Partitioner partitioner = PartitionUtil.getPartitioner(partitionInfo);
- return createPartitionFilterTree(expressionTree, partitionInfo).applyFilter(partitioner);
- }
-
- /**
- * create partition filter by basing on pushed-down filter
- * @param expressionTree
- * @param partitionInfo
- * @return
- */
- private PartitionFilterIntf createPartitionFilterTree(Expression expressionTree,
- PartitionInfo partitionInfo) {
- ExpressionType filterExpressionType = expressionTree.getFilterExpressionType();
- String partitionColumnName = partitionInfo.getColumnSchemaList().get(0).getColumnName();
- BinaryExpression currentExpression = null;
- ColumnExpression left = null;
- switch (filterExpressionType) {
- case OR:
- currentExpression = (BinaryExpression) expressionTree;
- return new OrFilterImpl(
- createPartitionFilterTree(currentExpression.getLeft(), partitionInfo),
- createPartitionFilterTree(currentExpression.getRight(), partitionInfo));
- case RANGE:
- case AND:
- currentExpression = (BinaryExpression) expressionTree;
- return new AndFilterImpl(
- createPartitionFilterTree(currentExpression.getLeft(), partitionInfo),
- createPartitionFilterTree(currentExpression.getRight(), partitionInfo));
- case EQUALS:
- EqualToExpression equalTo = (EqualToExpression) expressionTree;
- if (equalTo.getLeft() instanceof ColumnExpression &&
- equalTo.getRight() instanceof LiteralExpression) {
- left = (ColumnExpression) equalTo.getLeft();
- if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
- return new EqualToFilterImpl(equalTo, partitionInfo);
- }
- }
- return new KeepAllPartitionFilterImpl();
- case IN:
- InExpression in = (InExpression) expressionTree;
- if (in.getLeft() instanceof ColumnExpression &&
- in.getRight() instanceof ListExpression) {
- left = (ColumnExpression) in.getLeft();
- if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
- return new InFilterImpl(in, partitionInfo);
- }
- }
- return new KeepAllPartitionFilterImpl();
- case FALSE:
- return new PruneAllPartitionFilterImpl();
- case TRUE:
- return new KeepAllPartitionFilterImpl();
- case GREATERTHAN:
- GreaterThanExpression greaterThan = (GreaterThanExpression) expressionTree;
- if (greaterThan.getLeft() instanceof ColumnExpression &&
- greaterThan.getRight() instanceof LiteralExpression) {
- left = (ColumnExpression) greaterThan.getLeft();
- if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
- return new RangeFilterImpl((LiteralExpression) greaterThan.getRight(), true, false,
- partitionInfo);
- }
- }
- return new KeepAllPartitionFilterImpl();
- case GREATERTHAN_EQUALTO:
- GreaterThanEqualToExpression greaterThanEqualTo =
- (GreaterThanEqualToExpression) expressionTree;
- if (greaterThanEqualTo.getLeft() instanceof ColumnExpression &&
- greaterThanEqualTo.getRight() instanceof LiteralExpression) {
- left = (ColumnExpression) greaterThanEqualTo.getLeft();
- if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
- return new RangeFilterImpl((LiteralExpression) greaterThanEqualTo.getRight(), true,
- true, partitionInfo);
- }
- }
- return new KeepAllPartitionFilterImpl();
- case LESSTHAN:
- LessThanExpression lessThan = (LessThanExpression) expressionTree;
- if (lessThan.getLeft() instanceof ColumnExpression &&
- lessThan.getRight() instanceof LiteralExpression) {
- left = (ColumnExpression) lessThan.getLeft();
- if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
- return new RangeFilterImpl((LiteralExpression) lessThan.getRight(), false, false,
- partitionInfo);
- }
- }
- return new KeepAllPartitionFilterImpl();
- case LESSTHAN_EQUALTO:
- LessThanEqualToExpression lessThanEqualTo = (LessThanEqualToExpression) expressionTree;
- if (lessThanEqualTo.getLeft() instanceof ColumnExpression &&
- lessThanEqualTo.getRight() instanceof LiteralExpression) {
- left = (ColumnExpression) lessThanEqualTo.getLeft();
- if (partitionColumnName.equals(left.getCarbonColumn().getColName())) {
- return new RangeFilterImpl((LiteralExpression) lessThanEqualTo.getRight(), false, true,
- partitionInfo);
- }
- }
- return new KeepAllPartitionFilterImpl();
- case NOT_IN:
- case NOT_EQUALS:
- default:
- return new KeepAllPartitionFilterImpl();
- }
- }
-
- /**
- * Selects the blocks based on col max and min value.
- *
- * @param listOfDataBlocksToScan
- * @param dataRefNode
- */
- private void addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter,
- List<DataRefNode> listOfDataBlocksToScan, DataRefNode dataRefNode, boolean[] isMinMaxSet) {
- if (null == dataRefNode.getColumnsMinValue() || null == dataRefNode.getColumnsMaxValue()) {
- listOfDataBlocksToScan.add(dataRefNode);
- return;
- }
- BitSet bitSet = filterExecuter
- .isScanRequired(dataRefNode.getColumnsMaxValue(), dataRefNode.getColumnsMinValue(),
- isMinMaxSet);
- if (!bitSet.isEmpty()) {
- listOfDataBlocksToScan.add(dataRefNode);
- }
- }
-
- /**
* API will return a filter resolver instance which will be used by
* executers to evaluate or execute the filters.
*
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
index 91cc0f8..a6f1dc6 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
@@ -18,10 +18,8 @@
package org.apache.carbondata.core.scan.filter;
import java.io.IOException;
-import java.util.BitSet;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
import org.apache.carbondata.core.scan.expression.Expression;
import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -41,9 +39,4 @@ public interface FilterProcessor {
AbsoluteTableIdentifier tableIdentifier)
throws FilterUnsupportedException, IOException;
- /**
- * This API will get the map of required partitions.
- * @return BitSet the value "1" represent the required partition.
- */
- BitSet getFilteredPartitions(Expression expressionTree, PartitionInfo partitionInfo);
}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/AndFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/AndFilterImpl.java
deleted file mode 100644
index 1d756f4..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/AndFilterImpl.java
+++ /dev/null
@@ -1,44 +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.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the implement of AND logical filter
- */
-public class AndFilterImpl implements PartitionFilterIntf {
-
- protected PartitionFilterIntf left;
- protected PartitionFilterIntf right;
-
- public AndFilterImpl(PartitionFilterIntf left, PartitionFilterIntf right) {
- this.left = left;
- this.right = right;
- }
-
- @Override
- public BitSet applyFilter(Partitioner partitioner) {
- BitSet leftBitSet = left.applyFilter(partitioner);
- BitSet rightBitSet = right.applyFilter(partitioner);
- leftBitSet.and(rightBitSet);
- return leftBitSet;
- }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/EqualToFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/EqualToFilterImpl.java
deleted file mode 100644
index 93657b2..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/EqualToFilterImpl.java
+++ /dev/null
@@ -1,60 +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.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
-import org.apache.carbondata.core.scan.expression.LiteralExpression;
-import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-import org.apache.carbondata.core.util.ByteUtil;
-
-/**
- * the implement of EqualTo filter
- */
-public class EqualToFilterImpl implements PartitionFilterIntf {
-
- private EqualToExpression equalTo;
- private PartitionInfo partitionInfo;
-
- public EqualToFilterImpl(EqualToExpression equalTo, PartitionInfo partitionInfo) {
- this.equalTo = equalTo;
- this.partitionInfo = partitionInfo;
- }
-
- @Override
- public BitSet applyFilter(Partitioner partitioner) {
- BitSet partitionMap = PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
- if (equalTo.isNull) {
- partitionMap.set(partitioner.getPartition(null));
- } else {
- LiteralExpression literal = (LiteralExpression) equalTo.getRight();
- Object value = PartitionUtil.getDataBasedOnDataTypeForFilter(
- literal.getLiteralExpValue().toString(),
- partitionInfo.getColumnSchemaList().get(0).getDataType());
- if (PartitionType.RANGE == partitionInfo.getPartitionType() && value instanceof String) {
- value = ByteUtil.toBytes((String)value);
- }
- partitionMap.set(partitioner.getPartition(value));
- }
- return partitionMap;
- }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/InFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/InFilterImpl.java
deleted file mode 100644
index 1974ee1..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/InFilterImpl.java
+++ /dev/null
@@ -1,61 +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.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
-import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.LiteralExpression;
-import org.apache.carbondata.core.scan.expression.conditional.InExpression;
-import org.apache.carbondata.core.scan.expression.conditional.ListExpression;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-import org.apache.carbondata.core.util.ByteUtil;
-
-/**
- * the implement of In filter
- */
-public class InFilterImpl implements PartitionFilterIntf {
-
- private InExpression in;
- private PartitionInfo partitionInfo;
-
- public InFilterImpl(InExpression in, PartitionInfo partitionInfo) {
- this.in = in;
- this.partitionInfo = partitionInfo;
- }
-
- @Override
- public BitSet applyFilter(Partitioner partitioner) {
- BitSet partitionMap = PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
- ListExpression list = (ListExpression) in.getRight();
- for (Expression expr : list.getChildren()) {
- LiteralExpression literal = (LiteralExpression) expr;
- Object value = PartitionUtil.getDataBasedOnDataTypeForFilter(
- literal.getLiteralExpValue().toString(),
- partitionInfo.getColumnSchemaList().get(0).getDataType());
- if (PartitionType.RANGE == partitionInfo.getPartitionType() && value instanceof String) {
- value = ByteUtil.toBytes((String)value);
- }
- partitionMap.set(partitioner.getPartition(value));
- }
- return partitionMap;
- }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/KeepAllPartitionFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/KeepAllPartitionFilterImpl.java
deleted file mode 100644
index 8ad3be6..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/KeepAllPartitionFilterImpl.java
+++ /dev/null
@@ -1,34 +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.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the implement of partition filter to keep all partitions
- */
-public class KeepAllPartitionFilterImpl implements PartitionFilterIntf {
-
- @Override
- public BitSet applyFilter(Partitioner partitioner) {
- return PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), true);
- }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/OrFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/OrFilterImpl.java
deleted file mode 100644
index 804a92a..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/OrFilterImpl.java
+++ /dev/null
@@ -1,44 +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.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the implement of OR logical filter
- */
-public class OrFilterImpl implements PartitionFilterIntf {
-
- protected PartitionFilterIntf left;
- protected PartitionFilterIntf right;
-
- public OrFilterImpl(PartitionFilterIntf left, PartitionFilterIntf right) {
- this.left = left;
- this.right = right;
- }
-
- @Override
- public BitSet applyFilter(Partitioner partitioner) {
- BitSet leftBitSet = left.applyFilter(partitioner);
- BitSet rightBitSet = right.applyFilter(partitioner);
- leftBitSet.or(rightBitSet);
- return leftBitSet;
- }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterIntf.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterIntf.java
deleted file mode 100644
index fae07b4..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterIntf.java
+++ /dev/null
@@ -1,36 +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.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the interface of partition filter
- */
-public interface PartitionFilterIntf {
-
- /**
- * apply partition filter on Partitioner to get the map of required partitions
- * @param partitioner
- * @return
- */
- BitSet applyFilter(Partitioner partitioner);
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java
deleted file mode 100644
index 9671199..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java
+++ /dev/null
@@ -1,209 +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.core.scan.filter.partition;
-
-import java.text.DateFormat;
-import java.util.BitSet;
-import java.util.List;
-
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.scan.partition.ListPartitioner;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.RangePartitioner;
-import org.apache.carbondata.core.util.ByteUtil;
-import org.apache.carbondata.core.util.comparator.Comparator;
-import org.apache.carbondata.core.util.comparator.SerializableComparator;
-
-public class PartitionFilterUtil {
-
- /**
- * get partition map of range filter on list partition table
- * @param partitionInfo
- * @param partitioner
- * @param filterValue
- * @param isGreaterThan
- * @param isEqualTo
- * @return
- */
- public static BitSet getPartitionMapForRangeFilter(PartitionInfo partitionInfo,
- ListPartitioner partitioner, Object filterValue, boolean isGreaterThan, boolean isEqualTo,
- DateFormat timestampFormatter, DateFormat dateFormatter) {
-
- List<List<String>> listInfo = partitionInfo.getListInfo();
- DataType partitionColumnDataType = partitionInfo.getColumnSchemaList().get(0).getDataType();
-
- SerializableComparator comparator =
- Comparator.getComparator(partitionColumnDataType);
-
- BitSet partitionMap = PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
- // add default partition
- partitionMap.set(0);
- int partitions = listInfo.size();
- if (isGreaterThan) {
- if (isEqualTo) {
- // GreaterThanEqualTo(>=)
- outer1:
- for (int i = 0; i < partitions; i++) {
- for (String value : listInfo.get(i)) {
- Object listValue = PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
- timestampFormatter, dateFormatter);
- if (listValue instanceof String) {
- listValue = ByteUtil.toBytes((String)listValue);
- }
- if (comparator.compare(listValue, filterValue) >= 0) {
- partitionMap.set(i + 1);
- continue outer1;
- }
- }
- }
- } else {
- // GreaterThan(>)
- outer2:
- for (int i = 0; i < partitions; i++) {
- for (String value : listInfo.get(i)) {
- Object listValue = PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
- timestampFormatter, dateFormatter);
- if (listValue instanceof String) {
- listValue = ByteUtil.toBytes((String)listValue);
- }
- if (comparator.compare(listValue, filterValue) > 0) {
- partitionMap.set(i + 1);
- continue outer2;
- }
- }
- }
- }
- } else {
- if (isEqualTo) {
- // LessThanEqualTo(<=)
- outer3:
- for (int i = 0; i < partitions; i++) {
- for (String value : listInfo.get(i)) {
- Object listValue = PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
- timestampFormatter, dateFormatter);
- if (listValue instanceof String) {
- listValue = ByteUtil.toBytes((String)listValue);
- }
- if (comparator.compare(listValue, filterValue) <= 0) {
- partitionMap.set(i + 1);
- continue outer3;
- }
- }
- }
- } else {
- // LessThan(<)
- outer4:
- for (int i = 0; i < partitions; i++) {
- for (String value : listInfo.get(i)) {
- Object listValue = PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
- timestampFormatter, dateFormatter);
- if (listValue instanceof String) {
- listValue = ByteUtil.toBytes((String)listValue);
- }
- if (comparator.compare(listValue, filterValue) < 0) {
- partitionMap.set(i + 1);
- continue outer4;
- }
- }
- }
- }
- }
-
- return partitionMap;
- }
-
- /**
- * get partition map of range filter on range partition table
- * @param partitionInfo
- * @param partitioner
- * @param filterValue
- * @param isGreaterThan
- * @param isEqualTo
- * @return
- */
- public static BitSet getPartitionMapForRangeFilter(PartitionInfo partitionInfo,
- RangePartitioner partitioner, Object filterValue, boolean isGreaterThan, boolean isEqualTo,
- DateFormat timestampFormatter, DateFormat dateFormatter) {
-
- List<String> values = partitionInfo.getRangeInfo();
- DataType partitionColumnDataType = partitionInfo.getColumnSchemaList().get(0).getDataType();
-
- SerializableComparator comparator =
- Comparator.getComparator(partitionColumnDataType);
-
- BitSet partitionMap = PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
-
- int numPartitions = values.size();
- int result = 0;
- // the partition index of filter value
- int partitionIndex = 0;
- // find the partition of filter value
- for (; partitionIndex < numPartitions; partitionIndex++) {
- Object value = PartitionUtil.getDataBasedOnDataType(
- values.get(partitionIndex), partitionColumnDataType, timestampFormatter, dateFormatter);
- if (value instanceof String) {
- value = ByteUtil.toBytes((String)value);
- }
- result = comparator.compare(filterValue, value);
- if (result <= 0) {
- break;
- }
- }
- if (partitionIndex == numPartitions) {
- // filter value is in default partition
- if (isGreaterThan) {
- // GreaterThan(>), GreaterThanEqualTo(>=)
- partitionMap.set(0);
- } else {
- // LessThan(<), LessThanEqualTo(<=)
- partitionMap.set(0, partitioner.numPartitions());
- }
- } else {
- // filter value is not in default partition
- if (result == 0) {
- // if result is 0, the filter value is a bound value of range partition.
- if (isGreaterThan) {
- // GreaterThan(>), GreaterThanEqualTo(>=)
- partitionMap.set(partitionIndex + 2, partitioner.numPartitions());
- partitionMap.set(0);
- } else {
- if (isEqualTo) {
- // LessThanEqualTo(<=)
- partitionMap.set(1, partitionIndex + 3);
- } else {
- // LessThan(<)
- partitionMap.set(1, partitionIndex + 2);
- }
- }
- } else {
- // the filter value is not a bound value of range partition
- if (isGreaterThan) {
- // GreaterThan(>), GreaterThanEqualTo(>=)
- partitionMap.set(partitionIndex + 1, partitioner.numPartitions());
- partitionMap.set(0);
- } else {
- // LessThan(<), LessThanEqualTo(<=)
- partitionMap.set(1, partitionIndex + 2);
- }
- }
- }
- return partitionMap;
- }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PruneAllPartitionFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PruneAllPartitionFilterImpl.java
deleted file mode 100644
index 4b4625e..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PruneAllPartitionFilterImpl.java
+++ /dev/null
@@ -1,34 +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.core.scan.filter.partition;
-
-import java.util.BitSet;
-
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-
-/**
- * the implement of partition filter to prune all partitions
- */
-public class PruneAllPartitionFilterImpl implements PartitionFilterIntf {
-
- @Override
- public BitSet applyFilter(Partitioner partitioner) {
- return PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), false);
- }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/RangeFilterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/RangeFilterImpl.java
deleted file mode 100644
index 6484f1d..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/partition/RangeFilterImpl.java
+++ /dev/null
@@ -1,88 +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.core.scan.filter.partition;
-
-import java.text.SimpleDateFormat;
-import java.util.BitSet;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.scan.expression.LiteralExpression;
-import org.apache.carbondata.core.scan.partition.ListPartitioner;
-import org.apache.carbondata.core.scan.partition.PartitionUtil;
-import org.apache.carbondata.core.scan.partition.Partitioner;
-import org.apache.carbondata.core.scan.partition.RangePartitioner;
-import org.apache.carbondata.core.util.ByteUtil;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-/**
- * the implement of Range filter(include <=, <, >=, >)
- */
-public class RangeFilterImpl implements PartitionFilterIntf {
-
- private LiteralExpression literal;
- private boolean isGreaterThan;
- private boolean isEqualTo;
- private PartitionInfo partitionInfo;
-
- public RangeFilterImpl(LiteralExpression literal, boolean isGreaterThan, boolean isEqualTo,
- PartitionInfo partitionInfo) {
- this.literal = literal;
- this.isGreaterThan = isGreaterThan;
- this.isEqualTo = isEqualTo;
- this.partitionInfo = partitionInfo;
- }
-
- @Override
- public BitSet applyFilter(Partitioner partitioner) {
-
- SimpleDateFormat timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
- CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-
- SimpleDateFormat dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
- CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
-
- switch (partitionInfo.getPartitionType()) {
- case LIST:
- Object filterValueOfList = PartitionUtil.getDataBasedOnDataTypeForFilter(
- literal.getLiteralExpValue().toString(),
- partitionInfo.getColumnSchemaList().get(0).getDataType());
- if (filterValueOfList instanceof String) {
- filterValueOfList = ByteUtil.toBytes((String)filterValueOfList);
- }
- return PartitionFilterUtil.getPartitionMapForRangeFilter(partitionInfo,
- (ListPartitioner) partitioner, filterValueOfList, isGreaterThan, isEqualTo,
- timestampFormatter, dateFormatter);
- case RANGE:
- Object filterValueOfRange = PartitionUtil.getDataBasedOnDataTypeForFilter(
- literal.getLiteralExpValue().toString(),
- partitionInfo.getColumnSchemaList().get(0).getDataType());
- if (filterValueOfRange instanceof String) {
- filterValueOfRange = ByteUtil.toBytes((String)filterValueOfRange);
- }
- return PartitionFilterUtil.getPartitionMapForRangeFilter(partitionInfo,
- (RangePartitioner) partitioner, filterValueOfRange, isGreaterThan, isEqualTo,
- timestampFormatter, dateFormatter);
- default:
- return PartitionUtil.generateBitSetBySize(partitioner.numPartitions(), true);
- }
- }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/HashPartitioner.java b/core/src/main/java/org/apache/carbondata/core/scan/partition/HashPartitioner.java
deleted file mode 100644
index 8562abb..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/HashPartitioner.java
+++ /dev/null
@@ -1,43 +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.core.scan.partition;
-
-/**
- * Hash Partitioner
- */
-public class HashPartitioner implements Partitioner {
-
- private int numPartitions = 0;
-
- public HashPartitioner(int numPartitions) {
- this.numPartitions = numPartitions;
- }
-
- @Override
- public int numPartitions() {
- return numPartitions;
- }
-
- @Override
- public int getPartition(Object key) {
- if (key == null) {
- return 0;
- }
- return (key.hashCode() & Integer.MAX_VALUE) % numPartitions;
- }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/ListPartitioner.java b/core/src/main/java/org/apache/carbondata/core/scan/partition/ListPartitioner.java
deleted file mode 100644
index 8936b0d..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/ListPartitioner.java
+++ /dev/null
@@ -1,79 +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.core.scan.partition;
-
-import java.text.SimpleDateFormat;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.util.CarbonProperties;
-
-/**
- * List Partitioner
- */
-public class ListPartitioner implements Partitioner {
-
- private SimpleDateFormat timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
- CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-
- private SimpleDateFormat dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
- CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
-
- /**
- * Map the value of ListPartition to partition id.
- */
- private Map<Object, Integer> map = new java.util.HashMap<Object, Integer>();
-
- private int numPartitions;
-
- public ListPartitioner(PartitionInfo partitionInfo) {
- List<List<String>> values = partitionInfo.getListInfo();
- DataType partitionColumnDataType = partitionInfo.getColumnSchemaList().get(0).getDataType();
- numPartitions = values.size();
- for (int i = 0; i < numPartitions; i++) {
- for (String value : values.get(i)) {
- map.put(PartitionUtil.getDataBasedOnDataType(value, partitionColumnDataType,
- timestampFormatter, dateFormatter), i + 1);
- }
- }
- }
-
- /**
- * Number of partitions
- * add extra default partition
- * @return
- */
- @Override
- public int numPartitions() {
- return numPartitions + 1;
- }
-
- @Override
- public int getPartition(Object key) {
- Integer partition = map.get(key);
- if (partition == null) {
- return 0;
- }
- return partition;
- }
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
deleted file mode 100644
index 66ceeb8..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/PartitionUtil.java
+++ /dev/null
@@ -1,129 +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.core.scan.partition;
-
-import java.math.BigDecimal;
-import java.text.DateFormat;
-import java.util.BitSet;
-
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-
-import org.apache.commons.lang.StringUtils;
-
-public class PartitionUtil {
-
- public static Partitioner getPartitioner(PartitionInfo partitionInfo) {
- switch (partitionInfo.getPartitionType()) {
- case HASH:
- return new HashPartitioner(partitionInfo.getNumPartitions());
- case LIST:
- return new ListPartitioner(partitionInfo);
- case RANGE:
- return new RangePartitioner(partitionInfo);
- default:
- throw new UnsupportedOperationException(
- "unsupported partition type: " + partitionInfo.getPartitionType().name());
- }
- }
-
- public static Object getDataBasedOnDataType(String data, DataType actualDataType,
- DateFormat timestampFormatter, DateFormat dateFormatter) {
- if (data == null) {
- return null;
- }
- if (actualDataType != DataTypes.STRING && StringUtils.isEmpty(data)) {
- return null;
- }
- try {
- if (actualDataType == DataTypes.STRING) {
- return data;
- } else if (actualDataType == DataTypes.INT) {
- return Integer.parseInt(data);
- } else if (actualDataType == DataTypes.SHORT) {
- return Short.parseShort(data);
- } else if (actualDataType == DataTypes.DOUBLE) {
- return Double.parseDouble(data);
- } else if (actualDataType == DataTypes.LONG) {
- return Long.parseLong(data);
- } else if (actualDataType == DataTypes.DATE) {
- return dateFormatter.parse(data).getTime();
- } else if (actualDataType == DataTypes.TIMESTAMP) {
- return timestampFormatter.parse(data).getTime();
- } else if (DataTypes.isDecimal(actualDataType)) {
- return new BigDecimal(data);
- } else {
- return data;
- }
- } catch (Exception ex) {
- return null;
- }
- }
-
- /**
- * convert the string value of partition filter to the Object
- * @param data
- * @param actualDataType
- * @return
- */
- public static Object getDataBasedOnDataTypeForFilter(String data, DataType actualDataType) {
- if (data == null) {
- return null;
- }
- if (actualDataType != DataTypes.STRING && StringUtils.isEmpty(data)) {
- return null;
- }
- try {
- if (actualDataType == DataTypes.STRING) {
- return data;
- } else if (actualDataType == DataTypes.INT) {
- return Integer.parseInt(data);
- } else if (actualDataType == DataTypes.SHORT) {
- return Short.parseShort(data);
- } else if (actualDataType == DataTypes.DOUBLE) {
- return Double.parseDouble(data);
- } else if (actualDataType == DataTypes.LONG) {
- return Long.parseLong(data);
- } else if (actualDataType == DataTypes.DATE || actualDataType == DataTypes.TIMESTAMP) {
- return Long.parseLong(data) / 1000;
- } else if (DataTypes.isDecimal(actualDataType)) {
- return new BigDecimal(data);
- } else {
- return data;
- }
- } catch (Exception ex) {
- return null;
- }
- }
-
- /**
- * generate a BitSet by size
- * @param size
- * @param initValue true: initialize all bits to true
- * @return
- */
- public static BitSet generateBitSetBySize(int size, boolean initValue) {
- BitSet bitSet = new BitSet(size);
- if (initValue) {
- bitSet.set(0, size);
- }
- return bitSet;
- }
-
-}
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java b/core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java
deleted file mode 100644
index 478196b..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java
+++ /dev/null
@@ -1,91 +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.core.scan.partition;
-
-import java.text.SimpleDateFormat;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
-import org.apache.carbondata.core.util.ByteUtil;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.comparator.Comparator;
-import org.apache.carbondata.core.util.comparator.SerializableComparator;
-
-/**
- * Range Partitioner
- */
-public class RangePartitioner implements Partitioner {
-
- private int numPartitions;
- private Object[] bounds;
- private SerializableComparator comparator;
-
- private SimpleDateFormat timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
- CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-
- private SimpleDateFormat dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance()
- .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
- CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
-
- public RangePartitioner(PartitionInfo partitionInfo) {
- List<String> values = partitionInfo.getRangeInfo();
- DataType partitionColumnDataType = partitionInfo.getColumnSchemaList().get(0).getDataType();
- numPartitions = values.size();
- bounds = new Object[numPartitions];
- if (partitionColumnDataType == DataTypes.STRING) {
- for (int i = 0; i < numPartitions; i++) {
- bounds[i] = ByteUtil.toBytes(values.get(i));
- }
- } else {
- for (int i = 0; i < numPartitions; i++) {
- bounds[i] = PartitionUtil.getDataBasedOnDataType(values.get(i), partitionColumnDataType,
- timestampFormatter, dateFormatter);
- }
- }
- comparator = Comparator.getComparator(partitionColumnDataType);
- }
-
- /**
- * number of partitions
- * add extra default partition
- *
- * @return
- */
- @Override
- public int numPartitions() {
- return numPartitions + 1;
- }
-
- @Override
- public int getPartition(Object key) {
- if (key == null) {
- return 0;
- } else {
- for (int i = 0; i < numPartitions; i++) {
- if (comparator.compare(key, bounds[i]) < 0) {
- return i + 1;
- }
- }
- return 0;
- }
- }
-}
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 00d8b2a..ddd0491 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
@@ -2034,16 +2034,6 @@ public final class CarbonUtil {
column.setDataType(DataTypes.createDecimalType(column.getPrecision(), column.getScale()));
}
}
- if (tableInfo.getFactTable().getPartitionInfo() != null) {
- List<ColumnSchema> partitionColumns =
- tableInfo.getFactTable().getPartitionInfo().getColumnSchemaList();
- for (ColumnSchema column : partitionColumns) {
- DataType dataType = column.getDataType();
- if (DataTypes.isDecimal(dataType)) {
- column.setDataType(DataTypes.createDecimalType(column.getPrecision(), column.getScale()));
- }
- }
- }
}
/**
diff --git a/docs/ddl-of-carbondata.md b/docs/ddl-of-carbondata.md
index 9c9a02f..0660504 100644
--- a/docs/ddl-of-carbondata.md
+++ b/docs/ddl-of-carbondata.md
@@ -58,10 +58,6 @@ CarbonData DDL statements are documented here,which includes:
* [PARTITION](#partition)
* [STANDARD PARTITION(HIVE)](#standard-partition)
* [INSERT OVERWRITE PARTITION](#insert-overwrite)
- * [CARBONDATA PARTITION](#create-hash-partition-table)
- * [HASH PARTITION](#create-hash-partition-table)
- * [RANGE PARTITION](#create-range-partition-table)
- * [LIST PARTITION](#create-list-partition-table)
* [SHOW PARTITIONS](#show-partitions)
* [ADD PARTITION](#add-a-new-partition)
* [SPLIT PARTITION](#split-a-partition)
@@ -960,95 +956,6 @@ Users can specify which columns to include and exclude for local dictionary gene
WHERE au.country = 'US';
```
-### CARBONDATA PARTITION(HASH,RANGE,LIST) -- Alpha feature, this partition feature does not support update and delete data.
-
- The partition supports three type:(Hash,Range,List), similar to other system's partition features, CarbonData's partition feature can be used to improve query performance by filtering on the partition column.
-
-### Create Hash Partition Table
-
- This command allows us to create hash partition.
-
- ```
- CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
- [(col_name data_type , ...)]
- PARTITIONED BY (partition_col_name data_type)
- STORED AS carbondata
- [TBLPROPERTIES ('PARTITION_TYPE'='HASH',
- 'NUM_PARTITIONS'='N' ...)]
- ```
-
- **NOTE:** N is the number of hash partitions
-
-
- Example:
- ```
- CREATE TABLE IF NOT EXISTS hash_partition_table(
- col_A STRING,
- col_B INT,
- col_C LONG,
- col_D DECIMAL(10,2),
- col_F TIMESTAMP
- ) PARTITIONED BY (col_E LONG)
- STORED AS carbondata TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
- ```
-
-### Create Range Partition Table
-
- This command allows us to create range partition.
- ```
- CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
- [(col_name data_type , ...)]
- PARTITIONED BY (partition_col_name data_type)
- STORED AS carbondata
- [TBLPROPERTIES ('PARTITION_TYPE'='RANGE',
- 'RANGE_INFO'='2014-01-01, 2015-01-01, 2016-01-01, ...')]
- ```
-
- **NOTE:**
- * The 'RANGE_INFO' must be defined in ascending order in the table properties.
- * The default format for partition column of Date/Timestamp type is yyyy-MM-dd. Alternate formats for Date/Timestamp could be defined in CarbonProperties.
-
- Example:
- ```
- CREATE TABLE IF NOT EXISTS range_partition_table(
- col_A STRING,
- col_B INT,
- col_C LONG,
- col_D DECIMAL(10,2),
- col_E LONG
- ) partitioned by (col_F Timestamp)
- STORED BY 'carbondata'
- TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- 'RANGE_INFO'='2015-01-01, 2016-01-01, 2017-01-01, 2017-02-01')
- ```
-
-### Create List Partition Table
-
- This command allows us to create list partition.
- ```
- CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
- [(col_name data_type , ...)]
- PARTITIONED BY (partition_col_name data_type)
- STORED AS carbondata
- [TBLPROPERTIES ('PARTITION_TYPE'='LIST',
- 'LIST_INFO'='A, B, C, ...')]
- ```
- **NOTE:** List partition supports list info in one level group.
-
- Example:
- ```
- CREATE TABLE IF NOT EXISTS list_partition_table(
- col_B INT,
- col_C LONG,
- col_D DECIMAL(10,2),
- col_E LONG,
- col_F TIMESTAMP
- ) PARTITIONED BY (col_A STRING)
- STORED AS carbondata
- TBLPROPERTIES('PARTITION_TYPE'='LIST',
- 'LIST_INFO'='aaaa, bbbb, (cccc, dddd), eeee')
- ```
-
### Show Partitions
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
deleted file mode 100644
index 1c4b1bc..0000000
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
+++ /dev/null
@@ -1,215 +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.examples
-
-import java.io.File
-
-import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
-import org.apache.spark.sql.SparkSession
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.examples.util.ExampleUtils
-import org.apache.carbondata.spark.exception.ProcessMetaDataException
-
-
-
-object CarbonPartitionExample {
-
- def main(args: Array[String]) {
- val spark = ExampleUtils.createCarbonSession("CarbonPartitionExample")
- exampleBody(spark)
- spark.close()
- }
-
- def exampleBody(spark : SparkSession): Unit = {
- val rootPath = new File(this.getClass.getResource("/").getPath
- + "../../../..").getCanonicalPath
- val testData = s"$rootPath/integration/spark-common-test/src/test/resources/partition_data.csv"
-
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
- val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
- // range partition with bucket defined
- spark.sql("DROP TABLE IF EXISTS t0")
- spark.sql("""
- | CREATE TABLE IF NOT EXISTS t0
- | (
- | id Int,
- | vin String,
- | phonenumber Long,
- | country String,
- | area String,
- | salary Int
- | )
- | PARTITIONED BY (logdate Timestamp)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01')
- """.stripMargin)
-
- // none partition table
- spark.sql("DROP TABLE IF EXISTS t1")
- spark.sql("""
- | CREATE TABLE IF NOT EXISTS t1
- | (
- | id Int,
- | vin String,
- | logdate Timestamp,
- | phonenumber Long,
- | country String,
- | area String
- | )
- | STORED BY 'carbondata'
- """.stripMargin)
-
- // list partition
- spark.sql("DROP TABLE IF EXISTS t2")
- spark.sql("""
- | CREATE TABLE IF NOT EXISTS t2
- | (
- | id Int,
- | vin String,
- | logdate Timestamp,
- | phonenumber Long,
- | country String,
- | salary Int
- | )
- | PARTITIONED BY (area String)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='Asia, America, Europe', 'DICTIONARY_EXCLUDE' ='area')
- """.stripMargin)
-
- // hash partition
- spark.sql("DROP TABLE IF EXISTS t3")
- spark.sql("""
- | CREATE TABLE IF NOT EXISTS t3
- | (
- | id Int,
- | logdate Timestamp,
- | phonenumber Long,
- | country String,
- | area String,
- | salary Int
- | )
- | PARTITIONED BY (vin String)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='5')
- """.stripMargin)
-
- // list partition
- spark.sql("DROP TABLE IF EXISTS t5")
- spark.sql("""
- | CREATE TABLE IF NOT EXISTS t5
- | (
- | id Int,
- | vin String,
- | logdate Timestamp,
- | phonenumber Long,
- | area String,
- | salary Int
- |)
- | PARTITIONED BY (country String)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, NotGood), Korea ')
- """.stripMargin)
-
- // load data into partition table
- spark.sql(s"""
- LOAD DATA LOCAL INPATH '$testData' into table t0 options('BAD_RECORDS_ACTION'='FORCE')
- """)
- spark.sql(s"""
- LOAD DATA LOCAL INPATH '$testData' into table t5 options('BAD_RECORDS_ACTION'='FORCE')
- """)
-
- // alter list partition table t5 to add a partition
- spark.sql(s"""Alter table t5 add partition ('OutSpace')""".stripMargin)
- // alter list partition table t5 to split partition 4 into 3 independent partition
- spark.sql(
- s"""
- Alter table t5 split partition(4) into ('Canada', 'Russia', '(Good, NotGood)')
- """.stripMargin)
-
- spark.sql("""select * from t5 where country = 'Good' """).show(100, false)
-
- spark.sql("select * from t0 order by salary ").show(100, false)
- spark.sql("select * from t5 order by salary ").show(100, false)
-
- // hive partition table
- spark.sql("DROP TABLE IF EXISTS t7")
- spark.sql("""
- | create table t7(id int, name string) partitioned by (city string)
- | row format delimited fields terminated by ','
- """.stripMargin)
- spark.sql("alter table t7 add partition (city = 'Hangzhou')")
-
- // not default db partition table
- try {
- spark.sql(s"DROP TABLE IF EXISTS partitionDB.t9")
- } catch {
- case ex: NoSuchDatabaseException => LOGGER.error(ex.getMessage())
- }
- spark.sql(s"DROP DATABASE IF EXISTS partitionDB")
- spark.sql(s"CREATE DATABASE partitionDB")
- spark.sql(s"""
- | CREATE TABLE IF NOT EXISTS partitionDB.t9(
- | id Int,
- | logdate Timestamp,
- | phonenumber Int,
- | country String,
- | area String
- | )
- | PARTITIONED BY (vin String)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='5')
- """.stripMargin)
-
- // show tables
- spark.sql("SHOW TABLES").show()
-
- // show partitions
- try {
- spark.sql("""SHOW PARTITIONS t1""").show(100, false)
- } catch {
- case ex: ProcessMetaDataException => LOGGER.error(ex.getMessage())
- }
- spark.sql("""SHOW PARTITIONS t0""").show(100, false)
- spark.sql("""SHOW PARTITIONS t3""").show(100, false)
- spark.sql("""SHOW PARTITIONS t5""").show(100, false)
- spark.sql("""SHOW PARTITIONS t7""").show(100, false)
- spark.sql("""SHOW PARTITIONS partitionDB.t9""").show(100, false)
-
- CarbonProperties.getInstance().addProperty(
- CarbonCommonConstants.CARBON_DATE_FORMAT,
- CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-
- // drop table
- spark.sql("DROP TABLE IF EXISTS t0")
- spark.sql("DROP TABLE IF EXISTS t1")
- spark.sql("DROP TABLE IF EXISTS t2")
- spark.sql("DROP TABLE IF EXISTS t3")
- spark.sql("DROP TABLE IF EXISTS t5")
- spark.sql("DROP TABLE IF EXISTS t7")
- spark.sql("DROP TABLE IF EXISTS partitionDB.t9")
- spark.sql(s"DROP DATABASE IF EXISTS partitionDB")
- }
-}
diff --git a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
index 853f778..cef2b97 100644
--- a/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
+++ b/examples/spark2/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
@@ -62,10 +62,6 @@ class RunExamples extends QueryTest with BeforeAndAfterAll {
CarbonDataFrameExample.exampleBody(spark)
}
- test("CarbonPartitionExample") {
- CarbonPartitionExample.exampleBody(spark)
- }
-
test("CarbonSessionExample") {
CarbonSessionExample.exampleBody(spark)
}
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 5daf767..1180991 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -64,6 +64,7 @@ enum Encoding{
DIRECT_COMPRESS_VARCHAR = 14; // Identifies that a columm is encoded using DirectCompressCodec, it is used for long string columns
}
+// Only NATIVE_HIVE is supported, others are deprecated since CarbonData 2.0
enum PartitionType{
RANGE = 0;
RANGE_INTERVAL = 1;
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 0fecb75..52796fb 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
@@ -20,7 +20,6 @@ package org.apache.carbondata.hadoop.api;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
-import java.util.BitSet;
import java.util.Collections;
import java.util.Comparator;
import java.util.LinkedList;
@@ -36,7 +35,6 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
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.SchemaReader;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
import org.apache.carbondata.core.metadata.schema.table.TableInfo;
@@ -131,7 +129,6 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
}
}
// this will be null in case of corrupt schema file.
- PartitionInfo partitionInfo = carbonTable.getPartitionInfo();
DataMapFilter filter = getFilterPredicates(job.getConfiguration());
// if external table Segments are found, add it to the List
@@ -171,7 +168,7 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
}
if (useBlockDataMap) {
// do block filtering and get split
- splits = getSplits(job, filter, externalTableSegments, null, partitionInfo, null);
+ splits = getSplits(job, filter, externalTableSegments);
} else {
List<CarbonFile> carbonFiles = null;
if (null != this.fileLists) {
@@ -252,17 +249,18 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
* @return
* @throws IOException
*/
- private List<InputSplit> getSplits(JobContext job, DataMapFilter dataMapFilter,
- List<Segment> validSegments, BitSet matchedPartitions, PartitionInfo partitionInfo,
- List<Integer> oldPartitionIdList) throws IOException {
+ private List<InputSplit> getSplits(
+ JobContext job,
+ DataMapFilter dataMapFilter,
+ List<Segment> validSegments) throws IOException {
numSegments = validSegments.size();
List<InputSplit> result = new LinkedList<InputSplit>();
// for each segment fetch blocks matching filter in Driver BTree
List<CarbonInputSplit> dataBlocksOfSegment =
- getDataBlocksOfSegment(job, carbonTable, dataMapFilter, matchedPartitions, validSegments,
- partitionInfo, oldPartitionIdList, new ArrayList<Segment>(), new ArrayList<String>());
+ getDataBlocksOfSegment(job, carbonTable, dataMapFilter, validSegments,
+ new ArrayList<Segment>(), new ArrayList<String>());
numBlocks = dataBlocksOfSegment.size();
result.addAll(dataBlocksOfSegment);
return result;
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
index 1382368..5f7814a 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java
@@ -22,7 +22,6 @@ import java.io.DataInputStream;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.util.ArrayList;
-import java.util.BitSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -48,9 +47,7 @@ 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.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.PartitionInfo;
import org.apache.carbondata.core.metadata.schema.SchemaReader;
-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.metadata.schema.table.column.ColumnSchema;
@@ -494,8 +491,7 @@ m filterExpression
* get data blocks of given segment
*/
protected List<CarbonInputSplit> getDataBlocksOfSegment(JobContext job, CarbonTable carbonTable,
- DataMapFilter expression, BitSet matchedPartitions, List<Segment> segmentIds,
- PartitionInfo partitionInfo, List<Integer> oldPartitionIdList,
+ DataMapFilter expression, List<Segment> segmentIds,
List<Segment> invalidSegments, List<String> segmentsToBeRefreshed)
throws IOException {
@@ -509,35 +505,13 @@ m filterExpression
getPrunedBlocklets(job, carbonTable, expression, segmentIds, invalidSegments,
segmentsToBeRefreshed);
List<CarbonInputSplit> resultFilteredBlocks = 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) {
-
- // 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) {
- long partitionId = Long.parseLong(CarbonTablePath.DataFileUtil
- .getTaskIdFromTaskNo(CarbonTablePath.DataFileUtil.getTaskNo(blocklet.getPath())));
- 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)) {
- resultFilteredBlocks.add(blocklet.getInputSplit());
- }
- }
+ // 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.
+ resultFilteredBlocks.add(blocklet.getInputSplit());
}
statistic
.addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER, System.currentTimeMillis());
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 deae842..6a0f946 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
@@ -20,7 +20,6 @@ package org.apache.carbondata.hadoop.api;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.BitSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
@@ -37,8 +36,6 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
import org.apache.carbondata.core.indexstore.PartitionSpec;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-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.mutate.CarbonUpdateUtil;
import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
@@ -48,7 +45,6 @@ import org.apache.carbondata.core.profiler.ExplainCollector;
import org.apache.carbondata.core.readcommitter.LatestFilesReadCommittedScope;
import org.apache.carbondata.core.readcommitter.ReadCommittedScope;
import org.apache.carbondata.core.readcommitter.TableStatusReadCommittedScope;
-import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
import org.apache.carbondata.core.statusmanager.FileFormat;
import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
@@ -83,11 +79,8 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
"mapreduce.input.carboninputformat.segmentnumbers";
// 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 Logger LOG =
LogServiceFactory.getLogService(CarbonTableInputFormat.class.getName());
- 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 CARBON_TRANSACTIONAL_TABLE =
"mapreduce.input.carboninputformat.transactional";
public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
@@ -133,12 +126,12 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
if (!accessStreamingSegments) {
List<Segment> validSegments = segments.getValidSegments();
streamSegments = segments.getStreamSegments();
- streamSegments = getFilteredSegment(job, streamSegments, true, readCommittedScope);
+ streamSegments = getFilteredSegment(job, streamSegments, readCommittedScope);
if (validSegments.size() == 0) {
return getSplitsOfStreaming(job, streamSegments, carbonTable);
}
List<Segment> filteredSegmentToAccess =
- getFilteredSegment(job, segments.getValidSegments(), true, readCommittedScope);
+ getFilteredSegment(job, segments.getValidSegments(), readCommittedScope);
if (filteredSegmentToAccess.size() == 0) {
return getSplitsOfStreaming(job, streamSegments, carbonTable);
} else {
@@ -146,7 +139,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
}
} else {
List<Segment> filteredNormalSegments =
- getFilteredNormalSegments(job, segments.getValidSegments(),
+ getFilteredNormalSegments(segments.getValidSegments(),
getSegmentsToAccess(job, readCommittedScope));
streamSegments = segments.getStreamSegments();
if (filteredNormalSegments.size() == 0) {
@@ -170,34 +163,20 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
validAndInProgressSegments.addAll(segments.getListOfInProgressSegments());
// get updated filtered list
List<Segment> filteredSegmentToAccess =
- getFilteredSegment(job, new ArrayList<>(validAndInProgressSegments), false,
- readCommittedScope);
+ getFilteredSegment(job, new ArrayList<>(validAndInProgressSegments), readCommittedScope);
// process and resolve the expression
DataMapFilter dataMapFilter = getFilterPredicates(job.getConfiguration());
- // this will be null in case of corrupt schema file.
- PartitionInfo partitionInfo = carbonTable.getPartitionInfo();
if (dataMapFilter != null) {
dataMapFilter.resolve(false);
}
- // prune partitions for filter query on partition table
- BitSet matchedPartitions = null;
- if (partitionInfo != null && partitionInfo.getPartitionType() != PartitionType.NATIVE_HIVE) {
- matchedPartitions = setMatchedPartitions(null, dataMapFilter, partitionInfo, null);
- if (matchedPartitions != null) {
- if (matchedPartitions.cardinality() == 0) {
- return new ArrayList<InputSplit>();
- } else if (matchedPartitions.cardinality() == partitionInfo.getNumPartitions()) {
- matchedPartitions = null;
- }
- }
- }
// do block filtering and get split
- List<InputSplit> splits =
- getSplits(job, dataMapFilter, filteredSegmentToAccess, matchedPartitions, partitionInfo,
- null, updateStatusManager, segments.getInvalidSegments());
+ List<InputSplit> splits = getSplits(
+ job, dataMapFilter, filteredSegmentToAccess,
+ updateStatusManager, segments.getInvalidSegments());
+
// add all splits of streaming
List<InputSplit> splitsOfStreaming = getSplitsOfStreaming(job, streamSegments, carbonTable);
if (!splitsOfStreaming.isEmpty()) {
@@ -222,12 +201,11 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
* For Pre Aggregate rules it will set all the valid segments for both streaming and
* and normal for fact table, so if any handoff happened in between it will
* select only new hand off segments segments for fact.
- * @param job
* @param validSegments
* @param segmentsToAccess
* @return
*/
- private List<Segment> getFilteredNormalSegments(JobContext job, List<Segment> validSegments,
+ private List<Segment> getFilteredNormalSegments(List<Segment> validSegments,
Segment[] segmentsToAccess) {
List<Segment> segmentToAccessSet = Arrays.asList(segmentsToAccess);
List<Segment> filteredSegment = new ArrayList<>();
@@ -244,7 +222,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
* `INPUT_SEGMENT_NUMBERS` in job configuration
*/
private List<Segment> getFilteredSegment(JobContext job, List<Segment> validSegments,
- boolean validationRequired, ReadCommittedScope readCommittedScope) {
+ ReadCommittedScope readCommittedScope) {
Segment[] segmentsToAccess = getSegmentsToAccess(job, readCommittedScope);
List<Segment> segmentToAccessSet =
new ArrayList<>(new HashSet<>(Arrays.asList(segmentsToAccess)));
@@ -343,91 +321,6 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
}
/**
- * Read data in one segment. For alter table partition statement
- * @param job
- * @param targetSegment
- * @param oldPartitionIdList get old partitionId before partitionInfo was changed
- * @return
- */
- public List<InputSplit> getSplitsOfOneSegment(JobContext job, String targetSegment,
- List<Integer> oldPartitionIdList, PartitionInfo partitionInfo) {
- try {
- carbonTable = getOrCreateCarbonTable(job.getConfiguration());
- ReadCommittedScope readCommittedScope =
- getReadCommitted(job, carbonTable.getAbsoluteTableIdentifier());
- this.readCommittedScope = readCommittedScope;
-
- List<Segment> segmentList = new ArrayList<>();
- Segment segment = Segment.getSegment(targetSegment, carbonTable.getTablePath());
- segmentList.add(
- new Segment(segment.getSegmentNo(), segment.getSegmentFileName(), readCommittedScope));
- setSegmentsToAccess(job.getConfiguration(), segmentList);
-
- // process and resolve the expression
- DataMapFilter filter = getFilterPredicates(job.getConfiguration());
- CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
- // this will be null in case of corrupt schema file.
- if (null == carbonTable) {
- throw new IOException("Missing/Corrupt schema file for table.");
- }
- if (filter != null) {
- filter.processFilterExpression();
- }
- // prune partitions for filter query on partition table
- String partitionIds = job.getConfiguration().get(ALTER_PARTITION_ID);
- // matchedPartitions records partitionIndex, not partitionId
- BitSet matchedPartitions = null;
- if (partitionInfo != null) {
- matchedPartitions =
- setMatchedPartitions(partitionIds, filter, partitionInfo, oldPartitionIdList);
- if (matchedPartitions != null) {
- if (matchedPartitions.cardinality() == 0) {
- return new ArrayList<InputSplit>();
- } else if (matchedPartitions.cardinality() == partitionInfo.getNumPartitions()) {
- matchedPartitions = null;
- }
- }
- }
-
- // do block filtering and get split
- List<InputSplit> splits = getSplits(job, filter, segmentList, matchedPartitions,
- partitionInfo, oldPartitionIdList, new SegmentUpdateStatusManager(carbonTable),
- new ArrayList<Segment>());
- return splits;
- } catch (IOException e) {
- throw new RuntimeException("Can't get splits of the target segment ", e);
- }
- }
-
- /**
- * set the matched partition indices into a BitSet
- * @param partitionIds from alter table command, for normal query, it's null
- * @param filter from query
- * @param partitionInfo
- * @param oldPartitionIdList only used in alter table command
- * @return
- */
- private BitSet setMatchedPartitions(String partitionIds, DataMapFilter filter,
- PartitionInfo partitionInfo, List<Integer> oldPartitionIdList) {
- BitSet matchedPartitions = null;
- if (null != partitionIds) {
- String[] partList = partitionIds.replace("[", "").replace("]", "").split(",");
- // partList[0] -> use the first element to initiate BitSet, will auto expand later
- matchedPartitions = new BitSet(Integer.parseInt(partList[0].trim()));
- for (String partitionId : partList) {
- Integer index = oldPartitionIdList.indexOf(Integer.parseInt(partitionId.trim()));
- matchedPartitions.set(index);
- }
- } else {
- if (null != filter) {
- matchedPartitions = new FilterExpressionProcessor()
- .getFilteredPartitions(filter.getExpression(), partitionInfo);
- }
- }
- return matchedPartitions;
- }
-
- /**
* {@inheritDoc}
* Configurations FileInputFormat.INPUT_DIR, CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS
* are used to get table path to read.
@@ -436,8 +329,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
* @throws IOException
*/
private List<InputSplit> getSplits(JobContext job, DataMapFilter expression,
- List<Segment> validSegments, BitSet matchedPartitions, PartitionInfo partitionInfo,
- List<Integer> oldPartitionIdList, SegmentUpdateStatusManager updateStatusManager,
+ List<Segment> validSegments, SegmentUpdateStatusManager updateStatusManager,
List<Segment> invalidSegments) throws IOException {
List<String> segmentsToBeRefreshed = new ArrayList<>();
@@ -455,14 +347,14 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
numSegments = validSegments.size();
List<InputSplit> result = new LinkedList<InputSplit>();
UpdateVO invalidBlockVOForSegmentId = null;
- Boolean isIUDTable = false;
+ boolean isIUDTable = false;
isIUDTable = (updateStatusManager.getUpdateStatusDetails().length != 0);
// for each segment fetch blocks matching filter in Driver BTree
List<org.apache.carbondata.hadoop.CarbonInputSplit> dataBlocksOfSegment =
- getDataBlocksOfSegment(job, carbonTable, expression, matchedPartitions, validSegments,
- partitionInfo, oldPartitionIdList, invalidSegments, segmentsToBeRefreshed);
+ getDataBlocksOfSegment(job, carbonTable, expression, validSegments,
+ invalidSegments, segmentsToBeRefreshed);
numBlocks = dataBlocksOfSegment.size();
for (org.apache.carbondata.hadoop.CarbonInputSplit inputSplit : dataBlocksOfSegment) {
@@ -535,7 +427,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
// TODO: currently only batch segment is supported, add support for streaming table
List<Segment> filteredSegment =
- getFilteredSegment(job, allSegments.getValidSegments(), false, readCommittedScope);
+ getFilteredSegment(job, allSegments.getValidSegments(), readCommittedScope);
boolean isIUDTable = (updateStatusManager.getUpdateStatusDetails().length != 0);
/* In the select * flow, getSplits() method was clearing the segmentMap if,
segment needs refreshing. same thing need for select count(*) flow also.
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
index f4c6c22..c63aa68 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/DictionaryDecodeReadSupport.java
@@ -40,10 +40,6 @@ public class DictionaryDecodeReadSupport<T> implements CarbonReadSupport<T> {
protected Dictionary[] dictionaries;
protected DataType[] dataTypes;
- /**
- * carbon columns
- */
- protected CarbonColumn[] carbonColumns;
/**
* This initialization is done inside executor task
@@ -55,7 +51,6 @@ public class DictionaryDecodeReadSupport<T> implements CarbonReadSupport<T> {
@Override
public void initialize(CarbonColumn[] carbonColumns,
CarbonTable carbonTable) throws IOException {
- this.carbonColumns = carbonColumns;
dictionaries = new Dictionary[carbonColumns.length];
dataTypes = new DataType[carbonColumns.length];
for (int i = 0; i < carbonColumns.length; i++) {
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BatchSortLoad3TestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BatchSortLoad3TestCase.scala
index ed803ca..14045a6 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BatchSortLoad3TestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BatchSortLoad3TestCase.scala
@@ -177,15 +177,6 @@ class BatchSortLoad3TestCase extends QueryTest with BeforeAndAfterAll {
sql(s"""drop table if exists uniqdata_c""").collect
}
-
- //Batch_sort_Loading_001-01-01-01_001-TC_064
-// test("Batch_sort_Loading_001-01-01-01_001-TC_064", Include) {
-// sql(s"""drop table if exists uniqdata""").collect
-// 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 (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='20160302,20150302')""").collect
-// sql(s"""LOAD DATA INPATH '$resourcesPath/Data/uniqdata/7000_UniqData.csv' into table uniqdata OPTIONS('DELIMITER'=',','QUOTECHAR'='"','COMMENTCHAR'='#','MULTILINE'='true','ESCAPECHAR'='\','BAD_RECORDS_ACTION'='REDIRECT','BAD_RECORDS_LOGGER_ENABLE'='TRUE','FILEHEADER'='CUST_ID,CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')""").collect
-// sql(s"""drop table if exists uniqdata""").collect
-// }
-
override def afterAll {
}
}
\ No newline at end of file
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PartitionTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PartitionTestCase.scala
deleted file mode 100644
index 31ec14e..0000000
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/PartitionTestCase.scala
+++ /dev/null
@@ -1,520 +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.cluster.sdv.generated
-
-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 PartitionTestCase extends QueryTest with BeforeAndAfterAll {
-
-
- //Verify exception if column in partitioned by is already specified in table schema
- test("Partition-Local-sort_TC001", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='List','LIST_INFO'='1,3')""").collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify table is created with Partition
- ignore("Partition-Local-sort_TC002", Include) {
- sql(s"""drop table if exists uniqdata""").collect
- 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 (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST','LIST_INFO'='3')""").collect
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception partitioned by is not specified in the DDL
- test("Partition-Local-sort_TC003", Include) {
- sql(s"""drop table if exists uniqdata""").collect
- 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) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='List','NUM_PARTITIONS'='3')""").collect
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception if List info is not given with List type partition
- test("Partition-Local-sort_TC004", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='List')""").collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //exception should not be thrown if Partition type is not given
- test("Partition-Local-sort_TC005", Include) {
- try {
- sql(s"""drop table if exists uniqdata""").collect
- 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 (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('LIST_INFO'='1,2')""").collect
- sql(s"""drop table if exists uniqdata""").collect
- } catch {
- case _ => assert(false)
- }
- }
-
-
- //Verify exception if Partition type is 'range' and LIST_INFO Is provided
- test("Partition-Local-sort_TC006", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'LIST_INFO'='1,2')""").collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception if Partition type is 'range' and NUM_PARTITIONS Is provided
- test("Partition-Local-sort_TC007", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'NUM_PARTITIONS'='1')""").collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify table is created if Partition type is 'range' and RANGE_INFO Is provided
- ignore("Partition-Local-sort_TC008", Include) {
- sql(s"""drop table if exists uniqdata""").collect
- 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 (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='20160302,20150302')""").collect
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify table is created if Partition type is 'LIST' and LIST_INFO Is provided
- test("Partition-Local-sort_TC009", Include) {
- sql(s"""drop table if exists uniqdata""").collect
- 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 int)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,2')""").collect
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception if Partition type is 'LIST' and NUM_PARTITIONS Is provided
- test("Partition-Local-sort_TC010", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 (DOJ int)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'NUM_PARTITIONS'='1')""").collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception if Partition type is 'LIST' and RANGE_INFO Is provided
- test("Partition-Local-sort_TC011", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'RANGE_INFO'='20160302,20150302')""").collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception if datatype is not provided with partition column
- test("Partition-Local-sort_TC012", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 (DOJ)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='20160302,20150302')""").collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception if a non existent file header is provided in partition
- test("Partition-Local-sort_TC013", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 (DOJ timestamp)STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='20160302,20150302')
-
- LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOJ,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception if Partition By Is empty
- test("Partition-Local-sort_TC014", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 ()STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')
- """).collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify load with List Partition
- test("Partition-Local-sort_TC015", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata""",
- Seq(Row(28)), "partitionTestCase_Partition-Local-sort_TC015")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify load with List Partition and limit 1
- ignore("Partition-Local-sort_TC016", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select * from uniqdata limit 1""",
- Seq(Row("CUST_NAME_00002","ACTIVE_EMUI_VERSION_00002",null,null,null,12345678903.0000000000,22345678903.0000000000,1.123456749E10,-1.123456749E10,3,null,2)), "partitionTestCase_Partition-Local-sort_TC016")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify load with List Partition and select partition column
- test("Partition-Local-sort_TC017", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , 'QUOTECHAR'='"','BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_NAME,ACTIVE_EMUI_VERSION,DOB,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1,DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1,DOJ,CUST_ID')""").collect
- checkAnswer(s"""select CUST_ID from uniqdata order by CUST_ID limit 1""",
- Seq(Row(0)), "partitionTestCase_Partition-Local-sort_TC017")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception if 2 partition columns are provided
- test("Partition-Local-sort_TC018", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify data load with range partition with limit 1
- ignore("Partition-Local-sort_TC019", Include) {
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='0,5,10,29')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select * from uniqdata limit 1""",
- Seq(Row("CUST_NAME_00003","ACTIVE_EMUI_VERSION_00003",null,null,null,12345678904.0000000000,22345678904.0000000000,1.123456749E10,-1.123456749E10,4,null,5)), "partitionTestCase_Partition-Local-sort_TC019")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify data load with range partition
- ignore("Partition-Local-sort_TC020", Include) {
- dropTable("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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='0,5,10,29')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- sql(s"""select count(*) from uniqdata limit 1""").collect
- sql(s"""Seq(Row(28))""").collect
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify data load with hash partition with limit 1
- ignore("Partition-Local-sort_TC021", Include) {
- sql(s"""drop table if exists uniqdata""").collect
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='5')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , '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')""").collect
- checkAnswer(s"""select * from uniqdata limit 1""",
- Seq(Row("CUST_NAME_00003","ACTIVE_EMUI_VERSION_00003",null,null,null,12345678904.0000000000,22345678904.0000000000,1.123456749E10,-1.123456749E10,4,null,5)), "partitionTestCase_Partition-Local-sort_TC021")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify data load with hash partition
- ignore("Partition-Local-sort_TC022", Include) {
- dropTable("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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='5')""").collect
-
- sql(s"""LOAD DATA INPATH ''$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata OPTIONS('DELIMITER'=',' , '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')""").collect
- checkAnswer(s"""select count(*) from uniqdata limit 1""",
- Seq(Row(28)), "partitionTestCase_Partition-Local-sort_TC022")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify data load with List partition after compaction
- test("Partition-Local-sort_TC023", Include) {
- dropTable("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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""alter table uniqdata compact 'minor'""").collect
- checkAnswer(s"""select count(*) from uniqdata limit 1""",
- Seq(Row(112)), "partitionTestCase_Partition-Local-sort_TC023")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify data load with Range partition after compaction
- test("Partition-Local-sort_TC024", Include) {
- dropTable("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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='Range', 'RANGE_INFO'='0,5,10,30')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""alter table uniqdata compact 'minor'""").collect
- checkAnswer(s"""select count(*) from uniqdata limit 1""",
- Seq(Row(112)), "partitionTestCase_Partition-Local-sort_TC024")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify data load with Hash partition after compaction
- test("Partition-Local-sort_TC025", Include) {
- dropTable("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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='5')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- sql(s"""alter table uniqdata compact 'minor'""").collect
- checkAnswer(s"""select count(*) from uniqdata limit 1""",
- Seq(Row(112)), "partitionTestCase_Partition-Local-sort_TC025")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify join operation on List partition
- test("Partition-Local-sort_TC026", Include) {
- sql(s"""drop table if exists uniqdata1""").collect
- sql(s"""drop table if exists uniqdata""").collect
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata1 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')""").collect
- sql(s"""select a.cust_id, b.cust_id from uniqdata a, uniqdata1 b where a.cust_id > b.cust_id""").collect
-
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify data when sublist is provided in LIST_INFO
- test("Partition-Local-sort_TC028", Include) {
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,(1,2),3')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata""",
- Seq(Row(28)), "partitionTestCase_Partition-Local-sort_TC028")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify exception is thrown if partition column is dropped
- test("Partition-Local-sort_TC029", Include) {
- intercept[Exception] {
- sql(s"""drop table if exists uniqdata""").collect
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')
-
- alter table uniqdata drop columns(CUST_ID)
-
- """).collect
- }
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify insert is successful on list partition
- ignore("Partition-Local-sort_TC030", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='0,1')""").collect
- 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)""").collect
- 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)""").collect
- checkAnswer(s"""select * from uniqdata""",
- Seq(Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",1),Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",0)), "partitionTestCase_Partition-Local-sort_TC030")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify insert is successful on range partition
- ignore("Partition-Local-sort_TC031", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='0,3,5')""").collect
- 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)""").collect
- 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)""").collect
- checkAnswer(s"""select * from uniqdata""",
- Seq(Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",1),Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",0)), "partitionTestCase_Partition-Local-sort_TC031")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify insert is successful on HASH partition
- ignore("Partition-Local-sort_TC032", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='10')""").collect
- 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)""").collect
- 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)""").collect
- sql(s"""select * from uniqdata""").collect
- sql(s"""Seq(Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",1),Row("a",1,"2015-07-01 00:00:00.0",5678,7654,23.4000000000,55.6000000000,7654.0,8765.0,33,"2015-07-01 00:00:00.0",0))""").collect
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify date with > filter condition and list partition
- test("Partition-Local-sort_TC033", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,0,3,4')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata where CUST_ID>3""",
- Seq(Row(4)), "partitionTestCase_Partition-Local-sort_TC033")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify date with = filter condition and list partition
- test("Partition-Local-sort_TC034", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,0,3,4')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata where CUST_ID=3""",
- Seq(Row(8)), "partitionTestCase_Partition-Local-sort_TC034")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify date with = value not in list_info and list partition
- test("Partition-Local-sort_TC035", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,0,3,4')""").collect
-
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata where CUST_ID=10""",
- Seq(Row(0)), "partitionTestCase_Partition-Local-sort_TC035")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify date with > filter condition and range partition
- ignore("Partition-Local-sort_TC036", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1,0,3,4')""").collect
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata where CUST_ID>3""",
- Seq(Row(4)), "partitionTestCase_Partition-Local-sort_TC036")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify date with = filter condition and list partition
- ignore("Partition-Local-sort_TC037", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1,0,3,4')""").collect
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata where CUST_ID=3""",
- Seq(Row(8)), "partitionTestCase_Partition-Local-sort_TC037")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify date with = value not in list_info and list partition
- ignore("Partition-Local-sort_TC038", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1,0,3,4')""").collect
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata where CUST_ID=10""",
- Seq(Row(0)), "partitionTestCase_Partition-Local-sort_TC038")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify date with > filter condition and hash partition
- ignore("Partition-Local-sort_TC039", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='1,0,3,4')""").collect
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata where CUST_ID>3""",
- Seq(Row(4)), "partitionTestCase_Partition-Local-sort_TC039")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify date with = filter condition and hash partition
- ignore("Partition-Local-sort_TC040", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='1,0,3,4')""").collect
- sql(s"""LOAD DATA INPATH '$resourcesPath/Data/partition/2000_UniqData_partition.csv' into table uniqdata 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')""").collect
- checkAnswer(s"""select count(*) from uniqdata where CUST_ID=3""",
- Seq(Row(8)), "partitionTestCase_Partition-Local-sort_TC040")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
-
- //Verify date with = value not in list_info and hash partition
- ignore("Partition-Local-sort_TC041", Include) {
- sql(s"""drop table if exists uniqdata""").collect
-
- 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 'org.apache.carbondata.format' TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='1,0,3,4')""").collect
- checkAnswer(s"""select count(*) from uniqdata where CUST_ID=10""",
- Seq(Row(0)), "partitionTestCase_Partition-Local-sort_TC041")
- sql(s"""drop table if exists uniqdata""").collect
- }
-
- override def afterAll {
- sql("drop table if exists uniqdata")
- }
-}
\ No newline at end of file
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
index b11a57c..812aab5 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
@@ -152,7 +152,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
""".stripMargin)
}
assert(exception.getMessage.contains(
- "LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column: binaryfield is not a string/complex/varchar datatype column. " +
+ "LOCAL_DICTIONARY_INCLUDE/LOCAL_DICTIONARY_EXCLUDE column: binaryField is not a string/complex/varchar datatype column. " +
"LOCAL_DICTIONARY_COLUMN should be no dictionary string/complex/varchar datatype column"))
}
@@ -190,7 +190,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
""".stripMargin)
}
assert(exception.getMessage.contains(
- "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryfield"))
+ "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryField"))
}
test("Unsupport DICTIONARY_INCLUDE for binary, multiple column") {
@@ -210,7 +210,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
""".stripMargin)
}
assert(exception.getMessage.contains(
- "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryfield"))
+ "DICTIONARY_INCLUDE is unsupported for binary data type column: binaryField"))
}
test("Supports DICTIONARY_EXCLUDE for binary") {
@@ -262,7 +262,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
| tblproperties('inverted_index'='binaryField','SORT_COLUMNS'='binaryField')
""".stripMargin)
}
- assert(exception.getMessage.contains("sort_columns is unsupported for binary datatype column: binaryfield"))
+ assert(exception.getMessage.contains("sort_columns is unsupported for binary datatype column: binaryField"))
}
test("COLUMN_META_CACHE doesn't support binary") {
@@ -1583,61 +1583,6 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
Seq(Row("binary"), Row("1"), Row("binary"), Row("1"), Row("binary"), Row("1")))
}
- test("Create table and load data with binary column for partition") {
- sql("DROP TABLE IF EXISTS binaryTable")
- sql(
- s"""
- | CREATE TABLE IF NOT EXISTS binaryTable (
- | id int,
- | label boolean,
- | name string,
- | autoLabel boolean)
- | PARTITIONED BY(binaryfield binary)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('SORT_COLUMNS'='','PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
- """.stripMargin)
- sql(
- s"""
- | LOAD DATA LOCAL INPATH '$resourcesPath/binarystringdatawithHead.csv'
- | INTO TABLE binaryTable
- | partition(binaryfield)
- | OPTIONS('header'='true','DELIMITER'='|')
- """.stripMargin)
-
- val result = sql("desc formatted binaryTable").collect()
- var flag = false
- result.foreach { each =>
- if ("binary".equals(each.get(1))) {
- flag = true
- }
- }
- assert(flag)
-
- checkAnswer(sql("SELECT COUNT(*) FROM binaryTable"), Seq(Row(3)))
- try {
- val df = sql("SELECT * FROM binaryTable").collect()
- assert(3 == df.length)
-
- df.foreach { each =>
- assert(5 == each.length)
- if (2 == each.get(0)) {
- assert("binary".equals(new String(each.getAs[Array[Byte]](4))))
- } else if (1 == each.get(0)) {
- assert("Hello world".equals(new String(each.getAs[Array[Byte]](4))))
- } else if (3 == each.get(0)) {
- assert("1".equals(new String(each.getAs[Array[Byte]](4))))
- } else {
- assert(false)
- }
- }
-
- } catch {
- case e: Exception =>
- e.printStackTrace()
- assert(false)
- }
- }
-
test("Select query with average function for substring of binary column is executed.") {
sql("DROP TABLE IF EXISTS uniqdata")
sql(
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala
index b12e269..9fbc061 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithCompression.scala
@@ -546,7 +546,7 @@ class TestLoadDataWithCompression extends QueryTest with BeforeAndAfterEach with
checkAnswer(sql(s"SELECT count(*) FROM $tableName"), Seq(Row(8)))
val carbonTable = CarbonEnv.getCarbonTable(Option("default"), tableName)(sqlContext.sparkSession)
val tableColumnCompressor = carbonTable.getTableInfo.getFactTable.getTableProperties.get(CarbonCommonConstants.COMPRESSOR)
- assertResult(compressorName.toLowerCase())(tableColumnCompressor)
+ assertResult(compressorName)(tableColumnCompressor)
sql(s"DROP TABLE IF EXISTS $tableName")
// reuse the customize compressor again
@@ -555,7 +555,7 @@ class TestLoadDataWithCompression extends QueryTest with BeforeAndAfterEach with
checkAnswer(sql(s"SELECT count(*) FROM $tableName"), Seq(Row(8)))
val carbonTable2 = CarbonEnv.getCarbonTable(Option("default"), tableName)(sqlContext.sparkSession)
val tableColumnCompressor2 = carbonTable2.getTableInfo.getFactTable.getTableProperties.get(CarbonCommonConstants.COMPRESSOR)
- assertResult(compressorName.toLowerCase())(tableColumnCompressor2)
+ assertResult(compressorName)(tableColumnCompressor2)
}
private def generateAllDataTypeFiles(lineNum: Int, csvDir: String,
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala
index 473c9c6..5e10a74 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateDDLForComplexMapType.scala
@@ -461,7 +461,7 @@ class TestCreateDDLForComplexMapType extends QueryTest with BeforeAndAfterAll {
}
assert(exception1.getMessage
.contains(
- "sort_columns is unsupported for map datatype column: mapfield"))
+ "sort_columns is unsupported for map datatype column: mapField"))
}
test("Data Load Fail Issue") {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
index 6059cb6..0815eab 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableLike.scala
@@ -153,20 +153,6 @@ class TestCreateTableLike extends QueryTest with BeforeAndAfterEach with BeforeA
checkTableProperties(TableIdentifier("sourceTable"), TableIdentifier("targetTable"))
}
- test("table with partition") {
- sql("""
- | CREATE TABLE IF NOT EXISTS pt_tbl(
- | a int, b string
- | )
- | PARTITIONED BY (area string)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='Asia, America, Europe')
- """.stripMargin)
- sql("create table targetTable like pt_tbl")
- checkTableProperties(TableIdentifier("pt_tbl"), TableIdentifier("targetTable"))
- }
-
test("table with hive partition") {
sql(
"""
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithSortScope.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithSortScope.scala
index dac9124..110319a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithSortScope.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithSortScope.scala
@@ -45,7 +45,7 @@ class TestCreateTableWithSortScope extends QueryTest with BeforeAndAfterAll {
| TBLPROPERTIES('SORT_COLUMNS'='stringField', 'SORT_SCOPE'='GLOBAL_SORT')
""".stripMargin)
- checkExistence(sql("DESCRIBE FORMATTED tableWithGlobalSort"), true, "global_sort")
+ checkExistence(sql("DESCRIBE FORMATTED tableWithGlobalSort"), true, "GLOBAL_SORT")
sql(
s"""
@@ -59,7 +59,7 @@ class TestCreateTableWithSortScope extends QueryTest with BeforeAndAfterAll {
sql("DESCRIBE FORMATTED tableWithLocalSort")
- checkExistence(sql("DESCRIBE FORMATTED tableWithLocalSort"), true, "local_sort")
+ checkExistence(sql("DESCRIBE FORMATTED tableWithLocalSort"), true, "LOCAL_SORT")
sql(
s"""
@@ -71,7 +71,7 @@ class TestCreateTableWithSortScope extends QueryTest with BeforeAndAfterAll {
| TBLPROPERTIES('SORT_COLUMNS'='stringField', 'SORT_SCOPE'='BATCH_SORT')
""".stripMargin)
- checkExistence(sql("DESCRIBE FORMATTED tableWithBatchSort"), true, "batch_sort")
+ checkExistence(sql("DESCRIBE FORMATTED tableWithBatchSort"), true, "BATCH_SORT")
sql(
s"""
@@ -83,7 +83,7 @@ class TestCreateTableWithSortScope extends QueryTest with BeforeAndAfterAll {
| TBLPROPERTIES('SORT_COLUMNS'='stringField', 'SORT_SCOPE'='NO_SORT')
""".stripMargin)
- checkExistence(sql("DESCRIBE FORMATTED tableWithNoSort"), true, "no_sort")
+ checkExistence(sql("DESCRIBE FORMATTED tableWithNoSort"), true, "NO_SORT")
}
test("test create table with sort scope in abnormal cases") {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
index 557e748..cfd23ed 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
@@ -709,11 +709,11 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
.contains("Unsupported operation on non transactional table"))
//9. Show partition
- exception = intercept[MalformedCarbonCommandException] {
+ val ex = intercept[AnalysisException] {
sql("Show partitions sdkOutputTable").show(false)
}
- assert(exception.getMessage()
- .contains("Unsupported operation on non transactional table"))
+ assert(ex.getMessage()
+ .contains("SHOW PARTITIONS is not allowed on a table that is not partitioned"))
//12. Streaming table creation
// No need as External table don't accept table properties
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
index e31896f..637611b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortBigFileTest.scala
@@ -60,7 +60,7 @@ class CompactionSupportGlobalSortBigFileTest extends QueryTest with BeforeAndAft
"""
| CREATE TABLE compaction_globalsort(id INT, name STRING, city STRING, age INT)
| STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT')
+ | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='global_sort')
""".stripMargin)
sql("DROP TABLE IF EXISTS carbon_localsort")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
index ed0f466..dd54971 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortFunctionTest.scala
@@ -42,7 +42,7 @@ class CompactionSupportGlobalSortFunctionTest extends QueryTest with BeforeAndAf
"""
| CREATE TABLE compaction_globalsort(id INT, name STRING, city STRING, age INT)
| STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT', 'GLOBAL_SORT_PARTITIONS'='1')
+ | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='global_sort', 'GLOBAL_SORT_PARTITIONS'='1')
""".stripMargin)
sql("DROP TABLE IF EXISTS carbon_localsort")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
index a1d0fe0..1563811 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSupportGlobalSortParameterTest.scala
@@ -43,7 +43,7 @@ class CompactionSupportGlobalSortParameterTest extends QueryTest with BeforeAndA
"""
| CREATE TABLE compaction_globalsort(id INT, name STRING, city STRING, age INT)
| STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='GLOBAL_SORT', 'GLOBAL_SORT_PARTITIONS'='1')
+ | TBLPROPERTIES('SORT_COLUMNS'='city,name', 'SORT_SCOPE'='global_sort', 'GLOBAL_SORT_PARTITIONS'='1')
""".stripMargin)
sql("DROP TABLE IF EXISTS carbon_localsort")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
index 255f399..4a0e74f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
@@ -138,20 +138,6 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
}
// ----------------------------------- Configuration Validity -----------------------------------
- test("Don't support GLOBAL_SORT on partitioned table") {
- sql("DROP TABLE IF EXISTS carbon_globalsort_partitioned")
- sql(
- """
- | CREATE TABLE carbon_globalsort_partitioned(name STRING, city STRING, age INT)
- | PARTITIONED BY (id INT)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3', 'SORT_SCOPE'='GLOBAL_SORT', 'sort_columns' = 'name, city')
- """.stripMargin)
-
- intercept[MalformedCarbonCommandException] {
- sql(s"LOAD DATA LOCAL INPATH '$filePath' INTO TABLE carbon_globalsort_partitioned")
- }
- }
test("Number of partitions should be greater than 0") {
intercept[MalformedCarbonCommandException] {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
index 724e9f4..8793608 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
@@ -85,7 +85,7 @@ class TestLoadDataWithMalformedCarbonCommandException extends QueryTest with Bef
}
assert(e.getMessage
.equals(
- "DICTIONARY_EXCLUDE column: ccc does not exist in table or unsupported for complex child " +
+ "DICTIONARY_EXCLUDE column: CCC does not exist in table or unsupported for complex child " +
"column. Please check the create table statement."))
}
@@ -95,7 +95,7 @@ class TestLoadDataWithMalformedCarbonCommandException extends QueryTest with Bef
}
assert(e.getMessage
.equals(
- "DICTIONARY_INCLUDE column: aaa does not exist in table or unsupported for complex child " +
+ "DICTIONARY_INCLUDE column: AAA does not exist in table or unsupported for complex child " +
"column. Please check the create table statement."))
}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAllDataTypeForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAllDataTypeForPartitionTable.scala
deleted file mode 100644
index 82c6d48..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAllDataTypeForPartitionTable.scala
+++ /dev/null
@@ -1,1238 +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.spark.testsuite.partition
-
-import java.sql.{Date, Timestamp}
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.Row
-import org.scalatest.BeforeAndAfterAll
-import scala.collection.mutable
-
-import org.apache.spark.sql.test.TestQueryExecutor
-import org.apache.spark.sql.test.util.QueryTest
-
-class TestAllDataTypeForPartitionTable 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")
-
- dropTable
- }
-
- override def afterAll = {
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
- CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-
- dropTable
- }
-
- def dropTable = {
- sql("drop table if exists allTypeTable_hash_smallInt")
- sql("drop table if exists allTypeTable_hash_int")
- sql("drop table if exists allTypeTable_hash_bigint")
- sql("drop table if exists allTypeTable_hash_float")
- sql("drop table if exists allTypeTable_hash_double")
- sql("drop table if exists allTypeTable_hash_decimal")
- sql("drop table if exists allTypeTable_hash_timestamp")
- sql("drop table if exists allTypeTable_hash_date")
- sql("drop table if exists allTypeTable_hash_string")
- sql("drop table if exists allTypeTable_hash_varchar")
- sql("drop table if exists allTypeTable_hash_char")
-
- sql("drop table if exists allTypeTable_list_smallInt")
- sql("drop table if exists allTypeTable_list_int")
- sql("drop table if exists allTypeTable_list_bigint")
- sql("drop table if exists allTypeTable_list_float")
- sql("drop table if exists allTypeTable_list_double")
- sql("drop table if exists allTypeTable_list_decimal")
- sql("drop table if exists allTypeTable_list_timestamp")
- sql("drop table if exists allTypeTable_list_date")
- sql("drop table if exists allTypeTable_list_string")
- sql("drop table if exists allTypeTable_list_varchar")
- sql("drop table if exists allTypeTable_list_char")
-
- sql("drop table if exists allTypeTable_range_smallInt")
- sql("drop table if exists allTypeTable_range_int")
- sql("drop table if exists allTypeTable_range_bigint")
- sql("drop table if exists allTypeTable_range_float")
- sql("drop table if exists allTypeTable_range_double")
- sql("drop table if exists allTypeTable_range_decimal")
- sql("drop table if exists allTypeTable_range_timestamp")
- sql("drop table if exists allTypeTable_range_date")
- sql("drop table if exists allTypeTable_range_string")
- sql("drop table if exists allTypeTable_range_varchar")
- sql("drop table if exists allTypeTable_range_char")
- }
-
-
- test("allTypeTable_hash_smallInt") {
- val tableName = "allTypeTable_hash_smallInt"
-
- sql(
- s"""create table $tableName(
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(smallIntField smallInt)
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = -32768"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 128"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 32767"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_hash_int") {
- val tableName = "allTypeTable_hash_int"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(intField int)
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = -2147483648"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 32768"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 2147483647"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_hash_bigint") {
- val tableName = "allTypeTable_hash_bigint"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(bigIntField bigint)
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = -9223372036854775808"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 2147483648"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 9223372036854775807"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_hash_float") {
- val tableName = "allTypeTable_hash_float"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(floatField float)
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = -2147483648.1"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483647.1"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483648.1"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_hash_double") {
- val tableName = "allTypeTable_hash_double"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(doubleField double)
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = -9223372036854775808.1"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = 9223372036854775807.1"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2")),
- Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-
- }
-
- test("allTypeTable_hash_decimal") {
- val tableName = "allTypeTable_hash_decimal"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(decimalField decimal(25, 4))
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('-9223372036854775808.1234' as decimal(25, 4))"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775807.1234' as decimal(25, 4))"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775808.1234' as decimal(25, 4))"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_hash_timestamp") {
- val tableName = "allTypeTable_hash_timestamp"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(timestampField timestamp)
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-11 00:00:01'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-12 23:59:59'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-13 23:59:59'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- ignore("allTypeTable_hash_date") {
- val tableName = "allTypeTable_hash_date"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(dateField date)
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-11'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-12'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-13'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_hash_string") {
- val tableName = "allTypeTable_hash_string"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(stringField string)
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc1'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc2'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc3'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_hash_varchar") {
- val tableName = "allTypeTable_hash_varchar"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(varcharField varchar(10))
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_hash_char") {
- val tableName = "allTypeTable_hash_char"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(charField char(10))
- | stored by 'carbondata'
- | tblproperties('partition_type'='hash','num_partitions'='3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_smallInt") {
- val tableName = "allTypeTable_list_smallInt"
-
- sql(
- s"""create table $tableName(
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(smallIntField smallInt)
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='-32768, 32767')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = -32768"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 128"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 32767"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_int") {
- val tableName = "allTypeTable_list_int"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(intField int)
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='-2147483648, 2147483647')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = -2147483648"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 32768"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 2147483647"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_bigint") {
- val tableName = "allTypeTable_list_bigint"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(bigIntField bigint)
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='-9223372036854775808, 9223372036854775807')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = -9223372036854775808"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 2147483648"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 9223372036854775807"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_float") {
- val tableName = "allTypeTable_list_float"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(floatField float)
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='-2147483648.1,2147483648.1')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = -2147483648.1"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483647.1"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483648.1"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_double") {
- val tableName = "allTypeTable_list_double"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(doubleField double)
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='-9223372036854775808.1,9223372036854775808.1')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = -9223372036854775808.1"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = 9223372036854775807.1"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2")),
- Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-
- }
-
- test("allTypeTable_list_decimal") {
- val tableName = "allTypeTable_list_decimal"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(decimalField decimal(25, 4))
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='-9223372036854775808.1234, 9223372036854775808.1234')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('-9223372036854775808.1234' as decimal(25, 4))"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775807.1234' as decimal(25, 4))"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775808.1234' as decimal(25, 4))"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_timestamp") {
- val tableName = "allTypeTable_list_timestamp"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(timestampField timestamp)
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='2017-06-11 00:00:01, 2017-06-13 23:59:59')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-11 00:00:01'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-12 23:59:59'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-13 23:59:59'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_date") {
- val tableName = "allTypeTable_list_date"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(dateField date)
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='2017-06-11,2017-06-13')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-11'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-12'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-13'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_string") {
- val tableName = "allTypeTable_list_string"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(stringField string)
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='abc1,abc2')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc1'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc2'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc3'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_varchar") {
- val tableName = "allTypeTable_list_varchar"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(varcharField varchar(10))
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='abcd1,abcd3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_list_char") {
- val tableName = "allTypeTable_list_char"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(charField char(10))
- | stored by 'carbondata'
- | tblproperties('partition_type'='list','list_info'='abcde1,abcde3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_range_smallInt") {
- val tableName = "allTypeTable_range_smallInt"
-
- sql(
- s"""create table $tableName(
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(smallIntField smallInt)
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='0, 129')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = -32768"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 128"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where smallIntField = 32767"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_range_int") {
- val tableName = "allTypeTable_range_int"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(intField int)
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='-1, 32769')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = -2147483648"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 32768"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where intField = 2147483647"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_range_bigint") {
- val tableName = "allTypeTable_range_bigint"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(bigIntField bigint)
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='-9223372036854775807, 2147483649')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = -9223372036854775808"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 2147483648"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where bigIntField = 9223372036854775807"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_range_float") {
- val tableName = "allTypeTable_range_float"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(floatField float)
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='-2147483647.1,2147483648.1')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = -2147483648.1"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483647.1"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where floatField = 2147483648.1"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_range_double") {
- val tableName = "allTypeTable_range_double"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(doubleField double)
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='-9223372036854775807.1,9223372036854775808.1')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = -9223372036854775808.1"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where doubleField = 9223372036854775807.1"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2")),
- Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
-
- }
-
- test("allTypeTable_range_decimal") {
- val tableName = "allTypeTable_range_decimal"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(decimalField decimal(25, 4))
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='-9223372036854775807.1234, 9223372036854775808.1234')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('-9223372036854775808.1234' as decimal(25, 4))"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775807.1234' as decimal(25, 4))"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where decimalField = cast('9223372036854775808.1234' as decimal(25, 4))"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_range_timestamp") {
- val tableName = "allTypeTable_range_timestamp"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(timestampField timestamp)
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='2017-06-11 00:00:02, 2017-06-13 23:59:59')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-11 00:00:01'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-12 23:59:59'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where timestampField = '2017-06-13 23:59:59'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- ignore("allTypeTable_range_date") {
- val tableName = "allTypeTable_range_date"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | stringField string,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(dateField date)
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='2017-06-12,2017-06-13')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-11'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-12'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where dateField = '2017-06-13'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_range_string") {
- val tableName = "allTypeTable_range_string"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | varcharField varchar(10),
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(stringField string)
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='abc2,abc3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc1'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc2'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where stringField = 'abc3'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_range_varchar") {
- val tableName = "allTypeTable_range_varchar"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | charField char(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(varcharField varchar(10))
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='abcd2,abcd3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-
- test("allTypeTable_range_char") {
- val tableName = "allTypeTable_range_char"
-
- sql(
- s"""create table $tableName(
- | smallIntField smallInt,
- | intField int,
- | bigIntField bigint,
- | floatField float,
- | doubleField double,
- | decimalField decimal(25, 4),
- | timestampField timestamp,
- | dateField date,
- | stringField string,
- | varcharField varchar(10),
- | arrayField array<string>,
- | structField struct<col1:string, col2:string, col3:string>)
- | partitioned by(charField char(10))
- | stored by 'carbondata'
- | tblproperties('partition_type'='range','range_info'='abcde2,abcde3')
- """.stripMargin)
-
- sql(s"load data local inpath '$resourcesPath/alldatatypeforpartition.csv' into table $tableName " +
- "options ('COMPLEX_DELIMITER_LEVEL_1'='$', 'COMPLEX_DELIMITER_LEVEL_2'=':')")
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd1'"),
- Seq(Row(-32768, -2147483648, -9223372036854775808L, -2147483648.1, -9223372036854775808.1, BigDecimal("-9223372036854775808.1234"), Timestamp.valueOf("2017-06-11 00:00:01"), Date.valueOf("2017-06-11"), "abc1", "abcd1", "abcde1", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "1")), Row("a", "b", "1"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd2'"),
- Seq(Row(128, 32768, 2147483648L, 2147483647.1, 9223372036854775807.1, BigDecimal("9223372036854775807.1234"), Timestamp.valueOf("2017-06-12 23:59:59"), Date.valueOf("2017-06-12"), "abc2", "abcd2", "abcde2", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "2")), Row("a", "b", "2"))))
-
- checkAnswer(sql(s"select smallIntField,intField,bigIntField,floatField,doubleField,decimalField,timestampField,dateField,stringField,varcharField,charField,arrayField,structField from $tableName where varcharField = 'abcd3'"),
- Seq(Row(32767, 2147483647, 9223372036854775807L, 2147483648.1, 9223372036854775808.1, BigDecimal("9223372036854775808.1234"), Timestamp.valueOf("2017-06-13 23:59:59"), Date.valueOf("2017-06-13"), "abc3", "abcd3", "abcde3", new mutable.WrappedArray.ofRef[String](Array("a", "b", "c", "3")), Row("a", "b", "3"))))
- }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestCompactionForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestCompactionForPartitionTable.scala
deleted file mode 100644
index 0217486..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestCompactionForPartitionTable.scala
+++ /dev/null
@@ -1,82 +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.spark.testsuite.partition
-
-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.test.TestQueryExecutor
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-class TestCompactionForPartitionTable extends QueryTest with BeforeAndAfterAll {
-
- override def beforeAll {
- dropTable
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
- sql(
- """
- | CREATE TABLE originTable (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | STORED BY 'org.apache.carbondata.format'
- """.stripMargin)
-
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- }
-
- test("minor compaction") {
- sql("create table part_minor_compact(a String, b int) partitioned by (c int) stored by 'carbondata' tblproperties('PARTITION_TYPE'='LIST','LIST_INFO'='1,2')")
- sql("insert into part_minor_compact select 'a', 2, 3 from originTable limit 1")
- sql("insert into part_minor_compact select 'b', 3, 4 from originTable limit 1")
- sql("insert into part_minor_compact select 'c', 4, 5 from originTable limit 1")
- sql("insert into part_minor_compact select 'd', 1, 2 from originTable limit 1")
-
- checkAnswer(sql("select * from part_minor_compact where c = 4"), Seq(Row("b", 3, 4)))
-
- sql("alter table part_minor_compact compact 'minor'")
-
- checkAnswer(sql("select * from part_minor_compact where c = 4"), Seq(Row("b", 3, 4)))
- }
-
- test("major compaction") {
- sql("create table part_major_compact(a String, b int) partitioned by (c int) stored by 'carbondata' tblproperties('PARTITION_TYPE'='LIST','LIST_INFO'='1,2')")
- sql("insert into part_major_compact select 'a', 2, 3 from originTable limit 1")
- sql("insert into part_major_compact select 'b', 3, 4 from originTable limit 1")
- sql("insert into part_major_compact select 'c', 4, 5 from originTable limit 1")
- sql("insert into part_major_compact select 'd', 1, 2 from originTable limit 1")
-
- checkAnswer(sql("select * from part_major_compact where c = 4"), Seq(Row("b", 3, 4)))
-
- sql("alter table part_major_compact compact 'major'")
-
- checkAnswer(sql("select * from part_major_compact where c = 4"), Seq(Row("b", 3, 4)))
- }
-
- override def afterAll = {
- dropTable
- }
-
- def dropTable = {
- sql("drop table if exists part_minor_compact")
- sql("drop table if exists part_major_compact")
- sql("drop table if exists originTable")
- }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
deleted file mode 100644
index b246839..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
+++ /dev/null
@@ -1,430 +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.spark.testsuite.partition
-
-import org.apache.spark.sql.{CarbonEnv, SQLContext}
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.test.util.QueryTest
-
-class TestDDLForPartitionTable 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")
- dropTable
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
- }
-
- test("create partition table: hash partition") {
- sql(
- """
- | CREATE TABLE default.hashTable (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
-
- val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "hashTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- assert(partitionInfo != null)
- assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
- assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.INT)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 0)
- assert(partitionInfo.getPartitionType == PartitionType.HASH)
- assert(partitionInfo.getNumPartitions == 3)
- }
-
- test("create partition table: range partition") {
- sql(
- """
- | CREATE TABLE default.rangeTable (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59', 'DICTIONARY_INCLUDE'='doj')
- """.stripMargin)
-
- val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "rangeTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- assert(partitionInfo != null)
- assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
- assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 2)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
- assert(partitionInfo.getPartitionType == PartitionType.RANGE)
- assert(partitionInfo.getRangeInfo.size == 2)
- assert(partitionInfo.getRangeInfo.get(0).equals("2017-06-11 00:00:02"))
- assert(partitionInfo.getRangeInfo.get(1).equals("2017-06-13 23:59:59"))
- }
-
- test("create partition table: list partition") {
- sql(
- """
- | CREATE TABLE default.listTable (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (workgroupcategory string)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3)')
- """.stripMargin)
- val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- assert(partitionInfo != null)
- assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("workgroupcategory"))
- assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.STRING)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 0)
- assert(partitionInfo.getPartitionType == PartitionType.LIST)
- assert(partitionInfo.getListInfo.size == 3)
- assert(partitionInfo.getListInfo.get(0).size == 1)
- assert(partitionInfo.getListInfo.get(0).get(0).equals("0"))
- assert(partitionInfo.getListInfo.get(1).size == 1)
- assert(partitionInfo.getListInfo.get(1).get(0).equals("1"))
- assert(partitionInfo.getListInfo.get(2).size == 2)
- assert(partitionInfo.getListInfo.get(2).get(0).equals("2"))
- assert(partitionInfo.getListInfo.get(2).get(1).equals("3"))
- }
-
- test("create partition table: list partition with duplicate value") {
- intercept[Exception] { sql(
- """
- | CREATE TABLE default.listTableError (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (workgroupcategory string)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3, 1)')
- """.stripMargin) }
- }
-
- test("test exception if partition column is dropped") {
- sql("drop table if exists test")
- sql(
- "create table test(a int, b string) partitioned by (c int) stored by 'carbondata' " +
- "tblproperties('PARTITION_TYPE'='LIST','list_info'='0,10,5,20')")
- intercept[Exception] { sql("alter table test drop columns(c)") }
- }
-
- test("test describe formatted for partition column") {
- sql("drop table if exists des")
- sql(
- """create table des(a int, b string) partitioned by (c string) stored by 'carbondata'
- |tblproperties ('partition_type'='list','list_info'='1,2')""".stripMargin)
- checkExistence(sql("describe formatted des"), true, "Partition Type")
- sql("drop table if exists des")
- }
-
- test("test exception if hash number is invalid") {
- sql("DROP TABLE IF EXISTS test_hash_1")
- val exception_test_hash_1: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_hash_1(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='2.1')
- """.stripMargin
- )
- }
- assert(exception_test_hash_1.getMessage.contains("Invalid partition definition"))
-
- sql("DROP TABLE IF EXISTS test_hash_2")
- val exception_test_hash_2: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_hash_2(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='abc')
- """.stripMargin
- )
- }
- assert(exception_test_hash_2.getMessage.contains("Invalid partition definition"))
-
- sql("DROP TABLE IF EXISTS test_hash_3")
- val exception_test_hash_3: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_hash_3(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='-2.1')
- """.stripMargin
- )
- }
- assert(exception_test_hash_3.getMessage.contains("Invalid partition definition"))
- }
-
-
- test("test exception when values in list_info can not match partition column type") {
- sql("DROP TABLE IF EXISTS test_list_int")
- val exception_test_list_int: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_list_int(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_list_small")
- val exception_test_list_small: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_list_small(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 SMALLINT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_list_small.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_list_float")
- val exception_test_list_float: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_list_float(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 FLOAT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_list_float.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_list_double")
- val exception_test_list_double: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_list_double(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 DOUBLE) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_list_double.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_list_bigint")
- val exception_test_list_bigint: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_list_bigint(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 BIGINT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_list_bigint.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_list_date")
- val exception_test_list_date: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_list_date(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 DATE) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_list_date.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_list_timestamp")
- val exception_test_list_timestamp: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_list_timestamp(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 TIMESTAMP) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_list_timestamp.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_list_decimal")
- val exception_test_list_decimal: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_list_decimal(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 DECIMAL(25, 4)) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='23.23111,2.32')
- """.stripMargin)
- }
- assert(exception_test_list_decimal.getMessage.contains("Invalid Partition Values"))
- }
-
- test("test exception when values in range_info can not match partition column type") {
- sql("DROP TABLE IF EXISTS test_range_int")
- val exception_test_range_int: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_int(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_range_int.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_range_smallint")
- val exception_test_range_smallint: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_smallint(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 SMALLINT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_range_smallint.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_range_float")
- val exception_test_range_float: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_float(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 FLOAT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_range_float.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_range_double")
- val exception_test_range_double: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_double(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 DOUBLE) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_range_double.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_range_bigint")
- val exception_test_range_bigint: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_bigint(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 BIGINT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_range_bigint.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_range_date")
- val exception_test_range_date: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_date(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 DATE) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_range_date.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_range_timestamp")
- val exception_test_range_timestamp: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_timestamp(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 TIMESTAMP) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_range_timestamp.getMessage.contains("Invalid Partition Values"))
-
- sql("DROP TABLE IF EXISTS test_range_decimal")
- val exception_test_range_decimal: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_decimal(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 DECIMAL(25, 4)) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
- """.stripMargin)
- }
- assert(exception_test_range_decimal.getMessage.contains("Invalid Partition Values"))
- }
-
- test("Invalid Partition Range") {
- val exceptionMessage: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE default.rangeTableInvalid (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2017-06-11 00:00:02')
- """.stripMargin)
- }
- assert(exceptionMessage.getMessage
- .contains("Range info must define a valid range.Please check again!"))
- }
-
- test("test number of partitions for default partition") {
- sql("drop table if exists desc")
- sql("create table desc(name string) partitioned by (num int) stored by 'carbondata'")
- sql("insert into desc select 'abc',3")
- sql("insert into desc select 'abc',5")
- val descFormatted1 = sql("describe formatted desc").collect
- descFormatted1.find(_.get(0).toString.contains("Number of Partitions")) match {
- case Some(row) => assert(row.get(1).toString.contains("2"))
- }
- }
-
- override def afterAll = {
- dropTable
- }
-
- def dropTable = {
- sql("drop table if exists desc")
- sql("drop table if exists hashTable")
- sql("drop table if exists rangeTable")
- sql("drop table if exists listTable")
- sql("drop table if exists test")
- sql("DROP TABLE IF EXISTS test_hash_1")
- sql("DROP TABLE IF EXISTS test_hash_2")
- sql("DROP TABLE IF EXISTS test_hash_3")
- sql("DROP TABLE IF EXISTS test_list_int")
- sql("DROP TABLE IF EXISTS test_list_smallint")
- sql("DROP TABLE IF EXISTS test_list_bigint")
- sql("DROP TABLE IF EXISTS test_list_float")
- sql("DROP TABLE IF EXISTS test_list_double")
- sql("DROP TABLE IF EXISTS test_list_date")
- sql("DROP TABLE IF EXISTS test_list_timestamp")
- sql("DROP TABLE IF EXISTS test_list_decimal")
- sql("DROP TABLE IF EXISTS test_range_int")
- sql("DROP TABLE IF EXISTS test_range_smallint")
- sql("DROP TABLE IF EXISTS test_range_bigint")
- sql("DROP TABLE IF EXISTS test_range_float")
- sql("DROP TABLE IF EXISTS test_range_double")
- sql("DROP TABLE IF EXISTS test_range_date")
- sql("DROP TABLE IF EXISTS test_range_timestamp")
- sql("DROP TABLE IF EXISTS test_range_decimal")
- sql("DROP TABLE IF EXISTS rangeTableInvalid")
- }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
deleted file mode 100644
index 0633d7f..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
+++ /dev/null
@@ -1,206 +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.spark.testsuite.partition
-
-import org.apache.spark.sql.CarbonEnv
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.util.CarbonProperties
-
-class TestDDLForPartitionTableWithDefaultProperties extends QueryTest with BeforeAndAfterAll {
-
- override def beforeAll = {
- dropTable
- }
-
- test("create partition table: hash partition") {
- sql(
- """
- | CREATE TABLE default.hashTable (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
-
- val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "hashTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- assert(partitionInfo != null)
- assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
- assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.INT)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 0)
- assert(partitionInfo.getPartitionType == PartitionType.HASH)
- assert(partitionInfo.getNumPartitions == 3)
- }
-
- test("create partition table: range partition") {
- sql(
- """
- | CREATE TABLE default.rangeTable (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59','DICTIONARY_INCLUDE'='doj')
- """.stripMargin)
-
- val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "rangeTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- assert(partitionInfo != null)
- assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
- assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 2)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
- assert(partitionInfo.getPartitionType == PartitionType.RANGE)
- assert(partitionInfo.getRangeInfo.size == 2)
- assert(partitionInfo.getRangeInfo.get(0).equals("2017-06-11 00:00:02"))
- assert(partitionInfo.getRangeInfo.get(1).equals("2017-06-13 23:59:59"))
- }
-
- test("create partition table: list partition with timestamp datatype") {
- sql(
- """
- | CREATE TABLE default.listTable (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (projectenddate Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59',
- | 'DICTIONARY_INCLUDE'='projectenddate')
- """.stripMargin)
- val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- assert(partitionInfo != null)
- assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
- assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 2)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
- assert(partitionInfo.getPartitionType == PartitionType.LIST)
- assert(partitionInfo.getListInfo.size == 2)
- assert(partitionInfo.getListInfo.get(0).size == 1)
- assert(partitionInfo.getListInfo.get(0).get(0).equals("2017-06-11 00:00:02"))
- assert(partitionInfo.getListInfo.get(1).size == 1)
- assert(partitionInfo.getListInfo.get(1).get(0).equals("2017-06-13 23:59:59"))
- }
-
- test("create partition table: list partition with date datatype") {
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
-
- sql(
- """
- | CREATE TABLE default.listTableDate (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (projectenddate date)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='2017-06-11 , 2017-06-13')
- """.stripMargin)
- val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTableDate")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- assert(partitionInfo != null)
- assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
- assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.DATE)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 2)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
- assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
- assert(partitionInfo.getPartitionType == PartitionType.LIST)
- assert(partitionInfo.getListInfo.size == 2)
- assert(partitionInfo.getListInfo.get(0).size == 1)
- assert(partitionInfo.getListInfo.get(0).get(0).equals("2017-06-11"))
- assert(partitionInfo.getListInfo.get(1).size == 1)
- assert(partitionInfo.getListInfo.get(1).get(0).equals("2017-06-13"))
- }
-
- test("test exception when values in list_info can not match partition column type") {
- sql("DROP TABLE IF EXISTS test_list_int")
- val exception_test_list_int: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_list_int(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,2,(abc,efg)')
- """.stripMargin)
- }
- assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
- }
-
- test("test exception when partition values in rangeTable are in group ") {
- sql("DROP TABLE IF EXISTS rangeTable")
- val exception_test_list_int: Exception = intercept[Exception] {
- sql(
- """
- |CREATE TABLE default.rangeTable (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2017-06-11 00:00:02, (2017-06-13 23:59:59, 2017-09-13 23:45:59)')
- """.stripMargin)
- }
- assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
- }
-
- test("test exception when values in rangeTable does not match partition column type") {
- sql("DROP TABLE IF EXISTS rangeTable")
- val exception_test_list_int: Exception = intercept[Exception] {
- sql(
- """
- |CREATE TABLE default.rangeTable (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2017-06-11 00:00:02, abc, 2017-09-13 23:45:59')
- """.stripMargin)
- }
- assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
- }
-
-
- override def afterAll = {
- dropTable
- }
-
- def dropTable = {
- sql("drop table if exists hashTable")
- sql("drop table if exists rangeTable")
- sql("drop table if exists listTable")
- sql("drop table if exists listTableDate")
- }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
deleted file mode 100644
index 723d969..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
+++ /dev/null
@@ -1,409 +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.spark.testsuite.partition
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.test.TestQueryExecutor
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.{CarbonMetadata, SegmentFileStore}
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.spark.sql.test.util.QueryTest
-
-import org.apache.carbondata.core.datamap.Segment
-
-class TestDataLoadingForPartitionTable extends QueryTest with BeforeAndAfterAll {
-
- override def beforeAll {
- dropTable
-
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
- sql(
- """
- | CREATE TABLE originTable (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | STORED BY 'org.apache.carbondata.format'
- """.stripMargin)
-
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
- sql(
- """
- | CREATE TABLE originMultiLoads (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | STORED BY 'org.apache.carbondata.format'
- """.stripMargin)
-
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originMultiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originMultiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originMultiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- }
-
- def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Seq[Int]): Unit = {
- val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
- val segmentDir = carbonTable.getSegmentPath(segmentId)
-
- val dataFiles = if (FileFactory.isFileExist(segmentDir)) {
- val carbonFile = FileFactory.getCarbonFile(segmentDir)
- carbonFile.listFiles(new CarbonFileFilter() {
- override def accept(file: CarbonFile): Boolean = {
- return file.getName.endsWith(".carbondata")
- }
- })
- } else {
- val segment = Segment.getSegment(segmentId, carbonTable.getTablePath)
- val store = new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName)
- store.readIndexFiles(new Configuration(false))
- store.getIndexFilesMap.asScala.flatMap(_._2.asScala).map(f => FileFactory.getCarbonFile(f)).toArray
- }
-
- assert(dataFiles.size == partitions.size)
-
- dataFiles.foreach { dataFile =>
- val taskId = CarbonTablePath.DataFileUtil.getTaskNo(dataFile.getName).split("_")(0).toInt
- assert(partitions.exists(_ == taskId))
- }
- }
-
- test("data loading for partition table: hash partition") {
- sql(
- """
- | CREATE TABLE hashTable (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- |
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE hashTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
- validateDataFiles("default_hashTable", "0", Seq(0, 1, 2))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTable order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
- }
-
- test("data loading for partition table: range partition") {
- sql(
- """
- | CREATE TABLE rangeTable (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='01-01-2010, 01-01-2015, 01-04-2015, 01-07-2015')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
- validateDataFiles("default_rangeTable", "0", Seq(0, 1, 2, 4))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
- }
-
- test("data loading for partition table: list partition") {
- sql(
- """
- | CREATE TABLE listTable (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (workgroupcategory int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3)')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
- validateDataFiles("default_listTable", "0", Seq(2, 3))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
- }
-
- test("single pass data loading for partition table: hash partition") {
- sql(
- """
- | CREATE TABLE hashTableSinglePass (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE hashTableSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-
- validateDataFiles("default_hashTableSinglePass", "0", Seq(0, 1, 2))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTableSinglePass order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
- }
-
- test("single pass data loading for partition table: range partition") {
- sql(
- """
- | CREATE TABLE rangeTableSinglePass (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='01-01-2010, 01-01-2015, 01-04-2015, 01-07-2015')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTableSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-
- validateDataFiles("default_rangeTableSinglePass", "0", Seq(0, 1, 2, 4))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableSinglePass order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
- }
-
- test("single pass data loading for partition table: list partition") {
- sql(
- """
- | CREATE TABLE listTableSinglePass (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (workgroupcategory int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3)')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTableSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-
- validateDataFiles("default_listTableSinglePass", "0", Seq(2, 3))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableSinglePass order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
- }
-
- test("Insert into for partition table: hash partition") {
- sql(
- """
- | CREATE TABLE hashTableForInsert (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
- sql("insert into hashTableForInsert select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
-
- validateDataFiles("default_hashTableForInsert", "0", Seq(0, 1, 2))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTableForInsert order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
- }
-
- test("Insert into for partition table: range partition") {
- sql(
- """
- | CREATE TABLE rangeTableForInsert (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='01-01-2010, 01-01-2015, 01-04-2015, 01-07-2015')
- """.stripMargin)
- sql("insert into rangeTableForInsert select empno, empname, designation, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, doj from originTable")
-
- validateDataFiles("default_rangeTableForInsert", "0", Seq(0, 1, 2, 4))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableForInsert order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
-
- }
-
- test("Insert into partition table: list partition") {
- sql(
- """
- | CREATE TABLE listTableForInsert (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (workgroupcategory int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3)')
- """.stripMargin)
- sql("insert into listTableForInsert select empno, empname, designation, doj, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, workgroupcategory from originTable")
-
- validateDataFiles("default_listTableForInsert", "0", Seq(2, 3))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableForInsert order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
- }
-
- test("multiple data loading for partition table") {
- sql(
- """
- | CREATE TABLE multiLoads (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoads OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from multiLoads order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originMultiLoads order by empno"))
- }
-
- test("multiple single pass data loading for partition table") {
- sql(
- """
- | CREATE TABLE multiLoadsSinglePass (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoadsSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoadsSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE multiLoadsSinglePass OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"', 'SINGLE_PASS'='TRUE')""")
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from multiLoadsSinglePass order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originMultiLoads order by empno"))
- }
-
- test("multiple insertInto for partition table") {
- sql(
- """
- | CREATE TABLE multiInserts (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
- sql("insert into multiInserts select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
- sql("insert into multiInserts select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
- sql("insert into multiInserts select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from multiInserts order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originMultiLoads order by empno"))
- }
-
- test("multiple data loading and insertInto for partition table") {
- sql(
- """
- | CREATE TABLE loadAndInsert (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE loadAndInsert OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql("insert into loadAndInsert select empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary, empno from originTable")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE loadAndInsert OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from loadAndInsert order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originMultiLoads order by empno"))
- }
-
- test("list partition with string coloum and list_info in upper case") {
- sql(
- """
- | CREATE TABLE listTableUpper (empno int, empname String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (designation string)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='SE,SSE')
- """.stripMargin)
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTableUpper OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
- validateDataFiles("default_listTableUpper", "0", Seq(0,1,2))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableUpper order by empno"),
- sql("select empno, empname, designation, doj, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable order by empno"))
- }
-
-
- test("badrecords on partition column") {
- sql("create table badrecordsPartition(intField1 int, stringField1 string) partitioned by (intField2 int) stored by 'carbondata' tblproperties('partition_type'='hash', 'num_partitions'='5')")
- sql(s"load data local inpath '$resourcesPath/data_partition_badrecords.csv' into table badrecordsPartition options('bad_records_action'='force')")
-
- checkAnswer(sql("select count(*) cnt from badrecordsPartition where intField2 = 13"), Seq(Row(1)))
- checkAnswer(sql("select count(*) cnt from badrecordsPartition where intField2 = 14"), Seq(Row(1)))
- checkAnswer(sql("select count(*) cnt from badrecordsPartition where intField2 is null"), Seq(Row(9)))
- checkAnswer(sql("select count(*) cnt from badrecordsPartition where intField2 is not null"), Seq(Row(2)))
- }
-
- override def afterAll = {
- dropTable
- }
-
- def dropTable = {
- sql("drop table if exists originTable")
- sql("drop table if exists hashTable")
- sql("drop table if exists rangeTable")
- sql("drop table if exists listTable")
- sql("drop table if exists hashTableForInsert")
- sql("drop table if exists rangeTableForInsert")
- sql("drop table if exists listTableForInsert")
- sql("drop table if exists originMultiLoads")
- sql("drop table if exists multiLoads")
- sql("drop table if exists multiInserts")
- sql("drop table if exists loadAndInsert")
- sql("drop table if exists listTableUpper")
- sql("drop table if exists badrecordsPartition")
- sql("drop table if exists hashTableSinglePass")
- sql("drop table if exists rangeTableSinglePass")
- sql("drop table if exists listTableSinglePass")
- sql("drop table if exists multiLoadsSinglePass")
- }
-
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestQueryForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestQueryForPartitionTable.scala
deleted file mode 100644
index df0eea3..0000000
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestQueryForPartitionTable.scala
+++ /dev/null
@@ -1,287 +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.spark.testsuite.partition
-
-import org.apache.spark.sql.test.TestQueryExecutor
-import org.scalatest.BeforeAndAfterAll
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.spark.sql.test.util.QueryTest
-
-class TestQueryForPartitionTable extends QueryTest with BeforeAndAfterAll {
- override def beforeAll = {
- dropTable
-
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
- // create normal table(not a partition table)
- sql(
- """
- | CREATE TABLE originTable (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | STORED BY 'org.apache.carbondata.format'
- """.stripMargin)
-
- sql(
- """
- | CREATE TABLE hashTable (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
-
- sql(
- """
- | CREATE TABLE rangeTable (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='01-01-2010, 01-01-2015, 01-04-2015, 01-07-2015')
- """.stripMargin)
-
- sql(
- """
- | CREATE TABLE rangeTableOnString (empno int, designation String,
- | doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int,
- | deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,
- | attendance int, utilization int,salary int)
- | PARTITIONED BY (empname String)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='Ben, Jack, Sam, Tom')
- """.stripMargin)
-
- sql(
- """
- | CREATE TABLE rangeTableOnStringNo (empno int, designation String,
- | doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int,
- | deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,
- | attendance int, utilization int,salary int)
- | PARTITIONED BY (empname String)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='Ben, Jack, Sam, Tom', 'DICTIONARY_EXCLUDE'='empname')
- """.stripMargin)
-
- sql(
- """
- | CREATE TABLE listTable (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (workgroupcategory int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3)')
- """.stripMargin)
-
- sql(
- """
- | CREATE TABLE listTableOnString (empno int, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empname String)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3)')
- """.stripMargin)
-
- sql(
- """
- | CREATE TABLE listTableOnStringNo (empno int, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empname String)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3)', 'DICTIONARY_EXCLUDE'='empname')
- """.stripMargin)
-
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE originTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE hashTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTableOnString OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE rangeTableOnStringNo OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTable OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTableOnString OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE listTableOnStringNo OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
- }
-
- test("detail query on partition table: hash table") {
- // EqualTo
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTable where empno = 13"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empno = 13"))
- // In
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from hashTable where empno in (11, 13)"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empno in (11, 13)"))
- }
-
- test("detail query on partition table: range partition") {
- // EqualTo
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj = '2009-07-07 00:00:00'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj = '2009-07-07 00:00:00'"))
- // In
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj in (cast('2014-08-15 00:00:00' as timestamp), cast('2009-07-07 00:00:00' as timestamp))"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj in (cast('2014-08-15 00:00:00' as timestamp), cast('2009-07-07 00:00:00' as timestamp))"))
- // Range
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj >= '2014-08-15 00:00:00'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj >= '2014-08-15 00:00:00'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj <= '2014-08-15 00:00:00'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj <= '2014-08-15 00:00:00'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj > '2014-08-15 00:00:00'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj > '2014-08-15 00:00:00'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTable where doj < '2014-08-15 00:00:00'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where doj < '2014-08-15 00:00:00'"))
- }
-
- test("detail query on partition table: range partition on string") {
- // EqualTo
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname = 'madhan'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname = 'madhan'"))
- // In
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname in ('tom', 'jack')"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname in ('tom', 'jack')"))
- // Range
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname >= 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname >= 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname <= 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname <= 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname > 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname > 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnString where empname < 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname < 'tom'"))
- }
-
- test("detail query on partition table: range partition on string no dictionary") {
- // EqualTo
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname = 'madhan'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname = 'madhan'"))
- // In
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname in ('tom', 'jack')"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname in ('tom', 'jack')"))
- // Range
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname >= 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname >= 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname <= 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname <= 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname > 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname > 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from rangeTableOnStringNo where empname < 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname < 'tom'"))
- }
-
- test("detail query on partition table: list partition") {
- // EqualTo
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory = 2"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory = 2"))
-
- // In
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory in (2, 3)"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory in (2, 3)"))
-
- // Range
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory >= 2"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory >= 2"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory > 2"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory > 2"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory <= 2"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory <= 2"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTable where workgroupcategory < 2"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where workgroupcategory < 2"))
- }
-
- test("detail query on partition table: list partition on string") {
- // EqualTo
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname = 'madhan'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname = 'madhan'"))
- // In
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname in ('tom', 'jack')"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname in ('tom', 'jack')"))
- // Range
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname >= 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname >= 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname <= 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname <= 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname > 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname > 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnString where empname < 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname < 'tom'"))
- }
-
- test("detail query on partition table: list partition on string no dictionary") {
- // EqualTo
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname = 'madhan'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname = 'madhan'"))
- // In
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname in ('tom', 'jack')"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname in ('tom', 'jack')"))
- // Range
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname >= 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname >= 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname <= 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname <= 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname > 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname > 'tom'"))
-
- checkAnswer(sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from listTableOnStringNo where empname < 'tom'"),
- sql("select empno, empname, designation, doj, workgroupcategory, workgroupcategoryname, deptno, deptname, projectcode, projectjoindate, projectenddate, attendance, utilization, salary from originTable where empname < 'tom'"))
- }
-
- override def afterAll = {
- dropTable
- }
-
- def dropTable = {
- sql("drop table if exists originTable")
- sql("drop table if exists hashTable")
- sql("drop table if exists rangeTable")
- sql("drop table if exists rangeTableOnString")
- sql("drop table if exists rangeTableOnStringNo")
- sql("drop table if exists listTable")
- sql("drop table if exists listTableOnString")
- sql("drop table if exists listTableOnStringNo")
- }
-}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
index 4825968..c9dd879 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestShowPartitions.scala
@@ -46,80 +46,7 @@ class TestShowPartition extends QueryTest with BeforeAndAfterAll {
| STORED BY 'carbondata'
""".stripMargin)
- sql("drop table if exists hashTable")
- sql(
- """
- | CREATE TABLE hashTable (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
-
- sql("drop table if exists rangeTable")
- sql(
- """
- | CREATE TABLE rangeTable (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='01-01-2010, 01-01-2015')
- """.stripMargin)
-
- sql("drop table if exists listTable")
- sql(
- """
- | CREATE TABLE listTable (empno int, empname String, designation String, doj Timestamp,
- | workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (workgroupcategory int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3)')
- """.stripMargin)
-
sql(s"CREATE DATABASE if not exists partitionDB")
- sql("drop table if exists partitionDB.hashTable")
- sql("drop table if exists partitionDB.rangeTable")
- sql("drop table if exists partitionDB.listTable")
- sql(
- """
- | CREATE TABLE partitionDB.hashTable (empname String, designation String, doj Timestamp,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (empno int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
- """.stripMargin)
- sql(
- """
- | CREATE TABLE partitionDB.rangeTable (empno int, empname String, designation String,
- | workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (doj Timestamp)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='01-01-2010, 01-01-2015')
- """.stripMargin)
- sql(
- """
- | CREATE TABLE partitionDB.listTable (empno int, empname String, designation String,
- | doj Timestamp,workgroupcategoryname String, deptno int, deptname String,
- | projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
- | utilization int,salary int)
- | PARTITIONED BY (workgroupcategory int)
- | STORED BY 'org.apache.carbondata.format'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='0, 1, (2, 3)')
- """.stripMargin)
sql("DROP TABLE IF EXISTS hiveTable")
sql("""
@@ -138,52 +65,13 @@ class TestShowPartition extends QueryTest with BeforeAndAfterAll {
}
test("show partition table: exception when show not partition table") {
- val errorMessage = intercept[ProcessMetaDataException] {
+ val errorMessage = intercept[AnalysisException] {
sql("show partitions notPartitionTable").show()
}
assert(errorMessage.getMessage.contains(
"SHOW PARTITIONS is not allowed on a table that is not partitioned"))
}
- test("show partition table: hash table") {
- // EqualTo
- checkAnswer(sql("show partitions hashTable"), Seq(Row("empno = HASH_NUMBER(3)"), Row("partitionIds = [0, 1, 2]")))
-
- }
-
- test("show partition table: desc formatted should show partition type"){
- //check for partition type exist in desc formatted
- val result:DataFrame = sql("describe formatted hashTable")
- checkExistence(result,true,"Partition Type")
- val row: Array[Row] = result.collect().filter{row: Row => row.getString(0).contains("Partition Type")}
- assert(row(0).getString(1).contains("HASH"))
- }
-
- test("show partition table: range partition") {
- // EqualTo
- checkAnswer(sql("show partitions rangeTable"), Seq(Row("0, doj = DEFAULT"),
- Row("1, doj < 01-01-2010"), Row("2, 01-01-2010 <= doj < 01-01-2015")))
- }
-
- test("show partition table: list partition") {
- // EqualTo
- checkAnswer(sql("show partitions listTable"), Seq(Row("0, workgroupcategory = DEFAULT"),
- Row("1, workgroupcategory = 0"), Row("2, workgroupcategory = 1"), Row("3, workgroupcategory = 2, 3")))
-
- }
- test("show partition table: not default db") {
- // EqualTo
- checkAnswer(sql("show partitions partitionDB.hashTable"), Seq(Row("empno = HASH_NUMBER(3)"),
- Row("partitionIds = [0, 1, 2]")))
- // EqualTo
- checkAnswer(sql("show partitions partitionDB.rangeTable"), Seq(Row("0, doj = DEFAULT"),
- Row("1, doj < 01-01-2010"), Row("2, 01-01-2010 <= doj < 01-01-2015")))
- // EqualTo
- checkAnswer(sql("show partitions partitionDB.listTable"), Seq(Row("0, workgroupcategory = DEFAULT"),
- Row("1, workgroupcategory = 0"), Row("2, workgroupcategory = 1"), Row("3, workgroupcategory = 2, 3")))
-
- }
-
test("show partition table: hive partition table") {
// EqualTo
checkAnswer(sql("show partitions hiveTable"), Seq(Row("city=Hangzhou")))
@@ -194,27 +82,8 @@ class TestShowPartition extends QueryTest with BeforeAndAfterAll {
override def afterAll = {
sql("drop table if exists notPartitionTable")
- sql("drop table if exists hashTable")
- sql("drop table if exists listTable")
- sql("drop table if exists rangeTable")
sql("drop table if exists hiveTable")
try {
- sql("drop table if exists partitionDB.hashTable")
-
- } catch {
- case ex: NoSuchDatabaseException => print(ex.getMessage())
- }
- try {
- sql("drop table if exists partitionDB.rangeTable")
- } catch {
- case ex: NoSuchDatabaseException => print(ex.getMessage())
- }
- try {
- sql("drop table if exists partitionDB.listTable")
- } catch {
- case ex: NoSuchDatabaseException => print(ex.getMessage())
- }
- try {
sql("drop table if exists hiveDB.hiveTable")
} catch {
case ex: NoSuchDatabaseException => print(ex.getMessage())
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestUpdateForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestUpdateForPartitionTable.scala
index 14dab1e..c91caf6 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestUpdateForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestUpdateForPartitionTable.scala
@@ -14,6 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.carbondata.spark.testsuite.partition
import org.apache.spark.sql.test.util.QueryTest
@@ -24,40 +25,14 @@ class TestUpdateForPartitionTable extends QueryTest with BeforeAndAfterAll {
override def beforeAll: Unit = {
dropTable
- sql("create table test_range_partition_table (id int) partitioned by (name string) " +
- "stored by 'carbondata' TBLPROPERTIES('PARTITION_TYPE' = 'RANGE','RANGE_INFO' = 'a,e,f')")
sql("create table test_hive_partition_table (id int) partitioned by (name string) " +
"stored by 'carbondata'")
- sql("create table test_hash_partition_table (id int) partitioned by (name string) " +
- "stored by 'carbondata' TBLPROPERTIES('PARTITION_TYPE' = 'HASH','NUM_PARTITIONS' = '2')")
- sql("create table test_list_partition_table (id int) partitioned by (name string) " +
- "stored by 'carbondata' TBLPROPERTIES('PARTITION_TYPE' = 'LIST','LIST_INFO' = 'a,e,f')")
}
def dropTable = {
- sql("drop table if exists test_hash_partition_table")
- sql("drop table if exists test_list_partition_table")
- sql("drop table if exists test_range_partition_table")
sql("drop table if exists test_hive_partition_table")
}
-
- test ("test update for unsupported partition table") {
- val updateTables = Array(
- "test_range_partition_table",
- "test_list_partition_table",
- "test_hash_partition_table")
-
- updateTables.foreach(table => {
- sql("insert into " + table + " select 1,'b' ")
- val ex = intercept[UnsupportedOperationException] {
- sql("update " + table + " set (name) = ('c') where id = 1").collect()
- }
- assertResult("Unsupported update operation for range/hash/list partition table")(ex.getMessage)
- })
-
- }
-
test ("test update for hive(standard) partition table") {
sql("insert into test_hive_partition_table select 1,'b' ")
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/DropPartitionCallable.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/DropPartitionCallable.java
deleted file mode 100644
index 50ed39d..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/DropPartitionCallable.java
+++ /dev/null
@@ -1,40 +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.spark.partition;
-
-import java.util.concurrent.Callable;
-
-import org.apache.carbondata.spark.rdd.PartitionDropper;
-
-import org.apache.spark.sql.execution.command.DropPartitionCallableModel;
-
-public class DropPartitionCallable implements Callable<Void> {
-
- private DropPartitionCallableModel dropPartitionCallableModel;
-
- public DropPartitionCallable(DropPartitionCallableModel dropPartitionCallableModel) {
- this.dropPartitionCallableModel = dropPartitionCallableModel;
- }
-
- @Override
- public Void call() {
- PartitionDropper.triggerPartitionDrop(dropPartitionCallableModel);
- return null;
- }
-}
-
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/SplitPartitionCallable.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/SplitPartitionCallable.java
deleted file mode 100644
index fb21308..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/SplitPartitionCallable.java
+++ /dev/null
@@ -1,42 +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.spark.partition;
-
-import java.util.concurrent.Callable;
-
-import org.apache.carbondata.spark.rdd.PartitionSplitter;
-
-import org.apache.spark.sql.execution.command.SplitPartitionCallableModel;
-
-/**
- * Callable class which is used to split the partition in a separate callable.
- */
-public class SplitPartitionCallable implements Callable<Void> {
-
- private final SplitPartitionCallableModel splitPartitionCallableModel;
-
- public SplitPartitionCallable(SplitPartitionCallableModel splitPartitionCallableModel) {
- this.splitPartitionCallableModel = splitPartitionCallableModel;
- }
-
- @Override
- public Void call() {
- PartitionSplitter.triggerPartitionSplit(splitPartitionCallableModel);
- return null;
- }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala
deleted file mode 100644
index 4da2d9a..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala
+++ /dev/null
@@ -1,65 +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.spark
-
-import org.apache.spark.Partitioner
-
-import org.apache.carbondata.core.metadata.schema.PartitionInfo
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.scan.partition.{HashPartitioner => JavaHashPartitioner, ListPartitioner => JavaListPartitioner, RangePartitioner => JavaRangePartitioner}
-import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
-
-object PartitionFactory {
-
- def getPartitioner(partitionInfo: PartitionInfo): Partitioner = {
- partitionInfo.getPartitionType match {
- case PartitionType.HASH => new HashPartitioner(partitionInfo.getNumPartitions)
- case PartitionType.LIST => new ListPartitioner(partitionInfo)
- case PartitionType.RANGE => new RangePartitioner(partitionInfo)
- case partitionType =>
- throw new CarbonDataLoadingException(s"Unsupported partition type: $partitionType")
- }
- }
-}
-
-class HashPartitioner(partitions: Int) extends Partitioner {
-
- private val partitioner = new JavaHashPartitioner(partitions)
-
- override def numPartitions: Int = partitioner.numPartitions()
-
- override def getPartition(key: Any): Int = partitioner.getPartition(key)
-}
-
-class ListPartitioner(partitionInfo: PartitionInfo) extends Partitioner {
-
- private val partitioner = new JavaListPartitioner(partitionInfo)
-
- override def numPartitions: Int = partitioner.numPartitions()
-
- override def getPartition(key: Any): Int = partitioner.getPartition(key)
-}
-
-class RangePartitioner(partitionInfo: PartitionInfo) extends Partitioner {
-
- private val partitioner = new JavaRangePartitioner(partitionInfo)
-
- override def numPartitions: Int = partitioner.numPartitions()
-
- override def getPartition(key: Any): Int = partitioner.getPartition(key)
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
deleted file mode 100644
index 33d5eaa..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
+++ /dev/null
@@ -1,106 +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.spark.rdd
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.{Partition, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.execution.command.AlterPartitionModel
-import org.apache.spark.util.PartitionUtils
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.processing.loading.TableProcessingOperations
-import org.apache.carbondata.processing.partition.spliter.RowResultProcessor
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
-import org.apache.carbondata.spark.AlterPartitionResult
-import org.apache.carbondata.spark.util.CommonUtil
-
-class AlterTableLoadPartitionRDD[K, V](alterPartitionModel: AlterPartitionModel,
- result: AlterPartitionResult[K, V],
- partitionIds: Seq[String],
- bucketId: Int,
- identifier: AbsoluteTableIdentifier,
- prev: RDD[Array[AnyRef]])
- extends CarbonRDD[(K, V)](alterPartitionModel.sqlContext.sparkSession, prev) {
-
- val carbonLoadModel = alterPartitionModel.carbonLoadModel
- val segmentId = alterPartitionModel.segmentId
- val oldPartitionIds = alterPartitionModel.oldPartitionIds
- val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val partitionInfo = carbonTable.getPartitionInfo()
-
- override protected def internalGetPartitions: Array[Partition] = {
- val sc = alterPartitionModel.sqlContext.sparkContext
- sc.setLocalProperty("spark.scheduler.pool", "DDL")
- sc.setLocalProperty("spark.job.interruptOnCancel", "true")
- firstParent[Array[AnyRef]].partitions
- }
-
- override def internalCompute(split: Partition, context: TaskContext): Iterator[(K, V)] = {
- val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
- val rows = firstParent[Array[AnyRef]].iterator(split, context).toList.asJava
- val iter = new Iterator[(K, V)] {
- val partitionId: Int = partitionInfo.getPartitionId(split.index)
- carbonLoadModel.setTaskNo(String.valueOf(partitionId))
- carbonLoadModel.setSegmentId(segmentId)
-
- CommonUtil.setTempStoreLocation(split.index, carbonLoadModel,
- isCompactionFlow = false, isAltPartitionFlow = true)
- val tempStoreLoc: Array[String] = CarbonDataProcessorUtil.getLocalDataFolderLocation(
- carbonTable, carbonLoadModel.getTaskNo, segmentId, false, true)
-
- val loadStatus: Boolean = if (rows.isEmpty) {
- LOGGER.info("After repartition this split, NO target rows to write back.")
- true
- } else {
- val segmentProperties = PartitionUtils.getSegmentProperties(identifier,
- segmentId, partitionIds.toList, oldPartitionIds, partitionInfo, carbonTable)
- val processor = new RowResultProcessor(
- carbonTable,
- carbonLoadModel,
- segmentProperties,
- tempStoreLoc,
- bucketId)
- try {
- processor.execute(rows)
- } catch {
- case e: Exception =>
- sys.error(s"Exception when executing Row result processor ${ e.getMessage }")
- } finally {
- TableProcessingOperations
- .deleteLocalDataLoadFolderLocation(carbonLoadModel, false, true)
- }
- }
-
- val loadResult = segmentId
- var finished = false
-
- override def hasNext: Boolean = {
- !finished
- }
-
- override def next(): (K, V) = {
- finished = true
- result.getKey(loadResult, loadStatus)
- }
- }
- iter
- }
-}
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 5e33ea7..65fc9f3 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
@@ -101,11 +101,7 @@ class CarbonMergerRDD[K, V](
val iter = new Iterator[(K, V)] {
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
val carbonSparkPartition = theSplit.asInstanceOf[CarbonSparkPartition]
- if (carbonTable.isPartitionTable) {
- carbonLoadModel.setTaskNo(String.valueOf(carbonSparkPartition.partitionId))
- } else {
- carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
- }
+ carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
val partitionSpec = if (carbonTable.isHivePartitionTable) {
carbonSparkPartition.partitionSpec.get
} else {
@@ -339,7 +335,6 @@ class CarbonMergerRDD[K, V](
var defaultParallelism = sparkContext.defaultParallelism
val result = new java.util.ArrayList[Partition](defaultParallelism)
var taskPartitionNo = 0
- var carbonPartitionId = 0
var noOfBlocks = 0
val taskInfoList = new java.util.ArrayList[Distributable]
@@ -464,7 +459,6 @@ class CarbonMergerRDD[K, V](
val partitionTaskMap = new util.HashMap[PartitionSpec, String]()
val counter = new AtomicInteger()
var indexOfRangeColumn = -1
- var taskIdCount = 0
// As we are already handling null values in the filter expression separately so we
// can remove the null from the ranges we get, else it may lead to duplicate data
val newRanges = allRanges.filter { range =>
@@ -609,7 +603,6 @@ class CarbonMergerRDD[K, V](
logInfo("no.of.nodes where data present=" + nodeBlockMap.size())
defaultParallelism = sparkContext.defaultParallelism
- val isPartitionTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isPartitionTable
// Create Spark Partition for each task and assign blocks
nodeBlockMap.asScala.foreach { case (nodeName, splitList) =>
val taskSplitList = new java.util.ArrayList[NodeInfo](0)
@@ -633,15 +626,11 @@ class CarbonMergerRDD[K, V](
splitListForRange,
Array(nodeName))
}
- if (isPartitionTable) {
- carbonPartitionId = Integer.parseInt(taskInfo.getTaskId)
- }
result.add(
new CarbonSparkPartition(
id,
taskPartitionNo,
multiBlockSplit,
- carbonPartitionId,
getPartitionNamesFromTask(taskInfo.getTaskId, partitionTaskMap)))
taskPartitionNo += 1
}
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
deleted file mode 100644
index 8941636..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
+++ /dev/null
@@ -1,271 +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.spark.rdd
-
-import java.util.ArrayList
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.Job
-import org.apache.spark.{Partition, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.execution.command.AlterPartitionModel
-import org.apache.spark.sql.hive.DistributionUtil
-import org.apache.spark.sql.util.CarbonException
-import org.apache.spark.unsafe.types.UTF8String
-import org.apache.spark.util.PartitionUtils
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.converter.SparkDataTypeConverterImpl
-import org.apache.carbondata.core.datastore.block.{Distributable, SegmentProperties, TaskBlockInfo}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.metadata.schema.table.column.{CarbonDimension, CarbonMeasure}
-import org.apache.carbondata.core.scan.result.iterator.PartitionSpliterRawResultIterator
-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.{CarbonInputFormat, CarbonTableInputFormat}
-import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
-import org.apache.carbondata.processing.merger.CarbonCompactionUtil
-import org.apache.carbondata.processing.partition.spliter.CarbonSplitExecutor
-import org.apache.carbondata.processing.util.CarbonLoaderUtil
-
-
-/**
- * This RDD is used in alter table partition statement to get data of target partitions,
- * then repartition data according to new partitionInfo
- * @param alterPartitionModel
- * @param absoluteTableIdentifier
- * @param partitionIds the ids of target partition to be scanned
- * @param bucketId
- */
-class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
- absoluteTableIdentifier: AbsoluteTableIdentifier,
- partitionIds: Seq[String],
- bucketId: Int)
- extends CarbonRDD[(AnyRef, Array[AnyRef])](alterPartitionModel.sqlContext.sparkSession, Nil) {
-
- private val queryId = alterPartitionModel.sqlContext.sparkContext.getConf
- .get("queryId", System.nanoTime() + "")
- val segmentId = alterPartitionModel.segmentId
- val carbonLoadModel = alterPartitionModel.carbonLoadModel
- val oldPartitionIdList = alterPartitionModel.oldPartitionIds
- var storeLocation: String = null
- var splitStatus: Boolean = false
- var blockId: String = null
- val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val dimensions = carbonTable.getAllDimensions.asScala
- val measures = carbonTable.getAllMeasures.asScala
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionColumn = partitionInfo.getColumnSchemaList().get(0)
- val partitionDataType = partitionColumn.getDataType
- val partitionColumnName = partitionColumn.getColumnName
- var isDimension: Boolean = false
- val encodingList = partitionColumn.getEncodingList
- var dimension: CarbonDimension = null
- var measure: CarbonMeasure = null
- val noDictionaryIndexGroup: ArrayBuffer[Int] = new ArrayBuffer[Int]()
- val dictionaryIndexGroup: ArrayBuffer[Int] = new ArrayBuffer[Int]()
- val measureIndexGroup: ArrayBuffer[Int] = new ArrayBuffer[Int]()
-
- override def internalGetPartitions: Array[Partition] = {
- val parallelism = sparkContext.defaultParallelism
- val jobConf = new JobConf(FileFactory.getConfiguration)
- val job = new Job(jobConf)
- val format = CarbonInputFormatUtil.createCarbonTableInputFormat(absoluteTableIdentifier,
- partitionIds.toList.asJava, job)
- CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
- job.getConfiguration.set("query.id", queryId)
-
- val splits = format.getSplitsOfOneSegment(job, segmentId,
- oldPartitionIdList.map(_.asInstanceOf[Integer]).asJava, partitionInfo)
- var partition_num = 0
- val result = new ArrayList[Partition](parallelism)
- val blockList = splits.asScala
- .filter(_.asInstanceOf[CarbonInputSplit].getBucketId.toInt == bucketId)
- .map(_.asInstanceOf[Distributable])
- if (!blockList.isEmpty) {
- val activeNodes = DistributionUtil.ensureExecutorsAndGetNodeList(blockList, sparkContext)
- val nodeBlockMapping = CarbonLoaderUtil.nodeBlockTaskMapping(blockList.asJava, -1,
- parallelism, activeNodes.toList.asJava)
- nodeBlockMapping.asScala.foreach { case (node, blockList) =>
- blockList.asScala.foreach { blocksPerTask =>
- val splits = blocksPerTask.asScala.map(_.asInstanceOf[CarbonInputSplit])
- if (blocksPerTask.size() != 0) {
- val multiBlockSplit =
- new CarbonMultiBlockSplit(splits.asJava, Array(node))
- val partition = new CarbonSparkPartition(id, partition_num, multiBlockSplit)
- result.add(partition)
- partition_num += 1
- }
- }
- }
- }
- result.toArray(new Array[Partition](result.size()))
- }
-
- override def internalCompute(split: Partition, context: TaskContext):
- Iterator[(AnyRef, Array[AnyRef])] = {
- val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
- var exec : CarbonSplitExecutor = null
- val rows : java.util.List[(AnyRef, Array[AnyRef])] = new ArrayList[(AnyRef, Array[AnyRef])]()
- try {
- val inputSplit = split.asInstanceOf[CarbonSparkPartition].split.value
- val splits = inputSplit.getAllSplits.asScala
- val tableBlockInfoList = CarbonInputSplit.createBlocks(splits.asJava)
- val segmentMapping: java.util.Map[String, TaskBlockInfo] =
- CarbonCompactionUtil.createMappingForSegments(tableBlockInfoList)
- val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- var result : java.util.List[PartitionSpliterRawResultIterator] = null
- try {
- exec = new CarbonSplitExecutor(segmentMapping, carbonTable)
- result = exec.processDataBlocks(segmentId, new SparkDataTypeConverterImpl(),
- FileFactory.getConfiguration)
- } catch {
- case e: Throwable =>
- LOGGER.error(e)
- if (null != e.getMessage) {
- CarbonException.analysisException(
- s"Exception occurred in query execution :: ${e.getMessage}")
- } else {
- CarbonException.analysisException(
- "Exception occurred in query execution. Please check logs.")
- }
- }
- val segmentProperties = PartitionUtils.getSegmentProperties(absoluteTableIdentifier,
- segmentId, partitionIds.toList, oldPartitionIdList, partitionInfo, carbonTable)
- val partColIdx = getPartitionColumnIndex(partitionColumnName, segmentProperties)
- indexInitialise()
- for (iterator <- result.asScala) {
- while (iterator.hasNext) {
- val row = iterator.next()
- val partitionColumnValue = getPartitionColumnValue(row, partColIdx,
- segmentProperties)
- rows.add((partitionColumnValue, row))
- }
- }
- } catch {
- case e: Exception =>
- LOGGER.error(e)
- throw e
- } finally {
- if (null != exec) {
- exec.finish
- }
- }
- val iter = rows.iterator().asScala
- iter
- }
-
- def getPartitionColumnValue(row: Array[AnyRef], partColIdx: Int,
- segmentProperties: SegmentProperties): AnyRef = {
- val dims: Array[Byte] = row(0).asInstanceOf[ByteArrayWrapper].getDictionaryKey
- val keyGen = segmentProperties.getDimensionKeyGenerator
- val keyArray: Array[Long] = keyGen.getKeyArray(dims)
- val encodings = partitionColumn.getEncodingList
- val partitionType = partitionInfo.getPartitionType
- var partitionValue: AnyRef = null
- val factor = 1000L
- if (isDimension) {
- // direct dictionary
- if (encodings.contains(Encoding.DIRECT_DICTIONARY)) {
- val directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
- .getDirectDictionaryGenerator(partitionDataType)
- val dictionaryIndex = dictionaryIndexGroup.indexOf(partColIdx)
- val surrogateValue = (keyArray(dictionaryIndex) / factor).toInt
- partitionValue = directDictionaryGenerator.getValueFromSurrogate(surrogateValue)
- } else if (!encodings.contains(Encoding.DICTIONARY)) {
- // no dictionary
- val byteArray = row(0).asInstanceOf[ByteArrayWrapper].getNoDictionaryKeys
- val index = noDictionaryIndexGroup.indexOf(partColIdx)
- partitionValue = DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(byteArray(index)
- , partitionDataType)
- if (partitionValue.isInstanceOf[UTF8String]) {
- partitionValue = partitionValue.toString
- }
- } else { // normal dictionary
- val dict = CarbonLoaderUtil.getDictionary(absoluteTableIdentifier,
- dimension.getColumnIdentifier, partitionDataType)
- if (partitionDataType == DataTypes.STRING) {
- if (partitionType == PartitionType.RANGE) {
- partitionValue = ByteUtil.
- toBytes(dict.getDictionaryValueForKey(keyArray(partColIdx).toInt))
- } else {
- partitionValue = dict.getDictionaryValueForKey(keyArray(partColIdx).toInt)
- }
- } else {
- partitionValue = dict.getDictionaryValueForKey(keyArray(partColIdx).toInt)
- }
-
- }
- } else {
- partitionValue = row(measureIndexGroup(partColIdx))
- }
- partitionValue
- }
-
- def indexInitialise(): Unit = {
- for (dim: CarbonDimension <- dimensions) {
- if (!dim.getEncoder.contains(Encoding.DICTIONARY)) {
- noDictionaryIndexGroup.append(dimensions.indexOf(dim))
- } else {
- dictionaryIndexGroup.append(dimensions.indexOf(dim))
- }
- }
- for (msr: CarbonMeasure <- measures) {
- // index of measure in row
- measureIndexGroup.append(measures.indexOf(msr) + 1)
- }
- }
-
- /**
- * get the index of partition column in dimension/measure
- * @param partitionColumnName
- * @param segmentProperties
- * @return
- */
- def getPartitionColumnIndex(partitionColumnName: String,
- segmentProperties: SegmentProperties): Int = {
- val dimensions = segmentProperties.getDimensions
- val measures = segmentProperties.getMeasures
- val columns = dimensions.asScala.map(_.getColName) ++ measures.asScala.map(_.getColName)
- var index = 0
- for (i <- 0 until columns.size) {
- if (columns(i) == partitionColumnName) {
- index = i
- }
- }
- if (index < dimensions.size()) {
- isDimension = true
- dimension = dimensions.get(index)
- } else {
- index = index - dimensions.size()
- measure = measures.get(index)
- }
- index
- }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonSparkPartition.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonSparkPartition.scala
index b473d35..563ab25 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonSparkPartition.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonSparkPartition.scala
@@ -26,7 +26,6 @@ class CarbonSparkPartition(
val rddId: Int,
val idx: Int,
@transient val multiBlockSplit: CarbonMultiBlockSplit,
- val partitionId: Int = 0,
val partitionSpec: Option[PartitionSpec] = None)
extends Partition {
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 8861863..7511502 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -463,13 +463,12 @@ class PartitionTableDataLoaderRDD[K, V](
val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
val model: CarbonLoadModel = carbonLoadModel
val carbonTable = model.getCarbonDataLoadSchema.getCarbonTable
- val partitionInfo = carbonTable.getPartitionInfo()
val uniqueLoadStatusId =
carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE + theSplit.index
try {
loadMetadataDetails.setSegmentStatus(SegmentStatus.SUCCESS)
- carbonLoadModel.setTaskNo(String.valueOf(partitionInfo.getPartitionId(theSplit.index)))
+ carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
carbonLoadModel.setPreFetch(false)
val recordReaders = Array[CarbonIterator[Array[AnyRef]]] {
new NewRddIterator(firstParent[Row].iterator(theSplit, context), carbonLoadModel, context)
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
deleted file mode 100644
index bc90b89..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
+++ /dev/null
@@ -1,118 +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.spark.rdd
-
-import java.io.IOException
-
-import org.apache.spark.sql.execution.command.{AlterPartitionModel, DropPartitionCallableModel}
-import org.apache.spark.util.PartitionUtils
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.spark.{AlterPartitionResultImpl, PartitionFactory}
-
-object PartitionDropper {
-
- val logger = LogServiceFactory.getLogService(PartitionDropper.getClass.getName)
-
- def triggerPartitionDrop(dropPartitionCallableModel: DropPartitionCallableModel): Unit = {
- val alterPartitionModel = new AlterPartitionModel(dropPartitionCallableModel.carbonLoadModel,
- dropPartitionCallableModel.segmentId.getSegmentNo,
- dropPartitionCallableModel.oldPartitionIds,
- dropPartitionCallableModel.sqlContext
- )
- val partitionId = dropPartitionCallableModel.partitionId
- val oldPartitionIds = dropPartitionCallableModel.oldPartitionIds
- val dropWithData = dropPartitionCallableModel.dropWithData
- val carbonTable = dropPartitionCallableModel.carbonTable
- val dbName = carbonTable.getDatabaseName
- val tableName = carbonTable.getTableName
- val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitioner = PartitionFactory.getPartitioner(partitionInfo)
-
- var finalDropStatus = false
- val bucketInfo = carbonTable.getBucketingInfo()
- val bucketNumber = bucketInfo match {
- case null => 1
- case _ => bucketInfo.getNumOfRanges
- }
- val partitionIndex = oldPartitionIds.indexOf(Integer.valueOf(partitionId))
- val targetPartitionId = partitionInfo.getPartitionType match {
- case PartitionType.RANGE => if (partitionIndex == oldPartitionIds.length - 1) {
- "0"
- } else {
- String.valueOf(oldPartitionIds(partitionIndex + 1))
- }
- case PartitionType.LIST => "0"
- case _ => throw new UnsupportedOperationException(
- s"${partitionInfo.getPartitionType} is not supported")
- }
-
- if (!dropWithData) {
- try {
- for (i <- 0 until bucketNumber) {
- val bucketId = i
- val rdd = new CarbonScanPartitionRDD(alterPartitionModel,
- absoluteTableIdentifier,
- Seq(partitionId, targetPartitionId),
- bucketId
- ).partitionBy(partitioner).map(_._2)
-
- val dropStatus = new AlterTableLoadPartitionRDD(alterPartitionModel,
- new AlterPartitionResultImpl(),
- Seq(partitionId),
- bucketId,
- absoluteTableIdentifier,
- rdd).collect()
-
- if (dropStatus.length == 0) {
- finalDropStatus = false
- } else {
- finalDropStatus = dropStatus.forall(_._2)
- }
- if (!finalDropStatus) {
- logger.error(s"Drop Partition request failed for table " +
- s"${ dbName }.${ tableName }")
- }
- }
-
- if (finalDropStatus) {
- try {
- PartitionUtils.deleteOriginalCarbonFile(alterPartitionModel, absoluteTableIdentifier,
- Seq(partitionId, targetPartitionId).toList, dbName,
- tableName, partitionInfo)
- } catch {
- case e: IOException =>
- throw new IOException("Exception while delete original carbon files ", e)
- }
- logger.info(s"Drop Partition request completed for table " +
- s"${ dbName }.${ tableName }")
- }
- } catch {
- case e: Exception =>
- throw new RuntimeException("Exception in dropping partition action", e)
- }
- } else {
- PartitionUtils.deleteOriginalCarbonFile(alterPartitionModel, absoluteTableIdentifier,
- Seq(partitionId).toList, dbName, tableName, partitionInfo)
- logger.info(s"Drop Partition request completed for table " +
- s"${ dbName }.${ tableName }")
- }
- }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
deleted file mode 100644
index bd46c24..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
+++ /dev/null
@@ -1,93 +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.spark.rdd
-
-import java.io.IOException
-
-import org.apache.spark.sql.execution.command.{AlterPartitionModel, SplitPartitionCallableModel}
-import org.apache.spark.util.PartitionUtils
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.spark.{AlterPartitionResultImpl, PartitionFactory}
-
-object PartitionSplitter {
-
- val logger = LogServiceFactory.getLogService(PartitionSplitter.getClass.getName)
-
- def triggerPartitionSplit(splitPartitionCallableModel: SplitPartitionCallableModel): Unit = {
-
- val alterPartitionModel = new AlterPartitionModel(splitPartitionCallableModel.carbonLoadModel,
- splitPartitionCallableModel.segmentId,
- splitPartitionCallableModel.oldPartitionIds,
- splitPartitionCallableModel.sqlContext
- )
- val partitionId = splitPartitionCallableModel.partitionId
- val carbonLoadModel = splitPartitionCallableModel.carbonLoadModel
- val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
- val tableName = carbonTable.getTableName
- val databaseName = carbonTable.getDatabaseName
- val bucketInfo = carbonTable.getBucketingInfo()
- var finalSplitStatus = false
- val bucketNumber = bucketInfo match {
- case null => 1
- case _ => bucketInfo.getNumOfRanges
- }
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitioner = PartitionFactory.getPartitioner(partitionInfo)
-
- for (i <- 0 until bucketNumber) {
- val bucketId = i
- val rdd = new CarbonScanPartitionRDD(
- alterPartitionModel,
- absoluteTableIdentifier,
- Seq(partitionId),
- bucketId
- ).partitionBy(partitioner).map(_._2)
-
- val splitStatus = new AlterTableLoadPartitionRDD(alterPartitionModel,
- new AlterPartitionResultImpl(),
- Seq(partitionId),
- bucketId,
- absoluteTableIdentifier,
- rdd).collect()
-
- if (splitStatus.length == 0) {
- finalSplitStatus = false
- } else {
- finalSplitStatus = splitStatus.forall(_._2)
- }
- if (!finalSplitStatus) {
- logger.error(s"Add/Split Partition request failed for table " +
- s"${ databaseName }.${ tableName }")
- }
- }
- if (finalSplitStatus) {
- try {
- PartitionUtils.
- deleteOriginalCarbonFile(alterPartitionModel, absoluteTableIdentifier,
- Seq(partitionId).toList, databaseName, tableName, partitionInfo)
- } catch {
- case e: IOException =>
- throw new IOException("Exception while delete original carbon files ", e)
- }
- logger.info(s"Add/Split Partition request completed for table " +
- s"${ databaseName }.${ tableName }")
- }
- }
-}
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index 5f9f9bb..fd0cf74 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -583,9 +583,9 @@ object CarbonScalaUtil {
.isDefined
if (isLocalDictIncludeDefined && isLocalDictExcludeDefined) {
val localDictIncludeCols = tableProperties(CarbonCommonConstants.LOCAL_DICTIONARY_INCLUDE)
- .split(",").map(_.trim)
+ .split(",").map(_.trim.toLowerCase)
val localDictExcludeCols = tableProperties(CarbonCommonConstants.LOCAL_DICTIONARY_EXCLUDE)
- .split(",").map(_.trim)
+ .split(",").map(_.trim.toLowerCase)
localDictIncludeCols.foreach { distCol =>
if (localDictExcludeCols.exists(x => x.equalsIgnoreCase(distCol.trim))) {
val duplicateColumns = (localDictIncludeCols ++ localDictExcludeCols)
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 b16c434..787b591 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
@@ -31,11 +31,8 @@ import scala.math.BigDecimal.RoundingMode
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
import org.apache.spark.{SparkContext, SparkEnv}
-import org.apache.spark.sql.{Row, RowFactory}
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.execution.command.{ColumnProperty, Field, PartitionerField}
-import org.apache.spark.sql.types.{MetadataBuilder, StringType}
import org.apache.spark.util.FileUtils
import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
@@ -44,14 +41,9 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
import org.apache.carbondata.core.datastore.impl.FileFactory
import org.apache.carbondata.core.memory.{UnsafeMemoryManager, UnsafeSortMemoryManager}
import org.apache.carbondata.core.metadata.CarbonMetadata
-import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes}
-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.scan.partition.PartitionUtil
import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties, CarbonUtil, ThreadLocalTaskInfo}
-import org.apache.carbondata.core.util.comparator.Comparator
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, ThreadLocalTaskInfo}
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -108,56 +100,6 @@ object CommonUtil {
isValid
}
- /**
- * 1. If partitioned by clause exists, then partition_type should be defined
- * 2. If partition_type is Hash, then num_partitions should be defined
- * 3. If partition_type is List, then list_info should be defined
- * 4. If partition_type is Range, then range_info should be defined
- * 5. Only support single level partition for now
- * @param tableProperties
- * @param partitionerFields
- * @return partition clause and definition in tblproperties are valid or not
- */
- def validatePartitionColumns(tableProperties: Map[String, String],
- partitionerFields: Seq[PartitionerField]): Boolean = {
- var isValid: Boolean = true
- val partitionType = tableProperties.get(CarbonCommonConstants.PARTITION_TYPE)
- val numPartitions = tableProperties.get(CarbonCommonConstants.NUM_PARTITIONS)
- val rangeInfo = tableProperties.get(CarbonCommonConstants.RANGE_INFO)
- val listInfo = tableProperties.get(CarbonCommonConstants.LIST_INFO)
-
- if (partitionType.isEmpty) {
- isValid = true
- } else {
- partitionType.get.toUpperCase() match {
- case "HASH" => if (!numPartitions.isDefined
- || scala.util.Try(numPartitions.get.toInt).isFailure
- || numPartitions.get.toInt <= 0) {
- isValid = false
- }
- case "LIST" => if (!listInfo.isDefined) {
- isValid = false
- } else {
- listInfo.get.replace("(", "").replace(")", "").split(",").map(_.trim).foreach(
- isValid &= validateTypeConvert(partitionerFields(0), _))
- }
- case "RANGE" => if (!rangeInfo.isDefined) {
- isValid = false
- } else {
- rangeInfo.get.split(",").map(_.trim).foreach(
- isValid &= validateTypeConvert(partitionerFields(0), _))
- }
- case "RANGE_INTERVAL" => isValid = false
- case _ => isValid = true
- }
- // only support one partition column for now
- if (partitionerFields.length > 1 && !partitionType.get.toUpperCase.equals("NATIVE_HIVE")) {
- isValid = false
- }
- }
- isValid
- }
-
def validateTypeConvertForSpark2(partitionerField: PartitionerField, value: String): Boolean = {
val result = partitionerField.dataType.get.toLowerCase match {
case "integertype" =>
@@ -259,86 +201,6 @@ object CommonUtil {
}
}
- /**
- * To verify the range info is in correct order
- * @param rangeInfo
- * @param columnDataType
- * @param timestampFormatter
- * @param dateFormatter
- */
- def validateRangeInfo(rangeInfo: List[String], columnDataType: DataType,
- timestampFormatter: SimpleDateFormat, dateFormatter: SimpleDateFormat): Unit = {
- if (rangeInfo.size <= 1) {
- throw new
- MalformedCarbonCommandException("Range info must define a valid range.Please check again!")
- }
- val comparator = Comparator.getComparator(columnDataType)
- var head = columnDataType match {
- case DataTypes.STRING => ByteUtil.toBytes(rangeInfo.head)
- case _ => PartitionUtil.getDataBasedOnDataType(rangeInfo.head, columnDataType,
- timestampFormatter, dateFormatter)
- }
- val iterator = rangeInfo.tail.toIterator
- while (iterator.hasNext) {
- val next = columnDataType match {
- case DataTypes.STRING => ByteUtil.toBytes(iterator.next())
- case _ => PartitionUtil.getDataBasedOnDataType(iterator.next(), columnDataType,
- timestampFormatter, dateFormatter)
- }
- if (next == null) {
- sys.error(
- "Data in range info must be the same type with the partition field's type "
- + columnDataType)
- }
- if (comparator.compare(head, next) < 0) {
- head = next
- } else {
- sys.error("Range info must be in ascending order, please check again!")
- }
- }
- }
-
- def validateSplitListInfo(originListInfo: List[String], newListInfo: List[String],
- originList: List[List[String]]): Unit = {
- if (originListInfo.size == 1) {
- sys.error("The target list partition cannot be split, please check again!")
- }
- if (newListInfo.size == 1) {
- sys.error("Can't split list to one partition, please check again!")
- }
- if (!(newListInfo.size < originListInfo.size)) {
- sys.error("The size of new list must be smaller than original list, please check again!")
- }
- val tempList = newListInfo.mkString(",").split(",")
- .map(_.replace("(", "").replace(")", "").trim)
- if (tempList.length != originListInfo.size) {
- sys.error("The total number of elements in new list must equal to original list!")
- }
- if (!(tempList diff originListInfo).isEmpty) {
- sys.error("The elements in new list must exist in original list")
- }
- }
-
- def validateAddListInfo(newListInfo: List[String], originList: List[List[String]]): Unit = {
- if (newListInfo.size < 1) {
- sys.error("Please add at least one new partition")
- }
- for (originElementGroup <- originList) {
- for (newElement <- newListInfo ) {
- if (originElementGroup.contains(newElement)) {
- sys.error(s"The partition $newElement is already exist! Please check again!")
- }
- }
- }
- }
-
- def validateListInfo(listInfo: List[List[String]]): Unit = {
- val list = listInfo.flatten
- if (list.distinct.size != list.size) {
- sys.error("Duplicate elements defined in LIST_INFO!")
- }
- }
-
def validateFields(key: String, fields: Seq[Field]): Boolean = {
var isValid: Boolean = false
fields.foreach { field =>
@@ -635,52 +497,6 @@ object CommonUtil {
}
}
- def getPartitionInfo(columnName: String, partitionType: PartitionType,
- partitionInfo: PartitionInfo): Seq[Row] = {
- var result = Seq.newBuilder[Row]
- partitionType match {
- case PartitionType.RANGE =>
- result.+=(RowFactory.create("0" + ", " + columnName + " = DEFAULT"))
- val rangeInfo = partitionInfo.getRangeInfo
- val size = rangeInfo.size() - 1
- for (index <- 0 to size) {
- if (index == 0) {
- val id = partitionInfo.getPartitionId(index + 1).toString
- val desc = columnName + " < " + rangeInfo.get(index)
- result.+=(RowFactory.create(id + ", " + desc))
- } else {
- val id = partitionInfo.getPartitionId(index + 1).toString
- val desc = rangeInfo.get(index - 1) + " <= " + columnName + " < " + rangeInfo.get(index)
- result.+=(RowFactory.create(id + ", " + desc))
- }
- }
- case PartitionType.RANGE_INTERVAL =>
- result.+=(RowFactory.create(columnName + " = "))
- case PartitionType.LIST =>
- result.+=(RowFactory.create("0" + ", " + columnName + " = DEFAULT"))
- val listInfo = partitionInfo.getListInfo
- listInfo.asScala.foreach {
- f =>
- val id = partitionInfo.getPartitionId(listInfo.indexOf(f) + 1).toString
- val desc = columnName + " = " + f.toArray().mkString(", ")
- result.+=(RowFactory.create(id + ", " + desc))
- }
- case PartitionType.HASH =>
- val hashNumber = partitionInfo.getNumPartitions
- result.+=(RowFactory.create(columnName + " = HASH_NUMBER(" + hashNumber.toString() + ")"))
- result.+=(RowFactory.create(s"partitionIds = ${partitionInfo.getPartitionIds}"))
- case others =>
- result.+=(RowFactory.create(columnName + " = "))
- }
- val rows = result.result()
- rows
- }
-
- def partitionInfoOutput: Seq[Attribute] = Seq(
- AttributeReference("Partition(Id, DESC)", StringType, false,
- new MetadataBuilder().putString("comment", "partition").build())()
- )
-
/**
* Method to clear the memory for a task
* if present
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index 90dfc7c..d21e6e5 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -414,7 +414,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
}
// get partitionInfo
- val partitionInfo = getPartitionInfo(partitionCols, tableProperties)
+ val partitionInfo = getPartitionInfo(partitionCols)
if (tableProperties.get(CarbonCommonConstants.COLUMN_META_CACHE).isDefined) {
// validate the column_meta_cache option
val tableColumns = dims.map(x => x.name.get) ++ msrs.map(x => x.name.get)
@@ -550,7 +550,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
throw new MalformedCarbonCommandException(errMsg)
}
- if (!dataTypeErr.isEmpty) {
+ if (dataTypeErr.nonEmpty) {
val errMsg = s"long_string_columns: ${
dataTypeErr.mkString(",")
} ,its data type is not string. Please check the create table statement."
@@ -558,45 +558,10 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
}
}
- /**
- * @param partitionCols
- * @param tableProperties
- */
- protected def getPartitionInfo(partitionCols: Seq[PartitionerField],
- tableProperties: Map[String, String]): Option[PartitionInfo] = {
- val timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance
- .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
- CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
- val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
- .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
- CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+ protected def getPartitionInfo(partitionCols: Seq[PartitionerField]): Option[PartitionInfo] = {
if (partitionCols.isEmpty) {
None
} else {
- var partitionType: String = ""
- var numPartitions = 0
- var rangeInfo = List[String]()
- var listInfo = List[List[String]]()
-
- val columnDataType = DataTypeConverterUtil.
- convertToCarbonType(partitionCols.head.dataType.get)
- if (tableProperties.get(CarbonCommonConstants.PARTITION_TYPE).isDefined) {
- partitionType = tableProperties(CarbonCommonConstants.PARTITION_TYPE)
- }
- if (tableProperties.get(CarbonCommonConstants.NUM_PARTITIONS).isDefined) {
- numPartitions = tableProperties(CarbonCommonConstants.NUM_PARTITIONS)
- .toInt
- }
- if (tableProperties.get(CarbonCommonConstants.RANGE_INFO).isDefined) {
- rangeInfo = tableProperties(CarbonCommonConstants.RANGE_INFO).split(",")
- .map(_.trim()).toList
- CommonUtil.validateRangeInfo(rangeInfo, columnDataType, timestampFormatter, dateFormatter)
- }
- if (tableProperties.get(CarbonCommonConstants.LIST_INFO).isDefined) {
- val originListInfo = tableProperties(CarbonCommonConstants.LIST_INFO)
- listInfo = PartitionUtils.getListInfo(originListInfo)
- CommonUtil.validateListInfo(listInfo)
- }
val cols : ArrayBuffer[ColumnSchema] = new ArrayBuffer[ColumnSchema]()
partitionCols.foreach(partition_col => {
val columnSchema = new ColumnSchema
@@ -605,21 +570,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
columnSchema.setColumnName(partition_col.partitionColumn)
cols += columnSchema
})
-
- var partitionInfo : PartitionInfo = null
- partitionType.toUpperCase() match {
- case "HASH" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.HASH)
- partitionInfo.initialize(numPartitions)
- case "RANGE" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.RANGE)
- partitionInfo.setRangeInfo(rangeInfo.asJava)
- partitionInfo.initialize(rangeInfo.size + 1)
- case "LIST" => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.LIST)
- partitionInfo.setListInfo(listInfo.map(_.asJava).asJava)
- partitionInfo.initialize(listInfo.size + 1)
- case _ => partitionInfo = new PartitionInfo(cols.asJava, PartitionType.NATIVE_HIVE)
- partitionInfo.setListInfo(listInfo.map(_.asJava).asJava)
- }
- Some(partitionInfo)
+ Some(new PartitionInfo(cols.asJava, PartitionType.NATIVE_HIVE))
}
}
@@ -759,8 +710,9 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
// All long_string cols should be there in create table cols and should be of string data type
if (tableProperties.get(CarbonCommonConstants.LONG_STRING_COLUMNS).isDefined) {
- varcharCols =
- tableProperties(CarbonCommonConstants.LONG_STRING_COLUMNS).split(",").map(_.trim)
+ varcharCols = tableProperties(CarbonCommonConstants.LONG_STRING_COLUMNS)
+ .split(",")
+ .map(_.trim.toLowerCase)
validateLongStringColumns(fields, varcharCols)
}
@@ -1088,12 +1040,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
case Token("TOK_TABLEPROPLIST", list) =>
list.map {
case Token("TOK_TABLEPROPERTY", Token(key, Nil) :: Token(value, Nil) :: Nil) =>
- val reslovedKey = unquoteString(key)
- if (needToConvertToLowerCase(reslovedKey)) {
- (reslovedKey, unquoteString(value))
- } else {
- (reslovedKey, unquoteStringWithoutLowerConversion(value))
- }
+ (unquoteString(key), unquoteStringWithoutLowerConversion(value))
}
}
}
@@ -1114,11 +1061,6 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
}
}
- private def needToConvertToLowerCase(key: String): Boolean = {
- val noConvertList = Array("LIST_INFO", "RANGE_INFO")
- !noConvertList.exists(x => x.equalsIgnoreCase(key))
- }
-
protected def validateOptions(optionList: Option[List[(String, String)]]): Unit = {
// validate with all supported options
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 1782cdf..61eeaf9 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
@@ -14,231 +14,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.util
-import java.io.{File, IOException}
-import java.text.SimpleDateFormat
-import java.util
+package org.apache.spark.util
-import scala.collection.JavaConverters._
import scala.collection.mutable
-import scala.collection.mutable.ListBuffer
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.Job
-import org.apache.spark.sql.execution.command.{AlterPartitionModel, DataMapField, Field, PartitionerField}
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.Segment
-import org.apache.carbondata.core.datastore.block.{SegmentProperties, TableBlockInfo}
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, SegmentFileStore}
-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.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.readcommitter.TableStatusReadCommittedScope
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-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.{CarbonInputFormat, CarbonTableInputFormat}
-import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.spark.util.CommonUtil
+import org.apache.spark.sql.execution.command.{DataMapField, Field, PartitionerField}
object PartitionUtils {
- def getListInfo(originListInfo: String): List[List[String]] = {
- var listInfo = ListBuffer[List[String]]()
- var templist = ListBuffer[String]()
- val arr = originListInfo.split(",")
- .map(_.trim())
- var groupEnd = true
- val iter = arr.iterator
- while (iter.hasNext) {
- val value = iter.next()
- if (value.startsWith("(")) {
- templist += value.replace("(", "").trim()
- groupEnd = false
- } else if (value.endsWith(")")) {
- templist += value.replace(")", "").trim()
- listInfo += templist.toList
- templist.clear()
- groupEnd = true
- } else {
- if (groupEnd) {
- templist += value
- listInfo += templist.toList
- templist.clear()
- } else {
- templist += value
- }
- }
- }
- listInfo.toList
- }
-
- /**
- * verify the add/split information and update the partitionInfo:
- * 1. update rangeInfo/listInfo
- * 2. update partitionIds
- */
- def updatePartitionInfo(partitionInfo: PartitionInfo, partitionIdList: List[Int],
- partitionId: Int, splitInfo: List[String], timestampFormatter: SimpleDateFormat,
- dateFormatter: SimpleDateFormat): Unit = {
- val columnDataType = partitionInfo.getColumnSchemaList.get(0).getDataType
- val index = partitionIdList.indexOf(partitionId)
- if (index < 0) {
- throw new IllegalArgumentException("Invalid Partition Id " + partitionId +
- "\n Use show partitions table_name to get the list of valid partitions")
- }
- if (partitionInfo.getPartitionType == PartitionType.RANGE) {
- val rangeInfo = partitionInfo.getRangeInfo.asScala.toList
- val newRangeInfo = partitionId match {
- case 0 => rangeInfo ++ splitInfo
- case _ => rangeInfo.take(index - 1) ++ splitInfo ++
- rangeInfo.takeRight(rangeInfo.size - index)
- }
- CommonUtil.validateRangeInfo(newRangeInfo, columnDataType,
- timestampFormatter, dateFormatter)
- partitionInfo.setRangeInfo(newRangeInfo.asJava)
- } else if (partitionInfo.getPartitionType == PartitionType.LIST) {
- val originList = partitionInfo.getListInfo.asScala.map(_.asScala.toList).toList
- if (partitionId != 0) {
- val targetListInfo = partitionInfo.getListInfo.get(index - 1)
- CommonUtil.validateSplitListInfo(targetListInfo.asScala.toList, splitInfo, originList)
- } else {
- CommonUtil.validateAddListInfo(splitInfo, originList)
- }
- val addListInfo = PartitionUtils.getListInfo(splitInfo.mkString(","))
- val newListInfo = partitionId match {
- case 0 => originList ++ addListInfo
- case _ => originList.take(index - 1) ++ addListInfo ++
- originList.takeRight(originList.size - index)
- }
- partitionInfo.setListInfo(newListInfo.map(_.asJava).asJava)
- }
-
- if (partitionId == 0) {
- partitionInfo.addPartition(splitInfo.size)
- } else {
- partitionInfo.splitPartition(index, splitInfo.size)
- }
- }
-
- /**
- * Used for alter table partition commands to get segmentProperties in spark node
- * @param identifier
- * @param segmentId
- * @param oldPartitionIdList Task id group before partition info is changed
- * @return
- */
- def getSegmentProperties(identifier: AbsoluteTableIdentifier, segmentId: String,
- partitionIds: List[String], oldPartitionIdList: List[Int],
- partitionInfo: PartitionInfo,
- carbonTable: CarbonTable): SegmentProperties = {
- val tableBlockInfoList =
- getPartitionBlockList(
- identifier,
- segmentId,
- partitionIds,
- oldPartitionIdList,
- partitionInfo,
- carbonTable)
- val footer = CarbonUtil.readMetadataFile(tableBlockInfoList.get(0))
- val segmentProperties = new SegmentProperties(footer.getColumnInTable,
- footer.getSegmentInfo.getColumnCardinality)
- segmentProperties
- }
-
- def getPartitionBlockList(identifier: AbsoluteTableIdentifier, segmentId: String,
- partitionIds: List[String], oldPartitionIdList: List[Int],
- partitionInfo: PartitionInfo,
- carbonTable: CarbonTable): java.util.List[TableBlockInfo] = {
- val jobConf = new JobConf(FileFactory.getConfiguration)
- val job = new Job(jobConf)
- val format = CarbonInputFormatUtil
- .createCarbonTableInputFormat(identifier, partitionIds.asJava, job)
- 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])
- val tableBlockInfoList = CarbonInputSplit.createBlocks(blockList.asJava)
- tableBlockInfoList
- }
-
- @throws(classOf[IOException])
- def deleteOriginalCarbonFile(alterPartitionModel: AlterPartitionModel,
- identifier: AbsoluteTableIdentifier,
- partitionIds: List[String], dbName: String, tableName: String,
- partitionInfo: PartitionInfo): Unit = {
- val carbonLoadModel = alterPartitionModel.carbonLoadModel
- val segmentId = alterPartitionModel.segmentId
- val oldPartitionIds = alterPartitionModel.oldPartitionIds
- val newTime = carbonLoadModel.getFactTimeStamp
- val tablePath = carbonLoadModel.getTablePath
- val tableBlockInfoList =
- getPartitionBlockList(identifier, segmentId, partitionIds, oldPartitionIds,
- partitionInfo, carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable).asScala
- val pathList: util.List[String] = new util.ArrayList[String]()
- tableBlockInfoList.foreach{ tableBlockInfo =>
- val path = tableBlockInfo.getFilePath
- val timestamp = CarbonTablePath.DataFileUtil.getTimeStampFromFileName(path)
- if (timestamp.toLong != newTime) {
- // add carbondata file
- pathList.add(path)
- // add index file
- val version = tableBlockInfo.getVersion
- val taskNo = CarbonTablePath.DataFileUtil.getTaskNo(path)
- val batchNo = CarbonTablePath.DataFileUtil.getBatchNoFromTaskNo(taskNo)
- val taskId = CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskNo)
- val bucketNumber = CarbonTablePath.DataFileUtil.getBucketNo(path)
- val indexFilePath =
- new Path(new Path(path).getParent,
- CarbonTablePath.getCarbonIndexFileName(taskId,
- bucketNumber.toInt,
- batchNo,
- timestamp,
- segmentId)).toString
- // indexFilePath could be duplicated when multiple data file related to one index file
- if (indexFilePath != null && !pathList.contains(indexFilePath)) {
- pathList.add(indexFilePath)
- }
- }
- }
- val files: util.List[File] = new util.ArrayList[File]()
- for (path <- pathList.asScala) {
- val file = new File(path)
- files.add(file)
- }
- CarbonUtil.deleteFiles(files.asScala.toArray)
- if (!files.isEmpty) {
- val carbonTable = alterPartitionModel.carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val updatedSegFile: String = mergeAndUpdateSegmentFile(alterPartitionModel,
- identifier,
- segmentId,
- carbonTable,
- files.asScala)
-
- val segmentFiles = Seq(new Segment(alterPartitionModel.segmentId, updatedSegFile, null))
- .asJava
- if (!CarbonUpdateUtil.updateTableMetadataStatus(
- new util.HashSet[Segment](Seq(new Segment(alterPartitionModel.segmentId,
- null, null)).asJava),
- carbonTable,
- alterPartitionModel.carbonLoadModel.getFactTimeStamp.toString,
- true,
- new util.ArrayList[Segment](0),
- new util.ArrayList[Segment](segmentFiles), "")) {
- throw new IOException("Data update failed due to failure in table status updation.")
- }
- }
- }
-
/**
* Used to extract PartitionerFields for aggregation datamaps.
* This method will keep generating partitionerFields until the sequence of
@@ -295,49 +79,4 @@ object PartitionUtils {
}
- private def mergeAndUpdateSegmentFile(alterPartitionModel: AlterPartitionModel,
- identifier: AbsoluteTableIdentifier,
- segmentId: String,
- carbonTable: CarbonTable, filesToBeDelete: Seq[File]) = {
- val metadataDetails =
- SegmentStatusManager.readTableStatusFile(
- CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath))
- val segmentFile =
- metadataDetails.find(_.getLoadName.equals(segmentId)).get.getSegmentFile
- var allSegmentFiles: Seq[CarbonFile] = Seq.empty[CarbonFile]
- val file = SegmentFileStore.writeSegmentFile(
- carbonTable,
- alterPartitionModel.segmentId,
- System.currentTimeMillis().toString)
- if (segmentFile != null) {
- allSegmentFiles ++= FileFactory.getCarbonFile(
- SegmentFileStore.getSegmentFilePath(carbonTable.getTablePath, segmentFile)) :: Nil
- }
- val updatedSegFile = {
- val carbonFile = FileFactory.getCarbonFile(
- SegmentFileStore.getSegmentFilePath(carbonTable.getTablePath, file))
- allSegmentFiles ++= carbonFile :: Nil
-
- val mergedSegFileName = SegmentFileStore.genSegmentFileName(
- segmentId,
- alterPartitionModel.carbonLoadModel.getFactTimeStamp.toString)
- val tmpFile = mergedSegFileName + "_tmp"
- val segmentStoreFile = SegmentFileStore.mergeSegmentFiles(
- tmpFile,
- CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath),
- allSegmentFiles.toArray)
- val indexFiles = segmentStoreFile.getLocationMap.values().asScala.head.getFiles
- filesToBeDelete.foreach(f => indexFiles.remove(f.getName))
- SegmentFileStore.writeSegmentFile(
- segmentStoreFile,
- CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath) +
- CarbonCommonConstants.FILE_SEPARATOR + mergedSegFileName + CarbonTablePath.SEGMENT_EXT)
- carbonFile.delete()
- FileFactory.getCarbonFile(
- SegmentFileStore.getSegmentFilePath(
- carbonTable.getTablePath, tmpFile + CarbonTablePath.SEGMENT_EXT)).delete()
- mergedSegFileName + CarbonTablePath.SEGMENT_EXT
- }
- updatedSegFile
- }
}
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 57973da..71c5a89 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -18,26 +18,22 @@
package org.apache.carbondata.spark.rdd
import java.io.File
-import java.text.SimpleDateFormat
import java.util
-import java.util.TimeZone
import java.util.concurrent._
import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
import scala.util.Random
-import scala.util.control.Breaks._
import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
-import org.apache.hadoop.io.NullWritable
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapreduce.Job
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
import org.apache.spark.{SparkEnv, TaskContext}
import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionCoalescer, NewHadoopRDD, RDD}
+import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionCoalescer}
import org.apache.spark.sql.{CarbonEnv, DataFrame, Row, SQLContext}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.execution.command.{CompactionModel, ExecutionErrors, UpdateTableModel}
@@ -57,27 +53,22 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
import org.apache.carbondata.core.dictionary.server.DictionaryServer
import org.apache.carbondata.core.exception.ConcurrentOperationException
import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnarFormatVersion, SegmentFileStore}
-import org.apache.carbondata.core.metadata.datatype.DataTypes
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnarFormatVersion, SegmentFileStore}
import org.apache.carbondata.core.metadata.schema.table.CarbonTable
import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.readcommitter.TableStatusReadCommittedScope
-import org.apache.carbondata.core.scan.partition.PartitionUtil
import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties, CarbonUtil, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, ThreadLocalSessionInfo}
import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.events.{IndexServerLoadEvent, OperationContext, OperationListenerBus}
+import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
import org.apache.carbondata.indexserver.{DistributedRDDUtils, IndexServer}
-import org.apache.carbondata.processing.exception.DataLoadingException
import org.apache.carbondata.processing.loading.FailureCauses
-import org.apache.carbondata.processing.loading.csvinput.{BlockDetails, CSVInputFormat, StringArrayWritable}
+import org.apache.carbondata.processing.loading.csvinput.BlockDetails
import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostStatusUpdateEvent, LoadTablePreStatusUpdateEvent}
import org.apache.carbondata.processing.loading.exception.NoRetryException
import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonDataMergerUtil, CompactionType}
-import org.apache.carbondata.processing.util.{Auditor, CarbonDataProcessorUtil, CarbonLoaderUtil}
-import org.apache.carbondata.spark.{DataLoadResultImpl, PartitionFactory, _}
+import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
+import org.apache.carbondata.spark.{DataLoadResultImpl, _}
import org.apache.carbondata.spark.load._
import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
@@ -369,12 +360,10 @@ object CarbonDataRDDFactory {
}
}
} else {
- status = if (carbonTable.getPartitionInfo() != null) {
- loadDataForPartitionTable(sqlContext, dataFrame, carbonLoadModel, hadoopConf)
- } else if (dataFrame.isEmpty && isSortTable &&
- carbonLoadModel.getRangePartitionColumn != null &&
- (sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT) ||
- sortScope.equals(SortScopeOptions.SortScope.LOCAL_SORT))) {
+ status = if (dataFrame.isEmpty && isSortTable &&
+ carbonLoadModel.getRangePartitionColumn != null &&
+ (sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT) ||
+ sortScope.equals(SortScopeOptions.SortScope.LOCAL_SORT))) {
DataLoadProcessBuilderOnSpark
.loadDataUsingRangeSort(sqlContext.sparkSession, carbonLoadModel, hadoopConf)
} else if (isSortTable && sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT)) {
@@ -987,141 +976,6 @@ object CarbonDataRDDFactory {
}
/**
- * repartition the input data for partition table.
- */
- private def repartitionInputData(
- sqlContext: SQLContext,
- dataFrame: Option[DataFrame],
- carbonLoadModel: CarbonLoadModel,
- hadoopConf: Configuration): RDD[Row] = {
- val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionColumn = partitionInfo.getColumnSchemaList.get(0).getColumnName
- val partitionColumnDataType = partitionInfo.getColumnSchemaList.get(0).getDataType
- val columns = carbonLoadModel.getCsvHeaderColumns
- var partitionColumnIndex = -1
- breakable {
- for (i <- 0 until columns.length) {
- if (partitionColumn.equalsIgnoreCase(columns(i))) {
- partitionColumnIndex = i
- break
- }
- }
- }
- if (partitionColumnIndex == -1) {
- throw new DataLoadingException("Partition column not found.")
- }
-
- val specificTimestampFormat = carbonLoadModel.getTimestampformat
- val specificDateFormat = carbonLoadModel.getDateFormat
- val timeStampFormat =
- if (specificTimestampFormat != null && !specificTimestampFormat.trim.isEmpty) {
- new SimpleDateFormat(specificTimestampFormat)
- } else {
- val timestampFormatString = CarbonProperties.getInstance().getProperty(
- CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
- CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
- new SimpleDateFormat(timestampFormatString)
- }
-
- val dateFormat = if (specificDateFormat != null && !specificDateFormat.trim.isEmpty) {
- new SimpleDateFormat(specificDateFormat)
- } else {
- val dateFormatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
- .CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
- new SimpleDateFormat(dateFormatString)
- }
- dateFormat.setTimeZone(TimeZone.getTimeZone("GMT"))
-
- // generate RDD[(K, V)] to use the partitionBy method of PairRDDFunctions
- val inputRDD: RDD[(String, Row)] = if (dataFrame.isDefined) {
- // input data from DataFrame
- val complexDelimiters = carbonLoadModel.getComplexDelimiters
- val serializationNullFormat =
- carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
- dataFrame.get.rdd.map { row =>
- if (null != row && row.length > partitionColumnIndex &&
- null != row.get(partitionColumnIndex)) {
- (CarbonScalaUtil.getString(row.get(partitionColumnIndex), serializationNullFormat,
- complexDelimiters, timeStampFormat, dateFormat), row)
- } else {
- (null, row)
- }
- }
- } else {
- // input data from csv files
- CommonUtil.configureCSVInputFormat(hadoopConf, carbonLoadModel)
- hadoopConf.set(FileInputFormat.INPUT_DIR, carbonLoadModel.getFactFilePath)
- val columnCount = columns.length
- val jobConf = new JobConf(hadoopConf)
- SparkHadoopUtil.get.addCredentials(jobConf)
- new NewHadoopRDD[NullWritable, StringArrayWritable](
- sqlContext.sparkContext,
- classOf[CSVInputFormat],
- classOf[NullWritable],
- classOf[StringArrayWritable],
- jobConf
- ).map { currentRow =>
- if (null == currentRow || null == currentRow._2) {
- val row = new StringArrayRow(new Array[String](columnCount))
- (null, row)
- } else {
- val row = new StringArrayRow(new Array[String](columnCount))
- val values = currentRow._2.get()
- if (values != null && values.length > partitionColumnIndex) {
- (currentRow._2.get()(partitionColumnIndex), row.setValues(currentRow._2.get()))
- } else {
- (null, row.setValues(currentRow._2.get()))
- }
- }
- }
- }
-
- val partitioner = PartitionFactory.getPartitioner(partitionInfo)
- if (partitionColumnDataType == DataTypes.STRING) {
- if (partitionInfo.getPartitionType == PartitionType.RANGE) {
- inputRDD.map { row => (ByteUtil.toBytes(row._1), row._2) }
- .partitionBy(partitioner)
- .map(_._2)
- } else {
- inputRDD.partitionBy(partitioner)
- .map(_._2)
- }
- } else {
- inputRDD.map { row =>
- (PartitionUtil.getDataBasedOnDataType(row._1, partitionColumnDataType, timeStampFormat,
- dateFormat), row._2)
- }
- .partitionBy(partitioner)
- .map(_._2)
- }
- }
-
- /**
- * Execute load process for partition table
- */
- private def loadDataForPartitionTable(
- sqlContext: SQLContext,
- dataFrame: Option[DataFrame],
- carbonLoadModel: CarbonLoadModel,
- hadoopConf: Configuration
- ): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
- try {
- val rdd = repartitionInputData(sqlContext, dataFrame, carbonLoadModel, hadoopConf)
- new PartitionTableDataLoaderRDD(
- sqlContext.sparkSession,
- new DataLoadResultImpl(),
- carbonLoadModel,
- rdd
- ).collect()
- } catch {
- case ex: Exception =>
- LOGGER.error("load data failed for partition table", ex)
- throw ex
- }
- }
-
- /**
* Execute load process to load from input dataframe
*/
private def loadDataFrame(
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
index 6792887..a187b6b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
@@ -70,14 +70,6 @@ private[sql] case class CarbonProjectForUpdateCommand(
return Array(Row(updatedRowCount)).toSeq
}
val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
- if (carbonTable.getPartitionInfo != null &&
- (carbonTable.getPartitionInfo.getPartitionType == PartitionType.RANGE ||
- carbonTable.getPartitionInfo.getPartitionType == PartitionType.HASH ||
- carbonTable.getPartitionInfo.getPartitionType == PartitionType.LIST)) {
- throw new UnsupportedOperationException("Unsupported update operation for range/" +
- "hash/list partition table")
- }
-
setAuditTable(carbonTable)
setAuditInfo(Map("plan" -> plan.simpleString))
columns.foreach { col =>
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
deleted file mode 100644
index 2a0d113..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
+++ /dev/null
@@ -1,273 +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.command.partition
-
-import java.util
-import java.util.concurrent.{Executors, ExecutorService, Future}
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.util.AlterTableUtil
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.CacheProvider
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.{DataMapStoreManager, Segment}
-import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.CarbonMetadata
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.processing.loading.TableProcessingOperations
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.spark.partition.DropPartitionCallable
-
-case class CarbonAlterTableDropPartitionCommand(
- model: AlterTableDropPartitionModel)
- extends AtomicRunnableCommand {
-
- private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
- private val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
-
- override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
- if (model.partitionId.equals("0")) {
- sys.error(s"Cannot drop default partition! Please use delete statement!")
- }
- val dbName = model.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
- val tableName = model.tableName
- setAuditTable(dbName, tableName)
- setAuditInfo(Map("partition" -> model.partitionId))
- val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetaStore
- val carbonTable = CarbonEnv.getCarbonTable(Option(dbName), tableName)(sparkSession)
- val tablePath = carbonTable.getTablePath
- val partitionInfo = carbonTable.getPartitionInfo()
- if (partitionInfo == null) {
- throwMetadataException(dbName, tableName, "table is not a partition table")
- }
- val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
- // keep a copy of partitionIdList before update partitionInfo.
- // will be used in partition data scan
- oldPartitionIds.addAll(partitionIds.asJava)
- val partitionIndex = partitionIds.indexOf(Integer.valueOf(model.partitionId))
- partitionInfo.getPartitionType match {
- case PartitionType.HASH => sys.error(s"Hash partition cannot be dropped!")
- case PartitionType.RANGE =>
- val rangeInfo = new util.ArrayList(partitionInfo.getRangeInfo)
- val rangeToRemove = partitionInfo.getRangeInfo.get(partitionIndex - 1)
- rangeInfo.remove(rangeToRemove)
- partitionInfo.setRangeInfo(rangeInfo)
- case PartitionType.LIST =>
- val listInfo = new util.ArrayList(partitionInfo.getListInfo)
- val listToRemove = partitionInfo.getListInfo.get(partitionIndex - 1)
- listInfo.remove(listToRemove)
- partitionInfo.setListInfo(listInfo)
- case PartitionType.RANGE_INTERVAL =>
- throwMetadataException(dbName, tableName,
- "Dropping range interval partition is unsupported")
- case _ => throw new UnsupportedOperationException(
- s"${partitionInfo.getPartitionType} is not supported")
- }
- partitionInfo.dropPartition(partitionIndex)
-
- // read TableInfo
- val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)
- val schemaConverter = new ThriftWrapperSchemaConverterImpl()
- val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
- dbName, tableName, tablePath)
- val tableSchema = wrapperTableInfo.getFactTable
- tableSchema.setPartitionInfo(partitionInfo)
- wrapperTableInfo.setFactTable(tableSchema)
- wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
- val thriftTable =
- schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
- thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
- .setTime_stamp(System.currentTimeMillis)
- carbonMetaStore.updateTableSchemaForAlter(
- carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
- carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
- thriftTable,
- null,
- carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
- // sparkSession.catalog.refreshTable(tableName)
- Seq.empty
- }
-
- override def processData(sparkSession: SparkSession): Seq[Row] = {
- val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
- val dbName = model.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
- val tableName = model.tableName
- var locks = List.empty[ICarbonLock]
- var success = false
- try {
- val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
- LockUsage.COMPACTION_LOCK,
- LockUsage.DELETE_SEGMENT_LOCK,
- LockUsage.DROP_TABLE_LOCK,
- LockUsage.CLEAN_FILES_LOCK,
- LockUsage.ALTER_PARTITION_LOCK)
- locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
- locksToBeAcquired)(sparkSession)
- val carbonLoadModel = new CarbonLoadModel()
- val table = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
- val dataLoadSchema = new CarbonDataLoadSchema(table)
- // Need to fill dimension relation
- carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
- carbonLoadModel.setTableName(table.getTableName)
- carbonLoadModel.setCarbonTransactionalTable(table.isTransactionalTable)
- carbonLoadModel.setDatabaseName(table.getDatabaseName)
- carbonLoadModel.setTablePath(table.getTablePath)
- val loadStartTime = CarbonUpdateUtil.readCurrentTime
- carbonLoadModel.setFactTimeStamp(loadStartTime)
- val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
- .getOrElse(CarbonCommonConstants.COMPRESSOR,
- CompressorFactory.getInstance().getCompressor.getName)
- carbonLoadModel.setColumnCompressor(columnCompressor)
- alterTableDropPartition(
- sparkSession.sqlContext,
- model.partitionId,
- carbonLoadModel,
- model.dropWithData,
- oldPartitionIds.asScala.toList
- )
- success = true
- } catch {
- case e: Exception =>
- sys.error(s"Drop Partition failed. Please check logs for more info. ${ e.getMessage } ")
- success = false
- } finally {
- CacheProvider.getInstance().dropAllCache()
- AlterTableUtil.releaseLocks(locks)
- LOGGER.info("Locks released after alter table drop partition action.")
- }
- LOGGER.info(s"Alter table drop partition is successful for table $dbName.$tableName")
- Seq.empty
- }
-
- private def alterTableDropPartition(sqlContext: SQLContext,
- partitionId: String,
- carbonLoadModel: CarbonLoadModel,
- dropWithData: Boolean,
- oldPartitionIds: List[Int]): Unit = {
- try {
- startDropThreads(
- sqlContext,
- carbonLoadModel,
- partitionId,
- dropWithData,
- oldPartitionIds)
- } catch {
- case e: Exception =>
- LOGGER.error(s"Exception in start dropping partition thread. ${ e.getMessage }")
- throw e
- }
- }
-
- private def startDropThreads(sqlContext: SQLContext,
- carbonLoadModel: CarbonLoadModel,
- partitionId: String,
- dropWithData: Boolean,
- oldPartitionIds: List[Int]): Unit = {
- val numberOfCores = CarbonProperties.getInstance().getNumberOfAltPartitionCores
- val executor : ExecutorService = Executors.newFixedThreadPool(numberOfCores.toInt)
- try {
- val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
- val segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier)
- val validSegments = segmentStatusManager.getValidAndInvalidSegments(carbonTable.isChildTable)
- .getValidSegments.asScala
- val threadArray: Array[Thread] = new Array[Thread](validSegments.size)
- var i = 0
- for (segmentId: Segment <- validSegments) {
- threadArray(i) = dropPartitionThread(sqlContext, carbonLoadModel, executor,
- segmentId, partitionId, dropWithData, oldPartitionIds)
- threadArray(i).start()
- i += 1
- }
- for (thread <- threadArray) {
- thread.join()
- }
- val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(carbonTable)
- refresher.refreshSegments(validSegments.map(_.getSegmentNo).asJava)
- } catch {
- case e: Exception =>
- LOGGER.error(s"Exception when dropping partition: ${ e.getMessage }")
- } finally {
- executor.shutdown()
- try {
- TableProcessingOperations
- .deletePartialLoadDataIfExist(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable,
- false)
- } catch {
- case e: Exception =>
- LOGGER.error(s"Exception in dropping partition thread while deleting partial load file" +
- s" ${ e.getMessage }")
- }
- }
- }
-
- override protected def opName: String = "DROP CUSTOM PARTITION"
-}
-
-case class dropPartitionThread(sqlContext: SQLContext,
- carbonLoadModel: CarbonLoadModel,
- executor: ExecutorService,
- segmentId: Segment,
- partitionId: String,
- dropWithData: Boolean,
- oldPartitionIds: List[Int]) extends Thread {
-
- private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-
- override def run(): Unit = {
- try {
- executeDroppingPartition(sqlContext, carbonLoadModel, executor,
- segmentId, partitionId, dropWithData, oldPartitionIds)
- } catch {
- case e: Exception =>
- val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
- LOGGER.error(s"Exception in dropping partition thread: ${ e.getMessage } }", e)
- }
- }
-
- private def executeDroppingPartition(sqlContext: SQLContext,
- carbonLoadModel: CarbonLoadModel,
- executor: ExecutorService,
- segmentId: Segment,
- partitionId: String,
- dropWithData: Boolean,
- oldPartitionIds: List[Int]): Unit = {
- val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val model = new DropPartitionCallableModel(carbonLoadModel,
- segmentId, partitionId, oldPartitionIds, dropWithData, carbonTable, sqlContext)
- val future: Future[Void] = executor.submit(new DropPartitionCallable(model))
- try {
- future.get
- } catch {
- case e: Exception =>
- LOGGER.error(s"Exception in partition drop thread ${ e.getMessage }", e)
- throw e
- }
- }
-}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
deleted file mode 100644
index d880ddf..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
+++ /dev/null
@@ -1,323 +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.command.partition
-
-import java.text.SimpleDateFormat
-import java.util
-import java.util.concurrent.{Executors, ExecutorService, Future}
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.util.{AlterTableUtil, PartitionUtils}
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.CacheProvider
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.DataMapStoreManager
-import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema.PartitionInfo
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.events.{OperationContext, OperationListenerBus, PostAlterTableHivePartitionCommandEvent, PreAlterTableHivePartitionCommandEvent}
-import org.apache.carbondata.processing.loading.TableProcessingOperations
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.spark.partition.SplitPartitionCallable
-
-/**
- * Command for Alter Table Add & Split partition
- * Add is a special case of Splitting the default partition (part0)
- */
-case class CarbonAlterTableSplitPartitionCommand(
- splitPartitionModel: AlterTableSplitPartitionModel)
- extends AtomicRunnableCommand {
-
- private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
- private val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
-
- override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
- val dbName = splitPartitionModel.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
- val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetaStore
- val tableName = splitPartitionModel.tableName
- setAuditTable(dbName, tableName)
- setAuditInfo(Map("partition" -> splitPartitionModel.partitionId))
- val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
- .asInstanceOf[CarbonRelation]
- val tablePath = relation.carbonTable.getTablePath
- if (relation == null) {
- throwMetadataException(dbName, tableName, "table not found")
- }
- if (null == (CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession))) {
- LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
- throwMetadataException(dbName, tableName, "table not found")
- }
- val carbonTable = relation.carbonTable
- val partitionInfo = carbonTable.getPartitionInfo
- val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
- // keep a copy of partitionIdList before update partitionInfo.
- // will be used in partition data scan
- oldPartitionIds.addAll(partitionIds.asJava)
-
- if (partitionInfo == null) {
- throwMetadataException(dbName, tableName, "Table is not a partition table.")
- }
- if (partitionInfo.getPartitionType == PartitionType.HASH) {
- throwMetadataException(dbName, tableName, "Hash partition table cannot be added or split!")
- }
-
- updatePartitionInfo(partitionInfo, partitionIds)
-
- // read TableInfo
- val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)
- val schemaConverter = new ThriftWrapperSchemaConverterImpl()
- val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
- dbName, tableName, tablePath)
- val tableSchema = wrapperTableInfo.getFactTable
- tableSchema.setPartitionInfo(partitionInfo)
- wrapperTableInfo.setFactTable(tableSchema)
- wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
- val thriftTable =
- schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
- carbonMetaStore.updateTableSchemaForAlter(
- carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
- carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
- thriftTable,
- null,
- carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
- Seq.empty
- }
-
- private def updatePartitionInfo(partitionInfo: PartitionInfo, partitionIds: List[Int]): Unit = {
- val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
- .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
- CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
-
- val timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance
- .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
- CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-
- PartitionUtils.updatePartitionInfo(
- partitionInfo,
- partitionIds,
- splitPartitionModel.partitionId.toInt,
- splitPartitionModel.splitInfo,
- timestampFormatter,
- dateFormatter)
- }
-
- override def processData(sparkSession: SparkSession): Seq[Row] = {
- val dbName = splitPartitionModel.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
- val tableName = splitPartitionModel.tableName
- var locks = List.empty[ICarbonLock]
- var success = false
- try {
- val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
- LockUsage.COMPACTION_LOCK,
- LockUsage.DELETE_SEGMENT_LOCK,
- LockUsage.DROP_TABLE_LOCK,
- LockUsage.CLEAN_FILES_LOCK,
- LockUsage.ALTER_PARTITION_LOCK)
- locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
- locksToBeAcquired)(sparkSession)
- val table = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
- val carbonLoadModel = new CarbonLoadModel()
- val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
- .getOrElse(CarbonCommonConstants.COMPRESSOR,
- CompressorFactory.getInstance().getCompressor.getName)
- carbonLoadModel.setColumnCompressor(columnCompressor)
- val tablePath = table.getTablePath
- val dataLoadSchema = new CarbonDataLoadSchema(table)
- carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
- carbonLoadModel.setTableName(table.getTableName)
- carbonLoadModel.setDatabaseName(table.getDatabaseName)
- carbonLoadModel.setCarbonTransactionalTable(table.isTransactionalTable)
- carbonLoadModel.setTablePath(tablePath)
- val loadStartTime = CarbonUpdateUtil.readCurrentTime
- carbonLoadModel.setFactTimeStamp(loadStartTime)
- val operationContext = new OperationContext
- val preAlterTableHivePartitionCommandEvent = PreAlterTableHivePartitionCommandEvent(
- sparkSession,
- table)
- OperationListenerBus.getInstance()
- .fireEvent(preAlterTableHivePartitionCommandEvent, operationContext)
- alterTableSplitPartition(
- sparkSession.sqlContext,
- splitPartitionModel.partitionId.toInt.toString,
- carbonLoadModel,
- oldPartitionIds.asScala.toList
- )
- val postAlterTableHivePartitionCommandEvent = PostAlterTableHivePartitionCommandEvent(
- sparkSession,
- table)
- OperationListenerBus.getInstance()
- .fireEvent(postAlterTableHivePartitionCommandEvent, operationContext)
- success = true
- } catch {
- case e: Exception =>
- success = false
- sys.error(s"Add/Split Partition failed. Please check logs for more info. ${ e.getMessage }")
- } finally {
- AlterTableUtil.releaseLocks(locks)
- CacheProvider.getInstance().dropAllCache()
- val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
- LOGGER.info("Locks released after alter table add/split partition action.")
- if (success) {
- LOGGER.info(s"Alter table add/split partition is successful for table $dbName.$tableName")
- }
- }
- Seq.empty
- }
-
- private def alterTableSplitPartition(
- sqlContext: SQLContext,
- partitionId: String,
- carbonLoadModel: CarbonLoadModel,
- oldPartitionIdList: List[Int]
- ): Unit = {
- try {
- startSplitThreads(sqlContext,
- carbonLoadModel,
- partitionId,
- oldPartitionIdList)
- } catch {
- case e: Exception =>
- LOGGER.error(s"Exception in start splitting partition thread. ${ e.getMessage }")
- throw e
- }
- }
-
- private def startSplitThreads(sqlContext: SQLContext,
- carbonLoadModel: CarbonLoadModel,
- partitionId: String,
- oldPartitionIdList: List[Int]): Unit = {
- val numberOfCores = CarbonProperties.getInstance().getNumberOfAltPartitionCores
- val executor : ExecutorService = Executors.newFixedThreadPool(numberOfCores)
- try {
- val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
- val segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier)
- val validSegments = segmentStatusManager.getValidAndInvalidSegments(carbonTable.isChildTable)
- .getValidSegments.asScala
- val threadArray: Array[SplitThread] = new Array[SplitThread](validSegments.size)
- var i = 0
- validSegments.foreach { segmentId =>
- threadArray(i) = SplitThread(sqlContext, carbonLoadModel, executor,
- segmentId.getSegmentNo, partitionId, oldPartitionIdList)
- threadArray(i).start()
- i += 1
- }
- threadArray.foreach {
- thread => thread.join()
- }
- val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(carbonTable)
- refresher.refreshSegments(validSegments.map(_.getSegmentNo).asJava)
- } catch {
- case e: Exception =>
- LOGGER.error(s"Exception when split partition: ${ e.getMessage }")
- throw e
- } finally {
- executor.shutdown()
- try {
- TableProcessingOperations
- .deletePartialLoadDataIfExist(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable,
- false)
- } catch {
- case e: Exception =>
- LOGGER.error(s"Exception in add/split partition thread while deleting partial load file" +
- s" ${ e.getMessage }")
- }
- }
- }
-
- override protected def opName: String = "SPLIT CUSTOM PARTITION"
-}
-
-case class SplitThread(sqlContext: SQLContext,
- carbonLoadModel: CarbonLoadModel,
- executor: ExecutorService,
- segmentId: String,
- partitionId: String,
- oldPartitionIdList: List[Int]) extends Thread {
-
- private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-
- override def run(): Unit = {
- var triggeredSplitPartitionStatus = false
- var exception: Exception = null
- try {
- executePartitionSplit(sqlContext,
- carbonLoadModel, executor, segmentId, partitionId, oldPartitionIdList)
- triggeredSplitPartitionStatus = true
- } catch {
- case e: Exception =>
- val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
- LOGGER.error(s"Exception in partition split thread: ${ e.getMessage } }")
- exception = e
- }
- if (!triggeredSplitPartitionStatus) {
- throw new Exception("Exception in split partition " + exception.getMessage)
- }
- }
-
- private def executePartitionSplit( sqlContext: SQLContext,
- carbonLoadModel: CarbonLoadModel,
- executor: ExecutorService,
- segment: String,
- partitionId: String,
- oldPartitionIdList: List[Int]): Unit = {
- val futureList: util.List[Future[Void]] = new util.ArrayList[Future[Void]](
- CarbonCommonConstants.DEFAULT_COLLECTION_SIZE
- )
- scanSegmentsForSplitPartition(futureList, executor, segment, partitionId,
- sqlContext, carbonLoadModel, oldPartitionIdList)
- try {
- futureList.asScala.foreach { future =>
- future.get
- }
- } catch {
- case e: Exception =>
- LOGGER.error(s"Exception in partition split thread ${ e.getMessage }", e)
- throw e
- }
- }
-
- private def scanSegmentsForSplitPartition(futureList: util.List[Future[Void]],
- executor: ExecutorService,
- segmentId: String,
- partitionId: String,
- sqlContext: SQLContext,
- carbonLoadModel: CarbonLoadModel,
- oldPartitionIdList: List[Int]): Unit = {
-
- val splitModel = SplitPartitionCallableModel(carbonLoadModel,
- segmentId,
- partitionId,
- oldPartitionIdList,
- sqlContext)
-
- val future: Future[Void] = executor.submit(new SplitPartitionCallable(splitModel))
- futureList.add(future)
- }
-}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala
deleted file mode 100644
index 4bcdec4b..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala
+++ /dev/null
@@ -1,56 +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.command.partition
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.execution.command.MetadataCommand
-import org.apache.spark.sql.hive.CarbonRelation
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.spark.util.CommonUtil
-
-/**
- * Command for show table partitions Command
- */
-private[sql] case class CarbonShowCarbonPartitionsCommand(
- tableIdentifier: TableIdentifier)
- extends MetadataCommand {
-
- override val output: Seq[Attribute] = CommonUtil.partitionInfoOutput
-
- override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
- val relation = CarbonEnv.getInstance(sparkSession).carbonMetaStore
- .lookupRelation(tableIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
- val carbonTable = relation.carbonTable
- setAuditTable(carbonTable)
- val partitionInfo = carbonTable.getPartitionInfo
- if (partitionInfo == null) {
- throwMetadataException(carbonTable.getDatabaseName, carbonTable.getTableName,
- "SHOW PARTITIONS is not allowed on a table that is not partitioned")
- }
- val partitionType = partitionInfo.getPartitionType
- val columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
- val LOGGER = LogServiceFactory.getLogService(CarbonShowCarbonPartitionsCommand.getClass.getName)
- LOGGER.info("partition column name:" + columnName)
- CommonUtil.getPartitionInfo(columnName, partitionType, partitionInfo)
- }
-
- override protected def opName: String = "SHOW CUSTOM PARTITION"
-}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
index d795d63..8810425 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.command.MetadataCommand
import org.apache.spark.sql.hive.CarbonRelation
import org.apache.carbondata.common.Strings
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
import org.apache.carbondata.core.metadata.datatype.DataTypes
import org.apache.carbondata.core.metadata.schema.PartitionInfo
@@ -197,14 +198,8 @@ private[sql] case class CarbonDescribeFormattedCommand(
("Partition Columns",
partitionInfo.getColumnSchemaList.asScala.map {
col => s"${col.getColumnName}:${col.getDataType.getName}"}.mkString(", "), ""),
- ("Number of Partitions", getNumberOfPartitions(carbonTable, sparkSession), ""),
- ("Partitions Ids", partitionInfo.getPartitionIds.asScala.mkString(","), "")
+ ("Number of Partitions", getNumberOfPartitions(carbonTable, sparkSession), "")
)
- if (partitionInfo.getPartitionType == PartitionType.RANGE) {
- results ++= Seq(("Range", partitionInfo.getRangeInfo.asScala.mkString(", "), ""))
- } else if (partitionInfo.getPartitionType == PartitionType.LIST) {
- results ++= Seq(("List", partitionInfo.getListInfo.asScala.mkString(", "), ""))
- }
}
if (partitionSpec.nonEmpty) {
val partitions = sparkSession.sessionState.catalog.getPartition(tblIdentifier, partitionSpec)
@@ -263,7 +258,7 @@ private[sql] case class CarbonDescribeFormattedCommand(
.listPartitions(new TableIdentifier(carbonTable.getTableName,
Some(carbonTable.getDatabaseName))).size.toString
case _ =>
- carbonTable.getPartitionInfo.getNumPartitions.toString
+ throw new DeprecatedFeatureException("Custom Partition")
}
}
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 37a33ec..d51e85a 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
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.command.management.{CarbonAlterTableCompactionCommand, CarbonInsertIntoCommand, CarbonLoadDataCommand, RefreshCarbonTableCommand}
-import org.apache.spark.sql.execution.command.partition.{CarbonAlterTableAddHivePartitionCommand, CarbonAlterTableDropHivePartitionCommand, CarbonShowCarbonPartitionsCommand}
+import org.apache.spark.sql.execution.command.partition.{CarbonAlterTableAddHivePartitionCommand, CarbonAlterTableDropHivePartitionCommand}
import org.apache.spark.sql.execution.command.schema._
import org.apache.spark.sql.execution.command.table.{CarbonCreateTableLikeCommand, CarbonDescribeFormattedCommand, CarbonDropTableCommand}
import org.apache.spark.sql.hive.execution.command.{CarbonDropDatabaseCommand, CarbonResetCommand, CarbonSetCommand}
@@ -35,6 +35,7 @@ import org.apache.spark.sql.parser.CarbonSpark2SqlParser
import org.apache.spark.sql.types.StructField
import org.apache.spark.util.{CarbonReflectionUtils, DataMapUtil, FileUtils, SparkUtil}
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
import org.apache.carbondata.common.logging.LogServiceFactory
import org.apache.carbondata.core.util.{CarbonProperties, DataTypeUtil, ThreadLocalSessionInfo}
@@ -228,24 +229,6 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
} else {
Nil
}
- case ShowPartitionsCommand(t, cols) =>
- val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetaStore
- .tableExists(t)(sparkSession)
- if (isCarbonTable) {
- val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetaStore
- .lookupRelation(t)(sparkSession).asInstanceOf[CarbonRelation].carbonTable
- if (carbonTable != null && !carbonTable.getTableInfo.isTransactionalTable) {
- throw new MalformedCarbonCommandException(
- "Unsupported operation on non transactional table")
- }
- if (!carbonTable.isHivePartitionTable) {
- ExecutedCommandExec(CarbonShowCarbonPartitionsCommand(t)) :: Nil
- } else {
- ExecutedCommandExec(ShowPartitionsCommand(t, cols)) :: Nil
- }
- } else {
- ExecutedCommandExec(ShowPartitionsCommand(t, cols)) :: Nil
- }
case adp@AlterTableDropPartitionCommand(tableName, specs, ifExists, purge, retainData) =>
val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetaStore
.tableExists(tableName)(sparkSession)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
index 82ea8f6..81923ea 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
@@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.execution.command.datamap.{CarbonCreateDataMapCommand, CarbonDataMapRebuildCommand, CarbonDataMapShowCommand, CarbonDropDataMapCommand}
import org.apache.spark.sql.execution.command.management._
-import org.apache.spark.sql.execution.command.partition.{CarbonAlterTableDropPartitionCommand, CarbonAlterTableSplitPartitionCommand}
import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableColRenameDataTypeChangeCommand, CarbonAlterTableDropColumnCommand}
import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
import org.apache.spark.sql.types.StructField
@@ -77,7 +76,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
protected lazy val startCommand: Parser[LogicalPlan] =
loadManagement | showLoads | alterTable | restructure | updateTable | deleteRecords |
- alterPartition | datamapManagement | alterTableFinishStreaming | stream | cli |
+ datamapManagement | alterTableFinishStreaming | stream | cli |
cacheManagement | alterDataMap
protected lazy val loadManagement: Parser[LogicalPlan] =
@@ -86,9 +85,6 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
protected lazy val restructure: Parser[LogicalPlan] =
alterTableColumnRenameAndModifyDataType | alterTableDropColumn | alterTableAddColumns
- protected lazy val alterPartition: Parser[LogicalPlan] =
- alterAddPartition | alterSplitPartition | alterDropPartition
-
protected lazy val datamapManagement: Parser[LogicalPlan] =
createDataMap | dropDataMap | showDataMap | refreshDataMap
@@ -98,40 +94,6 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
protected lazy val cacheManagement: Parser[LogicalPlan] =
showCache | dropCache
- protected lazy val alterAddPartition: Parser[LogicalPlan] =
- ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (ADD ~> PARTITION ~>
- "(" ~> repsep(stringLit, ",") <~ ")") <~ opt(";") ^^ {
- case dbName ~ table ~ addInfo =>
- val alterTableAddPartitionModel = AlterTableSplitPartitionModel(dbName, table, "0", addInfo)
- CarbonAlterTableSplitPartitionCommand(alterTableAddPartitionModel)
- }
-
- protected lazy val alterSplitPartition: Parser[LogicalPlan] =
- ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (SPLIT ~> PARTITION ~>
- "(" ~> numericLit <~ ")") ~ (INTO ~> "(" ~> repsep(stringLit, ",") <~ ")") <~ opt(";") ^^ {
- case dbName ~ table ~ partitionId ~ splitInfo =>
- val alterTableSplitPartitionModel =
- AlterTableSplitPartitionModel(dbName, table, partitionId, splitInfo)
- if (partitionId == 0) {
- sys.error("Please use [Alter Table Add Partition] statement to split default partition!")
- }
- CarbonAlterTableSplitPartitionCommand(alterTableSplitPartitionModel)
- }
-
- protected lazy val alterDropPartition: Parser[LogicalPlan] =
- ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (DROP ~> PARTITION ~>
- "(" ~> numericLit <~ ")") ~ (WITH ~> DATA).? <~ opt(";") ^^ {
- case dbName ~ table ~ partitionId ~ withData =>
- val dropWithData = withData.getOrElse("NO") match {
- case "NO" => false
- case _ => true
- }
- val alterTableDropPartitionModel =
- AlterTableDropPartitionModel(dbName, table, partitionId, dropWithData)
- CarbonAlterTableDropPartitionCommand(alterTableDropPartitionModel)
- }
-
-
protected lazy val alterTable: Parser[LogicalPlan] =
ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (COMPACT ~ stringLit) ~
(WHERE ~> (SEGMENT ~ "." ~ ID) ~> IN ~> "(" ~> repsep(segmentId, ",") <~ ")").? <~
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
index b720a19..9591bc0 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
@@ -39,7 +39,7 @@ import org.apache.carbondata.core.metadata.schema.SchemaReader
import org.apache.carbondata.core.util.CarbonProperties
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.spark.CarbonOption
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
/**
* Utility class to validate the create table and CTAS command,
@@ -271,9 +271,6 @@ object CarbonSparkSqlParserUtil {
}
// validate partition clause
if (partitionerFields.nonEmpty) {
- if (!CommonUtil.validatePartitionColumns(tableProperties, partitionerFields)) {
- throw new MalformedCarbonCommandException("Error: Invalid partition definition")
- }
// partition columns should not be part of the schema
val badPartCols = partitionerFields.map(_.partitionColumn.toLowerCase).toSet
.intersect(colNames.map(_.toLowerCase).toSet)
@@ -302,26 +299,11 @@ object CarbonSparkSqlParserUtil {
s"Values must be specified for key(s): ${ badKeys.mkString("[", ",", "]") }", ctx)
}
props.map { case (key, value) =>
- if (needToConvertToLowerCase(key)) {
- (key.toLowerCase, value.toLowerCase)
- } else {
- (key.toLowerCase, value)
- }
+ (key.toLowerCase, value)
}
}
/**
- * check's whether need to convert to lower case
- *
- * @param key <String> property key
- * @return returns <true> if lower case conversion is needed else <false>
- */
- def needToConvertToLowerCase(key: String): Boolean = {
- val noConvertList = Array("LIST_INFO", "RANGE_INFO", "PATH")
- !noConvertList.exists(x => x.equalsIgnoreCase(key))
- }
-
- /**
* The method validate the create table command and returns the table's columns.
*
* @param tableHeader An instance of CreateTableHeaderContext having parser rules for
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
deleted file mode 100644
index bb04d0a..0000000
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
+++ /dev/null
@@ -1,926 +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.spark.testsuite.partition
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
-
-import org.apache.hadoop.fs.Path
-import org.apache.spark.sql.CarbonEnv
-import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.Segment
-import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.metadata.{CarbonMetadata, SegmentFileStore}
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-
-class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
-
-
- override def beforeAll {
- dropTable
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
- // deactivating the merge-index for old partition implimentation because we are not supporting
- // merge-index for the same currently.
- .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,"false")
- /**
- * list_table_area_origin
- * list_table_area
- */
- sql("""
- | CREATE TABLE IF NOT EXISTS list_table_area_origin
- | (
- | id Int,
- | vin string,
- | logdate Timestamp,
- | phonenumber Long,
- | country string,
- | salary Int
- | )
- | PARTITIONED BY (area string)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='Asia, America, Europe')
- """.stripMargin)
- sql("""
- | CREATE TABLE IF NOT EXISTS list_table_area
- | (
- | id Int,
- | vin string,
- | logdate Timestamp,
- | phonenumber Long,
- | country string,
- | salary Int
- | )
- | PARTITIONED BY (area string)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='Asia, America, Europe')
- """.stripMargin)
-
- /**
- * range_table_logdate_origin
- * range_table_logdate
- */
- sql(
- """
- | CREATE TABLE IF NOT EXISTS range_table_logdate_origin
- | (
- | id Int,
- | vin string,
- | phonenumber Long,
- | country string,
- | area string,
- | salary Int
- | )
- | PARTITIONED BY (logdate Timestamp)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01')
- """.stripMargin)
- sql(
- """
- | CREATE TABLE IF NOT EXISTS range_table_logdate
- | (
- | id Int,
- | vin string,
- | phonenumber Long,
- | country string,
- | area string,
- | salary Int
- | )
- | PARTITIONED BY (logdate Timestamp)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01', 'DICTIONARY_INCLUDE'='logdate')
- """.stripMargin)
-
- /**
- * list_table_country_origin
- * list_table_country
- */
- sql(
- """
- | CREATE TABLE IF NOT EXISTS list_table_country_origin
- | (
- | id Int,
- | vin string,
- | logdate Timestamp,
- | phonenumber Long,
- | area string,
- | salary Int
- | )
- | PARTITIONED BY (country string)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, NotGood), Korea ')
- """.stripMargin)
- sql(
- """
- | CREATE TABLE IF NOT EXISTS list_table_country
- | (
- | id Int,
- | vin string,
- | logdate Timestamp,
- | phonenumber Long,
- | area string,
- | salary Int
- | )
- | PARTITIONED BY (country string)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='LIST',
- | 'LIST_INFO'='(China, US),UK ,Japan,(Canada,Russia, Good, NotGood), Korea ')
- """.stripMargin)
-
- /**
- * range_table_logdate_split_origin
- * range_table_logdate_split
- */
- sql(
- """
- | CREATE TABLE IF NOT EXISTS range_table_logdate_split_origin
- | (
- | id Int,
- | vin string,
- | phonenumber Long,
- | country string,
- | area string,
- | salary Int
- | )
- | PARTITIONED BY (logdate Timestamp)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01, 2018/01/01')
- """.stripMargin)
- sql(
- """
- | CREATE TABLE IF NOT EXISTS range_table_logdate_split
- | (
- | id Int,
- | vin string,
- | phonenumber Long,
- | country string,
- | area string,
- | salary Int
- | )
- | PARTITIONED BY (logdate Timestamp)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01, 2018/01/01',
- | 'DICTIONARY_INCLUDE'='logdate')
- """.stripMargin)
-
- /**
- * range_table_bucket_origin
- * range_table_bucket
- */
- sql(
- """
- | CREATE TABLE IF NOT EXISTS range_table_bucket_origin
- | (
- | id Int,
- | vin string,
- | phonenumber Long,
- | country string,
- | area string,
- | salary Int
- | )
- | PARTITIONED BY (logdate Timestamp)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01, 2018/01/01',
- | 'BUCKETNUMBER'='3',
- | 'BUCKETCOLUMNS'='country')
- """.stripMargin)
- sql(
- """
- | CREATE TABLE IF NOT EXISTS range_table_bucket
- | (
- | id Int,
- | vin string,
- | phonenumber Long,
- | country string,
- | area string,
- | salary Int
- | )
- | PARTITIONED BY (logdate Timestamp)
- | STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
- | 'RANGE_INFO'='2014/01/01, 2015/01/01, 2016/01/01, 2018/01/01',
- | 'DICTIONARY_INCLUDE'='logdate',
- | 'BUCKETNUMBER'='3',
- | 'BUCKETCOLUMNS'='country')
- """.stripMargin)
-
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE list_table_area_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_logdate_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE list_table_country_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_logdate_split_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_bucket_origin OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE list_table_area OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_logdate OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE list_table_country OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_logdate_split OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
- sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/partition_data.csv' INTO TABLE range_table_bucket OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
-
- }
-
- test("Alter table add partition: List Partition") {
- sql("""ALTER TABLE list_table_area ADD PARTITION ('OutSpace', 'Hi')""".stripMargin)
- val carbonTable = CarbonEnv
- .getCarbonTable(Option("default"), "list_table_area")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionIds = partitionInfo.getPartitionIds
- val list_info = partitionInfo.getListInfo
- assert(partitionIds == List(0, 1, 2, 3, 4, 5).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo.getMaxPartitionId == 5)
- assert(partitionInfo.getNumPartitions == 6)
- assert(list_info.get(0).get(0) == "Asia")
- assert(list_info.get(1).get(0) == "America")
- assert(list_info.get(2).get(0) == "Europe")
- assert(list_info.get(3).get(0) == "OutSpace")
- assert(list_info.get(4).get(0) == "Hi")
- validateDataFiles("default_list_table_area", "0", Seq(1, 2, 4))
- val result_after = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area")
- val result_origin = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin")
- checkAnswer(result_after, result_origin)
-
- val result_after1 = sql(s"select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area < 'OutSpace' ")
- val result_origin1 = sql(s"select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area < 'OutSpace' ")
- checkAnswer(result_after1, result_origin1)
-
- val result_after2 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area <= 'OutSpace' ")
- val result_origin2 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <= 'OutSpace' ")
- checkAnswer(result_after2, result_origin2)
-
- val result_after3 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area = 'OutSpace' ")
- val result_origin3 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area = 'OutSpace' ")
- checkAnswer(result_after3, result_origin3)
-
- val result_after4 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area > 'OutSpace' ")
- val result_origin4 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area > 'OutSpace' ")
- checkAnswer(result_after4, result_origin4)
-
- val result_after5 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area >= 'OutSpace' ")
- val result_origin5 = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area >= 'OutSpace' ")
- checkAnswer(result_after5, result_origin5)
-
- intercept[Exception] { sql("""ALTER TABLE DROP PARTITION(0)""")}
- intercept[Exception] { sql("""ALTER TABLE DROP PARTITION(0) WITH DATA""")}
-
- sql("""ALTER TABLE list_table_area DROP PARTITION(2) WITH DATA""")
- val carbonTable2 = CarbonMetadata.getInstance().getCarbonTable("default", "list_table_area")
- val partitionInfo2 = carbonTable2.getPartitionInfo()
- val partitionIds2 = partitionInfo2.getPartitionIds
- val list_info2 = partitionInfo2.getListInfo
- assert(partitionIds2 == List(0, 1, 3, 4, 5).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo2.getMaxPartitionId == 5)
- assert(partitionInfo2.getNumPartitions == 5)
- assert(list_info2.get(0).get(0) == "Asia")
- assert(list_info2.get(1).get(0) == "Europe")
- assert(list_info2.get(2).get(0) == "OutSpace")
- assert(list_info2.get(3).get(0) == "Hi")
- validateDataFiles("default_list_table_area", "0", Seq(1, 4))
- checkAnswer(sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area"),
- sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <> 'America' "))
- }
-
- ignore("Alter table add partition: Range Partition") {
- sql("""ALTER TABLE range_table_logdate ADD PARTITION ('2017/01/01', '2018/01/01')""")
- val carbonTable = CarbonEnv
- .getCarbonTable(Option("default"), "range_table_logdate")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionIds = partitionInfo.getPartitionIds
- val range_info = partitionInfo.getRangeInfo
- assert(partitionIds == List(0, 1, 2, 3, 4, 5).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo.getMaxPartitionId == 5)
- assert(partitionInfo.getNumPartitions == 6)
- assert(range_info.get(0) == "2014/01/01")
- assert(range_info.get(1) == "2015/01/01")
- assert(range_info.get(2) == "2016/01/01")
- assert(range_info.get(3) == "2017/01/01")
- assert(range_info.get(4) == "2018/01/01")
- validateDataFiles("default_range_table_logdate", "0", Seq(1, 2, 3, 4, 5))
- val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate""")
- val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin""")
- checkAnswer(result_after, result_origin)
-
- val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after1, result_origin1)
-
- val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after2, result_origin2)
-
- val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after3, result_origin3)
-
- val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after4, result_origin4)
-
- val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after5, result_origin5)
-
- sql("""ALTER TABLE range_table_logdate DROP PARTITION(3) WITH DATA;""")
- val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable(
- "default",
- "range_table_logdate"
- )
- val partitionInfo1 = carbonTable1.getPartitionInfo()
- val partitionIds1 = partitionInfo1.getPartitionIds
- val range_info1 = partitionInfo1.getRangeInfo
- assert(partitionIds1 == List(0, 1, 2, 4, 5).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo1.getMaxPartitionId == 5)
- assert(partitionInfo1.getNumPartitions == 5)
- assert(range_info1.get(0) == "2014/01/01")
- assert(range_info1.get(1) == "2015/01/01")
- assert(range_info1.get(2) == "2017/01/01")
- assert(range_info1.get(3) == "2018/01/01")
- assert(range_info1.size() == 4)
- validateDataFiles("default_range_table_logdate", "0", Seq(1, 2, 4, 5))
- val result_after6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate""")
- val result_origin6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_origin where logdate < '2015/01/01 00:00:00' or logdate >= '2016/01/01 00:00:00' """)
- checkAnswer(result_after6, result_origin6)
- }
-
- test("test exception if invalid partition id is provided in alter command") {
- sql("drop table if exists test_invalid_partition_id")
-
- sql("CREATE TABLE test_invalid_partition_id (CUST_NAME String,ACTIVE_EMUI_VERSION string,DOB Timestamp,DOJ 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)" +
- " STORED BY 'org.apache.carbondata.format' " +
- "TBLPROPERTIES ('PARTITION_TYPE'='RANGE','RANGE_INFO'='9090,9500,9800',\"TABLE_BLOCKSIZE\"= \"256 MB\")")
- intercept[IllegalArgumentException] { sql("ALTER TABLE test_invalid_partition_id SPLIT PARTITION(6) INTO ('9800','9900')") }
- }
-
- test("Alter table split partition: List Partition") {
- sql("""ALTER TABLE list_table_country SPLIT PARTITION(4) INTO ('Canada', 'Russia', '(Good, NotGood)')""".stripMargin)
- val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "list_table_country")
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionIds = partitionInfo.getPartitionIds
- val list_info = partitionInfo.getListInfo
- assert(partitionIds == List(0, 1, 2, 3, 6, 7, 8, 5).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo.getMaxPartitionId == 8)
- assert(partitionInfo.getNumPartitions == 8)
- assert(list_info.get(0).get(0) == "China")
- assert(list_info.get(0).get(1) == "US")
- assert(list_info.get(1).get(0) == "UK")
- assert(list_info.get(2).get(0) == "Japan")
- assert(list_info.get(3).get(0) == "Canada")
- assert(list_info.get(4).get(0) == "Russia")
- assert(list_info.get(5).get(0) == "Good")
- assert(list_info.get(5).get(1) == "NotGood")
- assert(list_info.get(6).get(0) == "Korea")
- validateDataFiles("default_list_table_country", "0", Seq(1, 2, 3, 8))
- val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country""")
- val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin""")
- checkAnswer(result_after, result_origin)
-
- val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country < 'NotGood' """)
- val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country < 'NotGood' """)
- checkAnswer(result_after1, result_origin1)
-
- val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country <= 'NotGood' """)
- val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country <= 'NotGood' """)
- checkAnswer(result_after2, result_origin2)
-
- val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country = 'NotGood' """)
- val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country = 'NotGood' """)
- checkAnswer(result_after3, result_origin3)
-
- val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country >= 'NotGood' """)
- val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country >= 'NotGood' """)
- checkAnswer(result_after4, result_origin4)
-
- val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country > 'NotGood' """)
- val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country > 'NotGood' """)
- checkAnswer(result_after5, result_origin5)
-
- sql("""ALTER TABLE list_table_country DROP PARTITION(8)""")
- val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("default", "list_table_country")
- val partitionInfo1 = carbonTable1.getPartitionInfo()
- val partitionIds1 = partitionInfo1.getPartitionIds
- val list_info1 = partitionInfo1.getListInfo
- assert(partitionIds1 == List(0, 1, 2, 3, 6, 7, 5).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo1.getMaxPartitionId == 8)
- assert(partitionInfo1.getNumPartitions == 7)
- assert(list_info1.get(0).get(0) == "China")
- assert(list_info1.get(0).get(1) == "US")
- assert(list_info1.get(1).get(0) == "UK")
- assert(list_info1.get(2).get(0) == "Japan")
- assert(list_info1.get(3).get(0) == "Canada")
- assert(list_info1.get(4).get(0) == "Russia")
- assert(list_info1.get(5).get(0) == "Korea")
- validateDataFiles("default_list_table_country", "0", Seq(0, 1, 2, 3))
- val result_origin6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country""")
- val result_after6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin""")
- checkAnswer(result_origin6, result_after6)
- }
-
- test("Alter table split partition with different List Sequence: List Partition") {
- sql("""ALTER TABLE list_table_country ADD PARTITION ('(Part1, Part2, Part3, Part4)')""".stripMargin)
- sql("""ALTER TABLE list_table_country SPLIT PARTITION(9) INTO ('Part4', 'Part2', '(Part1, Part3)')""".stripMargin)
- val carbonTable = CarbonEnv
- .getCarbonTable(Option("default"), "list_table_country")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionIds = partitionInfo.getPartitionIds
- val list_info = partitionInfo.getListInfo
- assert(partitionIds == List(0, 1, 2, 3, 6, 7, 5, 10, 11, 12).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo.getMaxPartitionId == 12)
- assert(partitionInfo.getNumPartitions == 10)
- assert(list_info.get(0).get(0) == "China")
- assert(list_info.get(0).get(1) == "US")
- assert(list_info.get(1).get(0) == "UK")
- assert(list_info.get(2).get(0) == "Japan")
- assert(list_info.get(3).get(0) == "Canada")
- assert(list_info.get(4).get(0) == "Russia")
- assert(list_info.get(5).get(0) == "Korea")
- assert(list_info.get(6).get(0) == "Part4")
- assert(list_info.get(7).get(0) == "Part2")
- assert(list_info.get(8).get(0) == "Part1")
- assert(list_info.get(8).get(1) == "Part3")
- validateDataFiles("default_list_table_country", "0", Seq(0, 1, 2, 3))
- val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country""")
- val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin""")
- checkAnswer(result_after, result_origin)
-
- val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country < 'NotGood' """)
- val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country < 'NotGood' """)
- checkAnswer(result_after1, result_origin1)
-
- val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country <= 'NotGood' """)
- val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country <= 'NotGood' """)
- checkAnswer(result_after2, result_origin2)
-
- val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country = 'NotGood' """)
- val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country = 'NotGood' """)
- checkAnswer(result_after3, result_origin3)
-
- val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country >= 'NotGood' """)
- val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country >= 'NotGood' """)
- checkAnswer(result_after4, result_origin4)
-
- val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country where country > 'NotGood' """)
- val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin where country > 'NotGood' """)
- checkAnswer(result_after5, result_origin5)
- }
-
- test("Alter table split partition with extra space in New SubList: List Partition") {
- sql("""ALTER TABLE list_table_area ADD PARTITION ('(One,Two, Three, Four)')""".stripMargin)
- sql("""ALTER TABLE list_table_area SPLIT PARTITION(6) INTO ('One', '(Two, Three )', 'Four')""".stripMargin)
- val carbonTable = CarbonEnv
- .getCarbonTable(Option("default"), "list_table_area")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionIds = partitionInfo.getPartitionIds
- val list_info = partitionInfo.getListInfo
- assert(partitionIds == List(0, 1, 3, 4, 5, 7, 8, 9).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo.getMaxPartitionId == 9)
- assert(partitionInfo.getNumPartitions == 8)
- assert(list_info.get(0).get(0) == "Asia")
- assert(list_info.get(1).get(0) == "Europe")
- assert(list_info.get(2).get(0) == "OutSpace")
- assert(list_info.get(3).get(0) == "Hi")
- assert(list_info.get(4).get(0) == "One")
- assert(list_info.get(5).get(0) == "Two")
- assert(list_info.get(5).get(1) == "Three")
- assert(list_info.get(6).get(0) == "Four")
- validateDataFiles("default_list_table_area", "0", Seq(1, 4))
- val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area""")
- val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <> 'America' """)
- checkAnswer(result_after, result_origin)
-
- val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area < 'Four' """)
- val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area < 'Four' and area <> 'America' """)
- checkAnswer(result_after1, result_origin1)
-
- val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area <= 'Four' """)
- val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <= 'Four' and area <> 'America' """)
- checkAnswer(result_after2, result_origin2)
-
- val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area = 'Four' """)
- val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area = 'Four' and area <> 'America' """)
- checkAnswer(result_after3, result_origin3)
-
- val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area >= 'Four' """)
- val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area >= 'Four' and area <> 'America' """)
- checkAnswer(result_after4, result_origin4)
-
- val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area > 'Four' """)
- val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area > 'Four' and area <> 'America' """)
- checkAnswer(result_after5, result_origin5)
- }
-
- test("Alter table split partition: Range Partition") {
- sql("""ALTER TABLE range_table_logdate_split SPLIT PARTITION(4) INTO ('2017/01/01', '2018/01/01')""")
- val carbonTable = CarbonEnv
- .getCarbonTable(Option("default"), "range_table_logdate_split")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionIds = partitionInfo.getPartitionIds
- val rangeInfo = partitionInfo.getRangeInfo
- assert(partitionIds == List(0, 1, 2, 3, 5, 6).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo.getMaxPartitionId == 6)
- assert(partitionInfo.getNumPartitions == 6)
- assert(rangeInfo.get(0) == "2014/01/01")
- assert(rangeInfo.get(1) == "2015/01/01")
- assert(rangeInfo.get(2) == "2016/01/01")
- assert(rangeInfo.get(3) == "2017/01/01")
- assert(rangeInfo.get(4) == "2018/01/01")
- validateDataFiles("default_range_table_logdate_split", "0", Seq(1, 2, 3, 5, 6))
- val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split""")
- val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin""")
- checkAnswer(result_after, result_origin)
-
- val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after1, result_origin1)
-
- val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after2, result_origin2)
-
- val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after3, result_origin3)
-
- val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after4, result_origin4)
-
- val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after5, result_origin5)
-
- sql("""ALTER TABLE range_table_logdate_split DROP PARTITION(6)""")
- val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable(
- "default",
- "range_table_logdate_split"
- )
- val partitionInfo1 = carbonTable1.getPartitionInfo()
- val partitionIds1 = partitionInfo1.getPartitionIds
- val rangeInfo1 = partitionInfo1.getRangeInfo
- assert(partitionIds1 == List(0, 1, 2, 3, 5).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo1.getMaxPartitionId == 6)
- assert(partitionInfo1.getNumPartitions == 5)
- assert(rangeInfo1.get(0) == "2014/01/01")
- assert(rangeInfo1.get(1) == "2015/01/01")
- assert(rangeInfo1.get(2) == "2016/01/01")
- assert(rangeInfo1.get(3) == "2017/01/01")
- assert(rangeInfo1.size() == 4)
- validateDataFiles("default_range_table_logdate_split", "0", Seq(0, 1, 2, 3, 5))
- val result_after6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split""")
- val result_origin6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_logdate_split_origin""")
- checkAnswer(result_after6, result_origin6)
- }
-
- ignore("Alter table split partition: Range Partition + Bucket") {
- sql("""ALTER TABLE range_table_bucket SPLIT PARTITION(4) INTO ('2017/01/01', '2018/01/01')""")
- val carbonTable = CarbonEnv
- .getCarbonTable(Option("default"), "range_table_bucket")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionIds = partitionInfo.getPartitionIds
- val rangeInfo = partitionInfo.getRangeInfo
- assert(partitionIds == List(0, 1, 2, 3, 5, 6).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo.getMaxPartitionId == 6)
- assert(partitionInfo.getNumPartitions == 6)
- assert(rangeInfo.get(0) == "2014/01/01")
- assert(rangeInfo.get(1) == "2015/01/01")
- assert(rangeInfo.get(2) == "2016/01/01")
- assert(rangeInfo.get(3) == "2017/01/01")
- assert(rangeInfo.get(4) == "2018/01/01")
- validateDataFiles("default_range_table_bucket", "0", Seq(1, 2, 3, 5, 6))
- val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket""")
- val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin""")
- checkAnswer(result_after, result_origin)
-
- val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate < cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after1, result_origin1)
-
- val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate <= cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after2, result_origin2)
-
- val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
- val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate = cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_origin3, result_after3)
-
- val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate >= cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after4, result_origin4)
-
- val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
- val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate > cast('2017/01/12 00:00:00' as timestamp) """)
- checkAnswer(result_after5, result_origin5)
-
- sql("""ALTER TABLE range_table_bucket DROP PARTITION(6) WITH DATA""")
- val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("default", "range_table_bucket")
- val partitionInfo1 = carbonTable1.getPartitionInfo()
- val partitionIds1 = partitionInfo1.getPartitionIds
- val rangeInfo1 = partitionInfo1.getRangeInfo
- assert(partitionIds1 == List(0, 1, 2, 3, 5).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo1.getMaxPartitionId == 6)
- assert(partitionInfo1.getNumPartitions == 5)
- assert(rangeInfo1.get(0) == "2014/01/01")
- assert(rangeInfo1.get(1) == "2015/01/01")
- assert(rangeInfo1.get(2) == "2016/01/01")
- assert(rangeInfo1.get(3) == "2017/01/01")
- assert(rangeInfo1.size() == 4)
- validateDataFiles("default_range_table_bucket", "0", Seq(1, 2, 3, 5))
- val result_after6 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket""")
- val result_origin6= sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate < '2017/01/01 00:00:00' or logdate >= '2018/01/01 00:00:00'""")
- checkAnswer(result_after6, result_origin6)
-
- sql("""ALTER TABLE range_table_bucket DROP PARTITION(3)""")
- val carbonTable2 = CarbonMetadata.getInstance().getCarbonTable("default", "range_table_bucket")
- val partitionInfo2 = carbonTable2.getPartitionInfo()
- val partitionIds2 = partitionInfo2.getPartitionIds
- val rangeInfo2 = partitionInfo2.getRangeInfo
- assert(partitionIds2 == List(0, 1, 2, 5).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo2.getMaxPartitionId == 6)
- assert(partitionInfo2.getNumPartitions == 4)
- assert(rangeInfo2.get(0) == "2014/01/01")
- assert(rangeInfo2.get(1) == "2015/01/01")
- assert(rangeInfo2.get(2) == "2017/01/01")
- assert(rangeInfo2.size() == 3)
- validateDataFiles("default_range_table_bucket", "0", Seq(1, 2, 5))
- val result_origin7 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket""")
- val result_after7 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate < '2017/01/01 00:00:00' or logdate >= '2018/01/01 00:00:00'""")
- checkAnswer(result_origin7, result_after7)
-
- sql("""ALTER TABLE range_table_bucket DROP PARTITION(5)""")
- val carbonTable3 = CarbonMetadata.getInstance().getCarbonTable("default", "range_table_bucket")
- val partitionInfo3 = carbonTable3.getPartitionInfo()
- val partitionIds3 = partitionInfo3.getPartitionIds
- val rangeInfo3 = partitionInfo3.getRangeInfo
- assert(partitionIds3 == List(0, 1, 2).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo3.getMaxPartitionId == 6)
- assert(partitionInfo3.getNumPartitions == 3)
- assert(rangeInfo3.get(0) == "2014/01/01")
- assert(rangeInfo3.get(1) == "2015/01/01")
- assert(rangeInfo3.size() == 2)
- validateDataFiles("default_range_table_bucket", "0", Seq(0, 1, 2))
- val result_after8 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket""")
- val result_origin8 = sql("""select id, vin, logdate, phonenumber, country, area, salary from range_table_bucket_origin where logdate < '2017/01/01 00:00:00' or logdate >= '2018/01/01 00:00:00'""")
- checkAnswer(result_after8, result_origin8)
- }
-
- test("test exception when alter partition and the values"
- + "in range_info can not match partition column type") {
- val exception_test_range_int: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_int(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='11,12')
- """.stripMargin)
- sql("ALTER TABLE test_range_int ADD PARTITION ('abc')")
- }
- assert(exception_test_range_int.getMessage
- .contains("Data in range info must be the same type with the partition field's type"))
-
- sql("DROP TABLE IF EXISTS test_range_smallint")
- val exception_test_range_smallint: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_smallint(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 SMALLINT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='11,12')
- """.stripMargin)
- sql("ALTER TABLE test_range_smallint ADD PARTITION ('abc')")
- }
- assert(exception_test_range_smallint.getMessage
- .contains("Data in range info must be the same type with the partition field's type"))
-
- sql("DROP TABLE IF EXISTS test_range_float")
- val exception_test_range_float: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_float(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 FLOAT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1.1,2.1')
- """.stripMargin)
- sql("ALTER TABLE test_range_float ADD PARTITION ('abc')")
- }
- assert(exception_test_range_float.getMessage
- .contains("Data in range info must be the same type with the partition field's type"))
-
- sql("DROP TABLE IF EXISTS test_range_double")
- val exception_test_range_double: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_double(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 DOUBLE) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='1000.005,2000.005')
- """.stripMargin)
- sql("ALTER TABLE test_range_double ADD PARTITION ('abc')")
- }
- assert(exception_test_range_double.getMessage
- .contains("Data in range info must be the same type with the partition field's type"))
-
- sql("DROP TABLE IF EXISTS test_range_bigint")
- val exception_test_range_bigint: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_bigint(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 BIGINT) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='123456789,223456789')
- """.stripMargin)
- sql("ALTER TABLE test_range_bigint ADD PARTITION ('abc')")
- }
- assert(exception_test_range_bigint.getMessage
- .contains("Data in range info must be the same type with the partition field's type"))
-
- sql("DROP TABLE IF EXISTS test_range_date")
- val exception_test_range_date: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_date(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 DATE) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2017-06-11, 2017-06-13')
- """.stripMargin)
- sql("ALTER TABLE test_range_date ADD PARTITION ('abc')")
- }
- assert(exception_test_range_date.getMessage
- .contains("Data in range info must be the same type with the partition field's type"))
-
- sql("DROP TABLE IF EXISTS test_range_timestamp")
- val exception_test_range_timestamp: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_timestamp(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 TIMESTAMP) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2017/06/11, 2017/06/13')
- """.stripMargin)
- sql("ALTER TABLE test_range_timestamp ADD PARTITION ('abc')")
- }
- assert(exception_test_range_timestamp.getMessage
- .contains("Data in range info must be the same type with the partition field's type"))
- sql("DROP TABLE IF EXISTS test_range_decimal")
- val exception_test_range_decimal: Exception = intercept[Exception] {
- sql(
- """
- | CREATE TABLE test_range_decimal(col1 INT, col2 STRING)
- | PARTITIONED BY (col3 DECIMAL(25, 4)) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='22.22,33.33')
- """.stripMargin)
- sql("ALTER TABLE test_range_decimal ADD PARTITION ('abc')")
- }
- assert(exception_test_range_decimal.getMessage
- .contains("Data in range info must be the same type with the partition field's type"))
- }
-
- test("Add partition to table in or not in default database") {
- sql("DROP TABLE IF EXISTS carbon_table_default_db")
- sql(
- """
- | CREATE TABLE carbon_table_default_db(id INT, name STRING) PARTITIONED BY (dt STRING)
- | STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2015,2016')
- """.stripMargin)
- sql("ALTER TABLE carbon_table_default_db ADD PARTITION ('2017')")
- val carbonTable = CarbonEnv
- .getCarbonTable(Option("default"), "carbon_table_default_db")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo()
- val partitionIds = partitionInfo.getPartitionIds
- val range_info = partitionInfo.getRangeInfo
- assert(partitionIds == List(0, 1, 2, 3).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo.getMaxPartitionId == 3)
- assert(partitionInfo.getNumPartitions == 4)
- assert(range_info.get(0) == "2015")
- assert(range_info.get(1) == "2016")
- assert(range_info.get(2) == "2017")
-
- sql("CREATE DATABASE IF NOT EXISTS carbondb")
- sql("DROP TABLE IF EXISTS carbondb.carbontable")
- sql(
- """
- | CREATE TABLE carbondb.carbontable(id INT, name STRING) PARTITIONED BY (dt STRING)
- | STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2015,2016')
- """.stripMargin)
- sql("ALTER TABLE carbondb.carbontable ADD PARTITION ('2017')")
-
- val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("carbondb", "carbontable")
- val partitionInfo1 = carbonTable1.getPartitionInfo()
- val partitionIds1 = partitionInfo1.getPartitionIds
- val range_info1 = partitionInfo1.getRangeInfo
- assert(partitionIds1 == List(0, 1, 2, 3).map(Integer.valueOf(_)).asJava)
- assert(partitionInfo1.getMaxPartitionId == 3)
- assert(partitionInfo1.getNumPartitions == 4)
- assert(range_info1.get(0) == "2015")
- assert(range_info1.get(1) == "2016")
- assert(range_info1.get(2) == "2017")
- }
-
- test("test exception when alter partition's table doesn't exist in a perticular database") {
- val exception_test_add_partition: Exception = intercept[Exception] {
- sql("CREATE DATABASE IF NOT EXISTS carbondb")
- sql("USE default")
- sql("drop table if exists carbon_table_in_default_db")
- sql(
- """
- | CREATE TABLE carbon_table_in_default_db(id INT, name STRING)
- | PARTITIONED BY (dt STRING) STORED BY 'carbondata'
- | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='2015,2016')
- """.stripMargin)
- sql("ALTER TABLE carbondb.carbon_table_in_default_db ADD PARTITION ('2017')")
- }
- assert(exception_test_add_partition.getMessage
- .contains("Table or view 'carbon_table_in_default_db' not found in database 'carbondb'"))
- }
-
- def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Seq[Int]): Unit = {
- val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
- val dataFiles = getDataFiles(carbonTable, segmentId)
- validatePartitionTableFiles(partitions, dataFiles)
- }
-
- def getDataFiles(carbonTable: CarbonTable, segmentId: String): Array[String] = {
- val segment = Segment.getSegment(segmentId, carbonTable.getTablePath)
- val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
- val carbonFile = FileFactory.getCarbonFile(segmentDir)
- val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
- override def accept(file: CarbonFile): Boolean = {
- return file.getName.endsWith(".carbondata")
- }
- })
- dataFiles.map(_.getName)
- }
-
- /**
- * should ensure answer equals to expected list, not only contains
- * @param partitions
- * @param dataFiles
- */
- def validatePartitionTableFiles(partitions: Seq[Int], dataFiles: Array[String]): Unit = {
- val partitionIds: ListBuffer[Int] = new ListBuffer[Int]()
- dataFiles.foreach { dataFile =>
- val partitionId = CarbonTablePath.DataFileUtil.getTaskNo(dataFile).split("_")(0).toInt
- partitionIds += partitionId
- assert(partitions.contains(partitionId))
- }
- partitions.foreach(id => assert(partitionIds.contains(id)))
- }
-
- override def afterAll = {
- sql("DROP TABLE IF EXISTS carbondb.carbontable")
- sql("DROP DATABASE IF EXISTS carbondb")
- dropTable
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
- CarbonProperties.getInstance()
- .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
- .addProperty(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,
- CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT)
- }
-
- def dropTable {
- sql("DROP TABLE IF EXISTS list_table_area_origin")
- sql("DROP TABLE IF EXISTS range_table_logdate_origin")
- sql("DROP TABLE IF EXISTS list_table_country_origin")
- sql("DROP TABLE IF EXISTS range_table_logdate_split_origin")
- sql("DROP TABLE IF EXISTS range_table_bucket_origin")
- sql("DROP TABLE IF EXISTS list_table_area")
- sql("DROP TABLE IF EXISTS range_table_logdate")
- sql("DROP TABLE IF EXISTS list_table_country")
- sql("DROP TABLE IF EXISTS range_table_logdate_split")
- sql("DROP TABLE IF EXISTS range_table_bucket")
- sql("DROP TABLE IF EXISTS test_range_int")
- sql("DROP TABLE IF EXISTS test_range_smallint")
- sql("DROP TABLE IF EXISTS test_range_bigint")
- sql("DROP TABLE IF EXISTS test_range_float")
- sql("DROP TABLE IF EXISTS test_range_double")
- sql("DROP TABLE IF EXISTS test_range_date")
- sql("DROP TABLE IF EXISTS test_range_timestamp")
- sql("DROP TABLE IF EXISTS test_range_decimal")
- sql("drop table if exists test_invalid_partition_id")
- }
-
-
-}
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
index 8d5ee0e..1cdf93b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
@@ -32,7 +32,6 @@ import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
import org.apache.carbondata.common.logging.LogServiceFactory;
import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
-import org.apache.carbondata.core.constants.SortScopeOptions;
import org.apache.carbondata.core.datastore.compression.CompressorFactory;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
@@ -176,7 +175,6 @@ public class CarbonLoadModelBuilder {
String column_dict = optionsFinal.get("columndict");
validateDateTimeFormat(timestampformat, "TimestampFormat");
validateDateTimeFormat(dateFormat, "DateFormat");
- validateSortScope(sort_scope);
if (Boolean.parseBoolean(bad_records_logger_enable) ||
LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
@@ -217,12 +215,7 @@ public class CarbonLoadModelBuilder {
List<String> columnNames = new ArrayList<>();
List<String> partitionColumns = new ArrayList<>();
for (int i = 0; i < columns.size(); i++) {
- if (table.getPartitionInfo() != null && table.getPartitionInfo().getColumnSchemaList()
- .contains(columns.get(i).getColumnSchema())) {
- partitionColumns.add(columns.get(i).getColName());
- } else {
- columnNames.add(columns.get(i).getColName());
- }
+ columnNames.add(columns.get(i).getColName());
}
columnNames.addAll(partitionColumns);
fileHeader = Strings.mkString(columnNames.toArray(new String[columnNames.size()]), ",");
@@ -391,19 +384,6 @@ public class CarbonLoadModelBuilder {
}
}
- private void validateSortScope(String sortScope) throws InvalidLoadOptionException {
- if (sortScope != null) {
- // We support global sort for Hive standard partition, but don't support
- // global sort for other partition type.
- if (table.getPartitionInfo() != null &&
- !table.isHivePartitionTable() &&
- sortScope.equalsIgnoreCase(SortScopeOptions.SortScope.GLOBAL_SORT.toString())) {
- throw new InvalidLoadOptionException("Don't support use global sort on "
- + table.getPartitionInfo().getPartitionType() + " partition table.");
- }
- }
- }
-
private void validateGlobalSortPartitions(String globalSortPartitions)
throws InvalidLoadOptionException {
if (globalSortPartitions != null) {
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
index cafac7c..b9f1896 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
@@ -33,7 +33,6 @@ import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
import org.apache.carbondata.core.keygenerator.KeyGenException;
import org.apache.carbondata.core.localdictionary.generator.LocalDictionaryGenerator;
import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.util.CarbonThreadFactory;
import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
import org.apache.carbondata.core.util.CarbonUtil;
@@ -66,8 +65,6 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
private boolean[] isNoDictionaryDimensionColumn;
- private DataType[] measureDataType;
-
private int dimensionCount;
private int measureCount;
@@ -76,8 +73,6 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
private long[] writeCounter;
- private int outputLength;
-
private CarbonTableIdentifier tableIdentifier;
private String tableName;
@@ -125,9 +120,7 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
dimensionCount = configuration.getDimensionCount() - noDictWithComplextCount;
isNoDictionaryDimensionColumn =
CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
- measureDataType = configuration.getMeasureDataType();
measureCount = configuration.getMeasureCount();
- outputLength = measureCount + (this.noDictWithComplextCount > 0 ? 1 : 0) + 1;
CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
.recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PARTITION_ID,
System.currentTimeMillis());
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java b/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java
deleted file mode 100644
index 649c18d..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java
+++ /dev/null
@@ -1,35 +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.processing.partition;
-
-import java.util.List;
-
-public interface DataPartitioner {
-
- /**
- * All the partitions built by the Partitioner
- */
- List<Partition> getAllPartitions();
-
- /**
- * Identifies the partitions applicable for the given filter (API used for For query)
- */
- List<Partition> getPartitions();
-
-}
-
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java b/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java
deleted file mode 100644
index 95cdfb3..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java
+++ /dev/null
@@ -1,35 +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.processing.partition;
-
-import java.io.Serializable;
-import java.util.List;
-
-public interface Partition extends Serializable {
- /**
- * unique identification for the partition in the cluster.
- */
- String getUniqueID();
-
- /**
- * result
- *
- * @return
- */
- List<String> getFilesPath();
-}
\ No newline at end of file
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
deleted file mode 100644
index c03f34e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
+++ /dev/null
@@ -1,90 +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.processing.partition.spliter;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.block.TaskBlockInfo;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.scan.executor.QueryExecutor;
-import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
-import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.result.RowBatch;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.log4j.Logger;
-
-public abstract class AbstractCarbonQueryExecutor {
-
- private static final Logger LOGGER =
- LogServiceFactory.getLogService(AbstractCarbonQueryExecutor.class.getName());
- protected CarbonTable carbonTable;
- protected QueryModel queryModel;
- private QueryExecutor queryExecutor;
- Map<String, TaskBlockInfo> segmentMapping;
-
- /**
- * get executor and execute the query model.
- *
- * @param blockList
- * @return
- */
- CarbonIterator<RowBatch> executeBlockList(List<TableBlockInfo> blockList,
- Configuration configuration)
- throws QueryExecutionException, IOException {
- queryModel.setTableBlockInfos(blockList);
- this.queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel, configuration);
- return queryExecutor.execute(queryModel);
- }
-
- /**
- * Below method will be used
- * for cleanup
- */
- public void finish() {
- try {
- queryExecutor.finish();
- } catch (QueryExecutionException e) {
- LOGGER.error("Problem while finish: ", e);
- }
... 334 lines suppressed ...