You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/08/09 09:42:36 UTC

[GitHub] [pulsar] codelipenghui commented on a change in pull request #11555: [Offload] Add offload performance metrics for tiered storage.

codelipenghui commented on a change in pull request #11555:
URL: https://github.com/apache/pulsar/pull/11555#discussion_r682726859



##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerOffloaderMXBeanImpl.java
##########
@@ -0,0 +1,269 @@
+/**
+ * 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.bookkeeper.mledger.impl;
+
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.mledger.LedgerOffloaderMXBean;
+import static org.apache.bookkeeper.mledger.util.SafeRun.safeRun;
+import org.apache.bookkeeper.mledger.util.StatsBuckets;
+import org.apache.pulsar.common.stats.Rate;
+
+public class LedgerOffloaderMXBeanImpl implements LedgerOffloaderMXBean {
+
+    public static final long[] READ_ENTRY_LATENCY_BUCKETS_USEC = {500, 1_000, 5_000, 10_000, 20_000, 50_000, 100_000,
+            200_000, 1000_000};
+
+    private final String name;
+
+    private final ScheduledExecutorService scheduler;
+    private long refreshIntervalSeconds;
+
+    // offloadTimeMap record the time cost by one round offload
+    private final ConcurrentHashMap<String, Rate> offloadTimeMap = new ConcurrentHashMap<>();
+    // offloadErrorMap record error ocurred
+    private final ConcurrentHashMap<String, Rate> offloadErrorMap = new ConcurrentHashMap<>();
+    // offloadRateMap record the offload rate
+    private final ConcurrentHashMap<String, Rate> offloadRateMap = new ConcurrentHashMap<>();
+
+
+    // readLedgerLatencyBucketsMap record the time cost by ledger read
+    private final ConcurrentHashMap<String, StatsBuckets> readLedgerLatencyBucketsMap = new ConcurrentHashMap<>();
+    // writeToStorageLatencyBucketsMap record the time cost by write to storage
+    private final ConcurrentHashMap<String, StatsBuckets> writeToStorageLatencyBucketsMap = new ConcurrentHashMap<>();
+    // writeToStorageErrorMap record the error occurred in write storage
+    private final ConcurrentHashMap<String, Rate> writeToStorageErrorMap = new ConcurrentHashMap<>();
+
+
+    // streamingWriteToStorageRateMap and streamingWriteToStorageErrorMap is for streamingOffload
+    private final ConcurrentHashMap<String, Rate> streamingWriteToStorageRateMap = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<String, Rate> streamingWriteToStorageErrorMap = new ConcurrentHashMap<>();
+
+    // readOffloadIndexLatencyBucketsMap and readOffloadDataLatencyBucketsMap are latency metrics about index and data
+    // readOffloadDataRateMap and readOffloadErrorMap is for reading offloaded data
+    private final ConcurrentHashMap<String, StatsBuckets> readOffloadIndexLatencyBucketsMap = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<String, StatsBuckets> readOffloadDataLatencyBucketsMap = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<String, Rate> readOffloadDataRateMap = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<String, Rate> readOffloadErrorMap = new ConcurrentHashMap<>();
+
+    public LedgerOffloaderMXBeanImpl(String name, long refreshIntervalSecond) {
+        this.name = name;
+        this.refreshIntervalSeconds = refreshIntervalSeconds;
+        this.scheduler = Executors.newSingleThreadScheduledExecutor(
+                new DefaultThreadFactory("ledger-offloader-metrics"));
+        this.scheduler.scheduleAtFixedRate(
+                safeRun(() -> refreshStats()), refreshIntervalSeconds, refreshIntervalSeconds, TimeUnit.SECONDS);
+    }
+
+    public void refreshStats() {
+        double seconds = refreshIntervalSeconds;
+
+        if (seconds <= 0.0) {
+            // skip refreshing stats
+            return;
+        }
+        offloadTimeMap.values().forEach(rate->rate.calculateRate(seconds));
+        offloadErrorMap.values().forEach(rate->rate.calculateRate(seconds));
+        offloadRateMap.values().forEach(rate-> rate.calculateRate(seconds));
+        readLedgerLatencyBucketsMap.values().forEach(stat-> stat.refresh());
+        writeToStorageLatencyBucketsMap.values().forEach(stat -> stat.refresh());
+        writeToStorageErrorMap.values().forEach(rate->rate.calculateRate(seconds));
+        streamingWriteToStorageRateMap.values().forEach(rate -> rate.calculateRate(seconds));
+        streamingWriteToStorageErrorMap.values().forEach(rate->rate.calculateRate(seconds));
+        readOffloadDataRateMap.values().forEach(rate->rate.calculateRate(seconds));
+        readOffloadErrorMap.values().forEach(rate->rate.calculateRate(seconds));
+        readOffloadIndexLatencyBucketsMap.values().forEach(stat->stat.refresh());
+        readOffloadDataLatencyBucketsMap.values().forEach(stat->stat.refresh());
+    }
+
+    //TODO metrics在namespace这个level的输出。

Review comment:
       Do we need this line?

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerOffloaderMXBeanImpl.java
##########
@@ -0,0 +1,269 @@
+/**
+ * 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.bookkeeper.mledger.impl;
+
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.mledger.LedgerOffloaderMXBean;
+import static org.apache.bookkeeper.mledger.util.SafeRun.safeRun;
+import org.apache.bookkeeper.mledger.util.StatsBuckets;
+import org.apache.pulsar.common.stats.Rate;
+
+public class LedgerOffloaderMXBeanImpl implements LedgerOffloaderMXBean {
+
+    public static final long[] READ_ENTRY_LATENCY_BUCKETS_USEC = {500, 1_000, 5_000, 10_000, 20_000, 50_000, 100_000,
+            200_000, 1000_000};
+
+    private final String name;
+
+    private final ScheduledExecutorService scheduler;
+    private long refreshIntervalSeconds;
+
+    // offloadTimeMap record the time cost by one round offload
+    private final ConcurrentHashMap<String, Rate> offloadTimeMap = new ConcurrentHashMap<>();

Review comment:
       Why not maintain a MXBean for each offloader?

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java
##########
@@ -1130,9 +1131,43 @@ public void testMetricsProvider() throws IOException {
         BufferedReader reader = new BufferedReader(isReader);
         StringBuffer sb = new StringBuffer();
         String str;
-        while((str = reader.readLine()) != null){
+        while ((str = reader.readLine()) != null) {
             sb.append(str);
         }
+        System.out.println(sb.toString());
+        Assert.assertTrue(sb.toString().contains("test_metrics"));
+    }
+
+    @Test
+    public void testTieredStorageMetrics() throws IOException, PulsarAdminException {
+        final String tenant = "public";
+        final String namespace = "public/default";
+        final String topic = "persistent://" + namespace + "/tiered-storage";
+        Set<String> allowedClusters = new HashSet<>();
+        allowedClusters.add("test");
+        admin.tenants().createTenant(tenant, TenantInfo.builder().allowedClusters(allowedClusters).build());
+        admin.namespaces().createNamespace(namespace, 4);
+        try {
+            Producer<byte[]> producer = pulsarClient.newProducer().topic(topic).create();
+            producer.close();
+        } catch (Exception e) {
+            fail(e.getMessage());
+        }
+        //
+        PrometheusRawMetricsProvider rawMetricsProvider = stream -> stream.write("test_metrics{label1=\"xyz\"} 10 \n");
+        getPulsar().addPrometheusRawMetricsProvider(rawMetricsProvider);
+        HttpClient httpClient = HttpClientBuilder.create().build();
+        final String metricsEndPoint = getPulsar().getWebServiceAddress() + "/metrics";
+        HttpResponse response = httpClient.execute(new HttpGet(metricsEndPoint));
+        InputStream inputStream = response.getEntity().getContent();
+        InputStreamReader isReader = new InputStreamReader(inputStream);
+        BufferedReader reader = new BufferedReader(isReader);
+        StringBuffer sb = new StringBuffer();
+        String str;
+        while ((str = reader.readLine()) != null) {
+            sb.append(str);
+        }
+        System.out.println(sb.toString());

Review comment:
       Replace with log?

##########
File path: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloaderMXBean.java
##########
@@ -0,0 +1,139 @@
+/**
+ * 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.bookkeeper.mledger;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.common.annotation.InterfaceAudience;
+import org.apache.bookkeeper.common.annotation.InterfaceStability;
+import org.apache.bookkeeper.mledger.util.StatsBuckets;
+import org.apache.pulsar.common.stats.Rate;
+
+import java.util.Map;
+import java.util.concurrent.atomic.LongAdder;
+
+/**
+ * Management Bean for a {@link LedgerOffloader}.
+ */
+@InterfaceAudience.LimitedPrivate
+@InterfaceStability.Stable
+public interface LedgerOffloaderMXBean {
+
+    /**
+     * The ledger offloader name.
+     *
+     * @return ledger offloader name.
+     */
+    String getName();

Review comment:
       Does the method for getting the managed ledger name? or the topic name? It's a little confuse when using the offloader name.




-- 
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: commits-unsubscribe@pulsar.apache.org

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