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
Expand Up @@ -30,7 +30,6 @@

import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.ExecutionOptions;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.table.api.config.TableConfigOptions;
Expand All @@ -47,7 +46,6 @@
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.types.logical.RowType;

import java.io.File;
import java.time.ZoneId;
import java.util.ArrayList;
import java.util.Arrays;
Expand Down Expand Up @@ -247,11 +245,6 @@ private static Configuration toFlussClientConfig(
// retry. The option 'client.lookup.max-retries' is only for dealing with the
// RetriableException return by server not all exceptions. Trace by:
// https://github.com/apache/fluss/issues/2099

// pass flink io tmp dir to fluss client.
flussConfig.setString(
ConfigOptions.CLIENT_SCANNER_IO_TMP_DIR,
new File(flinkConfig.get(CoreOptions.TMP_DIRS), "/fluss").getAbsolutePath());
return flussConfig;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,9 @@

import javax.annotation.Nullable;

import static org.apache.fluss.config.ConfigOptions.CLIENT_SCANNER_IO_TMP_DIR;
import static org.apache.fluss.flink.utils.FlinkConnectorOptionsUtils.getClientScannerIoTmpDir;

/** Flink source for Fluss. */
public class FlinkSource<OUT>
implements Source<OUT, SourceSplitBase, SourceEnumeratorState>, ResultTypeQueryable {
Expand Down Expand Up @@ -181,6 +184,9 @@ public SourceReader<OUT, SourceSplitBase> createReader(SourceReaderContext conte
FlinkSourceReaderMetrics flinkSourceReaderMetrics =
new FlinkSourceReaderMetrics(context.metricGroup());

flussConf.set(
CLIENT_SCANNER_IO_TMP_DIR,
getClientScannerIoTmpDir(flussConf, context.getConfiguration()));
deserializationSchema.open(
new DeserializerInitContextImpl(
context.metricGroup().addGroup("deserializer"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,17 @@

package org.apache.fluss.flink.utils;

import org.apache.fluss.config.Configuration;
import org.apache.fluss.flink.FlinkConnectorOptions;
import org.apache.fluss.flink.FlinkConnectorOptions.ScanStartupMode;

import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.api.config.TableConfigOptions;
import org.apache.flink.table.types.logical.RowType;

import java.io.File;
import java.time.LocalDateTime;
import java.time.ZoneId;
import java.time.format.DateTimeFormatter;
Expand All @@ -34,6 +37,8 @@
import java.util.Optional;
import java.util.stream.Collectors;

import static org.apache.flink.configuration.CoreOptions.TMP_DIRS;
import static org.apache.fluss.config.ConfigOptions.CLIENT_SCANNER_IO_TMP_DIR;
import static org.apache.fluss.flink.FlinkConnectorOptions.SCAN_STARTUP_MODE;
import static org.apache.fluss.flink.FlinkConnectorOptions.SCAN_STARTUP_TIMESTAMP;
import static org.apache.fluss.flink.FlinkConnectorOptions.ScanStartupMode.TIMESTAMP;
Expand Down Expand Up @@ -148,6 +153,17 @@ public static long parseTimestamp(String timestampStr, String optionKey, ZoneId
}
}

public static String getClientScannerIoTmpDir(
Configuration flussConf, org.apache.flink.configuration.Configuration flinkConfig) {
if (!flussConf.contains(CLIENT_SCANNER_IO_TMP_DIR)) {
if (flinkConfig.contains(TMP_DIRS)) {
// pass flink io tmp dir to fluss client.
return new File(flinkConfig.get(CoreOptions.TMP_DIRS), "/fluss").getAbsolutePath();
}
}
return flussConf.getString(CLIENT_SCANNER_IO_TMP_DIR);
}

/** Fluss startup options. * */
public static class StartupOptions {
public ScanStartupMode startupMode;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,16 @@

package org.apache.fluss.flink.utils;

import org.apache.fluss.config.Configuration;

import org.apache.flink.table.api.ValidationException;
import org.junit.jupiter.api.Test;

import java.time.ZoneId;
import java.util.TimeZone;

import static org.apache.flink.configuration.CoreOptions.TMP_DIRS;
import static org.apache.fluss.config.ConfigOptions.CLIENT_SCANNER_IO_TMP_DIR;
import static org.apache.fluss.flink.FlinkConnectorOptions.SCAN_STARTUP_TIMESTAMP;
import static org.apache.fluss.flink.utils.FlinkConnectorOptionsUtils.parseTimestamp;
import static org.assertj.core.api.Assertions.assertThat;
Expand Down Expand Up @@ -58,4 +62,31 @@ void testParseTimestamp() {
+ "or 'timestamp', but is '2023-12-09T23:09:12'. "
+ "You can config like: '2023-12-09 23:09:12' or '1678883047356'.");
}

@Test
void testGetClientScannerIoTmpDir() {
Configuration flussConfig =
new Configuration().set(CLIENT_SCANNER_IO_TMP_DIR, "/fluss_tmp_dir");
org.apache.flink.configuration.Configuration flinkConfig =
new org.apache.flink.configuration.Configuration().set(TMP_DIRS, "/flink_tmp_dir");
assertThat(
FlinkConnectorOptionsUtils.getClientScannerIoTmpDir(
flussConfig, new org.apache.flink.configuration.Configuration()))
.isEqualTo("/fluss_tmp_dir");
assertThat(FlinkConnectorOptionsUtils.getClientScannerIoTmpDir(flussConfig, flinkConfig))
.isEqualTo("/fluss_tmp_dir");
String property = System.getProperty("java.io.tmpdir");
assertThat(
FlinkConnectorOptionsUtils.getClientScannerIoTmpDir(
new Configuration(),
new org.apache.flink.configuration.Configuration()))
.isEqualTo(property + "/fluss");

// only replace when flussConfig not contains CLIENT_SCANNER_IO_TMP_DIR while flinkConfig
// contains TMP_DIRS.
assertThat(
FlinkConnectorOptionsUtils.getClientScannerIoTmpDir(
new Configuration(), flinkConfig))
.isEqualTo("/flink_tmp_dir/fluss");
}
}