[GitHub] carbondata pull request #1011: [carbondata-1149] Fix issue of mismatch type ...

2017-06-11 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1011#discussion_r121313645
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
 ---
@@ -122,6 +122,254 @@ class TestDDLForPartitionTable  extends QueryTest 
with BeforeAndAfterAll {
 intercept[Exception] { sql("alter table test drop columns(c)") }
   }
 
+  test("test exception if hash number is invalid") {
+sql("DROP TABLE IF EXISTS test_hash_1")
+try {
+  sql(
+"""
+  | CREATE TABLE test_hash_1(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='2.1')
+""".stripMargin
+  )
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+sql("DROP TABLE IF EXISTS test_hash_2")
+try {
+  sql(
+"""
+  | CREATE TABLE test_hash_2(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='abc')
+""".stripMargin
+  )
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+sql("DROP TABLE IF EXISTS test_hash_3")
+try {
+  sql(
+"""
+  | CREATE TABLE test_hash_3(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='-2.1')
+""".stripMargin
+  )
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+  }
+
+  test("test exception when values in list_info can not match partition 
column type") {
+sql("DROP TABLE IF EXISTS test_list_1")
+try {
+  sql(
+"""
+  | CREATE TABLE test_list_1(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
+""".stripMargin)
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+// scalastyle:off
+sql("DROP TABLE IF EXISTS test_list_2")
+try {
+  sql("CREATE TABLE test_list_2(col1 INT, col2 STRING) PARTITIONED BY 
(col3 SHORT) STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='LIST', 
'LIST_INFO'='abc,def')")
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+// scalastyle:on
+sql("DROP TABLE IF EXISTS test_list_3")
+try {
+  sql(
+"""
+  | CREATE TABLE test_list_3(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 FLOAT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
+""".stripMargin)
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+sql("DROP TABLE IF EXISTS test_list_4")
+try {
+  sql(
+"""
+  | CREATE TABLE test_list_4(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 DOUBLE) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
+""".stripMargin)
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+sql("DROP TABLE IF EXISTS test_list_5")
+try {
+  sql(
+"""
+  | CREATE TABLE test_list_5(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 LONG) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
+""".stripMargin)
+} catch {
+  case e:Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+//sql("DROP TABLE IF EXISTS test_list_6")
+//try {
+//  val errorMessage_list_6 = intercept[Exception] {
+//sql(
+//  """
+//|CREATE TABLE test_list_6(col1 INT, col2 STRING) 
PARTITIONED BY (col3 DECIMAL)
+//|STORED BY 'carbondata' 
TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
+//  """.stripMargin)
+//  }
+//  

[GitHub] carbondata pull request #1011: [carbondata-1149] Fix issue of mismatch type ...

2017-06-11 Thread kunal642
Github user kunal642 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1011#discussion_r121313629
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
 ---
@@ -122,6 +122,254 @@ class TestDDLForPartitionTable  extends QueryTest 
with BeforeAndAfterAll {
 intercept[Exception] { sql("alter table test drop columns(c)") }
   }
 
+  test("test exception if hash number is invalid") {
+sql("DROP TABLE IF EXISTS test_hash_1")
+try {
+  sql(
+"""
+  | CREATE TABLE test_hash_1(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='2.1')
+""".stripMargin
+  )
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+sql("DROP TABLE IF EXISTS test_hash_2")
+try {
+  sql(
+"""
+  | CREATE TABLE test_hash_2(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='abc')
+""".stripMargin
+  )
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+sql("DROP TABLE IF EXISTS test_hash_3")
+try {
+  sql(
+"""
+  | CREATE TABLE test_hash_3(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='-2.1')
+""".stripMargin
+  )
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+  }
+
+  test("test exception when values in list_info can not match partition 
column type") {
+sql("DROP TABLE IF EXISTS test_list_1")
+try {
+  sql(
+"""
+  | CREATE TABLE test_list_1(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
+""".stripMargin)
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+// scalastyle:off
+sql("DROP TABLE IF EXISTS test_list_2")
+try {
+  sql("CREATE TABLE test_list_2(col1 INT, col2 STRING) PARTITIONED BY 
(col3 SHORT) STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='LIST', 
'LIST_INFO'='abc,def')")
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+// scalastyle:on
+sql("DROP TABLE IF EXISTS test_list_3")
+try {
+  sql(
+"""
+  | CREATE TABLE test_list_3(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 FLOAT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
+""".stripMargin)
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+sql("DROP TABLE IF EXISTS test_list_4")
+try {
+  sql(
+"""
+  | CREATE TABLE test_list_4(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 DOUBLE) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
+""".stripMargin)
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+sql("DROP TABLE IF EXISTS test_list_5")
+try {
+  sql(
+"""
+  | CREATE TABLE test_list_5(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 LONG) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
+""".stripMargin)
+} catch {
+  case e:Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+//sql("DROP TABLE IF EXISTS test_list_6")
--- End diff --

remove the commented test cases


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #1011: [carbondata-1149] Fix issue of mismatch type ...

2017-06-11 Thread gvramana
Github user gvramana commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1011#discussion_r121312645
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
 ---
@@ -122,6 +122,254 @@ class TestDDLForPartitionTable  extends QueryTest 
with BeforeAndAfterAll {
 intercept[Exception] { sql("alter table test drop columns(c)") }
   }
 
+  test("test exception if hash number is invalid") {
+sql("DROP TABLE IF EXISTS test_hash_1")
+try {
+  sql(
+"""
+  | CREATE TABLE test_hash_1(col1 INT, col2 STRING)
+  | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+  | TBLPROPERTIES('PARTITION_TYPE'='HASH', 'NUM_PARTITIONS'='2.1')
+""".stripMargin
+  )
+} catch {
+  case e: Exception =>
+assert(e.getMessage.contains("Invalid partition definition"))
+}
+
+sql("DROP TABLE IF EXISTS test_hash_2")
--- End diff --

One test for failure validation is sufficient. As all of them are covered 
in parse exception


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #1001: [CARBONDATA-1135] Added partition column info...

2017-06-11 Thread gvramana
Github user gvramana commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1001#discussion_r121313696
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
 ---
@@ -130,6 +138,7 @@ class TestDDLForPartitionTable  extends QueryTest with 
BeforeAndAfterAll {
 sql("drop table if exists hashTable")
 sql("drop table if exists rangeTable")
 sql("drop table if exists listTable")
+sql("drop table test")
--- End diff --

Use drop table if exists


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2373/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread watermen
Github user watermen commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121306203
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSort.scala
 ---
@@ -0,0 +1,152 @@
+/*
+ * 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.load
+
+import java.util.Comparator
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.row.CarbonRow
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
+import org.apache.carbondata.processing.csvload.{CSVInputFormat, 
StringArrayWritable}
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
+import 
org.apache.carbondata.processing.sortandgroupby.sortdata.{NewRowComparator, 
NewRowComparatorForNormalDims, SortParameters}
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+import org.apache.carbondata.spark.util.CommonUtil
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.NullWritable
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.NewHadoopRDD
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.execution.command.ExecutionErrors
+import org.apache.spark.storage.StorageLevel
+
+/**
+  * Use sortBy operator in spark to load the data
+  */
+object GlobalSort {
--- End diff --

Name it to `DataLoadProcess`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread watermen
Github user watermen commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121305832
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortOperates.scala
 ---
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.load
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import 
org.apache.carbondata.core.datastore.exception.CarbonDataWriterException
+import org.apache.carbondata.core.datastore.row.CarbonRow
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.processing.csvload.StringArrayWritable
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
+import 
org.apache.carbondata.processing.newflow.converter.impl.RowConverterImpl
+import 
org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
+import org.apache.carbondata.processing.newflow.parser.impl.RowParserImpl
+import org.apache.carbondata.processing.newflow.sort.SortHelper
+import 
org.apache.carbondata.processing.newflow.steps.{DataConverterProcessorStepImpl, 
DataWriterProcessorStepImpl}
+import 
org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters
+import org.apache.carbondata.processing.store.{CarbonFactHandler, 
CarbonFactHandlerFactory}
+import org.apache.carbondata.spark.rdd.{NewRddIterator, StringArrayRow}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.sql.Row
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.{SparkEnv, TaskContext}
+
+import scala.util.Random
+
+object GlobalSortOperates {
+  private val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def toStringArrayRow(row: StringArrayWritable, columnCount: Int): 
StringArrayRow = {
+val outRow = new StringArrayRow(new Array[String](columnCount))
+outRow.setValues(row.get())
+  }
+
+  def toRDDIterator(
+  rows: Iterator[Row],
+  modelBroadcast: Broadcast[CarbonLoadModel]): Iterator[Array[AnyRef]] 
= {
+new Iterator[Array[AnyRef]] {
+  val iter = new NewRddIterator(rows, modelBroadcast.value, 
TaskContext.get())
+
+  override def hasNext: Boolean = iter.hasNext
+
+  override def next(): Array[AnyRef] = iter.next
+}
+  }
+
+  def inputFunc(
+  rows: Iterator[Array[AnyRef]],
+  index: Int,
+  currentLoadCount: Int,
+  modelBroadcast: Broadcast[CarbonLoadModel],
+  rowNumber: LongAccumulator): Iterator[CarbonRow] = {
+val model: CarbonLoadModel = getModelCopy(index, currentLoadCount, 
modelBroadcast)
+val conf = DataLoadProcessBuilder.createConfiguration(model)
+val rowParser = new RowParserImpl(conf.getDataFields, conf)
+
+TaskContext.get().addTaskFailureListener { (t: TaskContext, e: 
Throwable) =>
+  wrapException(e, model)
+}
+
+new Iterator[CarbonRow] {
+  override def hasNext: Boolean = rows.hasNext
+
+  override def next(): CarbonRow = {
+rowNumber.add(1)
--- End diff --

Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread watermen
Github user watermen commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121305557
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortOperates.scala
 ---
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.load
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import 
org.apache.carbondata.core.datastore.exception.CarbonDataWriterException
+import org.apache.carbondata.core.datastore.row.CarbonRow
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.processing.csvload.StringArrayWritable
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
+import 
org.apache.carbondata.processing.newflow.converter.impl.RowConverterImpl
+import 
org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
+import org.apache.carbondata.processing.newflow.parser.impl.RowParserImpl
+import org.apache.carbondata.processing.newflow.sort.SortHelper
+import 
org.apache.carbondata.processing.newflow.steps.{DataConverterProcessorStepImpl, 
DataWriterProcessorStepImpl}
+import 
org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters
+import org.apache.carbondata.processing.store.{CarbonFactHandler, 
CarbonFactHandlerFactory}
+import org.apache.carbondata.spark.rdd.{NewRddIterator, StringArrayRow}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.sql.Row
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.{SparkEnv, TaskContext}
+
+import scala.util.Random
+
+object GlobalSortOperates {
--- End diff --

Name it to `DataLoadProcessorStep`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread watermen
Github user watermen commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121305190
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSort.scala
 ---
@@ -0,0 +1,152 @@
+/*
+ * 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.load
+
+import java.util.Comparator
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.row.CarbonRow
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
+import org.apache.carbondata.processing.csvload.{CSVInputFormat, 
StringArrayWritable}
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
+import 
org.apache.carbondata.processing.sortandgroupby.sortdata.{NewRowComparator, 
NewRowComparatorForNormalDims, SortParameters}
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+import org.apache.carbondata.spark.util.CommonUtil
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.NullWritable
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.NewHadoopRDD
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.execution.command.ExecutionErrors
+import org.apache.spark.storage.StorageLevel
+
+/**
+  * Use sortBy operator in spark to load the data
+  */
+object GlobalSort {
+  private val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def loadDataUsingGlobalSort(
+  sc: SparkContext,
+  dataFrame: Option[DataFrame],
+  model: CarbonLoadModel,
+  currentLoadCount: Int): Array[(String, (LoadMetadataDetails, 
ExecutionErrors))] = {
+val originRDD = if (dataFrame.isDefined) {
+  dataFrame.get.rdd
+} else {
+  // input data from files
+  val hadoopConfiguration = new Configuration()
+  CommonUtil.configureCSVInputFormat(hadoopConfiguration, model)
+  hadoopConfiguration.set(FileInputFormat.INPUT_DIR, 
model.getFactFilePath)
+  val columnCount = model.getCsvHeaderColumns.length
+  new NewHadoopRDD[NullWritable, StringArrayWritable](
+sc,
+classOf[CSVInputFormat],
+classOf[NullWritable],
+classOf[StringArrayWritable],
+hadoopConfiguration)
+.map(x => GlobalSortOperates.toStringArrayRow(x._2, columnCount))
+}
+
+val modelBroadcast = sc.broadcast(model)
+val partialSuccessAccum = sc.longAccumulator("Partial Success 
Accumulator")
+
+val inputStepRowNumber = sc.longAccumulator("Input Processor 
Accumulator")
+val convertStepRowNumber = sc.longAccumulator("Convert Processor 
Accumulator")
+val sortStepRowNumber = sc.longAccumulator("Sort Processor 
Accumulator")
+val writeStepRowNumber = sc.longAccumulator("Write Processor 
Accumulator")
+
+// 1. Input
+val inputRDD = originRDD.mapPartitions(rows => 
GlobalSortOperates.toRDDIterator(rows, modelBroadcast))
+  .mapPartitionsWithIndex { case (index, rows) =>
+GlobalSortOperates.inputFunc(rows, index, currentLoadCount, 
modelBroadcast, inputStepRowNumber)
+  }
+
+// 2. Convert
+val convertRDD = inputRDD.mapPartitionsWithIndex { case (index, rows) 
=>
+  GlobalSortOperates.convertFunc(rows, index, currentLoadCount, 
modelBroadcast, partialSuccessAccum,
+convertStepRowNumber)
+}.filter(_ != null)// Filter the bad record
+
+// 3. Sort
+val configuration = DataLoadProcessBuilder.createConfiguration(model)
+val sortParameters = SortParameters.createSortParameters(configuration)
+object RowOrdering extends Ordering[Array[AnyRef]] {
   

[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2372/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread watermen
Github user watermen commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121304371
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortHelper.scala
 ---
@@ -0,0 +1,53 @@
+/*
+ * 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.load
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import 
org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
+import 
org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordsLogger
+import org.apache.spark.util.LongAccumulator
+
+object GlobalSortHelper {
+  private val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def badRecordsLogger(loadModel: CarbonLoadModel, badRecordsAccum: 
LongAccumulator): Unit = {
+val key = new CarbonTableIdentifier(loadModel.getDatabaseName, 
loadModel.getTableName, null).getBadRecordLoggerKey
+if (null != BadRecordsLogger.hasBadRecord(key)) {
+  LOGGER.error("Data Load is partially success for table " + 
loadModel.getTableName)
+  badRecordsAccum.add(1)
+} else {
+  LOGGER.info("Data loading is successful for table " + 
loadModel.getTableName)
+}
+  }
+
+  def tryWithSafeFinally[T](tableName: String, block: => T)
--- End diff --

Delete


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121301080
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSort.scala
 ---
@@ -0,0 +1,152 @@
+/*
+ * 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.load
+
+import java.util.Comparator
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.row.CarbonRow
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
+import org.apache.carbondata.processing.csvload.{CSVInputFormat, 
StringArrayWritable}
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
+import 
org.apache.carbondata.processing.sortandgroupby.sortdata.{NewRowComparator, 
NewRowComparatorForNormalDims, SortParameters}
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+import org.apache.carbondata.spark.util.CommonUtil
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.NullWritable
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.NewHadoopRDD
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.execution.command.ExecutionErrors
+import org.apache.spark.storage.StorageLevel
+
+/**
+  * Use sortBy operator in spark to load the data
+  */
+object GlobalSort {
+  private val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def loadDataUsingGlobalSort(
+  sc: SparkContext,
+  dataFrame: Option[DataFrame],
+  model: CarbonLoadModel,
+  currentLoadCount: Int): Array[(String, (LoadMetadataDetails, 
ExecutionErrors))] = {
+val originRDD = if (dataFrame.isDefined) {
+  dataFrame.get.rdd
+} else {
+  // input data from files
+  val hadoopConfiguration = new Configuration()
+  CommonUtil.configureCSVInputFormat(hadoopConfiguration, model)
+  hadoopConfiguration.set(FileInputFormat.INPUT_DIR, 
model.getFactFilePath)
+  val columnCount = model.getCsvHeaderColumns.length
+  new NewHadoopRDD[NullWritable, StringArrayWritable](
+sc,
+classOf[CSVInputFormat],
+classOf[NullWritable],
+classOf[StringArrayWritable],
+hadoopConfiguration)
+.map(x => GlobalSortOperates.toStringArrayRow(x._2, columnCount))
+}
+
+val modelBroadcast = sc.broadcast(model)
+val partialSuccessAccum = sc.longAccumulator("Partial Success 
Accumulator")
+
+val inputStepRowNumber = sc.longAccumulator("Input Processor 
Accumulator")
+val convertStepRowNumber = sc.longAccumulator("Convert Processor 
Accumulator")
+val sortStepRowNumber = sc.longAccumulator("Sort Processor 
Accumulator")
+val writeStepRowNumber = sc.longAccumulator("Write Processor 
Accumulator")
+
+// 1. Input
+val inputRDD = originRDD.mapPartitions(rows => 
GlobalSortOperates.toRDDIterator(rows, modelBroadcast))
+  .mapPartitionsWithIndex { case (index, rows) =>
+GlobalSortOperates.inputFunc(rows, index, currentLoadCount, 
modelBroadcast, inputStepRowNumber)
+  }
+
+// 2. Convert
+val convertRDD = inputRDD.mapPartitionsWithIndex { case (index, rows) 
=>
+  GlobalSortOperates.convertFunc(rows, index, currentLoadCount, 
modelBroadcast, partialSuccessAccum,
+convertStepRowNumber)
+}.filter(_ != null)// Filter the bad record
+
+// 3. Sort
+val configuration = DataLoadProcessBuilder.createConfiguration(model)
+val sortParameters = SortParameters.createSortParameters(configuration)
+object RowOrdering extends Ordering[Array[AnyRef]] {

[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121300832
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortHelper.scala
 ---
@@ -0,0 +1,53 @@
+/*
+ * 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.load
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import 
org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
+import 
org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordsLogger
+import org.apache.spark.util.LongAccumulator
+
+object GlobalSortHelper {
+  private val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def badRecordsLogger(loadModel: CarbonLoadModel, badRecordsAccum: 
LongAccumulator): Unit = {
+val key = new CarbonTableIdentifier(loadModel.getDatabaseName, 
loadModel.getTableName, null).getBadRecordLoggerKey
+if (null != BadRecordsLogger.hasBadRecord(key)) {
+  LOGGER.error("Data Load is partially success for table " + 
loadModel.getTableName)
+  badRecordsAccum.add(1)
+} else {
+  LOGGER.info("Data loading is successful for table " + 
loadModel.getTableName)
+}
+  }
+
+  def tryWithSafeFinally[T](tableName: String, block: => T)
--- End diff --

Seems no one is using this function


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121301479
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortOperates.scala
 ---
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.load
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import 
org.apache.carbondata.core.datastore.exception.CarbonDataWriterException
+import org.apache.carbondata.core.datastore.row.CarbonRow
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.processing.csvload.StringArrayWritable
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
+import 
org.apache.carbondata.processing.newflow.converter.impl.RowConverterImpl
+import 
org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
+import org.apache.carbondata.processing.newflow.parser.impl.RowParserImpl
+import org.apache.carbondata.processing.newflow.sort.SortHelper
+import 
org.apache.carbondata.processing.newflow.steps.{DataConverterProcessorStepImpl, 
DataWriterProcessorStepImpl}
+import 
org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters
+import org.apache.carbondata.processing.store.{CarbonFactHandler, 
CarbonFactHandlerFactory}
+import org.apache.carbondata.spark.rdd.{NewRddIterator, StringArrayRow}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.sql.Row
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.{SparkEnv, TaskContext}
+
+import scala.util.Random
+
+object GlobalSortOperates {
--- End diff --

Name it similar to `AbstractDataLoadProcessorStep`, it contains process 
functions for all steps


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121302968
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortOperates.scala
 ---
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.load
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import 
org.apache.carbondata.core.datastore.exception.CarbonDataWriterException
+import org.apache.carbondata.core.datastore.row.CarbonRow
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.processing.csvload.StringArrayWritable
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
+import 
org.apache.carbondata.processing.newflow.converter.impl.RowConverterImpl
+import 
org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
+import org.apache.carbondata.processing.newflow.parser.impl.RowParserImpl
+import org.apache.carbondata.processing.newflow.sort.SortHelper
+import 
org.apache.carbondata.processing.newflow.steps.{DataConverterProcessorStepImpl, 
DataWriterProcessorStepImpl}
+import 
org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters
+import org.apache.carbondata.processing.store.{CarbonFactHandler, 
CarbonFactHandlerFactory}
+import org.apache.carbondata.spark.rdd.{NewRddIterator, StringArrayRow}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.sql.Row
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.{SparkEnv, TaskContext}
+
+import scala.util.Random
+
+object GlobalSortOperates {
+  private val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def toStringArrayRow(row: StringArrayWritable, columnCount: Int): 
StringArrayRow = {
+val outRow = new StringArrayRow(new Array[String](columnCount))
+outRow.setValues(row.get())
+  }
+
+  def toRDDIterator(
+  rows: Iterator[Row],
+  modelBroadcast: Broadcast[CarbonLoadModel]): Iterator[Array[AnyRef]] 
= {
+new Iterator[Array[AnyRef]] {
+  val iter = new NewRddIterator(rows, modelBroadcast.value, 
TaskContext.get())
+
+  override def hasNext: Boolean = iter.hasNext
+
+  override def next(): Array[AnyRef] = iter.next
+}
+  }
+
+  def inputFunc(
+  rows: Iterator[Array[AnyRef]],
+  index: Int,
+  currentLoadCount: Int,
+  modelBroadcast: Broadcast[CarbonLoadModel],
+  rowNumber: LongAccumulator): Iterator[CarbonRow] = {
+val model: CarbonLoadModel = getModelCopy(index, currentLoadCount, 
modelBroadcast)
+val conf = DataLoadProcessBuilder.createConfiguration(model)
+val rowParser = new RowParserImpl(conf.getDataFields, conf)
+
+TaskContext.get().addTaskFailureListener { (t: TaskContext, e: 
Throwable) =>
+  wrapException(e, model)
+}
+
+new Iterator[CarbonRow] {
+  override def hasNext: Boolean = rows.hasNext
+
+  override def next(): CarbonRow = {
+rowNumber.add(1)
+new CarbonRow(rowParser.parseRow(rows.next()))
+  }
+}
+  }
+
+  def convertFunc(
+  rows: Iterator[CarbonRow],
+  index: Int,
+  currentLoadCount: Int,
+  modelBroadcast: Broadcast[CarbonLoadModel],
+  partialSuccessAccum: LongAccumulator,
+  rowNumber: LongAccumulator): Iterator[CarbonRow] = {
+val model: CarbonLoadModel = getModelCopy(index, currentLoadCount, 
modelBroadcast)
+val conf = DataLoadProcessBuilder.createConfiguration(model)
+val badRecordLogger = 
DataConverterProcessorStepImpl.createBadRecordLogger(conf)
+val rowConverter = new RowConverterImpl(conf.getDataFields, conf, 
badRecordLogger)
+rowConverter.initialize()
+
+

[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2371/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/1011
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/249/Failed Tests: 
2carbondata-pr-spark-1.6/org.apache.carbondata:carbondata-spark-common-test:
 2org.apache.carbondata.spark.testsuite.partition.TestDDLForPartitionTable.test
 exception when values in list_info can not match partition column 
typeorg.apache.carbondata.spark.testsuite.partition.TestDDLForPartitionTable.test
 exception when values in range_info can not match partition column 
type



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121300326
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSort.scala
 ---
@@ -0,0 +1,152 @@
+/*
+ * 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.load
+
+import java.util.Comparator
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.row.CarbonRow
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
+import org.apache.carbondata.processing.csvload.{CSVInputFormat, 
StringArrayWritable}
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
+import 
org.apache.carbondata.processing.sortandgroupby.sortdata.{NewRowComparator, 
NewRowComparatorForNormalDims, SortParameters}
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+import org.apache.carbondata.spark.util.CommonUtil
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.NullWritable
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.NewHadoopRDD
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.execution.command.ExecutionErrors
+import org.apache.spark.storage.StorageLevel
+
+/**
+  * Use sortBy operator in spark to load the data
+  */
+object GlobalSort {
--- End diff --

Name it similar to `DataLoadProcessBuilder`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process

2017-06-11 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/910#discussion_r121300564
  
--- Diff: 
processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortHelper.java
 ---
@@ -0,0 +1,85 @@
+/*
+ * 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.processing.newflow.sort;
+
+import 
org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
+import org.apache.carbondata.processing.util.NonDictionaryUtil;
+
+public class SortHelper {
--- End diff --

Name it `SortStepRowUtil` like `WriteStepRowUtil`, it is for changing the 
format of carbon row. This behavior may change in future, so we better to put 
all such conversion in one place. So please change 
`SortTempFileChunkHolder.getRowFromStream` to use this utility. And 
`UnsafeSingleThreadFinalSortFilesMerger.convertRow` as well


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2370/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/1011
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/248/Failed Tests: 
2carbondata-pr-spark-1.6/org.apache.carbondata:carbondata-spark-common-test:
 2org.apache.carbondata.spark.testsuite.partition.TestDDLForPartitionTable.test
 exception when values in list_info can not match partition column 
typeorg.apache.carbondata.spark.testsuite.partition.TestDDLForPartitionTable.test
 exception when values in range_info can not match partition column 
type



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2369/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2368/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #1018: [Just For Test] Just test PR1105

2017-06-11 Thread chenerlu
Github user chenerlu closed the pull request at:

https://github.com/apache/carbondata/pull/1018


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #890: [CARBONDATA-1008] Make Caron table schema compatible ...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/890
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/246/Failed Tests: 
2carbondata-pr-spark-1.6/org.apache.carbondata:carbondata-core:
 2org.apache.carbondata.core.dictionary.client.DictionaryClientTest.testClientorg.apache.carbonda
 
ta.core.dictionary.client.DictionaryClientTest.testToCheckIfCorrectTimeOutExceptionMessageIsThrown



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #984: [CARBONDATA-1008] Make Caron table schema compatible ...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/984
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2366/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Updated] (CARBONDATA-1153) Can not add column

2017-06-11 Thread cen yuhai (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-1153?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

cen yuhai updated CARBONDATA-1153:
--
Summary: Can not add column  (was: Can not add column because it is aborted)

> Can not add column
> --
>
> Key: CARBONDATA-1153
> URL: https://issues.apache.org/jira/browse/CARBONDATA-1153
> Project: CarbonData
>  Issue Type: Bug
>  Components: spark-integration
>Affects Versions: 1.2.0
>Reporter: cen yuhai
>
> Sometimes it will throws exception as below. why can't I add column? no one 
> are altering the table... 
> {code}
> scala> carbon.sql("alter table temp.yuhai_carbon add columns(test1 string)")
> 17/06/11 22:09:13 AUDIT 
> [org.apache.spark.sql.execution.command.AlterTableAddColumns(207) -- main]: 
> [sh-hadoop-datanode-250-104.elenet.me][master][Thread-1]Alter table add 
> columns request has been received for temp.yuhai_carbon
> 17/06/11 22:10:22 ERROR [org.apache.spark.scheduler.TaskSetManager(70) -- 
> task-result-getter-3]: Task 0 in stage 0.0 failed 4 times; aborting job
> 17/06/11 22:10:22 ERROR 
> [org.apache.spark.sql.execution.command.AlterTableAddColumns(141) -- main]: 
> main Alter table add columns failed :Job aborted due to stage failure: Task 0 
> in stage 0.0 failed 4 times, most recent failure: Lost task 0.3 in stage 0.0 
> (TID 3, sh-hadoop-datanode-368.elenet.me, executor 7): 
> java.lang.RuntimeException: Dictionary file test1 is locked for updation. 
> Please try after some time
> at scala.sys.package$.error(package.scala:27)
> at 
> org.apache.carbondata.spark.util.GlobalDictionaryUtil$.loadDefaultDictionaryValueForNewColumn(GlobalDictionaryUtil.scala:857)
> at 
> org.apache.carbondata.spark.rdd.AlterTableAddColumnRDD$$anon$1.(AlterTableAddColumnRDD.scala:83)
> at 
> org.apache.carbondata.spark.rdd.AlterTableAddColumnRDD.compute(AlterTableAddColumnRDD.scala:68)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:331)
> at org.apache.spark.rdd.RDD.iterator(RDD.scala:295)
> at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:88)
> at org.apache.spark.scheduler.Task.run(Task.scala:104)
> at 
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:351)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:745)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2365/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/1011
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/244/Build result: 
FAILURE[...truncated 64.62 KB...][ERROR] Re-run Maven using the -X 
switch to enable full debug logging.[ERROR] [ERROR] For more information about 
the errors and possible solutions, please read the following articles:[ERROR] 
[Help 1] 
http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException[ERROR] 
[ERROR] After correcting the problems, you can resume the build with the 
command[ERROR]   mvn  -rf :carbondata-spark-common[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/processing/pom.xml
 to 
org.apache.carbondata/carbondata-processing/1.2.0-SNAPSHOT/carbondata-processing-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integration/spark-common/pom.xml
 to 
org.apache.carbondata/carbondata-spark-common/1.2.0-SNAPSHOT/carbondata-spark-common-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving /home/jenkins/jenkin
 s-slave/workspace/carbondata-pr-spark-1.6/examples/spark/pom.xml to 
org.apache.carbondata/carbondata-examples-spark/1.2.0-SNAPSHOT/carbondata-examples-spark-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integration/hive/pom.xml
 to 
org.apache.carbondata/carbondata-hive/1.2.0-SNAPSHOT/carbondata-hive-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/common/pom.xml to 
org.apache.carbondata/carbondata-common/1.2.0-SNAPSHOT/carbondata-common-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/assembly/pom.xml 
to 
org.apache.carbondata/carbondata-assembly/1.2.0-SNAPSHOT/carbondata-assembly-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/pom.xml to 
org.apache.carbondata/carbondata-parent/1.2.0-SNAPSHOT/carbondata-parent-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving /home/jenkins/jenkins-slave/works
 pace/carbondata-pr-spark-1.6/examples/flink/pom.xml to 
org.apache.carbondata/carbondata-examples-flink/1.2.0-SNAPSHOT/carbondata-examples-flink-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/core/pom.xml to 
org.apache.carbondata/carbondata-core/1.2.0-SNAPSHOT/carbondata-core-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/hadoop/pom.xml to 
org.apache.carbondata/carbondata-hadoop/1.2.0-SNAPSHOT/carbondata-hadoop-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integration/presto/pom.xml
 to 
org.apache.carbondata/carbondata-presto/1.2.0-SNAPSHOT/carbondata-presto-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integration/spark-common-test/pom.xml
 to 
org.apache.carbondata/carbondata-spark-common-test/1.2.0-SNAPSHOT/carbondata-spark-common-test-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integration/spark/pom.xml
 to 
org.apache.carbondata/carbondata-spark/1.2.0-SNAPSHOT/carbondata-spark-1.2.0-SNAPSHOT.pomchannel
 stoppedSetting status of 989d8658edb2bad11148781ae5a489bfd8859be3 to FAILURE 
with url https://builds.apache.org/job/carbondata-pr-spark-1.6/244/ and 
message: 'Tests Failed for Spark1.6 'Using context: Jenkins(Spark1.6): mvn 
clean test -Pspark-1.6



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Updated] (CARBONDATA-1153) Can not add column because it is aborted

2017-06-11 Thread cen yuhai (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-1153?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

cen yuhai updated CARBONDATA-1153:
--
Description: 
Sometimes it will throws exception as below. why can't I add column? no one are 
altering the table... 
{code}
scala> carbon.sql("alter table temp.yuhai_carbon add columns(test1 string)")
17/06/11 22:09:13 AUDIT 
[org.apache.spark.sql.execution.command.AlterTableAddColumns(207) -- main]: 
[sh-hadoop-datanode-250-104.elenet.me][master][Thread-1]Alter table add columns 
request has been received for temp.yuhai_carbon
17/06/11 22:10:22 ERROR [org.apache.spark.scheduler.TaskSetManager(70) -- 
task-result-getter-3]: Task 0 in stage 0.0 failed 4 times; aborting job
17/06/11 22:10:22 ERROR 
[org.apache.spark.sql.execution.command.AlterTableAddColumns(141) -- main]: 
main Alter table add columns failed :Job aborted due to stage failure: Task 0 
in stage 0.0 failed 4 times, most recent failure: Lost task 0.3 in stage 0.0 
(TID 3, sh-hadoop-datanode-368.elenet.me, executor 7): 
java.lang.RuntimeException: Dictionary file test1 is locked for updation. 
Please try after some time
at scala.sys.package$.error(package.scala:27)
at 
org.apache.carbondata.spark.util.GlobalDictionaryUtil$.loadDefaultDictionaryValueForNewColumn(GlobalDictionaryUtil.scala:857)
at 
org.apache.carbondata.spark.rdd.AlterTableAddColumnRDD$$anon$1.(AlterTableAddColumnRDD.scala:83)
at 
org.apache.carbondata.spark.rdd.AlterTableAddColumnRDD.compute(AlterTableAddColumnRDD.scala:68)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:331)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:295)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:88)
at org.apache.spark.scheduler.Task.run(Task.scala:104)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:351)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{code}

  was:
why can't I add column? no one are altering the table...
{code}
scala> carbon.sql("alter table temp.yuhai_carbon add columns(test1 string)")
17/06/11 22:09:13 AUDIT 
[org.apache.spark.sql.execution.command.AlterTableAddColumns(207) -- main]: 
[sh-hadoop-datanode-250-104.elenet.me][master][Thread-1]Alter table add columns 
request has been received for temp.yuhai_carbon
17/06/11 22:10:22 ERROR [org.apache.spark.scheduler.TaskSetManager(70) -- 
task-result-getter-3]: Task 0 in stage 0.0 failed 4 times; aborting job
17/06/11 22:10:22 ERROR 
[org.apache.spark.sql.execution.command.AlterTableAddColumns(141) -- main]: 
main Alter table add columns failed :Job aborted due to stage failure: Task 0 
in stage 0.0 failed 4 times, most recent failure: Lost task 0.3 in stage 0.0 
(TID 3, sh-hadoop-datanode-368.elenet.me, executor 7): 
java.lang.RuntimeException: Dictionary file test1 is locked for updation. 
Please try after some time
at scala.sys.package$.error(package.scala:27)
at 
org.apache.carbondata.spark.util.GlobalDictionaryUtil$.loadDefaultDictionaryValueForNewColumn(GlobalDictionaryUtil.scala:857)
at 
org.apache.carbondata.spark.rdd.AlterTableAddColumnRDD$$anon$1.(AlterTableAddColumnRDD.scala:83)
at 
org.apache.carbondata.spark.rdd.AlterTableAddColumnRDD.compute(AlterTableAddColumnRDD.scala:68)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:331)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:295)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:88)
at org.apache.spark.scheduler.Task.run(Task.scala:104)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:351)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{code}


> Can not add column because it is aborted
> 
>
> Key: CARBONDATA-1153
> URL: https://issues.apache.org/jira/browse/CARBONDATA-1153
> Project: CarbonData
>  Issue Type: Bug
>  Components: spark-integration
>Affects Versions: 1.2.0
>Reporter: cen yuhai
>
> Sometimes it will throws exception as below. why can't I add column? no one 
> are altering the table... 
> {code}
> scala> carbon.sql("alter table temp.yuhai_carbon add columns(test1 string)")
> 17/06/11 22:09:13 AUDIT 
> [org.apache.spark.sql.execution.command.AlterTableAddColumns(207) -- main]: 
> [sh-hadoop-datanode-250-104.elenet.me][master][Thread-1]Alter table add 
> columns request has been received for temp.yuhai_carbon
> 17/06/11 22:10:22 ERROR 

[jira] [Created] (CARBONDATA-1153) Can not add column because it is aborted

2017-06-11 Thread cen yuhai (JIRA)
cen yuhai created CARBONDATA-1153:
-

 Summary: Can not add column because it is aborted
 Key: CARBONDATA-1153
 URL: https://issues.apache.org/jira/browse/CARBONDATA-1153
 Project: CarbonData
  Issue Type: Bug
  Components: spark-integration
Affects Versions: 1.2.0
Reporter: cen yuhai


why can't I add column? no one are altering the table...
{code}
scala> carbon.sql("alter table temp.yuhai_carbon add columns(test1 string)")
17/06/11 22:09:13 AUDIT 
[org.apache.spark.sql.execution.command.AlterTableAddColumns(207) -- main]: 
[sh-hadoop-datanode-250-104.elenet.me][master][Thread-1]Alter table add columns 
request has been received for temp.yuhai_carbon
17/06/11 22:10:22 ERROR [org.apache.spark.scheduler.TaskSetManager(70) -- 
task-result-getter-3]: Task 0 in stage 0.0 failed 4 times; aborting job
17/06/11 22:10:22 ERROR 
[org.apache.spark.sql.execution.command.AlterTableAddColumns(141) -- main]: 
main Alter table add columns failed :Job aborted due to stage failure: Task 0 
in stage 0.0 failed 4 times, most recent failure: Lost task 0.3 in stage 0.0 
(TID 3, sh-hadoop-datanode-368.elenet.me, executor 7): 
java.lang.RuntimeException: Dictionary file test1 is locked for updation. 
Please try after some time
at scala.sys.package$.error(package.scala:27)
at 
org.apache.carbondata.spark.util.GlobalDictionaryUtil$.loadDefaultDictionaryValueForNewColumn(GlobalDictionaryUtil.scala:857)
at 
org.apache.carbondata.spark.rdd.AlterTableAddColumnRDD$$anon$1.(AlterTableAddColumnRDD.scala:83)
at 
org.apache.carbondata.spark.rdd.AlterTableAddColumnRDD.compute(AlterTableAddColumnRDD.scala:68)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:331)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:295)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:88)
at org.apache.spark.scheduler.Task.run(Task.scala:104)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:351)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[GitHub] carbondata issue #958: [CARBONDATA-1088] Added interfaces for Data Map frame...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/958
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/242/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2364/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #958: [CARBONDATA-1088] Added interfaces for Data Map frame...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/958
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2363/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/1011
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/243/Build result: 
FAILURE[...truncated 66.32 KB...][JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/processing/pom.xml
 to 
org.apache.carbondata/carbondata-processing/1.2.0-SNAPSHOT/carbondata-processing-1.2.0-SNAPSHOT.pomNo
 artifacts from carbondata-pr-spark-1.6 » Apache CarbonData :: Processing #242 
to compare, so performing full copy of artifacts[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/integration/spark-common/pom.xml
 to 
org.apache.carbondata/carbondata-spark-common/1.2.0-SNAPSHOT/carbondata-spark-common-1.2.0-SNAPSHOT.pomNo
 artifacts from carbondata-pr-spark-1.6 » Apache CarbonData :: Spark Common 
#242 to compare, so performing full copy of artifacts[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/examples/spark/pom.xml
 to org.apache.carbondata/carbondata-examples-spark/1.2.0-SNAPSHOT/carbondata
 -examples-spark-1.2.0-SNAPSHOT.pom[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/integration/hive/pom.xml
 to 
org.apache.carbondata/carbondata-hive/1.2.0-SNAPSHOT/carbondata-hive-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/common/pom.xml 
to 
org.apache.carbondata/carbondata-common/1.2.0-SNAPSHOT/carbondata-common-1.2.0-SNAPSHOT.pomNo
 artifacts from carbondata-pr-spark-1.6 » Apache CarbonData :: Common #242 to 
compare, so performing full copy of artifacts[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/assembly/pom.xml
 to 
org.apache.carbondata/carbondata-assembly/1.2.0-SNAPSHOT/carbondata-assembly-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/pom.xml to 
org.apache.carbondata/carbondata-parent/1.2.0-SNAPSHOT/carbondata-parent-1.2.0-SNAPSHOT.pomNo
 artifacts from carbondata-pr-spark-1.6 » Apache CarbonDa
 ta :: Parent #242 to compare, so performing full copy of artifacts[JENKINS] 
Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/examples/flink/pom.xml
 to 
org.apache.carbondata/carbondata-examples-flink/1.2.0-SNAPSHOT/carbondata-examples-flink-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/core/pom.xml to 
org.apache.carbondata/carbondata-core/1.2.0-SNAPSHOT/carbondata-core-1.2.0-SNAPSHOT.pomNo
 artifacts from carbondata-pr-spark-1.6 » Apache CarbonData :: Core #242 to 
compare, so performing full copy of artifacts[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/hadoop/pom.xml 
to 
org.apache.carbondata/carbondata-hadoop/1.2.0-SNAPSHOT/carbondata-hadoop-1.2.0-SNAPSHOT.pomNo
 artifacts from carbondata-pr-spark-1.6 » Apache CarbonData :: Hadoop #242 to 
compare, so performing full copy of artifacts[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/
 integration/presto/pom.xml to 
org.apache.carbondata/carbondata-presto/1.2.0-SNAPSHOT/carbondata-presto-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/integration/spark-common-test/pom.xml
 to 
org.apache.carbondata/carbondata-spark-common-test/1.2.0-SNAPSHOT/carbondata-spark-common-test-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6@2/integration/spark/pom.xml
 to 
org.apache.carbondata/carbondata-spark/1.2.0-SNAPSHOT/carbondata-spark-1.2.0-SNAPSHOT.pomNo
 artifacts from carbondata-pr-spark-1.6 » Apache CarbonData :: Spark #242 to 
compare, so performing full copy of artifactschannel stoppedSetting status of 
df2daa0c967b3756a818ebf51aeb701fcda4e3c7 to FAILURE with url 
https://builds.apache.org/job/carbondata-pr-spark-1.6/243/ and message: 'Tests 
Failed for Spark1.6 'Using context: Jenkins(Spark1.6): mvn clean test 
-Pspark-1.6



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #958: [CARBONDATA-1088] Added interfaces for Data Map frame...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/958
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2362/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #958: [CARBONDATA-1088] Added interfaces for Data Map frame...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/958
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/240/Build result: 
FAILURE[...truncated 60.68 KB...][ERROR] For more information about 
the errors and possible solutions, please read the following articles:[ERROR] 
[Help 1] 
http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException[ERROR] 
[ERROR] After correcting the problems, you can resume the build with the 
command[ERROR]   mvn  -rf :carbondata-hadoop[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/processing/pom.xml
 to 
org.apache.carbondata/carbondata-processing/1.2.0-SNAPSHOT/carbondata-processing-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integration/spark-common/pom.xml
 to 
org.apache.carbondata/carbondata-spark-common/1.2.0-SNAPSHOT/carbondata-spark-common-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/examples/spark/pom.xml
 to org.apache.carbo
 
ndata/carbondata-examples-spark/1.2.0-SNAPSHOT/carbondata-examples-spark-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integration/hive/pom.xml
 to 
org.apache.carbondata/carbondata-hive/1.2.0-SNAPSHOT/carbondata-hive-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/common/pom.xml to 
org.apache.carbondata/carbondata-common/1.2.0-SNAPSHOT/carbondata-common-1.2.0-SNAPSHOT.pomNo
 artifacts from carbondata-pr-spark-1.6 » Apache CarbonData :: Common #241 to 
compare, so performing full copy of artifacts[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/assembly/pom.xml 
to 
org.apache.carbondata/carbondata-assembly/1.2.0-SNAPSHOT/carbondata-assembly-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/pom.xml to 
org.apache.carbondata/carbondata-parent/1.2.0-SNAPSHOT/carbondata-parent-1.2.0-SNAPSHOT.pomNo
 artifact
 s from carbondata-pr-spark-1.6 » Apache CarbonData :: Parent #241 to compare, 
so performing full copy of artifacts[JENKINS] Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/examples/flink/pom.xml
 to 
org.apache.carbondata/carbondata-examples-flink/1.2.0-SNAPSHOT/carbondata-examples-flink-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/core/pom.xml to 
org.apache.carbondata/carbondata-core/1.2.0-SNAPSHOT/carbondata-core-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/hadoop/pom.xml to 
org.apache.carbondata/carbondata-hadoop/1.2.0-SNAPSHOT/carbondata-hadoop-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integration/presto/pom.xml
 to 
org.apache.carbondata/carbondata-presto/1.2.0-SNAPSHOT/carbondata-presto-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving /home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integr
 ation/spark-common-test/pom.xml to 
org.apache.carbondata/carbondata-spark-common-test/1.2.0-SNAPSHOT/carbondata-spark-common-test-1.2.0-SNAPSHOT.pom[JENKINS]
 Archiving 
/home/jenkins/jenkins-slave/workspace/carbondata-pr-spark-1.6/integration/spark/pom.xml
 to 
org.apache.carbondata/carbondata-spark/1.2.0-SNAPSHOT/carbondata-spark-1.2.0-SNAPSHOT.pomchannel
 stoppedSetting status of 7240ac9dc4a86f3268a07ca472e9028fd9905f4e to FAILURE 
with url https://builds.apache.org/job/carbondata-pr-spark-1.6/240/ and 
message: 'Tests Failed for Spark1.6 'Using context: Jenkins(Spark1.6): mvn 
clean test -Pspark-1.6



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #958: [CARBONDATA-1088] Added interfaces for Data Map frame...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/958
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2360/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #958: [CARBONDATA-1088] Added interfaces for Data Map frame...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/958
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/239/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1018: [Just For Test] Just test PR1105

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1018
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2359/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2358/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1018: [Just For Test] Just test PR1105

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1018
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2355/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Updated] (CARBONDATA-1146) V3 format support for delete operation in IUD.

2017-06-11 Thread Ravindra Pesala (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-1146?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ravindra Pesala updated CARBONDATA-1146:

Fix Version/s: 1.1.1

> V3 format support for delete operation in IUD.
> --
>
> Key: CARBONDATA-1146
> URL: https://issues.apache.org/jira/browse/CARBONDATA-1146
> Project: CarbonData
>  Issue Type: Sub-task
>Reporter: Manohar Vanam
>Priority: Minor
> Fix For: 1.2.0, 1.1.1
>
>  Time Spent: 5h
>  Remaining Estimate: 0h
>
> delete operation should handle V3 format. i.e pages of blocklet should be 
> considered in the delete operation.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Resolved] (CARBONDATA-1146) V3 format support for delete operation in IUD.

2017-06-11 Thread Ravindra Pesala (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-1146?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ravindra Pesala resolved CARBONDATA-1146.
-
   Resolution: Fixed
Fix Version/s: 1.2.0

> V3 format support for delete operation in IUD.
> --
>
> Key: CARBONDATA-1146
> URL: https://issues.apache.org/jira/browse/CARBONDATA-1146
> Project: CarbonData
>  Issue Type: Sub-task
>Reporter: Manohar Vanam
>Priority: Minor
> Fix For: 1.2.0
>
>  Time Spent: 5h
>  Remaining Estimate: 0h
>
> delete operation should handle V3 format. i.e pages of blocklet should be 
> considered in the delete operation.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[GitHub] carbondata issue #1017: [CARBONDATA-1105] support spark 2.1.1

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/1017
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/233/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1017: [CARBONDATA-1105] support spark 2.1.1

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1017
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2354/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1017: [CARBONDATA-1105] support spark 2.1.1

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/1017
  
Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #1017: [CARBONDATA-1105] support spark 2.1.1

2017-06-11 Thread cenyuhai
GitHub user cenyuhai opened a pull request:

https://github.com/apache/carbondata/pull/1017

[CARBONDATA-1105] support spark 2.1.1


support spark 2.1.1

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/cenyuhai/incubator-carbondata CARBONDATA-1105

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/carbondata/pull/1017.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #1017


commit ec3f00133073428bc2d31796c441e0c4c79fb130
Author: cenyuhai <261810...@qq.com>
Date:   2017-06-11T09:21:58Z

support spark 2.1.1




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Commented] (CARBONDATA-1105) ClassNotFoundException: org.apache.spark.sql.catalyst.CatalystConf

2017-06-11 Thread cen yuhai (JIRA)

[ 
https://issues.apache.org/jira/browse/CARBONDATA-1105?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16045876#comment-16045876
 ] 

cen yuhai commented on CARBONDATA-1105:
---

I think we should support spark2.1.1, right? spark2.1.0 is not stable

> ClassNotFoundException: org.apache.spark.sql.catalyst.CatalystConf
> --
>
> Key: CARBONDATA-1105
> URL: https://issues.apache.org/jira/browse/CARBONDATA-1105
> Project: CarbonData
>  Issue Type: Bug
>  Components: core
>Affects Versions: 1.2.0
> Environment: spark 2.1.1
>Reporter: cen yuhai
>
> I think it is related to SPARK-19944
> https://github.com/apache/spark/pull/17301
> {code}
> scala> carbon.sql("create table temp.test_carbon(id int, name string, scale 
> decimal, country string, salary double) STORED BY 'carbondata'")
> java.lang.NoClassDefFoundError: org/apache/spark/sql/catalyst/CatalystConf
>   at 
> org.apache.spark.sql.hive.CarbonSessionState.analyzer$lzycompute(CarbonSessionState.scala:127)
>   at 
> org.apache.spark.sql.hive.CarbonSessionState.analyzer(CarbonSessionState.scala:126)
>   at 
> org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:69)
>   at 
> org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:67)
>   at 
> org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:50)
>   at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:63)
>   at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:593)
>   ... 52 elided
> Caused by: java.lang.ClassNotFoundException: 
> org.apache.spark.sql.catalyst.CatalystConf
>   at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
>   at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
>   at java.security.AccessController.doPrivileged(Native Method)
>   at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
>   at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
>   at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>   ... 59 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Assigned] (CARBONDATA-1105) ClassNotFoundException: org.apache.spark.sql.catalyst.CatalystConf

2017-06-11 Thread cen yuhai (JIRA)

 [ 
https://issues.apache.org/jira/browse/CARBONDATA-1105?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

cen yuhai reassigned CARBONDATA-1105:
-

Assignee: cen yuhai

> ClassNotFoundException: org.apache.spark.sql.catalyst.CatalystConf
> --
>
> Key: CARBONDATA-1105
> URL: https://issues.apache.org/jira/browse/CARBONDATA-1105
> Project: CarbonData
>  Issue Type: Bug
>  Components: core
>Affects Versions: 1.2.0
> Environment: spark 2.1.1
>Reporter: cen yuhai
>Assignee: cen yuhai
>
> I think it is related to SPARK-19944
> https://github.com/apache/spark/pull/17301
> {code}
> scala> carbon.sql("create table temp.test_carbon(id int, name string, scale 
> decimal, country string, salary double) STORED BY 'carbondata'")
> java.lang.NoClassDefFoundError: org/apache/spark/sql/catalyst/CatalystConf
>   at 
> org.apache.spark.sql.hive.CarbonSessionState.analyzer$lzycompute(CarbonSessionState.scala:127)
>   at 
> org.apache.spark.sql.hive.CarbonSessionState.analyzer(CarbonSessionState.scala:126)
>   at 
> org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:69)
>   at 
> org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:67)
>   at 
> org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:50)
>   at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:63)
>   at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:593)
>   ... 52 elided
> Caused by: java.lang.ClassNotFoundException: 
> org.apache.spark.sql.catalyst.CatalystConf
>   at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
>   at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
>   at java.security.AccessController.doPrivileged(Native Method)
>   at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
>   at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
>   at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>   ... 59 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2353/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #890: [CARBONDATA-1008] Make Caron table schema compatible ...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/890
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2352/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/1011
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/232/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #984: [CARBONDATA-1008] Make Caron table schema compatible ...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/984
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2351/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #890: [CARBONDATA-1008] Make Caron table schema compatible ...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/890
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/231/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #984: [CARBONDATA-1008] Make Caron table schema compatible ...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/984
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/230/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata pull request #988: [CARBONDATA-1110] put if clause out of the for...

2017-06-11 Thread mayunSaicmotor
GitHub user mayunSaicmotor reopened a pull request:

https://github.com/apache/carbondata/pull/988

[CARBONDATA-1110] put if clause out of the for clause

it should be better to put if clause out of the for clause

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/mayunSaicmotor/incubator-carbondata 
CARBON-1110-NEW

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/carbondata/pull/988.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #988


commit 324ecffb98d58362f5e2a3560aa0ecac3956848f
Author: mayun 
Date:   2017-06-02T04:55:36Z

put if clause out of the for clause

commit a725ab6766f3cbe89974b09aa819aebd85c53846
Author: mayun 
Date:   2017-06-03T01:51:34Z

add comments

commit 5d7767d110f1421275733bebccbcf560dff7e1b2
Author: mayun 
Date:   2017-06-05T09:56:00Z

remove comments

commit eca1c4339b180e8232236f4a9d2729dcf59389f0
Author: mayun 
Date:   2017-06-08T04:51:53Z

remove comments

commit 1dd10202c8048d323d8960a6eb192ab217d8b0b5
Author: mayun 
Date:   2017-06-08T08:41:58Z

add comment

commit 5827e68446a8436de5571164d43a73b1a1f4d759
Author: mayun 
Date:   2017-06-08T08:49:47Z

remove comment




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/1011
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/228/Failed Tests: 
4carbondata-pr-spark-1.6/org.apache.carbondata:carbondata-spark-common-test:
 4org.apache.carbondata.spark.testsuite.allqueries.InsertIntoCarbonTableTestCase.insert
 into carbon table from carbon table union queryorg.apache.carbondata.spark.testsuite.dataretention.DataRetentionConcurrencyTestCase.DataRetention_Concurrency_load_dateorg.apache.carbondata.spark.testsuite.partition.TestDDLForPartitionTable.test
 exception when values in list_info can not match partition column 
typeorg.apache.carbondata.spark.testsuite.partition.TestDDLForPartitionTable.test
 exception when values in range_info can not match partition column 
type



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2349/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread asfgit
Github user asfgit commented on the issue:

https://github.com/apache/carbondata/pull/1011
  

Refer to this link for build results (access rights to CI server needed): 
https://builds.apache.org/job/carbondata-pr-spark-1.6/227/Failed Tests: 
2carbondata-pr-spark-1.6/org.apache.carbondata:carbondata-spark-common-test:
 2org.apache.carbondata.spark.testsuite.partition.TestDDLForPartitionTable.test
 exception when values in list_info can not match partition column 
typeorg.apache.carbondata.spark.testsuite.partition.TestDDLForPartitionTable.test
 exception when values in range_info can not match partition column 
type



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] carbondata issue #1011: [carbondata-1149] Fix issue of mismatch type of part...

2017-06-11 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1011
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2348/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---