Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id ED0A1200B67 for ; Tue, 2 Aug 2016 03:12:13 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id EBAE8160AA7; Tue, 2 Aug 2016 01:12:13 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id EAA80160A6C for ; Tue, 2 Aug 2016 03:12:11 +0200 (CEST) Received: (qmail 95769 invoked by uid 500); 2 Aug 2016 01:12:11 -0000 Mailing-List: contact commits-help@carbondata.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@carbondata.incubator.apache.org Delivered-To: mailing list commits@carbondata.incubator.apache.org Received: (qmail 95760 invoked by uid 99); 2 Aug 2016 01:12:11 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 02 Aug 2016 01:12:11 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 9A8B1C0B9A for ; Tue, 2 Aug 2016 01:12:10 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -4.646 X-Spam-Level: X-Spam-Status: No, score=-4.646 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-1.426] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id Fd6JLub2xqgB for ; Tue, 2 Aug 2016 01:11:59 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id 1C8485F248 for ; Tue, 2 Aug 2016 01:11:56 +0000 (UTC) Received: (qmail 95653 invoked by uid 99); 2 Aug 2016 01:11:56 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 02 Aug 2016 01:11:56 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 0D396E058E; Tue, 2 Aug 2016 01:11:56 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: chenliang613@apache.org To: commits@carbondata.incubator.apache.org Date: Tue, 02 Aug 2016 01:11:56 -0000 Message-Id: <6928e19ff7874c42bf086d9540a030e4@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/3] incubator-carbondata git commit: Refactored code to improve performance by using removing unnecessary conversion. archived-at: Tue, 02 Aug 2016 01:12:14 -0000 Repository: incubator-carbondata Updated Branches: refs/heads/master 645eafadb -> 29f9cf226 http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java b/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java index 0f9c339..e58e830 100644 --- a/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java +++ b/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java @@ -23,7 +23,7 @@ import org.carbondata.common.logging.LogService; import org.carbondata.common.logging.LogServiceFactory; import org.carbondata.core.carbon.datastore.block.SegmentProperties; import org.carbondata.core.keygenerator.KeyGenException; -import org.carbondata.scan.result.BatchRawResult; +import org.carbondata.scan.result.BatchResult; import org.carbondata.scan.wrappers.ByteArrayWrapper; /** @@ -39,7 +39,7 @@ public class RawResultIterator extends CarbonIterator { /** * Iterator of the Batch raw result. */ - private CarbonIterator detailRawQueryResultIterator; + private CarbonIterator detailRawQueryResultIterator; /** * Counter to maintain the row counter. @@ -57,9 +57,9 @@ public class RawResultIterator extends CarbonIterator { /** * batch of the result. */ - private BatchRawResult batch; + private BatchResult batch; - public RawResultIterator(CarbonIterator detailRawQueryResultIterator, + public RawResultIterator(CarbonIterator detailRawQueryResultIterator, SegmentProperties sourceSegProperties, SegmentProperties destinationSegProperties) { this.detailRawQueryResultIterator = detailRawQueryResultIterator; this.sourceSegProperties = sourceSegProperties; @@ -157,7 +157,7 @@ public class RawResultIterator extends CarbonIterator { * @param batch * @return */ - private boolean checkIfBatchIsProcessedCompletely(BatchRawResult batch){ + private boolean checkIfBatchIsProcessedCompletely(BatchResult batch){ if(counter < batch.getSize()) { return false; http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/preparator/QueryResultPreparator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/carbondata/scan/result/preparator/QueryResultPreparator.java b/core/src/main/java/org/carbondata/scan/result/preparator/QueryResultPreparator.java deleted file mode 100644 index 7ef5b6d..0000000 --- a/core/src/main/java/org/carbondata/scan/result/preparator/QueryResultPreparator.java +++ /dev/null @@ -1,10 +0,0 @@ -package org.carbondata.scan.result.preparator; - -import org.carbondata.scan.result.BatchResult; -import org.carbondata.scan.result.Result; - -public interface QueryResultPreparator { - - public BatchResult prepareQueryResult(Result scannedResult); - -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/preparator/impl/AbstractQueryResultPreparator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/carbondata/scan/result/preparator/impl/AbstractQueryResultPreparator.java b/core/src/main/java/org/carbondata/scan/result/preparator/impl/AbstractQueryResultPreparator.java deleted file mode 100644 index fd6f1c8..0000000 --- a/core/src/main/java/org/carbondata/scan/result/preparator/impl/AbstractQueryResultPreparator.java +++ /dev/null @@ -1,87 +0,0 @@ -package org.carbondata.scan.result.preparator.impl; - -import java.util.List; - -import org.carbondata.core.carbon.metadata.encoder.Encoding; -import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator; -import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory; -import org.carbondata.core.util.CarbonUtil; -import org.carbondata.core.util.DataTypeUtil; -import org.carbondata.scan.executor.impl.QueryExecutorProperties; -import org.carbondata.scan.model.QueryDimension; -import org.carbondata.scan.model.QueryModel; -import org.carbondata.scan.result.BatchResult; -import org.carbondata.scan.result.preparator.QueryResultPreparator; - -public abstract class AbstractQueryResultPreparator implements QueryResultPreparator { - - /** - * query properties - */ - protected QueryExecutorProperties queryExecuterProperties; - - /** - * query model - */ - protected QueryModel queryModel; - - public AbstractQueryResultPreparator(QueryExecutorProperties executerProperties, - QueryModel queryModel) { - this.queryExecuterProperties = executerProperties; - this.queryModel = queryModel; - } - - protected void fillDimensionData(Object[][] convertedResult, List queryDimensions, - int dimensionCount, Object[] row, int rowIndex) { - QueryDimension queryDimension; - for (int i = 0; i < dimensionCount; i++) { - queryDimension = queryDimensions.get(i); - if (!CarbonUtil - .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)) { - row[queryDimension.getQueryOrder()] = convertedResult[i][rowIndex]; - } else if (CarbonUtil - .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DIRECT_DICTIONARY)) { - DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory - .getDirectDictionaryGenerator(queryDimension.getDimension().getDataType()); - row[queryDimension.getQueryOrder()] = directDictionaryGenerator - .getValueFromSurrogate((Integer) convertedResult[i][rowIndex]); - } else { - if (queryExecuterProperties.sortDimIndexes[i] == 1) { - row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType( - queryExecuterProperties.columnToDictionayMapping - .get(queryDimension.getDimension().getColumnId()) - .getDictionaryValueFromSortedIndex((Integer) convertedResult[i][rowIndex]), - queryDimension.getDimension().getDataType()); - } else { - row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType( - queryExecuterProperties.columnToDictionayMapping - .get(queryDimension.getDimension().getColumnId()) - .getDictionaryValueForKey((Integer) convertedResult[i][rowIndex]), - queryDimension.getDimension().getDataType()); - } - } - } - } - - protected Object[][] encodeToRows(Object[][] data) { - if (data.length == 0) { - return data; - } - Object[][] rData = new Object[data[0].length][data.length]; - int len = data.length; - for (int i = 0; i < rData.length; i++) { - for (int j = 0; j < len; j++) { - rData[i][j] = data[j][i]; - } - } - return rData; - } - - protected BatchResult getEmptyChunkResult(int size) { - Object[][] row = new Object[size][1]; - BatchResult chunkResult = new BatchResult(); - chunkResult.setRows(row); - return chunkResult; - } - -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/preparator/impl/DetailQueryResultPreparatorImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/carbondata/scan/result/preparator/impl/DetailQueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/scan/result/preparator/impl/DetailQueryResultPreparatorImpl.java deleted file mode 100644 index 8a7d610..0000000 --- a/core/src/main/java/org/carbondata/scan/result/preparator/impl/DetailQueryResultPreparatorImpl.java +++ /dev/null @@ -1,148 +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.carbondata.scan.result.preparator.impl; - -import java.nio.ByteBuffer; -import java.nio.charset.Charset; -import java.util.List; - -import org.carbondata.common.logging.LogService; -import org.carbondata.common.logging.LogServiceFactory; -import org.carbondata.core.carbon.metadata.datatype.DataType; -import org.carbondata.core.carbon.metadata.encoder.Encoding; -import org.carbondata.core.constants.CarbonCommonConstants; -import org.carbondata.core.util.CarbonUtil; -import org.carbondata.core.util.DataTypeUtil; -import org.carbondata.scan.executor.impl.QueryExecutorProperties; -import org.carbondata.scan.model.QueryDimension; -import org.carbondata.scan.model.QueryMeasure; -import org.carbondata.scan.model.QueryModel; -import org.carbondata.scan.result.BatchResult; -import org.carbondata.scan.result.ListBasedResultWrapper; -import org.carbondata.scan.result.Result; -import org.carbondata.scan.wrappers.ByteArrayWrapper; - -/** - * Below class will be used to get the result by converting to actual data - * Actual data conversion can be converting the surrogate key to actual data - * - * @TODO there are many things in class which is very confusing, need to check - * why it was handled like that and how we can handle that in a better - * way.Need to revisit this class. IF aggregation is push down to spark - * layer and if we can process the data in byte array format then this - * class wont be useful so in future we can delete this class. - * @TODO need to expose one interface which will return the result based on required type - * for example its implementation case return converted result or directly result with out - * converting to actual value - */ -public class DetailQueryResultPreparatorImpl - extends AbstractQueryResultPreparator, Object> { - - private static final LogService LOGGER = - LogServiceFactory.getLogService(DetailQueryResultPreparatorImpl.class.getName()); - - public DetailQueryResultPreparatorImpl(QueryExecutorProperties executerProperties, - QueryModel queryModel) { - super(executerProperties, queryModel); - } - - @Override public BatchResult prepareQueryResult( - Result, Object> scannedResult) { - if ((null == scannedResult || scannedResult.size() < 1)) { - return new BatchResult(); - } - List queryDimension = queryModel.getQueryDimension(); - int dimensionCount = queryDimension.size(); - int totalNumberOfColumn = dimensionCount + queryExecuterProperties.measureDataTypes.length; - Object[][] resultData = new Object[scannedResult.size()][totalNumberOfColumn]; - if (!queryExecuterProperties.isFunctionQuery && totalNumberOfColumn == 0 - && scannedResult.size() > 0) { - return getEmptyChunkResult(scannedResult.size()); - } - int currentRow = 0; - long[] surrogateResult = null; - int noDictionaryColumnIndex = 0; - int complexTypeColumnIndex = 0; - ByteArrayWrapper key = null; - Object[] value = null; - while (scannedResult.hasNext()) { - key = scannedResult.getKey(); - value = scannedResult.getValue(); - if (key != null) { - surrogateResult = queryExecuterProperties.keyStructureInfo.getKeyGenerator() - .getKeyArray(key.getDictionaryKey(), - queryExecuterProperties.keyStructureInfo.getMaskedBytes()); - for (int i = 0; i < dimensionCount; i++) { - if (!CarbonUtil.hasEncoding(queryDimension.get(i).getDimension().getEncoder(), - Encoding.DICTIONARY)) { - resultData[currentRow][i] = DataTypeUtil.getDataBasedOnDataType( - new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++), - Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)), - queryDimension.get(i).getDimension().getDataType()); - } else if (CarbonUtil.hasDataType(queryDimension.get(i).getDimension().getDataType(), - new DataType[] { DataType.ARRAY, DataType.STRUCT })) { - resultData[currentRow][i] = queryExecuterProperties.complexDimensionInfoMap - .get(queryDimension.get(i).getDimension().getOrdinal()) - .getDataBasedOnDataTypeFromSurrogates( - ByteBuffer.wrap(key.getComplexTypeByIndex(complexTypeColumnIndex++))); - } else { - resultData[currentRow][i] = - (int) surrogateResult[queryDimension.get(i).getDimension().getKeyOrdinal()]; - } - } - } - if (value != null) { - System.arraycopy(value, 0, resultData[currentRow], dimensionCount, - queryExecuterProperties.measureDataTypes.length); - } - currentRow++; - noDictionaryColumnIndex = 0; - } - if (resultData.length > 0) { - resultData = encodeToRows(resultData); - } - return getResult(queryModel, resultData); - } - - private BatchResult getResult(QueryModel queryModel, Object[][] convertedResult) { - - int rowSize = convertedResult[0].length; - Object[][] rows = new Object[rowSize][]; - List queryDimensions = queryModel.getQueryDimension(); - int dimensionCount = queryDimensions.size(); - int msrCount = queryExecuterProperties.measureDataTypes.length; - Object[] row; - for (int rowIndex = 0; rowIndex < rowSize; rowIndex++) { - row = new Object[dimensionCount + msrCount]; - fillDimensionData(convertedResult, queryDimensions, dimensionCount, row, rowIndex); - - QueryMeasure msr; - for (int i = 0; i < queryModel.getQueryMeasures().size(); i++) { - msr = queryModel.getQueryMeasures().get(i); - row[msr.getQueryOrder()] = convertedResult[dimensionCount + i][rowIndex]; - } - rows[rowIndex] = row; - } - LOGGER.info( - "###########################################------ Total Number of records" + rowSize); - BatchResult chunkResult = new BatchResult(); - chunkResult.setRows(rows); - return chunkResult; - } -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/core/src/main/java/org/carbondata/scan/result/preparator/impl/RawQueryResultPreparatorImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/carbondata/scan/result/preparator/impl/RawQueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/scan/result/preparator/impl/RawQueryResultPreparatorImpl.java deleted file mode 100644 index 161b58e..0000000 --- a/core/src/main/java/org/carbondata/scan/result/preparator/impl/RawQueryResultPreparatorImpl.java +++ /dev/null @@ -1,136 +0,0 @@ -package org.carbondata.scan.result.preparator.impl; - -import java.nio.ByteBuffer; -import java.util.List; - -import org.carbondata.common.logging.LogService; -import org.carbondata.common.logging.LogServiceFactory; -import org.carbondata.core.carbon.metadata.encoder.Encoding; -import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator; -import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory; -import org.carbondata.core.util.CarbonUtil; -import org.carbondata.core.util.DataTypeUtil; -import org.carbondata.scan.executor.impl.QueryExecutorProperties; -import org.carbondata.scan.model.QueryDimension; -import org.carbondata.scan.model.QueryMeasure; -import org.carbondata.scan.model.QueryModel; -import org.carbondata.scan.model.QuerySchemaInfo; -import org.carbondata.scan.result.BatchRawResult; -import org.carbondata.scan.result.BatchResult; -import org.carbondata.scan.result.ListBasedResultWrapper; -import org.carbondata.scan.result.Result; -import org.carbondata.scan.wrappers.ByteArrayWrapper; - -/** - * It does not decode the dictionary. - */ -public class RawQueryResultPreparatorImpl - extends AbstractQueryResultPreparator, Object> { - - private static final LogService LOGGER = - LogServiceFactory.getLogService(RawQueryResultPreparatorImpl.class.getName()); - - private QuerySchemaInfo querySchemaInfo; - - public RawQueryResultPreparatorImpl(QueryExecutorProperties executerProperties, - QueryModel queryModel) { - super(executerProperties, queryModel); - querySchemaInfo = new QuerySchemaInfo(); - querySchemaInfo.setKeyGenerator(queryExecuterProperties.keyStructureInfo.getKeyGenerator()); - querySchemaInfo.setMaskedByteIndexes(queryExecuterProperties.keyStructureInfo.getMaskedBytes()); - querySchemaInfo.setQueryDimensions(queryModel.getQueryDimension() - .toArray(new QueryDimension[queryModel.getQueryDimension().size()])); - querySchemaInfo.setQueryMeasures(queryModel.getQueryMeasures() - .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()])); - int msrSize = queryExecuterProperties.measureDataTypes.length; - int dimensionCount = queryModel.getQueryDimension().size(); - int[] queryOrder = new int[dimensionCount + msrSize]; - int[] queryReverseOrder = new int[dimensionCount + msrSize]; - for (int i = 0; i < dimensionCount; i++) { - queryOrder[queryModel.getQueryDimension().get(i).getQueryOrder()] = i; - queryReverseOrder[i] = queryModel.getQueryDimension().get(i).getQueryOrder(); - } - for (int i = 0; i < msrSize; i++) { - queryOrder[queryModel.getQueryMeasures().get(i).getQueryOrder()] = i + dimensionCount; - queryReverseOrder[i + dimensionCount] = queryModel.getQueryMeasures().get(i).getQueryOrder(); - } - querySchemaInfo.setQueryOrder(queryOrder); - querySchemaInfo.setQueryReverseOrder(queryReverseOrder); - } - - @Override public BatchResult prepareQueryResult( - Result, Object> scannedResult) { - if ((null == scannedResult || scannedResult.size() < 1)) { - return new BatchRawResult(); - } - QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions(); - int msrSize = queryExecuterProperties.measureDataTypes.length; - int dimSize = queryDimensions.length; - int[] order = querySchemaInfo.getQueryReverseOrder(); - Object[][] resultData = new Object[scannedResult.size()][]; - Object[] value; - Object[] row; - int counter = 0; - if (queryModel.isRawBytesDetailQuery()) { - while (scannedResult.hasNext()) { - value = scannedResult.getValue(); - row = new Object[msrSize + 1]; - row[0] = scannedResult.getKey(); - if (value != null) { - assert (value.length == msrSize); - System.arraycopy(value, 0, row, 1, msrSize); - } - resultData[counter] = row; - counter++; - } - } else { - while (scannedResult.hasNext()) { - value = scannedResult.getValue(); - row = new Object[msrSize + dimSize]; - ByteArrayWrapper key = scannedResult.getKey(); - if (key != null) { - long[] surrogateResult = querySchemaInfo.getKeyGenerator() - .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes()); - int noDictionaryColumnIndex = 0; - int complexTypeColumnIndex = 0; - for (int i = 0; i < dimSize; i++) { - if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) { - row[order[i]] = DataTypeUtil.getDataBasedOnDataType( - new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)), - queryDimensions[i].getDimension().getDataType()); - } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) { - DirectDictionaryGenerator directDictionaryGenerator = - DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator( - queryDimensions[i].getDimension().getDataType()); - if (directDictionaryGenerator != null) { - row[order[i]] = directDictionaryGenerator.getValueFromSurrogate( - (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]); - } - } else if (CarbonUtil - .hasComplexDataType(queryDimensions[i].getDimension().getDataType())) { - row[order[i]] = queryExecuterProperties.complexDimensionInfoMap - .get(queryDimensions[i].getDimension().getOrdinal()) - .getDataBasedOnDataTypeFromSurrogates( - ByteBuffer.wrap(key.getComplexTypeByIndex(complexTypeColumnIndex++))); - } else { - row[order[i]] = - (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]; - } - } - } - for (int i = 0; i < msrSize; i++) { - row[order[i + queryDimensions.length]] = value[i]; - } - resultData[counter] = row; - counter++; - } - } - - LOGGER.info("###########################---- Total Number of records" + scannedResult.size()); - BatchRawResult result = new BatchRawResult(); - result.setRows(resultData); - return result; - } - -} - http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java ---------------------------------------------------------------------- diff --git a/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java index 279c8ab..0934862 100644 --- a/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java +++ b/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java @@ -77,7 +77,6 @@ public class CarbonInputFormatUtil { } plan.setLimit(-1); - plan.setRawDetailQuery(true); plan.setQueryId(System.nanoTime() + ""); return plan; } http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java ---------------------------------------------------------------------- diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java index 43921f4..5c853c0 100644 --- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java +++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java @@ -43,7 +43,7 @@ import org.carbondata.scan.executor.exception.QueryExecutionException; import org.carbondata.scan.model.QueryDimension; import org.carbondata.scan.model.QueryMeasure; import org.carbondata.scan.model.QueryModel; -import org.carbondata.scan.result.BatchRawResult; +import org.carbondata.scan.result.BatchResult; import org.carbondata.scan.result.iterator.RawResultIterator; /** @@ -143,12 +143,12 @@ public class CarbonCompactionExecutor { * @param blockList * @return */ - private CarbonIterator executeBlockList(List blockList) + private CarbonIterator executeBlockList(List blockList) throws QueryExecutionException { queryModel.setTableBlockInfos(blockList); this.queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel); - CarbonIterator iter = null; + CarbonIterator iter = null; try { iter = queryExecutor.execute(queryModel); } catch (QueryExecutionException e) { @@ -187,7 +187,6 @@ public class CarbonCompactionExecutor { model.setCountStarQuery(false); model.setDetailQuery(true); model.setForcedDetailRawQuery(true); - model.setRawBytesDetailQuery(true); model.setFilterExpressionResolverTree(null); List dims = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala ---------------------------------------------------------------------- diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala index 21aded4..8265a1b 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala @@ -46,61 +46,14 @@ case class CarbonScan( val selectedMsrs = scala.collection.mutable.MutableList[QueryMeasure]() @transient val carbonCatalog = ocRaw.catalog.asInstanceOf[CarbonMetastoreCatalog] - val attributesNeedToDecode = new java.util.HashSet[AttributeReference]() + val attributesNeedToDecode = new java.util.LinkedHashSet[AttributeReference]() val unprocessedExprs = new ArrayBuffer[Expression]() val buildCarbonPlan: CarbonQueryPlan = { val plan: CarbonQueryPlan = new CarbonQueryPlan(relationRaw.databaseName, relationRaw.tableName) - val dimensions = carbonTable.getDimensionByTableName(carbonTable.getFactTableName) - val measures = carbonTable.getMeasureByTableName(carbonTable.getFactTableName) - val dimAttr = new Array[Attribute](dimensions.size()) - val msrAttr = new Array[Attribute](measures.size()) - attributesRaw.foreach { attr => - val carbonDimension = - carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name) - if(carbonDimension != null) { - dimAttr(dimensions.indexOf(carbonDimension)) = attr - } else { - val carbonMeasure = - carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name) - if(carbonMeasure != null) { - msrAttr(measures.indexOf(carbonMeasure)) = attr - } - } - } - - attributesRaw = dimAttr.filter(f => f != null) ++ msrAttr.filter(f => f != null) - - var queryOrder: Integer = 0 - attributesRaw.foreach { attr => - val carbonDimension = - carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name) - if (carbonDimension != null) { - val dim = new QueryDimension(attr.name) - dim.setQueryOrder(queryOrder) - queryOrder = queryOrder + 1 - selectedDims += dim - } else { - val carbonMeasure = - carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name) - if (carbonMeasure != null) { - val m1 = new QueryMeasure(attr.name) - m1.setQueryOrder(queryOrder) - queryOrder = queryOrder + 1 - selectedMsrs += m1 - } - } - } - - // Fill the selected dimensions & measures obtained from - // attributes to query plan for detailed query - selectedDims.foreach(plan.addDimension) - selectedMsrs.foreach(plan.addMeasure) - plan.setSortedDimemsions(new ArrayList[QueryDimension]) - plan.setRawDetailQuery(true) plan.setOutLocationPath( CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS)) plan.setQueryId(System.nanoTime() + "") @@ -129,32 +82,59 @@ case class CarbonScan( if (attributesNeedToDecode.size() > 0) { val attributeOut = new ArrayBuffer[Attribute]() ++ attributesRaw - attributesNeedToDecode.asScala.map { attr => - val dims = plan.getDimensions.asScala.filter(f => f.getColumnName.equals(attr.name)) - val msrs = plan.getMeasures.asScala.filter(f => f.getColumnName.equals(attr.name)) - var order = plan.getDimensions.size() + plan.getMeasures.size() - if (dims.isEmpty && msrs.isEmpty) { - val dimension = carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name) - if (dimension != null) { - val qDim = new QueryDimension(dimension.getColName) - qDim.setQueryOrder(order) - plan.addDimension(qDim) - attributeOut += attr - order += 1 - } else { - val measure = carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name) - if (measure != null) { - val qMsr = new QueryMeasure(measure.getColName) - qMsr.setQueryOrder(order) - plan.addMeasure(qMsr) - order += 1 - attributeOut += attr - } - } + attributesNeedToDecode.asScala.foreach { attr => + if (!attributesRaw.exists(_.name.equalsIgnoreCase(attr.name))) { + attributeOut += attr } } attributesRaw = attributeOut } + + val dimensions = carbonTable.getDimensionByTableName(carbonTable.getFactTableName) + val measures = carbonTable.getMeasureByTableName(carbonTable.getFactTableName) + val dimAttr = new Array[Attribute](dimensions.size()) + val msrAttr = new Array[Attribute](measures.size()) + attributesRaw.foreach { attr => + val carbonDimension = + carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name) + if(carbonDimension != null) { + dimAttr(dimensions.indexOf(carbonDimension)) = attr + } else { + val carbonMeasure = + carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name) + if(carbonMeasure != null) { + msrAttr(measures.indexOf(carbonMeasure)) = attr + } + } + } + + attributesRaw = dimAttr.filter(f => f != null) ++ msrAttr.filter(f => f != null) + + var queryOrder: Integer = 0 + attributesRaw.foreach { attr => + val carbonDimension = + carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name) + if (carbonDimension != null) { + val dim = new QueryDimension(attr.name) + dim.setQueryOrder(queryOrder) + queryOrder = queryOrder + 1 + selectedDims += dim + } else { + val carbonMeasure = + carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name) + if (carbonMeasure != null) { + val m1 = new QueryMeasure(attr.name) + m1.setQueryOrder(queryOrder) + queryOrder = queryOrder + 1 + selectedMsrs += m1 + } + } + } + + // Fill the selected dimensions & measures obtained from + // attributes to query plan for detailed query + selectedDims.foreach(plan.addDimension) + selectedMsrs.foreach(plan.addMeasure) } http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala ---------------------------------------------------------------------- diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala index 995c109..2d20b73 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.types.{IntegerType, StringType} @@ -82,6 +83,9 @@ class ResolveCarbonFunctions(relations: Seq[CarbonDecoderRelation]) */ def transformCarbonPlan(plan: LogicalPlan, relations: Seq[CarbonDecoderRelation]): LogicalPlan = { + if (plan.isInstanceOf[RunnableCommand]) { + return plan + } var decoder = false val aliasMap = CarbonAliasDecoderRelation() // collect alias information before hand. http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9be7451e/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala ---------------------------------------------------------------------- diff --git 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 index fe56a58..95f36bb 100644 --- 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 @@ -40,7 +40,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { val currentDirectory = new File(this.getClass.getResource("/").getPath + "/../../") .getCanonicalPath - sql("create table Carbon_automation_test (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, Active_phonePADParti tionedVersions 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) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Latest_MONTH,Latest_DAY,deviceInformationId')"); + sql("create table if not exists Carbon_automation_test (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, Active _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) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Latest_MONTH,Latest_DAY,deviceInformationId')"); CarbonProperties.getInstance() .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT) sql("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/100_olap.csv' INTO table Carbon_automation_test options('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,Late st_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')"); @@ -48,7 +48,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/100_olap.csv' INTO table Carbon_automation_hive "); //hive table - sql("create table Carbon_automation_test_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,contractNumber int, 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 stri ng, Active_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, gamePointId int,gamePointDescription string)row format delimited fields terminated by ','"); + sql("create table if not exists Carbon_automation_test_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,contractNumber int, 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_operato rsVersion string, Active_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, gamePointId int,gamePointDescription string)row format delimited fields terminated by ','"); sql("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/100_olap.csv' INTO table Carbon_automation_test_hive"); } @@ -56,6 +56,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { override def afterAll { sql("drop table Carbon_automation_test") sql("drop table Carbon_automation_hive") + sql("drop table Carbon_automation_test_hive") CarbonProperties.getInstance() .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy") @@ -68,7 +69,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { Seq(Row("1", 132), Row("2", 110), Row("3", 176), Row("4", 132), Row("5", 132), Row("6", 209), Row("7", 198))) } - + test("select channelsId, Latest_DAY from Carbon_automation_test where count(channelsId) = 1") { try { sql("select channelsId, Latest_DAY from Carbon_automation_test where count(channelsId) = 1").collect @@ -77,7 +78,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { case ce: Exception => ce.getMessage } } - + //Test-24 test("select channelsId, sum(channelsId+ 10) Total from Carbon_automation_test group by channelsId order by Total") { checkAnswer( @@ -109,7 +110,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { Seq(Row("1", 11), Row("2", 11), Row("3", 11), Row("4", 11), Row("5", 11), Row("6", 11), Row("7", 11))) } - + //Test-29 test("select channelsId, avg(channelsId+ 10) Total from Carbon_automation_test group by channelsId order by Total") { checkAnswer( @@ -134,8 +135,8 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { Seq(Row("1", 12), Row("2", 10), Row("3", 16), Row("4", 12), Row("5", 12), Row("6", 19), Row("7", 18))) } - - + + //Test-33 test("select channelsId, count(channelsId+ 10) Total from Carbon_automation_test group by channelsId order by channelsId") { checkAnswer( @@ -159,8 +160,8 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { Seq(Row("1", 11), Row("2", 11), Row("3", 11), Row("4", 11), Row("5", 11), Row("6", 11), Row("7", 11))) } - - + + //Test-37 test("select channelsId, min(channelsId+ 10) Total from Carbon_automation_test group by channelsId order by Total") { checkAnswer( @@ -184,8 +185,8 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { Seq(Row("1", 11), Row("2", 11), Row("3", 11), Row("4", 11), Row("5", 11), Row("6", 11), Row("7", 11))) } - - + + //Test-41 test("select channelsId, max(channelsId+ 10) Total from Carbon_automation_test group by channelsId order by Total")({ @@ -218,15 +219,15 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select Latest_YEAR ,sum(distinct Latest_YEAR)+10 from Carbon_automation_test group by Latest_YEAR"), Seq(Row(2015, 2025))) }) - + //Test-47 test("select sum(gamepointid) +10 as a ,series from Carbon_automation_test group by series")({ checkAnswer( sql("select sum(gamepointid) +10 as a ,series from Carbon_automation_test group by series"), sql("select sum(gamepointid) +10 as a ,series from Carbon_automation_test_hive group by series")) - }) - + }) + //Test-50 test("select sum(gamepointid) +10.36 as a ,series from Carbon_automation_test group by series")({ @@ -234,7 +235,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select sum(gamepointid) +10.36 as a ,series from Carbon_automation_test group by series"), sql("select sum(gamepointid) +10.36 as a ,series from Carbon_automation_test_hive group by series")) }) - + //TC_055 test("select count(deviceinformationid)+10.32 as a ,series from Carbon_automation_test group by series")({ checkAnswer( @@ -348,7 +349,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select sum( DISTINCT Latest_MONTH) a from Carbon_automation_test"), Seq(Row(7))) }) - + //TC_088 test("select avg (if(deviceInformationId>100,NULL,deviceInformationId)) a from Carbon_automation_test")({ checkAnswer( @@ -382,7 +383,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select min(Latest_MONTH) a from Carbon_automation_test"), Seq(Row(7))) }) - + //TC_095 test("select min (if(deviceInformationId>100,NULL,deviceInformationId)) a from Carbon_automation_test")({ checkAnswer( @@ -416,7 +417,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select max(Latest_MONTH) a from Carbon_automation_test"), Seq(Row(7))) }) - + //TC_102 test("select max (if(deviceInformationId>100,NULL,deviceInformationId)) a from Carbon_automation_test")({ checkAnswer( @@ -503,7 +504,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select series,count(imei) a from Carbon_automation_test group by series order by series"), Seq(Row("0Series", 15), Row("1Series", 3), Row("2Series", 9), Row("3Series", 8), Row("4Series", 8), Row("5Series", 17), Row("6Series", 9), Row("7Series", 11), Row("8Series", 11), Row("9Series", 8))) }) - + //TC_138 test("select series,ActiveProvince,count(imei) a from Carbon_automation_test group by ActiveProvince,series order by series,ActiveProvince")({ checkAnswer( @@ -517,14 +518,14 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select count(distinct deviceColor) a,deliveryProvince from Carbon_automation_test group by deliveryProvince"), Seq(Row(10, "Hunan Province"), Row(10, "Guangdong Province"), Row(10, "Hubei Province"))) }) - + //TC_141 test("select series,sum(gamePointId) a from Carbon_automation_test group by series order by series")({ checkAnswer( sql("select series,sum(gamePointId) a from Carbon_automation_test group by series order by series"), sql("select series,sum(gamePointId) a from Carbon_automation_test_hive group by series order by series")) }) - + //TC_162 test("select imei,series from Carbon_automation_test where Carbon_automation_test.series IN ('1Series','7Series')")({ checkAnswer( @@ -538,7 +539,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select imei,series from Carbon_automation_test where Carbon_automation_test.series NOT IN ('1Series','7Series')"), Seq(Row("1AA100", "5Series"), Row("1AA1000", "5Series"), Row("1AA100000", "9Series"), Row("1AA100001", "0Series"), Row("1AA100002", "0Series"), Row("1AA100003", "5Series"), Row("1AA100004", "4Series"), Row("1AA100006", "6Series"), Row("1AA100007", "9Series"), Row("1AA100008", "8Series"), Row("1AA100009", "0Series"), Row("1AA10001", "2Series"), Row("1AA100010", "3Series"), Row("1AA100011", "0Series"), Row("1AA100012", "4Series"), Row("1AA100014", "5Series"), Row("1AA100015", "4Series"), Row("1AA100016", "3Series"), Row("1AA100017", "9Series"), Row("1AA100018", "8Series"), Row("1AA100019", "5Series"), Row("1AA10002", "0Series"), Row("1AA100020", "5Series"), Row("1AA100021", "0Series"), Row("1AA100022", "5Series"), Row("1AA100023", "5Series"), Row("1AA100024", "6Series"), Row("1AA100025", "0Series"), Row("1AA100027", "0Series"), Row("1AA100028", "5Series"), Row("1AA100029", "2Series"), Row("1AA100033", "8Series"), Row("1AA100034", "2Series"), Row("1AA100035", "5Series"), Row("1AA 100036", "5Series"), Row("1AA100038", "6Series"), Row("1AA100039", "8Series"), Row("1AA10004", "5Series"), Row("1AA100040", "8Series"), Row("1AA100041", "5Series"), Row("1AA100042", "3Series"), Row("1AA100043", "9Series"), Row("1AA100044", "8Series"), Row("1AA100045", "2Series"), Row("1AA100046", "3Series"), Row("1AA100047", "9Series"), Row("1AA100048", "3Series"), Row("1AA100049", "0Series"), Row("1AA10005", "8Series"), Row("1AA100050", "2Series"), Row("1AA100051", "2Series"), Row("1AA100052", "6Series"), Row("1AA100053", "2Series"), Row("1AA100056", "6Series"), Row("1AA100057", "9Series"), Row("1AA100058", "5Series"), Row("1AA100059", "4Series"), Row("1AA10006", "3Series"), Row("1AA100060", "8Series"), Row("1AA100061", "6Series"), Row("1AA100062", "9Series"), Row("1AA100063", "2Series"), Row("1AA100064", "6Series"), Row("1AA100065", "0Series"), Row("1AA100066", "6Series"), Row("1AA100067", "4Series"), Row("1AA100068", "8Series"), Row("1AA100069", "8Series"), Row("1AA10007", "8Seri es"), Row("1AA100070", "0Series"), Row("1AA100071", "0Series"), Row("1AA100072", "4Series"), Row("1AA100073", "4Series"), Row("1AA100074", "6Series"), Row("1AA100075", "3Series"), Row("1AA100076", "0Series"), Row("1AA100077", "3Series"), Row("1AA100078", "2Series"), Row("1AA100079", "4Series"), Row("1AA10008", "5Series"), Row("1AA100080", "9Series"), Row("1AA100081", "5Series"), Row("1AA100082", "5Series"), Row("1AA100083", "0Series"), Row("1AA100084", "0Series"))) }) - + //TC_166 test("select Upper(series) a from Carbon_automation_test")({ checkAnswer( @@ -559,7 +560,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select imei,series from Carbon_automation_test limit 10"), Seq(Row("1AA1", "7Series"), Row("1AA10", "7Series"), Row("1AA100", "5Series"), Row("1AA1000", "5Series"), Row("1AA10000", "7Series"), Row("1AA100000", "9Series"), Row("1AA1000000", "7Series"), Row("1AA100001", "0Series"), Row("1AA100002", "0Series"), Row("1AA100003", "5Series"))) }) - + //TC_171 test("select Lower(series) a from Carbon_automation_test")({ checkAnswer( @@ -580,7 +581,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select distinct Latest_DAY from Carbon_automation_test"), Seq(Row(1))) }) - + //TC_175 test("select distinct channelsId from Carbon_automation_test")({ checkAnswer( @@ -671,14 +672,14 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select series,ActiveProvince,sum(gamePointId) a from Carbon_automation_test group by series,ActiveProvince order by series desc,ActiveProvince asc"), sql("select series,ActiveProvince,sum(gamePointId) a from Carbon_automation_test_hive group by series,ActiveProvince order by series desc,ActiveProvince asc")) }) - + //TC_208 test("select Latest_DAY as a from Carbon_automation_test where Latest_DAY<=>Latest_areaId")({ checkAnswer( sql("select Latest_DAY as a from Carbon_automation_test where Latest_DAY<=>Latest_areaId"), Seq(Row(1), Row(1), Row(1), Row(1), Row(1), Row(1), Row(1), Row(1), Row(1), Row(1), Row(1), Row(1))) }) - + //TC_210 test("select Latest_DAY from Carbon_automation_test where Latest_DAY<>Latest_areaId")({ checkAnswer( @@ -720,7 +721,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select imei, Latest_DAY from Carbon_automation_test where Latest_DAY NOT BETWEEN Latest_areaId AND Latest_HOUR"), Seq(Row("1AA1", 1), Row("1AA10", 1), Row("1AA100", 1), Row("1AA1000", 1), Row("1AA10000", 1), Row("1AA100000", 1), Row("1AA100001", 1), Row("1AA100002", 1), Row("1AA100003", 1), Row("1AA100004", 1), Row("1AA100006", 1), Row("1AA100007", 1), Row("1AA100008", 1), Row("1AA100009", 1), Row("1AA10001", 1), Row("1AA100010", 1), Row("1AA100011", 1), Row("1AA100012", 1), Row("1AA100013", 1), Row("1AA100014", 1), Row("1AA100015", 1), Row("1AA100016", 1), Row("1AA100017", 1), Row("1AA100018", 1), Row("1AA100019", 1), Row("1AA10002", 1), Row("1AA100020", 1), Row("1AA100021", 1), Row("1AA100022", 1), Row("1AA100023", 1), Row("1AA100024", 1), Row("1AA100029", 1), Row("1AA10003", 1), Row("1AA100030", 1), Row("1AA100031", 1), Row("1AA100032", 1), Row("1AA100033", 1), Row("1AA100035", 1), Row("1AA100036", 1), Row("1AA100037", 1), Row("1AA100038", 1), Row("1AA10004", 1), Row("1AA100040", 1), Row("1AA100041", 1), Row("1AA100042", 1), Row("1AA100043", 1), Row("1AA100044", 1), Row("1AA100045", 1) , Row("1AA100046", 1), Row("1AA100047", 1), Row("1AA100048", 1), Row("1AA100049", 1), Row("1AA10005", 1), Row("1AA100051", 1), Row("1AA100053", 1), Row("1AA100054", 1), Row("1AA100055", 1), Row("1AA100056", 1), Row("1AA100057", 1), Row("1AA100058", 1), Row("1AA100059", 1), Row("1AA10006", 1), Row("1AA100060", 1), Row("1AA100062", 1), Row("1AA100063", 1), Row("1AA100064", 1), Row("1AA100065", 1), Row("1AA100066", 1), Row("1AA100067", 1), Row("1AA100068", 1), Row("1AA100069", 1), Row("1AA10007", 1), Row("1AA100070", 1), Row("1AA100071", 1), Row("1AA100072", 1), Row("1AA100073", 1), Row("1AA100075", 1), Row("1AA100076", 1), Row("1AA100077", 1), Row("1AA100078", 1), Row("1AA100079", 1), Row("1AA10008", 1), Row("1AA100080", 1), Row("1AA100081", 1), Row("1AA100082", 1), Row("1AA100083", 1), Row("1AA100084", 1))) }) - + //TC_219 test("select imei, Latest_DAY from Carbon_automation_test where Latest_DAY IS NOT NULL")({ checkAnswer( @@ -734,14 +735,14 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select imei, Latest_DAY from Carbon_automation_test where imei IS NOT NULL"), Seq(Row("1AA1", 1), Row("1AA10", 1), Row("1AA100", 1), Row("1AA1000", 1), Row("1AA10000", 1), Row("1AA100000", 1), Row("1AA1000000", 1), Row("1AA100001", 1), Row("1AA100002", 1), Row("1AA100003", 1), Row("1AA100004", 1), Row("1AA100005", 1), Row("1AA100006", 1), Row("1AA100007", 1), Row("1AA100008", 1), Row("1AA100009", 1), Row("1AA10001", 1), Row("1AA100010", 1), Row("1AA100011", 1), Row("1AA100012", 1), Row("1AA100013", 1), Row("1AA100014", 1), Row("1AA100015", 1), Row("1AA100016", 1), Row("1AA100017", 1), Row("1AA100018", 1), Row("1AA100019", 1), Row("1AA10002", 1), Row("1AA100020", 1), Row("1AA100021", 1), Row("1AA100022", 1), Row("1AA100023", 1), Row("1AA100024", 1), Row("1AA100025", 1), Row("1AA100026", 1), Row("1AA100027", 1), Row("1AA100028", 1), Row("1AA100029", 1), Row("1AA10003", 1), Row("1AA100030", 1), Row("1AA100031", 1), Row("1AA100032", 1), Row("1AA100033", 1), Row("1AA100034", 1), Row("1AA100035", 1), Row("1AA100036", 1), Row("1AA100037", 1), Row("1AA100038", 1), Row("1AA100039", 1), Row("1AA10004", 1), Row("1AA100040", 1), Row("1AA100041", 1), Row("1AA100042", 1), Row("1AA100043", 1), Row("1AA100044", 1), Row("1AA100045", 1), Row("1AA100046", 1), Row("1AA100047", 1), Row("1AA100048", 1), Row("1AA100049", 1), Row("1AA10005", 1), Row("1AA100050", 1), Row("1AA100051", 1), Row("1AA100052", 1), Row("1AA100053", 1), Row("1AA100054", 1), Row("1AA100055", 1), Row("1AA100056", 1), Row("1AA100057", 1), Row("1AA100058", 1), Row("1AA100059", 1), Row("1AA10006", 1), Row("1AA100060", 1), Row("1AA100061", 1), Row("1AA100062", 1), Row("1AA100063", 1), Row("1AA100064", 1), Row("1AA100065", 1), Row("1AA100066", 1), Row("1AA100067", 1), Row("1AA100068", 1), Row("1AA100069", 1), Row("1AA10007", 1), Row("1AA100070", 1), Row("1AA100071", 1), Row("1AA100072", 1), Row("1AA100073", 1), Row("1AA100074", 1), Row("1AA100075", 1), Row("1AA100076", 1), Row("1AA100077", 1), Row("1AA100078", 1), Row("1AA100079", 1), Row("1AA10008", 1), Row("1AA100080", 1), Row("1AA100 081", 1), Row("1AA100082", 1), Row("1AA100083", 1), Row("1AA100084", 1))) }) - + //TC_223 test("select * from (select if( Latest_areaId=7,7,NULL) as babu,Latest_MONTH from Carbon_automation_test) qq where babu LIKE Latest_MONTH")({ checkAnswer( sql("select * from (select if( Latest_areaId=7,7,NULL) as babu,Latest_MONTH from Carbon_automation_test) qq where babu LIKE Latest_MONTH"), Seq(Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7), Row(7, 7))) }) - + //TC_263 test("SELECT AMSize, ActiveAreaId, SUM(gamePointId) AS Sum_gamePointId FROM (select * from Carbon_automation_test) SUB_QRY GROUP BY AMSize, ActiveAreaId ORDER BY AMSize ASC, ActiveAreaId ASC")({ checkAnswer( @@ -755,7 +756,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("SELECT AMSize, ActiveAreaId, SUM(gamePointId) AS Sum_gamePointId FROM (select * from Carbon_automation_test) SUB_QRY WHERE NOT(AMSize = \"\") GROUP BY AMSize, ActiveAreaId ORDER BY AMSize ASC, ActiveAreaId ASC"), sql("SELECT AMSize, ActiveAreaId, SUM(gamePointId) AS Sum_gamePointId FROM (select * from Carbon_automation_test_hive) SUB_QRY WHERE NOT(AMSize = \"\") GROUP BY AMSize, ActiveAreaId ORDER BY AMSize ASC, ActiveAreaId ASC")) }) - + //TC_274 test("SELECT ActiveCountry, ActiveDistrict, Activecity, SUM(gamepointid) AS Sum_gamepointid FROM Carbon_automation_test group by ActiveCountry,ActiveDistrict,Activecity")({ checkAnswer( @@ -783,14 +784,14 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("SELECT ActiveCountry, COUNT(DISTINCT imei) AS DistinctCount_imei FROM (select * from Carbon_automation_test) SUB_QRY GROUP BY ActiveCountry ORDER BY ActiveCountry ASC"), Seq(Row("Chinese", 99))) }) - + //TC_282 test("SELECT ActiveCountry, ActiveDistrict, Activecity, SUM(gamepointid) AS Sum_gamepointid FROM (select * from Carbon_automation_test) SUB_QRY GROUP BY ActiveCountry, ActiveDistrict, Activecity ORDER BY ActiveCountry ASC, ActiveDistrict ASC, Activecity ASC")({ checkAnswer( sql("SELECT ActiveCountry, ActiveDistrict, Activecity, SUM(gamepointid) AS Sum_gamepointid FROM (select * from Carbon_automation_test) SUB_QRY GROUP BY ActiveCountry, ActiveDistrict, Activecity ORDER BY ActiveCountry ASC, ActiveDistrict ASC, Activecity ASC"), sql("SELECT ActiveCountry, ActiveDistrict, Activecity, SUM(gamepointid) AS Sum_gamepointid FROM (select * from Carbon_automation_test_hive) SUB_QRY GROUP BY ActiveCountry, ActiveDistrict, Activecity ORDER BY ActiveCountry ASC, ActiveDistrict ASC, Activecity ASC")) }) - + //TC_317 test("select channelsId from Carbon_automation_test order by channelsId")({ checkAnswer( @@ -818,7 +819,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("SELECT ActiveCountry, ActiveDistrict, Activecity, SUM(gamePointId) AS Sum_gamePointId FROM (select * from Carbon_automation_test) SUB_QRY WHERE imei = \"1AA100000\" GROUP BY ActiveCountry, ActiveDistrict, Activecity ORDER BY ActiveCountry ASC, ActiveDistrict ASC, Activecity ASC"), Seq(Row("Chinese", "yichang", "yichang", 136.0))) }) - + //TC_384 test("SELECT series, SUM(gamePointId) AS Sum_gamePointId FROM (select * from Carbon_automation_test) SUB_QRY GROUP BY series ORDER BY series ASC")({ checkAnswer( @@ -846,7 +847,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("SELECT imei, channelsId, COUNT(deliveryTime) AS Count_deliveryTime FROM (select * from Carbon_automation_test) SUB_QRY GROUP BY imei, channelsId ORDER BY imei ASC, channelsId ASC"), Seq(Row("1AA1", "4", 1), Row("1AA10", "4", 1), Row("1AA100", "6", 1), Row("1AA1000", "3", 1), Row("1AA10000", "1", 1), Row("1AA100000", "6", 1), Row("1AA1000000", "6", 1), Row("1AA100001", "7", 1), Row("1AA100002", "3", 1), Row("1AA100003", "3", 1), Row("1AA100004", "2", 1), Row("1AA100005", "1", 1), Row("1AA100006", "2", 1), Row("1AA100007", "3", 1), Row("1AA100008", "1", 1), Row("1AA100009", "3", 1), Row("1AA10001", "5", 1), Row("1AA100010", "6", 1), Row("1AA100011", "1", 1), Row("1AA100012", "2", 1), Row("1AA100013", "6", 1), Row("1AA100014", "3", 1), Row("1AA100015", "1", 1), Row("1AA100016", "3", 1), Row("1AA100017", "3", 1), Row("1AA100018", "4", 1), Row("1AA100019", "3", 1), Row("1AA10002", "5", 1), Row("1AA100020", "7", 1), Row("1AA100021", "6", 1), Row("1AA100022", "5", 1), Row("1AA100023", "3", 1), Row("1AA100024", "2", 1), Row("1AA100025", "1", 1), Row("1AA100026", "7", 1), Row("1AA100027", "4", 1), Row("1AA100028", "5", 1), Row("1AA100029", "5", 1), Row("1AA10003", "4", 1), Row("1AA100030", "3", 1), Row("1AA100031", "1", 1), Row("1AA100032", "7", 1), Row("1AA100033", "6", 1), Row("1AA100034", "6", 1), Row("1AA100035", "7", 1), Row("1AA100036", "5", 1), Row("1AA100037", "6", 1), Row("1AA100038", "3", 1), Row("1AA100039", "1", 1), Row("1AA10004", "4", 1), Row("1AA100040", "7", 1), Row("1AA100041", "1", 1), Row("1AA100042", "5", 1), Row("1AA100043", "6", 1), Row("1AA100044", "2", 1), Row("1AA100045", "6", 1), Row("1AA100046", "4", 1), Row("1AA100047", "1", 1), Row("1AA100048", "1", 1), Row("1AA100049", "6", 1), Row("1AA10005", "2", 1), Row("1AA100050", "1", 1), Row("1AA100051", "7", 1), Row("1AA100052", "7", 1), Row("1AA100053", "3", 1), Row("1AA100054", "2", 1), Row("1AA100055", "7", 1), Row("1AA100056", "5", 1), Row("1AA100057", "6", 1), Row("1AA100058", "4", 1), Row("1AA100059", "7", 1), Row("1AA10006", "5", 1), Row("1AA100060", "4", 1), Row("1AA100061", "6", 1), Row("1AA100062", "6", 1), Row("1AA100063", "3", 1), Row("1AA100064", "7", 1), Ro w("1AA100065", "7", 1), Row("1AA100066", "4", 1), Row("1AA100067", "7", 1), Row("1AA100068", "7", 1), Row("1AA100069", "5", 1), Row("1AA10007", "3", 1), Row("1AA100070", "3", 1), Row("1AA100071", "5", 1), Row("1AA100072", "7", 1), Row("1AA100073", "2", 1), Row("1AA100074", "7", 1), Row("1AA100075", "6", 1), Row("1AA100076", "7", 1), Row("1AA100077", "6", 1), Row("1AA100078", "5", 1), Row("1AA100079", "6", 1), Row("1AA10008", "4", 1), Row("1AA100080", "6", 1), Row("1AA100081", "2", 1), Row("1AA100082", "7", 1), Row("1AA100083", "2", 1), Row("1AA100084", "4", 1))) }) - + //TC_408 test("select imei,series from Carbon_automation_test where series='7Series' order by imei limit 10")({ checkAnswer( @@ -874,7 +875,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select avg(channelsName) from Carbon_automation_test where modelId is null"), Seq(Row(null))) }) - + //TC_424 test("SELECT count(DISTINCT gamePointId) FROM Carbon_automation_test where imei is null")({ checkAnswer( @@ -937,14 +938,14 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select channelsName from Carbon_automation_test where latest_day is NOT null"), Seq(Row("guomei"), Row("guomei"), Row("yidong"), Row("shuling"), Row("taobao"), Row("yidong"), Row("yidong"), Row("liantong"), Row("shuling"), Row("shuling"), Row("jingdong"), Row("taobao"), Row("jingdong"), Row("shuling"), Row("taobao"), Row("shuling"), Row("shishang"), Row("yidong"), Row("taobao"), Row("jingdong"), Row("yidong"), Row("shuling"), Row("taobao"), Row("shuling"), Row("shuling"), Row("guomei"), Row("shuling"), Row("shishang"), Row("liantong"), Row("yidong"), Row("shishang"), Row("shuling"), Row("jingdong"), Row("taobao"), Row("liantong"), Row("guomei"), Row("shishang"), Row("shishang"), Row("guomei"), Row("shuling"), Row("taobao"), Row("liantong"), Row("yidong"), Row("yidong"), Row("liantong"), Row("shishang"), Row("yidong"), Row("shuling"), Row("taobao"), Row("guomei"), Row("liantong"), Row("taobao"), Row("shishang"), Row("yidong"), Row("jingdong"), Row("yidong"), Row("guomei"), Row("taobao"), Row("taobao"), Row("yidong"), Row("jingdong"), Row("taobao"), Row("li antong"), Row("liantong"), Row("shuling"), Row("jingdong"), Row("liantong"), Row("shishang"), Row("yidong"), Row("guomei"), Row("liantong"), Row("shishang"), Row("guomei"), Row("yidong"), Row("yidong"), Row("shuling"), Row("liantong"), Row("liantong"), Row("guomei"), Row("liantong"), Row("liantong"), Row("shishang"), Row("shuling"), Row("shuling"), Row("shishang"), Row("liantong"), Row("jingdong"), Row("liantong"), Row("yidong"), Row("liantong"), Row("yidong"), Row("shishang"), Row("yidong"), Row("guomei"), Row("yidong"), Row("jingdong"), Row("liantong"), Row("jingdong"), Row("guomei"))) }) - + //TC_439 test("SELECT min(AMSize) FROM Carbon_automation_test where imei is NOT null")({ checkAnswer( sql("SELECT min(AMSize) FROM Carbon_automation_test where imei is NOT null"), Seq(Row("0RAM size"))) }) - + //TC_448 test("select var_samp(Latest_YEAR) from Carbon_automation_test")({ checkAnswer( @@ -958,7 +959,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select var_samp(AMSize) from Carbon_automation_test"), Seq(Row(null))) }) - + //TC_451 test("select stddev_pop(bomcode)from Carbon_automation_test")({ checkAnswer( @@ -979,7 +980,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select stddev_pop(AMSIZE)from Carbon_automation_test"), Seq(Row(null))) }) - + //TC_457 test("select stddev_samp(deviceInformationId)from Carbon_automation_test1")({ checkAnswer( @@ -1034,8 +1035,8 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { checkAnswer( sql("Select percentile(1,1.0) from Carbon_automation_test"), Seq(Row(1.0))) - }) - + }) + //TC_481 test("select percentile_approx(1, 0.5 ,5000) from Carbon_automation_test")({ checkAnswer( @@ -1056,7 +1057,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { sql("select percentile_approx(1, 0.5 ,700) from Carbon_automation_test"), Seq(Row(1.0))) }) - + //TC_484 test("select percentile_approx(1, 0.5 ,500) from Carbon_automation_test")({ checkAnswer( @@ -1098,7 +1099,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll { checkAnswer( sql("select c1,count(c1) from (select c1 as c1,c2 as c2 from carbonunion union all select c2 as c1,c1 as c2 from carbonunion)t where c1='200' group by c1"), sql("select c1,count(c1) from (select c1 as c1,c2 as c2 from sparkunion union all select c2 as c1,c1 as c2 from sparkunion)t where c1='200' group by c1")) + sql("drop table if exists carbonunion") }) - sql("drop table if exists carbonunion") } \ No newline at end of file