HIVE-16215 : counter recording for text cache may not fully work (Sergey Shelukhin, reviewed by Prasanth Jayachandran)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a086d6c7 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a086d6c7 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a086d6c7 Branch: refs/heads/hive-14535 Commit: a086d6c7c072589dd58ee6a696c9d03c9486d24b Parents: 7f05f0c Author: Sergey Shelukhin <[email protected]> Authored: Wed Mar 15 11:35:17 2017 -0700 Committer: Sergey Shelukhin <[email protected]> Committed: Wed Mar 15 11:35:17 2017 -0700 ---------------------------------------------------------------------- .../daemon/impl/StatsRecordingThreadPool.java | 8 +++---- .../llap/io/encoded/OrcEncodedDataReader.java | 4 ++-- .../llap/io/encoded/SerDeEncodedDataReader.java | 3 ++- .../hive/llap/io/encoded/TezCounterSource.java | 24 ++++++++++++++++++++ 4 files changed, 32 insertions(+), 7 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hive/blob/a086d6c7/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StatsRecordingThreadPool.java ---------------------------------------------------------------------- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StatsRecordingThreadPool.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StatsRecordingThreadPool.java index 363b9b1..e850135 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StatsRecordingThreadPool.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StatsRecordingThreadPool.java @@ -31,7 +31,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.llap.LlapUtil; -import org.apache.hadoop.hive.llap.io.encoded.OrcEncodedDataReader; +import org.apache.hadoop.hive.llap.io.encoded.TezCounterSource; import org.apache.log4j.MDC; import org.apache.log4j.NDC; import org.apache.tez.common.CallableWithNdc; @@ -157,9 +157,9 @@ public class StatsRecordingThreadPool extends ThreadPoolExecutor { TaskRunner2Callable taskRunner2Callable = (TaskRunner2Callable) actualCallable; // counters for task execution side tezCounters = taskRunner2Callable.addAndGetTezCounter(FileSystemCounter.class.getName()); - } else if (actualCallable instanceof OrcEncodedDataReader) { - // counters for llap io side - tezCounters = ((OrcEncodedDataReader) actualCallable).getTezCounters(); + } else if (actualCallable instanceof TezCounterSource) { + // Other counter sources (currently used in LLAP IO). + tezCounters = ((TezCounterSource) actualCallable).getTezCounters(); } if (tezCounters != null) { http://git-wip-us.apache.org/repos/asf/hive/blob/a086d6c7/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java ---------------------------------------------------------------------- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java index 6554fa2..ce0ffe6 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java @@ -26,7 +26,6 @@ import java.util.Collections; import java.util.List; import org.apache.hadoop.hive.llap.counters.LlapIOCounters; -import org.apache.orc.OrcUtils; import org.apache.orc.TypeDescription; import org.apache.orc.impl.DataReaderProperties; import org.apache.orc.impl.OrcIndex; @@ -92,7 +91,7 @@ import org.apache.tez.common.CallableWithNdc; * consumer. It also serves as ConsumerFeedback that receives processed EncodedColumnBatch-es. */ public class OrcEncodedDataReader extends CallableWithNdc<Void> - implements ConsumerFeedback<OrcEncodedColumnBatch> { + implements ConsumerFeedback<OrcEncodedColumnBatch>, TezCounterSource { private static final Logger LOG = LoggerFactory.getLogger(OrcEncodedDataReader.class); public static final FixedSizedObjectPool<ColumnStreamData> CSD_POOL = new FixedSizedObjectPool<>(8192, new PoolObjectHelper<ColumnStreamData>() { @@ -935,6 +934,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void> } } + @Override public TezCounters getTezCounters() { return counters.getTezCounters(); } http://git-wip-us.apache.org/repos/asf/hive/blob/a086d6c7/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java ---------------------------------------------------------------------- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java index 221c99e..6aab6de 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/SerDeEncodedDataReader.java @@ -100,7 +100,7 @@ import org.apache.tez.common.counters.TezCounters; import com.google.common.collect.Lists; public class SerDeEncodedDataReader extends CallableWithNdc<Void> - implements ConsumerFeedback<OrcEncodedColumnBatch> { + implements ConsumerFeedback<OrcEncodedColumnBatch>, TezCounterSource { public static final FixedSizedObjectPool<ColumnStreamData> CSD_POOL = new FixedSizedObjectPool<>(8192, new PoolObjectHelper<ColumnStreamData>() { @@ -1657,6 +1657,7 @@ public class SerDeEncodedDataReader extends CallableWithNdc<Void> ECB_POOL.offer(ecb); } + @Override public TezCounters getTezCounters() { return counters.getTezCounters(); } http://git-wip-us.apache.org/repos/asf/hive/blob/a086d6c7/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/TezCounterSource.java ---------------------------------------------------------------------- diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/TezCounterSource.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/TezCounterSource.java new file mode 100644 index 0000000..717c169 --- /dev/null +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/TezCounterSource.java @@ -0,0 +1,24 @@ +/** + * 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.hadoop.hive.llap.io.encoded; + +import org.apache.tez.common.counters.TezCounters; + +public interface TezCounterSource { + TezCounters getTezCounters(); +} \ No newline at end of file
