Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* 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
*
* https://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.server.metrics;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import io.micrometer.core.instrument.DistributionSummary;
import io.micrometer.core.instrument.Tag;
import io.micrometer.core.instrument.Tags;
import io.micrometer.core.instrument.distribution.HistogramSnapshot;

/**
* Provides a default DistributionSummary that does not do anything. This can be used to prevent NPE
* if metrics have not been initialized when a DistributionSummary is expected.
* <p>
* Normally DistributionSummaries are created using a builder that takes a registry.
*
* <pre>
* DistributionSummary distSum;
* ...
* public void registerMetrics(MeterRegistry registry) {
* ...
* distSum = DistributionSummary.builder("metric name").description("...").register(registry);
* ...
* }
* </pre>
*
* Until the registration is called, the instance variable is null. If code then tries to update the
* metric, a NPE is thrown. Using this class to provide a default value prevents this from occurring
* and on registration, it is replaced with a valid instance.
*/
public class NoOpDistributionSummary implements DistributionSummary {

private static final Logger LOG = LoggerFactory.getLogger(NoOpDistributionSummary.class);

@Override
public void record(double v) {
LOG.debug("record ignored - distribution summary will not be available.");
}

@Override
public long count() {
return 0;
}

@Override
public double totalAmount() {
return 0;
}

@Override
public double max() {
return 0;
}

@Override
public HistogramSnapshot takeSnapshot() {
return new HistogramSnapshot(0L, 0.0, 0.0, null, null, null);
}

@Override
public Id getId() {
return new Id("thrift.metrics.uninitialized", Tags.of(Tag.of("none", "uninitialized")), null,
"placeholder for uninitialized thrift metrics", Type.OTHER);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@

public class ThriftMetrics implements MetricsProducer {

private DistributionSummary idle;
private DistributionSummary execute;
private DistributionSummary idle = new NoOpDistributionSummary();
private DistributionSummary execute = new NoOpDistributionSummary();

public ThriftMetrics() {}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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
*
* https://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.server.metrics;

import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;

import org.junit.jupiter.api.Test;

class NoOpDistributionSummaryTest {
@Test
public void testNoOp() {
NoOpDistributionSummary noop = new NoOpDistributionSummary();
assertDoesNotThrow(() -> noop.getId());
assertDoesNotThrow(() -> noop.takeSnapshot());
assertDoesNotThrow(() -> noop.max());
assertDoesNotThrow(() -> noop.count());
assertDoesNotThrow(() -> noop.totalAmount());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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
*
* https://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.server.metrics;

import static org.apache.accumulo.core.metrics.MetricsProducer.METRICS_THRIFT_EXECUTE;
import static org.apache.accumulo.core.metrics.MetricsProducer.METRICS_THRIFT_IDLE;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertTrue;

import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import io.micrometer.core.instrument.DistributionSummary;
import io.micrometer.core.instrument.MeterRegistry;
import io.micrometer.core.instrument.simple.SimpleMeterRegistry;

class ThriftMetricsTest {
private final static Logger LOG = LoggerFactory.getLogger(ThriftMetricsTest.class);

@Test
void testNoNPE() {
ThriftMetrics tm = new ThriftMetrics();
assertDoesNotThrow(() -> tm.addIdle(1));
assertDoesNotThrow(() -> tm.addExecute(1));
}

@Test
void registerMetrics() {
MeterRegistry registry = new SimpleMeterRegistry();
ThriftMetrics tm = new ThriftMetrics();
tm.registerMetrics(registry);
tm.addExecute(1000);
tm.addIdle(1000);

registry.forEachMeter(m -> {
LOG.trace("meter: {}", m.getId());
assertInstanceOf(DistributionSummary.class, m);
assertFalse(m instanceof NoOpDistributionSummary);
});
assertTrue(registry.get(METRICS_THRIFT_IDLE).summary().count() > 0);
assertTrue(registry.get(METRICS_THRIFT_EXECUTE).summary().count() > 0);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.util.concurrent.atomic.LongAdder;

import org.apache.accumulo.core.metrics.MetricsProducer;
import org.apache.accumulo.server.metrics.NoOpDistributionSummary;

import io.micrometer.core.instrument.Counter;
import io.micrometer.core.instrument.DistributionSummary;
Expand All @@ -34,8 +35,8 @@ public class TabletServerScanMetrics implements MetricsProducer {

private final AtomicInteger openFiles = new AtomicInteger(0);
private Timer scans;
private DistributionSummary resultsPerScan;
private DistributionSummary yields;
private DistributionSummary resultsPerScan = new NoOpDistributionSummary();
private DistributionSummary yields = new NoOpDistributionSummary();
private Counter startScanCalls;
private Counter continueScanCalls;
private Counter closeScanCalls;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.time.Duration;

import org.apache.accumulo.core.metrics.MetricsProducer;
import org.apache.accumulo.server.metrics.NoOpDistributionSummary;

import io.micrometer.core.instrument.Counter;
import io.micrometer.core.instrument.DistributionSummary;
Expand All @@ -35,7 +36,7 @@ public class TabletServerUpdateMetrics implements MetricsProducer {
private Timer commitPrepStat;
private Timer walogWriteTimeStat;
private Timer commitTimeStat;
private DistributionSummary mutationArraySizeStat;
private DistributionSummary mutationArraySizeStat = new NoOpDistributionSummary();

public void addPermissionErrors(long value) {
permissionErrorsCounter.increment(value);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
import org.apache.accumulo.core.master.thrift.TabletServerStatus;
import org.apache.accumulo.core.metrics.MetricsInfo;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
Expand Down Expand Up @@ -135,6 +136,10 @@ public static void main(String[] args) throws Exception {

ServiceLock zlock = new ServiceLock(zoo.getZooKeeper(), zLockPath, UUID.randomUUID());

MetricsInfo metricsInfo = context.getMetricsInfo();
metricsInfo.addServiceTags("zombie.server", serverPort.address);
metricsInfo.init();

LockWatcher lw = new LockWatcher() {

@SuppressFBWarnings(value = "DM_EXIT",
Expand Down