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

[INLONG-11349][Sort] Integrate opentelemetry for sort-connectors-v1.15 #11351

Open
wants to merge 15 commits into
base: master
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 @@ -375,6 +375,9 @@ public class Constants {
public static final ConfigOption<String> SQL_SCRIPT_FILE = key("sql.script.file").noDefaultValue()
.withDescription("The file which is sql script and contains multi statement");

public static final ConfigOption<Boolean> ENABLE_LOG_REPORT = key("enable.log.report").defaultValue(false)
.withDescription("Whether to enable openTelemetry log report or not");

// ------------------------------------------------------------------------
// File format and compression related
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,8 @@ public static void main(String[] args) throws Exception {
config.getString(Constants.UPSERT_MATERIALIZE));
tableEnv.getConfig().getConfiguration().setString(Constants.TABLE_EXEC_SINK_NOT_NULL_ENFORCER,
config.getString(Constants.NOT_NULL_ENFORCER));
tableEnv.getConfig().getConfiguration().setBoolean(Constants.ENABLE_LOG_REPORT.key(),
config.getBoolean(Constants.ENABLE_LOG_REPORT));
String sqlFile = config.getString(Constants.SQL_SCRIPT_FILE);
Parser parser;
if (StringUtils.isEmpty(sqlFile)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.inlong.sort.tests.utils.FlinkContainerTestEnvJRE8;
import org.apache.inlong.sort.tests.utils.JdbcProxy;
import org.apache.inlong.sort.tests.utils.MySqlContainer;
import org.apache.inlong.sort.tests.utils.OpenTelemetryContainer;
import org.apache.inlong.sort.tests.utils.PlaceholderResolver;
import org.apache.inlong.sort.tests.utils.StarRocksContainer;
import org.apache.inlong.sort.tests.utils.TestUtils;
Expand All @@ -34,6 +35,7 @@
import org.testcontainers.containers.KafkaContainer;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.utility.DockerImageName;
import org.testcontainers.utility.MountableFile;

import java.io.IOException;
import java.net.URI;
Expand Down Expand Up @@ -111,6 +113,15 @@ public class Kafka2StarRocksTest extends FlinkContainerTestEnvJRE8 {
.withNetworkAliases("mysql")
.withLogConsumer(new Slf4jLogConsumer(MYSQL_LOG));

@ClassRule
public static final OpenTelemetryContainer OPEN_TELEMETRY_CONTAINER =
(OpenTelemetryContainer) new OpenTelemetryContainer()
.withCopyFileToContainer(MountableFile.forClasspathResource("/env/otel-config.yaml"),
"/otel-config.yaml")
.withCommand("--config=/otel-config.yaml")
.withNetwork(NETWORK)
.withNetworkAliases("logcollector");

@Before
public void setup() {
waitUntilJobRunning(Duration.ofSeconds(30));
Expand Down Expand Up @@ -152,6 +163,10 @@ public static void teardown() {
if (STAR_ROCKS != null) {
STAR_ROCKS.stop();
}

if (OPEN_TELEMETRY_CONTAINER != null) {
OPEN_TELEMETRY_CONTAINER.stop();
}
}

private void initializeKafkaTable(String topic) {
Expand Down Expand Up @@ -223,5 +238,10 @@ public void testKafkaWithSqlFile() throws Exception {
"test_output1",
3,
60000L);
// check log appender
String logs = OPEN_TELEMETRY_CONTAINER.getLogs();
if (!logs.contains("OpenTelemetryLogger installed")) {
throw new Exception("Failure to append logs to OpenTelemetry");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.inlong.sort.tests.utils.FlinkContainerTestEnvJRE8;
import org.apache.inlong.sort.tests.utils.JdbcProxy;
import org.apache.inlong.sort.tests.utils.MySqlContainer;
import org.apache.inlong.sort.tests.utils.OpenTelemetryContainer;
import org.apache.inlong.sort.tests.utils.StarRocksContainer;
import org.apache.inlong.sort.tests.utils.TestUtils;

Expand All @@ -30,6 +31,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.utility.MountableFile;

import java.net.URISyntaxException;
import java.nio.file.Path;
Expand Down Expand Up @@ -85,6 +87,15 @@ public class Mysql2StarRocksTest extends FlinkContainerTestEnvJRE8 {
.withNetworkAliases("mysql")
.withLogConsumer(new Slf4jLogConsumer(LOG));

@ClassRule
public static final OpenTelemetryContainer OPEN_TELEMETRY_CONTAINER =
(OpenTelemetryContainer) new OpenTelemetryContainer()
.withCopyFileToContainer(MountableFile.forClasspathResource("/env/otel-config.yaml"),
"/otel-config.yaml")
.withCommand("--config=/otel-config.yaml")
.withNetwork(NETWORK)
.withNetworkAliases("logcollector");

@Before
public void setup() {
waitUntilJobRunning(Duration.ofSeconds(30));
Expand Down Expand Up @@ -121,6 +132,9 @@ public static void teardown() {
if (STAR_ROCKS != null) {
STAR_ROCKS.stop();
}
if (OPEN_TELEMETRY_CONTAINER != null) {
OPEN_TELEMETRY_CONTAINER.stop();
}
}

/**
Expand Down Expand Up @@ -161,6 +175,11 @@ public void testMysqlUpdateAndDelete() throws Exception {
expectResult,
"test_output1",
3,
60000L);
80000L);
// check log appender
String logs = OPEN_TELEMETRY_CONTAINER.getLogs();
if (!logs.contains("OpenTelemetryLogger installed")) {
throw new Exception("Failure to append logs to OpenTelemetry");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.inlong.sort.tests.utils.FlinkContainerTestEnvJRE8;
import org.apache.inlong.sort.tests.utils.JdbcProxy;
import org.apache.inlong.sort.tests.utils.OpenTelemetryContainer;
import org.apache.inlong.sort.tests.utils.StarRocksContainer;
import org.apache.inlong.sort.tests.utils.TestUtils;

Expand All @@ -35,6 +36,7 @@
import org.testcontainers.containers.PulsarContainer;
import org.testcontainers.containers.output.Slf4jLogConsumer;
import org.testcontainers.utility.DockerImageName;
import org.testcontainers.utility.MountableFile;

import java.net.URI;
import java.net.URISyntaxException;
Expand Down Expand Up @@ -91,6 +93,15 @@ public class Pulsar2StarRocksTest extends FlinkContainerTestEnvJRE8 {
.withNetworkAliases(INTER_CONTAINER_STAR_ROCKS_ALIAS)
.withLogConsumer(new Slf4jLogConsumer(STAR_ROCKS_LOG));

@ClassRule
public static final OpenTelemetryContainer OPEN_TELEMETRY_CONTAINER =
(OpenTelemetryContainer) new OpenTelemetryContainer()
.withCopyFileToContainer(MountableFile.forClasspathResource("/env/otel-config.yaml"),
"/otel-config.yaml")
.withCommand("--config=/otel-config.yaml")
.withNetwork(NETWORK)
.withNetworkAliases("logcollector");

@Before
public void setup() {
waitUntilJobRunning(Duration.ofSeconds(30));
Expand Down Expand Up @@ -119,6 +130,9 @@ public static void teardown() {
if (STAR_ROCKS != null) {
STAR_ROCKS.stop();
}
if (OPEN_TELEMETRY_CONTAINER != null) {
OPEN_TELEMETRY_CONTAINER.stop();
}
}

@Test
Expand All @@ -144,6 +158,11 @@ public void testPulsarToStarRocks() throws Exception {
"1,Alice,Hello, Pulsar",
"2,Bob,Goodbye, Pulsar");
proxy.checkResultWithTimeout(expectedResult, "test_output1", 3, 60000L);
// check log appender
String logs = OPEN_TELEMETRY_CONTAINER.getLogs();
if (!logs.contains("OpenTelemetryLogger installed")) {
throw new Exception("Failure to append logs to OpenTelemetry");
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -130,6 +130,7 @@ public void submitSQLJob(String sqlFile, Path... jars)
commands.add(copyToContainerTmpPath(jobManager, constructDistJar(jars)));
commands.add("--sql.script.file");
commands.add(containerSqlFile);
commands.add("--enable.log.report true");

ExecResult execResult =
jobManager.execInContainer("bash", "-c", String.join(" ", commands));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ public static void before() {
.withNetworkAliases(INTER_CONTAINER_TM_ALIAS)
.withExposedPorts(DEBUG_PORT)
.withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES)
.withEnv("OTEL_EXPORTER_ENDPOINT", "logcollector:4317")
.dependsOn(jobManager)
.withLogConsumer(new Slf4jLogConsumer(TM_LOG));

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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.inlong.sort.tests.utils;

import org.testcontainers.containers.GenericContainer;

public class OpenTelemetryContainer extends GenericContainer {

public static final String IMAGE = "otel/opentelemetry-collector-contrib:0.110.0";
public static final Integer PORT = 4317;
public OpenTelemetryContainer() {
super(IMAGE);
addExposedPort(PORT);
}
}
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
#
# 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.
#

receivers:
otlp:
protocols:
grpc:
endpoint: logcollector:4317
processors:
batch:

exporters:
debug:
verbosity: detailed

service:
pipelines:
logs:
receivers: [otlp]
processors: [batch]
exporters: [debug]
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@ CREATE TABLE test_input1 (
'password' = 'inlong',
'database-name' = 'test',
'table-name' = 'test_input1',
'scan.incremental.snapshot.enabled' = 'false',
'scan.incremental.snapshot.enabled' = 'true',
'jdbc.properties.useSSL' = 'false',
'jdbc.properties.allowPublicKeyRetrieval' = 'true'
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -469,4 +469,10 @@ public final class Constants {
.withDescription(
"pulsar client auth params");

public static final ConfigOption<Boolean> ENABLE_LOG_REPORT =
ConfigOptions.key("enable.log.report")
.booleanType()
.defaultValue(false)
.withDescription("Whether enable openTelemetry log report or not.");

}
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,8 @@
<createDependencyReducedPom>false</createDependencyReducedPom>
<artifactSet>
<includes>
<include>io.opentelemetry*</include>
<include>com.squareup.*</include>
<include>org.apache.inlong:*</include>
<include>com.google.protobuf:*</include>
<include>org.apache.kafka:*</include>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.inlong.sort.iceberg.source;

import org.apache.inlong.sort.base.Constants;
import org.apache.inlong.sort.base.metric.MetricOption;
import org.apache.inlong.sort.iceberg.IcebergReadableMetadata.MetadataConverter;
import org.apache.inlong.sort.iceberg.source.reader.IcebergSourceReader;
Expand Down Expand Up @@ -94,6 +95,7 @@ public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEn
private final SplitAssignerFactory assignerFactory;

private final MetricOption metricOption;
private final boolean enableLogReport;

// Can't use SerializableTable as enumerator needs a regular table
// that can discover table changes
Expand All @@ -105,13 +107,15 @@ public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEn
ReaderFunction<T> readerFunction,
SplitAssignerFactory assignerFactory,
Table table,
MetricOption metricOption) {
MetricOption metricOption,
boolean enableLogReport) {
this.tableLoader = tableLoader;
this.scanContext = scanContext;
this.readerFunction = readerFunction;
this.assignerFactory = assignerFactory;
this.table = table;
this.metricOption = metricOption;
this.enableLogReport = enableLogReport;
}

String name() {
Expand Down Expand Up @@ -167,7 +171,7 @@ public SourceReader<T, IcebergSourceSplit> createReader(SourceReaderContext read
InlongIcebergSourceReaderMetrics<T> metrics =
new InlongIcebergSourceReaderMetrics<>(readerContext.metricGroup(), lazyTable().name());
metrics.registerMetrics(metricOption);
return new IcebergSourceReader<>(metrics, readerFunction, readerContext);
return new IcebergSourceReader<>(metrics, readerFunction, readerContext, enableLogReport);
}

@Override
Expand Down Expand Up @@ -522,9 +526,10 @@ public IcebergSource<T> build() {
}
resolveMetricOption();
checkRequired();
boolean enableLogReport = flinkConfig.get(Constants.ENABLE_LOG_REPORT);
// Since builder already load the table, pass it to the source to avoid double loading
return new IcebergSource<T>(
tableLoader, context, readerFunction, splitAssignerFactory, table, metricOption);
tableLoader, context, readerFunction, splitAssignerFactory, table, metricOption, enableLogReport);
}

private void checkRequired() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.inlong.sort.iceberg.source;

import org.apache.inlong.sort.base.Constants;
import org.apache.inlong.sort.iceberg.IcebergReadableMetadata;
import org.apache.inlong.sort.iceberg.IcebergReadableMetadata.MetadataConverter;

Expand Down Expand Up @@ -83,6 +84,7 @@ public class IcebergTableSource
private final Map<String, String> properties;
private final boolean isLimitPushDown;
private final ReadableConfig readableConfig;
private final boolean enableLogReport;

private IcebergTableSource(IcebergTableSource toCopy) {
this.loader = toCopy.loader;
Expand All @@ -95,6 +97,7 @@ private IcebergTableSource(IcebergTableSource toCopy) {
this.readableConfig = toCopy.readableConfig;
this.producedDataType = toCopy.producedDataType;
this.metadataKeys = toCopy.metadataKeys;
this.enableLogReport = toCopy.enableLogReport;
}

public IcebergTableSource(
Expand Down Expand Up @@ -124,6 +127,7 @@ private IcebergTableSource(
this.readableConfig = readableConfig;
this.producedDataType = schema.toPhysicalRowDataType();
this.metadataKeys = new ArrayList<>();
this.enableLogReport = readableConfig.get(Constants.ENABLE_LOG_REPORT);
}

@Override
Expand Down
Loading
Loading