You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2016/08/01 10:05:02 UTC

[04/47] incubator-carbondata git commit: [CARBONDATA-17] - select count(*) from table where column_x = 'value' is not returning the correct count (#771)

[CARBONDATA-17] - select count(*) from table where column_x = 'value' is not returning the correct count (#771)

if the first measure is having all values null.

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

Branch: refs/heads/master
Commit: 34741b48681b1a50bb46d84615dd77c17a6c44f7
Parents: 6b7b41a
Author: Mohammad Shahid Khan <mo...@gmail.com>
Authored: Fri Jul 22 15:12:17 2016 +0530
Committer: Kumar Vishal <ku...@gmail.com>
Committed: Fri Jul 22 15:12:17 2016 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  4 ++
 .../query/aggregator/impl/CountAggregator.java  |  2 +-
 .../aggregator/impl/CountStarAggregator.java    | 51 ++++++++++++++
 .../util/MeasureAggregatorFactory.java          | 28 ++------
 .../org/apache/spark/sql/CarbonOperators.scala  |  4 +-
 .../src/test/resources/datanullmeasurecol.csv   |  3 +
 .../filterexpr/CountStarTestCase.scala          | 72 ++++++++++++++++++++
 7 files changed, 137 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/34741b48/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
index 8dbe4dd..a7a8c07 100644
--- a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
@@ -155,6 +155,10 @@ public final class CarbonCommonConstants {
    */
   public static final String COUNT = "count";
   /**
+   * Count constant
+   */
+  public static final String COUNT_STAR = "countstar";
+  /**
    * Max constant
    */
   public static final String MAX = "max";

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/34741b48/core/src/main/java/org/carbondata/query/aggregator/impl/CountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/CountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/CountAggregator.java
index 12c2061..1a040b6 100644
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/CountAggregator.java
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/CountAggregator.java
@@ -42,7 +42,7 @@ public class CountAggregator implements MeasureAggregator {
   /**
    * aggregate value
    */
-  private double aggVal;
+  protected double aggVal;
 
   /**
    * Count Aggregate function which update the total count

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/34741b48/core/src/main/java/org/carbondata/query/aggregator/impl/CountStarAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/CountStarAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/CountStarAggregator.java
new file mode 100644
index 0000000..3b7e78c
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/CountStarAggregator.java
@@ -0,0 +1,51 @@
+/*
+ * 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.query.aggregator.impl;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+/**
+ * Class Description : It will return total count of values
+ */
+public class CountStarAggregator extends CountAggregator {
+
+  @Override public void agg(double newVal) {
+    super.agg(newVal);
+  }
+
+  @Override public void agg(Object newVal) {
+    aggVal++;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    aggVal++;
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    CountStarAggregator aggregator = new CountStarAggregator();
+    aggregator.aggVal = aggVal;
+    return aggregator;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new CountStarAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/34741b48/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java b/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
index a49c374..3fbf819 100644
--- a/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
+++ b/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
@@ -24,30 +24,7 @@ import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.query.aggregator.CustomMeasureAggregator;
 import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.AvgBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.AvgDoubleAggregator;
-import org.carbondata.query.aggregator.impl.AvgLongAggregator;
-import org.carbondata.query.aggregator.impl.AvgTimestampAggregator;
-import org.carbondata.query.aggregator.impl.CountAggregator;
-import org.carbondata.query.aggregator.impl.DistinctCountAggregatorObjectSet;
-import org.carbondata.query.aggregator.impl.DistinctCountBigDecimalAggregatorObjectSet;
-import org.carbondata.query.aggregator.impl.DistinctCountLongAggregatorObjectSet;
-import org.carbondata.query.aggregator.impl.DummyBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.DummyDoubleAggregator;
-import org.carbondata.query.aggregator.impl.DummyLongAggregator;
-import org.carbondata.query.aggregator.impl.MaxAggregator;
-import org.carbondata.query.aggregator.impl.MaxBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.MaxLongAggregator;
-import org.carbondata.query.aggregator.impl.MinAggregator;
-import org.carbondata.query.aggregator.impl.MinBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.MinLongAggregator;
-import org.carbondata.query.aggregator.impl.SumBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.SumDistinctBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.SumDistinctDoubleAggregator;
-import org.carbondata.query.aggregator.impl.SumDistinctLongAggregator;
-import org.carbondata.query.aggregator.impl.SumDoubleAggregator;
-import org.carbondata.query.aggregator.impl.SumLongAggregator;
-import org.carbondata.query.aggregator.impl.SumTimestampAggregator;
+import org.carbondata.query.aggregator.impl.*;
 import org.carbondata.query.carbon.model.CustomAggregateExpression;
 
 /**
@@ -106,6 +83,9 @@ public class MeasureAggregatorFactory {
     } else if (CarbonCommonConstants.COUNT.equalsIgnoreCase(aggregatorType)) {
       return new CountAggregator();
     }
+    else if (CarbonCommonConstants.COUNT_STAR.equalsIgnoreCase(aggregatorType)) {
+      return new CountStarAggregator();
+    }
     //
     else if (CarbonCommonConstants.MAX.equalsIgnoreCase(aggregatorType)) {
       switch (dataType) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/34741b48/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 9946dfd..76b7e8c 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
@@ -113,7 +113,7 @@ case class CarbonTableScan(
 
       case CountCarbon(posLiteral@PositionLiteral(Literal(star, _), _)) if star == "*" =>
         val m1 = new QueryMeasure("count(*)")
-        m1.setAggregateFunction(CarbonCommonConstants.COUNT)
+        m1.setAggregateFunction(CarbonCommonConstants.COUNT_STAR)
         m1.setQueryOrder(queryOrder)
         posLiteral.setPosition(queryOrder)
         plan.addMeasure(m1)
@@ -123,7 +123,7 @@ case class CarbonTableScan(
 
       case curr@CountCarbon(posLiteral@PositionLiteral(one, _)) =>
         val m1 = new QueryMeasure("count(*)")
-        m1.setAggregateFunction(CarbonCommonConstants.COUNT)
+        m1.setAggregateFunction(CarbonCommonConstants.COUNT_STAR)
         m1.setQueryOrder(queryOrder)
         posLiteral.setPosition(queryOrder)
         plan.addMeasure(m1)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/34741b48/integration/spark/src/test/resources/datanullmeasurecol.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/datanullmeasurecol.csv b/integration/spark/src/test/resources/datanullmeasurecol.csv
new file mode 100644
index 0000000..40692d8
--- /dev/null
+++ b/integration/spark/src/test/resources/datanullmeasurecol.csv
@@ -0,0 +1,3 @@
+ID,date,country,name,phonetype,serialname,salary
+1,2015/7/23,china,aaa1,phone197,A234
+2,2015/7/24,china,aaa2,phone756,A453

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/34741b48/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/CountStarTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/CountStarTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/CountStarTestCase.scala
new file mode 100644
index 0000000..20881c5
--- /dev/null
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/CountStarTestCase.scala
@@ -0,0 +1,72 @@
+/*
+ * 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.spark.testsuite.filterexpr
+
+import java.io.File
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.carbondata.core.constants.CarbonCommonConstants
+import org.carbondata.core.util.CarbonProperties
+import org.scalatest.BeforeAndAfterAll
+
+/**
+  * Test Class for filter expression query on String datatypes
+  *
+  * @author N00902756
+  *
+  */
+class CountStarTestCase extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("drop table if exists filtertestTables")
+
+
+    sql("CREATE TABLE filterTimestampDataType (ID int, date Timestamp, country String, " +
+      "name String, phonetype String, serialname String, salary int) " +
+      "STORED BY 'org.apache.carbondata.format'"
+    )
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/mm/dd")
+    val currentDirectory = new File(this.getClass.getResource("/").getPath + "/../../")
+      .getCanonicalPath
+    var csvFilePath = currentDirectory + "/src/test/resources/datanullmeasurecol.csv";
+      sql(
+      s"LOAD DATA LOCAL INPATH '" + csvFilePath + "' INTO TABLE " +
+        s"filterTimestampDataType " +
+        s"OPTIONS('DELIMITER'= ',', " +
+        s"'FILEHEADER'= '')"
+    )
+  }
+
+  test("select count ") {
+    checkAnswer(
+      sql("select count(*) from filterTimestampDataType where country='china'"),
+      Seq(Row(2))
+    )
+  }
+
+  override def afterAll {
+    sql("drop table noloadtable")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+  }
+}
\ No newline at end of file