Skip to content

Commit

Permalink
Add an SdkHarnessOption that controls whether logging is redirected t…
Browse files Browse the repository at this point in the history
…hrough the FnApi logging service. Redirection through the logging service is enabled by default.
  • Loading branch information
t2h6 authored Dec 18, 2024
1 parent 5eed396 commit 85450f4
Show file tree
Hide file tree
Showing 6 changed files with 67 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,13 @@ enum LogLevel {

void setLogMdc(boolean value);

/** This option controls whether logging will be redirected through the logging service. */
@Description("Controls whether logging will be redirected through the logging service.")
@Default.Boolean(true)
boolean getEnableLoggingService();

void setEnableLoggingService(boolean enableLoggingService);

/**
* Size (in MB) of each grouping table used to pre-combine elements. Larger values may reduce the
* amount of data shuffled. If unset, defaults to 100 MB.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,8 @@
import org.apache.beam.fn.harness.control.ProcessBundleHandler;
import org.apache.beam.fn.harness.data.BeamFnDataGrpcClient;
import org.apache.beam.fn.harness.debug.DataSampler;
import org.apache.beam.fn.harness.logging.BeamFnLoggingClient;
import org.apache.beam.fn.harness.logging.LoggingClient;
import org.apache.beam.fn.harness.logging.LoggingClientFactory;
import org.apache.beam.fn.harness.state.BeamFnStateGrpcClientCache;
import org.apache.beam.fn.harness.status.BeamFnStatusClient;
import org.apache.beam.fn.harness.stream.HarnessStreamObserverFactories;
Expand Down Expand Up @@ -283,8 +284,8 @@ public static void main(

// The logging client variable is not used per se, but during its lifetime (until close()) it
// intercepts logging and sends it to the logging service.
try (BeamFnLoggingClient logging =
BeamFnLoggingClient.createAndStart(
try (LoggingClient logging =
LoggingClientFactory.createAndStart(
options, loggingApiServiceDescriptor, channelFactory::forDescriptor)) {
LOG.info("Fn Harness started");
// Register standard file systems.
Expand Down Expand Up @@ -410,7 +411,7 @@ private BeamFnApi.ProcessBundleDescriptor loadDescriptor(String id) {
outboundObserverFactory,
executorService,
handlers);
CompletableFuture.anyOf(control.terminationFuture(), logging.terminationFuture()).get();
CompletableFuture.allOf(control.terminationFuture(), logging.terminationFuture()).get();
if (beamFnStatusClient != null) {
beamFnStatusClient.close();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@
/**
* Configures {@link java.util.logging} to send all {@link LogRecord}s via the Beam Fn Logging API.
*/
public class BeamFnLoggingClient implements AutoCloseable {
public class BeamFnLoggingClient implements LoggingClient {
private static final String ROOT_LOGGER_NAME = "";
private static final ImmutableMap<Level, BeamFnApi.LogEntry.Severity.Enum> LOG_LEVEL_MAP =
ImmutableMap.<Level, BeamFnApi.LogEntry.Severity.Enum>builder()
Expand Down Expand Up @@ -119,7 +119,7 @@ public class BeamFnLoggingClient implements AutoCloseable {
*/
private @Nullable Thread logEntryHandlerThread = null;

public static BeamFnLoggingClient createAndStart(
static BeamFnLoggingClient createAndStart(
PipelineOptions options,
Endpoints.ApiServiceDescriptor apiServiceDescriptor,
Function<Endpoints.ApiServiceDescriptor, ManagedChannel> channelFactory) {
Expand Down Expand Up @@ -383,6 +383,7 @@ void flushFinalLogs(@UnderInitialization BeamFnLoggingClient this) {
}
}

@Override
public CompletableFuture<?> terminationFuture() {
checkNotNull(bufferedLogConsumer, "BeamFnLoggingClient not fully started");
return bufferedLogConsumer;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,8 @@
package org.apache.beam.fn.harness.logging;

import java.util.concurrent.CompletableFuture;

public interface LoggingClient extends AutoCloseable {

CompletableFuture<?> terminationFuture();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
package org.apache.beam.fn.harness.logging;

import io.grpc.ManagedChannel;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import org.apache.beam.model.pipeline.v1.Endpoints;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.SdkHarnessOptions;

/**
* A factory for {@link LoggingClient}s. Provides {@link BeamFnLoggingClient} if the logging service
* is enabled, otherwise provides a no-op client.
*/
public class LoggingClientFactory {

private LoggingClientFactory() {}

/**
* A factory for {@link LoggingClient}s. Provides {@link BeamFnLoggingClient} if the logging
* service is enabled, otherwise provides a no-op client.
*/
public static LoggingClient createAndStart(
PipelineOptions options,
Endpoints.ApiServiceDescriptor apiServiceDescriptor,
Function<Endpoints.ApiServiceDescriptor, ManagedChannel> channelFactory) {
if (options.as(SdkHarnessOptions.class).getEnableLoggingService()) {
return BeamFnLoggingClient.createAndStart(options, apiServiceDescriptor, channelFactory);
} else {
return new NoOpLoggingClient();
}
}

static final class NoOpLoggingClient implements LoggingClient {
@Override
public CompletableFuture<?> terminationFuture() {
return CompletableFuture.completedFuture(new Object());
}

@Override
public void close() throws Exception {}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import org.apache.beam.fn.harness.debug.DataSampler;
import org.apache.beam.fn.harness.logging.BeamFnLoggingClient;
import org.apache.beam.fn.harness.logging.BeamFnLoggingMDC;
import org.apache.beam.fn.harness.logging.LoggingClientFactory;
import org.apache.beam.model.fnexecution.v1.BeamFnApi;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleDescriptor;
import org.apache.beam.model.fnexecution.v1.BeamFnLoggingGrpc;
Expand Down Expand Up @@ -648,7 +649,7 @@ public StreamObserver<BeamFnApi.LogEntry.List> logging(
// that was set up
// earlier.
try (BeamFnLoggingClient ignored =
BeamFnLoggingClient.createAndStart(
LoggingClientFactory.createAndStart(
PipelineOptionsFactory.create(),
apiServiceDescriptor,
(Endpoints.ApiServiceDescriptor descriptor) -> channel)) {
Expand Down

0 comments on commit 85450f4

Please sign in to comment.