This post was updated on .
Hi Community,
I load records by date in batch with save mode append, but the same query get different result when I hit query. And I make sure I did not update data and the first time query result is right. My env is carbondata 1.5.0 & spark2.3. 1. How I create table and datamap def store_ddl(carbon: SparkSession, logger: SLOGGER, parallelism: String): Unit = { val createStoreTableSql = s""" | CREATE TABLE IF NOT EXISTS store( | market_code STRING, | device_code STRING, | country_code STRING, | category_id INTEGER, | product_id LONG, | date TIMESTAMP, | est_free_app_download LONG, | est_paid_app_download LONG, | est_revenue LONG | ) | STORED BY 'carbondata' | TBLPROPERTIES( | 'SORT_COLUMNS'='market_code, device_code, country_code, category_id, date, product_id', | 'NO_INVERTED_INDEX'='est_free_app_download, est_paid_app_download, est_revenue', | 'LOCAL_DICTIONARY_ENABLE'='true', | 'LOCAL_DICTIONARY_THRESHOLD'='1000', | 'LOCAL_DICTIONARY_INCLUDE'='market_code, device_code, country_code', | 'SORT_SCOPE'='LOCAL_SORT', | 'CACHE_LEVEL'='BLOCKLET', | 'TABLE_BLOCKSIZE'='256', | 'GLOBAL_SORT_PARTITIONS'='${parallelism}' | ) """.stripMargin carbon.sql(createStoreTableSql) val createTimeSeriesDayNoProductTableSql = s""" | CREATE DATAMAP IF NOT EXISTS agg_by_day ON TABLE store | USING 'timeSeries' | DMPROPERTIES ( | 'EVENT_TIME'='date', | 'DAY_GRANULARITY'='1') | AS SELECT date, market_code, device_code, country_code, category_id, | COUNT(product_id), COUNT(est_free_app_download), COUNT(est_free_app_download), COUNT(est_revenue), | SUM(est_free_app_download), MIN(est_free_app_download), MAX(est_free_app_download), | SUM(est_paid_app_download), MIN(est_paid_app_download), MAX(est_paid_app_download), | SUM(est_revenue), MIN(est_revenue), MAX(est_revenue) | FROM store | GROUP BY date, market_code, device_code, country_code, category_id """.stripMargin carbon.sql(createTimeSeriesDayNoProductTableSql) val createTimeSeriesMonthNoProductTableSql = s""" | CREATE DATAMAP IF NOT EXISTS agg_by_month ON TABLE store | USING 'timeSeries' | DMPROPERTIES ( | 'EVENT_TIME'='date', | 'MONTH_GRANULARITY'='1') | AS SELECT date, market_code, device_code, country_code, category_id, | COUNT(product_id), COUNT(est_free_app_download), COUNT(est_free_app_download), COUNT(est_revenue), | SUM(est_free_app_download), MIN(est_free_app_download), MAX(est_free_app_download), | SUM(est_paid_app_download), MIN(est_paid_app_download), MAX(est_paid_app_download), | SUM(est_revenue), MIN(est_revenue), MAX(est_revenue) | FROM store | GROUP BY date, market_code, device_code, country_code, category_id """.stripMargin carbon.sql(createTimeSeriesMonthNoProductTableSql) val createTimeSeriesYearNoProductTableSql = s""" | CREATE DATAMAP IF NOT EXISTS agg_by_year ON TABLE store | USING 'timeSeries' | DMPROPERTIES ( | 'EVENT_TIME'='date', | 'YEAR_GRANULARITY'='1') | AS SELECT date, market_code, device_code, country_code, category_id, | COUNT(product_id), COUNT(est_free_app_download), COUNT(est_free_app_download), COUNT(est_revenue), | SUM(est_free_app_download), MIN(est_free_app_download), MAX(est_free_app_download), | SUM(est_paid_app_download), MIN(est_paid_app_download), MAX(est_paid_app_download), | SUM(est_revenue), MIN(est_revenue), MAX(est_revenue) | FROM store | GROUP BY date, market_code, device_code, country_code, category_id """.stripMargin carbon.sql(createTimeSeriesYearNoProductTableSql) } 2. Query and result carbon.time(carbon.sql( s""" |EXPLAIN SELECT date, market_code, device_code, country_code, category_id, product_id, est_free_app_download, est_paid_app_download, est_revenue |FROM store |WHERE date = '2016-09-01' AND device_code='ios-phone' AND country_code='EE' AND product_id IN (590416158, 590437560)""" .stripMargin).show(truncate=false) ) 2.1) First time query +-------------------+-----------+-----------+------------+-----------+----------+---------------------+---------------------+-----------+ |date |market_code|device_code|country_code|category_id|product_id|est_free_app_download|est_paid_app_download|est_revenue| +-------------------+-----------+-----------+------------+-----------+----------+---------------------+---------------------+-----------+ |2016-09-01 00:00:00|apple-store|ios-phone |EE |100000 |590416158 |1 |null |null | |2016-09-01 00:00:00|apple-store|ios-phone |EE |100021 |590416158 |1 |null |null | |2016-09-01 00:00:00|apple-store|ios-phone |EE |100021 |590437560 |null |1 |null | |2016-09-01 00:00:00|apple-store|ios-phone |EE |100070 |590437560 |null |1 |null | |2016-09-01 00:00:00|apple-store|ios-phone |EE |100076 |590416158 |1 |null |null | |2016-09-01 00:00:00|apple-store|ios-phone |EE |100000 |590437560 |null |1 |null | +-------------------+-----------+-----------+------------+-----------+----------+---------------------+---------------------+-----------+ 2.2) Second time query +-------------------+-----------+-----------+------------+-----------+----------+---------------------+---------------------+-----------+ |date |market_code|device_code|country_code|category_id|product_id|est_free_app_download|est_paid_app_download|est_revenue| +-------------------+-----------+-----------+------------+-----------+----------+---------------------+---------------------+-----------+ |2016-09-01 00:00:00|apple-store|ios-phone |EE |100000 |590416158 |null |1 |null | |2016-09-01 00:00:00|apple-store|ios-phone |EE |100023 |590416158 |null |1 |null | |2016-09-01 00:00:00|apple-store|ios-phone |EE |100023 |590437560 |null |1 |null | |2016-09-01 00:00:00|apple-store|ios-phone |EE |100023 |590437560 |null |1 |null | |2016-09-01 00:00:00|apple-store|ios-phone |EE |100000 |590437560 |null |1 |null | +-------------------+-----------+-----------+------------+-----------+----------+---------------------+---------------------+-----------+ category 100021, 100070, 100076 missed and 100023 should not be here because 2016-09-01 does not have this record in source data. And I have execute tons of queries, but they can not return right result as first time query. I doubt that it has something with compaction and local dictionary. Could anyone give any suggestions about this? Thanks Aaron -- Sent from: http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ |
Hi Aaron,
Thanks for reporting issue. Can you help me narrow down the issue? as I cannot reproduce locally with the information given in your mail. a) First can you disable local dictionary and try the same scenario? b) Can drop datamp and try the same scenario? -- If data is coming from data map (can see this in explain command) c) Avoid compaction and try the same scenario. d) If you can share, give me test data and complete steps. (Because compaction and other steps are not there in your previous mail) Mean while, I will try to reproduce locally again but I don't have complete steps you executed. Thanks, Ajantha On Wed, Sep 26, 2018 at 9:17 PM aaron <[hidden email]> wrote: > Hi Community, > > It seems that rows disappeared, same query get different result > > carbon.time(carbon.sql( > s""" > |EXPLAIN SELECT date, market_code, device_code, country_code, > category_id, product_id, est_free_app_download, est_paid_app_download, > est_revenue > |FROM store > |WHERE date = '2016-09-01' AND device_code='ios-phone' AND > country_code='EE' AND product_id IN (590416158, 590437560)""" > .stripMargin).show(truncate=false) > ) > > > Screen_Shot_2018-09-26_at_11.png > < > http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/file/t357/Screen_Shot_2018-09-26_at_11.png> > > Screen_Shot_2018-09-26_at_11.png > < > http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/file/t357/Screen_Shot_2018-09-26_at_11.png> > > > > > -- > Sent from: > http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ > |
**************************************************************************
a) First can you disable local dictionary and try the same scenario? I would test in other time Good idea, and I think this works, when I use global dictionary, query can return right result. But the question is, global dictionary also introduce a bug in spark 2.3, which I described in another issue. http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/Issue-Dictionary-and-S3-td63106.html ************************************************************************** b) Can drop datamp and try the same scenario? -- If data is coming from data map (can see this in explain command) I have confirmed this, datamap is not the reason for this. because this can reproduce without datamap. ************************************************************************** c) Avoid compaction and try the same scenario. I've confirmed, if no compaction, query works well. ************************************************************************** d) If you can share, give me test data and complete steps. (Because compaction and other steps are not there in your previous mail) The data is kind of huge, the table holds on about 7T csv raw data. I have no good idea to give you test data:) -- Sent from: http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ |
In reply to this post by Ajantha Bhat
Another comment, this issue can be reproduces on spark2.3.1 +
carbondata1.5.0, spark2.2.2 + carbondata1.5.0, I can send you the jar I compiled to you, hope this could help you. -- Sent from: http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ |
So, both local dictionary and compaction is required to reproduce the
issue? Without any one of them. Issue will not occur right? On Thu 27 Sep, 2018, 6:54 PM aaron, <[hidden email]> wrote: > Another comment, this issue can be reproduces on spark2.3.1 + > carbondata1.5.0, spark2.2.2 + carbondata1.5.0, I can send you the jar I > compiled to you, hope this could help you. > > > > -- > Sent from: > http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ > |
This post was updated on .
Yes, I think you're right. But the compaction comes from auto load merge
-- Sent from: http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ |
This post was updated on .
In reply to this post by Ajantha Bhat
This is the method I construct carbon instance, hope this can help you.
def carbonSession(appName: String, masterUrl: String, parallelism: String, logLevel: String, hdfsUrl: String="hdfs://xxx:9000"): SparkSession = { val storeLocation = s"${hdfsUrl}/usr/carbon/data" CarbonProperties.getInstance() .addProperty(CarbonCommonConstants.STORE_LOCATION, storeLocation) .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, "true") .addProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT, "true") .addProperty(CarbonCommonConstants.CARBON_TASK_DISTRIBUTION, CarbonCommonConstants.CARBON_TASK_DISTRIBUTION_BLOCKLET) .addProperty(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION, "false") .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true") //.addProperty(CarbonCommonConstants.ENABLE_AUTO_HANDOFF, "true") .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true") .addProperty(CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD, "4,3") .addProperty(CarbonCommonConstants.DAYS_ALLOWED_TO_COMPACT, "0") .addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC, s"${hdfsUrl}/usr/carbon/badrecords") .addProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED, "true") .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "false") .addProperty(CarbonCommonConstants.ENABLE_DATA_LOADING_STATISTICS, "false") .addProperty(CarbonCommonConstants.MAX_QUERY_EXECUTION_TIME, "2") // 2 minutes .addProperty(CarbonCommonConstants.LOCK_TYPE, "HDFSLOCK") .addProperty(CarbonCommonConstants.LOCK_PATH, s"${hdfsUrl}/usr/carbon/lock") .addProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD, s"${parallelism}") .addProperty(CarbonCommonConstants.CARBON_INVISIBLE_SEGMENTS_PRESERVE_COUNT, "100") .addProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS, s"${parallelism}") .addProperty(CarbonCommonConstants.LOAD_SORT_SCOPE, "LOCAL_SORT") .addProperty(CarbonCommonConstants.NUM_CORES_COMPACTING, s"${parallelism}") .addProperty(CarbonCommonConstants.UNSAFE_WORKING_MEMORY_IN_MB, "4096") .addProperty(CarbonCommonConstants.NUM_CORES_LOADING, s"${parallelism}") .addProperty(CarbonCommonConstants.CARBON_MAJOR_COMPACTION_SIZE, "1024") .addProperty(CarbonCommonConstants.BLOCKLET_SIZE, "64") //.addProperty(CarbonCommonConstants.TABLE_BLOCKLET_SIZE, "64") import org.apache.spark.sql.CarbonSession._ val carbon = SparkSession .builder() .master(masterUrl) .appName(appName) .config("spark.hadoop.fs.s3a.impl", "org.apache.hadoop.fs.s3a.S3AFileSystem") .config("spark.hadoop.dfs.replication", 1) .config("spark.cores.max", s"${parallelism}") .getOrCreateCarbonSession(storeLocation) carbon.sparkContext.hadoopConfiguration.setInt("dfs.replication", 1) carbon.sql(s"SET spark.default.parallelism=${parallelism}") carbon.sql(s"SET spark.sql.shuffle.partitions=${parallelism}") carbon.sql(s"SET spark.sql.cbo.enabled=true") carbon.sql(s"SET carbon.options.bad.records.logger.enable=true") carbon.sparkContext.setLogLevel(logLevel) carbon } -- Sent from: http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ |
Hi Aaron,
Can you please run compaction again with *carbon.local.dictionary.decoder.fallback=false *and share the result for the same. -Regards Kumar Vishal On Thu, Sep 27, 2018 at 7:37 PM aaron <[hidden email]> wrote: > This is the method I construct carbon instance, hope this can help you. > > def carbonSession(appName: String, masterUrl: String, parallelism: String, > logLevel: String, hdfsUrl: > String="hdfs://ec2-dca-aa-p-sdn-16.appannie.org:9000"): SparkSession = { > val storeLocation = s"${hdfsUrl}/usr/carbon/data" > > CarbonProperties.getInstance() > .addProperty(CarbonCommonConstants.STORE_LOCATION, storeLocation) > .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, "true") > .addProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT, "true") > .addProperty(CarbonCommonConstants.CARBON_TASK_DISTRIBUTION, > CarbonCommonConstants.CARBON_TASK_DISTRIBUTION_BLOCKLET) > .addProperty(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION, > "false") > .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true") > //.addProperty(CarbonCommonConstants.ENABLE_AUTO_HANDOFF, "true") > .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true") > > .addProperty(CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD, > "4,3") > .addProperty(CarbonCommonConstants.DAYS_ALLOWED_TO_COMPACT, "0") > .addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC, > s"${hdfsUrl}/usr/carbon/badrecords") > .addProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED, > "true") > .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, "false") > .addProperty(CarbonCommonConstants.ENABLE_DATA_LOADING_STATISTICS, > "false") > .addProperty(CarbonCommonConstants.MAX_QUERY_EXECUTION_TIME, "2") // > 2 minutes > .addProperty(CarbonCommonConstants.LOCK_TYPE, "HDFSLOCK") > .addProperty(CarbonCommonConstants.LOCK_PATH, > s"${hdfsUrl}/usr/carbon/lock") > .addProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD, > s"${parallelism}") > > > .addProperty(CarbonCommonConstants.CARBON_INVISIBLE_SEGMENTS_PRESERVE_COUNT, > "100") > .addProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS, > s"${parallelism}") > .addProperty(CarbonCommonConstants.LOAD_SORT_SCOPE, "LOCAL_SORT") > .addProperty(CarbonCommonConstants.NUM_CORES_COMPACTING, > s"${parallelism}") > .addProperty(CarbonCommonConstants.UNSAFE_WORKING_MEMORY_IN_MB, > "4096") > .addProperty(CarbonCommonConstants.NUM_CORES_LOADING, > s"${parallelism}") > .addProperty(CarbonCommonConstants.CARBON_MAJOR_COMPACTION_SIZE, > "1024") > .addProperty(CarbonCommonConstants.BLOCKLET_SIZE, "64") > //.addProperty(CarbonCommonConstants.TABLE_BLOCKLET_SIZE, "64") > > import org.apache.spark.sql.CarbonSession._ > > val carbon = SparkSession > .builder() > .master(masterUrl) > .appName(appName) > .config("spark.hadoop.fs.s3a.impl", > "org.apache.hadoop.fs.s3a.S3AFileSystem") > .config("spark.hadoop.dfs.replication", 1) > .config("spark.cores.max", s"${parallelism}") > .getOrCreateCarbonSession(storeLocation) > > carbon.sparkContext.hadoopConfiguration.setInt("dfs.replication", 1) > > carbon.sql(s"SET spark.default.parallelism=${parallelism}") > carbon.sql(s"SET spark.sql.shuffle.partitions=${parallelism}") > carbon.sql(s"SET spark.sql.cbo.enabled=true") > carbon.sql(s"SET carbon.options.bad.records.logger.enable=true") > > carbon.sparkContext.setLogLevel(logLevel) > carbon > } > > > > -- > Sent from: > http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ >
kumar vishal
|
@Aaron: I was able to reproduce the issue with my own dataset. (total 350
KB data) Issue is nothing to do with local dictionary. I have narrowed down the scenario, it is with sort columns + compaction. I will fix soon and update you Thanks, Ajantha On Thu, Sep 27, 2018 at 8:05 PM Kumar Vishal <[hidden email]> wrote: > Hi Aaron, > Can you please run compaction again with > *carbon.local.dictionary.decoder.fallback=false > *and share the result for the same. > > -Regards > Kumar Vishal > > On Thu, Sep 27, 2018 at 7:37 PM aaron <[hidden email]> wrote: > > > This is the method I construct carbon instance, hope this can help you. > > > > def carbonSession(appName: String, masterUrl: String, parallelism: > String, > > logLevel: String, hdfsUrl: > > String="hdfs://ec2-dca-aa-p-sdn-16.appannie.org:9000"): SparkSession = { > > val storeLocation = s"${hdfsUrl}/usr/carbon/data" > > > > CarbonProperties.getInstance() > > .addProperty(CarbonCommonConstants.STORE_LOCATION, storeLocation) > > .addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, "true") > > .addProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT, "true") > > .addProperty(CarbonCommonConstants.CARBON_TASK_DISTRIBUTION, > > CarbonCommonConstants.CARBON_TASK_DISTRIBUTION_BLOCKLET) > > > .addProperty(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION, > > "false") > > .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "true") > > //.addProperty(CarbonCommonConstants.ENABLE_AUTO_HANDOFF, "true") > > .addProperty(CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE, "true") > > > > .addProperty(CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD, > > "4,3") > > .addProperty(CarbonCommonConstants.DAYS_ALLOWED_TO_COMPACT, "0") > > .addProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC, > > s"${hdfsUrl}/usr/carbon/badrecords") > > .addProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED, > > "true") > > .addProperty(CarbonCommonConstants.ENABLE_QUERY_STATISTICS, > "false") > > .addProperty(CarbonCommonConstants.ENABLE_DATA_LOADING_STATISTICS, > > "false") > > .addProperty(CarbonCommonConstants.MAX_QUERY_EXECUTION_TIME, "2") > // > > 2 minutes > > .addProperty(CarbonCommonConstants.LOCK_TYPE, "HDFSLOCK") > > .addProperty(CarbonCommonConstants.LOCK_PATH, > > s"${hdfsUrl}/usr/carbon/lock") > > .addProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD, > > s"${parallelism}") > > > > > > > .addProperty(CarbonCommonConstants.CARBON_INVISIBLE_SEGMENTS_PRESERVE_COUNT, > > "100") > > .addProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS, > > s"${parallelism}") > > .addProperty(CarbonCommonConstants.LOAD_SORT_SCOPE, "LOCAL_SORT") > > .addProperty(CarbonCommonConstants.NUM_CORES_COMPACTING, > > s"${parallelism}") > > .addProperty(CarbonCommonConstants.UNSAFE_WORKING_MEMORY_IN_MB, > > "4096") > > .addProperty(CarbonCommonConstants.NUM_CORES_LOADING, > > s"${parallelism}") > > .addProperty(CarbonCommonConstants.CARBON_MAJOR_COMPACTION_SIZE, > > "1024") > > .addProperty(CarbonCommonConstants.BLOCKLET_SIZE, "64") > > //.addProperty(CarbonCommonConstants.TABLE_BLOCKLET_SIZE, "64") > > > > import org.apache.spark.sql.CarbonSession._ > > > > val carbon = SparkSession > > .builder() > > .master(masterUrl) > > .appName(appName) > > .config("spark.hadoop.fs.s3a.impl", > > "org.apache.hadoop.fs.s3a.S3AFileSystem") > > .config("spark.hadoop.dfs.replication", 1) > > .config("spark.cores.max", s"${parallelism}") > > .getOrCreateCarbonSession(storeLocation) > > > > carbon.sparkContext.hadoopConfiguration.setInt("dfs.replication", 1) > > > > carbon.sql(s"SET spark.default.parallelism=${parallelism}") > > carbon.sql(s"SET spark.sql.shuffle.partitions=${parallelism}") > > carbon.sql(s"SET spark.sql.cbo.enabled=true") > > carbon.sql(s"SET carbon.options.bad.records.logger.enable=true") > > > > carbon.sparkContext.setLogLevel(logLevel) > > carbon > > } > > > > > > > > -- > > Sent from: > > http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ > > > |
@Ajantha, Great! looking forward to your fix:)
-- Sent from: http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ |
@Aaron:
Please find the issue fix changes in the below PR. *https://github.com/apache/carbondata/pull/2784 <https://github.com/apache/carbondata/pull/2784>* I added a test case also and it is passed after my fix. Thanks, Ajantha On Fri, Sep 28, 2018 at 4:57 AM aaron <[hidden email]> wrote: > @Ajantha, Great! looking forward to your fix:) > > > > -- > Sent from: > http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ > |
Great and I will have a try later
-- Sent from: http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ |
In reply to this post by Ajantha Bhat
Cool! It works now. Thanks a lot!
-- Sent from: http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/ |
Free forum by Nabble | Edit this page |