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 2017/12/14 06:27:21 UTC

carbondata git commit: [CARBONDATA-1878] [DataMap] Fix bugs in unsafe datamap store

Repository: carbondata
Updated Branches:
  refs/heads/master 6026680a1 -> d504e067d


[CARBONDATA-1878] [DataMap] Fix bugs in unsafe datamap store

This closes #1633


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

Branch: refs/heads/master
Commit: d504e067d15013a91dd6192a46688b8b93f2e204
Parents: 6026680
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Fri Dec 8 12:03:09 2017 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Dec 14 11:57:01 2017 +0530

----------------------------------------------------------------------
 .../core/indexstore/UnsafeMemoryDMStore.java    |  3 +-
 ...tDictionaryWithOffHeapSortDisabledTest.scala | 81 ++++++++++++++++++++
 2 files changed, 83 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d504e067/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
index 450796a..73b7b60 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
@@ -147,7 +147,8 @@ public class UnsafeMemoryDMStore {
         break;
       case VARIABLE:
         byte[] data = row.getByteArray(index);
-        getUnsafe().putShort(memoryBlock.getBaseOffset() + runningLength, (short) data.length);
+        getUnsafe().putShort(memoryBlock.getBaseObject(),
+            memoryBlock.getBaseOffset() + runningLength, (short) data.length);
         runningLength += 2;
         getUnsafe().copyMemory(data, BYTE_ARRAY_OFFSET, memoryBlock.getBaseObject(),
             memoryBlock.getBaseOffset() + runningLength, data.length);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d504e067/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithOffHeapSortDisabledTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithOffHeapSortDisabledTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithOffHeapSortDisabledTest.scala
new file mode 100644
index 0000000..dc80374
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/directdictionary/DateDataTypeDirectDictionaryWithOffHeapSortDisabledTest.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.directdictionary
+
+import java.sql.Date
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.common.constants.LoggerAction
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+/**
+ * test case copied from `DateDataTypeDirectDictionaryTest` to verify CARBONDATA-1878
+ */
+class DateDataTypeDirectDictionaryWithOffHeapSortDisabledTest
+  extends QueryTest with BeforeAndAfterAll {
+  private val originBadRecordsAction: String = CarbonProperties.getInstance()
+    .getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+      CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT)
+  private val originOffHeapSortStatus: String = CarbonProperties.getInstance()
+    .getProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT,
+      CarbonCommonConstants.ENABLE_OFFHEAP_SORT_DEFAULT)
+
+  override def beforeAll {
+    try {
+      CarbonProperties.getInstance().addProperty("carbon.direct.dictionary", "true")
+      CarbonProperties.getInstance().addProperty(
+        CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, LoggerAction.FORCE.name())
+      CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT, "false")
+
+      sql("drop table if exists directDictionaryTable ")
+      sql("CREATE TABLE if not exists directDictionaryTable (empno int,doj date, salary int) " +
+        "STORED BY 'org.apache.carbondata.format'")
+
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
+      val csvFilePath = s"$resourcesPath/datasamplefordate.csv"
+      sql("LOAD DATA local inpath '" + csvFilePath + "' INTO TABLE directDictionaryTable OPTIONS" +
+          "('DELIMITER'= ',', 'QUOTECHAR'= '\"')" )
+    } catch {
+      case x: Throwable =>
+        x.printStackTrace()
+        CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+    }
+  }
+
+  test("test direct dictionary for not null condition") {
+    checkAnswer(sql("select doj from directDictionaryTable where doj is not null"),
+      Seq(Row(Date.valueOf("2016-03-14")), Row(Date.valueOf("2016-04-14"))))
+  }
+
+  override def afterAll {
+    sql("drop table directDictionaryTable")
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+      originBadRecordsAction)
+    CarbonProperties.getInstance().addProperty("carbon.direct.dictionary", "false")
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT,
+      originOffHeapSortStatus)
+  }
+}