Repository: accumulo Updated Branches: refs/heads/master 856496138 -> 2821a4815
http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsUtil.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsUtil.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsUtil.java new file mode 100644 index 0000000..5905aea --- /dev/null +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMetricsUtil.java @@ -0,0 +1,138 @@ +/* + * 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.accumulo.tserver.metrics; + +import org.apache.accumulo.tserver.TabletServer; +import org.apache.accumulo.tserver.tablet.Tablet; + +/** + * Wrapper around extracting metrics from a TabletServer instance + * + * Necessary to support both old custom JMX metrics and Hadoop Metrics2 + */ +public class TabletServerMetricsUtil { + + private final TabletServer tserver; + + public TabletServerMetricsUtil(TabletServer tserver) { + this.tserver = tserver; + } + + public long getEntries() { + long result = 0; + for (Tablet tablet : tserver.getOnlineTablets()) { + result += tablet.getNumEntries(); + } + return result; + } + + public long getEntriesInMemory() { + long result = 0; + for (Tablet tablet : tserver.getOnlineTablets()) { + result += tablet.getNumEntriesInMemory(); + } + return result; + } + + public long getIngest() { + long result = 0; + for (Tablet tablet : tserver.getOnlineTablets()) { + result += tablet.getNumEntriesInMemory(); + } + return result; + } + + public int getMajorCompactions() { + int result = 0; + for (Tablet tablet : tserver.getOnlineTablets()) { + if (tablet.isMajorCompactionRunning()) + result++; + } + return result; + } + + public int getMajorCompactionsQueued() { + int result = 0; + for (Tablet tablet : tserver.getOnlineTablets()) { + if (tablet.isMajorCompactionQueued()) + result++; + } + return result; + } + + public int getMinorCompactions() { + int result = 0; + for (Tablet tablet : tserver.getOnlineTablets()) { + if (tablet.isMinorCompactionRunning()) + result++; + } + return result; + } + + public int getMinorCompactionsQueued() { + int result = 0; + for (Tablet tablet : tserver.getOnlineTablets()) { + if (tablet.isMinorCompactionQueued()) + result++; + } + return result; + } + + public int getOnlineCount() { + return tserver.getOnlineTablets().size(); + } + + public int getOpeningCount() { + return tserver.getOpeningCount(); + } + + public long getQueries() { + long result = 0; + for (Tablet tablet : tserver.getOnlineTablets()) { + result += tablet.totalQueries(); + } + return result; + } + + public int getUnopenedCount() { + return tserver.getUnopenedCount(); + } + + public String getName() { + return tserver.getClientAddressString(); + } + + public long getTotalMinorCompactions() { + return tserver.getTotalMinorCompactions(); + } + + public double getHoldTime() { + return tserver.getHoldTimeMillis() / 1000.; + } + + public double getAverageFilesPerTablet() { + int count = 0; + long result = 0; + for (Tablet tablet : tserver.getOnlineTablets()) { + result += tablet.getDatafiles().size(); + count++; + } + if (count == 0) + return 0; + return result / (double) count; + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java index a89bb45..f4d61ca 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetrics.java @@ -21,14 +21,14 @@ import javax.management.ObjectName; import org.apache.accumulo.server.metrics.AbstractMetricsImpl; public class TabletServerMinCMetrics extends AbstractMetricsImpl implements TabletServerMinCMetricsMBean { - + static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(TabletServerMinCMetrics.class); - + private static final String METRICS_PREFIX = "tserver.minc"; - + private static ObjectName OBJECT_NAME = null; - - public TabletServerMinCMetrics() { + + TabletServerMinCMetrics() { super(); reset(); try { @@ -37,52 +37,61 @@ public class TabletServerMinCMetrics extends AbstractMetricsImpl implements Tabl log.error("Exception setting MBean object name", e); } } - + @Override protected ObjectName getObjectName() { return OBJECT_NAME; } - + @Override protected String getMetricsPrefix() { return METRICS_PREFIX; } - + + @Override public long getMinorCompactionMinTime() { - return this.getMetricMin(minc); + return this.getMetricMin(MINC); } - + + @Override public long getMinorCompactionAvgTime() { - return this.getMetricAvg(minc); + return this.getMetricAvg(MINC); } - + + @Override public long getMinorCompactionCount() { - return this.getMetricCount(minc); + return this.getMetricCount(MINC); } - + + @Override public long getMinorCompactionMaxTime() { - return this.getMetricMax(minc); + return this.getMetricMax(MINC); } - + + @Override public long getMinorCompactionQueueAvgTime() { - return this.getMetricAvg(queue); + return this.getMetricAvg(QUEUE); } - + + @Override public long getMinorCompactionQueueCount() { - return this.getMetricCount(queue); + return this.getMetricCount(QUEUE); } - + + @Override public long getMinorCompactionQueueMaxTime() { - return this.getMetricMax(queue); + return this.getMetricMax(QUEUE); } - + + @Override public long getMinorCompactionQueueMinTime() { - return this.getMetricMin(minc); + return this.getMetricMin(MINC); } - + + @Override public void reset() { createMetric("minc"); createMetric("queue"); } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsKeys.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsKeys.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsKeys.java new file mode 100644 index 0000000..29d4846 --- /dev/null +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsKeys.java @@ -0,0 +1,27 @@ +/* + * 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.accumulo.tserver.metrics; + +/** + * Keys to reference minor compaction metrics + */ +public interface TabletServerMinCMetricsKeys { + + String MINC = "minc"; + String QUEUE = "queue"; + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java index 0dc6cf3..2af820c 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerMinCMetricsMBean.java @@ -16,27 +16,24 @@ */ package org.apache.accumulo.tserver.metrics; -public interface TabletServerMinCMetricsMBean { - - static final String minc = "minc"; - static final String queue = "queue"; - +public interface TabletServerMinCMetricsMBean extends TabletServerMinCMetricsKeys { + long getMinorCompactionCount(); - + long getMinorCompactionAvgTime(); - + long getMinorCompactionMinTime(); - + long getMinorCompactionMaxTime(); - + long getMinorCompactionQueueCount(); - + long getMinorCompactionQueueAvgTime(); - + long getMinorCompactionQueueMinTime(); - + long getMinorCompactionQueueMaxTime(); - + void reset(); - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java index 142f171..6bcabfd 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetrics.java @@ -21,14 +21,14 @@ import javax.management.ObjectName; import org.apache.accumulo.server.metrics.AbstractMetricsImpl; public class TabletServerScanMetrics extends AbstractMetricsImpl implements TabletServerScanMetricsMBean { - + static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(TabletServerScanMetrics.class); - + public static final String METRICS_PREFIX = "tserver.scan"; - + static ObjectName OBJECT_NAME = null; - - public TabletServerScanMetrics() { + + TabletServerScanMetrics() { super(); reset(); try { @@ -37,52 +37,61 @@ public class TabletServerScanMetrics extends AbstractMetricsImpl implements Tabl log.error("Exception setting MBean object name", e); } } - + @Override protected ObjectName getObjectName() { return OBJECT_NAME; } - + @Override protected String getMetricsPrefix() { return METRICS_PREFIX; } - + + @Override public long getResultAvgSize() { - return this.getMetricAvg(resultSize); + return this.getMetricAvg(RESULT_SIZE); } - + + @Override public long getResultCount() { - return this.getMetricCount(resultSize); + return this.getMetricCount(RESULT_SIZE); } - + + @Override public long getResultMaxSize() { - return this.getMetricMax(resultSize); + return this.getMetricMax(RESULT_SIZE); } - + + @Override public long getResultMinSize() { - return this.getMetricMin(resultSize); + return this.getMetricMin(RESULT_SIZE); } - + + @Override public long getScanAvgTime() { - return this.getMetricAvg(scan); + return this.getMetricAvg(SCAN); } - + + @Override public long getScanCount() { - return this.getMetricCount(scan); + return this.getMetricCount(SCAN); } - + + @Override public long getScanMaxTime() { - return this.getMetricMax(scan); + return this.getMetricMax(SCAN); } - + + @Override public long getScanMinTime() { - return this.getMetricMin(scan); + return this.getMetricMin(SCAN); } - + + @Override public void reset() { - createMetric(scan); - createMetric(resultSize); + createMetric(SCAN); + createMetric(RESULT_SIZE); } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsKeys.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsKeys.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsKeys.java new file mode 100644 index 0000000..7d33a84 --- /dev/null +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsKeys.java @@ -0,0 +1,27 @@ +/* + * 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.accumulo.tserver.metrics; + +/** + * Keys for referencing scan metrics + */ +public interface TabletServerScanMetricsKeys { + + String SCAN = "scan"; + String RESULT_SIZE = "result"; + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java index b532cfa..e262fc4 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerScanMetricsMBean.java @@ -16,27 +16,24 @@ */ package org.apache.accumulo.tserver.metrics; -public interface TabletServerScanMetricsMBean { - - static final String scan = "scan"; - static final String resultSize = "result"; - +public interface TabletServerScanMetricsMBean extends TabletServerScanMetricsKeys { + long getScanCount(); - + long getScanAvgTime(); - + long getScanMinTime(); - + long getScanMaxTime(); - + long getResultCount(); - + long getResultAvgSize(); - + long getResultMinSize(); - + long getResultMaxSize(); - + void reset(); - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java index b23fb1a..48b78b0 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetrics.java @@ -21,14 +21,14 @@ import javax.management.ObjectName; import org.apache.accumulo.server.metrics.AbstractMetricsImpl; public class TabletServerUpdateMetrics extends AbstractMetricsImpl implements TabletServerUpdateMetricsMBean { - + static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(TabletServerUpdateMetrics.class); - + private static final String METRICS_PREFIX = "tserver.update"; - + private static ObjectName OBJECT_NAME = null; - - public TabletServerUpdateMetrics() { + + TabletServerUpdateMetrics() { super(); reset(); try { @@ -38,97 +38,116 @@ public class TabletServerUpdateMetrics extends AbstractMetricsImpl implements Ta log.error("Exception setting MBean object name", e); } } - + @Override protected ObjectName getObjectName() { return OBJECT_NAME; } - + @Override protected String getMetricsPrefix() { return METRICS_PREFIX; } - + + @Override public long getPermissionErrorCount() { - return this.getMetricCount(permissionErrors); + return this.getMetricCount(PERMISSION_ERRORS); } - + + @Override public long getUnknownTabletErrorCount() { - return this.getMetricCount(unknownTabletErrors); + return this.getMetricCount(UNKNOWN_TABLET_ERRORS); } - + + @Override public long getMutationArrayAvgSize() { - return this.getMetricAvg(mutationArraySize); + return this.getMetricAvg(MUTATION_ARRAY_SIZE); } - + + @Override public long getMutationArrayMinSize() { - return this.getMetricMin(mutationArraySize); + return this.getMetricMin(MUTATION_ARRAY_SIZE); } - + + @Override public long getMutationArrayMaxSize() { - return this.getMetricMax(mutationArraySize); + return this.getMetricMax(MUTATION_ARRAY_SIZE); } - + + @Override public long getCommitPrepCount() { - return this.getMetricCount(commitPrep); + return this.getMetricCount(COMMIT_PREP); } - + + @Override public long getCommitPrepMinTime() { - return this.getMetricMin(commitPrep); + return this.getMetricMin(COMMIT_PREP); } - + + @Override public long getCommitPrepMaxTime() { - return this.getMetricMax(commitPrep); + return this.getMetricMax(COMMIT_PREP); } - + + @Override public long getCommitPrepAvgTime() { - return this.getMetricAvg(commitPrep); + return this.getMetricAvg(COMMIT_PREP); } - + + @Override public long getConstraintViolationCount() { - return this.getMetricCount(constraintViolations); + return this.getMetricCount(CONSTRAINT_VIOLATIONS); } - + + @Override public long getWALogWriteCount() { - return this.getMetricCount(waLogWriteTime); + return this.getMetricCount(WALOG_WRITE_TIME); } - + + @Override public long getWALogWriteMinTime() { - return this.getMetricMin(waLogWriteTime); + return this.getMetricMin(WALOG_WRITE_TIME); } - + + @Override public long getWALogWriteMaxTime() { - return this.getMetricMax(waLogWriteTime); + return this.getMetricMax(WALOG_WRITE_TIME); } - + + @Override public long getWALogWriteAvgTime() { - return this.getMetricAvg(waLogWriteTime); + return this.getMetricAvg(WALOG_WRITE_TIME); } - + + @Override public long getCommitCount() { - return this.getMetricCount(commitTime); + return this.getMetricCount(COMMIT_TIME); } - + + @Override public long getCommitMinTime() { - return this.getMetricMin(commitTime); + return this.getMetricMin(COMMIT_TIME); } - + + @Override public long getCommitMaxTime() { - return this.getMetricMax(commitTime); + return this.getMetricMax(COMMIT_TIME); } - + + @Override public long getCommitAvgTime() { - return this.getMetricAvg(commitTime); + return this.getMetricAvg(COMMIT_TIME); } - + + @Override public void reset() { - createMetric(permissionErrors); - createMetric(unknownTabletErrors); - createMetric(mutationArraySize); - createMetric(commitPrep); - createMetric(constraintViolations); - createMetric(waLogWriteTime); - createMetric(commitTime); - } - + createMetric(PERMISSION_ERRORS); + createMetric(UNKNOWN_TABLET_ERRORS); + createMetric(MUTATION_ARRAY_SIZE); + createMetric(COMMIT_PREP); + createMetric(CONSTRAINT_VIOLATIONS); + createMetric(WALOG_WRITE_TIME); + createMetric(COMMIT_TIME); + } + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsKeys.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsKeys.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsKeys.java new file mode 100644 index 0000000..f06751b --- /dev/null +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsKeys.java @@ -0,0 +1,32 @@ +/* + * 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.accumulo.tserver.metrics; + +/** + * Keys to identify which update metric is being altered + */ +public interface TabletServerUpdateMetricsKeys { + + static String PERMISSION_ERRORS = "permissionErrors"; + static String UNKNOWN_TABLET_ERRORS = "unknownTabletErrors"; + static String MUTATION_ARRAY_SIZE = "mutationArraysSize"; + static String COMMIT_PREP = "commitPrep"; + static String CONSTRAINT_VIOLATIONS = "constraintViolations"; + static String WALOG_WRITE_TIME = "waLogWriteTime"; + static String COMMIT_TIME = "commitTime"; + +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java index 0292947..867f2f7 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/TabletServerUpdateMetricsMBean.java @@ -16,51 +16,43 @@ */ package org.apache.accumulo.tserver.metrics; -public interface TabletServerUpdateMetricsMBean { - - final static String permissionErrors = "permissionErrors"; - final static String unknownTabletErrors = "unknownTabletErrors"; - final static String mutationArraySize = "mutationArraysSize"; - final static String commitPrep = "commitPrep"; - final static String constraintViolations = "constraintViolations"; - final static String waLogWriteTime = "waLogWriteTime"; - final static String commitTime = "commitTime"; - +public interface TabletServerUpdateMetricsMBean extends TabletServerUpdateMetricsKeys { + long getPermissionErrorCount(); - + long getUnknownTabletErrorCount(); - + long getMutationArrayAvgSize(); - + long getMutationArrayMinSize(); - + long getMutationArrayMaxSize(); - + long getCommitPrepCount(); - + long getCommitPrepMinTime(); - + long getCommitPrepMaxTime(); - + long getCommitPrepAvgTime(); - + long getConstraintViolationCount(); - + long getWALogWriteCount(); - + long getWALogWriteMinTime(); - + long getWALogWriteMaxTime(); - + long getWALogWriteAvgTime(); - + long getCommitCount(); - + long getCommitMinTime(); - + long getCommitMaxTime(); - + long getCommitAvgTime(); - + void reset(); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java index 82e2be2..c0c8b53 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java @@ -98,6 +98,7 @@ import org.apache.accumulo.server.fs.VolumeUtil; import org.apache.accumulo.server.fs.VolumeUtil.TabletFiles; import org.apache.accumulo.server.master.state.TServerInstance; import org.apache.accumulo.server.master.tableOps.UserCompactionConfig; +import org.apache.accumulo.server.metrics.Metrics; import org.apache.accumulo.server.problems.ProblemReport; import org.apache.accumulo.server.problems.ProblemReports; import org.apache.accumulo.server.problems.ProblemType; @@ -972,13 +973,13 @@ public class Tablet implements TabletCommitter { if (!failed) { lastMinorCompactionFinishTime = System.currentTimeMillis(); } - TabletServerMinCMetrics minCMetrics = getTabletServer().getMinCMetrics(); + Metrics minCMetrics = getTabletServer().getMinCMetrics(); if (minCMetrics.isEnabled()) - minCMetrics.add(TabletServerMinCMetrics.minc, (lastMinorCompactionFinishTime - start)); + minCMetrics.add(TabletServerMinCMetrics.MINC, (lastMinorCompactionFinishTime - start)); if (hasQueueTime) { timer.updateTime(Operation.MINOR, queued, start, count, failed); if (minCMetrics.isEnabled()) - minCMetrics.add(TabletServerMinCMetrics.queue, (start - queued)); + minCMetrics.add(TabletServerMinCMetrics.QUEUE, (start - queued)); } else timer.updateTime(Operation.MINOR, start, count, failed); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java index ef18efd..4094c5f 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java @@ -102,7 +102,7 @@ public class ZombieTServer { TransactionWatcher watcher = new TransactionWatcher(); final ThriftClientHandler tch = new ThriftClientHandler(context, watcher); Processor<Iface> processor = new Processor<Iface>(tch); - ServerAddress serverPort = TServerUtils.startTServer(HostAndPort.fromParts("0.0.0.0", port), processor, "ZombieTServer", "walking dead", 2, 1, 1000, + ServerAddress serverPort = TServerUtils.startTServer(context.getConfiguration(), HostAndPort.fromParts("0.0.0.0", port), processor, "ZombieTServer", "walking dead", 2, 1, 1000, 10 * 1024 * 1024, null, -1); String addressString = serverPort.address.toString(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/2821a481/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java ---------------------------------------------------------------------- diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java index ae07925..671ead6 100644 --- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java +++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java @@ -247,17 +247,17 @@ public class NullTserver { Opts opts = new Opts(); opts.parseArgs(NullTserver.class.getName(), args); + // modify metadata + ZooKeeperInstance zki = new ZooKeeperInstance(new ClientConfiguration().withInstance(opts.iname).withZkHosts(opts.keepers)); + AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(zki)); + TransactionWatcher watcher = new TransactionWatcher(); ThriftClientHandler tch = new ThriftClientHandler(new AccumuloServerContext(new ServerConfigurationFactory(HdfsZooInstance.getInstance())), watcher); Processor<Iface> processor = new Processor<Iface>(tch); - TServerUtils.startTServer(HostAndPort.fromParts("0.0.0.0", opts.port), processor, "NullTServer", "null tserver", 2, 1, 1000, 10 * 1024 * 1024, null, -1); + TServerUtils.startTServer(context.getConfiguration(), HostAndPort.fromParts("0.0.0.0", opts.port), processor, "NullTServer", "null tserver", 2, 1, 1000, 10 * 1024 * 1024, null, -1); HostAndPort addr = HostAndPort.fromParts(InetAddress.getLocalHost().getHostName(), opts.port); - // modify metadata - ZooKeeperInstance zki = new ZooKeeperInstance(new ClientConfiguration().withInstance(opts.iname).withZkHosts(opts.keepers)); - AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(zki)); - String tableId = Tables.getTableId(zki, opts.tableName); // read the locations for the table
