RE: Change for DELETE SEGMENT FROM TABLE syntax
I am definitely not fan of introducing new syntax, but considering this is a unique feature of Carbon, I would like to suggest to use ALTER TABLE statement: ALTER TABLE DROP SEGMENTS [segment-spec] Segment-spec can be id or range. Regards. Jenny -Original Message- From: Ravindra Pesala [mailto:ravi.pes...@gmail.com] Sent: Friday, September 23, 2016 9:56 AM To: dev Subject: Re: Change for DELETE SEGMENT FROM TABLE syntax Hi Jay, Here if you use carbonsqlparser first then it won't be any issue. But if you use hive parser first then this issue arises. But I think there is a same case even with the 'Load' command, how that is handled here. And also the delete commands you mentioned does not look proper. DELETE FROM TABLE WHERE SEGMENT EQUAL id DELETE FROM TABLE WHERE SEGMENTS STARTTIME BEFORE date Here it seems like SEGMENT and SEGMENTS are like columns in table. I prefer the old DELETE commands over new commands. Thanks & Regards, Ravindra. On 23 September 2016 at 12:19, 杰 <2550062...@qq.com> wrote: > Hi,all >as we know, carbon have below DELETE syntax, > >DELETE SEGMENT id FROM TABLE >DELETE SEGMENTS FROM TABLE WHERE STARTTIME BEFORE date > >while in hive, there is similar syntax as below, > > > DELETE FILE | JAR | ARCHIVE > > >then, when we run DELETE SEGMENT in spark-sql that integrating hive > parser and carbon parser, it may give below > error message cause the parse in hive > > > ERROR DeleteResourceProcessor: Usage: delete [FILE|JAR|ARCHIVE] > []* > > which does not impact the usage of carbon, but looks not good. > So after a small discussion, we think that it may be better to make > change on our syntax and change to > > DELETE FROM TABLE WHERE SEGMENT EQUAL id > DELETE FROM TABLE WHERE SEGMENTS STARTTIME BEFORE date > > > i have raised JIRA[CARBONDATA-269] and PR#192 for this issue, please > give your suggestion about this change. > > > > Thanks > Jay -- Thanks & Regards, Ravi
RE: carbondata and idempotence
Hi I second Jenny here. It's not yet supported but definitely a good feature. Regards JB On Sep 23, 2016, 14:03, at 14:03, Jihong Ma wrote: >Hi Vincent, > >Are you referring to writing out Spark streaming data to Carbon file? >we don't support it yet, but it is in our near term plan to add the >integration, we will start the discussion in the dev list soon and >would love to hear your input, we will take into account the old >DStream interface as well as Spark 2.0 structured streaming, we would >like to ensure exactly-once semantics and design Carbon as an >idempotent sink. > >At the moment, we have fully integrated with Spark SQL with both SQL >and API interface, with the help multi-level indexes, we have seen >dramatic performance boost compared to other columnar file format on >hadoop eco-system. You are welcome to try it out for your batch >processing workload, the streaming ingest will come out a little later. > > >Regards. > >Jenny > >-Original Message- >From: vincent gromakowski [mailto:vincent.gromakow...@gmail.com] >Sent: Friday, September 23, 2016 7:33 AM >To: dev@carbondata.incubator.apache.org >Subject: carbondata and idempotence > >Hi Carbondata community, >I am evaluating various file format right now and found Carbondata to >be >interesting specially with the multiple index used to avoid full scan >but I >am asking if there is any way to achieve idem potence when writing to >Carbondata from Spark (or alternative) ? >A strong requirement is to avoid a Spark worker crash to write >duplicated >entries in Carbon... >Tx > >Vincent
RE: carbondata and idempotence
Hi Vincent, Are you referring to writing out Spark streaming data to Carbon file? we don't support it yet, but it is in our near term plan to add the integration, we will start the discussion in the dev list soon and would love to hear your input, we will take into account the old DStream interface as well as Spark 2.0 structured streaming, we would like to ensure exactly-once semantics and design Carbon as an idempotent sink. At the moment, we have fully integrated with Spark SQL with both SQL and API interface, with the help multi-level indexes, we have seen dramatic performance boost compared to other columnar file format on hadoop eco-system. You are welcome to try it out for your batch processing workload, the streaming ingest will come out a little later. Regards. Jenny -Original Message- From: vincent gromakowski [mailto:vincent.gromakow...@gmail.com] Sent: Friday, September 23, 2016 7:33 AM To: dev@carbondata.incubator.apache.org Subject: carbondata and idempotence Hi Carbondata community, I am evaluating various file format right now and found Carbondata to be interesting specially with the multiple index used to avoid full scan but I am asking if there is any way to achieve idem potence when writing to Carbondata from Spark (or alternative) ? A strong requirement is to avoid a Spark worker crash to write duplicated entries in Carbon... Tx Vincent
Re: Change for DELETE SEGMENT FROM TABLE syntax
Hi Jay, Here if you use carbonsqlparser first then it won't be any issue. But if you use hive parser first then this issue arises. But I think there is a same case even with the 'Load' command, how that is handled here. And also the delete commands you mentioned does not look proper. DELETE FROM TABLE WHERE SEGMENT EQUAL id DELETE FROM TABLE WHERE SEGMENTS STARTTIME BEFORE date Here it seems like SEGMENT and SEGMENTS are like columns in table. I prefer the old DELETE commands over new commands. Thanks & Regards, Ravindra. On 23 September 2016 at 12:19, 杰 <2550062...@qq.com> wrote: > Hi,all >as we know, carbon have below DELETE syntax, > >DELETE SEGMENT id FROM TABLE >DELETE SEGMENTS FROM TABLE WHERE STARTTIME BEFORE date > >while in hive, there is similar syntax as below, > > > DELETE FILE | JAR | ARCHIVE > > >then, when we run DELETE SEGMENT in spark-sql that integrating hive > parser and carbon parser, it may give below > error message cause the parse in hive > > > ERROR DeleteResourceProcessor: Usage: delete [FILE|JAR|ARCHIVE] > []* > > which does not impact the usage of carbon, but looks not good. > So after a small discussion, we think that it may be better to make > change on our syntax and change to > > DELETE FROM TABLE WHERE SEGMENT EQUAL id > DELETE FROM TABLE WHERE SEGMENTS STARTTIME BEFORE date > > > i have raised JIRA[CARBONDATA-269] and PR#192 for this issue, please > give your suggestion about this change. > > > > Thanks > Jay -- Thanks & Regards, Ravi
Re: carbondata and idempotence
Hi Vincent Happy to hear you are interested in Apache CarbonData. Write to CarbonData file from Spark, please refer to the example : DataFrameAPIExample. Can you explain more about this requirement:A strong requirement is to avoid a Spark worker crash to write duplicated entries in Carbon... ? Regards Liang -- View this message in context: http://apache-carbondata-mailing-list-archive.1130556.n5.nabble.com/carbondata-and-idempotence-tp1416p1417.html Sent from the Apache CarbonData Mailing List archive mailing list archive at Nabble.com.
carbondata and idempotence
Hi Carbondata community, I am evaluating various file format right now and found Carbondata to be interesting specially with the multiple index used to avoid full scan but I am asking if there is any way to achieve idem potence when writing to Carbondata from Spark (or alternative) ? A strong requirement is to avoid a Spark worker crash to write duplicated entries in Carbon... Tx Vincent
[GitHub] incubator-carbondata pull request #192: [CARBONDATA-269]change delete segmen...
Github user Jay357089 closed the pull request at: https://github.com/apache/incubator-carbondata/pull/192 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #190: [CARBONDATA-268]Improve carbonoptimi...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-carbondata/pull/190 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #195: FIX CI
Github user asfgit closed the pull request at: https://github.com/apache/incubator-carbondata/pull/195 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Created] (CARBONDATA-271) Non Filter data mismatch issue
kumar vishal created CARBONDATA-271: --- Summary: Non Filter data mismatch issue Key: CARBONDATA-271 URL: https://issues.apache.org/jira/browse/CARBONDATA-271 Project: CarbonData Issue Type: Bug Reporter: kumar vishal Assignee: kumar vishal Problem: While generating the default end key we are taking LONG.MAX key and using segment key generator we are generating the end key if cardinality is less than it will give some value with in its cardinality and btree searching will fail Solution: From segment property get the dimension cardinality as this is the max value for segment -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-carbondata pull request #196: [CARBONDATA-271]Fixed non filter dat...
GitHub user kumarvishal09 opened a pull request: https://github.com/apache/incubator-carbondata/pull/196 [CARBONDATA-271]Fixed non filter data mismatch issue. Problem: While generating the default end key we are taking LONG.MAX key and using segment key generator we are generating the end key if cardinality is less than it will give some value with in its cardinality and btree searching will fail Solution: From segment property get the dimension cardinality as this is the max value for segment You can merge this pull request into a Git repository by running: $ git pull https://github.com/kumarvishal09/incubator-carbondata NonFilterDataMismatchIssue Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-carbondata/pull/196.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 #196 commit db865d428639be0ab7e593f92931f85fcf71 Author: kumarvishal Date: 2016-09-23T10:48:52Z Fixed non filter data mismatch issue. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #194: [CARBONDATA-270] Double data type va...
Github user sujith71955 commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/194#discussion_r80221578 --- Diff: core/src/main/java/org/apache/carbondata/scan/filter/FilterUtil.java --- @@ -1401,8 +1401,7 @@ public static void logError(Throwable e, boolean invalidRowsPresent) { public static boolean nanSafeEqualsDoubles(Double d1, Double d2) { Boolean xIsNan = Double.isNaN(d1); Boolean yIsNan = Double.isNaN(d2); -if ((xIsNan && yIsNan) || (d1.doubleValue() == d2.doubleValue())) { - +if ((d1.doubleValue() == d2.doubleValue()) || (xIsNan && yIsNan)) { --- End diff -- yes ramana, right, i will fix it --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #190: [CARBONDATA-268]Improve carbonoptimi...
Github user gvramana commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/190#discussion_r80220071 --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala --- @@ -68,9 +70,50 @@ object CarbonOptimizer { */ class ResolveCarbonFunctions(relations: Seq[CarbonDecoderRelation]) extends Rule[LogicalPlan] with PredicateHelper { - + val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) def apply(plan: LogicalPlan): LogicalPlan = { -transformCarbonPlan(plan, relations) +if (relations.nonEmpty && !isOptimized(plan)) { + LOGGER.info("Starting to optimize plan") + val startTime = System.currentTimeMillis + val result = transformCarbonPlan(plan, relations) + LOGGER.info("Time taken to optimize plan: " + ( System.currentTimeMillis - startTime)) --- End diff -- add this to STATISTICS logger type --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #190: [CARBONDATA-268]Improve carbonoptimi...
Github user gvramana commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/190#discussion_r80220168 --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala --- @@ -397,20 +452,38 @@ class ResolveCarbonFunctions(relations: Seq[CarbonDecoderRelation]) case others => others } +} + +val tStartTime = System.currentTimeMillis --- End diff -- this variable not used --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #194: [CARBONDATA-270] Double data type va...
Github user gvramana commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/194#discussion_r80221140 --- Diff: core/src/main/java/org/apache/carbondata/scan/filter/FilterUtil.java --- @@ -1401,8 +1401,7 @@ public static void logError(Throwable e, boolean invalidRowsPresent) { public static boolean nanSafeEqualsDoubles(Double d1, Double d2) { Boolean xIsNan = Double.isNaN(d1); Boolean yIsNan = Double.isNaN(d2); -if ((xIsNan && yIsNan) || (d1.doubleValue() == d2.doubleValue())) { - +if ((d1.doubleValue() == d2.doubleValue()) || (xIsNan && yIsNan)) { --- End diff -- Boolean xIsNan = Double.isNaN(d1); these variables are already computed, no use of moving them. Instead use ( Double.isNaN(d1) && Double.isNaN(d2)) --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #190: [CARBONDATA-268]Improve carbonoptimi...
Github user gvramana commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/190#discussion_r80219989 --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala --- @@ -68,9 +70,50 @@ object CarbonOptimizer { */ class ResolveCarbonFunctions(relations: Seq[CarbonDecoderRelation]) extends Rule[LogicalPlan] with PredicateHelper { - + val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) def apply(plan: LogicalPlan): LogicalPlan = { -transformCarbonPlan(plan, relations) +if (relations.nonEmpty && !isOptimized(plan)) { + LOGGER.info("Starting to optimize plan") + val startTime = System.currentTimeMillis + val result = transformCarbonPlan(plan, relations) + LOGGER.info("Time taken to optimize plan: " + ( System.currentTimeMillis - startTime)) --- End diff -- change the statement to "Time taken for Carbon Optimizer to optimize" --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #190: [CARBONDATA-268]Improve carbonoptimi...
Github user gvramana commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/190#discussion_r80219576 --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala --- @@ -87,20 +130,32 @@ class ResolveCarbonFunctions(relations: Seq[CarbonDecoderRelation]) return plan } var decoder = false +val mapOfNodes = new java.util.HashMap[LogicalPlan, ExtraNodeInfo] +fillNodeInfo(plan, mapOfNodes) val aliasMap = CarbonAliasDecoderRelation() // collect alias information before hand. collectInformationOnAttributes(plan, aliasMap) -val transFormedPlan = - plan transformDown { -case cd: CarbonDictionaryTempDecoder if cd.isOuter => - decoder = true - cd + +def hasCarbonRelation(currentPlan: LogicalPlan): Boolean = { + val extraNodeInfo = mapOfNodes.get(currentPlan) --- End diff -- Change the name of mapofNodes to mapOfNonCarbonPlanNodes --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #194: [CARBONDATA-270] Double data type va...
Github user ManoharVanam commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/194#discussion_r80212063 --- Diff: core/src/main/java/org/apache/carbondata/scan/filter/FilterUtil.java --- @@ -1401,8 +1401,7 @@ public static void logError(Throwable e, boolean invalidRowsPresent) { public static boolean nanSafeEqualsDoubles(Double d1, Double d2) { Boolean xIsNan = Double.isNaN(d1); Boolean yIsNan = Double.isNaN(d2); -if ((xIsNan && yIsNan) || (d1.doubleValue() == d2.doubleValue())) { - +if ((d1.doubleValue() == d2.doubleValue()) || (xIsNan && yIsNan)) { --- End diff -- I think we can use Double.compare instead of these two checks --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #176: [CARBONDATA-208] Add configuration e...
Github user foryou2030 commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/176#discussion_r80208061 --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorderImpl.java --- @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.carbondata.core.carbon.querystatistics; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.apache.carbondata.common.logging.LogService; +import org.apache.carbondata.common.logging.LogServiceFactory; + +import static org.apache.carbondata.core.util.CarbonUtil.printLine; + +/** + * Class will be used to record and log the query statistics + */ +public class QueryStatisticsRecorderImpl implements QueryStatisticsRecorder,Serializable { + + private static final LogService LOGGER = + LogServiceFactory.getLogService(QueryStatisticsRecorderImpl.class.getName()); + + /** + * serialization version + */ + private static final long serialVersionUID = -5719752001674467864L; + + /** + * list for statistics to record time taken + * by each phase of the query for example aggregation + * scanning,block loading time etc. + */ + private List queryStatistics; + + /** + * query with taskd + */ + private String queryIWthTask; + + /** + * lock for log statistics table + */ + private static final Object lock = new Object(); + + public QueryStatisticsRecorderImpl(String queryId) { +queryStatistics = new ArrayList(); +this.queryIWthTask = queryId; + } + + /** + * Below method will be used to add the statistics + * + * @param statistic + */ + public synchronized void recordStatistics(QueryStatistic statistic) { +queryStatistics.add(statistic); + } + + /** + * Below method will be used to log the statistic + */ + public void logStatistics() { +for (QueryStatistic statistic : queryStatistics) { + LOGGER.statistic(statistic.getStatistics(queryIWthTask)); +} + } + + /** + * Below method will be used to show statistic log as table + */ + public void logStatisticsAsTableExecutor() { +synchronized (lock) { --- End diff -- yes, i think no need lock. For executor, one task one recorder. what do u think? @Vimal-Das --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #195: FIX CI
GitHub user Zhangshunyu opened a pull request: https://github.com/apache/incubator-carbondata/pull/195 FIX CI You can merge this pull request into a Git repository by running: $ git pull https://github.com/Zhangshunyu/incubator-carbondata FIXCI Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-carbondata/pull/195.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 #195 commit 22b1f1491d5e5306db012a7541aa30790d11cdae Author: Zhangshunyu Date: 2016-09-23T08:08:16Z FIX CI --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #176: [CARBONDATA-208] Add configuration e...
Github user foryou2030 commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/176#discussion_r80198494 --- Diff: core/src/main/java/org/apache/carbondata/core/util/CarbonTimeStatisticsFactory.java --- @@ -17,13 +17,14 @@ package org.apache.carbondata.core.util; -import org.apache.carbondata.core.carbon.querystatistics.DriverQueryStatisticsRecorder; +import org.apache.carbondata.core.carbon.querystatistics.*; import org.apache.carbondata.core.constants.CarbonCommonConstants; public class CarbonTimeStatisticsFactory { private static String LoadStatisticsInstanceType; private static LoadStatistics LoadStatisticsInstance; - private static DriverQueryStatisticsRecorder QueryStatisticsRecorderInstance; + private static String queryStatisticsRecorderInstanceType; + private static QueryStatisticsRecorder QueryStatisticsRecorderInstance; --- End diff -- already initialized ``` static { CarbonTimeStatisticsFactory.updateTimeStatisticsUtilStatus(); LoadStatisticsInstance = genLoadStatisticsInstance(); QueryStatisticsRecorderInstance = genQueryStatisticsRecorderInstance(); } ``` --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #176: [CARBONDATA-208] Add configuration e...
Github user foryou2030 commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/176#discussion_r80197685 --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java --- @@ -461,8 +460,8 @@ private Object getFilterPredicates(Configuration configuration) { FilterExpressionProcessor filterExpressionProcessor, AbsoluteTableIdentifier absoluteTableIdentifier, FilterResolverIntf resolver, String segmentId) throws IndexBuilderException, IOException { - -QueryStatisticsRecorder recorder = new QueryStatisticsRecorder(""); +QueryStatisticsRecorder recorder = + CarbonTimeStatisticsFactory.getQueryStatisticsRecorderInstance(); --- End diff -- already fixed --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #176: [CARBONDATA-208] Add configuration e...
Github user foryou2030 commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/176#discussion_r80197172 --- Diff: core/src/main/java/org/apache/carbondata/core/util/CarbonTimeStatisticsFactory.java --- @@ -52,12 +56,27 @@ public static LoadStatistics getLoadStatisticsInstance() { return LoadStatisticsInstance; } - private static DriverQueryStatisticsRecorder genQueryStatisticsRecorderInstance() { -return DriverQueryStatisticsRecorder.getInstance(); + private static QueryStatisticsRecorder genQueryStatisticsRecorderInstance() { +if (queryStatisticsRecorderInstanceType.equalsIgnoreCase("true")) { + return DriverQueryStatisticsRecorderImpl.getInstance(); +} else { + return DriverQueryStatisticsRecorderDummy.getInstance(); +} } - public static DriverQueryStatisticsRecorder getQueryStatisticsRecorderInstance() { + public static QueryStatisticsRecorder getQueryStatisticsRecorderInstance() { return QueryStatisticsRecorderInstance; } + public static QueryStatisticsRecorder getQueryStatisticsRecorder(String queryId) { --- End diff -- fixed --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #176: [CARBONDATA-208] Add configuration e...
Github user foryou2030 commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/176#discussion_r80197104 --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java --- @@ -461,8 +459,15 @@ private Object getFilterPredicates(Configuration configuration) { FilterExpressionProcessor filterExpressionProcessor, AbsoluteTableIdentifier absoluteTableIdentifier, FilterResolverIntf resolver, String segmentId) throws IndexBuilderException, IOException { - -QueryStatisticsRecorder recorder = new QueryStatisticsRecorder(""); +String queryStatisticsRecorderInstanceType = CarbonProperties.getInstance() --- End diff -- fixed --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #194: [CARBONDATA-270] Double data type va...
GitHub user sujith71955 opened a pull request: https://github.com/apache/incubator-carbondata/pull/194 [CARBONDATA-270] Double data type value comparison optimization [Description] Double data type value comparison optimization,EqualsToExpression evaluation for double values first check for the equality of nan values and then the double value comparison happens, since nan comparison scenarios are rare we can push the comparison of nan after the double value comparison. Already present UT testcases can verify the feature modification impact. You can merge this pull request into a Git repository by running: $ git pull https://github.com/sujith71955/incubator-carbondata master_doubleCmprOptimizations Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-carbondata/pull/194.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 #194 commit f6770ee14f737911071ba94252089587efb3c6d4 Author: sujith71955 Date: 2016-09-23T07:07:12Z [Issue Number] CARBONDATA-270] [Description] Double data type value comparison optimization,EqualsToExpression evaluation for double values first check for the equality of nan values and then the double value comparison happens, since nan comparison scenarios are rare we can push the comparison of nan after the double value comparison. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Created] (CARBONDATA-270) [Filter Optimization] double data type value comparison optimization
Sujith created CARBONDATA-270: - Summary: [Filter Optimization] double data type value comparison optimization Key: CARBONDATA-270 URL: https://issues.apache.org/jira/browse/CARBONDATA-270 Project: CarbonData Issue Type: Improvement Components: core Reporter: Sujith Assignee: Sujith Priority: Minor EqualsToExpression evaluation for double values first check for the equality of nan values and then the double value comparison happens, since nan comparison scenarios are rare we can push the comparison of nan after the double value comparison. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-carbondata pull request #192: [CARBONDATA-269]change delete segmen...
Github user ManoharVanam commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/192#discussion_r80060992 --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala --- @@ -1181,23 +1182,21 @@ class CarbonSqlParser() p.getClass.getSimpleName.equals("DecimalLit") } ) ^^ (_.chars) protected lazy val deleteLoadsByID: Parser[LogicalPlan] = -DELETE ~> SEGMENT ~> repsep(segmentId, ",") ~ (FROM ~> TABLE ~> - (ident <~ ".").? ~ ident) <~ +DELETE ~> FROM ~ TABLE ~> (ident <~ ".").? ~ ident ~ + (WHERE ~> SEGMENT ~> EQUAL ~> repsep(segmentId, ",")) <~ --- End diff -- I think its better to use segments in both places deleteLoadsByID , deleteLoadsByLoadDate to avoid confusion --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---