You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by xubo245 <gi...@git.apache.org> on 2018/01/26 09:58:26 UTC

[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] It better ...

GitHub user xubo245 opened a pull request:

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

     [CARBONDATA-2088][CARBONDATA-1516] It better using 'timeseries_cube'  instead of preaggregate for timeseries pre-aggregate table

    
    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     change using 'timeseries_cube'  instead of using  preaggregat
    change timeseries.eventTime  and  timeseries.hierarchy
     - [ ] Any backward compatibility impacted?
     NA
     - [ ] Document update required?
    NA
     - [ ] Testing done
        Add new test case
           
     - [ ] 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/xubo245/carbondata usingTimeseries

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

    https://github.com/apache/carbondata/pull/1865.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 #1865
    
----
commit 707be22b2a084e24e2d56d2bd85bd09b2d9ff5d5
Author: xubo245 <60...@...>
Date:   2018-01-26T09:47:46Z

     [CARBONDATA-2088][CARBONDATA-1516] It better using 'timeseries_cube' instead of preaggregate got timeseries pre-aggregate table

----


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164328911
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      return YEAR.getName
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      return MONTH.getName
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      return DAY.getName
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      return HOUR.getName
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      return MINUTE.getName
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      return SECOND.getName
    +    }
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var sum = 0
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +
    +    val granularity = if (sum > 1 || sum < 0) {
    +      throw new UnsupportedDataMapException(
    +        s"Granularity only support one")
    +    } else if (sum == 1) {
    +      true
    +    } else {
    +      false
    +    }
    +
    +    if (granularity && !dmClassName.equalsIgnoreCase(TIMESERIES.getName)) {
    +      throw new CarbonIllegalArgumentException(
    +        s"It should using ${TIMESERIES.getName}")
    +    } else if (!granularity && dmClassName.equalsIgnoreCase(TIMESERIES.getName)) {
    +      throw new CarbonIllegalArgumentException(
    +        s"${TIMESERIES.getName} should define time granularity")
    +    } else if (granularity) {
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  def getTimeSeriesGranularityDetails(
    --- End diff --
    
    ok, done


---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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 pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164411124
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +55,67 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    for (granularity <- Granularity.values()) {
    +      if (dmProperties.get(granularity.getName).isDefined) {
    +        return granularity.getName
    +      }
    +    }
    +
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var count = 0
    --- End diff --
    
    How to do it?


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164413731
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +55,67 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    for (granularity <- Granularity.values()) {
    +      if (dmProperties.get(granularity.getName).isDefined) {
    +        return granularity.getName
    +      }
    +    }
    +
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var count = 0
    --- End diff --
    
    ok, done


---

[GitHub] carbondata issue #1865: [WIP] [CARBONDATA-2088][CARBONDATA-1516] It better u...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

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



---

[GitHub] carbondata issue #1865: [WIP] [CARBONDATA-2088][CARBONDATA-1516] It better u...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1951/



---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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 #1865: [WIP] [CARBONDATA-2088][CARBONDATA-1516] It better u...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

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



---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on the issue:

    https://github.com/apache/carbondata/pull/1865
  
    @jackylk @kumarvishal09 @sraghunandan Please review it.


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on the issue:

    https://github.com/apache/carbondata/pull/1865
  
    @jackylk @ravipesala @sraghunandan @kumarvishal09 CI pass, please review it again. Other PR dependency it.


---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/2026/



---

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

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

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


---

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

Posted by jackylk <gi...@git.apache.org>.
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 issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1929/



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on the issue:

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


---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1917/



---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1965/



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

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



---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164416824
  
    --- Diff: integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/UnsupportedDataMapException.java ---
    @@ -0,0 +1,36 @@
    +/*
    + * 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;
    +
    +import static org.apache.carbondata.core.metadata.schema.table.DataMapClassName.PREAGGREGATE;
    +import static org.apache.carbondata.core.metadata.schema.table.DataMapClassName.TIMESERIES;
    +
    +/**
    + * Throw exception when using unsupported datamap type
    + */
    +public class UnsupportedDataMapException extends MalformedCarbonCommandException {
    +  /**
    +   * default serial version ID.
    +   */
    +  private static final long serialVersionUID = 1L;
    +
    +  public UnsupportedDataMapException(String dataMapType) {
    +    super("Unknown data map type " + dataMapType +
    +            ", Please use one of " + PREAGGREGATE + " or " + TIMESERIES);
    --- End diff --
    
    ok, done


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164400397
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala ---
    @@ -277,12 +281,21 @@ test("test PreAggregate table selection with timeseries and normal together") {
         sql(
           "create datamap agg0 on table maintabletime using 'preaggregate' as select dob,name from " +
           "maintabletime group by dob,name")
    -    sql(
    -      "create datamap agg1 on table maintabletime using 'preaggregate' DMPROPERTIES ('timeseries" +
    -      ".eventTime'='dob', 'timeseries.hierarchy'='hour=1,day=1,month=1,year=1') as select dob," +
    -      "name from maintabletime group by dob,name")
    +
    +  sql(
    +    s"""
    +       | create datamap agg1_year on table maintabletime
    +       | using '$timeSeries'
    +       | DMPROPERTIES (
    +       | 'event_time'='dob',
    +       | 'year_granularity'='1')
    --- End diff --
    
    ok, There are some test case for value is 2, I will add some test case when use other value.


---

[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] It better ...

Posted by xubo245 <gi...@git.apache.org>.
GitHub user xubo245 reopened a pull request:

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

     [CARBONDATA-2088][CARBONDATA-1516] It better using 'timeseries_cube'  instead of preaggregate for timeseries pre-aggregate table

    
    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     change using 'timeseries_cube'  instead of using  preaggregat
    change timeseries.eventTime  and  timeseries.hierarchy
     - [ ] Any backward compatibility impacted?
     NA
     - [ ] Document update required?
    NA
     - [ ] Testing done
        Add new test case
           
     - [ ] 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/xubo245/carbondata usingTimeseries

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

    https://github.com/apache/carbondata/pull/1865.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 #1865
    
----
commit 707be22b2a084e24e2d56d2bd85bd09b2d9ff5d5
Author: xubo245 <60...@...>
Date:   2018-01-26T09:47:46Z

     [CARBONDATA-2088][CARBONDATA-1516] It better using 'timeseries_cube' instead of preaggregate got timeseries pre-aggregate table

commit bf42301283d67905eb7e3845040cf91ab5ba42c3
Author: xubo245 <60...@...>
Date:   2018-01-26T12:45:04Z

    fix error

commit 60f259f40594b7cc5e80166b58b31b3e9a602f17
Author: xubo245 <60...@...>
Date:   2018-01-27T02:49:24Z

    fix findbugs

commit b1fb16d6022faab7c873708459b94b0224b4a9bd
Author: xubo245 <60...@...>
Date:   2018-01-27T03:43:53Z

    drop table to fix CI test error

----


---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by ravipesala <gi...@git.apache.org>.
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/3156/



---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164665299
  
    --- 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 --
    
    ok, done


---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

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



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

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



---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164326333
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      return YEAR.getName
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      return MONTH.getName
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      return DAY.getName
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      return HOUR.getName
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      return MINUTE.getName
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      return SECOND.getName
    +    }
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var sum = 0
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +
    +    val granularity = if (sum > 1 || sum < 0) {
    +      throw new UnsupportedDataMapException(
    +        s"Granularity only support one")
    --- End diff --
    
    ok, done


---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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 #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/3166/



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

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



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1928/



---

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

Posted by ravipesala <gi...@git.apache.org>.
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/3158/



---

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

Posted by jackylk <gi...@git.apache.org>.
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`


---

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

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on the issue:

    https://github.com/apache/carbondata/pull/1865
  
    @xubo245 Please update design document for This pr as you are changing the ddl for timeseries and add The new DDL in PR Description 


---

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

Posted by jackylk <gi...@git.apache.org>.
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...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164668854
  
    --- 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 --
    
    ok, done


---

[GitHub] carbondata issue #1865: [WIP] [CARBONDATA-2088][CARBONDATA-1516] It better u...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

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



---

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

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164379203
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +55,67 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    for (granularity <- Granularity.values()) {
    +      if (dmProperties.get(granularity.getName).isDefined) {
    +        return granularity.getName
    +      }
    +    }
    +
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var count = 0
    --- End diff --
    
    Please use boolean isFound instead for int 


---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/3198/



---

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

Posted by xubo245 <gi...@git.apache.org>.
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 pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164379370
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +55,67 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    for (granularity <- Granularity.values()) {
    +      if (dmProperties.get(granularity.getName).isDefined) {
    +        return granularity.getName
    +      }
    +    }
    +
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var count = 0
    +
    +    for (granularity <- Granularity.values()) {
    --- End diff --
    
    Break the loop when found the granularity level in DM properites


---

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

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

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


---

[GitHub] carbondata pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] It better ...

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 closed the pull request at:

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


---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1968/



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1945/



---

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

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164323882
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      return YEAR.getName
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      return MONTH.getName
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      return DAY.getName
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      return HOUR.getName
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      return MINUTE.getName
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      return SECOND.getName
    +    }
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var sum = 0
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +
    +    val granularity = if (sum > 1 || sum < 0) {
    +      throw new UnsupportedDataMapException(
    +        s"Granularity only support one")
    --- End diff --
    
    Only one granularity level can be defined


---

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

Posted by jackylk <gi...@git.apache.org>.
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 issue #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize syntax f...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/3199/



---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164326340
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      return YEAR.getName
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      return MONTH.getName
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      return DAY.getName
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      return HOUR.getName
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      return MINUTE.getName
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      return SECOND.getName
    +    }
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var sum = 0
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +
    +    val granularity = if (sum > 1 || sum < 0) {
    +      throw new UnsupportedDataMapException(
    +        s"Granularity only support one")
    +    } else if (sum == 1) {
    +      true
    +    } else {
    +      false
    +    }
    +
    +    if (granularity && !dmClassName.equalsIgnoreCase(TIMESERIES.getName)) {
    +      throw new CarbonIllegalArgumentException(
    +        s"It should using ${TIMESERIES.getName}")
    --- End diff --
    
    ok, done


---

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

Posted by jackylk <gi...@git.apache.org>.
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 issue #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize syntax f...

Posted by ravipesala <gi...@git.apache.org>.
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/3217/



---

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

Posted by jackylk <gi...@git.apache.org>.
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`


---

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

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164381889
  
    --- Diff: integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/UnsupportedDataMapException.java ---
    @@ -0,0 +1,36 @@
    +/*
    + * 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;
    +
    +import static org.apache.carbondata.core.metadata.schema.table.DataMapClassName.PREAGGREGATE;
    +import static org.apache.carbondata.core.metadata.schema.table.DataMapClassName.TIMESERIES;
    +
    +/**
    + * Throw exception when using unsupported datamap type
    + */
    +public class UnsupportedDataMapException extends MalformedCarbonCommandException {
    +  /**
    +   * default serial version ID.
    +   */
    +  private static final long serialVersionUID = 1L;
    +
    +  public UnsupportedDataMapException(String dataMapType) {
    +    super("Unknown data map type " + dataMapType +
    +            ", Please use one of " + PREAGGREGATE + " or " + TIMESERIES);
    --- End diff --
    
    Please update the message "Unknow data map type" + dataMapType 


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164326309
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    --- End diff --
    
    I need know what is the key, and then remove it for updatedDmProperties 
    
     val updatedDmProperties = dmproperties - TimeSeriesUtil.getGranularityKey(dmproperties)


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164326326
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      return YEAR.getName
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      return MONTH.getName
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      return DAY.getName
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      return HOUR.getName
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      return MINUTE.getName
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      return SECOND.getName
    +    }
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var sum = 0
    --- End diff --
    
    ok, done


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on the issue:

    https://github.com/apache/carbondata/pull/1865
  
    Thanks


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164665828
  
    --- 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 --
    
    ok,done


---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/3261/



---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1960/



---

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

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164323979
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      return YEAR.getName
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      return MONTH.getName
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      return DAY.getName
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      return HOUR.getName
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      return MINUTE.getName
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      return SECOND.getName
    +    }
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var sum = 0
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +
    +    val granularity = if (sum > 1 || sum < 0) {
    +      throw new UnsupportedDataMapException(
    +        s"Granularity only support one")
    +    } else if (sum == 1) {
    +      true
    +    } else {
    +      false
    +    }
    +
    +    if (granularity && !dmClassName.equalsIgnoreCase(TIMESERIES.getName)) {
    +      throw new CarbonIllegalArgumentException(
    +        s"It should using ${TIMESERIES.getName}")
    --- End diff --
    
    ${TIMESERIES.getName} keyword missing


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

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


---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1932/



---

[GitHub] carbondata issue #1865: [WIP] [CARBONDATA-2088][CARBONDATA-1516] It better u...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1948/



---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1998/



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by ravipesala <gi...@git.apache.org>.
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/3126/



---

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

Posted by ravipesala <gi...@git.apache.org>.
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 pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164374509
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala ---
    @@ -212,6 +215,60 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
         sql("drop datamap agg0 on table maintable")
       }
     
    +  val timeSeries = TIMESERIES.getName
    +
    +  test("test PreAggregate table selection: create with preaggregate and hierarchy") {
    +    sql("drop table if exists maintabletime")
    +    sql(
    +      """
    +        | create table maintabletime(year int,month int,name string,salary int,dob string)
    +        | stored by 'carbondata'
    +        | tblproperties(
    +        |   'sort_scope'='Global_sort',
    +        |   'table_blocksize'='23',
    +        |   'sort_columns'='month,year,name')
    +      """.stripMargin)
    +    sql("insert into maintabletime select 10,11,'x',12,'2014-01-01 00:00:00'")
    +    sql(
    +      s"""
    +         | create datamap agg0 on table maintabletime
    +         | using 'preaggregate'
    +         | as select dob,name from maintabletime
    +         | group by dob,name
    +       """.stripMargin)
    +    val e = intercept[MalformedCarbonCommandException] {
    +      sql(
    +        s"""
    +           | create datamap agg1 on table maintabletime
    +           | using 'preaggregate'
    +           | DMPROPERTIES (
    +           |  'event_time'='dob',
    +           |  'second_granularity'='1')
    +           | as select dob,name from maintabletime
    +           | group by dob,name
    +       """.stripMargin)
    +    }
    +    assert(e.getMessage.contains(s"$timeSeries keyword missing"))
    +    sql("drop table if exists maintabletime")
    +  }
    +
    +  test("test pre agg create table 21: using") {
    +    sql("drop datamap agg0 on table maintable")
    +
    +    val e: Exception = intercept[Exception] {
    +      sql(
    +        """
    +          | create datamap agg0 on table mainTable
    +          | using 'abc'
    +          | as select column3, sum(column3),column5, sum(column5)
    +          | from maintable
    +          | group by column3,column5,column2
    +        """.stripMargin)
    +    }
    +    assert(e.getMessage.contains(
    +      s"Unknown data map type abc, Please use one of $PREAGGREGATE or $TIMESERIES"))
    --- End diff --
    
    Please update the message to Unknown Data map type , 
    Please use PreAggregate or timeseries is not required as there can be n number of data map type


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on the issue:

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


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164409707
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala ---
    @@ -212,6 +215,60 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
         sql("drop datamap agg0 on table maintable")
       }
     
    +  val timeSeries = TIMESERIES.getName
    +
    +  test("test PreAggregate table selection: create with preaggregate and hierarchy") {
    +    sql("drop table if exists maintabletime")
    +    sql(
    +      """
    +        | create table maintabletime(year int,month int,name string,salary int,dob string)
    +        | stored by 'carbondata'
    +        | tblproperties(
    +        |   'sort_scope'='Global_sort',
    +        |   'table_blocksize'='23',
    +        |   'sort_columns'='month,year,name')
    +      """.stripMargin)
    +    sql("insert into maintabletime select 10,11,'x',12,'2014-01-01 00:00:00'")
    +    sql(
    +      s"""
    +         | create datamap agg0 on table maintabletime
    +         | using 'preaggregate'
    +         | as select dob,name from maintabletime
    +         | group by dob,name
    +       """.stripMargin)
    +    val e = intercept[MalformedCarbonCommandException] {
    +      sql(
    +        s"""
    +           | create datamap agg1 on table maintabletime
    +           | using 'preaggregate'
    +           | DMPROPERTIES (
    +           |  'event_time'='dob',
    +           |  'second_granularity'='1')
    +           | as select dob,name from maintabletime
    +           | group by dob,name
    +       """.stripMargin)
    +    }
    +    assert(e.getMessage.contains(s"$timeSeries keyword missing"))
    +    sql("drop table if exists maintabletime")
    +  }
    +
    +  test("test pre agg create table 21: using") {
    +    sql("drop datamap agg0 on table maintable")
    +
    +    val e: Exception = intercept[Exception] {
    +      sql(
    +        """
    +          | create datamap agg0 on table mainTable
    +          | using 'abc'
    +          | as select column3, sum(column3),column5, sum(column5)
    +          | from maintable
    +          | group by column3,column5,column2
    +        """.stripMargin)
    +    }
    +    assert(e.getMessage.contains(
    +      s"Unknown data map type abc, Please use one of $PREAGGREGATE or $TIMESERIES"))
    --- End diff --
    
    okļ¼Œdone


---

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

Posted by jackylk <gi...@git.apache.org>.
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 pull request #1865: [CARBONDATA-2088][CARBONDATA-1516] Optimize s...

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164323056
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    --- End diff --
    
    why to return the same value? if required only for validation, can change method name and return true/false


---

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

Posted by ravipesala <gi...@git.apache.org>.
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] It better using '...

Posted by ravipesala <gi...@git.apache.org>.
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/3132/



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

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



---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/3232/



---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164670599
  
    --- 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 --
    
    ok, move it to TimeSeriesUtil


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164667994
  
    --- 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 --
    
    ok, done


---

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

Posted by jackylk <gi...@git.apache.org>.
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


---

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

Posted by xubo245 <gi...@git.apache.org>.
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 issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1920/



---

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

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164381676
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala ---
    @@ -79,32 +79,27 @@ case class CarbonCreateDataMapCommand(
           }
           createPreAggregateTableCommands.flatMap(_.processMetadata(sparkSession))
         } else {
    -      val dataMapSchema = new DataMapSchema(dataMapName, dmClassName)
    -      dataMapSchema.setProperties(new java.util.HashMap[String, String](dmproperties.asJava))
    -      val dbName = CarbonEnv.getDatabaseName(tableIdentifier.database)(sparkSession)
    -      // upadting the parent table about dataschema
    -      PreAggregateUtil.updateMainTable(dbName, tableIdentifier.table, dataMapSchema, sparkSession)
    +      throw new UnsupportedDataMapException(dmClassName)
         }
         LOGGER.audit(s"DataMap $dataMapName successfully added to Table ${ tableIdentifier.table }")
         Seq.empty
       }
     
       override def processData(sparkSession: SparkSession): Seq[Row] = {
    -    if (dmClassName.equals("org.apache.carbondata.datamap.AggregateDataMapHandler") ||
    -        dmClassName.equalsIgnoreCase("preaggregate")) {
    +    if (dmClassName.equalsIgnoreCase(PREAGGREGATE.getName) ||
    +      dmClassName.equalsIgnoreCase(TIMESERIES.getName)) {
           createPreAggregateTableCommands.flatMap(_.processData(sparkSession))
         } else {
    -      Seq.empty
    +      throw new UnsupportedDataMapException(dmClassName)
         }
       }
     
       override def undoMetadata(sparkSession: SparkSession, exception: Exception): Seq[Row] = {
    -    if (dmClassName.equals("org.apache.carbondata.datamap.AggregateDataMapHandler") ||
    -        dmClassName.equalsIgnoreCase("preaggregate")) {
    -      val timeHierarchyString = dmproperties.get(CarbonCommonConstants.TIMESERIES_HIERARCHY)
    +    if (dmClassName.equalsIgnoreCase(PREAGGREGATE.getName) ||
    --- End diff --
    
    Same as above comment


---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1964/



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

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



---

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

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164323109
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      return YEAR.getName
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      return MONTH.getName
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      return DAY.getName
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      return HOUR.getName
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      return MINUTE.getName
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      return SECOND.getName
    +    }
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var sum = 0
    --- End diff --
    
    change to count


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164326318
  
    --- Diff: integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/UnsupportedDataMapException.java ---
    @@ -0,0 +1,36 @@
    +/*
    + * 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;
    +
    +import static org.apache.carbondata.core.metadata.schema.table.DataMapClassName.PREAGGREGATE;
    +import static org.apache.carbondata.core.metadata.schema.table.DataMapClassName.TIMESERIES;
    +
    +/**
    + * Throw exception when using unsupported datamap type
    + */
    +public class UnsupportedDataMapException extends MalformedCarbonCommandException {
    +  /**
    +   * default serial version ID.
    +   */
    +  private static final long serialVersionUID = 1L;
    +
    +  public UnsupportedDataMapException(String dataMapType) {
    +    super("Don't support using " + dataMapType +
    --- End diff --
    
    ok, done


---

[GitHub] carbondata issue #1865: [WIP] [CARBONDATA-2088][CARBONDATA-1516] It better u...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/3179/



---

[GitHub] carbondata issue #1865: [WIP] [CARBONDATA-2088][CARBONDATA-1516] It better u...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

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



---

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

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164322700
  
    --- Diff: integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/UnsupportedDataMapException.java ---
    @@ -0,0 +1,36 @@
    +/*
    + * 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;
    +
    +import static org.apache.carbondata.core.metadata.schema.table.DataMapClassName.PREAGGREGATE;
    +import static org.apache.carbondata.core.metadata.schema.table.DataMapClassName.TIMESERIES;
    +
    +/**
    + * Throw exception when using unsupported datamap type
    + */
    +public class UnsupportedDataMapException extends MalformedCarbonCommandException {
    +  /**
    +   * default serial version ID.
    +   */
    +  private static final long serialVersionUID = 1L;
    +
    +  public UnsupportedDataMapException(String dataMapType) {
    +    super("Don't support using " + dataMapType +
    --- End diff --
    
    change the sentence as "Unknown data map type " + dataMaType.Please use one of 


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164668021
  
    --- 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 --
    
    ok, done


---

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

Posted by jackylk <gi...@git.apache.org>.
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...

Posted by xubo245 <gi...@git.apache.org>.
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...

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164323826
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +56,107 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      return YEAR.getName
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      return MONTH.getName
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      return DAY.getName
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      return HOUR.getName
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      return MINUTE.getName
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      return SECOND.getName
    +    }
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var sum = 0
    +
    +    if (dmProperties.get(YEAR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MONTH.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(DAY.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(HOUR.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(MINUTE.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +    if (dmProperties.get(SECOND.getName).isDefined) {
    +      sum = sum + 1
    +    }
    +
    +    val granularity = if (sum > 1 || sum < 0) {
    +      throw new UnsupportedDataMapException(
    +        s"Granularity only support one")
    +    } else if (sum == 1) {
    +      true
    +    } else {
    +      false
    +    }
    +
    +    if (granularity && !dmClassName.equalsIgnoreCase(TIMESERIES.getName)) {
    +      throw new CarbonIllegalArgumentException(
    +        s"It should using ${TIMESERIES.getName}")
    +    } else if (!granularity && dmClassName.equalsIgnoreCase(TIMESERIES.getName)) {
    +      throw new CarbonIllegalArgumentException(
    +        s"${TIMESERIES.getName} should define time granularity")
    +    } else if (granularity) {
    +      true
    +    } else {
    +      false
    +    }
    +  }
    +
    +  def getTimeSeriesGranularityDetails(
    --- End diff --
    
    Better to iterate over enum values and check the default values.


---

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

Posted by ravipesala <gi...@git.apache.org>.
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/3163/



---

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

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164381622
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala ---
    @@ -79,32 +79,27 @@ case class CarbonCreateDataMapCommand(
           }
           createPreAggregateTableCommands.flatMap(_.processMetadata(sparkSession))
         } else {
    -      val dataMapSchema = new DataMapSchema(dataMapName, dmClassName)
    -      dataMapSchema.setProperties(new java.util.HashMap[String, String](dmproperties.asJava))
    -      val dbName = CarbonEnv.getDatabaseName(tableIdentifier.database)(sparkSession)
    -      // upadting the parent table about dataschema
    -      PreAggregateUtil.updateMainTable(dbName, tableIdentifier.table, dataMapSchema, sparkSession)
    +      throw new UnsupportedDataMapException(dmClassName)
         }
         LOGGER.audit(s"DataMap $dataMapName successfully added to Table ${ tableIdentifier.table }")
         Seq.empty
       }
     
       override def processData(sparkSession: SparkSession): Seq[Row] = {
    -    if (dmClassName.equals("org.apache.carbondata.datamap.AggregateDataMapHandler") ||
    --- End diff --
    
    I think process meta will handle exception part if class name mentioned in create data map statement is not valid. When call will come to processData or undoMeta validation is already passed so no need to add class validation again


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164413704
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -54,6 +55,67 @@ object TimeSeriesUtil {
         }
       }
     
    +  def getGranularityKey(dmProperties: Map[String, String]): String = {
    +
    +    for (granularity <- Granularity.values()) {
    +      if (dmProperties.get(granularity.getName).isDefined) {
    +        return granularity.getName
    +      }
    +    }
    +
    +    throw new CarbonIllegalArgumentException(
    +      s"${TIMESERIES.getName} should define time granularity")
    +  }
    +
    +  def validateTimeSeriesGranularity(
    +      dmProperties: Map[String, String],
    +      dmClassName: String): Boolean = {
    +    var count = 0
    +
    +    for (granularity <- Granularity.values()) {
    --- End diff --
    
    when we found the second granularity, we can break the loop, but  first can not.


---

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

Posted by ravipesala <gi...@git.apache.org>.
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/3171/



---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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/3202/



---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164400095
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala ---
    @@ -212,6 +215,60 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
         sql("drop datamap agg0 on table maintable")
       }
     
    +  val timeSeries = TIMESERIES.getName
    +
    +  test("test PreAggregate table selection: create with preaggregate and hierarchy") {
    +    sql("drop table if exists maintabletime")
    +    sql(
    +      """
    +        | create table maintabletime(year int,month int,name string,salary int,dob string)
    +        | stored by 'carbondata'
    +        | tblproperties(
    +        |   'sort_scope'='Global_sort',
    +        |   'table_blocksize'='23',
    +        |   'sort_columns'='month,year,name')
    +      """.stripMargin)
    +    sql("insert into maintabletime select 10,11,'x',12,'2014-01-01 00:00:00'")
    +    sql(
    +      s"""
    +         | create datamap agg0 on table maintabletime
    +         | using 'preaggregate'
    +         | as select dob,name from maintabletime
    +         | group by dob,name
    +       """.stripMargin)
    +    val e = intercept[MalformedCarbonCommandException] {
    +      sql(
    +        s"""
    +           | create datamap agg1 on table maintabletime
    +           | using 'preaggregate'
    +           | DMPROPERTIES (
    +           |  'event_time'='dob',
    +           |  'second_granularity'='1')
    +           | as select dob,name from maintabletime
    +           | group by dob,name
    +       """.stripMargin)
    +    }
    +    assert(e.getMessage.contains(s"$timeSeries keyword missing"))
    +    sql("drop table if exists maintabletime")
    +  }
    +
    +  test("test pre agg create table 21: using") {
    +    sql("drop datamap agg0 on table maintable")
    +
    +    val e: Exception = intercept[Exception] {
    +      sql(
    +        """
    +          | create datamap agg0 on table mainTable
    +          | using 'abc'
    +          | as select column3, sum(column3),column5, sum(column5)
    +          | from maintable
    +          | group by column3,column5,column2
    +        """.stripMargin)
    +    }
    +    assert(e.getMessage.contains(
    +      s"Unknown data map type abc, Please use one of $PREAGGREGATE or $TIMESERIES"))
    --- End diff --
    
    I think we should remind user how to use it. We can print out all data map type. How do you think?


---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164416505
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala ---
    @@ -79,32 +79,27 @@ case class CarbonCreateDataMapCommand(
           }
           createPreAggregateTableCommands.flatMap(_.processMetadata(sparkSession))
         } else {
    -      val dataMapSchema = new DataMapSchema(dataMapName, dmClassName)
    -      dataMapSchema.setProperties(new java.util.HashMap[String, String](dmproperties.asJava))
    -      val dbName = CarbonEnv.getDatabaseName(tableIdentifier.database)(sparkSession)
    -      // upadting the parent table about dataschema
    -      PreAggregateUtil.updateMainTable(dbName, tableIdentifier.table, dataMapSchema, sparkSession)
    +      throw new UnsupportedDataMapException(dmClassName)
         }
         LOGGER.audit(s"DataMap $dataMapName successfully added to Table ${ tableIdentifier.table }")
         Seq.empty
       }
     
       override def processData(sparkSession: SparkSession): Seq[Row] = {
    -    if (dmClassName.equals("org.apache.carbondata.datamap.AggregateDataMapHandler") ||
    --- End diff --
    
    We can discuss with jacky li, and raise another  PR for this if it necessary


---

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

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

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

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164374779
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala ---
    @@ -277,12 +281,21 @@ test("test PreAggregate table selection with timeseries and normal together") {
         sql(
           "create datamap agg0 on table maintabletime using 'preaggregate' as select dob,name from " +
           "maintabletime group by dob,name")
    -    sql(
    -      "create datamap agg1 on table maintabletime using 'preaggregate' DMPROPERTIES ('timeseries" +
    -      ".eventTime'='dob', 'timeseries.hierarchy'='hour=1,day=1,month=1,year=1') as select dob," +
    -      "name from maintabletime group by dob,name")
    +
    +  sql(
    +    s"""
    +       | create datamap agg1_year on table maintabletime
    +       | using '$timeSeries'
    +       | DMPROPERTIES (
    +       | 'event_time'='dob',
    +       | 'year_granularity'='1')
    --- End diff --
    
    Can u please add some test case for granularity value validation , for example when use has given value 1.5 or 2 


---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

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



---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
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 #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by CarbonDataQA <gi...@git.apache.org>.
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/1958/



---

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

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

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



---

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

Posted by xubo245 <gi...@git.apache.org>.
Github user xubo245 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/1865#discussion_r164665269
  
    --- 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 --
    
    ok, create datamap package in schema package  and move it 


---

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

Posted by ravipesala <gi...@git.apache.org>.
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/3162/



---

[GitHub] carbondata issue #1865: [CARBONDATA-2088][CARBONDATA-1516] It better using '...

Posted by ravipesala <gi...@git.apache.org>.
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/3123/



---

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

Posted by ravipesala <gi...@git.apache.org>.
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/



---