You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by kumarvishal09 <gi...@git.apache.org> on 2017/11/24 16:55:04 UTC

[GitHub] carbondata pull request #1565: [CARBONDATA-1518]Support creating timeseries ...

GitHub user kumarvishal09 opened a pull request:

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

    [CARBONDATA-1518]Support creating timeseries while creating main table.

    Support creating timeseries while creating main table.
    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     
     - [ ] Any backward compatibility impacted?
     
     - [ ] Document update required?
    
     - [ ] Testing done
            Please provide details on 
            - Whether new unit test cases have been added or why no new tests are required?
            - How it is tested? Please attach test report.
            - Is it a performance related change? Please attach the performance test report.
            - Any additional information to help reviewers in testing this change.
           
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    


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

    $ git pull https://github.com/kumarvishal09/incubator-carbondata master_timeseries

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

    https://github.com/apache/carbondata/pull/1565.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 #1565
    
----
commit fc76b22e90a3836bae68a6c3061781faf3a047e8
Author: kumarvishal <ku...@gmail.com>
Date:   2017-11-15T16:27:24Z

    Create Timeseries table
    
    timeseries support

----


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

    https://github.com/apache/carbondata/pull/1565
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/455/



---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154992762
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java ---
    @@ -126,8 +126,16 @@
        */
       private String aggFunction = "";
     
    +  /**
    +   * list of parent column relations
    +   */
       private List<ParentColumnTableRelation> parentColumnTableRelations;
     
    +  /**
    +   * timeseries function applied on column
    +   */
    +  private String timeSeriesFunction = "";
    --- End diff --
    
    it is specific to column schema and it has only two property timeseriesfunction and aggregation type 


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

    https://github.com/apache/carbondata/pull/1565
  
    Build Failed with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/494/



---

[GitHub] carbondata issue #1565: [WIP][CARBONDATA-1518][Pre-Aggregate]Support creatin...

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

    https://github.com/apache/carbondata/pull/1565
  
    please rebase


---

[GitHub] carbondata pull request #1565: [WIP][CARBONDATA-1518][Pre-Aggregate]Support ...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154094933
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/preagg/TimeSeriesUDF.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.preagg;
    +
    +import java.sql.Timestamp;
    +import java.util.Calendar;
    +import java.util.GregorianCalendar;
    +import java.util.Locale;
    +
    +/**
    + * class for applying timeseries udf
    + */
    +public class TimeSeriesUDF {
    +
    +  // thread local for keeping calender instance
    +  private ThreadLocal<Calendar> calanderThreadLocal = new ThreadLocal<>();
    +
    +  /**
    +   * singleton instance
    +   */
    +  public static final TimeSeriesUDF INSTANCE = new TimeSeriesUDF();
    +
    +  private TimeSeriesUDF() {
    +    initialize();
    +  }
    +
    +  /**
    +   * Below method will be used to apply udf on data provided
    +   * Method will work based on below logic.
    +   * Data: 2016-7-23 01:01:30,10
    +   * Year Level UDF will return: 2016-1-1 00:00:00,0
    +   * Month Level UDF will return: 2016-7-1 00:00:00,0
    +   * Day Level UDF will return: 2016-7-23 00:00:00,0
    +   * Hour Level UDF will return: 2016-7-23 01:00:00,0
    +   * Minute Level UDF will return: 2016-7-23 01:01:00,0
    +   * Second Level UDF will return: 2016-7-23 01:01:30,0
    +   * If function does not match with any of the above functions
    +   * it will throw exception
    +   *
    +   * @param data     timestamp data
    +   * @param function time series function name
    +   * @return data after applying udf
    +   */
    +  public Timestamp applyUDF(Timestamp data, String function) {
    +    if (null == data) {
    +      return data;
    +    }
    +    initialize();
    +    Calendar calendar = calanderThreadLocal.get();
    +    calendar.clear();
    +    calendar.setTimeInMillis(data.getTime());
    +    switch (function.toLowerCase(Locale.getDefault())) {
    +      case "second":
    --- End diff --
    
    Can you use enums


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

    https://github.com/apache/carbondata/pull/1565
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/528/



---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r155293926
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java ---
    @@ -126,8 +126,16 @@
        */
       private String aggFunction = "";
     
    +  /**
    +   * list of parent column relations
    +   */
       private List<ParentColumnTableRelation> parentColumnTableRelations;
     
    +  /**
    +   * timeseries function applied on column
    +   */
    +  private String timeSeriesFunction = "";
    --- End diff --
    
    @kumarvishal09 I don't think it is required to add timeSeriesFunction here, just rename aggFunction to function and use the same for both aggFunction and timeseries function.


---

[GitHub] carbondata pull request #1565: [WIP][CARBONDATA-1518][Pre-Aggregate]Support ...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154095105
  
    --- Diff: format/src/main/thrift/schema.thrift ---
    @@ -122,6 +122,12 @@ struct ColumnSchema{
       *  will be usefull in case of pre-aggregate
       **/
     	17: optional list<ParentColumnTableRelation> parentColumnTableRelations;
    +
    +  /**
    +  * function applied on timestamp column for timeseries for ex: hourTimestamp, dayTimestamp,
    +   * monthTimestamp, yearTimestamp
    +  **/
    +	18: optional string timeSeriesFunction;
    --- End diff --
    
    Keep as tableProperty only


---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154990920
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java ---
    @@ -439,6 +447,14 @@ public void setAggFunction(String aggFunction) {
         this.aggFunction = aggFunction;
       }
     
    +  public String getTimeSeriesFunction() {
    +    return timeSeriesFunction;
    +  }
    +
    +  public void setTimeSeriesFunction(String timeSeriesFunction) {
    +    this.timeSeriesFunction = timeSeriesFunction;
    --- End diff --
    
    should check it is not null


---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r155172136
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala ---
    @@ -50,13 +51,30 @@ case class CarbonCreateDataMapCommand(
         val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
         if (dmClassName.equals("org.apache.carbondata.datamap.AggregateDataMapHandler") ||
             dmClassName.equalsIgnoreCase("preaggregate")) {
    -      CreatePreAggregateTableCommand(
    -        dataMapName,
    -        tableIdentifier,
    -        dmClassName,
    -        dmproperties,
    -        queryString.get
    -      ).processMetadata(sparkSession)
    +      val timeHierarchyString = dmproperties.get("timeseries.hierarchy")
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518]Support creating timeseries while c...

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

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



---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154990854
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java ---
    @@ -476,6 +492,7 @@ public void write(DataOutput out) throws IOException {
         out.writeBoolean(invisible);
         out.writeBoolean(isSortColumn);
         out.writeUTF(null != aggFunction ? aggFunction : "");
    +    out.writeUTF(null != timeSeriesFunction ? timeSeriesFunction : "");
    --- End diff --
    
    It will not be null since initialization value is ""


---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154988752
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/preagg/TimeSeriesUDF.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.preagg;
    +
    +import java.sql.Timestamp;
    +import java.util.ArrayList;
    +import java.util.Calendar;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +
    +/**
    + * class for applying timeseries udf
    + */
    +public class TimeSeriesUDF {
    +
    +  public final List<String> TIMESERIES_FUNCTION = new ArrayList<>();
    +
    +  // thread local for keeping calender instance
    +  private ThreadLocal<Calendar> calanderThreadLocal = new ThreadLocal<>();
    +
    +  /**
    +   * singleton instance
    +   */
    +  public static final TimeSeriesUDF INSTANCE = new TimeSeriesUDF();
    +
    +  private TimeSeriesUDF() {
    +    initialize();
    +  }
    +
    +  /**
    +   * Below method will be used to apply udf on data provided
    +   * Method will work based on below logic.
    +   * Data: 2016-7-23 01:01:30,10
    +   * Year Level UDF will return: 2016-1-1 00:00:00,0
    +   * Month Level UDF will return: 2016-7-1 00:00:00,0
    +   * Day Level UDF will return: 2016-7-23 00:00:00,0
    +   * Hour Level UDF will return: 2016-7-23 01:00:00,0
    +   * Minute Level UDF will return: 2016-7-23 01:01:00,0
    +   * Second Level UDF will return: 2016-7-23 01:01:30,0
    +   * If function does not match with any of the above functions
    +   * it will throw exception
    --- End diff --
    
    mention it is IllegalArgumentException explicitly


---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154990053
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeseriesUtil.scala ---
    @@ -0,0 +1,158 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.sql.execution.command.timeseries
    +
    +import java.sql.Timestamp
    +
    +import org.apache.spark.sql.execution.command.{DataMapField, Field}
    +
    +import org.apache.carbondata.core.metadata.datatype.DataTypes
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.core.preagg.TimeSeriesUDF
    +import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
    +
    +/**
    + * Utility class for time series to keep
    + */
    +object TimeSeriesUtil {
    --- End diff --
    
    Can you move it to java code into core module


---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154989550
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala ---
    @@ -50,13 +51,30 @@ case class CarbonCreateDataMapCommand(
         val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
         if (dmClassName.equals("org.apache.carbondata.datamap.AggregateDataMapHandler") ||
             dmClassName.equalsIgnoreCase("preaggregate")) {
    -      CreatePreAggregateTableCommand(
    -        dataMapName,
    -        tableIdentifier,
    -        dmClassName,
    -        dmproperties,
    -        queryString.get
    -      ).processMetadata(sparkSession)
    +      val timeHierarchyString = dmproperties.get("timeseries.hierarchy")
    --- End diff --
    
    please collect all datamap property key string for pre-agg table and put them into a new file, so that it is easier to find all the properties for pre-agg table


---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r164043115
  
    --- Diff: integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala ---
    @@ -0,0 +1,93 @@
    +package org.apache.carbondata.integration.spark.testsuite.timeseries
    +
    +import org.apache.spark.sql.test.util.QueryTest
    +import org.scalatest.BeforeAndAfterAll
    +
    +class TestTimeSeriesCreateTable extends QueryTest with BeforeAndAfterAll {
    +
    +  override def beforeAll: Unit = {
    +    sql("drop table if exists mainTable")
    +    sql("CREATE TABLE mainTable(dataTime timestamp, name string, city string, age int) STORED BY 'org.apache.carbondata.format'")
    +    sql("create datamap agg0 on table mainTable using 'preaggregate' DMPROPERTIES ('timeseries.eventTime'='dataTime', 'timeseries.hierarchy'='second=1,hour=1,day=1,month=1,year=1') as select dataTime, sum(age) from mainTable group by dataTime")
    +  }
    +
    +  test("test timeseries create table Zero") {
    +    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg0_second"), true, "maintable_agg0_second")
    +    sql("drop datamap agg0_second on table mainTable")
    +  }
    +
    +  test("test timeseries create table One") {
    +    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg0_hour"), true, "maintable_agg0_hour")
    +    sql("drop datamap agg0_hour on table mainTable")
    +  }
    +  test("test timeseries create table two") {
    +    checkExistence(sql("DESCRIBE FORMATTED maintable_agg0_day"), true, "maintable_agg0_day")
    +    sql("drop datamap agg0_day on table mainTable")
    +  }
    +  test("test timeseries create table three") {
    +    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg0_month"), true, "maintable_agg0_month")
    +    sql("drop datamap agg0_month on table mainTable")
    +  }
    +  test("test timeseries create table four") {
    +    checkExistence(sql("DESCRIBE FORMATTED mainTable_agg0_year"), true, "maintable_agg0_year")
    +    sql("drop datamap agg0_year on table mainTable")
    +  }
    +
    +  test("test timeseries create table five") {
    +    try {
    +      sql(
    +        "create datamap agg0 on table mainTable using 'preaggregate' DMPROPERTIES ('timeseries.eventTime'='dataTime', 'timeseries.hierarchy'='sec=1,hour=1,day=1,month=1,year=1') as select dataTime, sum(age) from mainTable group by dataTime")
    +      assert(false)
    +    } catch {
    +      case _:Exception =>
    +        assert(true)
    +    }
    +  }
    +
    +  test("test timeseries create table Six") {
    +    try {
    +      sql(
    +        "create datamap agg0 on table mainTable using 'preaggregate' DMPROPERTIES ('timeseries.eventTime'='dataTime', 'timeseries.hierarchy'='hour=2') as select dataTime, sum(age) from mainTable group by dataTime")
    +      assert(false)
    +    } catch {
    +      case _:Exception =>
    +        assert(true)
    +    }
    +  }
    +
    +  test("test timeseries create table seven") {
    +    try {
    +      sql(
    +        "create datamap agg0 on table mainTable using 'preaggregate' DMPROPERTIES ('timeseries.eventTime'='dataTime', 'timeseries.hierarchy'='hour=1,day=1,year=1,month=1') as select dataTime, sum(age) from mainTable group by dataTime")
    +      assert(false)
    +    } catch {
    +      case _:Exception =>
    +        assert(true)
    +    }
    +  }
    +
    +  test("test timeseries create table Eight") {
    +    try {
    +      sql(
    +        "create datamap agg0 on table mainTable using 'preaggregate' DMPROPERTIES ('timeseries.eventTime'='name', 'timeseries.hierarchy'='hour=1,day=1,year=1,month=1') as select name, sum(age) from mainTable group by name")
    +      assert(false)
    --- End diff --
    
    It must run success whatever the sql run success...


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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


---

[GitHub] carbondata pull request #1565: [WIP][CARBONDATA-1518][Pre-Aggregate]Support ...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154095462
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala ---
    @@ -465,13 +472,22 @@ class TableNewProcessor(cm: TableModel) {
             encoders.add(Encoding.DICTIONARY)
             encoders
           }
    +      val timeSeriesFunction = if (cm.parentTable.isDefined && cm.timSeriesColumn.isDefined &&
    +                                   null != cm.parentTable.get.getColumnByName(
    +                                     cm.parentTable.get.getFactTableName,
    +                                       cm.timSeriesColumn.get)) {
    --- End diff --
    
    Format it properly


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r163140644
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeseriesUtil.scala ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.sql.execution.command.timeseries
    +
    +import java.sql.Timestamp
    +
    +import org.apache.spark.sql.execution.command.{DataMapField, Field}
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.metadata.datatype.DataTypes
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.core.preagg.TimeSeriesUDF
    +import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
    +
    +/**
    + * Utility class for time series to keep
    + */
    +object TimeSeriesUtil {
    +
    +  /**
    +   * Below method will be used to validate whether column mentioned in time series
    +   * is timestamp column or not
    +   *
    +   * @param dmproperties
    +   * data map properties
    +   * @param parentTable
    +   * parent table
    +   * @return whether time stamp column
    +   */
    +  def validateTimeSeriesEventTime(dmproperties: Map[String, String],
    +      parentTable: CarbonTable) {
    +    val eventTime = dmproperties.get(CarbonCommonConstants.TIMESERIES_EVENTTIME)
    +    if (!eventTime.isDefined) {
    +      throw new MalformedCarbonCommandException("Eventtime not defined in time series")
    +    } else {
    +      val carbonColumn = parentTable.getColumnByName(parentTable.getTableName, eventTime.get)
    +      if (carbonColumn.getDataType != DataTypes.TIMESTAMP) {
    +        throw new MalformedCarbonCommandException(
    +          "Timeseries event time is only supported on Timestamp " +
    +          "column")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Below method will be used to validate the hierarchy of time series and its value
    +   * validation will be done whether hierarchy order is proper or not and hierarchy level
    +   * value
    +   *
    +   * @param timeSeriesHierarchyDetails
    +   * time series hierarchy string
    +   */
    +  def validateAndGetTimeSeriesHierarchyDetails(timeSeriesHierarchyDetails: String): Array[
    +    (String, String)] = {
    +    val updatedtimeSeriesHierarchyDetails = timeSeriesHierarchyDetails.toLowerCase
    +    val timeSeriesHierarchy = updatedtimeSeriesHierarchyDetails.split(",")
    +    val hierBuffer = timeSeriesHierarchy.map {
    +      case f =>
    +        val splits = f.split("=")
    +        // checking hierarchy name is valid or not
    +        if (!TimeSeriesUDF.INSTANCE.TIMESERIES_FUNCTION.contains(splits(0).toLowerCase)) {
    +          throw new MalformedCarbonCommandException(s"Not supported heirarchy type: ${ splits(0) }")
    +
    +        }
    +        // validating hierarchy level is valid or not
    +        if (!splits(1).equals("1")) {
    +          throw new MalformedCarbonCommandException(
    +            s"Unsupported Value for hierarchy:" +
    +            s"${ splits(0) }=${ splits(1) }")
    +        }
    +        (splits(0), splits(1))
    +    }
    +    // checking whether hierarchy is in proper order or not
    +    // get the index of first hierarchy
    +    val indexOfFirstHierarchy = TimeSeriesUDF.INSTANCE.TIMESERIES_FUNCTION
    +      .indexOf(hierBuffer(0)._1.toLowerCase)
    +    val index = 0
    --- End diff --
    
    What usage has the index?


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

    https://github.com/apache/carbondata/pull/1565
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/472/



---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

    https://github.com/apache/carbondata/pull/1565
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/522/



---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

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


---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154990490
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java ---
    @@ -126,8 +126,16 @@
        */
       private String aggFunction = "";
     
    +  /**
    +   * list of parent column relations
    +   */
       private List<ParentColumnTableRelation> parentColumnTableRelations;
     
    +  /**
    +   * timeseries function applied on column
    +   */
    +  private String timeSeriesFunction = "";
    --- End diff --
    
    I think we should create one class to hold all information needed for pre-agg table, instead of putting them here directly in ColumnSchema


---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r155172378
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeseriesUtil.scala ---
    @@ -0,0 +1,158 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.sql.execution.command.timeseries
    +
    +import java.sql.Timestamp
    +
    +import org.apache.spark.sql.execution.command.{DataMapField, Field}
    +
    +import org.apache.carbondata.core.metadata.datatype.DataTypes
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.core.preagg.TimeSeriesUDF
    +import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
    +
    +/**
    + * Utility class for time series to keep
    + */
    +object TimeSeriesUtil {
    --- End diff --
    
    Some method is specific which we cannot move to core because of dependency that is why timeseries util is in integration 


---

[GitHub] carbondata pull request #1565: [WIP][CARBONDATA-1518][Pre-Aggregate]Support ...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154097078
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala ---
    @@ -0,0 +1,138 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.sql.execution.command.timeseries
    +
    +import java.sql.Timestamp
    +
    +import org.apache.spark.sql.execution.command.{DataMapField, Field}
    +
    +import org.apache.carbondata.core.metadata.datatype.DataTypes
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.core.preagg.TimeSeriesUDF
    +import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
    +
    +/**
    + * Utility class for time series to keep
    + */
    +object TimeSeriesUtil {
    --- End diff --
    
    I think this can be moved to core module


---

[GitHub] carbondata pull request #1565: [WIP][CARBONDATA-1518][Pre-Aggregate]Support ...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154094455
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java ---
    @@ -197,6 +197,7 @@
           thriftColumnSchema.setColumnProperties(properties);
         }
         thriftColumnSchema.setAggregate_function(wrapperColumnSchema.getAggFunction());
    +    thriftColumnSchema.setTimeSeriesFunction(wrapperColumnSchema.getTimeSeriesFunction());
    --- End diff --
    
    I don't think we should add a new field to schema about this, let it be as tableProperty only.


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r155170079
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/preagg/TimeSeriesUDF.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.preagg;
    +
    +import java.sql.Timestamp;
    +import java.util.ArrayList;
    +import java.util.Calendar;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +
    +/**
    + * class for applying timeseries udf
    + */
    +public class TimeSeriesUDF {
    +
    +  public final List<String> TIMESERIES_FUNCTION = new ArrayList<>();
    +
    +  // thread local for keeping calender instance
    +  private ThreadLocal<Calendar> calanderThreadLocal = new ThreadLocal<>();
    +
    +  /**
    +   * singleton instance
    +   */
    +  public static final TimeSeriesUDF INSTANCE = new TimeSeriesUDF();
    +
    +  private TimeSeriesUDF() {
    +    initialize();
    +  }
    +
    +  /**
    +   * Below method will be used to apply udf on data provided
    +   * Method will work based on below logic.
    +   * Data: 2016-7-23 01:01:30,10
    +   * Year Level UDF will return: 2016-1-1 00:00:00,0
    +   * Month Level UDF will return: 2016-7-1 00:00:00,0
    +   * Day Level UDF will return: 2016-7-23 00:00:00,0
    +   * Hour Level UDF will return: 2016-7-23 01:00:00,0
    +   * Minute Level UDF will return: 2016-7-23 01:01:00,0
    +   * Second Level UDF will return: 2016-7-23 01:01:30,0
    +   * If function does not match with any of the above functions
    +   * it will throw exception
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518]Support creating timeseries while c...

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

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



---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

    https://github.com/apache/carbondata/pull/1565
  
    rest this please


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r163249066
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeseriesUtil.scala ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.sql.execution.command.timeseries
    +
    +import java.sql.Timestamp
    +
    +import org.apache.spark.sql.execution.command.{DataMapField, Field}
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.metadata.datatype.DataTypes
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.core.preagg.TimeSeriesUDF
    +import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
    +
    +/**
    + * Utility class for time series to keep
    + */
    +object TimeSeriesUtil {
    +
    +  /**
    +   * Below method will be used to validate whether column mentioned in time series
    +   * is timestamp column or not
    +   *
    +   * @param dmproperties
    +   * data map properties
    +   * @param parentTable
    +   * parent table
    +   * @return whether time stamp column
    +   */
    +  def validateTimeSeriesEventTime(dmproperties: Map[String, String],
    +      parentTable: CarbonTable) {
    +    val eventTime = dmproperties.get(CarbonCommonConstants.TIMESERIES_EVENTTIME)
    +    if (!eventTime.isDefined) {
    +      throw new MalformedCarbonCommandException("Eventtime not defined in time series")
    +    } else {
    +      val carbonColumn = parentTable.getColumnByName(parentTable.getTableName, eventTime.get)
    +      if (carbonColumn.getDataType != DataTypes.TIMESTAMP) {
    +        throw new MalformedCarbonCommandException(
    +          "Timeseries event time is only supported on Timestamp " +
    +          "column")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Below method will be used to validate the hierarchy of time series and its value
    +   * validation will be done whether hierarchy order is proper or not and hierarchy level
    +   * value
    +   *
    +   * @param timeSeriesHierarchyDetails
    +   * time series hierarchy string
    +   */
    +  def validateAndGetTimeSeriesHierarchyDetails(timeSeriesHierarchyDetails: String): Array[
    +    (String, String)] = {
    +    val updatedtimeSeriesHierarchyDetails = timeSeriesHierarchyDetails.toLowerCase
    +    val timeSeriesHierarchy = updatedtimeSeriesHierarchyDetails.split(",")
    +    val hierBuffer = timeSeriesHierarchy.map {
    +      case f =>
    +        val splits = f.split("=")
    +        // checking hierarchy name is valid or not
    +        if (!TimeSeriesUDF.INSTANCE.TIMESERIES_FUNCTION.contains(splits(0).toLowerCase)) {
    +          throw new MalformedCarbonCommandException(s"Not supported heirarchy type: ${ splits(0) }")
    +
    +        }
    +        // validating hierarchy level is valid or not
    +        if (!splits(1).equals("1")) {
    --- End diff --
    
    Why we should "splits(1).equals("1")“?
    Cann't we support hout=2 or other?


---

[GitHub] carbondata pull request #1565: [WIP][CARBONDATA-1518][Pre-Aggregate]Support ...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r154095701
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala ---
    @@ -491,13 +507,22 @@ class TableNewProcessor(cm: TableModel) {
               encoders.add(Encoding.DICTIONARY)
               encoders
             }
    +        val timeSeriesFunction = if (cm.parentTable.isDefined && cm.timSeriesColumn.isDefined &&
    +                                    null != cm.parentTable.get.getColumnByName(
    +                                      cm.parentTable.get.getFactTableName,
    +                                      cm.timSeriesColumn.get)) {
    --- End diff --
    
    Please format properly


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

    https://github.com/apache/carbondata/pull/1565
  
    Build Success with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/511/



---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r163140389
  
    --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeseriesUtil.scala ---
    @@ -0,0 +1,159 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.spark.sql.execution.command.timeseries
    +
    +import java.sql.Timestamp
    +
    +import org.apache.spark.sql.execution.command.{DataMapField, Field}
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.metadata.datatype.DataTypes
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable
    +import org.apache.carbondata.core.preagg.TimeSeriesUDF
    +import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
    +
    +/**
    + * Utility class for time series to keep
    + */
    +object TimeSeriesUtil {
    +
    +  /**
    +   * Below method will be used to validate whether column mentioned in time series
    +   * is timestamp column or not
    +   *
    +   * @param dmproperties
    +   * data map properties
    +   * @param parentTable
    +   * parent table
    +   * @return whether time stamp column
    +   */
    +  def validateTimeSeriesEventTime(dmproperties: Map[String, String],
    +      parentTable: CarbonTable) {
    +    val eventTime = dmproperties.get(CarbonCommonConstants.TIMESERIES_EVENTTIME)
    +    if (!eventTime.isDefined) {
    +      throw new MalformedCarbonCommandException("Eventtime not defined in time series")
    +    } else {
    +      val carbonColumn = parentTable.getColumnByName(parentTable.getTableName, eventTime.get)
    +      if (carbonColumn.getDataType != DataTypes.TIMESTAMP) {
    +        throw new MalformedCarbonCommandException(
    +          "Timeseries event time is only supported on Timestamp " +
    +          "column")
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Below method will be used to validate the hierarchy of time series and its value
    +   * validation will be done whether hierarchy order is proper or not and hierarchy level
    +   * value
    +   *
    +   * @param timeSeriesHierarchyDetails
    +   * time series hierarchy string
    +   */
    +  def validateAndGetTimeSeriesHierarchyDetails(timeSeriesHierarchyDetails: String): Array[
    +    (String, String)] = {
    +    val updatedtimeSeriesHierarchyDetails = timeSeriesHierarchyDetails.toLowerCase
    +    val timeSeriesHierarchy = updatedtimeSeriesHierarchyDetails.split(",")
    +    val hierBuffer = timeSeriesHierarchy.map {
    +      case f =>
    +        val splits = f.split("=")
    +        // checking hierarchy name is valid or not
    +        if (!TimeSeriesUDF.INSTANCE.TIMESERIES_FUNCTION.contains(splits(0).toLowerCase)) {
    +          throw new MalformedCarbonCommandException(s"Not supported heirarchy type: ${ splits(0) }")
    --- End diff --
    
    should heirarchy be hierarchy?


---

[GitHub] carbondata pull request #1565: [CARBONDATA-1518][Pre-Aggregate]Support creat...

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

    https://github.com/apache/carbondata/pull/1565#discussion_r155172117
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/preagg/TimeSeriesUDF.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.preagg;
    +
    +import java.sql.Timestamp;
    +import java.util.ArrayList;
    +import java.util.Calendar;
    +import java.util.GregorianCalendar;
    +import java.util.List;
    +
    +/**
    + * class for applying timeseries udf
    + */
    +public class TimeSeriesUDF {
    +
    +  public final List<String> TIMESERIES_FUNCTION = new ArrayList<>();
    +
    +  // thread local for keeping calender instance
    +  private ThreadLocal<Calendar> calanderThreadLocal = new ThreadLocal<>();
    +
    +  /**
    +   * singleton instance
    +   */
    +  public static final TimeSeriesUDF INSTANCE = new TimeSeriesUDF();
    +
    +  private TimeSeriesUDF() {
    +    initialize();
    +  }
    +
    +  /**
    +   * Below method will be used to apply udf on data provided
    +   * Method will work based on below logic.
    +   * Data: 2016-7-23 01:01:30,10
    +   * Year Level UDF will return: 2016-1-1 00:00:00,0
    +   * Month Level UDF will return: 2016-7-1 00:00:00,0
    +   * Day Level UDF will return: 2016-7-23 00:00:00,0
    +   * Hour Level UDF will return: 2016-7-23 01:00:00,0
    +   * Minute Level UDF will return: 2016-7-23 01:01:00,0
    +   * Second Level UDF will return: 2016-7-23 01:01:30,0
    +   * If function does not match with any of the above functions
    +   * it will throw exception
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #1565: [CARBONDATA-1518]Support creating timeseries while c...

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

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



---

[GitHub] carbondata issue #1565: [CARBONDATA-1518][Pre-Aggregate]Support creating tim...

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

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



---