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,61 @@
/*
* 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.iceberg.rest;

import java.util.Map;
import java.util.function.Supplier;
import org.apache.iceberg.metrics.MetricsReport;
import org.apache.iceberg.metrics.MetricsReporter;
import org.apache.iceberg.rest.requests.ReportMetricsRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

class RESTMetricsReporter implements MetricsReporter {
Copy link
Contributor

Choose a reason for hiding this comment

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

Serializable? Or is this just to produce a separate one as a minimal refactor?

Copy link
Contributor Author

@nastra nastra Apr 24, 2023

Choose a reason for hiding this comment

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

for easier reviewing, the goal of this PR was to move REST metrics reporting into its own class before making it fully serializable (as that involves slightly more changes). I have a WIP PR (currently outdated) for this that also includes the required changes from #7144

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

private final RESTClient client;
private final String metricsEndpoint;
private final Supplier<Map<String, String>> headers;

RESTMetricsReporter(
RESTClient client, String metricsEndpoint, Supplier<Map<String, String>> headers) {
this.client = client;
this.metricsEndpoint = metricsEndpoint;
this.headers = headers;
}

@Override
public void report(MetricsReport report) {
if (null == report) {
LOG.warn("Received invalid metrics report: null");
return;
}

try {
client.post(
metricsEndpoint,
ReportMetricsRequest.of(report),
null,
headers,
ErrorHandlers.defaultErrorHandler());
} catch (Exception e) {
LOG.warn("Failed to report metrics to REST endpoint {}", metricsEndpoint, e);
}
}
}
36 changes: 13 additions & 23 deletions core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java
Original file line number Diff line number Diff line change
Expand Up @@ -57,8 +57,8 @@
import org.apache.iceberg.hadoop.Configurable;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.ResolvingFileIO;
import org.apache.iceberg.metrics.MetricsReport;
import org.apache.iceberg.metrics.MetricsReporter;
import org.apache.iceberg.metrics.MetricsReporters;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
Expand All @@ -68,7 +68,6 @@
import org.apache.iceberg.rest.requests.CreateNamespaceRequest;
import org.apache.iceberg.rest.requests.CreateTableRequest;
import org.apache.iceberg.rest.requests.RenameTableRequest;
import org.apache.iceberg.rest.requests.ReportMetricsRequest;
import org.apache.iceberg.rest.requests.UpdateNamespacePropertiesRequest;
import org.apache.iceberg.rest.responses.ConfigResponse;
import org.apache.iceberg.rest.responses.CreateNamespaceResponse;
Expand Down Expand Up @@ -347,35 +346,26 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) {
tableFileIO(context, response.config()),
tableMetadata);

TableIdentifier tableIdentifier = loadedIdent;
BaseTable table =
new BaseTable(
ops,
fullTableName(loadedIdent),
report -> reportMetrics(tableIdentifier, report, session::headers));
metricsReporter(paths.metrics(loadedIdent), session::headers));
if (metadataType != null) {
return MetadataTableUtils.createMetadataTableInstance(table, metadataType);
}

return table;
}

private void reportMetrics(
TableIdentifier tableIdentifier,
MetricsReport report,
Supplier<Map<String, String>> headers) {
try {
reporter.report(report);
if (reportingViaRestEnabled) {
client.post(
paths.metrics(tableIdentifier),
ReportMetricsRequest.of(report),
null,
headers,
ErrorHandlers.defaultErrorHandler());
}
} catch (Exception e) {
LOG.warn("Failed to report metrics to REST endpoint for table {}", tableIdentifier, e);
private MetricsReporter metricsReporter(
Copy link
Contributor Author

@nastra nastra Apr 24, 2023

Choose a reason for hiding this comment

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

rather than wrapping the combined reporter in a lambda for the purpose of reporting, we just return the combined reporter here.

String metricsEndpoint, Supplier<Map<String, String>> headers) {
if (reportingViaRestEnabled) {
RESTMetricsReporter restMetricsReporter =
new RESTMetricsReporter(client, metricsEndpoint, headers);
return MetricsReporters.combine(reporter, restMetricsReporter);
} else {
return this.reporter;
}
}

Expand Down Expand Up @@ -603,7 +593,7 @@ public Table create() {
response.tableMetadata());

return new BaseTable(
ops, fullTableName(ident), report -> reportMetrics(ident, report, session::headers));
ops, fullTableName(ident), metricsReporter(paths.metrics(ident), session::headers));
}

@Override
Expand All @@ -625,7 +615,7 @@ public Transaction createTransaction() {
meta);

return Transactions.createTableTransaction(
fullName, ops, meta, report -> reportMetrics(ident, report, session::headers));
fullName, ops, meta, metricsReporter(paths.metrics(ident), session::headers));
}

@Override
Expand Down Expand Up @@ -676,7 +666,7 @@ public Transaction replaceTransaction() {
base);

return Transactions.replaceTableTransaction(
fullName, ops, replacement, report -> reportMetrics(ident, report, session::headers));
fullName, ops, replacement, metricsReporter(paths.metrics(ident), session::headers));
}

@Override
Expand Down