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 30404200C87 for ; Wed, 17 May 2017 19:35:58 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 2EAE6160BBA; Wed, 17 May 2017 17:35:58 +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 F3D7E160BAC for ; Wed, 17 May 2017 19:35:56 +0200 (CEST) Received: (qmail 88087 invoked by uid 500); 17 May 2017 17:35:56 -0000 Mailing-List: contact commits-help@carbondata.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@carbondata.apache.org Delivered-To: mailing list commits@carbondata.apache.org Received: (qmail 88056 invoked by uid 99); 17 May 2017 17:35:55 -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; Wed, 17 May 2017 17:35:55 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CEC5CDFF93; Wed, 17 May 2017 17:35:55 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ravipesala@apache.org To: commits@carbondata.apache.org Date: Wed, 17 May 2017 17:35:55 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/2] carbondata git commit: added check for starting dictionary server archived-at: Wed, 17 May 2017 17:35:58 -0000 Repository: carbondata Updated Branches: refs/heads/master ca4d4baac -> 781d6fa73 added check for starting dictionary server moved single pass test suite to common module Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/caa93291 Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/caa93291 Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/caa93291 Branch: refs/heads/master Commit: caa93291c3e4256b06e23b74405493c89ce6bd5c Parents: ca4d4ba Author: kunal642 Authored: Wed May 17 13:07:12 2017 +0530 Committer: ravipesala Committed: Wed May 17 23:05:01 2017 +0530 ---------------------------------------------------------------------- .../generator/ServerDictionaryGenerator.java | 4 +- .../dataload/TestLoadDataWithSinglePass.scala | 129 +++++++++++++++++++ .../execution/command/carbonTableSchema.scala | 21 ++- .../dataload/TestLoadDataWithSinglePass.scala | 111 ---------------- .../spark/rdd/CarbonDataRDDFactory.scala | 4 +- .../execution/command/carbonTableSchema.scala | 21 ++- 6 files changed, 166 insertions(+), 124 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/carbondata/blob/caa93291/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java index cd168b8..456e885 100644 --- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java +++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java @@ -73,7 +73,9 @@ public class ServerDictionaryGenerator implements DictionaryGenerator carbonDimension.hasEncoding(Encoding.DICTIONARY) && + !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) + } + val server: Option[DictionaryServer] = if (createDictionary) { + val dictionaryServer = DictionaryServer + .getInstance(dictionaryServerPort.toInt) + carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort) + Some(dictionaryServer) + } else { + None + } CarbonDataRDDFactory.loadCarbonData(sqlContext, carbonLoadModel, relation.tableMeta.storePath, columnar, partitionStatus, - Some(server), + server, dataFrame, updateModel) } else { http://git-wip-us.apache.org/repos/asf/carbondata/blob/caa93291/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala ---------------------------------------------------------------------- diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala deleted file mode 100644 index 1d456d3..0000000 --- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.carbondata.spark.testsuite.dataload - -import org.apache.spark.sql.common.util.QueryTest -import org.scalatest.BeforeAndAfterAll - -/** - * Test Class for data loading use one pass - * - */ -class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll { - - override def beforeAll { - sql("DROP TABLE IF EXISTS table_two_pass") - sql("DROP TABLE IF EXISTS table_one_pass") - sql("DROP TABLE IF EXISTS table_one_pass_2") - - sql( - """ - |CREATE TABLE table_two_pass (ID int, date Timestamp, country String, - |name String, phonetype String, serialname String, salary int) - |STORED BY 'org.apache.carbondata.format' - """.stripMargin) - - sql( - s""" - |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_two_pass - |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='false') - """.stripMargin) - - sql( - """ - |CREATE TABLE table_one_pass (ID int, date Timestamp, country String, - |name String, phonetype String, serialname String, salary int) - |STORED BY 'org.apache.carbondata.format' - """.stripMargin) - - sql( - s""" - |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_one_pass - |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true') - """.stripMargin) - } - - test("test data loading use one pass") { - checkAnswer( - sql("select * from table_one_pass"), - sql("select * from table_two_pass") - ) - } - - test("test data loading use one pass when offer column dictionary file") { - sql( - """ - |CREATE TABLE table_one_pass_2 (ID int, date Timestamp, country String, - |name String, phonetype String, serialname String, salary int) - |STORED BY 'org.apache.carbondata.format' - """.stripMargin) - sql( - s""" - |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_one_pass_2 - |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true', 'COLUMNDICT'= - |'country:$resourcesPath/columndictionary/country.csv, name:$resourcesPath/columndictionary/name.csv') - """.stripMargin) - - checkAnswer( - sql("select * from table_one_pass_2"), - sql("select * from table_two_pass") - ) - } - - test("test data loading use one pass when do incremental load") { - sql( - s""" - |LOAD DATA local inpath '$resourcesPath/dataIncrement.csv' INTO TABLE table_two_pass - |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='false') - """.stripMargin) - sql( - s""" - |LOAD DATA local inpath '$resourcesPath/dataIncrement.csv' INTO TABLE table_one_pass - |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true') - """.stripMargin) - - checkAnswer( - sql("select * from table_one_pass"), - sql("select * from table_two_pass") - ) - } - - override def afterAll { - sql("DROP TABLE IF EXISTS table_two_pass") - sql("DROP TABLE IF EXISTS table_one_pass") - sql("DROP TABLE IF EXISTS table_one_pass_2") - } -} http://git-wip-us.apache.org/repos/asf/carbondata/blob/caa93291/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala ---------------------------------------------------------------------- diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala index 812151e..b01602f 100644 --- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala +++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala @@ -963,7 +963,7 @@ object CarbonDataRDDFactory { private def writeDictionary(carbonLoadModel: CarbonLoadModel, result: Option[DictionaryServer], writeAll: Boolean) = { - // write dictionary file and shutdown dictionary server + // write dictionary file val uniqueTableName: String = s"${ carbonLoadModel.getDatabaseName }_${ carbonLoadModel.getTableName }" @@ -976,7 +976,7 @@ object CarbonDataRDDFactory { server.writeTableDictionary(uniqueTableName) } } catch { - case ex: Exception => + case _: Exception => LOGGER.error(s"Error while writing dictionary file for $uniqueTableName") throw new Exception("Dataload failed due to error while writing dictionary file!") } http://git-wip-us.apache.org/repos/asf/carbondata/blob/caa93291/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---------------------------------------------------------------------- diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala index 11b3115..94a95fd 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala @@ -504,16 +504,27 @@ case class LoadTable( val sparkDriverHost = sparkSession.sqlContext.sparkContext. getConf.get("spark.driver.host") carbonLoadModel.setDictionaryServerHost(sparkDriverHost) - // start dictionary server when use one pass load. - val server: DictionaryServer = DictionaryServer - .getInstance(dictionaryServerPort.toInt) - carbonLoadModel.setDictionaryServerPort(server.getPort) + // start dictionary server when use one pass load and dimension with DICTIONARY + // encoding is present. + val allDimensions = table.getAllDimensions.asScala.toList + val createDictionary = allDimensions.exists { + carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) && + !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) + } + val server: Option[DictionaryServer] = if (createDictionary) { + val dictionaryServer = DictionaryServer + .getInstance(dictionaryServerPort.toInt) + carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort) + Some(dictionaryServer) + } else { + None + } CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext, carbonLoadModel, relation.tableMeta.storePath, columnar, partitionStatus, - Some(server), + server, dataFrame, updateModel) }