carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbono...@apache.org
Subject [48/56] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into merge3
Date Thu, 23 Jun 2016 14:16:36 GMT
Merge remote-tracking branch 'carbon_master/master' into merge3

Conflicts:
	core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
	core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
	core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
	core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
	integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
	integration/spark/src/main/scala/org/carbondata/spark/agg/CarbonAggregates.scala
	integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
	integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
	integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
	integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/7972709b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/7972709b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/7972709b

Branch: refs/heads/master
Commit: 7972709bcf70a4278535b5195d7c2c3ae934ddc5
Parents: bf7557d 5e2e66f
Author: ravipesala <ravi.pesala@gmail.com>
Authored: Mon Jun 20 21:43:22 2016 +0530
Committer: ravipesala <ravi.pesala@gmail.com>
Committed: Mon Jun 20 21:43:22 2016 +0530

----------------------------------------------------------------------
 .../core/carbon/path/CarbonTablePath.java       |  20 +-
 .../core/constants/CarbonCommonConstants.java   |  27 +
 .../fileperations/AtomicFileOperationsImpl.java |   3 +-
 .../store/filesystem/AbstractDFSCarbonFile.java | 217 +++++++
 .../store/filesystem/HDFSCarbonFile.java        | 249 ++------
 .../store/filesystem/LocalCarbonFile.java       |   1 +
 .../store/filesystem/ViewFSCarbonFile.java      | 126 ++++
 .../store/impl/DFSFileHolderImpl.java           | 183 ++++++
 .../datastorage/store/impl/FileFactory.java     |  36 +-
 .../store/impl/HDFSFileHolderImpl.java          | 186 ------
 .../TimeStampDirectDictionaryGenerator.java     |  17 +-
 .../core/locks/CarbonLockFactory.java           |  10 +-
 .../carbondata/core/locks/ZooKeeperLocking.java |  53 +-
 .../carbondata/core/locks/ZookeeperInit.java    |  64 ++
 .../org/carbondata/core/util/CarbonUtil.java    |  23 +-
 .../aggregator/impl/AvgTimestampAggregator.java | 113 ++++
 .../aggregator/impl/SumTimestampAggregator.java |  89 +++
 .../DirectDictionaryDimensionAggregator.java    | 117 ++++
 .../resolver/ConditionalFilterResolverImpl.java |   9 +-
 .../core/locks/ZooKeeperLockingTest.java        |   9 +-
 .../org/carbondata/examples/CarbonExample.scala |   8 +-
 .../examples/GenerateDictionaryExample.scala    |   2 +-
 .../AbstractDictionaryDecodedReadSupport.java   |   8 +-
 .../carbondata/hadoop/util/SchemaReader.java    |   3 +-
 .../hadoop/ft/CarbonInputMapperTest.java        |  12 +-
 .../hadoop/test/util/StoreCreator.java          |   4 +-
 integration-testcases/pom.xml                   |   2 +-
 .../sql/common/util/CarbonHiveContext.scala     |   4 +-
 .../allqueries/AllDataTypesTestCase2.scala      |   9 +-
 .../allqueries/AllDataTypesTestCase4.scala      |  72 +--
 .../allqueries/AllDataTypesTestCase5.scala      |   9 +-
 .../allqueries/AllDataTypesTestCase6.scala      |  27 +-
 integration/spark/pom.xml                       |   2 +-
 .../spark/merger/CarbonCompactionExecutor.java  |   1 +
 .../spark/merger/CarbonCompactionUtil.java      |  33 --
 .../spark/merger/CompactionCallable.java        |  68 +++
 .../spark/merger/RowResultMerger.java           |  17 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |  20 +-
 .../spark/merger/CarbonDataMergerUtil.java      |  28 +-
 .../readsupport/SparkRowReadSupportImpl.java    |   7 +
 .../spark/sql/CarbonCatalystOperators.scala     |   2 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  46 +-
 .../execution/command/carbonTableSchema.scala   |  37 +-
 .../spark/sql/hive/CarbonMetastoreCatalog.scala |  11 +
 .../scala/org/carbondata/spark/KeyVal.scala     |   9 +-
 .../spark/csv/CarbonCsvRelation.scala           | 242 ++++++++
 .../carbondata/spark/csv/DefaultSource.scala    |  94 +--
 .../spark/rdd/CarbonDataLoadRDD.scala           |  20 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        | 245 +++++---
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  27 +-
 .../org/carbondata/spark/rdd/Compactor.scala    |  15 +-
 .../spark/util/GlobalDictionaryUtil.scala       |  53 +-
 .../src/test/resources/InvalidCsvFormatdata.csv |   3 -
 .../spark/src/test/resources/avgTest.csv        |  30 +-
 .../test/resources/compaction/compaction1.csv   |  10 +-
 .../test/resources/compaction/compaction2.csv   |  10 +-
 .../test/resources/compaction/compaction3.csv   |  10 +-
 integration/spark/src/test/resources/data2.csv  |   4 +-
 .../spark/src/test/resources/datasamplenull.csv |   3 +
 .../TestLoadDataWithEmptyArrayColumns.scala     |  11 +-
 .../AllDataTypesTestCaseAggregate.scala         |   4 +
 .../aggquery/AverageQueryTestCase.scala         |   7 +-
 .../AllDataTypesTestCaseAggregate.scala         |  10 +-
 .../testsuite/bigdecimal/TestBigDecimal.scala   |   7 +
 .../spark/testsuite/bigdecimal/TestBigInt.scala |   7 +
 .../DataCompactionCardinalityBoundryTest.scala  |  10 +-
 .../DataCompactionNoDictionaryTest.scala        |  29 +-
 .../datacompaction/DataCompactionTest.scala     |  12 +-
 .../dataload/DefaultSourceTestCase.scala        |  25 +-
 .../TestDataWithDicExcludeAndInclude.scala      |   9 +-
 .../dataload/TestLoadDataWithHiveSyntax.scala   |   8 +-
 .../dataload/TestLoadTblNameIsKeyword.scala     |  96 +++
 .../dataretention/DataRetentionTestCase.scala   |  11 +-
 .../TimestampDataTypeNullDataTest.scala         |  92 +++
 .../filterexpr/FilterProcessorTestCase.scala    |   9 +
 .../NullMeasureValueTestCaseAggregate.scala     |  14 +-
 .../csvreaderstep/BlockDataHandler.java         | 579 +++++++++----------
 .../processing/csvreaderstep/CsvInput.java      |  10 +-
 .../processing/csvreaderstep/CsvInputMeta.java  |  66 +--
 .../graphgenerator/GraphGenerator.java          |  14 +-
 .../processing/mdkeygen/MDKeyGenStep.java       |   2 +-
 .../sortandgroupby/sortdata/SortDataRows.java   |   2 +-
 .../store/CarbonFactDataHandlerColumnar.java    |  30 +-
 .../store/CarbonFactDataHandlerModel.java       |  22 +
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |  29 +-
 .../FileStoreSurrogateKeyGenForCSV.java         |   2 +-
 .../util/CarbonDataProcessorUtil.java           |  48 +-
 87 files changed, 2620 insertions(+), 1304 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
index d46725f,060adcb..771e5f5
--- a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
@@@ -124,9 -122,10 +127,10 @@@ public class TimeStampDirectDictionaryG
      try {
        dateToStr = timeParser.parse(memberStr);
      } catch (ParseException e) {
 -      LOGGER.error(
 -          "Cannot convert" + TIMESTAMP.toString() + " to Time/Long type value" + e.getMessage());
 +      LOGGER.error("Cannot convert" + memberStr + " to Time/Long type value"
 +          + e.getMessage());
      }
+     //adding +2 to reserve the first cuttOffDiff value for null or empty date
      if (null == dateToStr) {
        return -1;
      } else {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
index 1fb595a,0000000..072d527
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
@@@ -1,239 -1,0 +1,242 @@@
 +/*
 + * 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.carbondata.scan.filter.resolver;
 +
 +import java.util.List;
 +import java.util.SortedMap;
 +
 +import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 +import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 +import org.carbondata.core.carbon.metadata.encoder.Encoding;
++import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 +import org.carbondata.scan.executor.exception.QueryExecutionException;
 +import org.carbondata.scan.expression.ColumnExpression;
 +import org.carbondata.scan.expression.DataType;
 +import org.carbondata.scan.expression.Expression;
 +import org.carbondata.scan.expression.conditional.BinaryConditionalExpression;
 +import org.carbondata.scan.expression.conditional.ConditionalExpression;
 +import org.carbondata.scan.expression.exception.FilterUnsupportedException;
 +import org.carbondata.scan.filter.FilterUtil;
 +import org.carbondata.scan.filter.intf.FilterExecuterType;
 +import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
 +import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 +import org.carbondata.scan.filter.resolver.resolverinfo.visitor.FilterInfoTypeVisitorFactory;
 +
 +public class ConditionalFilterResolverImpl implements FilterResolverIntf {
 +
 +  private static final long serialVersionUID = 1838955268462201691L;
 +  protected Expression exp;
 +  protected boolean isExpressionResolve;
 +  protected boolean isIncludeFilter;
 +  private DimColumnResolvedFilterInfo dimColResolvedFilterInfo;
 +
 +  public ConditionalFilterResolverImpl(Expression exp, boolean isExpressionResolve,
 +      boolean isIncludeFilter) {
 +    this.exp = exp;
 +    this.isExpressionResolve = isExpressionResolve;
 +    this.isIncludeFilter = isIncludeFilter;
 +    this.dimColResolvedFilterInfo = new DimColumnResolvedFilterInfo();
 +  }
 +
 +  /**
 +   * This API will resolve the filter expression and generates the
 +   * dictionaries for executing/evaluating the filter expressions in the
 +   * executer layer.
 +   *
 +   * @throws QueryExecutionException
 +   * @throws FilterUnsupportedException
 +   */
 +  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier)
 +      throws FilterUnsupportedException {
 +    FilterResolverMetadata metadata = new FilterResolverMetadata();
 +    metadata.setTableIdentifier(absoluteTableIdentifier);
 +    if ((!isExpressionResolve) && exp instanceof BinaryConditionalExpression) {
 +      BinaryConditionalExpression binaryConditionalExpression = (BinaryConditionalExpression)
exp;
 +      Expression leftExp = binaryConditionalExpression.getLeft();
 +      Expression rightExp = binaryConditionalExpression.getRight();
 +      if (leftExp instanceof ColumnExpression) {
 +        ColumnExpression columnExpression = (ColumnExpression) leftExp;
 +        metadata.setColumnExpression(columnExpression);
 +        metadata.setExpression(rightExp);
 +        metadata.setIncludeFilter(isIncludeFilter);
 +        // If imei=imei comes in filter condition then we need to
 +        // skip processing of right expression.
 +        // This flow has reached here assuming that this is a single
 +        // column expression.
 +        // we need to check if the other expression contains column
 +        // expression or not in depth.
-         if (FilterUtil.checkIfExpressionContainsColumn(rightExp)||
-             FilterUtil.isExpressionNeedsToResolved(rightExp,isIncludeFilter) &&
-             columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)){
++        CarbonDimension dimension = columnExpression.getDimension();
++        if (FilterUtil.checkIfExpressionContainsColumn(rightExp)
++            || FilterUtil.isExpressionNeedsToResolved(rightExp, isIncludeFilter) &&
++            dimension.hasEncoding(Encoding.DICTIONARY) && !dimension
++            .hasEncoding(Encoding.DIRECT_DICTIONARY)) {
 +          isExpressionResolve = true;
 +        } else {
 +          //Visitor pattern is been used in this scenario inorder to populate the
 +          // dimColResolvedFilterInfo
 +          //visitable object with filter member values based on the visitor type, currently
there
 +          //3 types of visitors custom,direct and no dictionary, all types of visitor populate
 +          //the visitable instance as per its buisness logic which is different for all
the
 +          // visitors.
 +          dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
 +              FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
 +              metadata);
 +        }
 +      } else if (rightExp instanceof ColumnExpression) {
 +        ColumnExpression columnExpression = (ColumnExpression) rightExp;
 +        metadata.setColumnExpression(columnExpression);
 +        metadata.setExpression(leftExp);
 +        metadata.setIncludeFilter(isIncludeFilter);
 +        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
 +          isExpressionResolve = true;
 +        } else {
 +          // if imei=imei comes in filter condition then we need to
 +          // skip processing of right expression.
 +          // This flow has reached here assuming that this is a single
 +          // column expression.
 +          // we need to check if the other expression contains column
 +          // expression or not in depth.
 +          if (FilterUtil.checkIfExpressionContainsColumn(leftExp)) {
 +            isExpressionResolve = true;
 +          } else {
 +
 +            dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
 +                FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
 +                metadata);
 +
 +          }
 +        }
 +      } else {
 +        isExpressionResolve = true;
 +      }
 +    }
 +    if (isExpressionResolve && exp instanceof ConditionalExpression) {
 +      ConditionalExpression conditionalExpression = (ConditionalExpression) exp;
 +      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
 +      metadata.setColumnExpression(columnList.get(0));
 +      metadata.setExpression(exp);
 +      metadata.setIncludeFilter(isIncludeFilter);
 +      if (!columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) || columnList.get(0)
 +          .getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
 +        dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
 +            FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnList.get(0)),
metadata);
 +
 +      } else if (columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) &&
!(
 +          columnList.get(0).getDimension().getDataType()
 +              == org.carbondata.core.carbon.metadata.datatype.DataType.STRUCT
 +              || columnList.get(0).getDimension().getDataType()
 +              == org.carbondata.core.carbon.metadata.datatype.DataType.ARRAY)) {
 +        dimColResolvedFilterInfo.setFilterValues(FilterUtil
 +            .getFilterListForAllValues(absoluteTableIdentifier, exp, columnList.get(0),
 +                isIncludeFilter));
 +
 +        dimColResolvedFilterInfo.setColumnIndex(columnList.get(0).getDimension().getOrdinal());
 +        dimColResolvedFilterInfo.setDimension(columnList.get(0).getDimension());
 +      }
 +    }
 +
 +  }
 +
 +  /**
 +   * Left node will not be presentin this scenario
 +   *
 +   * @return left node of type FilterResolverIntf instance
 +   */
 +  public FilterResolverIntf getLeft() {
 +    // TODO Auto-generated method stub
 +    return null;
 +  }
 +
 +  /**
 +   * Right node will not be presentin this scenario
 +   *
 +   * @return left node of type FilterResolverIntf instance
 +   */
 +  @Override public FilterResolverIntf getRight() {
 +    // TODO Auto-generated method stub
 +    return null;
 +  }
 +
 +  /**
 +   * Method will return the DimColumnResolvedFilterInfo instance which consists
 +   * the mapping of the respective dimension and its surrogates involved in
 +   * filter expression.
 +   *
 +   * @return DimColumnResolvedFilterInfo
 +   */
 +  public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
 +    return dimColResolvedFilterInfo;
 +  }
 +
 +  /**
 +   * method will calculates the start key based on the filter surrogates
 +   */
 +  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
 +      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
 +    if (null == dimColResolvedFilterInfo.getStarIndexKey()) {
 +      FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
 +          setOfStartKeyByteArray);
 +    }
 +  }
 +
 +  /**
 +   * method will get the start key based on the filter surrogates
 +   *
 +   * @return end IndexKey
 +   */
 +  @Override public void getEndKey(SegmentProperties segmentProperties,
 +      AbsoluteTableIdentifier absoluteTableIdentifier, long[] endKeys,
 +      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
 +    if (null == dimColResolvedFilterInfo.getEndIndexKey()) {
 +      try {
 +        FilterUtil.getEndKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
 +            absoluteTableIdentifier, endKeys, segmentProperties);
 +        FilterUtil.getEndKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
 +            setOfEndKeyByteArray);
 +      } catch (QueryExecutionException e) {
 +        // TODO Auto-generated catch block
 +        e.printStackTrace();
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Method will return the executer type for particular conditional resolver
 +   * basically two types of executers will be formed for the conditional query.
 +   *
 +   * @return the filter executer type
 +   */
 +  @Override public FilterExecuterType getFilterExecuterType() {
 +    switch (exp.getFilterExpressionType()) {
 +      case NOT_EQUALS:
 +      case NOT_IN:
 +        return FilterExecuterType.EXCLUDE;
 +
 +      default:
 +        return FilterExecuterType.INCLUDE;
 +    }
 +
 +  }
 +
 +  @Override public Expression getFilterExpression() {
 +    // TODO Auto-generated method stub
 +    return exp;
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
----------------------------------------------------------------------
diff --cc examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
index a549409,2d37bda..db05a51
--- a/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
+++ b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
@@@ -63,9 -63,9 +63,9 @@@ object GenerateDictionaryExample 
      val tableName = carbonTableIdentifier.getTableName
      val carbonRelation = CarbonEnv.getInstance(carbonContext).carbonCatalog.
        lookupRelation1(Option(dataBaseName),
 -        tableName, None) (carbonContext).asInstanceOf[CarbonRelation]
 +        tableName) (carbonContext).asInstanceOf[CarbonRelation]
      val carbonTable = carbonRelation.cubeMeta.carbonTable
-     val dimensions = carbonTable.getDimensionByTableName(tableName)
+     val dimensions = carbonTable.getDimensionByTableName(tableName.toLowerCase())
        .toArray.map(_.asInstanceOf[CarbonDimension])
      // scalastyle:off println
      // print dictionary information

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputMapperTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
----------------------------------------------------------------------
diff --cc integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
index 88ba722,14824ce..da328a1
--- a/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
+++ b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
@@@ -113,9 -93,9 +119,10 @@@ class AllDataTypesTestCase2 extends Que
    }
  
    override def afterAll {
+     //CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
"dd-MM-yyyy")
      try {
        sql("drop cube Carbon_automation_test2")
 +      sql("drop table Carbon_automation_hive2")
      } catch {
        case e: Exception => print("ERROR: DROP Carbon_automation_test2 ")
      }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase6.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
----------------------------------------------------------------------
diff --cc integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
index 85a3418,48b3668..9cf0ea7
--- a/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
@@@ -3,9 -3,11 +3,11 @@@ package org.carbondata.spark.readsuppor
  import java.sql.Timestamp;
  
  import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+ import org.carbondata.core.carbon.metadata.datatype.DataType;
+ import org.carbondata.core.carbon.metadata.encoder.Encoding;
  import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
  import org.carbondata.hadoop.readsupport.impl.AbstractDictionaryDecodedReadSupport;
 -import org.carbondata.query.carbon.util.DataTypeUtil;
 +import org.carbondata.scan.util.DataTypeUtil;
  
  import org.apache.spark.sql.Row;
  import org.apache.spark.sql.catalyst.expressions.GenericRow;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index e36d148,655141c..3f4dac5
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@@ -25,9 -25,10 +25,11 @@@ import java.util.UUI
  import scala.collection.JavaConverters._
  import scala.collection.mutable.ArrayBuffer
  import scala.language.implicitConversions
+ import scala.util.Random
  
+ import org.apache.spark.SparkEnv
  import org.apache.spark.sql._
 +import org.apache.spark.sql.catalyst.TableIdentifier
  import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
  import org.apache.spark.sql.execution.{RunnableCommand, SparkPlan}
  import org.apache.spark.sql.hive.HiveContext
@@@ -1395,16 -1412,15 +1398,16 @@@ private[sql] case class LoadTable
  
    def run(sqlContext: SQLContext): Seq[Row] = {
  
 -    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
 +    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
 +    val identifier = TableIdentifier(tableName, Option(dbName))
      if (isOverwriteExist) {
 -      sys.error("Overwrite is not supported for carbon table with " + schemaName + "." +
tableName)
 +      sys.error("Overwrite is not supported for carbon table with " + dbName + "." + tableName)
      }
      if (null == org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
 -      .getCarbonTable(schemaName + "_" + tableName)) {
 -      logError("Data loading failed. table not found: " + schemaName + "." + tableName)
 -      LOGGER.audit("Data loading failed. table not found: " + schemaName + "." + tableName)
 -      sys.error("Data loading failed. table not found: " + schemaName + "." + tableName)
 +      .getCarbonTable(dbName + "_" + tableName)) {
-       logError("Data loading failed. table not found: " + dbName + "_" + tableName)
-       LOGGER.audit("Data loading failed. table not found: " + dbName + "_" + tableName)
-       sys.error("Data loading failed. table not found: " + dbName + "_" + tableName)
++      logError("Data loading failed. table not found: " + dbName + "." + tableName)
++      LOGGER.audit("Data loading failed. table not found: " + dbName + "." + tableName)
++      sys.error("Data loading failed. table not found: " + dbName + "." + tableName)
      }
      CarbonProperties.getInstance().addProperty("zookeeper.enable.lock", "false")
      val carbonLock = CarbonLockFactory.getCarbonLockObj(org.carbondata.core.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
index cb87818,66bb0c0..52103be
--- a/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
@@@ -26,30 -26,33 +26,29 @@@
  package org.carbondata.spark
  
  import org.carbondata.core.load.LoadMetadataDetails
 -import org.carbondata.query.carbon.result.BatchRawResult
 -import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
 -
 -trait KeyVal[K, V] extends Serializable {
 -  def getKey(key: CarbonKey, value: CarbonValue): (K, V)
  
 +trait Value[V] extends Serializable {
 +  def getValue(value: Array[Object]): V
  }
  
 -class KeyValImpl extends KeyVal[CarbonKey, CarbonValue] {
 -  override def getKey(key: CarbonKey, value: CarbonValue): (CarbonKey, CarbonValue) = (key,
value)
 +class ValueImpl extends Value[Array[Object]] {
 +  override def getValue(value: Array[Object]): Array[Object] = value
  }
  
 -trait RawKeyVal[K, V] extends Serializable {
 -  def getKey(key: BatchRawResult, value: Any): (K, V)
 -
 +trait RawValue[V] extends Serializable {
 +  def getValue(value: Array[Any]): V
  }
  
 -class RawKeyValImpl extends RawKeyVal[BatchRawResult, Any] {
 -  override def getKey(key: BatchRawResult, value: Any): (BatchRawResult, Any) = (key, value)
 +class RawValueImpl extends RawValue[Array[Any]] {
 +  override def getValue(value: Array[Any]): Array[Any] = value
  }
  
- trait Result[K, V] extends Serializable {
-   def getKey(key: Int, value: LoadMetadataDetails): (K, V)
- 
+ trait DataLoadResult[K, V] extends Serializable {
+   def getKey(key: String, value: LoadMetadataDetails): (K, V)
  }
  
- class ResultImpl extends Result[Int, LoadMetadataDetails] {
-   override def getKey(key: Int, value: LoadMetadataDetails): (Int, LoadMetadataDetails)
= {
+ class DataLoadResultImpl extends DataLoadResult[String, LoadMetadataDetails] {
+   override def getKey(key: String, value: LoadMetadataDetails): (String, LoadMetadataDetails)
= {
      (key, value)
    }
  }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index af2271f,839a02e..87b0952
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@@ -40,11 -41,12 +41,11 @@@ import org.carbondata.core.carbon.metad
  import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
  import org.carbondata.core.constants.CarbonCommonConstants
  import org.carbondata.core.load.{BlockDetails, LoadMetadataDetails}
- import org.carbondata.core.locks.{CarbonLockFactory, LockUsage}
- import org.carbondata.core.util.CarbonUtil
- import org.carbondata.integration.spark.merger.CompactionType
+ import org.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
+ import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
+ import org.carbondata.integration.spark.merger.{CompactionCallable, CompactionType}
  import org.carbondata.lcm.status.SegmentStatusManager
  import org.carbondata.processing.util.CarbonDataProcessorUtil
 -import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
  import org.carbondata.spark._
  import org.carbondata.spark.load._
  import org.carbondata.spark.merger.CarbonDataMergerUtil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
index e18b64d,c418216..c1361f5
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
@@@ -34,9 -34,10 +34,9 @@@ import org.carbondata.core.carbon.metad
  import org.carbondata.core.constants.CarbonCommonConstants
  import org.carbondata.core.util.CarbonProperties
  import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
- import org.carbondata.integration.spark.merger.{CarbonCompactionExecutor, CarbonCompactionUtil,
- RowResultMerger}
+ import org.carbondata.integration.spark.merger.{CarbonCompactionExecutor, CarbonCompactionUtil,
RowResultMerger}
+ import org.carbondata.processing.util.CarbonDataProcessorUtil
 -import org.carbondata.query.carbon.result.RowResult
 -import org.carbondata.query.carbon.result.iterator.RawResultIterator
 +import org.carbondata.scan.result.iterator.RawResultIterator
  import org.carbondata.spark.MergeResult
  import org.carbondata.spark.load.{CarbonLoaderUtil, CarbonLoadModel}
  import org.carbondata.spark.merger.CarbonDataMergerUtil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
index c2a2277,3b389a6..b6bda0c
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
@@@ -37,18 -39,18 +39,22 @@@ class AllDataTypesTestCaseAggregate ext
  
      val currentDirectory = new File(this.getClass.getResource("/").getPath + "/../../")
        .getCanonicalPath
-     CarbonProperties.getInstance().addProperty("carbon.direct.surrogate","false")
+ 
      sql("create cube Carbon_automation_test dimensions(imei string,deviceInformationId integer,MAC
string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize
string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode
string,internalModels string, deliveryTime string, channelsId string, channelsName string
, deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict
string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId
string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string,
ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion
string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string,
Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string,
Active_operatorsVersion string, Active
 _phonePADPartitionedVersions string, Latest_YEAR integer, Latest_MONTH integer, Latest_DAY
integer, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province
string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId
string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string,
Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber
string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId
string, gamePointDescription string)  measures(gamePointId integer,contractNumber integer)
OPTIONS (PARTITIONER [CLASS = 'org.carbondata.spark.partition.api.impl.SampleDataPartitionerImpl'
,COLUMNS= (imei) , PARTITION_COUNT=2] )");
+     CarbonProperties.getInstance()
+       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
      sql("LOAD DATA FACT FROM'"+currentDirectory+"/src/test/resources/100_olap.csv' INTO
Cube Carbon_automation_test partitionData(DELIMITER ',' ,QUOTECHAR '\"', FILEHEADER 'imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUI
 Version,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')");
 +    sql("create table if not exists Carbon_automation_hive (imei string,deviceInformationId
int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize
string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode
string,internalModels string, deliveryTime string, channelsId string, channelsName string
, deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict
string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId
string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string,
ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion
string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string,
Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string,
Active_operatorsVersion string, Activ
 e_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int,
Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city
string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion
string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string,
Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string,
Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId
string, gamePointDescription string, gamePointId int,contractNumber int) row format delimited
fields terminated by ','");
 +    sql("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/100_olap.csv' INTO
table Carbon_automation_hive ");
  
    }
  
    override def afterAll {
      sql("drop cube Carbon_automation_test")
 +    sql("drop cube Carbon_automation_hive")
 +
+     CarbonProperties.getInstance()
+       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
    }
  
    //Test-22



Mime
View raw message