You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@carbondata.apache.org by lion-x <gi...@git.apache.org> on 2016/10/09 00:58:55 UTC

[GitHub] incubator-carbondata pull request #219: [WIP] [CARBONDATA-37]Support differe...

GitHub user lion-x opened a pull request:

    https://github.com/apache/incubator-carbondata/pull/219

    [WIP] [CARBONDATA-37]Support different time format input style

    # Why raise this PR?
    support different time format input style. In some scenarios different time dimensions may use different time format, we should support these requirements.
    


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

    $ git pull https://github.com/lion-x/incubator-carbondata timeformat

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

    https://github.com/apache/incubator-carbondata/pull/219.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 #219
    
----
commit 15d1e640f96b15411fb39eaf7208b3b656ebda0a
Author: X-Lion <xl...@gmail.com>
Date:   2016-09-29T14:33:18Z

    Lionx0929

----


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85460088
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -1244,6 +1260,25 @@ case class LoadTableUsingKettle(
         Seq.empty
       }
     
    +  private def validateDateFormat(dateFormat: String, dateDimensionsName: ArrayBuffer[String]):
    +  Unit = {
    +    if (dateFormat == "") {
    +      throw new MalformedCarbonCommandException("Error: Option DateFormat is set an empty string.")
    +    } else {
    +      var dateFormats: Array[String] = dateFormat.split(",")
    +      for (singleDateFormat <- dateFormats) {
    +        var dateFormatSplits: Array[String] = singleDateFormat.split(":", 2)
    +        if (!dateDimensionsName.contains(dateFormatSplits(0))) {
    --- End diff --
    
    take care case-insensitive


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85040709
  
    --- Diff: processing/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGeneratorTest.java ---
    @@ -37,7 +37,7 @@
       private int surrogateKey = -1;
     
       @Before public void setUp() throws Exception {
    -    TimeStampDirectDictionaryGenerator generator = TimeStampDirectDictionaryGenerator.instance;
    +    TimeStampDirectDictionaryGenerator generator = new TimeStampDirectDictionaryGenerator(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
    --- End diff --
    
    Should use carbon property to  create generator, not default value.
    please correct all.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85040387
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java ---
    @@ -111,7 +110,7 @@
       /**
        * timeFormat
        */
    -  protected SimpleDateFormat timeFormat;
    +  protected String dateFormat;
    --- End diff --
    
    better to complete modification for all method and keep the order.
    e.g.  saveRep, readRep...


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r82719262
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -117,15 +117,24 @@ private TimeStampDirectDictionaryGenerator() {
        * @return dictionary value
        */
       @Override public int generateDirectSurrogateKey(String memberStr) {
    -    SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    -            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
    +    String timeString;
    +    String formatString;
    +    if (memberStr.contains(CarbonCommonConstants.COLON_SPC_CHARACTER)){
    --- End diff --
    
    What is the reason the data contain COLON_SPC_CHARACTER?


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85040702
  
    --- Diff: processing/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGeneratorTest.java ---
    @@ -37,7 +37,7 @@
       private int surrogateKey = -1;
     
       @Before public void setUp() throws Exception {
    -    TimeStampDirectDictionaryGenerator generator = TimeStampDirectDictionaryGenerator.instance;
    +    TimeStampDirectDictionaryGenerator generator = new TimeStampDirectDictionaryGenerator(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
    --- End diff --
    
    Should use carbon property to  create generator, not default value.
    please correct all.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85038170
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -1129,6 +1130,9 @@ case class LoadTable(
           carbonLoadModel.setEscapeChar(escapeChar)
           carbonLoadModel.setQuoteChar(quoteChar)
           carbonLoadModel.setCommentChar(commentchar)
    +      carbonLoadModel.setDateFormat(dateFormat)
    --- End diff --
    
    It is necessary to validate input "dateFormat" before dataloading


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85632892
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -470,6 +472,34 @@ public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws K
                   break;
               }
             }
    +        HashMap<String, String> dateformatsHashMap = new HashMap<String, String>();
    +        if (meta.dateFormat != null) {
    +          String[] dateformats = meta.dateFormat.split(CarbonCommonConstants.COMMA);
    +          for (String dateFormat:dateformats) {
    +            String[] dateFormatSplits = dateFormat.split(":", 2);
    +            dateformatsHashMap.put(dateFormatSplits[0].toLowerCase().trim(),
    +                dateFormatSplits[1].trim());
    +          }
    +        }
    +        String[] DimensionColumnIds = meta.getDimensionColumnIds();
    +        directDictionaryGenerators =
    +            new DirectDictionaryGenerator[DimensionColumnIds.length];
    +        for (int i = 0; i < DimensionColumnIds.length; i++) {
    +          ColumnSchemaDetails columnSchemaDetails = columnSchemaDetailsWrapper.get(
    +              DimensionColumnIds[i]);
    +          if (columnSchemaDetails.isDirectDictionary()) {
    +            String columnName = columnSchemaDetails.getColumnName();
    +            DataType columnType = columnSchemaDetails.getColumnType();
    +            if (dateformatsHashMap.containsKey(columnName)) {
    --- End diff --
    
    better to use "get" method, just look up map once.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85632919
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -1244,6 +1246,29 @@ case class LoadTableUsingKettle(
         Seq.empty
       }
     
    +  private  def validateDateFormat(dateFormat: String, table: CarbonTable): Unit = {
    +    val dimensions = table.getDimensionByTableName(tableName).asScala
    +    if (dateFormat != null) {
    +      if (dateFormat.trim == "") {
    +        throw new MalformedCarbonCommandException("Error: Option DateFormat is set an empty " +
    --- End diff --
    
    better to remove "Error: " for all exception message


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85459810
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -1143,6 +1141,21 @@ case class LoadTableUsingKettle(
           val allDictionaryPath = options.getOrElse("all_dictionary_path", "")
           val complex_delimiter_level_1 = options.getOrElse("complex_delimiter_level_1", "\\$")
           val complex_delimiter_level_2 = options.getOrElse("complex_delimiter_level_2", "\\:")
    +      val timeFormat = options.getOrElse("timeformat", null)
    +      val dateFormat = options.getOrElse("dateformat", null)
    +      val tableDimensions: util.List[CarbonDimension] = table.getDimensionByTableName(tableName)
    +      val dateDimensionsName = new ArrayBuffer[String]
    +      tableDimensions.toArray.foreach {
    +        dimension => {
    +          val columnSchema: ColumnSchema = dimension.asInstanceOf[CarbonDimension].getColumnSchema
    +          if (columnSchema.getDataType.name == "TIMESTAMP") {
    +            dateDimensionsName += columnSchema.getColumnName
    +          }
    +        }
    +      }
    +      if (dateFormat != null) {
    +        validateDateFormat(dateFormat, dateDimensionsName)
    +      }
    --- End diff --
    
    please move these code into method validateDateFormat


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85039488
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -470,6 +474,36 @@ public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws K
                   break;
               }
             }
    +        HashMap<String, String> dateformatsHashMap = new HashMap<String, String>();
    +        if (meta.dateFormat != null) {
    +          String[] dateformats = meta.dateFormat.split(",");
    +          for (String dateFormat:dateformats) {
    +            String[] dateFormatSplits = dateFormat.split(":", 2);
    +            dateformatsHashMap.put(dateFormatSplits[0],dateFormatSplits[1]);
    +            // TODO  verify the dateFormatSplits is valid or not
    +          }
    +        }
    +        directDictionaryGenerators =
    +                new DirectDictionaryGenerator[meta.getDimensionColumnIds().length];
    +        for (int i = 0; i < meta.getDimensionColumnIds().length; i++) {
    +          ColumnSchemaDetails columnSchemaDetails = columnSchemaDetailsWrapper.get(
    +                  meta.getDimensionColumnIds()[i]);
    +          if (columnSchemaDetails.isDirectDictionary()) {
    +            if (dateformatsHashMap.containsKey(columnSchemaDetails.getColumnName())) {
    +              directDictionaryGenerators[i] =
    +                      DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(
    +                              columnSchemaDetails.getColumnType(),
    +                              dateformatsHashMap.get(columnSchemaDetails.getColumnName()));
    +            } else {
    +              String dateFormat = CarbonProperties.getInstance()
    +                      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    +                              CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
    +              directDictionaryGenerators[i] =
    +                      DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(
    +                              columnSchemaDetails.getColumnType(), dateFormat);
    --- End diff --
    
    1. move out CarbonProperties.getInstance().getProperty  from for loop
    2. for defaut  dataformat, use  method getDirectDictionaryGenerator(DataType dataType)


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85038702
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -117,9 +113,11 @@ private TimeStampDirectDictionaryGenerator() {
        * @return dictionary value
        */
       @Override public int generateDirectSurrogateKey(String memberStr) {
    -    SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    -            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
    +    SimpleDateFormat timeParser = threadLocal.get();
    +    if(timeParser == null){
    +      timeParser = new SimpleDateFormat(dateFormat);
    +      threadLocal.set(timeParser);
    +    }
         timeParser.setLenient(false);
    --- End diff --
    
    Please extract above codes to a new initial method,  and invoke this method in different thread.
    It it not good to run these codes in generateDirectSurrogateKey method.



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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85250472
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java ---
    @@ -651,6 +654,7 @@ public void setDefault() {
         columnSchemaDetails = "";
         columnsDataTypeString="";
         tableOption = "";
    +    dateFormat = CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT;
    --- End diff --
    
    ok


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85256460
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java ---
    @@ -111,7 +110,7 @@
       /**
        * timeFormat
        */
    -  protected SimpleDateFormat timeFormat;
    +  protected String dateFormat;
    --- End diff --
    
    ok



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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85248921
  
    --- Diff: processing/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGeneratorTest.java ---
    @@ -37,7 +37,7 @@
       private int surrogateKey = -1;
     
       @Before public void setUp() throws Exception {
    -    TimeStampDirectDictionaryGenerator generator = TimeStampDirectDictionaryGenerator.instance;
    +    TimeStampDirectDictionaryGenerator generator = new TimeStampDirectDictionaryGenerator(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
    --- End diff --
    
    This file is a test file, I think the TimeStampDirectDictionaryGenerator should be set 'CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT' for testing. pls check again.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85459633
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -1156,6 +1169,9 @@ case class LoadTableUsingKettle(
           carbonLoadModel.setEscapeChar(escapeChar)
           carbonLoadModel.setQuoteChar(quoteChar)
           carbonLoadModel.setCommentChar(commentchar)
    +      carbonLoadModel.setDateFormat(dateFormat)
    +      carbonLoadModel.setSerializationNullFormat("serialization_null_format" + "," +
    +        serializationNullFormat)
    --- End diff --
    
    this code is useless


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85465773
  
    --- Diff: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala ---
    @@ -0,0 +1,117 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +
    +package org.apache.carbondata.spark.testsuite.dataload
    +
    +import org.apache.spark.sql.common.util.CarbonHiveContext._
    +import org.apache.spark.sql.common.util.QueryTest
    +import org.scalatest.BeforeAndAfterAll
    +import java.sql.Timestamp
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
    +import org.apache.spark.sql.Row
    +
    +class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterAll {
    +  override def beforeAll {
    +    sql("DROP TABLE IF EXISTS t3")
    +    sql("""
    +           CREATE TABLE IF NOT EXISTS t3
    +           (ID Int, date Timestamp, starttime Timestamp, country String,
    +           name String, phonetype String, serialname String, salary Int)
    +           STORED BY 'carbondata'
    +        """)
    +    CarbonProperties.getInstance()
    +      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
    +  }
    +
    +  test("test load data with different timestamp format") {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = 'starttime:yyyy-MM-dd HH:mm:ss')
    +           """)
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData2.csv' into table t3
    +           OPTIONS('dateformat' = 'date:yyyy-MM-dd,starttime:yyyy/MM/dd HH:mm:ss')
    +           """)
    +      checkAnswer(
    +        sql("SELECT date FROM t3 WHERE ID = 1"),
    +        Seq(Row(Timestamp.valueOf("2015-07-23 00:00:00.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT starttime FROM t3 WHERE ID = 1"),
    +        Seq(Row(Timestamp.valueOf("2016-07-23 01:01:30.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT date FROM t3 WHERE ID = 18"),
    +        Seq(Row(Timestamp.valueOf("2015-07-25 00:00:00.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT starttime FROM t3 WHERE ID = 18"),
    +        Seq(Row(Timestamp.valueOf("2016-07-25 02:32:02.0")))
    +      )
    +  }
    +
    +  test("test load data with different timestamp format with being set an empty string") {
    +    try {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = '')
    +           """)
    +      assert(false)
    +    } catch {
    +      case ex: MalformedCarbonCommandException =>
    +        assertResult(ex.getMessage)("Error: Option DateFormat is set an empty string.")
    +      case _ => assert(false)
    +    }
    +  }
    +
    +  test("test load data with different timestamp format with a wrong column name") {
    +    try {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = 'fasfdas:yyyy/MM/dd')
    +           """)
    +      assert(false)
    +    } catch {
    +      case ex: MalformedCarbonCommandException =>
    +        assertResult(ex.getMessage)("Error: Wrong Column Name fasfdas is provided in Option DateFormat.")
    +      case _ => assert(false)
    +    }
    +  }
    +
    +  test("test load data with different timestamp format with a timestamp column is set an empty string") {
    +    try {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = 'fasfdas:')
    +           """)
    +      assert(false)
    +    } catch {
    +      case ex: MalformedCarbonCommandException =>
    +        assertResult(ex.getMessage)("Error: Wrong Column Name fasfdas is provided in Option DateFormat.")
    +      case _ => assert(false)
    +    }
    +  }
    +
    +  override def afterAll {
    --- End diff --
    
    ok, support


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85460571
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -1176,11 +1215,8 @@ else if(isComplexTypeColumn[j]) {
                 }
                 ColumnSchemaDetails details = columnSchemaDetailsWrapper.get(dimensionColumnIds[m]);
                 if (details.isDirectDictionary()) {
    -              DirectDictionaryGenerator directDictionaryGenerator1 =
    -                  DirectDictionaryKeyGeneratorFactory
    -                      .getDirectDictionaryGenerator(details.getColumnType());
                   surrogateKeyForHrrchy[0] =
    -                  directDictionaryGenerator1.generateDirectSurrogateKey(tuple);
    +              directDictionaryGenerators[m].generateDirectSurrogateKey(tuple);
    --- End diff --
    
    take care code style


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r82720605
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -1171,6 +1171,14 @@ else if(isComplexTypeColumn[j]) {
                   DirectDictionaryGenerator directDictionaryGenerator1 =
                       DirectDictionaryKeyGeneratorFactory
                           .getDirectDictionaryGenerator(details.getColumnType());
    +              String[] timeformats = meta.timeFormat.split(",");
    +              for(String timeformat:timeformats){
    +                if(timeformat.startsWith(details.getColumnName())){
    +                  timeformat = timeformat.replaceFirst(":",
    +                          CarbonCommonConstants.COLON_SPC_CHARACTER);
    +                  tuple = timeformat.replace(details.getColumnName(), tuple);
    +                }
    +              }
    --- End diff --
    
    better to not modify tuple value


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85466134
  
    --- Diff: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +
    +package org.apache.carbondata.spark.testsuite.dataload
    +
    +import org.apache.spark.sql.common.util.CarbonHiveContext._
    +import org.apache.spark.sql.common.util.QueryTest
    +import org.scalatest.BeforeAndAfterAll
    +import java.sql.Timestamp
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.spark.sql.Row
    +
    +class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterAll {
    +  override def beforeAll {
    +    sql("DROP TABLE IF EXISTS t3")
    +    sql("""
    +           CREATE TABLE IF NOT EXISTS t3
    +           (ID Int, date Timestamp, starttime Timestamp, country String,
    +           name String, phonetype String, serialname String, salary Int)
    +           STORED BY 'carbondata'
    +        """)
    +    CarbonProperties.getInstance()
    +      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
    +  }
    +
    +  test("test load data with different timestamp format") {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = 'starttime:yyyy-MM-dd HH:mm:ss')
    +           """)
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData2.csv' into table t3
    +           OPTIONS('dateformat' = 'date:yyyy-MM-dd,starttime:yyyy/MM/dd HH:mm:ss')
    +           """)
    +      checkAnswer(
    +        sql("SELECT date FROM t3 WHERE ID = 1"),
    +        Seq(Row(Timestamp.valueOf("2015-07-23 00:00:00.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT starttime FROM t3 WHERE ID = 1"),
    +        Seq(Row(Timestamp.valueOf("2016-07-23 01:01:30.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT date FROM t3 WHERE ID = 18"),
    +        Seq(Row(Timestamp.valueOf("2015-07-25 00:00:00.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT starttime FROM t3 WHERE ID = 18"),
    +        Seq(Row(Timestamp.valueOf("2016-07-25 02:32:02.0")))
    +      )
    +  }
    --- End diff --
    
    ok


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85039860
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -39,37 +39,32 @@
      */
     public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGenerator {
     
    -  private TimeStampDirectDictionaryGenerator() {
    +  private ThreadLocal<SimpleDateFormat> threadLocal = new ThreadLocal<>();
     
    -  }
    -
    -  public static TimeStampDirectDictionaryGenerator instance =
    -      new TimeStampDirectDictionaryGenerator();
    +  private String dateFormat;
     
       /**
        * The value of 1 unit of the SECOND, MINUTE, HOUR, or DAY in millis.
        */
    -  public static final long granularityFactor;
    +  public  long granularityFactor;
       /**
        * The date timestamp to be considered as start date for calculating the timestamp
        * java counts the number of milliseconds from  start of "January 1, 1970", this property is
        * customized the start of position. for example "January 1, 2000"
        */
    -  public static final long cutOffTimeStamp;
    +  public  long cutOffTimeStamp;
       /**
        * Logger instance
        */
    +
       private static final LogService LOGGER =
    -      LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
    +          LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
     
    -  /**
    -   * initialization block for granularityFactor and cutOffTimeStamp
    -   */
    -  static {
    +  public TimeStampDirectDictionaryGenerator(String dateFormat) {
    --- End diff --
    
    please keep default dateformat TimeStampDirectDictionaryGenerator() construct method, If DataLoading command didn't provide dateformat option for some column, we can use none-parameter construct method.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r84612211
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -117,15 +117,24 @@ private TimeStampDirectDictionaryGenerator() {
        * @return dictionary value
        */
       @Override public int generateDirectSurrogateKey(String memberStr) {
    -    SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    -            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
    +    String timeString;
    --- End diff --
    
    for 3, provide different date format key generate for different date format is complex, because the date format style is defined by user freely.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85039192
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -470,6 +474,36 @@ public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws K
                   break;
               }
             }
    +        HashMap<String, String> dateformatsHashMap = new HashMap<String, String>();
    +        if (meta.dateFormat != null) {
    +          String[] dateformats = meta.dateFormat.split(",");
    +          for (String dateFormat:dateformats) {
    +            String[] dateFormatSplits = dateFormat.split(":", 2);
    +            dateformatsHashMap.put(dateFormatSplits[0],dateFormatSplits[1]);
    +            // TODO  verify the dateFormatSplits is valid or not
    +          }
    +        }
    +        directDictionaryGenerators =
    +                new DirectDictionaryGenerator[meta.getDimensionColumnIds().length];
    +        for (int i = 0; i < meta.getDimensionColumnIds().length; i++) {
    --- End diff --
    
    not good to invoke getDimensionColumnIds many times


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85460589
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -343,7 +345,8 @@ public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws K
               }
     
               data.setGenerator(
    -              KeyGeneratorFactory.getKeyGenerator(getUpdatedLens(meta.dimLens, meta.dimPresent)));
    +              KeyGeneratorFactory.getKeyGenerator(
    +                  getUpdatedLens(meta.dimLens, meta.dimPresent)));
    --- End diff --
    
    keep code style


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85038977
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java ---
    @@ -651,6 +654,7 @@ public void setDefault() {
         columnSchemaDetails = "";
         columnsDataTypeString="";
         tableOption = "";
    +    dateFormat = CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT;
    --- End diff --
    
    Here should be empty string


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85249559
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -39,37 +39,32 @@
      */
     public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGenerator {
     
    -  private TimeStampDirectDictionaryGenerator() {
    +  private ThreadLocal<SimpleDateFormat> threadLocal = new ThreadLocal<>();
     
    -  }
    -
    -  public static TimeStampDirectDictionaryGenerator instance =
    -      new TimeStampDirectDictionaryGenerator();
    +  private String dateFormat;
     
       /**
        * The value of 1 unit of the SECOND, MINUTE, HOUR, or DAY in millis.
        */
    -  public static final long granularityFactor;
    +  public  long granularityFactor;
       /**
        * The date timestamp to be considered as start date for calculating the timestamp
        * java counts the number of milliseconds from  start of "January 1, 1970", this property is
        * customized the start of position. for example "January 1, 2000"
        */
    -  public static final long cutOffTimeStamp;
    +  public  long cutOffTimeStamp;
       /**
        * Logger instance
        */
    +
       private static final LogService LOGGER =
    -      LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
    +          LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
    --- End diff --
    
    done


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85460491
  
    --- Diff: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala ---
    @@ -0,0 +1,117 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +
    +package org.apache.carbondata.spark.testsuite.dataload
    +
    +import org.apache.spark.sql.common.util.CarbonHiveContext._
    +import org.apache.spark.sql.common.util.QueryTest
    +import org.scalatest.BeforeAndAfterAll
    +import java.sql.Timestamp
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
    +import org.apache.spark.sql.Row
    +
    +class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterAll {
    +  override def beforeAll {
    +    sql("DROP TABLE IF EXISTS t3")
    +    sql("""
    +           CREATE TABLE IF NOT EXISTS t3
    +           (ID Int, date Timestamp, starttime Timestamp, country String,
    +           name String, phonetype String, serialname String, salary Int)
    +           STORED BY 'carbondata'
    +        """)
    +    CarbonProperties.getInstance()
    +      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
    +  }
    +
    +  test("test load data with different timestamp format") {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = 'starttime:yyyy-MM-dd HH:mm:ss')
    +           """)
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData2.csv' into table t3
    +           OPTIONS('dateformat' = 'date:yyyy-MM-dd,starttime:yyyy/MM/dd HH:mm:ss')
    +           """)
    +      checkAnswer(
    +        sql("SELECT date FROM t3 WHERE ID = 1"),
    +        Seq(Row(Timestamp.valueOf("2015-07-23 00:00:00.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT starttime FROM t3 WHERE ID = 1"),
    +        Seq(Row(Timestamp.valueOf("2016-07-23 01:01:30.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT date FROM t3 WHERE ID = 18"),
    +        Seq(Row(Timestamp.valueOf("2015-07-25 00:00:00.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT starttime FROM t3 WHERE ID = 18"),
    +        Seq(Row(Timestamp.valueOf("2016-07-25 02:32:02.0")))
    +      )
    +  }
    +
    +  test("test load data with different timestamp format with being set an empty string") {
    +    try {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = '')
    +           """)
    +      assert(false)
    +    } catch {
    +      case ex: MalformedCarbonCommandException =>
    +        assertResult(ex.getMessage)("Error: Option DateFormat is set an empty string.")
    +      case _ => assert(false)
    +    }
    +  }
    +
    +  test("test load data with different timestamp format with a wrong column name") {
    +    try {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = 'fasfdas:yyyy/MM/dd')
    +           """)
    +      assert(false)
    +    } catch {
    +      case ex: MalformedCarbonCommandException =>
    +        assertResult(ex.getMessage)("Error: Wrong Column Name fasfdas is provided in Option DateFormat.")
    +      case _ => assert(false)
    +    }
    +  }
    +
    +  test("test load data with different timestamp format with a timestamp column is set an empty string") {
    +    try {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = 'fasfdas:')
    +           """)
    +      assert(false)
    +    } catch {
    +      case ex: MalformedCarbonCommandException =>
    +        assertResult(ex.getMessage)("Error: Wrong Column Name fasfdas is provided in Option DateFormat.")
    +      case _ => assert(false)
    +    }
    +  }
    +
    +  override def afterAll {
    --- End diff --
    
    dateformat='startTime'
    dateformat='startTime : '
    dateformat=' startTime : yyyy-MM-dd HH:mm:ss '
    dateformat='startTime:ydm1'
    dateformat=':yyyy/MM/dd'


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85036431
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -39,37 +39,32 @@
      */
     public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGenerator {
     
    -  private TimeStampDirectDictionaryGenerator() {
    +  private ThreadLocal<SimpleDateFormat> threadLocal = new ThreadLocal<>();
     
    -  }
    -
    -  public static TimeStampDirectDictionaryGenerator instance =
    -      new TimeStampDirectDictionaryGenerator();
    +  private String dateFormat;
     
       /**
        * The value of 1 unit of the SECOND, MINUTE, HOUR, or DAY in millis.
        */
    -  public static final long granularityFactor;
    +  public  long granularityFactor;
       /**
        * The date timestamp to be considered as start date for calculating the timestamp
        * java counts the number of milliseconds from  start of "January 1, 1970", this property is
        * customized the start of position. for example "January 1, 2000"
        */
    -  public static final long cutOffTimeStamp;
    +  public  long cutOffTimeStamp;
       /**
        * Logger instance
        */
    +
       private static final LogService LOGGER =
    -      LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
    +          LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
    --- End diff --
    
    please correct code style


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r82719466
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -117,15 +117,24 @@ private TimeStampDirectDictionaryGenerator() {
        * @return dictionary value
        */
       @Override public int generateDirectSurrogateKey(String memberStr) {
    -    SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    -            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
    +    String timeString;
    --- End diff --
    
    please use word "date" instead of "time"


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85466025
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -1129,6 +1130,9 @@ case class LoadTable(
           carbonLoadModel.setEscapeChar(escapeChar)
           carbonLoadModel.setQuoteChar(quoteChar)
           carbonLoadModel.setCommentChar(commentchar)
    +      carbonLoadModel.setDateFormat(dateFormat)
    --- End diff --
    
    ok


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85255469
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -470,6 +474,36 @@ public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws K
                   break;
               }
             }
    +        HashMap<String, String> dateformatsHashMap = new HashMap<String, String>();
    +        if (meta.dateFormat != null) {
    +          String[] dateformats = meta.dateFormat.split(",");
    +          for (String dateFormat:dateformats) {
    +            String[] dateFormatSplits = dateFormat.split(":", 2);
    +            dateformatsHashMap.put(dateFormatSplits[0],dateFormatSplits[1]);
    +            // TODO  verify the dateFormatSplits is valid or not
    +          }
    +        }
    +        directDictionaryGenerators =
    +                new DirectDictionaryGenerator[meta.getDimensionColumnIds().length];
    +        for (int i = 0; i < meta.getDimensionColumnIds().length; i++) {
    +          ColumnSchemaDetails columnSchemaDetails = columnSchemaDetailsWrapper.get(
    +                  meta.getDimensionColumnIds()[i]);
    +          if (columnSchemaDetails.isDirectDictionary()) {
    +            if (dateformatsHashMap.containsKey(columnSchemaDetails.getColumnName())) {
    +              directDictionaryGenerators[i] =
    +                      DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(
    +                              columnSchemaDetails.getColumnType(),
    +                              dateformatsHashMap.get(columnSchemaDetails.getColumnName()));
    +            } else {
    +              String dateFormat = CarbonProperties.getInstance()
    +                      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    +                              CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
    +              directDictionaryGenerators[i] =
    +                      DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(
    +                              columnSchemaDetails.getColumnType(), dateFormat);
    --- End diff --
    
    OK


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r82720651
  
    --- Diff: hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java ---
    @@ -356,6 +356,7 @@ public static void executeGraph(LoadModel loadModel, String storeLocation, Strin
         schmaModel.setEscapeCharacter("\\");
         schmaModel.setQuoteCharacter("\"");
         schmaModel.setCommentCharacter("#");
    +    schmaModel.setTimeFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
    --- End diff --
    
    No need to modify this file


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r82720156
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -1171,6 +1171,14 @@ else if(isComplexTypeColumn[j]) {
                   DirectDictionaryGenerator directDictionaryGenerator1 =
                       DirectDictionaryKeyGeneratorFactory
                           .getDirectDictionaryGenerator(details.getColumnType());
    +              String[] timeformats = meta.timeFormat.split(",");
    +              for(String timeformat:timeformats){
    --- End diff --
    
    Style, need space:  'for(' => 'for ('  , the same to '){' => ') {', and some other places.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85036554
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -39,37 +39,32 @@
      */
     public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGenerator {
     
    -  private TimeStampDirectDictionaryGenerator() {
    +  private ThreadLocal<SimpleDateFormat> threadLocal = new ThreadLocal<>();
     
    -  }
    -
    -  public static TimeStampDirectDictionaryGenerator instance =
    -      new TimeStampDirectDictionaryGenerator();
    +  private String dateFormat;
     
       /**
        * The value of 1 unit of the SECOND, MINUTE, HOUR, or DAY in millis.
        */
    -  public static final long granularityFactor;
    +  public  long granularityFactor;
       /**
        * The date timestamp to be considered as start date for calculating the timestamp
        * java counts the number of milliseconds from  start of "January 1, 1970", this property is
        * customized the start of position. for example "January 1, 2000"
        */
    -  public static final long cutOffTimeStamp;
    +  public  long cutOffTimeStamp;
       /**
        * Logger instance
        */
    +
       private static final LogService LOGGER =
    -      LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
    +          LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
    --- End diff --
    
    please correct all code style
    wrap line indentation length is 4 


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85460233
  
    --- Diff: processing/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGeneratorTest.java ---
    @@ -37,7 +37,7 @@
       private int surrogateKey = -1;
     
       @Before public void setUp() throws Exception {
    -    TimeStampDirectDictionaryGenerator generator = TimeStampDirectDictionaryGenerator.instance;
    +    TimeStampDirectDictionaryGenerator generator = new TimeStampDirectDictionaryGenerator(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
    --- End diff --
    
    ok


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85633017
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -470,6 +472,34 @@ public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws K
                   break;
               }
             }
    +        HashMap<String, String> dateformatsHashMap = new HashMap<String, String>();
    +        if (meta.dateFormat != null) {
    +          String[] dateformats = meta.dateFormat.split(CarbonCommonConstants.COMMA);
    +          for (String dateFormat:dateformats) {
    +            String[] dateFormatSplits = dateFormat.split(":", 2);
    +            dateformatsHashMap.put(dateFormatSplits[0].toLowerCase().trim(),
    +                dateFormatSplits[1].trim());
    +          }
    +        }
    +        String[] DimensionColumnIds = meta.getDimensionColumnIds();
    +        directDictionaryGenerators =
    +            new DirectDictionaryGenerator[DimensionColumnIds.length];
    +        for (int i = 0; i < DimensionColumnIds.length; i++) {
    +          ColumnSchemaDetails columnSchemaDetails = columnSchemaDetailsWrapper.get(
    +              DimensionColumnIds[i]);
    +          if (columnSchemaDetails.isDirectDictionary()) {
    +            String columnName = columnSchemaDetails.getColumnName();
    +            DataType columnType = columnSchemaDetails.getColumnType();
    +            if (dateformatsHashMap.containsKey(columnName)) {
    --- End diff --
    
    ok



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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85038276
  
    --- Diff: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +
    +package org.apache.carbondata.spark.testsuite.dataload
    +
    +import org.apache.spark.sql.common.util.CarbonHiveContext._
    +import org.apache.spark.sql.common.util.QueryTest
    +import org.scalatest.BeforeAndAfterAll
    +import java.sql.Timestamp
    +
    +import org.apache.carbondata.core.constants.CarbonCommonConstants
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.spark.sql.Row
    +
    +class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterAll {
    +  override def beforeAll {
    +    sql("DROP TABLE IF EXISTS t3")
    +    sql("""
    +           CREATE TABLE IF NOT EXISTS t3
    +           (ID Int, date Timestamp, starttime Timestamp, country String,
    +           name String, phonetype String, serialname String, salary Int)
    +           STORED BY 'carbondata'
    +        """)
    +    CarbonProperties.getInstance()
    +      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
    +  }
    +
    +  test("test load data with different timestamp format") {
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData1.csv' into table t3
    +           OPTIONS('dateformat' = 'starttime:yyyy-MM-dd HH:mm:ss')
    +           """)
    +      sql(s"""
    +           LOAD DATA LOCAL INPATH './src/test/resources/timeStampFormatData2.csv' into table t3
    +           OPTIONS('dateformat' = 'date:yyyy-MM-dd,starttime:yyyy/MM/dd HH:mm:ss')
    +           """)
    +      checkAnswer(
    +        sql("SELECT date FROM t3 WHERE ID = 1"),
    +        Seq(Row(Timestamp.valueOf("2015-07-23 00:00:00.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT starttime FROM t3 WHERE ID = 1"),
    +        Seq(Row(Timestamp.valueOf("2016-07-23 01:01:30.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT date FROM t3 WHERE ID = 18"),
    +        Seq(Row(Timestamp.valueOf("2015-07-25 00:00:00.0")))
    +      )
    +      checkAnswer(
    +        sql("SELECT starttime FROM t3 WHERE ID = 18"),
    +        Seq(Row(Timestamp.valueOf("2016-07-25 02:32:02.0")))
    +      )
    +  }
    --- End diff --
    
    please add more test cases to test wrong dateformat, not exists columns, empty string


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85036534
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java ---
    @@ -39,14 +40,26 @@ private DirectDictionaryKeyGeneratorFactory() {
        * @param dataType DataType
        * @return the generator instance
        */
    -  public static DirectDictionaryGenerator getDirectDictionaryGenerator(DataType dataType) {
    +  public static DirectDictionaryGenerator getDirectDictionaryGenerator(DataType dataType,
    +                                                                       String dateFormat) {
    --- End diff --
    
    please keep java code style


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85035902
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java ---
    @@ -39,14 +40,26 @@ private DirectDictionaryKeyGeneratorFactory() {
        * @param dataType DataType
        * @return the generator instance
        */
    -  public static DirectDictionaryGenerator getDirectDictionaryGenerator(DataType dataType) {
    +  public static DirectDictionaryGenerator getDirectDictionaryGenerator(DataType dataType,
    +                                                                       String dateFormat) {
         DirectDictionaryGenerator directDictionaryGenerator = null;
         switch (dataType) {
           case TIMESTAMP:
    -        directDictionaryGenerator = TimeStampDirectDictionaryGenerator.instance;
    +        directDictionaryGenerator = new TimeStampDirectDictionaryGenerator(dateFormat);
             break;
           default:
    +    }
    +    return directDictionaryGenerator;
    +  }
     
    +  public static DirectDictionaryGenerator getDirectDictionaryGenerator(DataType dataType) {
    +    DirectDictionaryGenerator directDictionaryGenerator = null;
    +    switch (dataType) {
    +      case TIMESTAMP:
    +        directDictionaryGenerator = new TimeStampDirectDictionaryGenerator(
    +                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
    --- End diff --
    
    here need to use CarbonProperty CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85461092
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -1244,6 +1260,25 @@ case class LoadTableUsingKettle(
         Seq.empty
       }
     
    +  private def validateDateFormat(dateFormat: String, dateDimensionsName: ArrayBuffer[String]):
    +  Unit = {
    +    if (dateFormat == "") {
    +      throw new MalformedCarbonCommandException("Error: Option DateFormat is set an empty string.")
    +    } else {
    +      var dateFormats: Array[String] = dateFormat.split(",")
    --- End diff --
    
    CarbonCommonConstant.COMMA


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85040305
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java ---
    @@ -111,7 +110,7 @@
       /**
        * timeFormat
    --- End diff --
    
    please correct comment


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r82925906
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -117,15 +117,24 @@ private TimeStampDirectDictionaryGenerator() {
        * @return dictionary value
        */
       @Override public int generateDirectSurrogateKey(String memberStr) {
    -    SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    -            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
    +    String timeString;
    +    String formatString;
    +    if (memberStr.contains(CarbonCommonConstants.COLON_SPC_CHARACTER)){
    --- End diff --
    
    It is not a good implement to append format to input data.
    No need to combine data with data format to a new member string.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r82720457
  
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java ---
    @@ -1171,6 +1171,14 @@ else if(isComplexTypeColumn[j]) {
                   DirectDictionaryGenerator directDictionaryGenerator1 =
                       DirectDictionaryKeyGeneratorFactory
                           .getDirectDictionaryGenerator(details.getColumnType());
    --- End diff --
    
    If the column type is TimeStamp, please provide dateformat to KeyGenerator.
    Better to provide different key generator for each date format.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85036811
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -92,23 +87,24 @@ private TimeStampDirectDictionaryGenerator() {
           cutOffTimeStampLocal = -1;
         } else {
           try {
    -        SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
    -            .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    -                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
    +        SimpleDateFormat timeParser = new SimpleDateFormat(
    +                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
    --- End diff --
    
    why just use default value? 


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r84612129
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -117,15 +117,24 @@ private TimeStampDirectDictionaryGenerator() {
        * @return dictionary value
        */
       @Override public int generateDirectSurrogateKey(String memberStr) {
    -    SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    -            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
    +    String timeString;
    --- End diff --
    
    ok, revised the code by your comment.


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85464806
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -1244,6 +1260,25 @@ case class LoadTableUsingKettle(
         Seq.empty
       }
     
    +  private def validateDateFormat(dateFormat: String, dateDimensionsName: ArrayBuffer[String]):
    +  Unit = {
    +    if (dateFormat == "") {
    +      throw new MalformedCarbonCommandException("Error: Option DateFormat is set an empty string.")
    +    } else {
    +      var dateFormats: Array[String] = dateFormat.split(",")
    +      for (singleDateFormat <- dateFormats) {
    +        var dateFormatSplits: Array[String] = singleDateFormat.split(":", 2)
    +        if (!dateDimensionsName.contains(dateFormatSplits(0))) {
    --- End diff --
    
    ok


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r82722044
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java ---
    @@ -117,15 +117,24 @@ private TimeStampDirectDictionaryGenerator() {
        * @return dictionary value
        */
       @Override public int generateDirectSurrogateKey(String memberStr) {
    -    SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()
    -        .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
    -            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
    +    String timeString;
    +    String formatString;
    +    if (memberStr.contains(CarbonCommonConstants.COLON_SPC_CHARACTER)){
    --- End diff --
    
    because in some format like XXXX-XX-XX 00:00:00.000, it has colon, it will make mistake when separating the memberstring.
    for example member string like, 2016-08-11 00:00:00.000:yyyy-MM-dd HH.mm.ss.SSS


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

[GitHub] incubator-carbondata pull request #219: [CARBONDATA-37]Support different tim...

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

    https://github.com/apache/incubator-carbondata/pull/219#discussion_r85459286
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala ---
    @@ -1143,6 +1141,21 @@ case class LoadTableUsingKettle(
           val allDictionaryPath = options.getOrElse("all_dictionary_path", "")
           val complex_delimiter_level_1 = options.getOrElse("complex_delimiter_level_1", "\\$")
           val complex_delimiter_level_2 = options.getOrElse("complex_delimiter_level_2", "\\:")
    +      val timeFormat = options.getOrElse("timeformat", null)
    --- End diff --
    
    "timeFormat" is useless


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