You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@carbondata.apache.org by foryou2030 <gi...@git.apache.org> on 2016/08/24 10:56:57 UTC

[GitHub] incubator-carbondata pull request #91: [WIP] Add performance statistics logs...

GitHub user foryou2030 opened a pull request:

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

    [WIP] Add performance statistics logs to record the query time taken by carbon

    

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

    $ git pull https://github.com/foryou2030/incubator-carbondata query_statistics

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

    https://github.com/apache/incubator-carbondata/pull/91.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 #91
    
----
commit 729e49ca40fc3dbc76ed7a03896ec15167d9e24b
Author: Jay357089 <li...@huawei.com>
Date:   2016-08-24T10:54:41Z

    add more query statistics

----


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76753482
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java ---
    @@ -61,14 +67,275 @@ public QueryStatisticsRecorder(String queryId) {
        */
       public synchronized void recordStatistics(QueryStatistic statistic) {
         queryStatistics.add(statistic);
    +    // refresh query Statistics Map
    +    String key = statistic.getQueryId();
    +    if (!StringUtils.isEmpty(key)) {
    +      // 240954528274124_0 and 240954528274124 is the same query id
    +      key = key.substring(0, 15);
    +    }
    +    if (queryStatisticsMap.get(key) != null) {
    --- End diff --
    
    how you are handling for each executor query 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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77120540
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsCommonConstants.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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;
    +
    +public final class QueryStatisticsCommonConstants {
    --- End diff --
    
    CarbonCommonConstants also we can change it to interface


---
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 #91: [CARBONDATA-200] Add performance stat...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77147007
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala ---
    @@ -32,6 +32,7 @@ import org.apache.carbondata.core.carbon.{AbsoluteTableIdentifier, ColumnIdentif
     import org.apache.carbondata.core.carbon.metadata.datatype.DataType
    --- End diff --
    
    handled


---
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 #91: [CARBONDATA-200] Add performance stat...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77147108
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/SingleQueryStatisticsRecorder.java ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.HashMap;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +
    +/**
    + * Class will be used to record and log the query statistics
    + */
    +public class SingleQueryStatisticsRecorder implements Serializable {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SingleQueryStatisticsRecorder.class.getName());
    +  /**
    +   * serialization version
    +   */
    +  private static final long serialVersionUID = -1L;
    +
    +  /**
    +   * singleton QueryStatisticsRecorder for driver
    +   */
    +  private HashMap<String, List<QueryStatistic>> queryStatisticsMap;
    +
    +  private SingleQueryStatisticsRecorder() {
    +    queryStatisticsMap = new HashMap<String, List<QueryStatistic>>();
    +  }
    +
    +  private static SingleQueryStatisticsRecorder carbonLoadStatisticsImplInstance =
    +      new SingleQueryStatisticsRecorder();
    +
    +  public static SingleQueryStatisticsRecorder getInstance() {
    +    return carbonLoadStatisticsImplInstance;
    +  }
    +
    +  /**
    +   * Below method will be used to add the statistics
    +   *
    +   * @param statistic
    +   */
    +  public synchronized void recordStatisticsForDriver(QueryStatistic statistic, String queryId) {
    +    // refresh query Statistics Map
    +    if (queryStatisticsMap.get(queryId) != null) {
    +      queryStatisticsMap.get(queryId).add(statistic);
    +    } else {
    +      List<QueryStatistic> newQueryStatistics = new ArrayList<QueryStatistic>();
    +      newQueryStatistics.add(statistic);
    +      queryStatisticsMap.put(queryId, newQueryStatistics);
    +    }
    +  }
    +
    +  /**
    +   * Below method will be used to show statistic log as table
    +   */
    +  public void logStatisticsAsTableDriver() {
    +    String tableInfo = collectDriverStatistics();
    +    if (null != tableInfo) {
    +      LOGGER.statistic(tableInfo);
    +    }
    +  }
    +
    +  /**
    +   * Below method will parse queryStatisticsMap and put time into table
    +   */
    +  public String collectDriverStatistics() {
    +    for (String key: queryStatisticsMap.keySet()) {
    +      try {
    +        // TODO: get the finished query, and print Statistics
    +        if (queryStatisticsMap.get(key).size() > 2) {
    +          String sql_parse_time = "";
    +          String load_meta_time = "";
    +          String block_allocation_time = "";
    +          String block_identification_time = "";
    +          Double driver_part_time_tmp = 0.0;
    +          String splitChar = " ";
    +          // get statistic time from the QueryStatistic
    +          for (QueryStatistic statistic : queryStatisticsMap.get(key)) {
    +            switch (statistic.getMessage()) {
    +              case QueryStatisticsConstants.SQL_PARSE:
    +                sql_parse_time += statistic.getTimeTaken() + splitChar;
    +                driver_part_time_tmp += statistic.getTimeTaken();
    +                break;
    +              case QueryStatisticsConstants.LOAD_META:
    +                load_meta_time += statistic.getTimeTaken() + splitChar;
    +                driver_part_time_tmp += statistic.getTimeTaken();
    +                break;
    +              case QueryStatisticsConstants.BLOCK_ALLOCATION:
    +                block_allocation_time += statistic.getTimeTaken() + splitChar;
    +                driver_part_time_tmp += statistic.getTimeTaken();
    +                break;
    +              case QueryStatisticsConstants.BLOCK_IDENTIFICATION:
    +                block_identification_time += statistic.getTimeTaken() + splitChar;
    +                driver_part_time_tmp += statistic.getTimeTaken();
    +                break;
    +              default:
    +                break;
    +            }
    +          }
    +          String driver_part_time = driver_part_time_tmp + splitChar;
    +          // structure the query statistics info table
    +          StringBuilder tableInfo = new StringBuilder();
    +          int len1 = 8;
    +          int len2 = 20;
    +          int len3 = 21;
    +          int len4 = 22;
    +          String line = "+" + printLine("-", len1) + "+" + printLine("-", len2) + "+" +
    +              printLine("-", len3) + "+" + printLine("-", len4) + "+";
    +          String line2 = "|" + printLine(" ", len1) + "+" + printLine("-", len2) + "+" +
    +              printLine(" ", len3) + "+" + printLine("-", len4) + "+";
    +          // table header
    +          tableInfo.append(line).append("\n");
    +          tableInfo.append("|" + printLine(" ", (len1 - "Module".length())) + "Module" + "|" +
    +              printLine(" ", (len2 - "Operation Step".length())) + "Operation Step" + "|" +
    +              printLine(" ", (len3 + len4 + 1 - "Query Cost".length())) +
    +              "Query Cost" + "|" + "\n");
    +          // driver part
    +          tableInfo.append(line).append("\n");
    +          tableInfo.append("|" + printLine(" ", len1) + "|" +
    +              printLine(" ", (len2 - "SQL parse".length())) + "SQL parse" + "|" +
    +              printLine(" ", len3) + "|" +
    +              printLine(" ", (len4 - sql_parse_time.length())) + sql_parse_time + "|" + "\n");
    +          tableInfo.append(line2).append("\n");
    +          tableInfo.append("|" +printLine(" ", (len1 - "Driver".length())) + "Driver" + "|" +
    +              printLine(" ", (len2 - "Load meta data".length())) + "Load meta data" + "|" +
    +              printLine(" ", (len3 - driver_part_time.length())) + driver_part_time + "|" +
    +              printLine(" ", (len4 - load_meta_time.length())) +
    +              load_meta_time + "|" + "\n");
    +          tableInfo.append(line2).append("\n");
    +          tableInfo.append("|" +
    +              printLine(" ", (len1 - "Part".length())) + "Part" + "|" +
    +              printLine(" ", (len2 - "Block allocation".length())) +
    +              "Block allocation" + "|" +
    +              printLine(" ", len3) + "|" +
    +              printLine(" ", (len4 - block_allocation_time.length())) +
    +              block_allocation_time + "|" + "\n");
    +          tableInfo.append(line2).append("\n");
    +          tableInfo.append("|" +
    +              printLine(" ", len1) + "|" +
    +              printLine(" ", (len2 - "Block identification".length())) +
    +              "Block identification" + "|" +
    +              printLine(" ", len3) + "|" +
    +              printLine(" ", (len4 - block_identification_time.length())) +
    +              block_identification_time + "|" + "\n");
    +          tableInfo.append(line).append("\n");
    +
    +          // once the statistics be printed, remove it from the map
    +          queryStatisticsMap.remove(key);
    +          // show query statistic as "query id" + "table"
    +          return "Print query statistic for query id: " + key + "\n" + tableInfo.toString();
    +        }
    +      } catch (Exception ex) {
    +        return "Put statistics into table failed, catch exception: " + ex.getMessage();
    +      }
    +    }
    +    return null;
    +  }
    +
    +  /**
    +   * Below method will create string like "***********"
    +   *
    +   * @param a
    +   * @param num
    +   */
    +  public static String printLine(String a, int num)
    --- End diff --
    
    handled


---
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 #91: [CARBONDATA-200] Add performance stat...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77147060
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/SingleQueryStatisticsRecorder.java ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.HashMap;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +
    +/**
    + * Class will be used to record and log the query statistics
    + */
    +public class SingleQueryStatisticsRecorder implements Serializable {
    --- End diff --
    
    handled


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76751559
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsCommonConstants.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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;
    +
    +public final class QueryStatisticsCommonConstants {
    --- End diff --
    
    change this class to interface as in interface all the property will be by default public static final, so not need to write in the code and no need to add any constructor for the same 


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77120933
  
    --- Diff: core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java ---
    @@ -45,10 +47,28 @@
     
       public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
         super(infos, queryModel);
    +    this.queryModel = queryModel;
    +  }
    +
    +  private Boolean flag;
    +
    +  private Long total = 0L;
    --- End diff --
    
    handled


---
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 #91: [CARBONDATA-200] Add performance stat...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77147039
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/SingleQueryStatisticsRecorder.java ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.HashMap;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +
    +/**
    + * Class will be used to record and log the query statistics
    + */
    +public class SingleQueryStatisticsRecorder implements Serializable {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SingleQueryStatisticsRecorder.class.getName());
    +  /**
    +   * serialization version
    +   */
    +  private static final long serialVersionUID = -1L;
    +
    +  /**
    +   * singleton QueryStatisticsRecorder for driver
    +   */
    +  private HashMap<String, List<QueryStatistic>> queryStatisticsMap;
    --- End diff --
    
    handled


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76752958
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java ---
    @@ -45,8 +47,16 @@
        */
       private long startTime;
     
    -  public QueryStatistic() {
    +  /**
    +   * number of count
    +   */
    +  private long count;
    +
    +  private String queryIdWthTask;
    +
    +  public QueryStatistic(String queryId) {
    --- End diff --
    
    Why queryId is required in QueryStatistic as queryId will be common for all the query statistics object, previously it was in query statistics recorder class as recorder was common, can we move it to recorder??  


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76760563
  
    --- Diff: core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java ---
    @@ -45,10 +47,28 @@
     
       public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
         super(infos, queryModel);
    +    this.queryModel = queryModel;
    +  }
    +
    +  private Boolean flag;
    +
    +  private Long total = 0L;
    --- End diff --
    
    Why wrapper Long is required why not primitive long ? 


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77131735
  
    --- Diff: integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala ---
    @@ -32,6 +32,7 @@ import org.apache.carbondata.core.carbon.{AbsoluteTableIdentifier, ColumnIdentif
     import org.apache.carbondata.core.carbon.metadata.datatype.DataType
    --- End diff --
    
    do not use System.nano time use system.currentTimeInMilii


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76751861
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java ---
    @@ -20,11 +20,14 @@
     
    --- End diff --
    
    Please add Driver Side block loading time in statistics


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77135543
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/SingleQueryStatisticsRecorder.java ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.HashMap;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +
    +/**
    + * Class will be used to record and log the query statistics
    + */
    +public class SingleQueryStatisticsRecorder implements Serializable {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SingleQueryStatisticsRecorder.class.getName());
    +  /**
    +   * serialization version
    +   */
    +  private static final long serialVersionUID = -1L;
    +
    +  /**
    +   * singleton QueryStatisticsRecorder for driver
    +   */
    +  private HashMap<String, List<QueryStatistic>> queryStatisticsMap;
    +
    +  private SingleQueryStatisticsRecorder() {
    +    queryStatisticsMap = new HashMap<String, List<QueryStatistic>>();
    +  }
    +
    +  private static SingleQueryStatisticsRecorder carbonLoadStatisticsImplInstance =
    +      new SingleQueryStatisticsRecorder();
    +
    +  public static SingleQueryStatisticsRecorder getInstance() {
    +    return carbonLoadStatisticsImplInstance;
    +  }
    +
    +  /**
    +   * Below method will be used to add the statistics
    +   *
    +   * @param statistic
    +   */
    +  public synchronized void recordStatisticsForDriver(QueryStatistic statistic, String queryId) {
    +    // refresh query Statistics Map
    +    if (queryStatisticsMap.get(queryId) != null) {
    +      queryStatisticsMap.get(queryId).add(statistic);
    +    } else {
    +      List<QueryStatistic> newQueryStatistics = new ArrayList<QueryStatistic>();
    +      newQueryStatistics.add(statistic);
    +      queryStatisticsMap.put(queryId, newQueryStatistics);
    +    }
    +  }
    +
    +  /**
    +   * Below method will be used to show statistic log as table
    +   */
    +  public void logStatisticsAsTableDriver() {
    +    String tableInfo = collectDriverStatistics();
    +    if (null != tableInfo) {
    +      LOGGER.statistic(tableInfo);
    +    }
    +  }
    +
    +  /**
    +   * Below method will parse queryStatisticsMap and put time into table
    +   */
    +  public String collectDriverStatistics() {
    +    for (String key: queryStatisticsMap.keySet()) {
    +      try {
    +        // TODO: get the finished query, and print Statistics
    +        if (queryStatisticsMap.get(key).size() > 2) {
    +          String sql_parse_time = "";
    +          String load_meta_time = "";
    +          String block_allocation_time = "";
    +          String block_identification_time = "";
    +          Double driver_part_time_tmp = 0.0;
    +          String splitChar = " ";
    +          // get statistic time from the QueryStatistic
    +          for (QueryStatistic statistic : queryStatisticsMap.get(key)) {
    +            switch (statistic.getMessage()) {
    +              case QueryStatisticsConstants.SQL_PARSE:
    +                sql_parse_time += statistic.getTimeTaken() + splitChar;
    +                driver_part_time_tmp += statistic.getTimeTaken();
    +                break;
    +              case QueryStatisticsConstants.LOAD_META:
    +                load_meta_time += statistic.getTimeTaken() + splitChar;
    +                driver_part_time_tmp += statistic.getTimeTaken();
    +                break;
    +              case QueryStatisticsConstants.BLOCK_ALLOCATION:
    +                block_allocation_time += statistic.getTimeTaken() + splitChar;
    +                driver_part_time_tmp += statistic.getTimeTaken();
    +                break;
    +              case QueryStatisticsConstants.BLOCK_IDENTIFICATION:
    +                block_identification_time += statistic.getTimeTaken() + splitChar;
    +                driver_part_time_tmp += statistic.getTimeTaken();
    +                break;
    +              default:
    +                break;
    +            }
    +          }
    +          String driver_part_time = driver_part_time_tmp + splitChar;
    +          // structure the query statistics info table
    +          StringBuilder tableInfo = new StringBuilder();
    +          int len1 = 8;
    +          int len2 = 20;
    +          int len3 = 21;
    +          int len4 = 22;
    +          String line = "+" + printLine("-", len1) + "+" + printLine("-", len2) + "+" +
    +              printLine("-", len3) + "+" + printLine("-", len4) + "+";
    +          String line2 = "|" + printLine(" ", len1) + "+" + printLine("-", len2) + "+" +
    +              printLine(" ", len3) + "+" + printLine("-", len4) + "+";
    +          // table header
    +          tableInfo.append(line).append("\n");
    +          tableInfo.append("|" + printLine(" ", (len1 - "Module".length())) + "Module" + "|" +
    +              printLine(" ", (len2 - "Operation Step".length())) + "Operation Step" + "|" +
    +              printLine(" ", (len3 + len4 + 1 - "Query Cost".length())) +
    +              "Query Cost" + "|" + "\n");
    +          // driver part
    +          tableInfo.append(line).append("\n");
    +          tableInfo.append("|" + printLine(" ", len1) + "|" +
    +              printLine(" ", (len2 - "SQL parse".length())) + "SQL parse" + "|" +
    +              printLine(" ", len3) + "|" +
    +              printLine(" ", (len4 - sql_parse_time.length())) + sql_parse_time + "|" + "\n");
    +          tableInfo.append(line2).append("\n");
    +          tableInfo.append("|" +printLine(" ", (len1 - "Driver".length())) + "Driver" + "|" +
    +              printLine(" ", (len2 - "Load meta data".length())) + "Load meta data" + "|" +
    +              printLine(" ", (len3 - driver_part_time.length())) + driver_part_time + "|" +
    +              printLine(" ", (len4 - load_meta_time.length())) +
    +              load_meta_time + "|" + "\n");
    +          tableInfo.append(line2).append("\n");
    +          tableInfo.append("|" +
    +              printLine(" ", (len1 - "Part".length())) + "Part" + "|" +
    +              printLine(" ", (len2 - "Block allocation".length())) +
    +              "Block allocation" + "|" +
    +              printLine(" ", len3) + "|" +
    +              printLine(" ", (len4 - block_allocation_time.length())) +
    +              block_allocation_time + "|" + "\n");
    +          tableInfo.append(line2).append("\n");
    +          tableInfo.append("|" +
    +              printLine(" ", len1) + "|" +
    +              printLine(" ", (len2 - "Block identification".length())) +
    +              "Block identification" + "|" +
    +              printLine(" ", len3) + "|" +
    +              printLine(" ", (len4 - block_identification_time.length())) +
    +              block_identification_time + "|" + "\n");
    +          tableInfo.append(line).append("\n");
    +
    +          // once the statistics be printed, remove it from the map
    +          queryStatisticsMap.remove(key);
    +          // show query statistic as "query id" + "table"
    +          return "Print query statistic for query id: " + key + "\n" + tableInfo.toString();
    +        }
    +      } catch (Exception ex) {
    +        return "Put statistics into table failed, catch exception: " + ex.getMessage();
    +      }
    +    }
    +    return null;
    +  }
    +
    +  /**
    +   * Below method will create string like "***********"
    +   *
    +   * @param a
    +   * @param num
    +   */
    +  public static String printLine(String a, int num)
    --- End diff --
    
    move it to util class..carbonUtil


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76754835
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java ---
    @@ -61,14 +67,275 @@ public QueryStatisticsRecorder(String queryId) {
        */
       public synchronized void recordStatistics(QueryStatistic statistic) {
         queryStatistics.add(statistic);
    +    // refresh query Statistics Map
    +    String key = statistic.getQueryId();
    +    if (!StringUtils.isEmpty(key)) {
    +      // 240954528274124_0 and 240954528274124 is the same query id
    +      key = key.substring(0, 15);
    +    }
    +    if (queryStatisticsMap.get(key) != null) {
    --- End diff --
    
    i think 240954528274124_0  means queryid for each segment? yes?
    each executor will print whole time for all segment 
    if we have 3 executors, 2 segments, print like:
    executor1 log:
    |        +--------------------+                     +----------------------+
    |        |     Dictionary load|                     |     0.002  0.001 |
    |        +--------------------+                     +----------------------+
    
    executor 2 log:
    |        +--------------------+                     +----------------------+
    |        |     Dictionary load|                     |     0.003  0.002 |
    |        +--------------------+                     +----------------------+
    
    executor3 log:
    |        +--------------------+                     +----------------------+
    |        |     Dictionary load|                     |     0.001  0.003 |
    |        +--------------------+                     +----------------------+


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77134233
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/SingleQueryStatisticsRecorder.java ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.HashMap;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +
    +/**
    + * Class will be used to record and log the query statistics
    + */
    +public class SingleQueryStatisticsRecorder implements Serializable {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(SingleQueryStatisticsRecorder.class.getName());
    +  /**
    +   * serialization version
    +   */
    +  private static final long serialVersionUID = -1L;
    +
    +  /**
    +   * singleton QueryStatisticsRecorder for driver
    +   */
    +  private HashMap<String, List<QueryStatistic>> queryStatisticsMap;
    --- End diff --
    
    use Map<String,List<QueryStatistic>>


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77121643
  
    --- Diff: core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java ---
    @@ -45,10 +47,28 @@
     
       public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
         super(infos, queryModel);
    +    this.queryModel = queryModel;
    +  }
    +
    +  private Boolean flag;
    +
    +  private Long total = 0L;
    +
    +  private QueryModel queryModel;
    +
    +  @Override public boolean hasNext() {
    +    flag = super.hasNext();
    --- End diff --
    
    you can have time and record count variable in AbstractDetailQueryResultIterator and in next you can update the value....and once has next is false you can record the statistics 


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77120377
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsCommonConstants.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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;
    +
    +public final class QueryStatisticsCommonConstants {
    --- End diff --
    
    QueryStatisticsConstants used to defined Constants, i write it refer to CarbonCommonConstants,
    i think need to change it to interface


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77120916
  
    --- Diff: core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java ---
    @@ -45,10 +47,28 @@
     
       public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
         super(infos, queryModel);
    +    this.queryModel = queryModel;
    +  }
    +
    +  private Boolean flag;
    +
    +  private Long total = 0L;
    +
    +  private QueryModel queryModel;
    +
    +  @Override public boolean hasNext() {
    +    flag = super.hasNext();
    --- End diff --
    
    in hasnext: i need to judge when  will it traverse the iterator over
    in next: i need to accumulate the total time
    but AbstractDetailQueryResultIterator didn't define 'next' 


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76760660
  
    --- Diff: core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java ---
    @@ -45,10 +47,28 @@
     
       public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
         super(infos, queryModel);
    +    this.queryModel = queryModel;
    +  }
    +
    +  private Boolean flag;
    +
    +  private Long total = 0L;
    +
    +  private QueryModel queryModel;
    +
    +  @Override public boolean hasNext() {
    +    flag = super.hasNext();
    +    if(!flag && total > 0) {
    --- End diff --
    
    why total >0 check is required? 


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77120457
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java ---
    @@ -20,11 +20,14 @@
     
    --- End diff --
    
    handled,  but i can't get the query id, so i print it alone


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77135304
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/SingleQueryStatisticsRecorder.java ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.HashMap;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +
    +/**
    + * Class will be used to record and log the query statistics
    + */
    +public class SingleQueryStatisticsRecorder implements Serializable {
    --- End diff --
    
    Change class name to DriverQueryStatisticsRecorder


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76751950
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java ---
    @@ -61,14 +67,275 @@ public QueryStatisticsRecorder(String queryId) {
        */
       public synchronized void recordStatistics(QueryStatistic statistic) {
         queryStatistics.add(statistic);
    +    // refresh query Statistics Map
    +    String key = statistic.getQueryId();
    +    if (!StringUtils.isEmpty(key)) {
    +      // 240954528274124_0 and 240954528274124 is the same query id
    +      key = key.substring(0, 15);
    +    }
    +    if (queryStatisticsMap.get(key) != null) {
    +      queryStatisticsMap.get(key).add(statistic);
    +    } else {
    +      List<QueryStatistic> newQueryStatistics = new ArrayList<QueryStatistic>();
    +      newQueryStatistics.add(statistic);
    +      queryStatisticsMap.put(key, newQueryStatistics);
    +    }
       }
     
       /**
        * Below method will be used to log the statistic
        */
       public void logStatistics() {
         for (QueryStatistic statistic : queryStatistics) {
    -      LOGGER.statistic(statistic.getStatistics(queryIWthTask));
    +      LOGGER.statistic(statistic.getStatistics());
    +    }
    +  }
    +
    +  /**
    +   * Below method will be used to show statistic log as table
    +   */
    +  public void logStatisticsTable() {
    +    String tableInfo = putStatisticsIntoTable();
    +    if (null != tableInfo) {
    +      LOGGER.statistic(tableInfo);
    +    }
    +  }
    +
    +  /**
    +   * Below method will parse queryStatisticsMap and put time into table
    +   */
    +  public String putStatisticsIntoTable() {
    +    for (String key: queryStatisticsMap.keySet()) {
    +      try {
    +        // TODO: get the finished query, and print Statistics
    +        if (queryStatisticsMap.get(key).size() > 8) {
    +          String jdbc_connection_time = "";
    +          String sql_parse_time = "";
    +          String load_meta_time = "";
    +          String block_identification_time = "";
    +          String schedule_time = "";
    +          String driver_part_time = "";
    +          String executor_part_time = "";
    +          String load_index_time = "";
    --- End diff --
    
    change load_index_time to block loading 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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76760461
  
    --- Diff: core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java ---
    @@ -45,10 +47,28 @@
     
       public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
         super(infos, queryModel);
    +    this.queryModel = queryModel;
    +  }
    +
    +  private Boolean flag;
    +
    +  private Long total = 0L;
    +
    +  private QueryModel queryModel;
    +
    +  @Override public boolean hasNext() {
    +    flag = super.hasNext();
    --- End diff --
    
    why we are overriding hasnext?? We can handle this in super class AbstractDetailQueryResultIterator 


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76751631
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsCommonConstants.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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;
    +
    +public final class QueryStatisticsCommonConstants {
    --- End diff --
    
    Change name to QueryStatisticsConstants


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77120500
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java ---
    @@ -45,8 +47,16 @@
        */
       private long startTime;
     
    -  public QueryStatistic() {
    +  /**
    +   * number of count
    +   */
    +  private long count;
    +
    +  private String queryIdWthTask;
    +
    +  public QueryStatistic(String queryId) {
    --- End diff --
    
    handled


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76752133
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java ---
    @@ -61,14 +67,275 @@ public QueryStatisticsRecorder(String queryId) {
        */
       public synchronized void recordStatistics(QueryStatistic statistic) {
         queryStatistics.add(statistic);
    +    // refresh query Statistics Map
    +    String key = statistic.getQueryId();
    +    if (!StringUtils.isEmpty(key)) {
    +      // 240954528274124_0 and 240954528274124 is the same query id
    +      key = key.substring(0, 15);
    +    }
    +    if (queryStatisticsMap.get(key) != null) {
    +      queryStatisticsMap.get(key).add(statistic);
    +    } else {
    +      List<QueryStatistic> newQueryStatistics = new ArrayList<QueryStatistic>();
    +      newQueryStatistics.add(statistic);
    +      queryStatisticsMap.put(key, newQueryStatistics);
    +    }
       }
     
       /**
        * Below method will be used to log the statistic
        */
       public void logStatistics() {
         for (QueryStatistic statistic : queryStatistics) {
    -      LOGGER.statistic(statistic.getStatistics(queryIWthTask));
    +      LOGGER.statistic(statistic.getStatistics());
    +    }
    +  }
    +
    +  /**
    +   * Below method will be used to show statistic log as table
    +   */
    +  public void logStatisticsTable() {
    +    String tableInfo = putStatisticsIntoTable();
    +    if (null != tableInfo) {
    +      LOGGER.statistic(tableInfo);
    +    }
    +  }
    +
    +  /**
    +   * Below method will parse queryStatisticsMap and put time into table
    +   */
    +  public String putStatisticsIntoTable() {
    +    for (String key: queryStatisticsMap.keySet()) {
    +      try {
    +        // TODO: get the finished query, and print Statistics
    +        if (queryStatisticsMap.get(key).size() > 8) {
    +          String jdbc_connection_time = "";
    +          String sql_parse_time = "";
    +          String load_meta_time = "";
    +          String block_identification_time = "";
    +          String schedule_time = "";
    +          String driver_part_time = "";
    +          String executor_part_time = "";
    +          String load_index_time = "";
    +          String scan_data_time = "";
    +          String dictionary_load_time = "";
    --- End diff --
    
    dictionary loading time required in both the side executor and driver, please add in driver side 


---
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 #91: [CARBONDATA-200] Add performance stat...

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

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


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r77135391
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/SingleQueryStatisticsRecorder.java ---
    @@ -0,0 +1,193 @@
    +/*
    + * 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.HashMap;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +
    +/**
    + * Class will be used to record and log the query statistics
    + */
    +public class SingleQueryStatisticsRecorder implements Serializable {
    --- End diff --
    
    remove Serializable as it not required 


---
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 #91: [WIP] Add performance statistics logs...

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

    https://github.com/apache/incubator-carbondata/pull/91#discussion_r76758327
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java ---
    @@ -61,14 +67,275 @@ public QueryStatisticsRecorder(String queryId) {
        */
       public synchronized void recordStatistics(QueryStatistic statistic) {
         queryStatistics.add(statistic);
    +    // refresh query Statistics Map
    +    String key = statistic.getQueryId();
    +    if (!StringUtils.isEmpty(key)) {
    +      // 240954528274124_0 and 240954528274124 is the same query id
    +      key = key.substring(0, 15);
    +    }
    +    if (queryStatisticsMap.get(key) != null) {
    --- End diff --
    
    query id is not based on segment it is based on task 


---
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.
---