diff --git a/fluss-common/src/main/java/org/apache/fluss/annotation/docs/ConfigOverrideDefault.java b/fluss-common/src/main/java/org/apache/fluss/annotation/docs/ConfigOverrideDefault.java new file mode 100644 index 0000000000..5b5a1e0b3f --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/annotation/docs/ConfigOverrideDefault.java @@ -0,0 +1,33 @@ +/* + * 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.fluss.annotation.docs; + +import org.apache.fluss.annotation.Internal; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** Annotation used to override the default value string in the documentation. */ +@Internal +@Target(ElementType.FIELD) +@Retention(RetentionPolicy.RUNTIME) +public @interface ConfigOverrideDefault { + String value(); +} diff --git a/fluss-common/src/main/java/org/apache/fluss/annotation/docs/ConfigSection.java b/fluss-common/src/main/java/org/apache/fluss/annotation/docs/ConfigSection.java new file mode 100644 index 0000000000..c9eab72692 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/annotation/docs/ConfigSection.java @@ -0,0 +1,33 @@ +/* + * 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.fluss.annotation.docs; + +import org.apache.fluss.annotation.Internal; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** Annotation used on config options to group them into logical sections. */ +@Internal +@Target({ElementType.FIELD, ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +public @interface ConfigSection { + String value(); +} diff --git a/fluss-docgen/README.md b/fluss-docgen/README.md new file mode 100644 index 0000000000..1ad5da0fca --- /dev/null +++ b/fluss-docgen/README.md @@ -0,0 +1,20 @@ +# Fluss Documentation Generator + +This module contains utilities to automatically generate documentation parts from the Fluss source code. This ensures that the documentation stays in sync with the actual implementation and default values. + +## Configuration Options Generator + +The `ConfigOptionsDocGenerator` scans the `ConfigOptions` class and generates an MDX partial file containing categorized tables of all available configuration settings. + +### How it works +1. It uses reflection to find all `ConfigOption` fields in the `ConfigOptions` class. +2. It groups options into sections based on the `@ConfigSection` annotation or key prefixes. +3. It handles special default value formatting via `@ConfigOverrideDefault`. +4. It outputs an MDX file (config_reference.mdx) using React-compatible HTML table syntax. + +### Running the Generator + +To update the configuration documentation, run the following command from the project root: + +```bash +./mvnw compile -pl fluss-docgen -am && ./mvnw exec:java -pl fluss-docgen -Dexec.mainClass="org.apache.fluss.docs.ConfigOptionsDocGenerator" \ No newline at end of file diff --git a/fluss-docgen/pom.xml b/fluss-docgen/pom.xml new file mode 100644 index 0000000000..3a3806bce5 --- /dev/null +++ b/fluss-docgen/pom.xml @@ -0,0 +1,79 @@ + + + + 4.0.0 + + + org.apache.fluss + fluss + 0.9-SNAPSHOT + + + fluss-docgen + Fluss : Documentation Generator + + + + org.apache.fluss + fluss-common + ${project.version} + + + + org.junit.jupiter + junit-jupiter + test + + + + org.assertj + assertj-core + test + + + + org.apache.fluss + fluss-test-utils + ${project.version} + test + + + + + + + org.codehaus.mojo + exec-maven-plugin + 3.1.0 + + + compile + java + + + + + org.apache.fluss.docs.ConfigOptionsDocGenerator + + + + + \ No newline at end of file diff --git a/fluss-docgen/src/main/java/org/apache/fluss/docs/ConfigDocUtils.java b/fluss-docgen/src/main/java/org/apache/fluss/docs/ConfigDocUtils.java new file mode 100644 index 0000000000..8bc2079134 --- /dev/null +++ b/fluss-docgen/src/main/java/org/apache/fluss/docs/ConfigDocUtils.java @@ -0,0 +1,70 @@ +/* + * 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.fluss.docs; + +import org.apache.fluss.config.ConfigOption; +import org.apache.fluss.config.MemorySize; + +import java.time.Duration; + +/** Utility class for formatting configuration options into human-readable documentation. */ +public class ConfigDocUtils { + + /** + * Formats the default value of a {@link ConfigOption} for documentation purposes. + * + * @param option The configuration option to format. + * @return A string representation of the default value. + */ + public static String formatDefaultValue(ConfigOption option) { + Object value = option.defaultValue(); + + if (value == null) { + return "none"; + } + + // Handle Duration: Convert ISO-8601 (PT15M) to human-readable (15 min). + if (value instanceof Duration) { + Duration d = (Duration) value; + long seconds = d.getSeconds(); // Use Java 8 compatible method. + + if (seconds == 0) { + return "0 s"; + } + if (seconds >= 3600 && seconds % 3600 == 0) { + return (seconds / 3600) + " hours"; + } + if (seconds >= 60 && seconds % 60 == 0) { + return (seconds / 60) + " min"; + } + return seconds + " s"; + } + + // Handle MemorySize: Uses internal toString() for human-readable units (e.g., 64 mb). + if (value instanceof MemorySize) { + return value.toString(); + } + + // Handle Strings: Specifically check for empty values. + if (value instanceof String && ((String) value).isEmpty()) { + return "(empty)"; + } + + return value.toString(); + } +} diff --git a/fluss-docgen/src/main/java/org/apache/fluss/docs/ConfigOptionsDocGenerator.java b/fluss-docgen/src/main/java/org/apache/fluss/docs/ConfigOptionsDocGenerator.java new file mode 100644 index 0000000000..560b5f141f --- /dev/null +++ b/fluss-docgen/src/main/java/org/apache/fluss/docs/ConfigOptionsDocGenerator.java @@ -0,0 +1,157 @@ +/* + * 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.fluss.docs; + +import org.apache.fluss.annotation.docs.ConfigOverrideDefault; +import org.apache.fluss.annotation.docs.ConfigSection; +import org.apache.fluss.config.ConfigOption; +import org.apache.fluss.config.ConfigOptions; + +import java.io.File; +import java.lang.reflect.Field; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** Generator for configuration documentation. */ +public class ConfigOptionsDocGenerator { + + public static void main(String[] args) throws Exception { + Path projectRoot = findProjectRoot(); + File outputFile = + projectRoot.resolve("website/docs/maintenance/config_reference.mdx").toFile(); + + System.out.println("Generating MDX partial: " + outputFile.getAbsolutePath()); + + if (!outputFile.getParentFile().exists()) { + outputFile.getParentFile().mkdirs(); + } + + String content = generateMDXContent(); + Files.write( + outputFile.toPath(), Collections.singletonList(content), StandardCharsets.UTF_8); + + System.out.println("SUCCESS: MDX partial generated."); + } + + private static String generateMDXContent() throws IllegalAccessException { + StringBuilder builder = new StringBuilder(); + builder.append("{/* This file is auto-generated. Do not edit directly. */}\n\n"); + + Field[] fields = ConfigOptions.class.getDeclaredFields(); + Map> sections = new TreeMap<>(); + + // 1. Group the fields first + for (Field field : fields) { + if (field.getType().equals(ConfigOption.class)) { + String section = "Common"; + if (field.isAnnotationPresent(ConfigSection.class)) { + section = field.getAnnotation(ConfigSection.class).value(); + } else { + ConfigOption option = (ConfigOption) field.get(null); + String key = option.key(); + if (key != null && key.contains(".")) { + section = capitalize(key.split("\\.")[0]); + } + } + sections.computeIfAbsent(section, k -> new ArrayList<>()).add(field); + } + } + + // 2. Generate the HTML for each section + for (Map.Entry> entry : sections.entrySet()) { + builder.append("## ").append(entry.getKey()).append(" Configurations\n\n"); + builder.append("\n") + .append(" \n \n") + .append(" \n") + .append(" \n") + .append(" \n") + .append(" \n") + .append(" \n \n \n"); + + for (Field field : entry.getValue()) { + ConfigOption option = (ConfigOption) field.get(null); + + String defaultValue = ConfigDocUtils.formatDefaultValue(option); + if (field.isAnnotationPresent(ConfigOverrideDefault.class)) { + defaultValue = field.getAnnotation(ConfigOverrideDefault.class).value(); + } + + // ESCAPE DESCRIPTION: Crucial for MDX/React rendering success + // We escape <, >, {, and } which are special JSX characters + String description = + option.description() + .replace("<", "<") + .replace(">", ">") + .replace("{", "{") + .replace("}", "}") + .replace("%s", ""); + + builder.append(" \n") + .append(" \n") + .append(" \n") + .append(" \n") + .append(" \n") + .append(" \n"); + } + builder.append(" \n
KeyDefaultTypeDescription
") + .append(option.key()) + .append("") + .append(defaultValue.replace("<", "<")) + .append("") + .append(getType(option)) + .append("") + .append(description) + .append("
\n\n"); + } + + // Mandatory export for MDX partials to render correctly in Docusaurus + builder.append("export default ({children}) => <>{children};\n"); + + return builder.toString(); + } + + private static String getType(ConfigOption option) { + Object def = option.defaultValue(); + if (def != null) { + return def.getClass().getSimpleName(); + } + return "String"; + } + + private static String capitalize(String str) { + if (str == null || str.isEmpty()) { + return str; + } + return str.substring(0, 1).toUpperCase() + str.substring(1); + } + + private static Path findProjectRoot() { + Path root = Paths.get(System.getProperty("user.dir")); + while (root != null && !Files.exists(root.resolve("pom.xml"))) { + root = root.getParent(); + } + return root; + } +} diff --git a/fluss-docgen/src/test/java/org/apache/fluss/docs/ConfigOptionsDocGeneratorTest.java b/fluss-docgen/src/test/java/org/apache/fluss/docs/ConfigOptionsDocGeneratorTest.java new file mode 100644 index 0000000000..27c1721a23 --- /dev/null +++ b/fluss-docgen/src/test/java/org/apache/fluss/docs/ConfigOptionsDocGeneratorTest.java @@ -0,0 +1,37 @@ +/* + * 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.fluss.docs; + +import org.junit.jupiter.api.Test; + +import java.io.File; + +import static org.assertj.core.api.Assertions.assertThat; + +class ConfigOptionsDocGeneratorTest { + + @Test + void testGeneratorRunsSuccessfully() throws Exception { + // This triggers your generator logic + ConfigOptionsDocGenerator.main(new String[] {}); + + // Verify the file was actually created + File generatedFile = new File("website/docs/maintenance/config_reference.mdx"); + assertThat(generatedFile).exists(); + } +} diff --git a/fluss-docgen/website/docs/maintenance/config_reference.mdx b/fluss-docgen/website/docs/maintenance/config_reference.mdx new file mode 100644 index 0000000000..984bd62ce5 --- /dev/null +++ b/fluss-docgen/website/docs/maintenance/config_reference.mdx @@ -0,0 +1,1493 @@ +{/* This file is auto-generated. Do not edit directly. */} + +## Acl Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
acl.notification.expiration-time15 minDurationThe duration for which ACL notifications are valid before they expire. This configuration determines the time window during which an ACL notification is considered active. After this duration, the notification will no longer be valid and will be discarded. The default value is 15 minutes. This setting is important to ensure that ACL changes are propagated in a timely manner and do not remain active longer than necessary.
+ +## Advertised Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
advertised.listenersnoneStringThe externally advertised address and port for client connections. Required in distributed environments when the bind address is not publicly reachable. Format matches `bind.listeners` (listener_name://host:port). Defaults to the value of `bind.listeners` if not explicitly configured.
+ +## Allow Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
allow.create.log.tablestrueBooleanWhether to allow creation of log tables. When set to false, attempts to create log tables (tables without primary key) will be rejected. The default value is true.
allow.create.kv.tablestrueBooleanWhether to allow creation of kv tables (primary key tables). When set to false, attempts to create kv tables (tables with primary key) will be rejected. The default value is true.
+ +## Authorizer Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
authorizer.enabledfalseBooleanSpecifies whether to enable the authorization feature. If enabled, access control is enforced based on the authorization rules defined in the configuration. If disabled, all operations and resources are accessible to all users.
authorizer.typedefaultStringSpecifies the type of authorizer to be used for access control. This value corresponds to the identifier of the authorization plugin. The default value is `default`, which indicates the built-in authorizer implementation. Custom authorizers can be implemented by providing a matching plugin identifier.
+ +## Auto-partition Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
auto-partition.check.interval10 minDurationThe interval of auto partition check. The default value is 10 minutes.
+ +## Bind Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
bind.listenersnoneStringThe network address and port to which the server binds for accepting connections. This defines the interface and port where the server will listen for incoming requests. The format is `listener_name://host:port`, and multiple addresses can be specified, separated by commas. Use `0.0.0.0` for the `host` to bind to all available interfaces which is dangerous on production and not suggested for production usage. The `listener_name` serves as an identifier for the address in the configuration. For example, `internal.listener.name` specifies the address used for internal server communication. If multiple addresses are configured, ensure that the `listener_name` values are unique.
+ +## Bootstrap Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
bootstrap.serversnoneStringA list of host/port pairs to use for establishing the initial connection to the Fluss cluster. The list should be in the form host1:port1,host2:port2,.... Since these servers are just used for the initial connection to discover the full cluster membership (which may change dynamically), this list need not contain the full set of servers (you may want more than one, though, in case a server is down)
+ +## Client Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
client.id(empty)StringAn id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging.
client.connect-timeout2 minDurationThe Netty client connect timeout.
client.writer.buffer.memory-size64 mbMemorySizeThe total bytes of memory the writer can use to buffer internal rows.
client.writer.buffer.page-size128 kbMemorySizeSize of every page in memory buffers (`client.writer.buffer.memory-size`).
client.writer.buffer.per-request-memory-size16 mbMemorySizeThe minimum number of bytes that will be allocated by the writer rounded down to the closes multiple of client.writer.buffer.page-sizeIt must be greater than or equal to client.writer.buffer.page-size. This option allows to allocate memory in batches to have better CPU-cached friendliness due to contiguous segments.
client.writer.buffer.wait-timeout9223372036 sDurationDefines how long the writer will block when waiting for segments to become available.
client.writer.batch-size2 mbMemorySizeThe writer or walBuilder will attempt to batch records together into one batch for the same bucket. This helps performance on both the client and the server.
client.writer.dynamic-batch-size.enabledtrueBooleanControls whether the client writer dynamically adjusts the batch size based on actual write throughput. Enabled by default. With dynamic batch sizing enabled, the writer adapts memory allocation per batch according to historical write sizes for the target table or partition. This ensures better memory utilization and performance under varying throughput conditions. The dynamic batch size is bounded: it will not exceed `client.writer.batch-size`, nor fall below `client.writer.buffer.page-size`.When disabled, the writer uses a fixed batch size (`client.writer.batch-size`) for all batches, this may lead to frequent memory waits and suboptimal write performance if the incoming data rate is inconsistent across partitions.
client.writer.batch-timeout0 sDurationThe writer groups ay rows that arrive in between request sends into a single batched request. Normally this occurs only under load when rows arrive faster than they can be sent out. However in some circumstances the writer may want toreduce the number of requests even under moderate load. This setting accomplishes this by adding a small amount of artificial delay, that is, rather than immediately sending out a row, the writer will wait for up to the given delay to allow other records to be sent so that the sends can be batched together. This can be thought of as analogous to Nagle's algorithm in TCP. This setting gives the upper bound on the delay for batching: once we get client.writer.batch-size worth of rows for a bucket it will be sent immediately regardless of this setting, however if we have fewer than this many bytes accumulated for this bucket we will delay for the specified time waiting for more records to show up.
client.writer.bucket.no-key-assignerSTICKYNoKeyAssignerThe bucket assigner for no key table. For table with bucket key or primary key, we choose a bucket based on a hash of the key. For these table without bucket key and primary key, we can use this option to specify bucket assigner, the candidate assigner is [ROUND_ROBIN, STICKY], the default assigner is STICKY. +ROUND_ROBIN: this strategy will assign the bucket id for the input row by round robin. +STICKY: this strategy will assign new bucket id only if the batch changed in record accumulator, otherwise the bucket id will be the same as the front record.
client.writer.acksallStringThe number of acknowledgments the writer requires the leader to have received before considering a request complete. This controls the durability of records that are sent. The following settings are allowed: +acks=0: If set to 0, then the writer will not wait for any acknowledgment from the server at all. No guarantee can be mode that the server has received the record in this case. +acks=1: This will mean the leader will write the record to its local log but will respond without awaiting full acknowledge the record but before the followers have replicated it then the record will be lost. +acks=-1 (all): This will mean the leader will wait for the full ser of in-sync replicas to acknowledge the record. This guarantees that the record will not be lost as long as at least one in-sync replica remains alive, This is the strongest available guarantee.
client.writer.request-max-size10 mbMemorySizeThe maximum size of a request in bytes. This setting will limit the number of record batches the writer will send in a single request to avoid sending huge requests. Note that this retry is no different than if the writer resent the row upon receiving the error.
client.writer.retries2147483647IntegerSetting a value greater than zero will cause the client to resend any record whose send fails with a potentially transient error.
client.writer.enable-idempotencetrueBooleanEnable idempotence for the writer. When idempotence is enabled, the writer will ensure that exactly one copy of each record is written in the stream. When idempotence is disabled, the writer retries due to server failures, etc., may write duplicates of the retried record in the stream. Note that enabling writer idempotence requires client.writer.retries to be greater than 0, and client.writer.acks must be `all`. +Writer idempotence is enabled by default if no conflicting config are set. If conflicting config are set and writer idempotence is not explicitly enabled, idempotence is disabled. If idempotence is explicitly enabled and conflicting config are set, a ConfigException is thrown
client.writer.max-inflight-requests-per-bucket5IntegerThe maximum number of unacknowledged requests per bucket for writer. This configuration can work only if client.writer.enable-idempotence is set to true. When the number of inflight requests per bucket exceeds this setting, the writer will wait for the inflight requests to complete before sending out new requests.
client.writer.dynamic-create-partition.enabledtrueBooleanWhether enable dynamic create partition for client writer. Enable by default. Dynamic partition strategy refers to creating partitions based on the data being written for partitioned table if the wrote partition don't exists.
client.request-timeout30 sDurationThe timeout for a request to complete. If user set the write ack to -1, this timeout is the max time that delayed write try to complete. The default setting is 30 seconds.
client.scanner.log.check-crctrueBooleanAutomatically check the CRC3 of the read records for LogScanner. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance.
client.scanner.log.max-poll-records500IntegerThe maximum number of records returned in a single call to poll() for LogScanner. Note that this config doesn't impact the underlying fetching behavior. The Scanner will cache the records from each fetch request and returns them incrementally from each poll.
client.security.protocolPLAINTEXTStringThe authentication protocol used to authenticate the client.
client.scanner.log.fetch.max-bytes16 mbMemorySizeThe maximum amount of data the server should return for a fetch request from client. Records are fetched in batches, and if the first record batch in the first non-empty bucket of the fetch is larger than this value, the record batch will still be returned to ensure that the fetch can make progress. As such, this is not a absolute maximum.
client.scanner.log.fetch.max-bytes-for-bucket1 mbMemorySizeThe maximum amount of data the server should return for a table bucket in fetch request from client. Records are fetched in batches, the max bytes size is config by this option.
client.scanner.log.fetch.wait-max-time0 sDurationThe maximum time to wait for enough bytes to be available for a fetch log request from client to response.
client.scanner.log.fetch.min-bytes1 bytesMemorySizeThe minimum bytes expected for each fetch log request from client to response. If not enough bytes, wait up to client.scanner.log.fetch.wait-max-time time to return.
client.lookup.queue-size25600IntegerThe maximum number of pending lookup operations.
client.lookup.max-batch-size128IntegerThe maximum batch size of merging lookup operations to one lookup request.
client.lookup.max-inflight-requests128IntegerThe maximum number of unacknowledged lookup requests for lookup operations.
client.lookup.batch-timeout0 sDurationThe maximum time to wait for the lookup batch to full, if this timeout is reached, the lookup batch will be closed to send.
client.lookup.max-retries2147483647IntegerSetting a value greater than zero will cause the client to resend any lookup request that fails with a potentially transient error.
client.scanner.remote-log.prefetch-num4IntegerThe number of remote log segments to keep in local temp file for LogScanner, which download from remote storage. The default setting is 4.
client.scanner.io.tmpdir/tmp/flussStringLocal directory that is used by client for storing the data files (like kv snapshot, log segment files) to read temporarily
client.remote-file.download-thread-num3IntegerThe number of threads the client uses to download remote files.
client.filesystem.security.token.renewal.backoff1 hoursDurationThe time period how long to wait before retrying to obtain new security tokens for filesystem after a failure.
client.filesystem.security.token.renewal.time-ratio0.75DoubleRatio of the token's expiration time when new credentials for access filesystem should be re-obtained.
client.metrics.enabledfalseBooleanEnable metrics for client. When metrics is enabled, the client will collect metrics and report by the JMX metrics reporter.
client.security.sasl.mechanismPLAINStringSASL mechanism to use for authentication.Currently, we only support plain.
client.security.sasl.jaas.confignoneStringJAAS configuration string for the client. If not provided, uses the JVM option -Djava.security.auth.login.config. +Example: org.apache.fluss.security.auth.sasl.plain.PlainLoginModule required username="admin" password="admin-secret";
client.security.sasl.usernamenoneStringThe password to use for client-side SASL JAAS authentication. This is used when the client connects to the Fluss cluster with SASL authentication enabled. If not provided, the username will be read from the JAAS configuration string specified by `client.security.sasl.jaas.config`.
client.security.sasl.passwordnoneStringThe username to use for client-side SASL JAAS authentication. This is used when the client connects to the Fluss cluster with SASL authentication enabled. If not provided, the password will be read from the JAAS configuration string specified by `client.security.sasl.jaas.config`.
+ +## Coordinator Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
coordinator.hostnoneStringThe config parameter defining the network address to connect to for communication with the coordinator server. If the coordinator server is used as a bootstrap server (discover all the servers in the cluster), the value of this config option should be a static hostname or address.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports
coordinator.port9123StringThe config parameter defining the network port to connect to for communication with the coordinator server. Like coordinator.host, if the coordinator server is used as a bootstrap server (discover all the servers in the cluster), the value of this config option should be a static port. Otherwise, the value can be set to "0" for a dynamic service name resolution. The value accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports
coordinator.io-pool.size10IntegerThe size of the IO thread pool to run blocking operations for coordinator server. This includes discard unnecessary snapshot files. Increase this value if you experience slow unnecessary snapshot files clean. The default value is 10. This option is deprecated. Please use server.io-pool.size instead.
+ +## Data Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
data.dir/tmp/fluss-dataStringThis configuration controls the directory where fluss will store its data. The default value is /tmp/fluss-data
+ +## Datalake Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
datalake.formatnoneStringThe datalake format used by of Fluss to be as lakehouse storage. Currently, supported formats are Paimon, Iceberg, and Lance. In the future, more kinds of data lake format will be supported, such as DeltaLake or Hudi.
+ +## Default Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
default.bucket.number1IntegerThe default number of buckets for a table in Fluss cluster. It's a cluster-level parameter, and all the tables without specifying bucket number in the cluster will use the value as the bucket number.
default.replication.factor1IntegerThe default replication factor for the log of a table in Fluss cluster. It's a cluster-level parameter, and all the tables without specifying replication factor in the cluster will use the value as replication factor.
+ +## Internal Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
internal.listener.nameFLUSSStringThe listener for server internal communication.
+ +## Kafka Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
kafka.enabledfalseBooleanWhether enable fluss kafka. Disabled by default. When this option is set to true, the fluss kafka will be enabled.
kafka.listener.names[KAFKA]ArrayListThe listener names for Kafka wire protocol communication. Support multiple listener names, separated by comma.
kafka.databasekafkaStringThe database for fluss kafka. The default database is `kafka`.
kafka.connection.max-idle-time1 minDurationClose kafka idle connections after the given time specified by this config.
+ +## Kv Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
kv.snapshot.interval10 minDurationThe interval to perform periodic snapshot for kv data. The default setting is 10 minutes.
kv.snapshot.scheduler-thread-num1IntegerThe number of threads that the server uses to schedule snapshot kv data for all the replicas in the server.
kv.snapshot.transfer-thread-num4IntegerThe number of threads the server uses to transfer (download and upload) kv snapshot files. This option is deprecated. Please use server.io-pool.size instead.
kv.snapshot.num-retained1IntegerThe maximum number of completed snapshots to retain.
kv.rocksdb.thread.num2IntegerThe maximum number of concurrent background flush and compaction jobs (per bucket of table). The default value is `2`.
kv.rocksdb.files.open-1IntegerThe maximum number of open files (per bucket of table) that can be used by the DB, `-1` means no limit. The default value is `-1`.
kv.rocksdb.log.max-file-size25 mbMemorySizeThe maximum size of RocksDB's file used for information logging. If the log files becomes larger than this, a new file will be created. If 0, all logs will be written to one log file. The default maximum file size is `25MB`.
kv.rocksdb.log.file-num4IntegerThe maximum number of files RocksDB should keep for information logging (Default setting: 4).
kv.rocksdb.log.dirnoneStringThe directory for RocksDB's information logging files. If empty (Fluss default setting), log files will be in the same directory as the Fluss log. If non-empty, this directory will be used and the data directory's absolute path will be used as the prefix of the log file name. If setting this option as a non-existing location, e.g `/dev/null`, RocksDB will then create the log under its own database folder as before.
kv.rocksdb.log.levelINFO_LEVELInfoLogLevelThe specified information logging level for RocksDB. Candidate log level is [DEBUG_LEVEL, INFO_LEVEL, WARN_LEVEL, ERROR_LEVEL, FATAL_LEVEL, HEADER_LEVEL, NUM_INFO_LOG_LEVELS]. If unset, Fluss will use INFO_LEVEL. Note: RocksDB info logs will not be written to the Fluss's tablet server logs and there is no rolling strategy, unless you configure kv.rocksdb.log.dir, kv.rocksdb.log.max-file-size, and kv.rocksdb.log.file-num accordingly. Without a rolling strategy, it may lead to uncontrolled disk space usage if configured with increased log levels! There is no need to modify the RocksDB log level, unless for troubleshooting RocksDB.
kv.rocksdb.write-batch-size2 mbMemorySizeThe max size of the consumed memory for RocksDB batch write, will flush just based on item count if this config set to 0.
kv.rocksdb.shared-rate-limiter.bytes-per-sec9223372036854775807 bytesMemorySizeThe shared rate limit in bytes per second for RocksDB flush and compaction operations across all RocksDB instances in the TabletServer. All KV tablets share a single global RateLimiter to prevent disk IO from being saturated. The RateLimiter is always enabled. The default value is Long.MAX_VALUE (effectively unlimited). Set to a lower value (e.g., 100MB) to limit the rate.
kv.rocksdb.compaction.styleLEVELCompactionStyleThe specified compaction style for DB. Candidate compaction style is LEVEL, FIFO, UNIVERSAL or NONE, and Fluss chooses `LEVEL` as default style.
kv.rocksdb.compaction.level.use-dynamic-sizefalseBooleanIf true, RocksDB will pick target size of each level dynamically. From an empty DB, RocksDB would make last level the base level, which means merging L0 data into the last level, until it exceeds max_bytes_for_level_base. And then repeat this process for second last level and so on. The default value is `false`. For more information, please refer to https://github.com/facebook/rocksdb/wiki/Leveled-Compaction#level_compaction_dynamic_level_bytes-is-trueRocksDB's doc.
kv.rocksdb.compression.per.level[LZ4, LZ4, LZ4, LZ4, LZ4, ZSTD, ZSTD]ArrayListA comma-separated list of Compression Type. Different levels can have different compression policies. In many cases, lower levels use fast compression algorithms, while higher levels with more data use slower but more effective compression algorithms. The N th element in the List corresponds to the compression type of the level N-1When `kv.rocksdb.compaction.level.use-dynamic-size` is true, compression_per_level[0] still determines L0, but other elements are based on the base level and may not match the level seen in the info log. Note: If the List size is smaller than the level number, the undefined lower level uses the last Compression Type in the List. The optional values include NO, SNAPPY, LZ4, ZSTD. For more information about compression type, please refer to doc https://github.com/facebook/rocksdb/wiki/Compression. The default value is ‘LZ4,LZ4,LZ4,LZ4,LZ4,ZSTD,ZSTD’, indicates there is lz4 compaction of level0 and level4,ZSTD compaction algorithm is used from level5 to level6. LZ4 is a lightweight compression algorithm so it usually strikes a good balance between space and CPU usage. ZSTD is more space save than LZ4, but it is more CPU-intensive. Different machines deploy compaction modes according to CPU and I/O resources. The default value is for the scenario that CPU resources are adequate. If you find the IO pressure of the system is not big when writing a lot of data, but CPU resources are inadequate, you can exchange I/O resources for CPU resources and change the compaction mode to `NO,NO,NO,LZ4,LZ4,ZSTD,ZSTD`.
kv.rocksdb.compaction.level.target-file-size-base64 mbMemorySizeThe target file size for compaction, which determines a level-1 file size. The default value is `64MB`.
kv.rocksdb.compaction.level.max-size-level-base256 mbMemorySizeThe upper-bound of the total size of level base files in bytes. The default value is `256MB`.
kv.rocksdb.writebuffer.size64 mbMemorySizeThe amount of data built up in memory (backed by an unsorted log on disk) before converting to a sorted on-disk files. The default writebuffer size is `64MB`.
kv.rocksdb.writebuffer.count2IntegerThe maximum number of write buffers that are built up in memory. The default value is `2`.
kv.rocksdb.writebuffer.number-to-merge1IntegerThe minimum number of write buffers that will be merged together before writing to storage. The default value is `1`.
kv.rocksdb.block.blocksize4 kbMemorySizeThe approximate size (in bytes) of user data packed per block. The default blocksize is `4KB`.
kv.rocksdb.block.metadata-blocksize4 kbMemorySizeApproximate size of partitioned metadata packed per block. Currently applied to indexes block when partitioned index/filters option is enabled. The default blocksize is `4KB`.
kv.rocksdb.block.cache-size8 mbMemorySizeThe amount of the cache for data blocks in RocksDB. The default block-cache size is `8MB`.
kv.rocksdb.use-bloom-filtertrueBooleanIf true, every newly created SST file will contain a Bloom filter. It is enabled by default.
kv.rocksdb.bloom-filter.bits-per-key10.0DoubleBits per key that bloom filter will use, this only take effect when bloom filter is used. The default value is 10.0.
kv.rocksdb.bloom-filter.block-based-modefalseBooleanIf true, RocksDB will use block-based filter instead of full filter, this only take effect when bloom filter is used. The default value is `false`.
kv.rocksdb.block.cache-index-and-filter-blocksfalseBooleanIf true, index and filter blocks will be stored in block cache, together with all other data blocks. This helps to limit memory usage so that the total memory used by RocksDB is bounded by block cache size. The default value is `false`.
kv.rocksdb.block.cache-index-and-filter-blocks-with-high-priorityfalseBooleanIf true and cache_index_and_filter_blocks is enabled, index and filter blocks will be stored with high priority in block cache, making them less likely to be evicted than data blocks. The default value is `false`.
kv.rocksdb.block.pin-l0-filter-and-index-blocks-in-cachefalseBooleanIf true and cache_index_and_filter_blocks is enabled, L0 index and filter blocks will be pinned in block cache and will not be evicted. This helps avoid performance degradation due to cache misses on L0 index/filter blocks. The default value is `false`.
kv.rocksdb.block.pin-top-level-index-and-filterfalseBooleanIf true, the top-level index of partitioned index/filter blocks will be pinned in block cache and will not be evicted. The default value is `false`.
kv.recover.log-record-batch.max-size16 mbMemorySizeThe max fetch size for fetching log to apply to kv during recovering kv.
+ +## Lake Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
lake.tiering.auto-expire-snapshotfalseBooleanIf true, snapshot expiration will be triggered automatically when tiering service commits to the datalake, even if Key: 'table.datalake.auto-expire-snapshot' , default: false (fallback keys: []) is false.
+ +## Log Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
log.segment.file-size1 gbMemorySizeThis configuration controls the segment file size for the log. Retention and cleaning is always done a file at a time so a larger segment size means fewer files but less granular control over retention.
log.index.file-size10 mbMemorySizeThis configuration controls the size of the index that maps offsets to file positions. We preallocate this index file and shrink it only after log rolls. You generally should not need to change this setting.
log.index.interval-size4 kbMemorySizeThis setting controls how frequently fluss adds an index entry to its offset index. The default setting ensures that we index a message roughly every 4096 bytes. More indexing allows reads to jump closer to the exact position in the log but makes the index larger. You probably don't need to change this.
log.file-preallocatefalseBooleanTrue if we should preallocate the file on disk when creating a new log segment.
log.flush.interval-messages9223372036854775807LongThis setting allows specifying an interval at which we will force a fsync of data written to the log. For example if this was set to 1, we would fsync after every message; if it were 5 we would fsync after every five messages.
log.replica.high-watermark.checkpoint-interval5 sDurationThe frequency with which the high watermark is saved out to disk. The default setting is 5 seconds.
log.replica.max-lag-time30 sDurationIf a follower replica hasn't sent any fetch log requests or hasn't consumed up the leaders log end offset for at least this time, the leader will remove the follower replica form isr
log.replica.write-operation-purge-number1000IntegerThe purge number (in number of requests) of the write operation manager, the default value is 1000.
log.replica.fetch-operation-purge-number1000IntegerThe purge number (in number of requests) of the fetch log operation manager, the default value is 1000.
log.replica.fetcher-number1IntegerNumber of fetcher threads used to replicate log records from each source tablet server. The total number of fetchers on each tablet server is bound by this parameter multiplied by the number of tablet servers in the cluster. Increasing this value can increase the degree of I/O parallelism in the follower and leader tablet server at the cost of higher CPU and memory utilization.
log.replica.fetch.backoff-interval1 sDurationThe amount of time to sleep when fetch bucket error occurs.
log.replica.fetch.max-bytes16 mbMemorySizeThe maximum amount of data the server should return for a fetch request from follower. Records are fetched in batches, and if the first record batch in the first non-empty bucket of the fetch is larger than this value, the record batch will still be returned to ensure that the fetch can make progress. As such, this is not a absolute maximum. Note that the fetcher performs multiple fetches in parallel.
log.replica.fetch.max-bytes-for-bucket1 mbMemorySizeThe maximum amount of data the server should return for a table bucket in fetch request from follower. Records are fetched in batches, the max bytes size is config by this option.
log.replica.fetch.wait-max-time0 sDurationThe maximum time to wait for enough bytes to be available for a fetch log request from follower to response. This value should always be less than the `log.replica.max-lag-time` at all times to prevent frequent shrinking of ISR for low throughput tables
log.replica.fetch.min-bytes1 bytesMemorySizeThe minimum bytes expected for each fetch log request from follower to response. If not enough bytes, wait up to log.replica.fetch.wait-max-time time to return.
log.replica.min-in-sync-replicas-number1IntegerWhen a writer set `client.writer.acks` to all (-1), this configuration specifies the minimum number of replicas that must acknowledge a write for the write to be considered successful. If this minimum cannot be met, then the writer will raise an exception (NotEnoughReplicas). when used together, this config and `client.writer.acks` allow you to enforce greater durability guarantees. A typical scenario would be to create a table with a replication factor of 3. set this conf to 2, and write with acks = -1. This will ensure that the writer raises an exception if a majority of replicas don't receive a write.
+ +## Max Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
max.partition.num1000IntegerLimits the maximum number of partitions that can be created for a partitioned table to avoid creating too many partitions.
max.bucket.num128000IntegerThe maximum number of buckets that can be created for a table.The default value is 128000
+ +## Metrics Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
metrics.reportersnoneStringAn optional list of reporter names. If configured, only reporters whose name matches in the list will be started
metrics.reporter.prometheus.port9249StringThe port the Prometheus reporter listens on.In order to be able to run several instances of the reporter on one host (e.g. when one TabletServer is colocated with the CoordinatorServer) it is advisable to use a port range like 9250-9260.
metrics.reporter.jmx.portnoneStringThe port for the JMXServer that JMX clients can connect to. If not set, the JMXServer won't start. In order to be able to run several instances of the reporter on one host (e.g. when one TabletServer is colocated with the CoordinatorServer) it is advisable to use a port range like 9990-9999.
+ +## Netty Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
netty.server.num-network-threads3IntegerThe number of threads that the server uses for receiving requests from the network and sending responses to the network.
netty.server.num-worker-threads8IntegerThe number of threads that the server uses for processing requests, which may include disk and remote I/O.
netty.server.max-queued-requests500IntegerThe number of queued requests allowed for worker threads, before blocking the I/O threads.
netty.connection.max-idle-time10 minDurationClose idle connections after the given time specified by this config.
netty.client.num-network-threads4IntegerThe number of threads that the client uses for sending requests to the network and receiving responses from network. The default value is 4
+ +## Plugin Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
plugin.classloader.parent-first-patterns.default[java., org.apache.fluss., javax.annotation., org.apache.hadoop., core-site, org.slf4j, org.apache.log4j, org.apache.logging, org.apache.commons.logging, ch.qos.logback]ArrayListA (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. This setting should generally not be modified. To add another pattern we recommend to use "plugin.classloader.parent-first-patterns.additional" instead.
plugin.classloader.parent-first-patterns.additional[]ArrayListA (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. These patterns are appended to "plugin.classloader.parent-first-patterns.default".
+ +## Remote Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
remote.data.dirnoneStringThe directory used for storing the kv snapshot data files and remote log for log tiered storage in a Fluss supported filesystem.
remote.fs.write-buffer-size4 kbMemorySizeThe default size of the write buffer for writing the local files to remote file systems.
remote.log.task-interval-duration1 minDurationInterval at which remote log manager runs the scheduled tasks like copy segments, clean up remote log segments, delete local log segments etc. If the value is set to 0, it means that the remote log storage is disabled.
remote.log.index-file-cache-size1 gbMemorySizeThe total size of the space allocated to store index files fetched from remote storage in the local storage.
remote.log-manager.thread-pool-size4IntegerSize of the thread pool used in scheduling tasks to copy segments, fetch remote log indexes and clean up remote log segments.
remote.log.data-transfer-thread-num4IntegerThe number of threads the server uses to transfer (download and upload) remote log file can be data file, index file and remote log metadata file. This option is deprecated. Please use server.io-pool.size instead.
+ +## Security Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
security.sasl.enabled.mechanismsnoneString
security.protocol.map{}EmptyMapA map defining the authentication protocol for each listener. The format is `listenerName1:protocol1,listenerName2:protocol2`, e.g., `INTERNAL:PLAINTEXT,CLIENT:GSSAPI`. Each listener can be associated with a specific authentication protocol. Listeners not included in the map will use PLAINTEXT by default, which does not require authentication.
+ +## Server Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
server.io-pool.size10IntegerThe size of the IO thread pool to run blocking operations for both coordinator and tablet servers. This includes discard unnecessary snapshot files, transfer kv snapshot files, and transfer remote log files. Increase this value if you experience slow IO operations. The default value is 10.
server.writer-id.expiration-time168 hoursDurationThe time that the tablet server will wait without receiving any write request from a client before expiring the related status. The default value is 7 days.
server.writer-id.expiration-check-interval10 minDurationThe interval at which to remove writer ids that have expired due to server.writer-id.expiration-time passing. The default value is 10 minutes.
server.background.threads10IntegerThe number of threads to use for various background processing tasks.
server.buffer.memory-size256 mbMemorySizeThe total bytes of memory the server can use, e.g, buffer write-ahead-log rows.
server.buffer.page-size128 kbMemorySizeSize of every page in memory buffers (`server.buffer.memory-size`).
server.buffer.per-request-memory-size16 mbMemorySizeThe minimum number of bytes that will be allocated by the writer rounded down to the closes multiple of server.buffer.page-sizeIt must be greater than or equal to server.buffer.page-size. This option allows to allocate memory in batches to have better CPU-cached friendliness due to contiguous segments.
server.buffer.wait-timeout9223372036 sDurationDefines how long the buffer pool will block when waiting for segments to become available.
+ +## Super Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
super.usersnoneStringA semicolon-separated list of superusers who have unrestricted access to all operations and resources. Note that the delimiter is semicolon since SSL user names may contain comma, and each super user should be specified in the format `principal_type:principal_name`, e.g., `User:admin;User:bob`. This configuration is critical for defining administrative privileges in the system.
+ +## Table Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
table.replication.factornoneStringThe replication factor for the log of the new table. When it's not set, Fluss will use the cluster's default replication factor configured by default.replication.factor. It should be a positive number and not larger than the number of tablet servers in the Fluss cluster. A value larger than the number of tablet servers in Fluss cluster will result in an error when the new table is created.
table.log.formatARROWLogFormatThe format of the log records in log store. The default value is `arrow`. The supported formats are `arrow`, `indexed` and `compacted`.
table.log.arrow.compression.typeZSTDArrowCompressionTypeThe compression type of the log records if the log format is set to `ARROW`. The candidate compression type is [NONE, LZ4_FRAME, ZSTD]
table.log.arrow.compression.zstd.level3IntegerThe compression level of ZSTD for the log records if the log format is set to `ARROW` and the compression type is set to `ZSTD`. The valid range is 1 to 22.
table.kv.formatCOMPACTEDKvFormatThe format of the kv records in kv store. The default value is `compacted`. The supported formats are `compacted` and `indexed`.
table.auto-partition.enabledfalseBooleanWhether enable auto partition for the table. Disable by default. When auto partition is enabled, the partitions of the table will be created automatically.
table.auto-partition.keynoneStringThis configuration defines the time-based partition key to be used for auto-partitioning when a table is partitioned with multiple keys. Auto-partitioning utilizes a time-based partition key to handle partitions automatically, including creating new ones and removing outdated ones, by comparing the time value of the partition with the current system time. In the case of a table using multiple partition keys (such as a composite partitioning strategy), this feature determines which key should serve as the primary time dimension for making auto-partitioning decisions.And If the table has only one partition key, this config is not necessary. Otherwise, it must be specified.
table.auto-partition.time-unitDAYAutoPartitionTimeUnitThe time granularity for auto created partitions. The default value is `DAY`. Valid values are `HOUR`, `DAY`, `MONTH`, `QUARTER`, `YEAR`. If the value is `HOUR`, the partition format for auto created is yyyyMMddHH. If the value is `DAY`, the partition format for auto created is yyyyMMdd. If the value is `MONTH`, the partition format for auto created is yyyyMM. If the value is `QUARTER`, the partition format for auto created is yyyyQ. If the value is `YEAR`, the partition format for auto created is yyyy.
table.auto-partition.time-zoneEtc/UTCStringThe time zone for auto partitions, which is by default the same as the system time zone.
table.auto-partition.num-precreate2IntegerThe number of partitions to pre-create for auto created partitions in each check for auto partition. For example, if the current check time is 2024-11-11 and the value is configured as 3, then partitions 20241111, 20241112, 20241113 will be pre-created. If any one partition exists, it'll skip creating the partition. The default value is 2, which means 2 partitions will be pre-created. If the `table.auto-partition.time-unit` is `DAY`(default), one precreated partition is for today and another one is for tomorrow.For a partition table with multiple partition keys, pre-create is unsupported and will be set to 0 automatically when creating table if it is not explicitly specified.
table.auto-partition.num-retention7IntegerThe number of history partitions to retain for auto created partitions in each check for auto partition. For example, if the current check time is 2024-11-11, time-unit is DAY, and the value is configured as 3, then the history partitions 20241108, 20241109, 20241110 will be retained. The partitions earlier than 20241108 will be deleted. The default value is 7.
table.log.ttl168 hoursDurationThe time to live for log segments. The configuration controls the maximum time we will retain a log before we will delete old segments to free up space. If set to -1, the log will not be deleted.
table.log.tiered.local-segments2IntegerThe number of log segments to retain in local for each table when log tiered storage is enabled. It must be greater that 0. The default is 2.
table.datalake.enabledfalseBooleanWhether enable lakehouse storage for the table. Disabled by default. When this option is set to ture and the datalake tiering service is up, the table will be tiered and compacted into datalake format stored on lakehouse storage.
table.datalake.formatnoneStringThe data lake format of the table specifies the tiered Lakehouse storage format. Currently, supported formats are `paimon`, `iceberg`, and `lance`. In the future, more kinds of data lake format will be supported, such as DeltaLake or Hudi. Once the `table.datalake.format` property is configured, Fluss adopts the key encoding and bucketing strategy used by the corresponding data lake format. This ensures consistency in key encoding and bucketing, enabling seamless **Union Read** functionality across Fluss and Lakehouse. The `table.datalake.format` can be pre-defined before enabling `table.datalake.enabled`. This allows the data lake feature to be dynamically enabled on the table without requiring table recreation. If `table.datalake.format` is not explicitly set during table creation, the table will default to the format specified by the `datalake.format` configuration in the Fluss cluster.
table.datalake.freshness3 minDurationIt defines the maximum amount of time that the datalake table's content should lag behind updates to the Fluss table. Based on this target freshness, the Fluss service automatically moves data from the Fluss table and updates to the datalake table, so that the data in the datalake table is kept up to date within this target. If the data does not need to be as fresh, you can specify a longer target freshness time to reduce costs.
table.datalake.auto-compactionfalseBooleanIf true, compaction will be triggered automatically when tiering service writes to the datalake. It is disabled by default.
table.datalake.auto-expire-snapshotfalseBooleanIf true, snapshot expiration will be triggered automatically when tiering service commits to the datalake. It is disabled by default.
table.merge-enginenoneStringDefines the merge engine for the primary key table. By default, primary key table doesn't have merge engine. The supported merge engines are `first_row`, `versioned`, and `aggregation`. The `first_row` merge engine will keep the first row of the same primary key. The `versioned` merge engine will keep the row with the largest version of the same primary key. The `aggregation` merge engine will aggregate rows with the same primary key using field-level aggregate functions.
table.merge-engine.versioned.ver-columnnoneStringThe column name of the version column for the `versioned` merge engine. If the merge engine is set to `versioned`, the version column must be set.
table.delete.behaviorALLOWDeleteBehaviorDefines the delete behavior for the primary key table. The supported delete behaviors are `allow`, `ignore`, and `disable`. The `allow` behavior allows normal delete operations (default for default merge engine). The `ignore` behavior silently skips delete requests without error. The `disable` behavior rejects delete requests with a clear error message. For tables with FIRST_ROW, VERSIONED, or AGGREGATION merge engines, this option defaults to `ignore`. Note: For AGGREGATION merge engine, when set to `allow`, delete operations will remove the entire record.
table.auto-increment.cache-size100000LongThe cache size of auto-increment IDs fetched from the distributed counter each time. This value determines the length of the locally cached ID segment. Default: 100000. A larger cache size may cause significant auto-increment ID gaps, especially when unused cached ID segments are discarded due to TabletServer restarts or abnormal terminations. Conversely, a smaller cache size increases the frequency of ID fetch requests to the distributed counter, introducing extra network overhead and reducing write throughput and performance.
table.changelog.imageFULLChangelogImageDefines the changelog image mode for the primary key table. This configuration is inspired by similar settings in database systems like MySQL's binlog_row_image and PostgreSQL's replica identity. The supported modes are `FULL` (default) and `WAL`. The `FULL` mode produces both UPDATE_BEFORE and UPDATE_AFTER records for update operations, capturing complete information about updates and allowing tracking of previous values. The `WAL` mode does not produce UPDATE_BEFORE records. Only INSERT, UPDATE_AFTER (and DELETE if allowed) records are emitted. When WAL mode is enabled, the default merge engine is used (no merge engine configured), updates are full row updates (not partial update), and there is no auto-increment column, an optimization is applied to skip looking up old values, and in this case INSERT operations are converted to UPDATE_AFTER events. This mode reduces storage and transmission costs but loses the ability to track previous values. This option only affects primary key tables.
+ +## Tablet-server Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
tablet-server.hostnoneStringThe external address of the network interface where the TabletServer is exposed. Because different TabletServer need different values for this option, usually it is specified in an additional non-shared TabletServer-specific config file.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports
tablet-server.port0StringThe external RPC port where the TabletServer is exposed.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports
tablet-server.idnoneStringThe id for the tablet server.
tablet-server.racknoneStringThe rack for the tabletServer. This will be used in rack aware bucket assignment for fault tolerance. Examples: `RACK1`, `cn-hangzhou-server10`
tablet-server.controlled-shutdown.max-retries3IntegerThe maximum number of retries for controlled shutdown of the tablet server. During controlled shutdown, the tablet server attempts to transfer leadership of its buckets to other servers. If the transfer fails, it will retry up to this number of times before proceeding with shutdown. The default value is 3.
tablet-server.controlled-shutdown.retry-interval1 sDurationThe interval between retries during controlled shutdown of the tablet server. When controlled shutdown fails to transfer bucket leadership, the tablet server will wait for this duration before attempting the next retry. The default value is 1000 milliseconds (1 second).
+ +## Zookeeper Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
zookeeper.addressnoneStringThe ZooKeeper address to use, when running Fluss with ZooKeeper.
zookeeper.path.root/flussStringThe root path under which Fluss stores its entries in ZooKeeper.
zookeeper.client.max-inflight-requests100IntegerThe maximum number of unacknowledged requests the client will send to ZooKeeper before blocking.
zookeeper.client.session-timeout1 minDurationDefines the session timeout for the ZooKeeper session.
zookeeper.client.connection-timeout15 sDurationDefines the connection timeout for ZooKeeper.
zookeeper.client.retry-wait5 sDurationDefines the pause between consecutive retries.
zookeeper.client.max-retry-attempts3IntegerDefines the number of connection retries before the client gives up.
zookeeper.client.tolerate-suspended-connectionsfalseBooleanDefines whether a suspended ZooKeeper connection will be treated as an error that causes the leader information to be invalidated or not. In case you set this option to , Fluss will wait until a ZooKeeper connection is marked as lost before it revokes the leadership of components. This has the effect that Fluss is more resilient against temporary connection instabilities at the cost of running more likely into timing issues with ZooKeeper.
zookeeper.client.ensemble-trackertrueBooleanDefines whether Curator should enable ensemble tracker. This can be useful in certain scenarios in which CuratorFramework is accessing to ZK clusters via load balancer or Virtual IPs. Default Curator EnsembleTracking logic watches CuratorEventType.GET_CONFIG events and changes ZooKeeper connection string. It is not desired behaviour when ZooKeeper is running under the Virtual IPs. Under certain configurations EnsembleTracking can lead to setting of ZooKeeper connection string with unresolvable hostnames.
zookeeper.client.config-pathnoneStringThe file path from which the ZooKeeper client reads its configuration. This allows each ZooKeeper client instance to load its own configuration file, instead of relying on shared JVM-level environment settings. This enables fine-grained control over ZooKeeper client behavior.
zookeeper.client.max-buffer-size104857600IntegerThe maximum buffer size (in bytes) for ZooKeeper client. This corresponds to the jute.maxbuffer property. Default is 100MB to match the RPC frame length limit.
+ +export default ({children}) => <>{children}; + diff --git a/pom.xml b/pom.xml index 07b7f90031..4abcce3489 100644 --- a/pom.xml +++ b/pom.xml @@ -67,6 +67,7 @@ fluss-lake fluss-kafka tools/ci/fluss-ci-tools + fluss-docgen diff --git a/website/docs/maintenance/config_reference.mdx b/website/docs/maintenance/config_reference.mdx new file mode 100644 index 0000000000..984bd62ce5 --- /dev/null +++ b/website/docs/maintenance/config_reference.mdx @@ -0,0 +1,1493 @@ +{/* This file is auto-generated. Do not edit directly. */} + +## Acl Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
acl.notification.expiration-time15 minDurationThe duration for which ACL notifications are valid before they expire. This configuration determines the time window during which an ACL notification is considered active. After this duration, the notification will no longer be valid and will be discarded. The default value is 15 minutes. This setting is important to ensure that ACL changes are propagated in a timely manner and do not remain active longer than necessary.
+ +## Advertised Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
advertised.listenersnoneStringThe externally advertised address and port for client connections. Required in distributed environments when the bind address is not publicly reachable. Format matches `bind.listeners` (listener_name://host:port). Defaults to the value of `bind.listeners` if not explicitly configured.
+ +## Allow Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
allow.create.log.tablestrueBooleanWhether to allow creation of log tables. When set to false, attempts to create log tables (tables without primary key) will be rejected. The default value is true.
allow.create.kv.tablestrueBooleanWhether to allow creation of kv tables (primary key tables). When set to false, attempts to create kv tables (tables with primary key) will be rejected. The default value is true.
+ +## Authorizer Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
authorizer.enabledfalseBooleanSpecifies whether to enable the authorization feature. If enabled, access control is enforced based on the authorization rules defined in the configuration. If disabled, all operations and resources are accessible to all users.
authorizer.typedefaultStringSpecifies the type of authorizer to be used for access control. This value corresponds to the identifier of the authorization plugin. The default value is `default`, which indicates the built-in authorizer implementation. Custom authorizers can be implemented by providing a matching plugin identifier.
+ +## Auto-partition Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
auto-partition.check.interval10 minDurationThe interval of auto partition check. The default value is 10 minutes.
+ +## Bind Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
bind.listenersnoneStringThe network address and port to which the server binds for accepting connections. This defines the interface and port where the server will listen for incoming requests. The format is `listener_name://host:port`, and multiple addresses can be specified, separated by commas. Use `0.0.0.0` for the `host` to bind to all available interfaces which is dangerous on production and not suggested for production usage. The `listener_name` serves as an identifier for the address in the configuration. For example, `internal.listener.name` specifies the address used for internal server communication. If multiple addresses are configured, ensure that the `listener_name` values are unique.
+ +## Bootstrap Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
bootstrap.serversnoneStringA list of host/port pairs to use for establishing the initial connection to the Fluss cluster. The list should be in the form host1:port1,host2:port2,.... Since these servers are just used for the initial connection to discover the full cluster membership (which may change dynamically), this list need not contain the full set of servers (you may want more than one, though, in case a server is down)
+ +## Client Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
client.id(empty)StringAn id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging.
client.connect-timeout2 minDurationThe Netty client connect timeout.
client.writer.buffer.memory-size64 mbMemorySizeThe total bytes of memory the writer can use to buffer internal rows.
client.writer.buffer.page-size128 kbMemorySizeSize of every page in memory buffers (`client.writer.buffer.memory-size`).
client.writer.buffer.per-request-memory-size16 mbMemorySizeThe minimum number of bytes that will be allocated by the writer rounded down to the closes multiple of client.writer.buffer.page-sizeIt must be greater than or equal to client.writer.buffer.page-size. This option allows to allocate memory in batches to have better CPU-cached friendliness due to contiguous segments.
client.writer.buffer.wait-timeout9223372036 sDurationDefines how long the writer will block when waiting for segments to become available.
client.writer.batch-size2 mbMemorySizeThe writer or walBuilder will attempt to batch records together into one batch for the same bucket. This helps performance on both the client and the server.
client.writer.dynamic-batch-size.enabledtrueBooleanControls whether the client writer dynamically adjusts the batch size based on actual write throughput. Enabled by default. With dynamic batch sizing enabled, the writer adapts memory allocation per batch according to historical write sizes for the target table or partition. This ensures better memory utilization and performance under varying throughput conditions. The dynamic batch size is bounded: it will not exceed `client.writer.batch-size`, nor fall below `client.writer.buffer.page-size`.When disabled, the writer uses a fixed batch size (`client.writer.batch-size`) for all batches, this may lead to frequent memory waits and suboptimal write performance if the incoming data rate is inconsistent across partitions.
client.writer.batch-timeout0 sDurationThe writer groups ay rows that arrive in between request sends into a single batched request. Normally this occurs only under load when rows arrive faster than they can be sent out. However in some circumstances the writer may want toreduce the number of requests even under moderate load. This setting accomplishes this by adding a small amount of artificial delay, that is, rather than immediately sending out a row, the writer will wait for up to the given delay to allow other records to be sent so that the sends can be batched together. This can be thought of as analogous to Nagle's algorithm in TCP. This setting gives the upper bound on the delay for batching: once we get client.writer.batch-size worth of rows for a bucket it will be sent immediately regardless of this setting, however if we have fewer than this many bytes accumulated for this bucket we will delay for the specified time waiting for more records to show up.
client.writer.bucket.no-key-assignerSTICKYNoKeyAssignerThe bucket assigner for no key table. For table with bucket key or primary key, we choose a bucket based on a hash of the key. For these table without bucket key and primary key, we can use this option to specify bucket assigner, the candidate assigner is [ROUND_ROBIN, STICKY], the default assigner is STICKY. +ROUND_ROBIN: this strategy will assign the bucket id for the input row by round robin. +STICKY: this strategy will assign new bucket id only if the batch changed in record accumulator, otherwise the bucket id will be the same as the front record.
client.writer.acksallStringThe number of acknowledgments the writer requires the leader to have received before considering a request complete. This controls the durability of records that are sent. The following settings are allowed: +acks=0: If set to 0, then the writer will not wait for any acknowledgment from the server at all. No guarantee can be mode that the server has received the record in this case. +acks=1: This will mean the leader will write the record to its local log but will respond without awaiting full acknowledge the record but before the followers have replicated it then the record will be lost. +acks=-1 (all): This will mean the leader will wait for the full ser of in-sync replicas to acknowledge the record. This guarantees that the record will not be lost as long as at least one in-sync replica remains alive, This is the strongest available guarantee.
client.writer.request-max-size10 mbMemorySizeThe maximum size of a request in bytes. This setting will limit the number of record batches the writer will send in a single request to avoid sending huge requests. Note that this retry is no different than if the writer resent the row upon receiving the error.
client.writer.retries2147483647IntegerSetting a value greater than zero will cause the client to resend any record whose send fails with a potentially transient error.
client.writer.enable-idempotencetrueBooleanEnable idempotence for the writer. When idempotence is enabled, the writer will ensure that exactly one copy of each record is written in the stream. When idempotence is disabled, the writer retries due to server failures, etc., may write duplicates of the retried record in the stream. Note that enabling writer idempotence requires client.writer.retries to be greater than 0, and client.writer.acks must be `all`. +Writer idempotence is enabled by default if no conflicting config are set. If conflicting config are set and writer idempotence is not explicitly enabled, idempotence is disabled. If idempotence is explicitly enabled and conflicting config are set, a ConfigException is thrown
client.writer.max-inflight-requests-per-bucket5IntegerThe maximum number of unacknowledged requests per bucket for writer. This configuration can work only if client.writer.enable-idempotence is set to true. When the number of inflight requests per bucket exceeds this setting, the writer will wait for the inflight requests to complete before sending out new requests.
client.writer.dynamic-create-partition.enabledtrueBooleanWhether enable dynamic create partition for client writer. Enable by default. Dynamic partition strategy refers to creating partitions based on the data being written for partitioned table if the wrote partition don't exists.
client.request-timeout30 sDurationThe timeout for a request to complete. If user set the write ack to -1, this timeout is the max time that delayed write try to complete. The default setting is 30 seconds.
client.scanner.log.check-crctrueBooleanAutomatically check the CRC3 of the read records for LogScanner. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance.
client.scanner.log.max-poll-records500IntegerThe maximum number of records returned in a single call to poll() for LogScanner. Note that this config doesn't impact the underlying fetching behavior. The Scanner will cache the records from each fetch request and returns them incrementally from each poll.
client.security.protocolPLAINTEXTStringThe authentication protocol used to authenticate the client.
client.scanner.log.fetch.max-bytes16 mbMemorySizeThe maximum amount of data the server should return for a fetch request from client. Records are fetched in batches, and if the first record batch in the first non-empty bucket of the fetch is larger than this value, the record batch will still be returned to ensure that the fetch can make progress. As such, this is not a absolute maximum.
client.scanner.log.fetch.max-bytes-for-bucket1 mbMemorySizeThe maximum amount of data the server should return for a table bucket in fetch request from client. Records are fetched in batches, the max bytes size is config by this option.
client.scanner.log.fetch.wait-max-time0 sDurationThe maximum time to wait for enough bytes to be available for a fetch log request from client to response.
client.scanner.log.fetch.min-bytes1 bytesMemorySizeThe minimum bytes expected for each fetch log request from client to response. If not enough bytes, wait up to client.scanner.log.fetch.wait-max-time time to return.
client.lookup.queue-size25600IntegerThe maximum number of pending lookup operations.
client.lookup.max-batch-size128IntegerThe maximum batch size of merging lookup operations to one lookup request.
client.lookup.max-inflight-requests128IntegerThe maximum number of unacknowledged lookup requests for lookup operations.
client.lookup.batch-timeout0 sDurationThe maximum time to wait for the lookup batch to full, if this timeout is reached, the lookup batch will be closed to send.
client.lookup.max-retries2147483647IntegerSetting a value greater than zero will cause the client to resend any lookup request that fails with a potentially transient error.
client.scanner.remote-log.prefetch-num4IntegerThe number of remote log segments to keep in local temp file for LogScanner, which download from remote storage. The default setting is 4.
client.scanner.io.tmpdir/tmp/flussStringLocal directory that is used by client for storing the data files (like kv snapshot, log segment files) to read temporarily
client.remote-file.download-thread-num3IntegerThe number of threads the client uses to download remote files.
client.filesystem.security.token.renewal.backoff1 hoursDurationThe time period how long to wait before retrying to obtain new security tokens for filesystem after a failure.
client.filesystem.security.token.renewal.time-ratio0.75DoubleRatio of the token's expiration time when new credentials for access filesystem should be re-obtained.
client.metrics.enabledfalseBooleanEnable metrics for client. When metrics is enabled, the client will collect metrics and report by the JMX metrics reporter.
client.security.sasl.mechanismPLAINStringSASL mechanism to use for authentication.Currently, we only support plain.
client.security.sasl.jaas.confignoneStringJAAS configuration string for the client. If not provided, uses the JVM option -Djava.security.auth.login.config. +Example: org.apache.fluss.security.auth.sasl.plain.PlainLoginModule required username="admin" password="admin-secret";
client.security.sasl.usernamenoneStringThe password to use for client-side SASL JAAS authentication. This is used when the client connects to the Fluss cluster with SASL authentication enabled. If not provided, the username will be read from the JAAS configuration string specified by `client.security.sasl.jaas.config`.
client.security.sasl.passwordnoneStringThe username to use for client-side SASL JAAS authentication. This is used when the client connects to the Fluss cluster with SASL authentication enabled. If not provided, the password will be read from the JAAS configuration string specified by `client.security.sasl.jaas.config`.
+ +## Coordinator Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
coordinator.hostnoneStringThe config parameter defining the network address to connect to for communication with the coordinator server. If the coordinator server is used as a bootstrap server (discover all the servers in the cluster), the value of this config option should be a static hostname or address.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports
coordinator.port9123StringThe config parameter defining the network port to connect to for communication with the coordinator server. Like coordinator.host, if the coordinator server is used as a bootstrap server (discover all the servers in the cluster), the value of this config option should be a static port. Otherwise, the value can be set to "0" for a dynamic service name resolution. The value accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports
coordinator.io-pool.size10IntegerThe size of the IO thread pool to run blocking operations for coordinator server. This includes discard unnecessary snapshot files. Increase this value if you experience slow unnecessary snapshot files clean. The default value is 10. This option is deprecated. Please use server.io-pool.size instead.
+ +## Data Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
data.dir/tmp/fluss-dataStringThis configuration controls the directory where fluss will store its data. The default value is /tmp/fluss-data
+ +## Datalake Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
datalake.formatnoneStringThe datalake format used by of Fluss to be as lakehouse storage. Currently, supported formats are Paimon, Iceberg, and Lance. In the future, more kinds of data lake format will be supported, such as DeltaLake or Hudi.
+ +## Default Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
default.bucket.number1IntegerThe default number of buckets for a table in Fluss cluster. It's a cluster-level parameter, and all the tables without specifying bucket number in the cluster will use the value as the bucket number.
default.replication.factor1IntegerThe default replication factor for the log of a table in Fluss cluster. It's a cluster-level parameter, and all the tables without specifying replication factor in the cluster will use the value as replication factor.
+ +## Internal Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
internal.listener.nameFLUSSStringThe listener for server internal communication.
+ +## Kafka Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
kafka.enabledfalseBooleanWhether enable fluss kafka. Disabled by default. When this option is set to true, the fluss kafka will be enabled.
kafka.listener.names[KAFKA]ArrayListThe listener names for Kafka wire protocol communication. Support multiple listener names, separated by comma.
kafka.databasekafkaStringThe database for fluss kafka. The default database is `kafka`.
kafka.connection.max-idle-time1 minDurationClose kafka idle connections after the given time specified by this config.
+ +## Kv Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
kv.snapshot.interval10 minDurationThe interval to perform periodic snapshot for kv data. The default setting is 10 minutes.
kv.snapshot.scheduler-thread-num1IntegerThe number of threads that the server uses to schedule snapshot kv data for all the replicas in the server.
kv.snapshot.transfer-thread-num4IntegerThe number of threads the server uses to transfer (download and upload) kv snapshot files. This option is deprecated. Please use server.io-pool.size instead.
kv.snapshot.num-retained1IntegerThe maximum number of completed snapshots to retain.
kv.rocksdb.thread.num2IntegerThe maximum number of concurrent background flush and compaction jobs (per bucket of table). The default value is `2`.
kv.rocksdb.files.open-1IntegerThe maximum number of open files (per bucket of table) that can be used by the DB, `-1` means no limit. The default value is `-1`.
kv.rocksdb.log.max-file-size25 mbMemorySizeThe maximum size of RocksDB's file used for information logging. If the log files becomes larger than this, a new file will be created. If 0, all logs will be written to one log file. The default maximum file size is `25MB`.
kv.rocksdb.log.file-num4IntegerThe maximum number of files RocksDB should keep for information logging (Default setting: 4).
kv.rocksdb.log.dirnoneStringThe directory for RocksDB's information logging files. If empty (Fluss default setting), log files will be in the same directory as the Fluss log. If non-empty, this directory will be used and the data directory's absolute path will be used as the prefix of the log file name. If setting this option as a non-existing location, e.g `/dev/null`, RocksDB will then create the log under its own database folder as before.
kv.rocksdb.log.levelINFO_LEVELInfoLogLevelThe specified information logging level for RocksDB. Candidate log level is [DEBUG_LEVEL, INFO_LEVEL, WARN_LEVEL, ERROR_LEVEL, FATAL_LEVEL, HEADER_LEVEL, NUM_INFO_LOG_LEVELS]. If unset, Fluss will use INFO_LEVEL. Note: RocksDB info logs will not be written to the Fluss's tablet server logs and there is no rolling strategy, unless you configure kv.rocksdb.log.dir, kv.rocksdb.log.max-file-size, and kv.rocksdb.log.file-num accordingly. Without a rolling strategy, it may lead to uncontrolled disk space usage if configured with increased log levels! There is no need to modify the RocksDB log level, unless for troubleshooting RocksDB.
kv.rocksdb.write-batch-size2 mbMemorySizeThe max size of the consumed memory for RocksDB batch write, will flush just based on item count if this config set to 0.
kv.rocksdb.shared-rate-limiter.bytes-per-sec9223372036854775807 bytesMemorySizeThe shared rate limit in bytes per second for RocksDB flush and compaction operations across all RocksDB instances in the TabletServer. All KV tablets share a single global RateLimiter to prevent disk IO from being saturated. The RateLimiter is always enabled. The default value is Long.MAX_VALUE (effectively unlimited). Set to a lower value (e.g., 100MB) to limit the rate.
kv.rocksdb.compaction.styleLEVELCompactionStyleThe specified compaction style for DB. Candidate compaction style is LEVEL, FIFO, UNIVERSAL or NONE, and Fluss chooses `LEVEL` as default style.
kv.rocksdb.compaction.level.use-dynamic-sizefalseBooleanIf true, RocksDB will pick target size of each level dynamically. From an empty DB, RocksDB would make last level the base level, which means merging L0 data into the last level, until it exceeds max_bytes_for_level_base. And then repeat this process for second last level and so on. The default value is `false`. For more information, please refer to https://github.com/facebook/rocksdb/wiki/Leveled-Compaction#level_compaction_dynamic_level_bytes-is-trueRocksDB's doc.
kv.rocksdb.compression.per.level[LZ4, LZ4, LZ4, LZ4, LZ4, ZSTD, ZSTD]ArrayListA comma-separated list of Compression Type. Different levels can have different compression policies. In many cases, lower levels use fast compression algorithms, while higher levels with more data use slower but more effective compression algorithms. The N th element in the List corresponds to the compression type of the level N-1When `kv.rocksdb.compaction.level.use-dynamic-size` is true, compression_per_level[0] still determines L0, but other elements are based on the base level and may not match the level seen in the info log. Note: If the List size is smaller than the level number, the undefined lower level uses the last Compression Type in the List. The optional values include NO, SNAPPY, LZ4, ZSTD. For more information about compression type, please refer to doc https://github.com/facebook/rocksdb/wiki/Compression. The default value is ‘LZ4,LZ4,LZ4,LZ4,LZ4,ZSTD,ZSTD’, indicates there is lz4 compaction of level0 and level4,ZSTD compaction algorithm is used from level5 to level6. LZ4 is a lightweight compression algorithm so it usually strikes a good balance between space and CPU usage. ZSTD is more space save than LZ4, but it is more CPU-intensive. Different machines deploy compaction modes according to CPU and I/O resources. The default value is for the scenario that CPU resources are adequate. If you find the IO pressure of the system is not big when writing a lot of data, but CPU resources are inadequate, you can exchange I/O resources for CPU resources and change the compaction mode to `NO,NO,NO,LZ4,LZ4,ZSTD,ZSTD`.
kv.rocksdb.compaction.level.target-file-size-base64 mbMemorySizeThe target file size for compaction, which determines a level-1 file size. The default value is `64MB`.
kv.rocksdb.compaction.level.max-size-level-base256 mbMemorySizeThe upper-bound of the total size of level base files in bytes. The default value is `256MB`.
kv.rocksdb.writebuffer.size64 mbMemorySizeThe amount of data built up in memory (backed by an unsorted log on disk) before converting to a sorted on-disk files. The default writebuffer size is `64MB`.
kv.rocksdb.writebuffer.count2IntegerThe maximum number of write buffers that are built up in memory. The default value is `2`.
kv.rocksdb.writebuffer.number-to-merge1IntegerThe minimum number of write buffers that will be merged together before writing to storage. The default value is `1`.
kv.rocksdb.block.blocksize4 kbMemorySizeThe approximate size (in bytes) of user data packed per block. The default blocksize is `4KB`.
kv.rocksdb.block.metadata-blocksize4 kbMemorySizeApproximate size of partitioned metadata packed per block. Currently applied to indexes block when partitioned index/filters option is enabled. The default blocksize is `4KB`.
kv.rocksdb.block.cache-size8 mbMemorySizeThe amount of the cache for data blocks in RocksDB. The default block-cache size is `8MB`.
kv.rocksdb.use-bloom-filtertrueBooleanIf true, every newly created SST file will contain a Bloom filter. It is enabled by default.
kv.rocksdb.bloom-filter.bits-per-key10.0DoubleBits per key that bloom filter will use, this only take effect when bloom filter is used. The default value is 10.0.
kv.rocksdb.bloom-filter.block-based-modefalseBooleanIf true, RocksDB will use block-based filter instead of full filter, this only take effect when bloom filter is used. The default value is `false`.
kv.rocksdb.block.cache-index-and-filter-blocksfalseBooleanIf true, index and filter blocks will be stored in block cache, together with all other data blocks. This helps to limit memory usage so that the total memory used by RocksDB is bounded by block cache size. The default value is `false`.
kv.rocksdb.block.cache-index-and-filter-blocks-with-high-priorityfalseBooleanIf true and cache_index_and_filter_blocks is enabled, index and filter blocks will be stored with high priority in block cache, making them less likely to be evicted than data blocks. The default value is `false`.
kv.rocksdb.block.pin-l0-filter-and-index-blocks-in-cachefalseBooleanIf true and cache_index_and_filter_blocks is enabled, L0 index and filter blocks will be pinned in block cache and will not be evicted. This helps avoid performance degradation due to cache misses on L0 index/filter blocks. The default value is `false`.
kv.rocksdb.block.pin-top-level-index-and-filterfalseBooleanIf true, the top-level index of partitioned index/filter blocks will be pinned in block cache and will not be evicted. The default value is `false`.
kv.recover.log-record-batch.max-size16 mbMemorySizeThe max fetch size for fetching log to apply to kv during recovering kv.
+ +## Lake Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
lake.tiering.auto-expire-snapshotfalseBooleanIf true, snapshot expiration will be triggered automatically when tiering service commits to the datalake, even if Key: 'table.datalake.auto-expire-snapshot' , default: false (fallback keys: []) is false.
+ +## Log Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
log.segment.file-size1 gbMemorySizeThis configuration controls the segment file size for the log. Retention and cleaning is always done a file at a time so a larger segment size means fewer files but less granular control over retention.
log.index.file-size10 mbMemorySizeThis configuration controls the size of the index that maps offsets to file positions. We preallocate this index file and shrink it only after log rolls. You generally should not need to change this setting.
log.index.interval-size4 kbMemorySizeThis setting controls how frequently fluss adds an index entry to its offset index. The default setting ensures that we index a message roughly every 4096 bytes. More indexing allows reads to jump closer to the exact position in the log but makes the index larger. You probably don't need to change this.
log.file-preallocatefalseBooleanTrue if we should preallocate the file on disk when creating a new log segment.
log.flush.interval-messages9223372036854775807LongThis setting allows specifying an interval at which we will force a fsync of data written to the log. For example if this was set to 1, we would fsync after every message; if it were 5 we would fsync after every five messages.
log.replica.high-watermark.checkpoint-interval5 sDurationThe frequency with which the high watermark is saved out to disk. The default setting is 5 seconds.
log.replica.max-lag-time30 sDurationIf a follower replica hasn't sent any fetch log requests or hasn't consumed up the leaders log end offset for at least this time, the leader will remove the follower replica form isr
log.replica.write-operation-purge-number1000IntegerThe purge number (in number of requests) of the write operation manager, the default value is 1000.
log.replica.fetch-operation-purge-number1000IntegerThe purge number (in number of requests) of the fetch log operation manager, the default value is 1000.
log.replica.fetcher-number1IntegerNumber of fetcher threads used to replicate log records from each source tablet server. The total number of fetchers on each tablet server is bound by this parameter multiplied by the number of tablet servers in the cluster. Increasing this value can increase the degree of I/O parallelism in the follower and leader tablet server at the cost of higher CPU and memory utilization.
log.replica.fetch.backoff-interval1 sDurationThe amount of time to sleep when fetch bucket error occurs.
log.replica.fetch.max-bytes16 mbMemorySizeThe maximum amount of data the server should return for a fetch request from follower. Records are fetched in batches, and if the first record batch in the first non-empty bucket of the fetch is larger than this value, the record batch will still be returned to ensure that the fetch can make progress. As such, this is not a absolute maximum. Note that the fetcher performs multiple fetches in parallel.
log.replica.fetch.max-bytes-for-bucket1 mbMemorySizeThe maximum amount of data the server should return for a table bucket in fetch request from follower. Records are fetched in batches, the max bytes size is config by this option.
log.replica.fetch.wait-max-time0 sDurationThe maximum time to wait for enough bytes to be available for a fetch log request from follower to response. This value should always be less than the `log.replica.max-lag-time` at all times to prevent frequent shrinking of ISR for low throughput tables
log.replica.fetch.min-bytes1 bytesMemorySizeThe minimum bytes expected for each fetch log request from follower to response. If not enough bytes, wait up to log.replica.fetch.wait-max-time time to return.
log.replica.min-in-sync-replicas-number1IntegerWhen a writer set `client.writer.acks` to all (-1), this configuration specifies the minimum number of replicas that must acknowledge a write for the write to be considered successful. If this minimum cannot be met, then the writer will raise an exception (NotEnoughReplicas). when used together, this config and `client.writer.acks` allow you to enforce greater durability guarantees. A typical scenario would be to create a table with a replication factor of 3. set this conf to 2, and write with acks = -1. This will ensure that the writer raises an exception if a majority of replicas don't receive a write.
+ +## Max Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
max.partition.num1000IntegerLimits the maximum number of partitions that can be created for a partitioned table to avoid creating too many partitions.
max.bucket.num128000IntegerThe maximum number of buckets that can be created for a table.The default value is 128000
+ +## Metrics Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
metrics.reportersnoneStringAn optional list of reporter names. If configured, only reporters whose name matches in the list will be started
metrics.reporter.prometheus.port9249StringThe port the Prometheus reporter listens on.In order to be able to run several instances of the reporter on one host (e.g. when one TabletServer is colocated with the CoordinatorServer) it is advisable to use a port range like 9250-9260.
metrics.reporter.jmx.portnoneStringThe port for the JMXServer that JMX clients can connect to. If not set, the JMXServer won't start. In order to be able to run several instances of the reporter on one host (e.g. when one TabletServer is colocated with the CoordinatorServer) it is advisable to use a port range like 9990-9999.
+ +## Netty Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
netty.server.num-network-threads3IntegerThe number of threads that the server uses for receiving requests from the network and sending responses to the network.
netty.server.num-worker-threads8IntegerThe number of threads that the server uses for processing requests, which may include disk and remote I/O.
netty.server.max-queued-requests500IntegerThe number of queued requests allowed for worker threads, before blocking the I/O threads.
netty.connection.max-idle-time10 minDurationClose idle connections after the given time specified by this config.
netty.client.num-network-threads4IntegerThe number of threads that the client uses for sending requests to the network and receiving responses from network. The default value is 4
+ +## Plugin Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
plugin.classloader.parent-first-patterns.default[java., org.apache.fluss., javax.annotation., org.apache.hadoop., core-site, org.slf4j, org.apache.log4j, org.apache.logging, org.apache.commons.logging, ch.qos.logback]ArrayListA (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. This setting should generally not be modified. To add another pattern we recommend to use "plugin.classloader.parent-first-patterns.additional" instead.
plugin.classloader.parent-first-patterns.additional[]ArrayListA (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. These patterns are appended to "plugin.classloader.parent-first-patterns.default".
+ +## Remote Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
remote.data.dirnoneStringThe directory used for storing the kv snapshot data files and remote log for log tiered storage in a Fluss supported filesystem.
remote.fs.write-buffer-size4 kbMemorySizeThe default size of the write buffer for writing the local files to remote file systems.
remote.log.task-interval-duration1 minDurationInterval at which remote log manager runs the scheduled tasks like copy segments, clean up remote log segments, delete local log segments etc. If the value is set to 0, it means that the remote log storage is disabled.
remote.log.index-file-cache-size1 gbMemorySizeThe total size of the space allocated to store index files fetched from remote storage in the local storage.
remote.log-manager.thread-pool-size4IntegerSize of the thread pool used in scheduling tasks to copy segments, fetch remote log indexes and clean up remote log segments.
remote.log.data-transfer-thread-num4IntegerThe number of threads the server uses to transfer (download and upload) remote log file can be data file, index file and remote log metadata file. This option is deprecated. Please use server.io-pool.size instead.
+ +## Security Configurations + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
security.sasl.enabled.mechanismsnoneString
security.protocol.map{}EmptyMapA map defining the authentication protocol for each listener. The format is `listenerName1:protocol1,listenerName2:protocol2`, e.g., `INTERNAL:PLAINTEXT,CLIENT:GSSAPI`. Each listener can be associated with a specific authentication protocol. Listeners not included in the map will use PLAINTEXT by default, which does not require authentication.
+ +## Server Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
server.io-pool.size10IntegerThe size of the IO thread pool to run blocking operations for both coordinator and tablet servers. This includes discard unnecessary snapshot files, transfer kv snapshot files, and transfer remote log files. Increase this value if you experience slow IO operations. The default value is 10.
server.writer-id.expiration-time168 hoursDurationThe time that the tablet server will wait without receiving any write request from a client before expiring the related status. The default value is 7 days.
server.writer-id.expiration-check-interval10 minDurationThe interval at which to remove writer ids that have expired due to server.writer-id.expiration-time passing. The default value is 10 minutes.
server.background.threads10IntegerThe number of threads to use for various background processing tasks.
server.buffer.memory-size256 mbMemorySizeThe total bytes of memory the server can use, e.g, buffer write-ahead-log rows.
server.buffer.page-size128 kbMemorySizeSize of every page in memory buffers (`server.buffer.memory-size`).
server.buffer.per-request-memory-size16 mbMemorySizeThe minimum number of bytes that will be allocated by the writer rounded down to the closes multiple of server.buffer.page-sizeIt must be greater than or equal to server.buffer.page-size. This option allows to allocate memory in batches to have better CPU-cached friendliness due to contiguous segments.
server.buffer.wait-timeout9223372036 sDurationDefines how long the buffer pool will block when waiting for segments to become available.
+ +## Super Configurations + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
super.usersnoneStringA semicolon-separated list of superusers who have unrestricted access to all operations and resources. Note that the delimiter is semicolon since SSL user names may contain comma, and each super user should be specified in the format `principal_type:principal_name`, e.g., `User:admin;User:bob`. This configuration is critical for defining administrative privileges in the system.
+ +## Table Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
table.replication.factornoneStringThe replication factor for the log of the new table. When it's not set, Fluss will use the cluster's default replication factor configured by default.replication.factor. It should be a positive number and not larger than the number of tablet servers in the Fluss cluster. A value larger than the number of tablet servers in Fluss cluster will result in an error when the new table is created.
table.log.formatARROWLogFormatThe format of the log records in log store. The default value is `arrow`. The supported formats are `arrow`, `indexed` and `compacted`.
table.log.arrow.compression.typeZSTDArrowCompressionTypeThe compression type of the log records if the log format is set to `ARROW`. The candidate compression type is [NONE, LZ4_FRAME, ZSTD]
table.log.arrow.compression.zstd.level3IntegerThe compression level of ZSTD for the log records if the log format is set to `ARROW` and the compression type is set to `ZSTD`. The valid range is 1 to 22.
table.kv.formatCOMPACTEDKvFormatThe format of the kv records in kv store. The default value is `compacted`. The supported formats are `compacted` and `indexed`.
table.auto-partition.enabledfalseBooleanWhether enable auto partition for the table. Disable by default. When auto partition is enabled, the partitions of the table will be created automatically.
table.auto-partition.keynoneStringThis configuration defines the time-based partition key to be used for auto-partitioning when a table is partitioned with multiple keys. Auto-partitioning utilizes a time-based partition key to handle partitions automatically, including creating new ones and removing outdated ones, by comparing the time value of the partition with the current system time. In the case of a table using multiple partition keys (such as a composite partitioning strategy), this feature determines which key should serve as the primary time dimension for making auto-partitioning decisions.And If the table has only one partition key, this config is not necessary. Otherwise, it must be specified.
table.auto-partition.time-unitDAYAutoPartitionTimeUnitThe time granularity for auto created partitions. The default value is `DAY`. Valid values are `HOUR`, `DAY`, `MONTH`, `QUARTER`, `YEAR`. If the value is `HOUR`, the partition format for auto created is yyyyMMddHH. If the value is `DAY`, the partition format for auto created is yyyyMMdd. If the value is `MONTH`, the partition format for auto created is yyyyMM. If the value is `QUARTER`, the partition format for auto created is yyyyQ. If the value is `YEAR`, the partition format for auto created is yyyy.
table.auto-partition.time-zoneEtc/UTCStringThe time zone for auto partitions, which is by default the same as the system time zone.
table.auto-partition.num-precreate2IntegerThe number of partitions to pre-create for auto created partitions in each check for auto partition. For example, if the current check time is 2024-11-11 and the value is configured as 3, then partitions 20241111, 20241112, 20241113 will be pre-created. If any one partition exists, it'll skip creating the partition. The default value is 2, which means 2 partitions will be pre-created. If the `table.auto-partition.time-unit` is `DAY`(default), one precreated partition is for today and another one is for tomorrow.For a partition table with multiple partition keys, pre-create is unsupported and will be set to 0 automatically when creating table if it is not explicitly specified.
table.auto-partition.num-retention7IntegerThe number of history partitions to retain for auto created partitions in each check for auto partition. For example, if the current check time is 2024-11-11, time-unit is DAY, and the value is configured as 3, then the history partitions 20241108, 20241109, 20241110 will be retained. The partitions earlier than 20241108 will be deleted. The default value is 7.
table.log.ttl168 hoursDurationThe time to live for log segments. The configuration controls the maximum time we will retain a log before we will delete old segments to free up space. If set to -1, the log will not be deleted.
table.log.tiered.local-segments2IntegerThe number of log segments to retain in local for each table when log tiered storage is enabled. It must be greater that 0. The default is 2.
table.datalake.enabledfalseBooleanWhether enable lakehouse storage for the table. Disabled by default. When this option is set to ture and the datalake tiering service is up, the table will be tiered and compacted into datalake format stored on lakehouse storage.
table.datalake.formatnoneStringThe data lake format of the table specifies the tiered Lakehouse storage format. Currently, supported formats are `paimon`, `iceberg`, and `lance`. In the future, more kinds of data lake format will be supported, such as DeltaLake or Hudi. Once the `table.datalake.format` property is configured, Fluss adopts the key encoding and bucketing strategy used by the corresponding data lake format. This ensures consistency in key encoding and bucketing, enabling seamless **Union Read** functionality across Fluss and Lakehouse. The `table.datalake.format` can be pre-defined before enabling `table.datalake.enabled`. This allows the data lake feature to be dynamically enabled on the table without requiring table recreation. If `table.datalake.format` is not explicitly set during table creation, the table will default to the format specified by the `datalake.format` configuration in the Fluss cluster.
table.datalake.freshness3 minDurationIt defines the maximum amount of time that the datalake table's content should lag behind updates to the Fluss table. Based on this target freshness, the Fluss service automatically moves data from the Fluss table and updates to the datalake table, so that the data in the datalake table is kept up to date within this target. If the data does not need to be as fresh, you can specify a longer target freshness time to reduce costs.
table.datalake.auto-compactionfalseBooleanIf true, compaction will be triggered automatically when tiering service writes to the datalake. It is disabled by default.
table.datalake.auto-expire-snapshotfalseBooleanIf true, snapshot expiration will be triggered automatically when tiering service commits to the datalake. It is disabled by default.
table.merge-enginenoneStringDefines the merge engine for the primary key table. By default, primary key table doesn't have merge engine. The supported merge engines are `first_row`, `versioned`, and `aggregation`. The `first_row` merge engine will keep the first row of the same primary key. The `versioned` merge engine will keep the row with the largest version of the same primary key. The `aggregation` merge engine will aggregate rows with the same primary key using field-level aggregate functions.
table.merge-engine.versioned.ver-columnnoneStringThe column name of the version column for the `versioned` merge engine. If the merge engine is set to `versioned`, the version column must be set.
table.delete.behaviorALLOWDeleteBehaviorDefines the delete behavior for the primary key table. The supported delete behaviors are `allow`, `ignore`, and `disable`. The `allow` behavior allows normal delete operations (default for default merge engine). The `ignore` behavior silently skips delete requests without error. The `disable` behavior rejects delete requests with a clear error message. For tables with FIRST_ROW, VERSIONED, or AGGREGATION merge engines, this option defaults to `ignore`. Note: For AGGREGATION merge engine, when set to `allow`, delete operations will remove the entire record.
table.auto-increment.cache-size100000LongThe cache size of auto-increment IDs fetched from the distributed counter each time. This value determines the length of the locally cached ID segment. Default: 100000. A larger cache size may cause significant auto-increment ID gaps, especially when unused cached ID segments are discarded due to TabletServer restarts or abnormal terminations. Conversely, a smaller cache size increases the frequency of ID fetch requests to the distributed counter, introducing extra network overhead and reducing write throughput and performance.
table.changelog.imageFULLChangelogImageDefines the changelog image mode for the primary key table. This configuration is inspired by similar settings in database systems like MySQL's binlog_row_image and PostgreSQL's replica identity. The supported modes are `FULL` (default) and `WAL`. The `FULL` mode produces both UPDATE_BEFORE and UPDATE_AFTER records for update operations, capturing complete information about updates and allowing tracking of previous values. The `WAL` mode does not produce UPDATE_BEFORE records. Only INSERT, UPDATE_AFTER (and DELETE if allowed) records are emitted. When WAL mode is enabled, the default merge engine is used (no merge engine configured), updates are full row updates (not partial update), and there is no auto-increment column, an optimization is applied to skip looking up old values, and in this case INSERT operations are converted to UPDATE_AFTER events. This mode reduces storage and transmission costs but loses the ability to track previous values. This option only affects primary key tables.
+ +## Tablet-server Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
tablet-server.hostnoneStringThe external address of the network interface where the TabletServer is exposed. Because different TabletServer need different values for this option, usually it is specified in an additional non-shared TabletServer-specific config file.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports
tablet-server.port0StringThe external RPC port where the TabletServer is exposed.This option is deprecated. Please use bind.listeners instead, which provides a more flexible configuration for multiple ports
tablet-server.idnoneStringThe id for the tablet server.
tablet-server.racknoneStringThe rack for the tabletServer. This will be used in rack aware bucket assignment for fault tolerance. Examples: `RACK1`, `cn-hangzhou-server10`
tablet-server.controlled-shutdown.max-retries3IntegerThe maximum number of retries for controlled shutdown of the tablet server. During controlled shutdown, the tablet server attempts to transfer leadership of its buckets to other servers. If the transfer fails, it will retry up to this number of times before proceeding with shutdown. The default value is 3.
tablet-server.controlled-shutdown.retry-interval1 sDurationThe interval between retries during controlled shutdown of the tablet server. When controlled shutdown fails to transfer bucket leadership, the tablet server will wait for this duration before attempting the next retry. The default value is 1000 milliseconds (1 second).
+ +## Zookeeper Configurations + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
zookeeper.addressnoneStringThe ZooKeeper address to use, when running Fluss with ZooKeeper.
zookeeper.path.root/flussStringThe root path under which Fluss stores its entries in ZooKeeper.
zookeeper.client.max-inflight-requests100IntegerThe maximum number of unacknowledged requests the client will send to ZooKeeper before blocking.
zookeeper.client.session-timeout1 minDurationDefines the session timeout for the ZooKeeper session.
zookeeper.client.connection-timeout15 sDurationDefines the connection timeout for ZooKeeper.
zookeeper.client.retry-wait5 sDurationDefines the pause between consecutive retries.
zookeeper.client.max-retry-attempts3IntegerDefines the number of connection retries before the client gives up.
zookeeper.client.tolerate-suspended-connectionsfalseBooleanDefines whether a suspended ZooKeeper connection will be treated as an error that causes the leader information to be invalidated or not. In case you set this option to , Fluss will wait until a ZooKeeper connection is marked as lost before it revokes the leadership of components. This has the effect that Fluss is more resilient against temporary connection instabilities at the cost of running more likely into timing issues with ZooKeeper.
zookeeper.client.ensemble-trackertrueBooleanDefines whether Curator should enable ensemble tracker. This can be useful in certain scenarios in which CuratorFramework is accessing to ZK clusters via load balancer or Virtual IPs. Default Curator EnsembleTracking logic watches CuratorEventType.GET_CONFIG events and changes ZooKeeper connection string. It is not desired behaviour when ZooKeeper is running under the Virtual IPs. Under certain configurations EnsembleTracking can lead to setting of ZooKeeper connection string with unresolvable hostnames.
zookeeper.client.config-pathnoneStringThe file path from which the ZooKeeper client reads its configuration. This allows each ZooKeeper client instance to load its own configuration file, instead of relying on shared JVM-level environment settings. This enables fine-grained control over ZooKeeper client behavior.
zookeeper.client.max-buffer-size104857600IntegerThe maximum buffer size (in bytes) for ZooKeeper client. This corresponds to the jute.maxbuffer property. Default is 100MB to match the RPC frame length limit.
+ +export default ({children}) => <>{children}; +