Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add support for tracking Hadoop and GCS API level metrics at a thread… #1299

Open
wants to merge 2 commits into
base: branch-3.0.x
Choose a base branch
from
Open
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
Expand Up @@ -78,6 +78,9 @@ public class GhfsGlobalStorageStatistics extends StorageStatistics {
// Initial requests are expected to take time due to warmup.
private static final int WARMUP_THRESHOLD_SEC = 30;

private static final GhfsThreadLocalStatistics threadLocalStatistics =
new GhfsThreadLocalStatistics();

private final Map<String, AtomicLong> opsCount = new HashMap<>();
private final Map<String, AtomicLong> minimums = new HashMap<>();
private final Map<String, AtomicLong> maximums = new HashMap<>();
Expand Down Expand Up @@ -124,6 +127,10 @@ static <B> B trackDuration(
}
}

public GhfsThreadLocalStatistics getThreadLocalStatistics() {
return threadLocalStatistics;
}

private String getNonZeroMetrics() {
// TreeMap to keep the result sorted.
TreeMap<String, Long> result = new TreeMap<>();
Expand Down Expand Up @@ -159,6 +166,7 @@ void increment(GoogleCloudStorageStatistics statistic) {
* @return the new value
*/
long incrementCounter(GhfsStatistic op, long count) {
threadLocalStatistics.increment(op, count);
return opsCount.get(op.getSymbol()).addAndGet(count);
}

Expand All @@ -170,6 +178,7 @@ long incrementCounter(GhfsStatistic op, long count) {
*/
void incrementCounter(GoogleCloudStorageStatistics op, long count) {
opsCount.get(op.getSymbol()).addAndGet(count);
threadLocalStatistics.increment(op, count);
}

@Override
Expand Down Expand Up @@ -245,7 +254,7 @@ void updateStats(
String symbol = statistic.getSymbol();
updateMinMaxStats(minLatency, maxLatency, context, symbol);
addMeanStatistic(statistic.getSymbol(), totalDuration, count);
opsCount.get(symbol).addAndGet(count);
incrementCounter(statistic, count);

updateConnectorHadoopApiTime(totalDuration);
}
Expand Down
56 changes: 41 additions & 15 deletions gcs/src/main/java/com/google/cloud/hadoop/fs/gcs/GhfsStatistic.java
Original file line number Diff line number Diff line change
Expand Up @@ -66,33 +66,48 @@ public enum GhfsStatistic {
"files_delete_rejected",
"Total number of files whose delete request was rejected",
TYPE_COUNTER),
INVOCATION_CREATE(StoreStatisticNames.OP_CREATE, "Calls of create()", TYPE_DURATION_TOTAL),
INVOCATION_DELETE(StoreStatisticNames.OP_DELETE, "Calls of delete()", TYPE_DURATION_TOTAL),
INVOCATION_EXISTS(StoreStatisticNames.OP_EXISTS, "Calls of exists()", TYPE_COUNTER),
INVOCATION_CREATE(StoreStatisticNames.OP_CREATE, "Calls of create()", TYPE_DURATION_TOTAL, true),
INVOCATION_DELETE(StoreStatisticNames.OP_DELETE, "Calls of delete()", TYPE_DURATION_TOTAL, true),
INVOCATION_EXISTS(StoreStatisticNames.OP_EXISTS, "Calls of exists()", TYPE_COUNTER, true),
INVOCATION_GET_FILE_STATUS(
StoreStatisticNames.OP_GET_FILE_STATUS, "Calls of getFileStatus()", TYPE_DURATION_TOTAL),
StoreStatisticNames.OP_GET_FILE_STATUS,
"Calls of getFileStatus()",
TYPE_DURATION_TOTAL,
true),
INVOCATION_GET_FILE_CHECKSUM(
StoreStatisticNames.OP_GET_FILE_CHECKSUM, "Calls of getFileChecksum()", TYPE_COUNTER),

INVOCATION_LIST_STATUS_RESULT_SIZE(
"op_get_list_status_result_size", "Number of files returned from list call", TYPE_COUNTER),
INVOCATION_GLOB_STATUS(
StoreStatisticNames.OP_GLOB_STATUS, "Calls of globStatus()", TYPE_DURATION_TOTAL),
INVOCATION_HFLUSH(StoreStatisticNames.OP_HFLUSH, "Calls of hflush()", TYPE_DURATION_TOTAL),
INVOCATION_HSYNC(StoreStatisticNames.OP_HSYNC, "Calls of hsync()", TYPE_DURATION_TOTAL),
StoreStatisticNames.OP_GLOB_STATUS, "Calls of globStatus()", TYPE_DURATION_TOTAL, true),
INVOCATION_HFLUSH(StoreStatisticNames.OP_HFLUSH, "Calls of hflush()", TYPE_DURATION_TOTAL, true),
INVOCATION_HSYNC(StoreStatisticNames.OP_HSYNC, "Calls of hsync()", TYPE_DURATION_TOTAL, true),
INVOCATION_LIST_STATUS(
StoreStatisticNames.OP_LIST_STATUS, "Calls of listStatus()", TYPE_DURATION_TOTAL),
INVOCATION_MKDIRS(StoreStatisticNames.OP_MKDIRS, "Calls of mkdirs()", TYPE_DURATION_TOTAL),
INVOCATION_OPEN(StoreStatisticNames.OP_OPEN, "Calls of open()", TYPE_DURATION_TOTAL),
INVOCATION_RENAME(StoreStatisticNames.OP_RENAME, "Calls of rename()", TYPE_DURATION_TOTAL),
StoreStatisticNames.OP_LIST_STATUS, "Calls of listStatus()", TYPE_DURATION_TOTAL, true),
INVOCATION_MKDIRS(StoreStatisticNames.OP_MKDIRS, "Calls of mkdirs()", TYPE_DURATION_TOTAL, true),
INVOCATION_OPEN(StoreStatisticNames.OP_OPEN, "Calls of open()", TYPE_DURATION_TOTAL, true),
INVOCATION_RENAME(StoreStatisticNames.OP_RENAME, "Calls of rename()", TYPE_DURATION_TOTAL, true),
INVOCATION_COPY_FROM_LOCAL_FILE(
StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE, "Calls of copyFromLocalFile()", TYPE_COUNTER),
StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE,
"Calls of copyFromLocalFile()",
TYPE_COUNTER,
true),
INVOCATION_CREATE_NON_RECURSIVE(
StoreStatisticNames.OP_CREATE_NON_RECURSIVE, "Calls of createNonRecursive()", TYPE_DURATION),
StoreStatisticNames.OP_CREATE_NON_RECURSIVE,
"Calls of createNonRecursive()",
TYPE_DURATION,
true),
INVOCATION_GET_DELEGATION_TOKEN(
StoreStatisticNames.OP_GET_DELEGATION_TOKEN, "Calls of getDelegationToken()", TYPE_COUNTER),
StoreStatisticNames.OP_GET_DELEGATION_TOKEN,
"Calls of getDelegationToken()",
TYPE_COUNTER,
true),
INVOCATION_LIST_LOCATED_STATUS(
StoreStatisticNames.OP_LIST_LOCATED_STATUS, "Calls of listLocatedStatus()", TYPE_COUNTER),
StoreStatisticNames.OP_LIST_LOCATED_STATUS,
"Calls of listLocatedStatus()",
TYPE_COUNTER,
true),

/** Stream reads */
STREAM_READ_BYTES(
Expand Down Expand Up @@ -188,6 +203,8 @@ public enum GhfsStatistic {
private static final ImmutableMap<String, GhfsStatistic> SYMBOL_MAP =
Maps.uniqueIndex(Iterators.forArray(values()), GhfsStatistic::getSymbol);

private final boolean isHadoopApi;

/**
* Statistic definition.
*
Expand All @@ -196,9 +213,14 @@ public enum GhfsStatistic {
* @param type type
*/
GhfsStatistic(String symbol, String description, StatisticTypeEnum type) {
this(symbol, description, type, false);
}

GhfsStatistic(String symbol, String description, StatisticTypeEnum type, boolean isHadoopApi) {
this.symbol = symbol;
this.description = description;
this.type = type;
this.isHadoopApi = isHadoopApi;
}

/** Statistic name. */
Expand Down Expand Up @@ -248,4 +270,8 @@ public String toString() {
public StatisticTypeEnum getType() {
return type;
}

boolean getIsHadoopApi() {
return this.isHadoopApi;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,125 @@
/*
* Copyright 2025 Google Inc.
*
* Licensed 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 com.google.cloud.hadoop.fs.gcs;

import static com.google.cloud.hadoop.fs.gcs.GhfsStatistic.GCS_CONNECTOR_TIME;

import com.google.cloud.hadoop.gcsio.GoogleCloudStorageStatistics;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import org.apache.hadoop.fs.StorageStatistics;

class GhfsThreadLocalStatistics extends StorageStatistics {
static final String NAME = "GhfsThreadLocalStatistics";
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

getClass().getName()?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These are string with special meaning. The consumer framework (eg. spark) will use this name to get the statistics.

static final String HADOOP_API_COUNT = "hadoopApiCount";
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Enum would be better to represent this I think.

So ThreadLocalValue and corresponding strings stay together and not set different one by mistake.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same as above. The consumer framework is not aware of the enums and will have to use string to adhere to the interface.

static final String HADOOP_API_TIME = "hadoopApiTime";
static final String GCS_API_COUNT = "gcsApiCount";
static final String GCS_API_TIME = "gcsApiTime";
static final String BACKOFF_COUNT = "backoffCount";
static final String BACKOFF_TIME = "backoffTime";

private final ThreadLocalValue hadoopApiCount;
private final ThreadLocalValue hadoopApiTime;
private final ThreadLocalValue gcsApiCount;
private final ThreadLocalValue gcsApiTime;
private final ThreadLocalValue backoffCount;
private final ThreadLocalValue backoffTime;

private Map<String, ThreadLocalValue> metrics = new HashMap<>();

GhfsThreadLocalStatistics() {
super(NAME);
this.hadoopApiCount = createMetric(HADOOP_API_COUNT);
this.hadoopApiTime = createMetric(HADOOP_API_TIME);
this.gcsApiCount = createMetric(GCS_API_COUNT);
this.gcsApiTime = createMetric(GCS_API_TIME);
this.backoffCount = createMetric(BACKOFF_COUNT);
this.backoffTime = createMetric(BACKOFF_TIME);
}

private ThreadLocalValue createMetric(String name) {
ThreadLocalValue result = new ThreadLocalValue();
metrics.put(name, result);

return result;
}

@Override
public Long getLong(String s) {
if (!metrics.containsKey(s)) {
return 0L;
}

return metrics.get(s).getValue();
}

@Override
public boolean isTracked(String s) {
return metrics.containsKey(s);
}

@Override
public void reset() {
for (ThreadLocalValue s : metrics.values()) {
s.reset();
}
}

void increment(GhfsStatistic statistic, long count) {
if (statistic == GCS_CONNECTOR_TIME) {
this.hadoopApiTime.increment(count);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I am not able to understand this, why are we incrementing hadoopApiTime for GCS_CONNECTOR_TIME?

Also, the below metric (2 lines below) is Count while here it is Time.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

GCS connector implements Hadoop FS APIs. So we are measuring the wall clock time taken by GcsConnectorHadoopAPIs.

Another way to look at this is: From the caller perspective this the "hadoopAPITime" and from the connector perspective it is the GCSConnectorTime. Also, from the caller perspective they need the metric to be generic since the connector could be for a different scheme.

} else if (statistic.getIsHadoopApi()) {
this.hadoopApiCount.increment(count);
}
}

void increment(GoogleCloudStorageStatistics op, long count) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: switch case?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am in double minds since we are not handling all the enum cases. Will change if we add values.

if (op == GoogleCloudStorageStatistics.GCS_API_TIME) {
this.gcsApiTime.increment(count);
} else if (op == GoogleCloudStorageStatistics.GCS_API_REQUEST_COUNT) {
this.gcsApiCount.increment(count);
} else if (op == GoogleCloudStorageStatistics.GCS_BACKOFF_COUNT) {
this.backoffCount.increment(count);
} else if (op == GoogleCloudStorageStatistics.GCS_BACKOFF_TIME) {
this.backoffTime.increment(count);
}
}

@Override
public Iterator<LongStatistic> getLongStatistics() {
return this.metrics.entrySet().stream()
.map(entry -> new LongStatistic(entry.getKey(), entry.getValue().getValue()))
.iterator();
}

private static class ThreadLocalValue {
private ThreadLocal<Long> value = ThreadLocal.withInitial(() -> 0L);

void increment(long count) {
value.set(value.get() + count);
}

Long getValue() {
return value.get();
}

void reset() {
value.set(0L);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,10 @@ public GoogleHadoopFileSystem() {
GlobalStorageStatistics.INSTANCE.put(
GhfsGlobalStorageStatistics.NAME, () -> new GhfsGlobalStorageStatistics());

GlobalStorageStatistics.INSTANCE.put(
GhfsThreadLocalStatistics.NAME,
() -> ((GhfsGlobalStorageStatistics) globalStats).getThreadLocalStatistics());

if (GhfsGlobalStorageStatistics.class.isAssignableFrom(globalStats.getClass())) {
globalStorageStatistics = (GhfsGlobalStorageStatistics) globalStats;
} else {
Expand Down
Loading