[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread xubo245
Github user xubo245 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164663023
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapClassName.java
 ---
@@ -0,0 +1,52 @@
+/*
+ * 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.core.metadata.schema.table;
+
+/**
+ * type for create datamap
+ * The syntax of datamap creation is as follows.
+ * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 
'DataMapClassName'
+ * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
+ *
+ * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
+ */
+
+public enum DataMapClassName {
+  PREAGGREGATE((short) 1, "preaggregate"),
+  TIMESERIES((short) 2, "timeseries");
+  private int value;
--- End diff --

ok, done


---


[GitHub] carbondata issue #1819: [CARBONDATA-1964] Fixed bug to set bad.records.actio...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1819
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3252/



---


[GitHub] carbondata issue #1819: [CARBONDATA-1964] Fixed bug to set bad.records.actio...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1819
  
Build Failed with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2017/



---


[GitHub] carbondata pull request #1856: [CARBONDATA-2073][CARBONDATA-1516][Tests] Add...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1856#discussion_r164660048
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesMatchStrategySuite.scala
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.integration.spark.testsuite.timeseries
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Row}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.test.util.QueryTest
+import org.apache.spark.util.SparkUtil4Test
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+class TestTimeSeriesMatchStrategySuite extends QueryTest with 
BeforeAndAfterAll with BeforeAndAfterEach {
+
+  override def beforeAll: Unit = {
+SparkUtil4Test.createTaskMockUp(sqlContext)
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+
+  }
+
+  override protected def beforeEach(): Unit = {
+sql("drop table if exists mainTable")
+sql(
+  """
+| CREATE TABLE mainTable(
+| mytime timestamp,
+| name string,
+| age int)
+| STORED BY 'org.apache.carbondata.format'
+  """.stripMargin)
+sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into 
table mainTable")
+  }
+
+  val timeSeries = "preaggregate"
+
+  test("test timeseries match 1: select small size when there are >1 
timeseries match") {
+
+dropDataMaps("maintable", "agg1_minute", "agg2_minute")
+sql(
+  s"""
+ | create datamap agg1 on table mainTable
--- End diff --

change datamap name to big_agg


---


[GitHub] carbondata pull request #1856: [CARBONDATA-2073][CARBONDATA-1516][Tests] Add...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1856#discussion_r164659399
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesMatchStrategySuite.scala
 ---
@@ -0,0 +1,294 @@
+/*
+ * 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.integration.spark.testsuite.timeseries
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Row}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.test.util.QueryTest
+import org.apache.spark.util.SparkUtil4Test
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+class TestTimeSeriesMatchStrategySuite extends QueryTest with 
BeforeAndAfterAll with BeforeAndAfterEach {
+
+  override def beforeAll: Unit = {
+SparkUtil4Test.createTaskMockUp(sqlContext)
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+
+  }
+
+  override protected def beforeEach(): Unit = {
+sql("drop table if exists mainTable")
+sql(
+  """
+| CREATE TABLE mainTable(
+| mytime timestamp,
+| name string,
+| age int)
+| STORED BY 'org.apache.carbondata.format'
+  """.stripMargin)
+sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/timeseriestest.csv' into 
table mainTable")
+  }
+
+  val timeSeries = "preaggregate"
+
+  test("test timeseries match 1: select small size when there are >1 
timeseries match") {
+
+dropDataMaps("maintable", "agg1_minute", "agg2_minute")
+sql(
+  s"""
+ | create datamap agg1 on table mainTable
+ | using '$timeSeries'
+ | DMPROPERTIES (
+ |   'timeseries.eventTime'='mytime',
+ |   'timeseries.hierarchy'='minute=1')
+ | as select mytime, sum(age), count(age), max(age), min(age)
+ | from mainTable
+ | group by mytime
+  """.stripMargin)
+
+val df1 = sql(
+  """
+| select
+|   timeseries(mytime,'minute') as minuteLevel,
+|   sum(age) as sum
+| from mainTable
+| where
+| timeseries(mytime,'minute')<'2016-02-23 01:02:00' and
+| timeseries(mytime,'minute')>='2016-02-23 01:01:00'
+| group by
+|   timeseries(mytime,'minute')
+| order by
+|   timeseries(mytime,'minute')
+  """.stripMargin)
+
+checkAnswer(df1, Seq(Row(Timestamp.valueOf("2016-02-23 01:01:00"), 
60)))
+
+preAggTableValidator(df1.queryExecution.analyzed, 
"maintable_agg1_minute")
+
+sql(
+  s"""
+ | create datamap agg2 on table mainTable
+ | using '$timeSeries'
+ | DMPROPERTIES (
+ |   'timeseries.eventTime'='mytime',
+ |   'timeseries.hierarchy'='minute=1')
+ | as select mytime, sum(age)
+ | from mainTable
+ | group by mytime
+  """.stripMargin)
+
+val df2 = sql(
+  """
+| select
+|   timeseries(mytime,'minute') as minuteLevel,
+|   sum(age) as sum
+| from mainTable
+| where
+| timeseries(mytime,'minute')<'2016-02-23 01:02:00' and
+| timeseries(mytime,'minute')>='2016-02-23 01:01:00'
+| group by
+|   timeseries(mytime,'minute')
+| order by
+|   timeseries(mytime,'minute')

[GitHub] carbondata issue #1879: [WIP] Fix measure min/max value problem while readin...

2018-01-29 Thread jackylk
Github user jackylk commented on the issue:

https://github.com/apache/carbondata/pull/1879
  
Can you add testcase for it?


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164658274
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapClassName.java
 ---
@@ -0,0 +1,52 @@
+/*
+ * 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.core.metadata.schema.table;
+
+/**
+ * type for create datamap
+ * The syntax of datamap creation is as follows.
+ * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 
'DataMapClassName'
+ * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
+ *
+ * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
+ */
+
+public enum DataMapClassName {
--- End diff --

I suggest `DataMapProvider`


---


[jira] [Assigned] (CARBONDATA-1948) Update help document for the change made for CARBONDATA-1929

2018-01-29 Thread Gururaj Shetty (JIRA)

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

Gururaj Shetty reassigned CARBONDATA-1948:
--

Assignee: Gururaj Shetty

> Update help document for the change made for CARBONDATA-1929
> 
>
> Key: CARBONDATA-1948
> URL: https://issues.apache.org/jira/browse/CARBONDATA-1948
> Project: CarbonData
>  Issue Type: Sub-task
>Reporter: Mohammad Shahid Khan
>Assignee: Gururaj Shetty
>Priority: Minor
>
> carbon.sort.file.write.buffer.size (minValue = 10 KB, maxValue=10MB, 
> defaultValue =16 KB )
> carbon.sort.intermediate.files.limit (minValue = 2, maxValue=50, defaultValue 
> =20 )



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[GitHub] carbondata issue #1104: [CARBONDATA-1239] Add validation for set command par...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1104
  
Build Success with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2016/



---


[GitHub] carbondata issue #1704: [CARBONDATA-1918] Incorrect data is displayed when S...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1704
  
SDV Build Fail , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3199/



---


[GitHub] carbondata issue #1104: [CARBONDATA-1239] Add validation for set command par...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1104
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3251/



---


[GitHub] carbondata issue #1831: [CARBONDATA-1993] Carbon properties default values f...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1831
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3250/



---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164657031
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ---
@@ -1525,9 +1525,7 @@
*/
   public static final long HANDOFF_SIZE_DEFAULT = 1024L * 1024 * 1024;
 
-  public static final String TIMESERIES_EVENTTIME = "timeseries.eventtime";
-
-  public static final String TIMESERIES_HIERARCHY = "timeseries.hierarchy";
+  public static final String TIMESERIES_EVENTTIME = "event_time";
--- End diff --

Declare the constant in datamap class, not in CarbonCommonConstaints


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164656822
  
--- Diff: 
integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/UnsupportedDataMapException.java
 ---
@@ -0,0 +1,32 @@
+/*
+ * 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.exception;
+
+/**
+ * Throw exception when using unsupported datamap type
+ */
+public class UnsupportedDataMapException extends 
MalformedCarbonCommandException {
--- End diff --

Rename to `MalformedDataMapCommandException`


---


[jira] [Assigned] (CARBONDATA-1948) Update help document for the change made for CARBONDATA-1929

2018-01-29 Thread Gururaj Shetty (JIRA)

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

Gururaj Shetty reassigned CARBONDATA-1948:
--

Assignee: (was: Gururaj Shetty)

> Update help document for the change made for CARBONDATA-1929
> 
>
> Key: CARBONDATA-1948
> URL: https://issues.apache.org/jira/browse/CARBONDATA-1948
> Project: CarbonData
>  Issue Type: Sub-task
>Reporter: Mohammad Shahid Khan
>Priority: Minor
>
> carbon.sort.file.write.buffer.size (minValue = 10 KB, maxValue=10MB, 
> defaultValue =16 KB )
> carbon.sort.intermediate.files.limit (minValue = 2, maxValue=50, defaultValue 
> =20 )



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164656572
  
--- Diff: 
integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/CarbonIllegalArgumentException.java
 ---
@@ -0,0 +1,32 @@
+/*
+ * 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.exception;
+
+/**
+ * Throw exception when using illegal argument
+ */
+public class CarbonIllegalArgumentException extends 
MalformedCarbonCommandException {
--- End diff --

Make it specific: UnsupportedDataMapException


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164656476
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/Granularity.java
 ---
@@ -0,0 +1,62 @@
+/*
+ * 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.core.metadata.schema.table;
+
+/**
+ * type for create datamap
+ * The syntax of datamap creation is as follows.
+ * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 
'DataMapClassName'
+ * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
+ *
+ * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
+ */
+
+public enum Granularity {
+  YEAR((short) 1, "year_granularity", "year"),
+  MONTH((short) 2, "month_granularity", "month"),
+  DAY((short) 3, "day_granularity", "day"),
+  HOUR((short) 4, "hour_granularity", "hour"),
+  MINUTE((short) 5, "minute_granularity", "minute"),
+  SECOND((short) 6, "second_granularity", "second");
+  private int value;
+  private String name;
+  private String time;
--- End diff --

This is not required


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164656340
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/Granularity.java
 ---
@@ -0,0 +1,62 @@
+/*
+ * 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.core.metadata.schema.table;
+
+/**
+ * type for create datamap
+ * The syntax of datamap creation is as follows.
+ * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 
'DataMapClassName'
+ * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
+ *
+ * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
+ */
+
+public enum Granularity {
+  YEAR((short) 1, "year_granularity", "year"),
+  MONTH((short) 2, "month_granularity", "month"),
+  DAY((short) 3, "day_granularity", "day"),
+  HOUR((short) 4, "hour_granularity", "hour"),
+  MINUTE((short) 5, "minute_granularity", "minute"),
+  SECOND((short) 6, "second_granularity", "second");
+  private int value;
--- End diff --

int is not required


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164656315
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/Granularity.java
 ---
@@ -0,0 +1,62 @@
+/*
+ * 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.core.metadata.schema.table;
--- End diff --

Move it to time series package


---


[GitHub] carbondata pull request #1876: [CARBONDATA-2093] Use small file feature of g...

2018-01-29 Thread QiangCai
Github user QiangCai commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1876#discussion_r164655879
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
 ---
@@ -414,6 +416,75 @@ class StandardPartitionTableLoadingTestCase extends 
QueryTest with BeforeAndAfte
   sql("select * from  casesensitivepartition where empno=17"))
   }
 
+  test("Partition LOAD with small files") {
+sql("DROP TABLE IF EXISTS smallpartitionfiles")
+sql(
+  """
+| CREATE TABLE smallpartitionfiles(id INT, name STRING, age INT) 
PARTITIONED BY(city STRING)
+| STORED BY 'org.apache.carbondata.format'
+  """.stripMargin)
+val inputPath = new File("target/small_files").getCanonicalPath
+val folder = new File(inputPath)
+if (folder.exists()) {
+  FileUtils.deleteDirectory(folder)
+}
+folder.mkdir()
+for (i <- 0 to 100) {
+  val file = s"$folder/file$i.csv"
+  val writer = new FileWriter(file)
+  writer.write("id,name,city,age\n")
+  writer.write(s"$i,name_$i,city_${i % 5},${ i % 100 }")
+  writer.close()
+}
+sql(s"LOAD DATA LOCAL INPATH '$inputPath' INTO TABLE 
smallpartitionfiles")
+FileUtils.deleteDirectory(folder)
+val carbonTable = 
CarbonMetadata.getInstance().getCarbonTable("default", "smallpartitionfiles")
+val carbonTablePath = 
CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
+val segmentDir = carbonTablePath.getSegmentDir("0", "0")
+assert(new File(segmentDir).listFiles().length < 50)
+  }
+
+  test("verify partition read with small files") {
+try {
+  
CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_TASK_DISTRIBUTION,
+CarbonCommonConstants.CARBON_TASK_DISTRIBUTION_MERGE_FILES)
+  sql("DROP TABLE IF EXISTS smallpartitionfilesread")
+  sql(
+"""
+  | CREATE TABLE smallpartitionfilesread(id INT, name STRING, age 
INT) PARTITIONED BY
+  | (city STRING)
+  | STORED BY 'org.apache.carbondata.format'
+""".stripMargin)
+  val inputPath = new File("target/small_files").getCanonicalPath
+  val folder = new File(inputPath)
+  if (folder.exists()) {
+FileUtils.deleteDirectory(folder)
+  }
+  folder.mkdir()
+  for (i <- 0 until 100) {
+val file = s"$folder/file$i.csv"
+val writer = new FileWriter(file)
+writer.write("id,name,city,age\n")
+writer.write(s"$i,name_$i,city_${ i },${ i % 100 }")
+writer.close()
+  }
+  sql(s"LOAD DATA LOCAL INPATH '$inputPath' INTO TABLE 
smallpartitionfilesread")
+  FileUtils.deleteDirectory(folder)
+  val dataFrame = sql("select * from smallpartitionfilesread")
+  val scanRdd = dataFrame.queryExecution.sparkPlan.collect {
+case b: BatchedDataSourceScanExec if 
b.rdd.isInstanceOf[CarbonScanRDD] => b.rdd
+  .asInstanceOf[CarbonScanRDD]
+  }.head
+  assert(scanRdd.getPartitions.length < 10)
+  assertResult(100)(dataFrame.collect().length)
--- End diff --

suggest to use dataFrame.count


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164655685
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapClassName.java
 ---
@@ -0,0 +1,52 @@
+/*
+ * 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.core.metadata.schema.table;
+
+/**
+ * type for create datamap
+ * The syntax of datamap creation is as follows.
+ * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 
'DataMapClassName'
+ * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
+ *
+ * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
+ */
+
+public enum DataMapClassName {
+  PREAGGREGATE((short) 1, "preaggregate"),
+  TIMESERIES((short) 2, "timeseries");
+  private int value;
--- End diff --

You do not need this int value


---


[jira] [Assigned] (CARBONDATA-1948) Update help document for the change made for CARBONDATA-1929

2018-01-29 Thread Gururaj Shetty (JIRA)

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

Gururaj Shetty reassigned CARBONDATA-1948:
--

Assignee: Gururaj Shetty  (was: Varun Saxena)

> Update help document for the change made for CARBONDATA-1929
> 
>
> Key: CARBONDATA-1948
> URL: https://issues.apache.org/jira/browse/CARBONDATA-1948
> Project: CarbonData
>  Issue Type: Sub-task
>Reporter: Mohammad Shahid Khan
>Assignee: Gururaj Shetty
>Priority: Minor
>
> carbon.sort.file.write.buffer.size (minValue = 10 KB, maxValue=10MB, 
> defaultValue =16 KB )
> carbon.sort.intermediate.files.limit (minValue = 2, maxValue=50, defaultValue 
> =20 )



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[GitHub] carbondata issue #1831: [CARBONDATA-1993] Carbon properties default values f...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1831
  
Build Success with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2014/



---


[GitHub] carbondata pull request #1878: [CARBONDATA-2094] Filter DataMap Tables in Sh...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1878#discussion_r164654501
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/ShowTablesCommand.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.table
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.execution.command.MetadataCommand
+import org.apache.spark.sql.types.{BooleanType, StringType}
+
+
+private[sql] case class ShowTablesCommand ( databaseName: Option[String],
+tableIdentifierPattern: Option[String])  extends MetadataCommand{
+
+  // The result of SHOW TABLES has three columns: database, tableName and 
isTemporary.
+  override val output: Seq[Attribute] = {
+AttributeReference("database", StringType, nullable = false)() ::
+AttributeReference("tableName", StringType, nullable = false)() ::
+AttributeReference("isTemporary", BooleanType, nullable = false)() :: 
Nil
+  }
+
+  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
+// Since we need to return a Seq of rows, we will call getTables 
directly
+// instead of calling tables in sparkSession.
+// filterDataMaps Method is to Filter the Table.
+val catalog = sparkSession.sessionState.catalog
+val db = databaseName.getOrElse(catalog.getCurrentDatabase)
+var tables =
+  tableIdentifierPattern.map(catalog.listTables(db, 
_)).getOrElse(catalog.listTables(db))
+tables = filterDataMaps(tables, sparkSession)
+tables.map { tableIdent =>
+  val isTemp = catalog.isTemporaryTable(tableIdent)
+  Row(tableIdent.database.getOrElse(""), tableIdent.table, isTemp)
+}
+  }
+
+  private def filterDataMaps(tables: Seq[TableIdentifier],
+  sparkSession: SparkSession): Seq[TableIdentifier] = {
+// Filter carbon Tables then get CarbonTable and getDataMap List and 
filter the same
+// as of now 2 times lookup is happening(filter  carbon table 
,getDataMapList)
+//  add another PR to improve  with 1 lookup
+val allDatamapTable = tables.filter(tbs => {
--- End diff --

`tables.filter(tbs => {` change to 
`tables.filter { table =>`


---


[GitHub] carbondata pull request #1878: [CARBONDATA-2094] Filter DataMap Tables in Sh...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1878#discussion_r164654398
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/ShowTablesCommand.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.table
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.execution.command.MetadataCommand
+import org.apache.spark.sql.types.{BooleanType, StringType}
+
+
+private[sql] case class ShowTablesCommand ( databaseName: Option[String],
+tableIdentifierPattern: Option[String])  extends MetadataCommand{
+
+  // The result of SHOW TABLES has three columns: database, tableName and 
isTemporary.
+  override val output: Seq[Attribute] = {
+AttributeReference("database", StringType, nullable = false)() ::
+AttributeReference("tableName", StringType, nullable = false)() ::
+AttributeReference("isTemporary", BooleanType, nullable = false)() :: 
Nil
+  }
+
+  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
+// Since we need to return a Seq of rows, we will call getTables 
directly
+// instead of calling tables in sparkSession.
+// filterDataMaps Method is to Filter the Table.
+val catalog = sparkSession.sessionState.catalog
+val db = databaseName.getOrElse(catalog.getCurrentDatabase)
+var tables =
+  tableIdentifierPattern.map(catalog.listTables(db, 
_)).getOrElse(catalog.listTables(db))
+tables = filterDataMaps(tables, sparkSession)
+tables.map { tableIdent =>
+  val isTemp = catalog.isTemporaryTable(tableIdent)
+  Row(tableIdent.database.getOrElse(""), tableIdent.table, isTemp)
+}
+  }
+
+  private def filterDataMaps(tables: Seq[TableIdentifier],
+  sparkSession: SparkSession): Seq[TableIdentifier] = {
+// Filter carbon Tables then get CarbonTable and getDataMap List and 
filter the same
+// as of now 2 times lookup is happening(filter  carbon table 
,getDataMapList)
+//  add another PR to improve  with 1 lookup
--- End diff --

change  to TODO
and add the JIRA number here


---


[GitHub] carbondata pull request #1878: [CARBONDATA-2094] Filter DataMap Tables in Sh...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1878#discussion_r164654277
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/ShowTablesCommand.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.table
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.execution.command.MetadataCommand
+import org.apache.spark.sql.types.{BooleanType, StringType}
+
+
+private[sql] case class ShowTablesCommand ( databaseName: Option[String],
+tableIdentifierPattern: Option[String])  extends MetadataCommand{
+
+  // The result of SHOW TABLES has three columns: database, tableName and 
isTemporary.
+  override val output: Seq[Attribute] = {
+AttributeReference("database", StringType, nullable = false)() ::
+AttributeReference("tableName", StringType, nullable = false)() ::
+AttributeReference("isTemporary", BooleanType, nullable = false)() :: 
Nil
+  }
+
+  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
+// Since we need to return a Seq of rows, we will call getTables 
directly
+// instead of calling tables in sparkSession.
+// filterDataMaps Method is to Filter the Table.
+val catalog = sparkSession.sessionState.catalog
+val db = databaseName.getOrElse(catalog.getCurrentDatabase)
+var tables =
+  tableIdentifierPattern.map(catalog.listTables(db, 
_)).getOrElse(catalog.listTables(db))
+tables = filterDataMaps(tables, sparkSession)
+tables.map { tableIdent =>
+  val isTemp = catalog.isTemporaryTable(tableIdent)
+  Row(tableIdent.database.getOrElse(""), tableIdent.table, isTemp)
+}
+  }
+
+  private def filterDataMaps(tables: Seq[TableIdentifier],
--- End diff --

please add comment


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread xubo245
Github user xubo245 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164654217
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapClassName.java
 ---
@@ -0,0 +1,52 @@
+/*
+ * 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.core.metadata.schema.table;
+
+/**
+ * type for create datamap
+ * The syntax of datamap creation is as follows.
+ * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 
'DataMapClassName'
+ * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
+ *
+ * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
+ */
+
+public enum DataMapClassName {
--- End diff --

But now we don't support className. How about change DataMapClassName to 
DataMapName?


---


[GitHub] carbondata pull request #1875: [CARBONDATA-2092] Fix compaction bug to preve...

2018-01-29 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1875#discussion_r164653466
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
 ---
@@ -165,6 +165,7 @@ private static BitSet getPresenceMeta(
 dataFileFooter.setBlockInfo(new BlockInfo(tableBlockInfo));
 dataFileFooter.setSegmentInfo(segmentInfo);
 dataFileFooter.setVersionId(tableBlockInfo.getVersion());
+
dataFileFooter.setSchemaUpdatedTimeStamp(readIndexHeader.getSchema_time_stamp());
--- End diff --

What if it is from old store? better check 
`readIndexHeader.isSetSchema_time_stamp` before setting it to footer


---


[GitHub] carbondata pull request #1878: [CARBONDATA-2094] Filter DataMap Tables in Sh...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1878#discussion_r164653281
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/ShowTablesCommand.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.table
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.execution.command.MetadataCommand
+import org.apache.spark.sql.types.{BooleanType, StringType}
+
+
+private[sql] case class ShowTablesCommand ( databaseName: Option[String],
+tableIdentifierPattern: Option[String])  extends MetadataCommand{
+
+  // The result of SHOW TABLES has three columns: database, tableName and 
isTemporary.
+  override val output: Seq[Attribute] = {
+AttributeReference("database", StringType, nullable = false)() ::
+AttributeReference("tableName", StringType, nullable = false)() ::
+AttributeReference("isTemporary", BooleanType, nullable = false)() :: 
Nil
+  }
+
+  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
+// Since we need to return a Seq of rows, we will call getTables 
directly
+// instead of calling tables in sparkSession.
+// filterDataMaps Method is to Filter the Table.
+val catalog = sparkSession.sessionState.catalog
+val db = databaseName.getOrElse(catalog.getCurrentDatabase)
+var tables =
+  tableIdentifierPattern.map(catalog.listTables(db, 
_)).getOrElse(catalog.listTables(db))
+tables = filterDataMaps(tables, sparkSession)
+tables.map { tableIdent =>
+  val isTemp = catalog.isTemporaryTable(tableIdent)
+  Row(tableIdent.database.getOrElse(""), tableIdent.table, isTemp)
--- End diff --

`tableIdent.database.getOrElse("")` is not right, default database is 
`default`


---


[GitHub] carbondata pull request #1878: [CARBONDATA-2094] Filter DataMap Tables in Sh...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1878#discussion_r164653059
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/ShowTablesCommand.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.table
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.execution.command.MetadataCommand
+import org.apache.spark.sql.types.{BooleanType, StringType}
+
+
+private[sql] case class ShowTablesCommand ( databaseName: Option[String],
+tableIdentifierPattern: Option[String])  extends MetadataCommand{
+
+  // The result of SHOW TABLES has three columns: database, tableName and 
isTemporary.
+  override val output: Seq[Attribute] = {
+AttributeReference("database", StringType, nullable = false)() ::
+AttributeReference("tableName", StringType, nullable = false)() ::
+AttributeReference("isTemporary", BooleanType, nullable = false)() :: 
Nil
+  }
+
+  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
+// Since we need to return a Seq of rows, we will call getTables 
directly
+// instead of calling tables in sparkSession.
+// filterDataMaps Method is to Filter the Table.
+val catalog = sparkSession.sessionState.catalog
+val db = databaseName.getOrElse(catalog.getCurrentDatabase)
--- End diff --

Use `CarbonEnv.getDatabaseName` instead


---


[GitHub] carbondata issue #1855: [CARBONDATA-1796]while submiting new job, pass crede...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1855
  
Build Failed with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2015/



---


[GitHub] carbondata pull request #1878: [CARBONDATA-2094] Filter DataMap Tables in Sh...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1878#discussion_r164652985
  
--- Diff: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
 ---
@@ -212,6 +212,21 @@ class TestPreAggCreateCommand extends QueryTest with 
BeforeAndAfterAll {
 sql("drop datamap agg0 on table maintable")
   }
 
+  test("remove agg tables from show table command") {
+sql("DROP TABLE IF EXISTS tbl_1")
+sql("DROP TABLE IF EXISTS sparktable")
+sql("create table if not exists  tbl_1(imei string,age int,mac string 
,prodate timestamp,update timestamp,gamepoint double,contrid double) stored by 
'carbondata' ")
+sql("create table if not exists sparktable(a int,b string)")
+sql(
+  s"""create datamap preagg_sum on table tbl_1 using 'preaggregate' as 
select mac,avg(age) from tbl_1 group by mac"""
+.stripMargin)
+sql(
+  "create datamap agg2 on table tbl_1 using 'preaggregate' 
DMPROPERTIES ('timeseries" +
--- End diff --

Please split this testcase into two testcases, one for preaggregation 
datamap, another for time series datamap


---


[GitHub] carbondata pull request #1867: [CARBONDATA-2055][Streaming][WIP]Support inte...

2018-01-29 Thread chenliang613
Github user chenliang613 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1867#discussion_r164652953
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/CarbonStreamSparkStreamFactory.scala
 ---
@@ -0,0 +1,58 @@
+/*
+ * 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.spark.sql
+
+import org.apache.commons.lang3.StringUtils
+
+import org.apache.carbondata.streaming.CarbonStreamException
+import org.apache.carbondata.streaming.CarbonStreamSparkStreaming
+import org.apache.carbondata.streaming.CarbonStreamSparkStreamingWriter
+
+/**
+ * Create CarbonStreamSparkStreamingWriter for stream table
+ * when integrate with Spark Streaming
+ */
+object CarbonStreamSparkStreamFactory {
--- End diff --

How about using "CarbonSparkStreamingFactory" as name?


---


[GitHub] carbondata issue #1781: [CARBONDATA-2012] Add support to load pre-aggregate ...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1781
  
Build Failed with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2012/



---


[GitHub] carbondata issue #1855: [CARBONDATA-1796]while submiting new job, pass crede...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1855
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3249/



---


[GitHub] carbondata pull request #1878: [CARBONDATA-2094] Filter DataMap Tables in Sh...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1878#discussion_r164652631
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/ShowTablesCommand.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.table
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, 
AttributeReference}
+import org.apache.spark.sql.execution.command.MetadataCommand
+import org.apache.spark.sql.types.{BooleanType, StringType}
+
+
+private[sql] case class ShowTablesCommand ( databaseName: Option[String],
--- End diff --

Change to `CarbonShowTablesCommand` like other commands


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread ravipesala
Github user ravipesala commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164652496
  
--- Diff: 
core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapClassName.java
 ---
@@ -0,0 +1,52 @@
+/*
+ * 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.core.metadata.schema.table;
+
+/**
+ * type for create datamap
+ * The syntax of datamap creation is as follows.
+ * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 
'DataMapClassName'
+ * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
+ *
+ * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
+ */
+
+public enum DataMapClassName {
--- End diff --

If you mention name as `DataMapClassName` name then you should have class 
names also as part of Enum. Like 
```
DataMapClassName(int value, String shortName,  String className)
```


---


[GitHub] carbondata issue #1781: [CARBONDATA-2012] Add support to load pre-aggregate ...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1781
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3246/



---


[GitHub] carbondata issue #1847: [WIP][CARBONDATA-2064] Add compaction listener

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1847
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3198/



---


[GitHub] carbondata pull request #1876: [CARBONDATA-2093] Use small file feature of g...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1876#discussion_r164652321
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
 ---
@@ -588,15 +576,13 @@ case class CarbonLoadDataCommand(
 }
 val len = rowDataTypes.length
 var rdd =
-  new NewHadoopRDD[NullWritable, StringArrayWritable](
-sparkSession.sparkContext,
-classOf[CSVInputFormat],
-classOf[NullWritable],
-classOf[StringArrayWritable],
-jobConf).map { case (key, value) =>
+  DataLoadingUtil.csvFileScanRDD(
+sparkSession,
+model = carbonLoadModel,
+hadoopConf).map { row =>
--- End diff --

move `.map` to next line


---


[GitHub] carbondata pull request #1860: [CARBONDATA-2080] [S3-Implementation] Propaga...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1860#discussion_r164651950
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
 ---
@@ -347,31 +347,10 @@ class NewDataFrameLoaderRDD[K, V](
 sc: SparkContext,
 result: DataLoadResult[K, V],
 carbonLoadModel: CarbonLoadModel,
-prev: DataLoadCoalescedRDD[Row],
-@transient hadoopConf: Configuration) extends CarbonRDD[(K, V)](prev) {
--- End diff --

This hadoopConf should be moved to CarbonRDD. 


---


[GitHub] carbondata pull request #1867: [CARBONDATA-2055][Streaming][WIP]Support inte...

2018-01-29 Thread chenliang613
Github user chenliang613 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1867#discussion_r164651907
  
--- Diff: 
examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStreamSparkStreamingExample.scala
 ---
@@ -0,0 +1,245 @@
+/*
+ * 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.examples
+
+import java.io.{File, PrintWriter}
+import java.net.ServerSocket
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.CarbonEnv
+import org.apache.spark.sql.CarbonStreamSparkStreamFactory
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.SaveMode
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.streaming.{Seconds, StreamingContext, Time}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.{CarbonStorePath, 
CarbonTablePath}
+import org.apache.carbondata.streaming.CarbonSparkStreamingListener
+import org.apache.carbondata.streaming.parser.CarbonStreamParser
+
+/**
+ * This example introduces how to use Spark Streaming to write data
+ * to CarbonData stream table.
+ */
+// scalastyle:off println
+
+case class StreamData(id: Int, name: String, city: String, salary: Float)
--- End diff --

Can you explain , what are the difference of 
"CarbonStreamSparkStreamingExample.scala" and 
"CarbonBatchSparkStreamingExample", whether can merge the two examples into one 
, or not ?


---


[GitHub] carbondata pull request #1860: [CARBONDATA-2080] [S3-Implementation] Propaga...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1860#discussion_r164651488
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
 ---
@@ -31,7 +37,7 @@ import org.apache.carbondata.core.util._
  */
 abstract class CarbonRDD[T: ClassTag](@transient sc: SparkContext,
 @transient private var deps: Seq[Dependency[_]]) extends RDD[T](sc, 
deps) {
-
+  @transient val hadoopConf: Configuration = sc.hadoopConfiguration
--- End diff --

This hadoopConf should be passed via constructor, so add a new parameter in 
CarbonRDD constructor: `@transient hadoopConf: Configuration`
And modify all subclass of CarbonRDD to pass this parameter.


---


[GitHub] carbondata pull request #1860: [CARBONDATA-2080] [S3-Implementation] Propaga...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1860#discussion_r164651172
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
 ---
@@ -31,7 +37,7 @@ import org.apache.carbondata.core.util._
  */
 abstract class CarbonRDD[T: ClassTag](@transient sc: SparkContext,
--- End diff --

Please change `sc:SparkContext` to `sparkSession:SparkSession`.


---


[GitHub] carbondata issue #1104: [CARBONDATA-1239] Add validation for set command par...

2018-01-29 Thread mohammadshahidkhan
Github user mohammadshahidkhan commented on the issue:

https://github.com/apache/carbondata/pull/1104
  
retest this please


---


[GitHub] carbondata issue #1834: [CARBONDATA-2056] Hadoop Configuration with access k...

2018-01-29 Thread mohammadshahidkhan
Github user mohammadshahidkhan commented on the issue:

https://github.com/apache/carbondata/pull/1834
  
retest SDV please


---


[GitHub] carbondata pull request #1860: [CARBONDATA-2080] [S3-Implementation] Propaga...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1860#discussion_r164650047
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
 ---
@@ -59,6 +74,33 @@ abstract class CarbonRDD[T: ClassTag](@transient sc: 
SparkContext,
   map(f => CarbonProperties.getInstance().addProperty(f._1, f._2))
 internalCompute(split, context)
   }
+
+  private def getConf = {
--- End diff --

Move this up to line 59, and provide comment for  `confBytes` and `getConf`


---


[GitHub] carbondata pull request #1860: [CARBONDATA-2080] [S3-Implementation] Propaga...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1860#discussion_r164649933
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
 ---
@@ -59,6 +74,33 @@ abstract class CarbonRDD[T: ClassTag](@transient sc: 
SparkContext,
   map(f => CarbonProperties.getInstance().addProperty(f._1, f._2))
 internalCompute(split, context)
   }
+
+  private def getConf = {
--- End diff --

provide return value type in function signature


---


[GitHub] carbondata pull request #1860: [CARBONDATA-2080] [S3-Implementation] Propaga...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1860#discussion_r164649657
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
 ---
@@ -31,7 +37,7 @@ import org.apache.carbondata.core.util._
  */
 abstract class CarbonRDD[T: ClassTag](@transient sc: SparkContext,
 @transient private var deps: Seq[Dependency[_]]) extends RDD[T](sc, 
deps) {
-
+  @transient val hadoopConf: Configuration = sc.hadoopConfiguration
--- End diff --

Please add comment for this variable


---


[GitHub] carbondata pull request #1855: [CARBONDATA-1796]while submiting new job, pas...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1855#discussion_r164649308
  
--- Diff: 
integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala 
---
@@ -52,12 +54,14 @@ object SparkUtil {
   // set folder or file
   hadoopConfiguration.set(FileInputFormat.INPUT_DIR, filePath)
   hadoopConfiguration.set(FileInputFormat.INPUT_DIR_RECURSIVE, "true")
+  val jobConf = new JobConf(hadoopConfiguration)
--- End diff --

Seems no one is using this function, please remove it


---


[GitHub] carbondata pull request #1873: [CARBONDATA-2090] Fix the error message of al...

2018-01-29 Thread chenliang613
Github user chenliang613 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1873#discussion_r164649281
  
--- Diff: 
integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
 ---
@@ -182,6 +182,10 @@ class TestStreamingTableOperation extends QueryTest 
with BeforeAndAfterAll {
 
   // normal table not support streaming ingest
   test("normal table not support streaming ingest") {
--- End diff --

Please add alter table description for this test case


---


[GitHub] carbondata issue #1855: [CARBONDATA-1796]while submiting new job, pass crede...

2018-01-29 Thread jackylk
Github user jackylk commented on the issue:

https://github.com/apache/carbondata/pull/1855
  
LGTM


---


[GitHub] carbondata issue #1856: [CARBONDATA-2073][CARBONDATA-1516][Tests] Add test c...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1856
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3197/



---


[GitHub] carbondata issue #1781: [CARBONDATA-2012] Add support to load pre-aggregate ...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1781
  
Build Failed with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2011/



---


[GitHub] carbondata issue #1781: [CARBONDATA-2012] Add support to load pre-aggregate ...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1781
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3245/



---


[GitHub] carbondata issue #1704: [CARBONDATA-1918] Incorrect data is displayed when S...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1704
  
Build Success with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2009/



---


[GitHub] carbondata issue #1704: [WIP][CARBONDATA-1918] Incorrect data is displayed w...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1704
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3243/



---


[GitHub] carbondata issue #1880: [CARBONDATA-1616] Add CarbonData Streaming Ingestion...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1880
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3196/



---


[GitHub] carbondata issue #1856: [CARBONDATA-2073][CARBONDATA-1516][Tests] Add test c...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1856
  
Build Success with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2008/



---


[GitHub] carbondata issue #1856: [CARBONDATA-2073][CARBONDATA-1516][Tests] Add test c...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1856
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3242/



---


[GitHub] carbondata issue #1880: [CARBONDATA-1616] Add CarbonData Streaming Ingestion...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1880
  
Build Success with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2007/



---


[GitHub] carbondata issue #1880: [CARBONDATA-1616] Add CarbonData Streaming Ingestion...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1880
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3241/



---


[GitHub] carbondata issue #1879: [WIP] Fix measure min/max value problem while readin...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1879
  
SDV Build Fail , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3195/



---


[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize syntax f...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1865
  
Build Success with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2006/



---


[GitHub] carbondata issue #1847: [WIP][CARBONDATA-2064] Add compaction listener

2018-01-29 Thread dhatchayani
Github user dhatchayani commented on the issue:

https://github.com/apache/carbondata/pull/1847
  
retest sdv please


---


[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize syntax f...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1865
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3240/



---


[GitHub] carbondata issue #1879: [WIP] Fix measure min/max value problem while readin...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1879
  
Build Success with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2005/



---


[GitHub] carbondata issue #1879: [WIP] Fix measure min/max value problem while readin...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1879
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3239/



---


[GitHub] carbondata issue #1713: [CARBONDATA-1899] Optimize CarbonData concurrency te...

2018-01-29 Thread xubo245
Github user xubo245 commented on the issue:

https://github.com/apache/carbondata/pull/1713
  
@ravipesala It's ok in my computer with Xmx=4g. You can increase the value 
of Xmx or decrease the value of  totalNum.

The  memory requirements is similar 
org.apache.carbondata.examples.CompareTest


---


[GitHub] carbondata pull request #1880: [CARBONDATA-1616] Add CarbonData Streaming In...

2018-01-29 Thread QiangCai
GitHub user QiangCai opened a pull request:

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

[CARBONDATA-1616] Add CarbonData Streaming Ingestion Guide

 - [x] Any interfaces changed?
 no
 - [x] Any backward compatibility impacted?
 no
 - [x] Document update required?
updated doc
 - [x] Testing done
Please provide details on 
- Whether new unit test cases have been added or why no new tests 
are required?
doc will be reviewed
- How it is tested? Please attach test report.
- Is it a performance related change? Please attach the performance 
test report.
- Any additional information to help reviewers in testing this 
change.
   
 - [x] For large changes, please consider breaking it into sub-tasks under 
an umbrella JIRA. 
 small changes


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

$ git pull https://github.com/QiangCai/carbondata streaming_doc

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

https://github.com/apache/carbondata/pull/1880.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 #1880


commit f44d7b3b04ec6a813153a7769bd7991e9b746950
Author: QiangCai 
Date:   2018-01-04T03:52:07Z

streaming doc




---


[GitHub] carbondata issue #1856: [CARBONDATA-2073][CARBONDATA-1516][Tests] Add test c...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1856
  
SDV Build Fail , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3194/



---


[GitHub] carbondata pull request #1877: [CARBONDATA-1616] Documentation for Streaming...

2018-01-29 Thread asfgit
Github user asfgit closed the pull request at:

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


---


[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize syntax f...

2018-01-29 Thread xubo245
Github user xubo245 commented on the issue:

https://github.com/apache/carbondata/pull/1865
  
retest this please


---


[GitHub] carbondata issue #1856: [CARBONDATA-2073][CARBONDATA-1516][Tests] Add test c...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1856
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3238/



---


[GitHub] carbondata issue #1856: [CARBONDATA-2073][CARBONDATA-1516][Tests] Add test c...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1856
  
Build Failed with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2004/



---


[GitHub] carbondata pull request #1879: [WIP] Fix measure min/max value problem while...

2018-01-29 Thread manishgupta88
GitHub user manishgupta88 opened a pull request:

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

[WIP] Fix measure min/max value problem while reading from old store

Modified code to fix measure min/max problem in old store. Problem in query 
comes when measure contains a negative value

 - [ ] Any interfaces changed?
 
 - [ ] Any backward compatibility impacted?
 
 - [ ] Document update required?

 - [ ] Testing done
Please provide details on 
- Whether new unit test cases have been added or why no new tests 
are required?
- How it is tested? Please attach test report.
- Is it a performance related change? Please attach the performance 
test report.
- Any additional information to help reviewers in testing this 
change.
   
 - [ ] For large changes, please consider breaking it into sub-tasks under 
an umbrella JIRA. 



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

$ git pull https://github.com/manishgupta88/carbondata measure_min_max_fix

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

https://github.com/apache/carbondata/pull/1879.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 #1879


commit 5c483fff32947c6495e3c231bb350fc816a04de4
Author: manishgupta88 
Date:   2018-01-30T03:26:13Z

Modified code to fix measure min/max problem in old store




---


[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize syntax f...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1865
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3193/



---


[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize syntax f...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1865
  
Build Failed with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2003/



---


[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize syntax f...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1865
  
Build Success with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3237/



---


[GitHub] carbondata issue #1877: [CARBONDATA-1616] Documentation for Streaming Ingest...

2018-01-29 Thread QiangCai
Github user QiangCai commented on the issue:

https://github.com/apache/carbondata/pull/1877
  
LGTM


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread xubo245
Github user xubo245 commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164629019
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
 ---
@@ -59,44 +59,37 @@ case class CarbonCreateDataMapCommand(
 val details = TimeSeriesUtil
   .getTimeSeriesGranularityDetails(dmproperties, dmClassName)
 val updatedDmProperties = dmproperties - 
TimeSeriesUtil.getGranularityKey(dmproperties)
-details.map { f =>
-  CreatePreAggregateTableCommand(dataMapName,
-tableIdentifier,
-dmClassName,
-updatedDmProperties,
-queryString.get,
-Some(f._1))
-}.toSeq
+CreatePreAggregateTableCommand(dataMapName,
+  tableIdentifier,
+  dmClassName,
+  updatedDmProperties,
+  queryString.get,
+  Some(details(0)._1))
   } else {
-Seq(CreatePreAggregateTableCommand(
+CreatePreAggregateTableCommand(
   dataMapName,
   tableIdentifier,
   dmClassName,
   dmproperties,
   queryString.get
-))
+)
   }
-  
createPreAggregateTableCommands.flatMap(_.processMetadata(sparkSession))
+  createPreAggregateTableCommands.processMetadata(sparkSession)
 } else {
   throw new UnsupportedDataMapException(dmClassName)
 }
-LOGGER.audit(s"DataMap $dataMapName successfully added to Table ${ 
tableIdentifier.table }")
+LOGGER.audit(s"DataMap $dataMapName successfully added to Table 
${tableIdentifier.table}")
 Seq.empty
   }
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
-if (dmClassName.equalsIgnoreCase(PREAGGREGATE.getName) ||
-  dmClassName.equalsIgnoreCase(TIMESERIES.getName)) {
-  createPreAggregateTableCommands.flatMap(_.processData(sparkSession))
-} else {
-  throw new UnsupportedDataMapException(dmClassName)
-}
+createPreAggregateTableCommands.processData(sparkSession)
--- End diff --

ok, done


---


[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

2018-01-29 Thread jackylk
Github user jackylk commented on a diff in the pull request:

https://github.com/apache/carbondata/pull/1865#discussion_r164617642
  
--- Diff: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
 ---
@@ -59,44 +59,37 @@ case class CarbonCreateDataMapCommand(
 val details = TimeSeriesUtil
   .getTimeSeriesGranularityDetails(dmproperties, dmClassName)
 val updatedDmProperties = dmproperties - 
TimeSeriesUtil.getGranularityKey(dmproperties)
-details.map { f =>
-  CreatePreAggregateTableCommand(dataMapName,
-tableIdentifier,
-dmClassName,
-updatedDmProperties,
-queryString.get,
-Some(f._1))
-}.toSeq
+CreatePreAggregateTableCommand(dataMapName,
+  tableIdentifier,
+  dmClassName,
+  updatedDmProperties,
+  queryString.get,
+  Some(details(0)._1))
   } else {
-Seq(CreatePreAggregateTableCommand(
+CreatePreAggregateTableCommand(
   dataMapName,
   tableIdentifier,
   dmClassName,
   dmproperties,
   queryString.get
-))
+)
   }
-  
createPreAggregateTableCommands.flatMap(_.processMetadata(sparkSession))
+  createPreAggregateTableCommands.processMetadata(sparkSession)
 } else {
   throw new UnsupportedDataMapException(dmClassName)
 }
-LOGGER.audit(s"DataMap $dataMapName successfully added to Table ${ 
tableIdentifier.table }")
+LOGGER.audit(s"DataMap $dataMapName successfully added to Table 
${tableIdentifier.table}")
 Seq.empty
   }
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
-if (dmClassName.equalsIgnoreCase(PREAGGREGATE.getName) ||
-  dmClassName.equalsIgnoreCase(TIMESERIES.getName)) {
-  createPreAggregateTableCommands.flatMap(_.processData(sparkSession))
-} else {
-  throw new UnsupportedDataMapException(dmClassName)
-}
+createPreAggregateTableCommands.processData(sparkSession)
--- End diff --

I think it is better not to remove the validation of the dmClassName, since 
we will refactor processData and processMeta and add test framework for it 
later. If we remove the validation, we may forget to do this check when 
refactoring


---


[GitHub] carbondata issue #1878: [CARBONDATA-2094] Filter DataMap Tables in Show Tabl...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1878
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3192/



---


[GitHub] carbondata issue #1867: [CARBONDATA-2055][Streaming][WIP]Support integrating...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1867
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3191/



---


[GitHub] carbondata issue #1089: [CARBONDATA-1224] Added page level reader instead of...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1089
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3190/



---


[GitHub] carbondata issue #1704: [WIP][CARBONDATA-1918] Incorrect data is displayed w...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1704
  
SDV Build Fail , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3189/



---


[GitHub] carbondata issue #1819: [CARBONDATA-1964] Fixed bug to set bad.records.actio...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1819
  
SDV Build Fail , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3188/



---


[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize syntax f...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1865
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3187/



---


[GitHub] carbondata issue #1786: [CARBONDATA-1988] Fixed bug to remove empty partitio...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1786
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3186/



---


[GitHub] carbondata issue #1878: [CARBONDATA-2094] Filter DataMap Tables in Show Tabl...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1878
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3236/



---


[GitHub] carbondata issue #1878: [CARBONDATA-2094] Filter DataMap Tables in Show Tabl...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1878
  
Build Failed with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2002/



---


[GitHub] carbondata issue #1874: [WIP] Refactor query scan process to improve readabi...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1874
  
SDV Build Fail , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3185/



---


[GitHub] carbondata pull request #1878: [CARBONDATA-2094] Filter DataMap Tables in Sh...

2018-01-29 Thread BJangir
GitHub user BJangir opened a pull request:

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

[CARBONDATA-2094] Filter DataMap Tables in Show Table Command

Currently Show Table command shows datamap tables (agg tablels) but show 
table command should not show aggregate tables.

Solution :- Handle "show table "command in carbon side and Filter the 
datamap table and return rest of  the tables.

Be sure to do all of the following checklist to help us incorporate 
your contribution quickly and easily:

 - [ ] Any interfaces changed? NO
 
 - [ ] Any backward compatibility impacted?
 NO
 - [ ] Document update required?
NO
 - [ ] Testing done
Please provide details on 
- Whether new unit test cases have been added or why no new tests 
are required?
UI Added
- How it is tested? Please attach test report.
UI added
- Is it a performance related change? Please attach the performance 
test report.
 NA
- Any additional information to help reviewers in testing this change.
  NA 
 - [ ] For large changes, please consider breaking it into sub-tasks under 
an umbrella JIRA. 
NA


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

$ git pull https://github.com/BJangir/incubator-carbondata block_agg_display

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

https://github.com/apache/carbondata/pull/1878.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 #1878


commit c5caa22a7cc6c164ac37cfe142c6516a4933b5bf
Author: BJangir 
Date:   2018-01-29T18:16:56Z

[CARBONDATA-2094] Filter DataMap Tables in Show Table Command




---


[jira] [Created] (CARBONDATA-2094) Filter DataMap Tables in "Show Table Command"

2018-01-29 Thread Babulal (JIRA)
Babulal created CARBONDATA-2094:
---

 Summary: Filter DataMap Tables in "Show Table Command"
 Key: CARBONDATA-2094
 URL: https://issues.apache.org/jira/browse/CARBONDATA-2094
 Project: CarbonData
  Issue Type: Bug
Reporter: Babulal


Currently Show Table command shows datamap tables (agg tablels) but show table 
command should not show aggregate tables. 





--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[GitHub] carbondata issue #1847: [WIP][CARBONDATA-2064] Add compaction listener

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1847
  
SDV Build Fail , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3184/



---


[GitHub] carbondata issue #1876: [CARBONDATA-2093] Use small file feature of global s...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1876
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3183/



---


[GitHub] carbondata issue #1856: [CARBONDATA-2073][CARBONDATA-1516][Tests] Add test c...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1856
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3182/



---


[GitHub] carbondata issue #1747: [Compatibility] Added changes for backward compatibi...

2018-01-29 Thread ravipesala
Github user ravipesala commented on the issue:

https://github.com/apache/carbondata/pull/1747
  
SDV Build Success , Please check CI 
http://144.76.159.231:8080/job/ApacheSDVTests/3181/



---


[GitHub] carbondata issue #1704: [WIP][CARBONDATA-1918] Incorrect data is displayed w...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1704
  
Build Failed  with Spark 2.1.0, Please check CI 
http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/3235/



---


[GitHub] carbondata issue #1704: [WIP][CARBONDATA-1918] Incorrect data is displayed w...

2018-01-29 Thread CarbonDataQA
Github user CarbonDataQA commented on the issue:

https://github.com/apache/carbondata/pull/1704
  
Build Failed with Spark 2.2.1, Please check CI 
http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/2001/



---


  1   2   >