You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@orc.apache.org by GitBox <gi...@apache.org> on 2022/06/13 04:33:28 UTC

[GitHub] [orc] wgtmac commented on a diff in pull request #1158: ORC-961: [C++] expose related metrics of the reader

wgtmac commented on code in PR #1158:
URL: https://github.com/apache/orc/pull/1158#discussion_r895301590


##########
c++/include/orc/Reader.hh:
##########
@@ -39,6 +39,29 @@ namespace orc {
   struct ReaderOptionsPrivate;
   struct RowReaderOptionsPrivate;
 
+  struct ReaderMetrics {
+    volatile uint64_t ReaderCount;

Review Comment:
   Use std::atomic_uint64_t to replace volatile keyword as it does not guarantee atomicity.



##########
c++/include/orc/Reader.hh:
##########
@@ -455,6 +478,12 @@ namespace orc {
      */
     virtual bool hasCorrectStatistics() const = 0;
 
+    /**
+     * Get metrics of the reader
+     * @return the accumulated reader metrics to current state.
+     */
+    virtual ReaderMetrics getReaderMetrics() const = 0;

Review Comment:
   const ReaderMetrics& ?



##########
tools/src/ToolsHelper.cc:
##########
@@ -80,10 +83,25 @@ bool parseOptions(int* argc, char** argv[], uint64_t* batchSize, orc::RowReaderO
         }
         break;
       }
+      case 'm' : {
+        *showMetrics = true;
+        break;
+      }
       default: break;
     }
   } while (opt != -1);
   *argc -= optind;
   *argv += optind;
   return true;
 }
+
+void printReaderMetrics(std::ostream& out, const orc::ReaderMetrics& metrics) {
+  out << "ReaderCount: " << metrics.ReaderCount << std::endl;
+  out << "ElapsedTimeUs: " << metrics.ReaderInclusiveLatencyUs << std::endl;

Review Comment:
   nit: user may expect to read time in seconds which is much easier to interpret.



##########
c++/include/orc/Reader.hh:
##########
@@ -39,6 +39,29 @@ namespace orc {
   struct ReaderOptionsPrivate;
   struct RowReaderOptionsPrivate;
 
+  struct ReaderMetrics {

Review Comment:
   Please add comment to help understanding.



##########
c++/include/orc/Reader.hh:
##########
@@ -39,6 +39,29 @@ namespace orc {
   struct ReaderOptionsPrivate;
   struct RowReaderOptionsPrivate;
 
+  struct ReaderMetrics {
+    volatile uint64_t ReaderCount;
+    volatile uint64_t ReaderInclusiveLatencyUs;
+    volatile uint64_t DecompCount;
+    volatile uint64_t DecompLatencyUs;
+    volatile uint64_t DecodingCount;
+    volatile uint64_t DecodingLatencyUs;
+    volatile uint64_t ByteDecodingCount;
+    volatile uint64_t ByteDecodingLatencyUs;
+
+    ReaderMetrics() {
+      ReaderCount = 0;
+      ReaderInclusiveLatencyUs = 0;
+      DecompCount = 0;
+      DecompLatencyUs = 0;
+      DecodingCount = 0;
+      DecodingLatencyUs = 0;
+      ByteDecodingCount = 0;
+      ByteDecodingLatencyUs = 0;

Review Comment:
   Can we support blocking io latency as well?



##########
c++/src/Reader.cc:
##########
@@ -1154,6 +1165,8 @@ namespace orc {
   }
 
   bool RowReaderImpl::next(ColumnVectorBatch& data) {
+    AutoStopwatch measure(&contents->readerMetrics->ReaderInclusiveLatencyUs,
+                  &contents->readerMetrics->ReaderCount);

Review Comment:
   Fix alignment



##########
c++/include/orc/Reader.hh:
##########
@@ -39,6 +39,29 @@ namespace orc {
   struct ReaderOptionsPrivate;
   struct RowReaderOptionsPrivate;
 
+  struct ReaderMetrics {
+    volatile uint64_t ReaderCount;
+    volatile uint64_t ReaderInclusiveLatencyUs;
+    volatile uint64_t DecompCount;
+    volatile uint64_t DecompLatencyUs;
+    volatile uint64_t DecodingCount;
+    volatile uint64_t DecodingLatencyUs;
+    volatile uint64_t ByteDecodingCount;
+    volatile uint64_t ByteDecodingLatencyUs;
+
+    ReaderMetrics() {
+      ReaderCount = 0;

Review Comment:
   nit: provide the default value in the declaration so we can avoid explicit constructor here. 



##########
c++/src/Utils.hh:
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_UTILS_HH
+#define ORC_UTILS_HH
+
+#include <stdint.h>
+#include <time.h>
+#include <sys/time.h>
+
+namespace orc {
+
+static inline uint64_t getCurrentTimeUs() {
+  timespec ts;
+  clock_gettime(CLOCK_MONOTONIC, &ts);
+  return static_cast<uint64_t>(ts.tv_sec) * 1000000ULL +
+         static_cast<uint64_t>(ts.tv_nsec) / 1000;
+}
+
+class AutoStopwatch {
+    uint64_t start;
+    volatile uint64_t* latencyUs;
+    volatile uint64_t* count;
+
+public:
+    AutoStopwatch(volatile uint64_t* latency,
+                  volatile uint64_t* cnt) {
+        start = getCurrentTimeUs();
+        latencyUs = latency;
+        count = cnt;
+    }
+
+    ~AutoStopwatch() {
+        if (latencyUs) {
+            uint64_t elapsedTime = getCurrentTimeUs() - start;
+             __sync_fetch_and_add(latencyUs, elapsedTime);

Review Comment:
   Windows does not provide this API, please use std::atomic instead.



##########
c++/src/Utils.hh:
##########
@@ -0,0 +1,61 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_UTILS_HH
+#define ORC_UTILS_HH
+
+#include <stdint.h>
+#include <time.h>
+#include <sys/time.h>

Review Comment:
   Windows may fail with this inclusion. You may want to use std chrono library. See : https://en.cppreference.com/w/cpp/chrono/high_resolution_clock/now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org