diff --git a/gluten-flink/core/pom.xml b/gluten-flink/core/pom.xml
new file mode 100644
index 000000000000..5a39a4487c7d
--- /dev/null
+++ b/gluten-flink/core/pom.xml
@@ -0,0 +1,84 @@
+
+
+
+ 4.0.0
+
+
+ org.apache.gluten
+ gluten-flink
+ 1.3.0-SNAPSHOT
+ ../pom.xml
+
+
+ gluten-flink-core
+ Gluten Flink Core
+ jar
+
+
+
+ org.apache.flink
+ flink-table-api-java
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-table-planner_${scala.binary.version}
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-clients
+ ${flink.version}
+ provided
+
+
+ com.google.protobuf
+ protobuf-java
+ ${protobuf.version}
+ provided
+
+
+
+
+
+
+
+ org.xolstice.maven.plugins
+ protobuf-maven-plugin
+
+
+ compile-substrait-proto
+ generate-sources
+
+ compile
+ test-compile
+
+
+
+ com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}
+
+ src/main/resources/substrait/proto
+ true
+
+
+
+
+
+
+
diff --git a/gluten-flink/core/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/gluten-flink/core/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
new file mode 100644
index 000000000000..6fba382c39c9
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java
@@ -0,0 +1,824 @@
+/*
+ * 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.flink.runtime.taskexecutor;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.JMXServerOptions;
+import org.apache.flink.configuration.RpcOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.configuration.TaskManagerOptionsInternal;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.plugin.PluginManager;
+import org.apache.flink.core.plugin.PluginUtils;
+import org.apache.flink.core.security.FlinkSecurityManager;
+import org.apache.flink.management.jmx.JMXService;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.blob.BlobCacheService;
+import org.apache.flink.runtime.blob.BlobUtils;
+import org.apache.flink.runtime.blob.TaskExecutorBlobService;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.entrypoint.ClusterEntrypointUtils;
+import org.apache.flink.runtime.entrypoint.DeterminismEnvelope;
+import org.apache.flink.runtime.entrypoint.FlinkParseException;
+import org.apache.flink.runtime.entrypoint.WorkingDirectory;
+import org.apache.flink.runtime.externalresource.ExternalResourceInfoProvider;
+import org.apache.flink.runtime.externalresource.ExternalResourceUtils;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTrackerImpl;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.metrics.MetricRegistryImpl;
+import org.apache.flink.runtime.metrics.ReporterSetup;
+import org.apache.flink.runtime.metrics.TraceReporterSetup;
+import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
+import org.apache.flink.runtime.metrics.util.MetricUtils;
+import org.apache.flink.runtime.rpc.AddressResolution;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.RpcSystem;
+import org.apache.flink.runtime.rpc.RpcSystemUtils;
+import org.apache.flink.runtime.rpc.RpcUtils;
+import org.apache.flink.runtime.security.SecurityConfiguration;
+import org.apache.flink.runtime.security.SecurityUtils;
+import org.apache.flink.runtime.security.token.DelegationTokenReceiverRepository;
+import org.apache.flink.runtime.state.changelog.StateChangelogStorageLoader;
+import org.apache.flink.runtime.taskmanager.MemoryLogger;
+import org.apache.flink.runtime.util.ConfigurationParserUtils;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.Hardware;
+import org.apache.flink.runtime.util.JvmShutdownSafeguard;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.runtime.util.SignalHandler;
+import org.apache.flink.util.AbstractID;
+import org.apache.flink.util.AutoCloseableAsync;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Reference;
+import org.apache.flink.util.ShutdownHookUtil;
+import org.apache.flink.util.StringUtils;
+import org.apache.flink.util.TaskManagerExceptionUtils;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+import org.apache.flink.util.concurrent.FutureUtils;
+import org.apache.flink.util.function.FunctionUtils;
+
+import org.apache.gluten.backendsapi.FlinkBackend;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.InetAddress;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is the executable entry point for the task manager in yarn or standalone mode. It
+ * constructs the related components (network, I/O manager, memory manager, RPC service, HA service)
+ * and starts them.
+ */
+public class TaskManagerRunner implements FatalErrorHandler {
+
+ private static final Logger LOG = LoggerFactory.getLogger(TaskManagerRunner.class);
+
+ private static final long FATAL_ERROR_SHUTDOWN_TIMEOUT_MS = 10000L;
+
+ private static final int SUCCESS_EXIT_CODE = 0;
+ @VisibleForTesting public static final int FAILURE_EXIT_CODE = 1;
+
+ private final Thread shutdownHook;
+
+ private final Object lock = new Object();
+
+ private final Configuration configuration;
+
+ private final Time timeout;
+
+ private final PluginManager pluginManager;
+
+ private final TaskExecutorServiceFactory taskExecutorServiceFactory;
+
+ private final CompletableFuture terminationFuture;
+
+ @GuardedBy("lock")
+ private DeterminismEnvelope resourceId;
+
+ /** Executor used to run future callbacks. */
+ @GuardedBy("lock")
+ private ExecutorService executor;
+
+ @GuardedBy("lock")
+ private RpcSystem rpcSystem;
+
+ @GuardedBy("lock")
+ private RpcService rpcService;
+
+ @GuardedBy("lock")
+ private HighAvailabilityServices highAvailabilityServices;
+
+ @GuardedBy("lock")
+ private MetricRegistryImpl metricRegistry;
+
+ @GuardedBy("lock")
+ private BlobCacheService blobCacheService;
+
+ @GuardedBy("lock")
+ private DeterminismEnvelope workingDirectory;
+
+ @GuardedBy("lock")
+ private TaskExecutorService taskExecutorService;
+
+ @GuardedBy("lock")
+ private boolean shutdown;
+
+ public TaskManagerRunner(
+ Configuration configuration,
+ PluginManager pluginManager,
+ TaskExecutorServiceFactory taskExecutorServiceFactory)
+ throws Exception {
+ this.configuration = checkNotNull(configuration);
+ this.pluginManager = checkNotNull(pluginManager);
+ this.taskExecutorServiceFactory = checkNotNull(taskExecutorServiceFactory);
+
+ timeout = Time.fromDuration(configuration.get(RpcOptions.ASK_TIMEOUT_DURATION));
+
+ this.terminationFuture = new CompletableFuture<>();
+ this.shutdown = false;
+
+ this.shutdownHook =
+ ShutdownHookUtil.addShutdownHook(
+ () -> this.closeAsync(Result.JVM_SHUTDOWN).join(),
+ getClass().getSimpleName(),
+ LOG);
+ }
+
+ private void startTaskManagerRunnerServices() throws Exception {
+ synchronized (lock) {
+ rpcSystem = RpcSystem.load(configuration);
+
+ this.executor =
+ Executors.newScheduledThreadPool(
+ Hardware.getNumberCPUCores(),
+ new ExecutorThreadFactory("taskmanager-future"));
+
+ highAvailabilityServices =
+ HighAvailabilityServicesUtils.createHighAvailabilityServices(
+ configuration,
+ executor,
+ AddressResolution.NO_ADDRESS_RESOLUTION,
+ rpcSystem,
+ this);
+
+ JMXService.startInstance(configuration.get(JMXServerOptions.JMX_SERVER_PORT));
+
+ rpcService = createRpcService(configuration, highAvailabilityServices, rpcSystem);
+
+ this.resourceId =
+ getTaskManagerResourceID(
+ configuration, rpcService.getAddress(), rpcService.getPort());
+
+ this.workingDirectory =
+ ClusterEntrypointUtils.createTaskManagerWorkingDirectory(
+ configuration, resourceId);
+
+ LOG.info("Using working directory: {}", workingDirectory);
+
+ HeartbeatServices heartbeatServices =
+ HeartbeatServices.fromConfiguration(configuration);
+
+ metricRegistry =
+ new MetricRegistryImpl(
+ MetricRegistryConfiguration.fromConfiguration(
+ configuration,
+ rpcSystem.getMaximumMessageSizeInBytes(configuration)),
+ ReporterSetup.fromConfiguration(configuration, pluginManager),
+ TraceReporterSetup.fromConfiguration(configuration, pluginManager));
+
+ final RpcService metricQueryServiceRpcService =
+ MetricUtils.startRemoteMetricsRpcService(
+ configuration,
+ rpcService.getAddress(),
+ configuration.get(TaskManagerOptions.BIND_HOST),
+ rpcSystem);
+ metricRegistry.startQueryService(metricQueryServiceRpcService, resourceId.unwrap());
+
+ blobCacheService =
+ BlobUtils.createBlobCacheService(
+ configuration,
+ Reference.borrowed(workingDirectory.unwrap().getBlobStorageDirectory()),
+ highAvailabilityServices.createBlobStore(),
+ null);
+
+ final ExternalResourceInfoProvider externalResourceInfoProvider =
+ ExternalResourceUtils.createStaticExternalResourceInfoProviderFromConfig(
+ configuration, pluginManager);
+
+ final DelegationTokenReceiverRepository delegationTokenReceiverRepository =
+ new DelegationTokenReceiverRepository(configuration, pluginManager);
+
+ taskExecutorService =
+ taskExecutorServiceFactory.createTaskExecutor(
+ this.configuration,
+ this.resourceId.unwrap(),
+ rpcService,
+ highAvailabilityServices,
+ heartbeatServices,
+ metricRegistry,
+ blobCacheService,
+ false,
+ externalResourceInfoProvider,
+ workingDirectory.unwrap(),
+ this,
+ delegationTokenReceiverRepository);
+
+ handleUnexpectedTaskExecutorServiceTermination();
+
+ MemoryLogger.startIfConfigured(
+ LOG, configuration, terminationFuture.thenAccept(ignored -> {}));
+ }
+ }
+
+ @GuardedBy("lock")
+ private void handleUnexpectedTaskExecutorServiceTermination() {
+ taskExecutorService
+ .getTerminationFuture()
+ .whenComplete(
+ (unused, throwable) -> {
+ synchronized (lock) {
+ if (!shutdown) {
+ onFatalError(
+ new FlinkException(
+ "Unexpected termination of the TaskExecutor.",
+ throwable));
+ }
+ }
+ });
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Lifecycle management
+ // --------------------------------------------------------------------------------------------
+
+ public void start() throws Exception {
+ synchronized (lock) {
+ startTaskManagerRunnerServices();
+ taskExecutorService.start();
+ }
+ }
+
+ public void close() throws Exception {
+ try {
+ closeAsync().get();
+ } catch (ExecutionException e) {
+ ExceptionUtils.rethrowException(ExceptionUtils.stripExecutionException(e));
+ }
+ }
+
+ public CompletableFuture closeAsync() {
+ return closeAsync(Result.SUCCESS);
+ }
+
+ private CompletableFuture closeAsync(Result terminationResult) {
+ synchronized (lock) {
+ // remove shutdown hook to prevent resource leaks
+ ShutdownHookUtil.removeShutdownHook(shutdownHook, this.getClass().getSimpleName(), LOG);
+
+ if (shutdown) {
+ return terminationFuture;
+ }
+
+ final CompletableFuture taskManagerTerminationFuture;
+ if (taskExecutorService != null) {
+ taskManagerTerminationFuture = taskExecutorService.closeAsync();
+ } else {
+ taskManagerTerminationFuture = FutureUtils.completedVoidFuture();
+ }
+
+ final CompletableFuture serviceTerminationFuture =
+ FutureUtils.composeAfterwards(
+ taskManagerTerminationFuture, this::shutDownServices);
+
+ final CompletableFuture workingDirCleanupFuture =
+ FutureUtils.runAfterwards(
+ serviceTerminationFuture, () -> deleteWorkingDir(terminationResult));
+
+ final CompletableFuture rpcSystemClassLoaderCloseFuture;
+
+ if (rpcSystem != null) {
+ rpcSystemClassLoaderCloseFuture =
+ FutureUtils.runAfterwards(workingDirCleanupFuture, rpcSystem::close);
+ } else {
+ rpcSystemClassLoaderCloseFuture = FutureUtils.completedVoidFuture();
+ }
+
+ rpcSystemClassLoaderCloseFuture.whenComplete(
+ (Void ignored, Throwable throwable) -> {
+ if (throwable != null) {
+ terminationFuture.completeExceptionally(throwable);
+ } else {
+ terminationFuture.complete(terminationResult);
+ }
+ });
+
+ shutdown = true;
+ return terminationFuture;
+ }
+ }
+
+ private void deleteWorkingDir(Result terminationResult) throws IOException {
+ synchronized (lock) {
+ if (workingDirectory != null) {
+ if (!workingDirectory.isDeterministic() || terminationResult == Result.SUCCESS) {
+ workingDirectory.unwrap().delete();
+ }
+ }
+ }
+ }
+
+ private CompletableFuture shutDownServices() {
+ synchronized (lock) {
+ Collection> terminationFutures = new ArrayList<>(3);
+ Exception exception = null;
+
+ try {
+ JMXService.stopInstance();
+ } catch (Exception e) {
+ exception = ExceptionUtils.firstOrSuppressed(e, exception);
+ }
+
+ if (blobCacheService != null) {
+ try {
+ blobCacheService.close();
+ } catch (Exception e) {
+ exception = ExceptionUtils.firstOrSuppressed(e, exception);
+ }
+ }
+
+ if (metricRegistry != null) {
+ try {
+ terminationFutures.add(metricRegistry.closeAsync());
+ } catch (Exception e) {
+ exception = ExceptionUtils.firstOrSuppressed(e, exception);
+ }
+ }
+
+ if (highAvailabilityServices != null) {
+ try {
+ highAvailabilityServices.close();
+ } catch (Exception e) {
+ exception = ExceptionUtils.firstOrSuppressed(e, exception);
+ }
+ }
+
+ if (rpcService != null) {
+ terminationFutures.add(rpcService.closeAsync());
+ }
+
+ if (executor != null) {
+ terminationFutures.add(
+ ExecutorUtils.nonBlockingShutdown(
+ timeout.toMilliseconds(), TimeUnit.MILLISECONDS, executor));
+ }
+
+ if (exception != null) {
+ terminationFutures.add(FutureUtils.completedExceptionally(exception));
+ }
+
+ return FutureUtils.completeAll(terminationFutures);
+ }
+ }
+
+ // export the termination future for caller to know it is terminated
+ public CompletableFuture getTerminationFuture() {
+ return terminationFuture;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // FatalErrorHandler methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void onFatalError(Throwable exception) {
+ TaskManagerExceptionUtils.tryEnrichTaskManagerError(exception);
+ LOG.error(
+ "Fatal error occurred while executing the TaskManager. Shutting it down...",
+ exception);
+
+ if (ExceptionUtils.isJvmFatalOrOutOfMemoryError(exception)) {
+ terminateJVM();
+ } else {
+ closeAsync(Result.FAILURE);
+
+ FutureUtils.orTimeout(
+ terminationFuture,
+ FATAL_ERROR_SHUTDOWN_TIMEOUT_MS,
+ TimeUnit.MILLISECONDS,
+ String.format(
+ "Waiting for TaskManager shutting down timed out after %s ms.",
+ FATAL_ERROR_SHUTDOWN_TIMEOUT_MS));
+ }
+ }
+
+ private void terminateJVM() {
+ FlinkSecurityManager.forceProcessExit(FAILURE_EXIT_CODE);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Static entry point
+ // --------------------------------------------------------------------------------------------
+
+ public static void main(String[] args) throws Exception {
+ // startup checks and logging
+ EnvironmentInformation.logEnvironmentInfo(LOG, "TaskManager", args);
+ SignalHandler.register(LOG);
+ JvmShutdownSafeguard.installAsShutdownHook(LOG);
+
+ long maxOpenFileHandles = EnvironmentInformation.getOpenFileHandlesLimit();
+
+ if (maxOpenFileHandles != -1L) {
+ LOG.info("Maximum number of open file descriptors is {}.", maxOpenFileHandles);
+ } else {
+ LOG.info("Cannot determine the maximum number of open file descriptors");
+ }
+
+ runTaskManagerProcessSecurely(args);
+ }
+
+ public static Configuration loadConfiguration(String[] args) throws FlinkParseException {
+ return ConfigurationParserUtils.loadCommonConfiguration(
+ args, TaskManagerRunner.class.getSimpleName());
+ }
+
+ public static int runTaskManager(Configuration configuration, PluginManager pluginManager)
+ throws Exception {
+ final TaskManagerRunner taskManagerRunner;
+
+ try {
+ taskManagerRunner =
+ new TaskManagerRunner(
+ configuration,
+ pluginManager,
+ TaskManagerRunner::createTaskExecutorService);
+ taskManagerRunner.start();
+ } catch (Exception exception) {
+ throw new FlinkException("Failed to start the TaskManagerRunner.", exception);
+ }
+
+ try {
+ return taskManagerRunner.getTerminationFuture().get().getExitCode();
+ } catch (Throwable t) {
+ throw new FlinkException(
+ "Unexpected failure during runtime of TaskManagerRunner.",
+ ExceptionUtils.stripExecutionException(t));
+ }
+ }
+
+ public static void runTaskManagerProcessSecurely(String[] args) {
+ Configuration configuration = null;
+
+ try {
+ configuration = loadConfiguration(args);
+ } catch (FlinkParseException fpe) {
+ LOG.error("Could not load the configuration.", fpe);
+ System.exit(FAILURE_EXIT_CODE);
+ }
+
+ runTaskManagerProcessSecurely(checkNotNull(configuration));
+ }
+
+ public static void runTaskManagerProcessSecurely(Configuration configuration) {
+ FlinkSecurityManager.setFromConfiguration(configuration);
+ final PluginManager pluginManager =
+ PluginUtils.createPluginManagerFromRootFolder(configuration);
+ FileSystem.initialize(configuration, pluginManager);
+
+ StateChangelogStorageLoader.initialize(pluginManager);
+ FlinkBackend.initialize();
+
+ int exitCode;
+ Throwable throwable = null;
+
+ ClusterEntrypointUtils.configureUncaughtExceptionHandler(configuration);
+ try {
+ SecurityUtils.install(new SecurityConfiguration(configuration));
+
+ exitCode =
+ SecurityUtils.getInstalledContext()
+ .runSecured(() -> runTaskManager(configuration, pluginManager));
+ } catch (Throwable t) {
+ throwable = ExceptionUtils.stripException(t, UndeclaredThrowableException.class);
+ exitCode = FAILURE_EXIT_CODE;
+ }
+
+ if (throwable != null) {
+ LOG.error("Terminating TaskManagerRunner with exit code {}.", exitCode, throwable);
+ } else {
+ LOG.info("Terminating TaskManagerRunner with exit code {}.", exitCode);
+ }
+
+ System.exit(exitCode);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Static utilities
+ // --------------------------------------------------------------------------------------------
+
+ public static TaskExecutorService createTaskExecutorService(
+ Configuration configuration,
+ ResourceID resourceID,
+ RpcService rpcService,
+ HighAvailabilityServices highAvailabilityServices,
+ HeartbeatServices heartbeatServices,
+ MetricRegistry metricRegistry,
+ BlobCacheService blobCacheService,
+ boolean localCommunicationOnly,
+ ExternalResourceInfoProvider externalResourceInfoProvider,
+ WorkingDirectory workingDirectory,
+ FatalErrorHandler fatalErrorHandler,
+ DelegationTokenReceiverRepository delegationTokenReceiverRepository)
+ throws Exception {
+
+ final TaskExecutor taskExecutor =
+ startTaskManager(
+ configuration,
+ resourceID,
+ rpcService,
+ highAvailabilityServices,
+ heartbeatServices,
+ metricRegistry,
+ blobCacheService,
+ localCommunicationOnly,
+ externalResourceInfoProvider,
+ workingDirectory,
+ fatalErrorHandler,
+ delegationTokenReceiverRepository);
+
+ return TaskExecutorToServiceAdapter.createFor(taskExecutor);
+ }
+
+ public static TaskExecutor startTaskManager(
+ Configuration configuration,
+ ResourceID resourceID,
+ RpcService rpcService,
+ HighAvailabilityServices highAvailabilityServices,
+ HeartbeatServices heartbeatServices,
+ MetricRegistry metricRegistry,
+ TaskExecutorBlobService taskExecutorBlobService,
+ boolean localCommunicationOnly,
+ ExternalResourceInfoProvider externalResourceInfoProvider,
+ WorkingDirectory workingDirectory,
+ FatalErrorHandler fatalErrorHandler,
+ DelegationTokenReceiverRepository delegationTokenReceiverRepository)
+ throws Exception {
+
+ checkNotNull(configuration);
+ checkNotNull(resourceID);
+ checkNotNull(rpcService);
+ checkNotNull(highAvailabilityServices);
+
+ LOG.info("Starting TaskManager with ResourceID: {}", resourceID.getStringWithMetadata());
+
+ SystemOutRedirectionUtils.redirectSystemOutAndError(configuration);
+
+ String externalAddress = rpcService.getAddress();
+
+ final TaskExecutorResourceSpec taskExecutorResourceSpec =
+ TaskExecutorResourceUtils.resourceSpecFromConfig(configuration);
+
+ TaskManagerServicesConfiguration taskManagerServicesConfiguration =
+ TaskManagerServicesConfiguration.fromConfiguration(
+ configuration,
+ resourceID,
+ externalAddress,
+ localCommunicationOnly,
+ taskExecutorResourceSpec,
+ workingDirectory);
+
+ Tuple2 taskManagerMetricGroup =
+ MetricUtils.instantiateTaskManagerMetricGroup(
+ metricRegistry,
+ externalAddress,
+ resourceID,
+ taskManagerServicesConfiguration.getSystemResourceMetricsProbingInterval());
+
+ final ExecutorService ioExecutor =
+ Executors.newFixedThreadPool(
+ taskManagerServicesConfiguration.getNumIoThreads(),
+ new ExecutorThreadFactory("flink-taskexecutor-io"));
+
+ TaskManagerServices taskManagerServices =
+ TaskManagerServices.fromConfiguration(
+ taskManagerServicesConfiguration,
+ taskExecutorBlobService.getPermanentBlobService(),
+ taskManagerMetricGroup.f1,
+ ioExecutor,
+ rpcService.getScheduledExecutor(),
+ fatalErrorHandler,
+ workingDirectory);
+
+ MetricUtils.instantiateFlinkMemoryMetricGroup(
+ taskManagerMetricGroup.f1,
+ taskManagerServices.getTaskSlotTable(),
+ taskManagerServices::getManagedMemorySize);
+
+ TaskManagerConfiguration taskManagerConfiguration =
+ TaskManagerConfiguration.fromConfiguration(
+ configuration,
+ taskExecutorResourceSpec,
+ externalAddress,
+ workingDirectory.getTmpDirectory());
+
+ String metricQueryServiceAddress = metricRegistry.getMetricQueryServiceGatewayRpcAddress();
+
+ return new TaskExecutor(
+ rpcService,
+ taskManagerConfiguration,
+ highAvailabilityServices,
+ taskManagerServices,
+ externalResourceInfoProvider,
+ heartbeatServices,
+ taskManagerMetricGroup.f0,
+ metricQueryServiceAddress,
+ taskExecutorBlobService,
+ fatalErrorHandler,
+ new TaskExecutorPartitionTrackerImpl(taskManagerServices.getShuffleEnvironment()),
+ delegationTokenReceiverRepository);
+ }
+
+ /**
+ * Create a RPC service for the task manager.
+ *
+ * @param configuration The configuration for the TaskManager.
+ * @param haServices to use for the task manager hostname retrieval
+ */
+ @VisibleForTesting
+ static RpcService createRpcService(
+ final Configuration configuration,
+ final HighAvailabilityServices haServices,
+ final RpcSystem rpcSystem)
+ throws Exception {
+
+ checkNotNull(configuration);
+ checkNotNull(haServices);
+
+ return RpcUtils.createRemoteRpcService(
+ rpcSystem,
+ configuration,
+ determineTaskManagerBindAddress(configuration, haServices, rpcSystem),
+ configuration.get(TaskManagerOptions.RPC_PORT),
+ configuration.get(TaskManagerOptions.BIND_HOST),
+ configuration.getOptional(TaskManagerOptions.RPC_BIND_PORT));
+ }
+
+ private static String determineTaskManagerBindAddress(
+ final Configuration configuration,
+ final HighAvailabilityServices haServices,
+ RpcSystemUtils rpcSystemUtils)
+ throws Exception {
+
+ final String configuredTaskManagerHostname = configuration.get(TaskManagerOptions.HOST);
+
+ if (configuredTaskManagerHostname != null) {
+ LOG.info(
+ "Using configured hostname/address for TaskManager: {}.",
+ configuredTaskManagerHostname);
+ return configuredTaskManagerHostname;
+ } else {
+ return determineTaskManagerBindAddressByConnectingToResourceManager(
+ configuration, haServices, rpcSystemUtils);
+ }
+ }
+
+ private static String determineTaskManagerBindAddressByConnectingToResourceManager(
+ final Configuration configuration,
+ final HighAvailabilityServices haServices,
+ RpcSystemUtils rpcSystemUtils)
+ throws LeaderRetrievalException {
+
+ final Duration lookupTimeout = configuration.get(RpcOptions.LOOKUP_TIMEOUT_DURATION);
+
+ final InetAddress taskManagerAddress =
+ LeaderRetrievalUtils.findConnectingAddress(
+ haServices.getResourceManagerLeaderRetriever(),
+ lookupTimeout,
+ rpcSystemUtils);
+
+ LOG.info(
+ "TaskManager will use hostname/address '{}' ({}) for communication.",
+ taskManagerAddress.getHostName(),
+ taskManagerAddress.getHostAddress());
+
+ HostBindPolicy bindPolicy =
+ HostBindPolicy.fromString(configuration.get(TaskManagerOptions.HOST_BIND_POLICY));
+ return bindPolicy == HostBindPolicy.IP
+ ? taskManagerAddress.getHostAddress()
+ : taskManagerAddress.getHostName();
+ }
+
+ @VisibleForTesting
+ static DeterminismEnvelope getTaskManagerResourceID(
+ Configuration config, String rpcAddress, int rpcPort) {
+
+ final String metadata =
+ config.get(TaskManagerOptionsInternal.TASK_MANAGER_RESOURCE_ID_METADATA, "");
+ return config.getOptional(TaskManagerOptions.TASK_MANAGER_RESOURCE_ID)
+ .map(
+ value ->
+ DeterminismEnvelope.deterministicValue(
+ new ResourceID(value, metadata)))
+ .orElseGet(
+ FunctionUtils.uncheckedSupplier(
+ () -> {
+ final String hostName =
+ InetAddress.getLocalHost().getHostName();
+ final String value =
+ StringUtils.isNullOrWhitespaceOnly(rpcAddress)
+ ? hostName
+ + "-"
+ + new AbstractID()
+ .toString()
+ .substring(0, 6)
+ : rpcAddress
+ + ":"
+ + rpcPort
+ + "-"
+ + new AbstractID()
+ .toString()
+ .substring(0, 6);
+ return DeterminismEnvelope.nondeterministicValue(
+ new ResourceID(value, metadata));
+ }));
+ }
+
+ /** Factory for {@link TaskExecutor}. */
+ public interface TaskExecutorServiceFactory {
+ TaskExecutorService createTaskExecutor(
+ Configuration configuration,
+ ResourceID resourceID,
+ RpcService rpcService,
+ HighAvailabilityServices highAvailabilityServices,
+ HeartbeatServices heartbeatServices,
+ MetricRegistry metricRegistry,
+ BlobCacheService blobCacheService,
+ boolean localCommunicationOnly,
+ ExternalResourceInfoProvider externalResourceInfoProvider,
+ WorkingDirectory workingDirectory,
+ FatalErrorHandler fatalErrorHandler,
+ DelegationTokenReceiverRepository delegationTokenReceiverRepository)
+ throws Exception;
+ }
+
+ public interface TaskExecutorService extends AutoCloseableAsync {
+ void start();
+
+ CompletableFuture getTerminationFuture();
+ }
+
+ public enum Result {
+ SUCCESS(SUCCESS_EXIT_CODE),
+ JVM_SHUTDOWN(FAILURE_EXIT_CODE),
+ FAILURE(FAILURE_EXIT_CODE);
+
+ private final int exitCode;
+
+ Result(int exitCode) {
+ this.exitCode = exitCode;
+ }
+
+ public int getExitCode() {
+ return exitCode;
+ }
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/flink/streaming/api/operators/GlutenOperator.java b/gluten-flink/core/src/main/java/org/apache/flink/streaming/api/operators/GlutenOperator.java
new file mode 100644
index 000000000000..6b4b3ec49a0e
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/flink/streaming/api/operators/GlutenOperator.java
@@ -0,0 +1,23 @@
+/*
+ * 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.flink.streaming.api.operators;
+
+/** Interface for all gluten operators. */
+public interface GlutenOperator {
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/flink/table/planner/loader/PlannerModule.java b/gluten-flink/core/src/main/java/org/apache/flink/table/planner/loader/PlannerModule.java
new file mode 100644
index 000000000000..59f5465eb97b
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/flink/table/planner/loader/PlannerModule.java
@@ -0,0 +1,206 @@
+/*
+ * 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.flink.table.planner.loader;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ConfigurationUtils;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.core.classloading.ComponentClassLoader;
+import org.apache.flink.core.classloading.SubmoduleClassLoader;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.delegation.ExecutorFactory;
+import org.apache.flink.table.delegation.PlannerFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.util.FileUtils;
+import org.apache.flink.util.IOUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Stream;
+
+/**
+ * This class will overwrite the PlannerModule in Flink to load gluten-flink-core.jar.
+ * So that it will load the classes in gluten code first.
+ */
+class PlannerModule {
+
+ /**
+ * The name of the table planner dependency jar, bundled with flink-table-planner-loader module
+ * artifact.
+ */
+ static final String FLINK_TABLE_PLANNER_FAT_JAR = "flink-table-planner.jar";
+
+ private static final String HINT_USAGE =
+ "mvn clean package -pl flink-table/flink-table-planner,flink-table/flink-table-planner-loader -DskipTests";
+
+ private static final String[] OWNER_CLASSPATH =
+ Stream.concat(
+ Arrays.stream(CoreOptions.PARENT_FIRST_LOGGING_PATTERNS),
+ Stream.of(
+ // These packages are shipped either by
+ // flink-table-runtime or flink-dist itself
+ "org.codehaus.janino",
+ "org.codehaus.commons",
+ "org.apache.commons.lang3",
+ "org.apache.commons.math3",
+ "com.google.protobuf",
+ // with hive dialect, hadoop jar should be in classpath,
+ // also, we should make it loaded by owner classloader,
+ // otherwise, it'll throw class not found exception
+ // when initialize HiveParser which requires hadoop
+ "org.apache.hadoop"))
+ .toArray(String[]::new);
+
+ private static final String[] COMPONENT_CLASSPATH = new String[] {"org.apache.flink"};
+
+ private static final Map KNOWN_MODULE_ASSOCIATIONS = new HashMap<>();
+
+ static {
+ KNOWN_MODULE_ASSOCIATIONS.put("org.apache.flink.table.runtime", "flink-table-runtime");
+ KNOWN_MODULE_ASSOCIATIONS.put("org.apache.flink.formats.raw", "flink-table-runtime");
+
+ KNOWN_MODULE_ASSOCIATIONS.put("org.codehaus.janino", "flink-table-runtime");
+ KNOWN_MODULE_ASSOCIATIONS.put("org.codehaus.commons", "flink-table-runtime");
+ KNOWN_MODULE_ASSOCIATIONS.put(
+ "org.apache.flink.table.shaded.com.jayway", "flink-table-runtime");
+ }
+
+ private final PlannerComponentClassLoader submoduleClassLoader;
+
+ private PlannerModule() {
+ try {
+ final ClassLoader flinkClassLoader = PlannerModule.class.getClassLoader();
+
+ final Path tmpDirectory =
+ Paths.get(ConfigurationUtils.parseTempDirectories(new Configuration())[0]);
+ Files.createDirectories(FileUtils.getTargetPathIfContainsSymbolicPath(tmpDirectory));
+ final Path tempFile =
+ Files.createFile(
+ tmpDirectory.resolve(
+ "flink-table-planner_" + UUID.randomUUID() + ".jar"));
+
+ final InputStream resourceStream =
+ flinkClassLoader.getResourceAsStream(FLINK_TABLE_PLANNER_FAT_JAR);
+ InputStream glutenStream =
+ flinkClassLoader.getResourceAsStream("gluten-flink-core.jar");
+ InputStream protobufStream =
+ flinkClassLoader.getResourceAsStream("protobuf-java.jar");
+ if (resourceStream == null || glutenStream == null || protobufStream == null) {
+ throw new TableException(
+ String.format(
+ "Flink Table planner could not be found. If this happened while running a test in the IDE, "
+ + "run '%s' on the command-line, "
+ + "or add a test dependency on the flink-table-planner-loader test-jar.",
+ HINT_USAGE));
+ }
+ final Path glutenFile =
+ Files.createFile(
+ tmpDirectory.resolve(
+ "gluten-flink-core_" + UUID.randomUUID() + ".jar"));
+ final Path protobufFile =
+ Files.createFile(
+ tmpDirectory.resolve(
+ "protobuf-java_" + UUID.randomUUID() + ".jar"));
+
+ IOUtils.copyBytes(resourceStream, Files.newOutputStream(tempFile));
+ IOUtils.copyBytes(protobufStream, Files.newOutputStream(protobufFile));
+ IOUtils.copyBytes(glutenStream, Files.newOutputStream(glutenFile));
+ tempFile.toFile().deleteOnExit();
+ protobufFile.toFile().deleteOnExit();
+ glutenFile.toFile().deleteOnExit();
+
+ this.submoduleClassLoader =
+ new PlannerComponentClassLoader(
+ new URL[] {
+ glutenFile.toUri().toURL(),
+ protobufFile.toUri().toURL(),
+ tempFile.toUri().toURL()},
+ flinkClassLoader,
+ OWNER_CLASSPATH,
+ COMPONENT_CLASSPATH,
+ KNOWN_MODULE_ASSOCIATIONS);
+ } catch (IOException e) {
+ throw new TableException(
+ "Could not initialize the table planner components loader.", e);
+ }
+ }
+
+ public void addUrlToClassLoader(URL url) {
+ // add the url to component url
+ this.submoduleClassLoader.addURL(url);
+ }
+
+ // Singleton lazy initialization
+
+ private static class PlannerComponentsHolder {
+ private static final PlannerModule INSTANCE = new PlannerModule();
+ }
+
+ public static PlannerModule getInstance() {
+ return PlannerComponentsHolder.INSTANCE;
+ }
+
+ // load methods for various components provided by the planner
+
+ public ExecutorFactory loadExecutorFactory() {
+ return FactoryUtil.discoverFactory(
+ this.submoduleClassLoader,
+ ExecutorFactory.class,
+ ExecutorFactory.DEFAULT_IDENTIFIER);
+ }
+
+ public PlannerFactory loadPlannerFactory() {
+ return FactoryUtil.discoverFactory(
+ this.submoduleClassLoader, PlannerFactory.class, "default");
+ }
+
+ /**
+ * A class loader extending {@link ComponentClassLoader} which overwrites method{@link #addURL}
+ * to enable it can add url to component classloader.
+ */
+ private static class PlannerComponentClassLoader extends ComponentClassLoader {
+
+ public PlannerComponentClassLoader(
+ URL[] classpath,
+ ClassLoader ownerClassLoader,
+ String[] ownerFirstPackages,
+ String[] componentFirstPackages,
+ Map knownPackagePrefixesModuleAssociation) {
+ super(
+ classpath,
+ ownerClassLoader,
+ ownerFirstPackages,
+ componentFirstPackages,
+ knownPackagePrefixesModuleAssociation);
+ }
+
+ @Override
+ public void addURL(URL url) {
+ super.addURL(url);
+ }
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java b/gluten-flink/core/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java
new file mode 100644
index 000000000000..bda387162174
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java
@@ -0,0 +1,130 @@
+/*
+ * 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.flink.table.planner.plan.nodes.exec.stream;
+
+import org.apache.flink.FlinkVersion;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.calcite.shaded.com.google.common.collect.Lists;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.*;
+import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCalc;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.TransformationMetadata;
+import org.apache.flink.table.runtime.operators.GlutenCalOperator;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.apache.calcite.rex.RexNode;
+import org.apache.gluten.rexnode.RexNodeConverter;
+import org.apache.gluten.substrait.SubstraitContext;
+import org.apache.gluten.substrait.plan.PlanBuilder;
+import org.apache.gluten.substrait.plan.PlanNode;
+import org.apache.gluten.substrait.rel.RelBuilder;
+import org.apache.gluten.substrait.rel.RelNode;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.List;
+
+/** Gluten Stream {@link ExecNode} for Calc to use {@link GlutenCalOperator}. */
+@ExecNodeMetadata(
+ name = "stream-exec-calc",
+ version = 1,
+ producedTransformations = CommonExecCalc.CALC_TRANSFORMATION,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
+public class StreamExecCalc extends CommonExecCalc implements StreamExecNode {
+
+ public StreamExecCalc(
+ ReadableConfig tableConfig,
+ List projection,
+ @Nullable RexNode condition,
+ InputProperty inputProperty,
+ RowType outputType,
+ String description) {
+ this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecCalc.class),
+ ExecNodeContext.newPersistedConfig(StreamExecCalc.class, tableConfig),
+ projection,
+ condition,
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
+ }
+
+ @JsonCreator
+ public StreamExecCalc(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig,
+ @JsonProperty(FIELD_NAME_PROJECTION) List projection,
+ @JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition,
+ @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties,
+ @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
+ @JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
+ super(
+ id,
+ context,
+ persistedConfig,
+ projection,
+ condition,
+ TableStreamOperator.class,
+ true, // retainHeader
+ inputProperties,
+ outputType,
+ description);
+ }
+
+ @Override
+ public Transformation translateToPlanInternal(
+ PlannerBase planner, ExecNodeConfig config) {
+ final ExecEdge inputEdge = getInputEdges().get(0);
+ final Transformation inputTransform =
+ (Transformation) inputEdge.translateToPlan(planner);
+ // TODO: whether need to generate substrait plan?
+ SubstraitContext context = new SubstraitContext();
+ RelNode filter = RelBuilder.makeFilterRel(
+ null,
+ RexNodeConverter.toExpressionNode(context, condition),
+ context,
+ (long) getId());
+ RelNode project = RelBuilder.makeProjectRel(
+ filter,
+ RexNodeConverter.toExpressionNode(context, projection),
+ context,
+ (long) ExecNodeContext.newNodeId());
+ PlanNode plan = PlanBuilder.makePlan(context, Lists.newArrayList(project), Lists.newArrayList());
+ final GlutenCalOperator calOperator = new GlutenCalOperator(plan.toProtobuf());
+ return ExecNodeUtil.createOneInputTransformation(
+ inputTransform,
+ new TransformationMetadata("gluten-calc", "Gluten cal operator"),
+ calOperator,
+ InternalTypeInfo.of(getOutputType()),
+ inputTransform.getParallelism(),
+ false);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/flink/table/runtime/operators/GlutenCalOperator.java b/gluten-flink/core/src/main/java/org/apache/flink/table/runtime/operators/GlutenCalOperator.java
new file mode 100644
index 000000000000..0235014fb27f
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/flink/table/runtime/operators/GlutenCalOperator.java
@@ -0,0 +1,64 @@
+/*
+ * 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.flink.table.runtime.operators;
+
+import org.apache.flink.streaming.api.operators.GlutenOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+
+import io.substrait.proto.Plan;
+import org.apache.gluten.backendsapi.FlinkBackend;
+import org.apache.gluten.vectorized.VLNativeRowVector;
+
+/** Calculate operator in gluten, which will call Velox to run. */
+public class GlutenCalOperator extends TableStreamOperator
+ implements OneInputStreamOperator, GlutenOperator {
+
+ private final byte[] glutenPlan;
+
+ private StreamRecord outElement = null;
+
+ private int nativeExecutor;
+
+ public GlutenCalOperator(Plan plan) {
+ this.glutenPlan = plan.toByteArray();
+ }
+
+ @Override
+ public void open() throws Exception {
+ super.open();
+ outElement = new StreamRecord(null);
+
+ nativeExecutor = FlinkBackend.flinkPlanExecApi().generateOperator(glutenPlan);
+ }
+
+ @Override
+ public void processElement(StreamRecord element) throws Exception {
+ // TODO: use velox jni methods to run?
+ long res = nativeProcessElement(
+ nativeExecutor,
+ VLNativeRowVector.fromRowData(element.getValue()).rowAddress());
+ if (res > 0) {
+ output.collect(outElement.replace(new VLNativeRowVector(res).toRowData()));
+ }
+ }
+
+ private native long nativeProcessElement(int executor, long data);
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/backendsapi/FlinkBackend.java b/gluten-flink/core/src/main/java/org/apache/gluten/backendsapi/FlinkBackend.java
new file mode 100644
index 000000000000..928b99aa1655
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/backendsapi/FlinkBackend.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.gluten.backendsapi;
+
+public class FlinkBackend {
+ private static VeloxFlinkPlanExecApi planExecApi;
+
+ public static VeloxFlinkPlanExecApi flinkPlanExecApi() {
+ if (planExecApi == null) {
+ planExecApi = new VeloxFlinkPlanExecApi();
+ }
+ return planExecApi;
+ }
+
+ public static void initialize() {
+ // TODO: use JniLibLoader to load velox flink libraries.
+ System.load("libgluten-flink.so");
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/backendsapi/VeloxFlinkPlanExecApi.java b/gluten-flink/core/src/main/java/org/apache/gluten/backendsapi/VeloxFlinkPlanExecApi.java
new file mode 100644
index 000000000000..52cfa3db1d5b
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/backendsapi/VeloxFlinkPlanExecApi.java
@@ -0,0 +1,22 @@
+/*
+ * 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.gluten.backendsapi;
+
+public class VeloxFlinkPlanExecApi {
+
+ public native int generateOperator(byte[] plan);
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/rexnode/RexNodeConverter.java b/gluten-flink/core/src/main/java/org/apache/gluten/rexnode/RexNodeConverter.java
new file mode 100644
index 000000000000..b36159b4e342
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/rexnode/RexNodeConverter.java
@@ -0,0 +1,108 @@
+/*
+ * 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.gluten.rexnode;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.gluten.substrait.SubstraitContext;
+import org.apache.gluten.substrait.expression.ExpressionBuilder;
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.type.TypeBuilder;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** Convertor to convert RexNode to ExpressionNode */
+public class RexNodeConverter {
+
+ public static ExpressionNode toExpressionNode(SubstraitContext context, RexNode rexNode) {
+ if (rexNode instanceof RexLiteral) {
+ RexLiteral literal = (RexLiteral) rexNode;
+ return ExpressionBuilder.makeLiteral(
+ literal.getValue(),
+ toTypeNode(literal.getType()));
+ } else if (rexNode instanceof RexCall) {
+ RexCall rexCall = (RexCall) rexNode;
+ Long functionId =
+ ExpressionBuilder.newScalarFunction(
+ context.registeredFunction(),
+ rexCall.getOperator().getName());
+ List params = toExpressionNode(context, rexCall.getOperands());
+ TypeNode typeNode = toTypeNode(rexCall.getType());
+ return ExpressionBuilder.makeScalarFunction(functionId, params, typeNode);
+ } else if (rexNode instanceof RexInputRef) {
+ RexInputRef inputRef = (RexInputRef) rexNode;
+ return ExpressionBuilder.makeSelection(inputRef.getIndex());
+ } else {
+ throw new RuntimeException("Unrecognized RexNode: " + rexNode);
+ }
+ }
+
+ public static List toExpressionNode(
+ SubstraitContext context,
+ List rexNodes) {
+ return rexNodes.stream()
+ .map(rexNode -> toExpressionNode(context, rexNode))
+ .collect(Collectors.toList());
+ }
+
+ public static TypeNode toTypeNode(RelDataType dataType) {
+ switch (dataType.getSqlTypeName()) {
+ case BOOLEAN:
+ return TypeBuilder.makeBoolean(dataType.isNullable());
+ case TINYINT:
+ return TypeBuilder.makeI8(dataType.isNullable());
+ case SMALLINT:
+ return TypeBuilder.makeI16(dataType.isNullable());
+ case INTEGER:
+ return TypeBuilder.makeI32(dataType.isNullable());
+ case BIGINT:
+ return TypeBuilder.makeI64(dataType.isNullable());
+ case FLOAT:
+ return TypeBuilder.makeFP32(dataType.isNullable());
+ case DOUBLE:
+ return TypeBuilder.makeFP64(dataType.isNullable());
+ case CHAR:
+ return TypeBuilder.makeFixedChar(dataType.isNullable(), 1);
+ case VARCHAR:
+ return TypeBuilder.makeString(dataType.isNullable());
+ case BINARY:
+ return TypeBuilder.makeBinary(dataType.isNullable());
+ case DECIMAL:
+ return TypeBuilder.makeDecimal(
+ dataType.isNullable(),
+ dataType.getPrecision(),
+ dataType.getScale());
+ case DATE:
+ return TypeBuilder.makeDate(dataType.isNullable());
+ case TIME:
+ return TypeBuilder.makeTimestamp(dataType.isNullable());
+ case MAP:
+ return TypeBuilder.makeMap(
+ dataType.isNullable(),
+ toTypeNode(dataType.getKeyType()),
+ toTypeNode(dataType.getValueType()));
+ default:
+ throw new RuntimeException("Unsupported rex node type: " + dataType);
+ }
+ }
+
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/FunctionMappings.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/FunctionMappings.java
new file mode 100644
index 000000000000..937db30e8864
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/FunctionMappings.java
@@ -0,0 +1,39 @@
+/*
+ * 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.gluten.substrait;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/** Mapping of flink function and substrait function. */
+public class FunctionMappings {
+ // A map stores the relationship between flink function name and substrait function.
+ private static Map functionMappings = new HashMap() {
+ {
+ put(">", "gt");
+ put("<", "lt");
+ }
+ };
+
+ public static String toSubstraitFunction(String funcName) {
+ if (functionMappings.containsKey(funcName)) {
+ return functionMappings.get(funcName);
+ } else {
+ throw new RuntimeException("Function not supported: " + funcName);
+ }
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/SubstraitContext.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/SubstraitContext.java
new file mode 100644
index 000000000000..e16a8a6a0468
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/SubstraitContext.java
@@ -0,0 +1,64 @@
+/*
+ * 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.gluten.substrait;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class SubstraitContext implements Serializable {
+ // A map stores the relationship between function name and function id.
+ private Map functionMap = new HashMap();
+
+ // A map stores the relationship between Spark operator id and its respective Substrait Rel ids.
+ private Map> operatorToRelsMap = new HashMap();
+
+ private long relId = 0;
+
+ public Long registerFunction(String funcName) {
+ if (!functionMap.containsKey(funcName)) {
+ Long newFunctionId = Long.valueOf(functionMap.size());
+ functionMap.put(funcName, newFunctionId);
+ return newFunctionId;
+ } else {
+ return functionMap.get(funcName);
+ }
+ }
+
+ public Map registeredFunction() {
+ return functionMap;
+ }
+
+ /**
+ * Register a rel to certain operator id.
+ * @param operatorId operator id
+ */
+ public void registerRelToOperator(Long operatorId) {
+ if (operatorToRelsMap.containsKey(operatorId)) {
+ List rels = operatorToRelsMap.get(operatorId);
+ rels.add(relId);
+ } else {
+ List rels = new ArrayList();
+ rels.add(relId);
+ operatorToRelsMap.put(operatorId, rels);
+ }
+ relId += 1;
+ }
+
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/BinaryOPNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/BinaryOPNode.java
new file mode 100644
index 000000000000..596be721cf4e
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/BinaryOPNode.java
@@ -0,0 +1,55 @@
+/*
+ * 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.gluten.substrait.derivation;
+
+import io.substrait.proto.DerivationExpression;
+
+import java.io.Serializable;
+
+public class BinaryOPNode implements DerivationExpressionNode, Serializable {
+ private final String op;
+ private final DerivationExpressionNode arg1;
+ private final DerivationExpressionNode arg2;
+
+ BinaryOPNode(String op, DerivationExpressionNode arg1, DerivationExpressionNode arg2) {
+ this.op = op;
+ this.arg1 = arg1;
+ this.arg2 = arg2;
+ }
+
+ @Override
+ public DerivationExpression toProtobuf() {
+ DerivationExpression.BinaryOp.Builder binaryBuilder =
+ DerivationExpression.BinaryOp.newBuilder();
+ switch (op) {
+ case "multiply":
+ binaryBuilder.setOpType(DerivationExpression.BinaryOp.BinaryOpType.BINARY_OP_TYPE_MULTIPLY);
+ break;
+ case "divide":
+ binaryBuilder.setOpType(DerivationExpression.BinaryOp.BinaryOpType.BINARY_OP_TYPE_DIVIDE);
+ break;
+ default:
+ System.out.println("Not supported.");
+ }
+ binaryBuilder.setArg1(arg1.toProtobuf());
+ binaryBuilder.setArg2(arg2.toProtobuf());
+
+ DerivationExpression.Builder builder = DerivationExpression.newBuilder();
+ builder.setBinaryOp(binaryBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/DerivationExpressionBuilder.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/DerivationExpressionBuilder.java
new file mode 100644
index 000000000000..c01ea659b536
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/DerivationExpressionBuilder.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gluten.substrait.derivation;
+
+public class DerivationExpressionBuilder {
+ private DerivationExpressionBuilder() {}
+
+ public static DerivationExpressionNode makeDerivationFP64(Boolean nullable) {
+ return new DerivationFP64TypeNode(nullable);
+ }
+
+ public static DerivationExpressionNode makeBinaryOP(
+ String op, DerivationExpressionNode arg1, DerivationExpressionNode arg2) {
+ return new BinaryOPNode(op, arg1, arg2);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/DerivationExpressionNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/DerivationExpressionNode.java
new file mode 100644
index 000000000000..7290645e2ca5
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/DerivationExpressionNode.java
@@ -0,0 +1,23 @@
+/*
+ * 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.gluten.substrait.derivation;
+
+import io.substrait.proto.DerivationExpression;
+
+public interface DerivationExpressionNode {
+ DerivationExpression toProtobuf();
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/DerivationFP64TypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/DerivationFP64TypeNode.java
new file mode 100644
index 000000000000..218ecb6f711e
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/derivation/DerivationFP64TypeNode.java
@@ -0,0 +1,44 @@
+/*
+ * 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.gluten.substrait.derivation;
+
+import io.substrait.proto.DerivationExpression;
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class DerivationFP64TypeNode implements DerivationExpressionNode, Serializable {
+ private final Boolean nullable;
+
+ DerivationFP64TypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public DerivationExpression toProtobuf() {
+ Type.FP64.Builder doubleBuilder = Type.FP64.newBuilder();
+ if (nullable) {
+ doubleBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ doubleBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ DerivationExpression.Builder builder = DerivationExpression.newBuilder();
+ builder.setFp64(doubleBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/AggregateFunctionNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/AggregateFunctionNode.java
new file mode 100644
index 000000000000..600f1570452d
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/AggregateFunctionNode.java
@@ -0,0 +1,79 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.AggregateFunction;
+import io.substrait.proto.AggregationPhase;
+import io.substrait.proto.FunctionArgument;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class AggregateFunctionNode implements Serializable {
+ private final Long functionId;
+ private final List expressionNodes = new ArrayList<>();
+ private final String phase;
+ private final TypeNode outputTypeNode;
+
+ AggregateFunctionNode(
+ Long functionId,
+ List expressionNodes,
+ String phase,
+ TypeNode outputTypeNode) {
+ this.functionId = functionId;
+ this.expressionNodes.addAll(expressionNodes);
+ this.phase = phase;
+ this.outputTypeNode = outputTypeNode;
+ }
+
+ public AggregateFunction toProtobuf() {
+ AggregateFunction.Builder aggBuilder = AggregateFunction.newBuilder();
+ aggBuilder.setFunctionReference(functionId.intValue());
+
+ if (phase == null) {
+ aggBuilder.setPhase(AggregationPhase.AGGREGATION_PHASE_UNSPECIFIED);
+ } else {
+ switch (phase) {
+ case "PARTIAL":
+ aggBuilder.setPhase(AggregationPhase.AGGREGATION_PHASE_INITIAL_TO_INTERMEDIATE);
+ break;
+ case "PARTIAL_MERGE":
+ aggBuilder.setPhase(AggregationPhase.AGGREGATION_PHASE_INTERMEDIATE_TO_INTERMEDIATE);
+ break;
+ case "COMPLETE":
+ aggBuilder.setPhase(AggregationPhase.AGGREGATION_PHASE_INITIAL_TO_RESULT);
+ break;
+ case "FINAL":
+ aggBuilder.setPhase(AggregationPhase.AGGREGATION_PHASE_INTERMEDIATE_TO_RESULT);
+ break;
+ default:
+ aggBuilder.setPhase(AggregationPhase.AGGREGATION_PHASE_INITIAL_TO_RESULT);
+ }
+ }
+ for (ExpressionNode expressionNode : expressionNodes) {
+ FunctionArgument.Builder functionArgument = FunctionArgument.newBuilder();
+ functionArgument.setValue(expressionNode.toProtobuf());
+ aggBuilder.addArguments(functionArgument.build());
+ }
+ aggBuilder.setOutputType(outputTypeNode.toProtobuf());
+
+ return aggBuilder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/BinaryLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/BinaryLiteralNode.java
new file mode 100644
index 000000000000..3d1ee51741ee
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/BinaryLiteralNode.java
@@ -0,0 +1,49 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.BinaryTypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import com.google.protobuf.ByteString;
+import io.substrait.proto.Expression.Literal.Builder;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+public class BinaryLiteralNode extends LiteralNodeWithValue {
+ public BinaryLiteralNode(byte[] value) {
+ super(value, new BinaryTypeNode(true));
+ }
+
+ public BinaryLiteralNode(byte[] value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, byte[] value) {
+ ByteString byteValue;
+ try {
+ Method m = ByteString.class.getDeclaredMethod("wrap", byte[].class);
+ m.setAccessible(true);
+ byteValue = (ByteString) m.invoke(null, value);
+ } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+ throw new RuntimeException(e);
+ }
+ literalBuilder.setBinary(byteValue);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/BooleanLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/BooleanLiteralNode.java
new file mode 100644
index 000000000000..dcee247b9df1
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/BooleanLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.BooleanTypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class BooleanLiteralNode extends LiteralNodeWithValue {
+ public BooleanLiteralNode(Boolean value) {
+ super(value, new BooleanTypeNode(true));
+ }
+
+ public BooleanLiteralNode(Boolean value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, Boolean value) {
+ literalBuilder.setBoolean(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ByteLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ByteLiteralNode.java
new file mode 100644
index 000000000000..7042ea9ca145
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ByteLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.I8TypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class ByteLiteralNode extends LiteralNodeWithValue {
+ public ByteLiteralNode(Byte value) {
+ super(value, new I8TypeNode(true));
+ }
+
+ public ByteLiteralNode(Byte value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, Byte value) {
+ literalBuilder.setI8(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/CastNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/CastNode.java
new file mode 100644
index 000000000000..1acbcf6acaa3
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/CastNode.java
@@ -0,0 +1,53 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression;
+
+import java.io.Serializable;
+
+public class CastNode implements ExpressionNode, Serializable {
+ private final TypeNode typeNode;
+ private final ExpressionNode expressionNode;
+
+ public final boolean ansiEnabled;
+
+ CastNode(TypeNode typeNode, ExpressionNode expressionNode, boolean ansiEnabled) {
+ this.typeNode = typeNode;
+ this.expressionNode = expressionNode;
+ this.ansiEnabled = ansiEnabled;
+ }
+
+ @Override
+ public Expression toProtobuf() {
+ Expression.Cast.Builder castBuilder = Expression.Cast.newBuilder();
+ castBuilder.setType(typeNode.toProtobuf());
+ castBuilder.setInput(expressionNode.toProtobuf());
+ if (ansiEnabled) {
+ // Throw exception on failure.
+ castBuilder.setFailureBehaviorValue(2);
+ } else {
+ // Return null on failure.
+ castBuilder.setFailureBehaviorValue(1);
+ }
+ Expression.Builder builder = Expression.newBuilder();
+ builder.setCast(castBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/DateLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/DateLiteralNode.java
new file mode 100644
index 000000000000..2c79f816124e
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/DateLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.DateTypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class DateLiteralNode extends LiteralNodeWithValue {
+ public DateLiteralNode(Integer value) {
+ super(value, new DateTypeNode(true));
+ }
+
+ public DateLiteralNode(Integer value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, Integer value) {
+ literalBuilder.setDate(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/DoubleLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/DoubleLiteralNode.java
new file mode 100644
index 000000000000..f36fc54859bd
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/DoubleLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.FP64TypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class DoubleLiteralNode extends LiteralNodeWithValue {
+ public DoubleLiteralNode(Double value) {
+ super(value, new FP64TypeNode(true));
+ }
+
+ public DoubleLiteralNode(Double value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, Double value) {
+ literalBuilder.setFp64(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ExpressionBuilder.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ExpressionBuilder.java
new file mode 100644
index 000000000000..523b25b59c79
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ExpressionBuilder.java
@@ -0,0 +1,217 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.*;
+
+import java.util.List;
+import java.util.Map;
+
+/** Contains helper functions for constructing substrait relations. */
+public class ExpressionBuilder {
+ private ExpressionBuilder() {}
+
+ public static Long newScalarFunction(Map functionMap, String functionName) {
+ if (!functionMap.containsKey(functionName)) {
+ Long functionId = (long) functionMap.size();
+ functionMap.put(functionName, functionId);
+ return functionId;
+ } else {
+ return functionMap.get(functionName);
+ }
+ }
+
+ public static NullLiteralNode makeNullLiteral(TypeNode typeNode) {
+ return new NullLiteralNode(typeNode);
+ }
+
+ public static BooleanLiteralNode makeBooleanLiteral(Boolean vBool) {
+ return new BooleanLiteralNode(vBool);
+ }
+
+ public static BooleanLiteralNode makeBooleanLiteral(Boolean vBool, TypeNode typeNode) {
+ return new BooleanLiteralNode(vBool, typeNode);
+ }
+
+ public static IntLiteralNode makeIntLiteral(Integer vInt) {
+ return new IntLiteralNode(vInt);
+ }
+
+ public static IntLiteralNode makeIntLiteral(Integer vInt, TypeNode typeNode) {
+ return new IntLiteralNode(vInt, typeNode);
+ }
+
+ public static ByteLiteralNode makeByteLiteral(Byte vByte) {
+ return new ByteLiteralNode(vByte);
+ }
+
+ public static ByteLiteralNode makeByteLiteral(Byte vByte, TypeNode typeNode) {
+ return new ByteLiteralNode(vByte, typeNode);
+ }
+
+ public static ShortLiteralNode makeShortLiteral(Short vShort) {
+ return new ShortLiteralNode(vShort);
+ }
+
+ public static ShortLiteralNode makeShortLiteral(Short vShort, TypeNode typeNode) {
+ return new ShortLiteralNode(vShort, typeNode);
+ }
+
+ public static LongLiteralNode makeLongLiteral(Long vLong) {
+ return new LongLiteralNode(vLong);
+ }
+
+ public static LongLiteralNode makeLongLiteral(Long vLong, TypeNode typeNode) {
+ return new LongLiteralNode(vLong, typeNode);
+ }
+
+ public static DoubleLiteralNode makeDoubleLiteral(Double vDouble) {
+ return new DoubleLiteralNode(vDouble);
+ }
+
+ public static DoubleLiteralNode makeDoubleLiteral(Double vDouble, TypeNode typeNode) {
+ return new DoubleLiteralNode(vDouble, typeNode);
+ }
+
+ public static FloatLiteralNode makeFloatLiteral(Float vFloat) {
+ return new FloatLiteralNode(vFloat);
+ }
+
+ public static FloatLiteralNode makeFloatLiteral(Float vFloat, TypeNode typeNode) {
+ return new FloatLiteralNode(vFloat, typeNode);
+ }
+
+ public static DateLiteralNode makeDateLiteral(Integer vDate) {
+ return new DateLiteralNode(vDate);
+ }
+
+ public static DateLiteralNode makeDateLiteral(Integer vDate, TypeNode typeNode) {
+ return new DateLiteralNode(vDate, typeNode);
+ }
+
+ public static TimestampLiteralNode makeTimestampLiteral(Long vTimestamp) {
+ return new TimestampLiteralNode(vTimestamp);
+ }
+
+ public static TimestampLiteralNode makeTimestampLiteral(Long vTimestamp, TypeNode typeNode) {
+ return new TimestampLiteralNode(vTimestamp, typeNode);
+ }
+
+ public static StringLiteralNode makeStringLiteral(String vString) {
+ return new StringLiteralNode(vString);
+ }
+
+ public static StringLiteralNode makeStringLiteral(String vString, TypeNode typeNode) {
+ return new StringLiteralNode(vString, typeNode);
+ }
+
+ public static BinaryLiteralNode makeBinaryLiteral(byte[] vBytes) {
+ return new BinaryLiteralNode(vBytes);
+ }
+
+ public static BinaryLiteralNode makeBinaryLiteral(byte[] vBytes, TypeNode typeNode) {
+ return new BinaryLiteralNode(vBytes, typeNode);
+ }
+
+ public static LiteralNode makeLiteral(Object obj, TypeNode typeNode) {
+ if (obj == null) {
+ return makeNullLiteral(typeNode);
+ }
+ if (typeNode instanceof BooleanTypeNode) {
+ return makeBooleanLiteral((Boolean) obj, typeNode);
+ }
+ if (typeNode instanceof I8TypeNode) {
+ return makeByteLiteral((Byte) obj, typeNode);
+ }
+ if (typeNode instanceof I16TypeNode) {
+ return makeShortLiteral((Short) obj, typeNode);
+ }
+ if (typeNode instanceof I32TypeNode) {
+ System.out.println("makeLiteral " + obj.getClass().getName());
+ return makeIntLiteral(Integer.valueOf(obj.toString()), typeNode);
+ }
+ if (typeNode instanceof I64TypeNode) {
+ return makeLongLiteral((Long) obj, typeNode);
+ }
+ if (typeNode instanceof FP32TypeNode) {
+ return makeFloatLiteral((Float) obj, typeNode);
+ }
+ if (typeNode instanceof FP64TypeNode) {
+ return makeDoubleLiteral((Double) obj, typeNode);
+ }
+ if (typeNode instanceof DateTypeNode) {
+ return makeDateLiteral((Integer) obj, typeNode);
+ }
+ if (typeNode instanceof TimestampTypeNode) {
+ return makeTimestampLiteral((Long) obj, typeNode);
+ }
+ if (typeNode instanceof StringTypeNode) {
+ return makeStringLiteral(obj.toString(), typeNode);
+ }
+ if (typeNode instanceof BinaryTypeNode) {
+ return makeBinaryLiteral((byte[]) obj, typeNode);
+ }
+
+ throw new RuntimeException(
+ String.format(
+ "Type not supported: %s, obj: %s, class: %s",
+ typeNode.toString(), obj.toString(), obj.getClass().toString()));
+ }
+
+ public static void checkDecimalScale(int scale) {
+ if (scale < 0) {
+ // Substrait don't support decimal type with negative scale.
+ throw new UnsupportedOperationException(
+ String.format("DecimalType with negative scale not supported: %s.", scale));
+ }
+ }
+
+ public static ScalarFunctionNode makeScalarFunction(
+ Long functionId, List expressionNodes, TypeNode typeNode) {
+ return new ScalarFunctionNode(functionId, expressionNodes, typeNode);
+ }
+
+ public static SelectionNode makeSelection(Integer fieldIdx) {
+ return new SelectionNode(fieldIdx);
+ }
+
+ public static SelectionNode makeSelection(Integer fieldIdx, Integer childFieldIdx) {
+ return new SelectionNode(fieldIdx, childFieldIdx);
+ }
+
+ public static AggregateFunctionNode makeAggregateFunction(
+ Long functionId,
+ List expressionNodes,
+ String phase,
+ TypeNode outputTypeNode) {
+ return new AggregateFunctionNode(functionId, expressionNodes, phase, outputTypeNode);
+ }
+
+ public static CastNode makeCast(
+ TypeNode typeNode, ExpressionNode expressionNode, boolean ansiEnabled) {
+ return new CastNode(typeNode, expressionNode, ansiEnabled);
+ }
+
+ public static StringMapNode makeStringMap(Map values) {
+ return new StringMapNode(values);
+ }
+
+ public static SingularOrListNode makeSingularOrListNode(
+ ExpressionNode value, List expressionNodes) {
+ return new SingularOrListNode(value, expressionNodes);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ExpressionNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ExpressionNode.java
new file mode 100644
index 000000000000..e0d248e79a5a
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ExpressionNode.java
@@ -0,0 +1,29 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import io.substrait.proto.Expression;
+
+/** Contains helper functions for constructing Substrait expressions. */
+public interface ExpressionNode {
+ /**
+ * Converts a Expression into a protobuf.
+ *
+ * @return A rel protobuf
+ */
+ Expression toProtobuf();
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/FloatLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/FloatLiteralNode.java
new file mode 100644
index 000000000000..5f1b5b0a66c1
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/FloatLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.FP32TypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class FloatLiteralNode extends LiteralNodeWithValue {
+ public FloatLiteralNode(Float value) {
+ super(value, new FP32TypeNode(true));
+ }
+
+ public FloatLiteralNode(Float value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, Float value) {
+ literalBuilder.setFp32(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/IfThenNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/IfThenNode.java
new file mode 100644
index 000000000000..3ea85ba8c96f
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/IfThenNode.java
@@ -0,0 +1,63 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import io.substrait.proto.Expression;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class IfThenNode implements ExpressionNode, Serializable {
+
+ private final List ifNodes = new ArrayList<>();
+ private final List thenNodes = new ArrayList<>();
+
+ private final ExpressionNode elseValue;
+
+ public IfThenNode(
+ List ifNodes, List thenNodes, ExpressionNode elseValue) {
+ this.ifNodes.addAll(ifNodes);
+ this.thenNodes.addAll(thenNodes);
+ this.elseValue = elseValue;
+ }
+
+ @Override
+ public Expression toProtobuf() {
+ if (ifNodes.size() != thenNodes.size()) {
+ throw new RuntimeException("The length of if nodes and then nodes is different.");
+ }
+ Expression.IfThen.Builder ifThenBuilder = Expression.IfThen.newBuilder();
+
+ int ifNodesLen = ifNodes.size();
+ for (int i = 0; i < ifNodesLen; i++) {
+ Expression.IfThen.IfClause.Builder ifClauseBuilder = Expression.IfThen.IfClause.newBuilder();
+ ifClauseBuilder.setIf(ifNodes.get(i).toProtobuf());
+ ifClauseBuilder.setThen(thenNodes.get(i).toProtobuf());
+
+ ifThenBuilder.addIfs(ifClauseBuilder.build());
+ }
+
+ if (elseValue != null) {
+ ifThenBuilder.setElse(elseValue.toProtobuf());
+ }
+
+ Expression.Builder builder = Expression.newBuilder();
+ builder.setIfThen(ifThenBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/IntLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/IntLiteralNode.java
new file mode 100644
index 000000000000..79ef659b1217
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/IntLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.I32TypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class IntLiteralNode extends LiteralNodeWithValue {
+ public IntLiteralNode(Integer value) {
+ super(value, new I32TypeNode(true));
+ }
+
+ public IntLiteralNode(Integer value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, Integer value) {
+ literalBuilder.setI32(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/LiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/LiteralNode.java
new file mode 100644
index 000000000000..5363bbe2ec78
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/LiteralNode.java
@@ -0,0 +1,44 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression;
+
+import java.io.Serializable;
+
+public abstract class LiteralNode implements ExpressionNode, Serializable {
+ private final TypeNode typeNode;
+
+ LiteralNode(TypeNode typeNode) {
+ this.typeNode = typeNode;
+ }
+
+ public TypeNode getTypeNode() {
+ return typeNode;
+ }
+
+ protected abstract Expression.Literal getLiteral();
+
+ @Override
+ public Expression toProtobuf() {
+ Expression.Builder builder = Expression.newBuilder();
+ builder.setLiteral(getLiteral());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/LiteralNodeWithValue.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/LiteralNodeWithValue.java
new file mode 100644
index 000000000000..d452d17e6f27
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/LiteralNodeWithValue.java
@@ -0,0 +1,44 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.*;
+
+import io.substrait.proto.Expression;
+
+public abstract class LiteralNodeWithValue extends LiteralNode {
+ private final T value;
+
+ LiteralNodeWithValue(T value, TypeNode typeNode) {
+ super(typeNode);
+ this.value = value;
+ }
+
+ public T getValue() {
+ return value;
+ }
+
+ @Override
+ protected Expression.Literal getLiteral() {
+ T value = getValue();
+ Expression.Literal.Builder literalBuilder = Expression.Literal.newBuilder();
+ updateLiteralBuilder(literalBuilder, value);
+ return literalBuilder.build();
+ }
+
+ protected abstract void updateLiteralBuilder(Expression.Literal.Builder literalBuilder, T value);
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/LongLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/LongLiteralNode.java
new file mode 100644
index 000000000000..334e09912d1c
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/LongLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.I64TypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class LongLiteralNode extends LiteralNodeWithValue {
+ public LongLiteralNode(Long value) {
+ super(value, new I64TypeNode(true));
+ }
+
+ public LongLiteralNode(Long value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, Long value) {
+ literalBuilder.setI64(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/NullLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/NullLiteralNode.java
new file mode 100644
index 000000000000..5eba55ddd27a
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/NullLiteralNode.java
@@ -0,0 +1,34 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression;
+
+public class NullLiteralNode extends LiteralNode {
+ public NullLiteralNode(TypeNode typeNode) {
+ super(typeNode);
+ }
+
+ @Override
+ protected Expression.Literal getLiteral() {
+ Expression.Literal.Builder literalBuilder = Expression.Literal.newBuilder();
+ literalBuilder.setNull(getTypeNode().toProtobuf());
+ return literalBuilder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ScalarFunctionNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ScalarFunctionNode.java
new file mode 100644
index 000000000000..fa24dad880c4
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ScalarFunctionNode.java
@@ -0,0 +1,54 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression;
+import io.substrait.proto.FunctionArgument;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ScalarFunctionNode implements ExpressionNode, Serializable {
+ private final Long functionId;
+ private final List expressionNodes = new ArrayList<>();
+ private final TypeNode typeNode;
+
+ ScalarFunctionNode(Long functionId, List expressionNodes, TypeNode typeNode) {
+ this.functionId = functionId;
+ this.expressionNodes.addAll(expressionNodes);
+ this.typeNode = typeNode;
+ }
+
+ @Override
+ public Expression toProtobuf() {
+ Expression.ScalarFunction.Builder scalarBuilder = Expression.ScalarFunction.newBuilder();
+ scalarBuilder.setFunctionReference(functionId.intValue());
+ for (ExpressionNode expressionNode : expressionNodes) {
+ FunctionArgument.Builder functionArgument = FunctionArgument.newBuilder();
+ functionArgument.setValue(expressionNode.toProtobuf());
+ scalarBuilder.addArguments(functionArgument.build());
+ }
+ scalarBuilder.setOutputType(typeNode.toProtobuf());
+
+ Expression.Builder builder = Expression.newBuilder();
+ builder.setScalarFunction(scalarBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/SelectionNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/SelectionNode.java
new file mode 100644
index 000000000000..e3a95bccc333
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/SelectionNode.java
@@ -0,0 +1,84 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import io.substrait.proto.Expression;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class SelectionNode implements ExpressionNode, Serializable {
+ private final Integer fieldIndex;
+
+ // The nested indices of child field. For case like a.b.c, the index of c is put at last.
+ private final List nestedChildIndices = new ArrayList<>();
+
+ SelectionNode(Integer fieldIndex) {
+ this.fieldIndex = fieldIndex;
+ }
+
+ SelectionNode(Integer fieldIndex, Integer childIndex) {
+ this.fieldIndex = fieldIndex;
+ this.nestedChildIndices.add(childIndex);
+ }
+
+ public SelectionNode addNestedChildIdx(Integer childIndex) {
+ this.nestedChildIndices.add(childIndex);
+ return this;
+ }
+
+ public Expression.ReferenceSegment createRef(
+ Integer childIdx, Expression.ReferenceSegment childRef) {
+ Expression.ReferenceSegment.StructField.Builder structBuilder =
+ Expression.ReferenceSegment.StructField.newBuilder();
+ structBuilder.setField(childIdx);
+ if (childRef != null) {
+ structBuilder.setChild(childRef);
+ }
+
+ Expression.ReferenceSegment.Builder refBuilder = Expression.ReferenceSegment.newBuilder();
+ refBuilder.setStructField(structBuilder.build());
+ return refBuilder.build();
+ }
+
+ @Override
+ public Expression toProtobuf() {
+ Expression.ReferenceSegment.StructField.Builder structBuilder =
+ Expression.ReferenceSegment.StructField.newBuilder();
+ structBuilder.setField(fieldIndex);
+
+ // Handle the nested field indices.
+ if (!nestedChildIndices.isEmpty()) {
+ Expression.ReferenceSegment childRef = null;
+ for (int i = nestedChildIndices.size() - 1; i >= 0; i--) {
+ childRef = createRef(nestedChildIndices.get(i), childRef);
+ }
+ structBuilder.setChild(childRef);
+ }
+
+ Expression.ReferenceSegment.Builder refBuilder = Expression.ReferenceSegment.newBuilder();
+ refBuilder.setStructField(structBuilder.build());
+
+ Expression.FieldReference.Builder fieldBuilder = Expression.FieldReference.newBuilder();
+ fieldBuilder.setDirectReference(refBuilder.build());
+
+ Expression.Builder builder = Expression.newBuilder();
+ builder.setSelection(fieldBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ShortLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ShortLiteralNode.java
new file mode 100644
index 000000000000..c352a1106197
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/ShortLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.I16TypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class ShortLiteralNode extends LiteralNodeWithValue {
+ public ShortLiteralNode(Short value) {
+ super(value, new I16TypeNode(true));
+ }
+
+ public ShortLiteralNode(Short value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, Short value) {
+ literalBuilder.setI16(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/SingularOrListNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/SingularOrListNode.java
new file mode 100644
index 000000000000..c55791a08535
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/SingularOrListNode.java
@@ -0,0 +1,45 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import io.substrait.proto.Expression;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class SingularOrListNode implements ExpressionNode, Serializable {
+ private final ExpressionNode value;
+ private final List listNodes = new ArrayList<>();
+
+ SingularOrListNode(ExpressionNode value, List listNodes) {
+ this.value = value;
+ this.listNodes.addAll(listNodes);
+ }
+
+ @Override
+ public Expression toProtobuf() {
+ Expression.SingularOrList.Builder builder = Expression.SingularOrList.newBuilder();
+ builder.setValue(value.toProtobuf());
+ for (ExpressionNode expressionNode : listNodes) {
+ builder.addOptions(expressionNode.toProtobuf());
+ }
+ Expression.Builder expressionBuilder = Expression.newBuilder();
+ expressionBuilder.setSingularOrList(builder);
+ return expressionBuilder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/StringLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/StringLiteralNode.java
new file mode 100644
index 000000000000..c17ecc5bb3e0
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/StringLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.StringTypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class StringLiteralNode extends LiteralNodeWithValue {
+ public StringLiteralNode(String value) {
+ super(value, new StringTypeNode(true));
+ }
+
+ public StringLiteralNode(String value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, String value) {
+ literalBuilder.setString(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/StringMapNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/StringMapNode.java
new file mode 100644
index 000000000000..9b69e3a79851
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/StringMapNode.java
@@ -0,0 +1,52 @@
+/*
+ * 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.gluten.substrait.expression;
+
+import io.substrait.proto.Expression;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+public class StringMapNode implements ExpressionNode, Serializable {
+ private final Map values = new HashMap<>();
+
+ public StringMapNode(Map values) {
+ this.values.putAll(values);
+ }
+
+ @Override
+ public Expression toProtobuf() {
+ Expression.Literal.Builder literalBuilder = Expression.Literal.newBuilder();
+ Expression.Literal.Map.KeyValue.Builder keyValueBuilder =
+ Expression.Literal.Map.KeyValue.newBuilder();
+ Expression.Literal.Map.Builder mapBuilder = Expression.Literal.Map.newBuilder();
+ for (Map.Entry entry : values.entrySet()) {
+ literalBuilder.setString(entry.getKey());
+ keyValueBuilder.setKey(literalBuilder.build());
+ literalBuilder.setString(entry.getValue());
+ keyValueBuilder.setValue(literalBuilder.build());
+ mapBuilder.addKeyValues(keyValueBuilder.build());
+ }
+ literalBuilder.setMap(mapBuilder.build());
+
+ Expression.Builder builder = Expression.newBuilder();
+ builder.setLiteral(literalBuilder.build());
+
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/TimestampLiteralNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/TimestampLiteralNode.java
new file mode 100644
index 000000000000..ec253edbc4a3
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/expression/TimestampLiteralNode.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.gluten.substrait.expression;
+
+import org.apache.gluten.substrait.type.TimestampTypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Expression.Literal.Builder;
+
+public class TimestampLiteralNode extends LiteralNodeWithValue {
+ public TimestampLiteralNode(Long value) {
+ super(value, new TimestampTypeNode(true));
+ }
+
+ public TimestampLiteralNode(Long value, TypeNode typeNode) {
+ super(value, typeNode);
+ }
+
+ @Override
+ protected void updateLiteralBuilder(Builder literalBuilder, Long value) {
+ literalBuilder.setTimestamp(value);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/extensions/AdvancedExtensionNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/extensions/AdvancedExtensionNode.java
new file mode 100644
index 000000000000..f313888d4ffb
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/extensions/AdvancedExtensionNode.java
@@ -0,0 +1,53 @@
+/*
+ * 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.gluten.substrait.extensions;
+
+import com.google.protobuf.Any;
+import io.substrait.proto.AdvancedExtension;
+
+import java.io.Serializable;
+
+public class AdvancedExtensionNode implements Serializable {
+
+ // An optimization is helpful information that don't influence semantics. May
+ // be ignored by a consumer.
+ private final Any optimization;
+
+ // An enhancement alter semantics. Cannot be ignored by a consumer.
+ private final Any enhancement;
+
+ public AdvancedExtensionNode(Any enhancement) {
+ this.optimization = null;
+ this.enhancement = enhancement;
+ }
+
+ public AdvancedExtensionNode(Any optimization, Any enhancement) {
+ this.optimization = optimization;
+ this.enhancement = enhancement;
+ }
+
+ public AdvancedExtension toProtobuf() {
+ AdvancedExtension.Builder extensionBuilder = AdvancedExtension.newBuilder();
+ if (optimization != null) {
+ extensionBuilder.setOptimization(optimization);
+ }
+ if (enhancement != null) {
+ extensionBuilder.setEnhancement(enhancement);
+ }
+ return extensionBuilder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/extensions/ExtensionBuilder.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/extensions/ExtensionBuilder.java
new file mode 100644
index 000000000000..215116a57d11
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/extensions/ExtensionBuilder.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gluten.substrait.extensions;
+
+import com.google.protobuf.Any;
+
+public class ExtensionBuilder {
+ private ExtensionBuilder() {}
+
+ public static FunctionMappingNode makeFunctionMapping(String name, Long functionId) {
+ return new FunctionMappingNode(name, functionId);
+ }
+
+ public static AdvancedExtensionNode makeAdvancedExtension(Any enhancement) {
+ return new AdvancedExtensionNode(enhancement);
+ }
+
+ public static AdvancedExtensionNode makeAdvancedExtension(Any optimization, Any enhancement) {
+ return new AdvancedExtensionNode(optimization, enhancement);
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/extensions/FunctionMappingNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/extensions/FunctionMappingNode.java
new file mode 100644
index 000000000000..71ce48a876c7
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/extensions/FunctionMappingNode.java
@@ -0,0 +1,42 @@
+/*
+ * 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.gluten.substrait.extensions;
+
+import io.substrait.proto.SimpleExtensionDeclaration;
+
+import java.io.Serializable;
+
+public class FunctionMappingNode implements Serializable {
+ private final String name;
+ private final Long functionId;
+
+ public FunctionMappingNode(String name, Long functionId) {
+ this.name = name;
+ this.functionId = functionId;
+ }
+
+ public SimpleExtensionDeclaration toProtobuf() {
+ SimpleExtensionDeclaration.ExtensionFunction.Builder funcBuilder =
+ SimpleExtensionDeclaration.ExtensionFunction.newBuilder();
+ funcBuilder.setFunctionAnchor(functionId.intValue());
+ funcBuilder.setName(name);
+
+ SimpleExtensionDeclaration.Builder declaration = SimpleExtensionDeclaration.newBuilder();
+ declaration.setExtensionFunction(funcBuilder.build());
+ return declaration.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/plan/PlanBuilder.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/plan/PlanBuilder.java
new file mode 100644
index 000000000000..4ece867be27a
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/plan/PlanBuilder.java
@@ -0,0 +1,91 @@
+/*
+ * 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.gluten.substrait.plan;
+
+import org.apache.gluten.substrait.FunctionMappings;
+import org.apache.gluten.substrait.SubstraitContext;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+import org.apache.gluten.substrait.extensions.ExtensionBuilder;
+import org.apache.gluten.substrait.extensions.FunctionMappingNode;
+import org.apache.gluten.substrait.rel.RelNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class PlanBuilder {
+
+ public static byte[] EMPTY_PLAN = empty().toProtobuf().toByteArray();
+
+ private PlanBuilder() {}
+
+ public static PlanNode makePlan(
+ List mappingNodes, List relNodes, List outNames) {
+ return new PlanNode(mappingNodes, relNodes, outNames);
+ }
+
+ public static PlanNode makePlan(
+ List mappingNodes,
+ List relNodes,
+ List outNames,
+ TypeNode outputSchema,
+ AdvancedExtensionNode extension) {
+ return new PlanNode(mappingNodes, relNodes, outNames, outputSchema, extension);
+ }
+
+ public static PlanNode makePlan(AdvancedExtensionNode extension) {
+ return new PlanNode(extension);
+ }
+
+ public static PlanNode makePlan(
+ SubstraitContext subCtx, List relNodes, List outNames) {
+ return makePlan(subCtx, relNodes, outNames, null, null);
+ }
+
+ public static PlanNode makePlan(
+ SubstraitContext subCtx,
+ List relNodes,
+ List outNames,
+ TypeNode outputSchema,
+ AdvancedExtensionNode extension) {
+ if (subCtx == null) {
+ throw new NullPointerException("ColumnarWholestageTransformer cannot doTansform.");
+ }
+ List mappingNodes = new ArrayList<>();
+
+ for (Map.Entry entry : subCtx.registeredFunction().entrySet()) {
+ FunctionMappingNode mappingNode =
+ ExtensionBuilder.makeFunctionMapping(
+ FunctionMappings.toSubstraitFunction(entry.getKey()),
+ entry.getValue());
+ mappingNodes.add(mappingNode);
+ }
+ if (extension != null || outputSchema != null) {
+ return makePlan(mappingNodes, relNodes, outNames, outputSchema, extension);
+ }
+ return makePlan(mappingNodes, relNodes, outNames);
+ }
+
+ public static PlanNode makePlan(SubstraitContext subCtx, ArrayList relNodes) {
+ return makePlan(subCtx, relNodes, new ArrayList<>());
+ }
+
+ public static PlanNode empty() {
+ return makePlan(new SubstraitContext(), new ArrayList<>(), new ArrayList<>());
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/plan/PlanNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/plan/PlanNode.java
new file mode 100644
index 000000000000..87273729194f
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/plan/PlanNode.java
@@ -0,0 +1,91 @@
+/*
+ * 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.gluten.substrait.plan;
+
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+import org.apache.gluten.substrait.extensions.FunctionMappingNode;
+import org.apache.gluten.substrait.rel.RelNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.Plan;
+import io.substrait.proto.PlanRel;
+import io.substrait.proto.RelRoot;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class PlanNode implements Serializable {
+ private final List mappingNodes = new ArrayList<>();
+ private final List relNodes = new ArrayList<>();
+ private final List outNames = new ArrayList<>();
+
+ private TypeNode outputSchema = null;
+ private AdvancedExtensionNode extension = null;
+
+ PlanNode(List mappingNodes, List relNodes, List outNames) {
+ this.mappingNodes.addAll(mappingNodes);
+ this.relNodes.addAll(relNodes);
+ this.outNames.addAll(outNames);
+ }
+
+ PlanNode(
+ List mappingNodes,
+ List relNodes,
+ List outNames,
+ TypeNode outputSchema,
+ AdvancedExtensionNode extension) {
+ this.mappingNodes.addAll(mappingNodes);
+ this.relNodes.addAll(relNodes);
+ this.outNames.addAll(outNames);
+ this.outputSchema = outputSchema;
+ this.extension = extension;
+ }
+
+ PlanNode(AdvancedExtensionNode extension) {
+ this.extension = extension;
+ }
+
+ public Plan toProtobuf() {
+ Plan.Builder planBuilder = Plan.newBuilder();
+ // add the extension functions
+ for (FunctionMappingNode mappingNode : mappingNodes) {
+ planBuilder.addExtensions(mappingNode.toProtobuf());
+ }
+
+ for (RelNode relNode : relNodes) {
+ PlanRel.Builder planRelBuilder = PlanRel.newBuilder();
+
+ RelRoot.Builder relRootBuilder = RelRoot.newBuilder();
+ relRootBuilder.setInput(relNode.toProtobuf());
+ for (String name : outNames) {
+ relRootBuilder.addNames(name);
+ }
+ if (outputSchema != null) {
+ relRootBuilder.setOutputSchema(outputSchema.toProtobuf().getStruct());
+ }
+ planRelBuilder.setRoot(relRootBuilder.build());
+
+ planBuilder.addRelations(planRelBuilder.build());
+ }
+
+ if (extension != null) {
+ planBuilder.setAdvancedExtensions(extension.toProtobuf());
+ }
+ return planBuilder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/AggregateRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/AggregateRelNode.java
new file mode 100644
index 000000000000..b75df46d639f
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/AggregateRelNode.java
@@ -0,0 +1,98 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.expression.AggregateFunctionNode;
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.AggregateRel;
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class AggregateRelNode implements RelNode, Serializable {
+ private final RelNode input;
+ private final List groupings = new ArrayList<>();
+ private final List aggregateFunctionNodes = new ArrayList<>();
+
+ private final List filters = new ArrayList<>();
+ private final AdvancedExtensionNode extensionNode;
+
+ AggregateRelNode(
+ RelNode input,
+ List groupings,
+ List aggregateFunctionNodes,
+ List filters) {
+ this.input = input;
+ this.groupings.addAll(groupings);
+ this.aggregateFunctionNodes.addAll(aggregateFunctionNodes);
+ this.filters.addAll(filters);
+ this.extensionNode = null;
+ }
+
+ AggregateRelNode(
+ RelNode input,
+ List groupings,
+ List aggregateFunctionNodes,
+ List filters,
+ AdvancedExtensionNode extensionNode) {
+ this.input = input;
+ this.groupings.addAll(groupings);
+ this.aggregateFunctionNodes.addAll(aggregateFunctionNodes);
+ this.filters.addAll(filters);
+ this.extensionNode = extensionNode;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+
+ AggregateRel.Grouping.Builder groupingBuilder = AggregateRel.Grouping.newBuilder();
+ for (ExpressionNode exprNode : groupings) {
+ groupingBuilder.addGroupingExpressions(exprNode.toProtobuf());
+ }
+
+ AggregateRel.Builder aggBuilder = AggregateRel.newBuilder();
+ aggBuilder.setCommon(relCommonBuilder.build());
+ aggBuilder.addGroupings(groupingBuilder.build());
+
+ for (int i = 0; i < aggregateFunctionNodes.size(); i++) {
+ AggregateRel.Measure.Builder measureBuilder = AggregateRel.Measure.newBuilder();
+ measureBuilder.setMeasure(aggregateFunctionNodes.get(i).toProtobuf());
+ // Set the filter expression if valid.
+ if (this.filters.get(i) != null) {
+ measureBuilder.setFilter(this.filters.get(i).toProtobuf());
+ }
+ aggBuilder.addMeasures(measureBuilder.build());
+ }
+
+ if (input != null) {
+ aggBuilder.setInput(input.toProtobuf());
+ }
+ if (extensionNode != null) {
+ aggBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+ Rel.Builder builder = Rel.newBuilder();
+ builder.setAggregate(aggBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/CrossRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/CrossRelNode.java
new file mode 100644
index 000000000000..338a10b4d6c3
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/CrossRelNode.java
@@ -0,0 +1,72 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.CrossRel;
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+
+import java.io.Serializable;
+
+public class CrossRelNode implements RelNode, Serializable {
+ private final RelNode left;
+ private final RelNode right;
+ private final CrossRel.JoinType joinType;
+ private final ExpressionNode expression;
+ private final AdvancedExtensionNode extensionNode;
+
+ CrossRelNode(
+ RelNode left,
+ RelNode right,
+ CrossRel.JoinType joinType,
+ ExpressionNode expression,
+ AdvancedExtensionNode extensionNode) {
+ this.left = left;
+ this.right = right;
+ this.joinType = joinType;
+ this.expression = expression;
+ this.extensionNode = extensionNode;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+
+ CrossRel.Builder crossRelBuilder = CrossRel.newBuilder();
+ crossRelBuilder.setCommon(relCommonBuilder.build());
+
+ crossRelBuilder.setType(joinType);
+
+ if (left != null) {
+ crossRelBuilder.setLeft(left.toProtobuf());
+ }
+ if (right != null) {
+ crossRelBuilder.setRight(right.toProtobuf());
+ }
+ if (expression != null) {
+ crossRelBuilder.setExpression(expression.toProtobuf());
+ }
+ if (extensionNode != null) {
+ crossRelBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+ return Rel.newBuilder().setCross(crossRelBuilder.build()).build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/ExpandRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/ExpandRelNode.java
new file mode 100644
index 000000000000..25d007e5641f
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/ExpandRelNode.java
@@ -0,0 +1,79 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.ExpandRel;
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ExpandRelNode implements RelNode, Serializable {
+ private final RelNode input;
+ private final List> projections = new ArrayList<>();
+
+ private final AdvancedExtensionNode extensionNode;
+
+ public ExpandRelNode(
+ RelNode input, List> projections, AdvancedExtensionNode extensionNode) {
+ this.input = input;
+ this.projections.addAll(projections);
+ this.extensionNode = extensionNode;
+ }
+
+ public ExpandRelNode(RelNode input, List> projections) {
+ this.input = input;
+ this.projections.addAll(projections);
+ this.extensionNode = null;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+
+ ExpandRel.Builder expandBuilder = ExpandRel.newBuilder();
+ expandBuilder.setCommon(relCommonBuilder.build());
+
+ if (input != null) {
+ expandBuilder.setInput(input.toProtobuf());
+ }
+
+ for (List projectList : projections) {
+ ExpandRel.ExpandField.Builder expandFieldBuilder = ExpandRel.ExpandField.newBuilder();
+ ExpandRel.SwitchingField.Builder switchingField = ExpandRel.SwitchingField.newBuilder();
+ for (ExpressionNode exprNode : projectList) {
+ switchingField.addDuplicates(exprNode.toProtobuf());
+ }
+ expandFieldBuilder.setSwitchingField(switchingField.build());
+ expandBuilder.addFields(expandFieldBuilder.build());
+ }
+
+ if (extensionNode != null) {
+ expandBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+
+ Rel.Builder builder = Rel.newBuilder();
+ builder.setExpand(expandBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/FetchRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/FetchRelNode.java
new file mode 100644
index 000000000000..3c8b991c5b5e
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/FetchRelNode.java
@@ -0,0 +1,69 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.FetchRel;
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+
+import java.io.Serializable;
+
+public class FetchRelNode implements RelNode, Serializable {
+ private final RelNode input;
+ private final Long offset;
+ private final Long count;
+
+ private final AdvancedExtensionNode extensionNode;
+
+ FetchRelNode(RelNode input, Long offset, Long count) {
+ this.input = input;
+ this.offset = offset;
+ this.count = count;
+ this.extensionNode = null;
+ }
+
+ FetchRelNode(RelNode input, Long offset, Long count, AdvancedExtensionNode extensionNode) {
+ this.input = input;
+ this.offset = offset;
+ this.count = count;
+ this.extensionNode = extensionNode;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+
+ FetchRel.Builder fetchRelBuilder = FetchRel.newBuilder();
+ fetchRelBuilder.setCommon(relCommonBuilder.build());
+ if (input != null) {
+ fetchRelBuilder.setInput(input.toProtobuf());
+ }
+ fetchRelBuilder.setOffset(offset);
+ fetchRelBuilder.setCount(count);
+
+ if (extensionNode != null) {
+ fetchRelBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+
+ Rel.Builder relBuilder = Rel.newBuilder();
+ relBuilder.setFetch(fetchRelBuilder.build());
+ return relBuilder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/FilterRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/FilterRelNode.java
new file mode 100644
index 000000000000..356b12b292ae
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/FilterRelNode.java
@@ -0,0 +1,63 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.FilterRel;
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+
+import java.io.Serializable;
+
+public class FilterRelNode implements RelNode, Serializable {
+ private final RelNode input;
+ private final ExpressionNode condition;
+ private final AdvancedExtensionNode extensionNode;
+
+ FilterRelNode(RelNode input, ExpressionNode condition) {
+ this.input = input;
+ this.condition = condition;
+ this.extensionNode = null;
+ }
+
+ FilterRelNode(RelNode input, ExpressionNode condition, AdvancedExtensionNode extensionNode) {
+ this.input = input;
+ this.condition = condition;
+ this.extensionNode = extensionNode;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+
+ FilterRel.Builder filterBuilder = FilterRel.newBuilder();
+ filterBuilder.setCommon(relCommonBuilder.build());
+ if (input != null) {
+ filterBuilder.setInput(input.toProtobuf());
+ }
+ filterBuilder.setCondition(condition.toProtobuf());
+ if (extensionNode != null) {
+ filterBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+ Rel.Builder builder = Rel.newBuilder();
+ builder.setFilter(filterBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/GenerateRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/GenerateRelNode.java
new file mode 100644
index 000000000000..efd470891905
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/GenerateRelNode.java
@@ -0,0 +1,84 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.GenerateRel;
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+
+import java.io.Serializable;
+import java.util.List;
+
+public class GenerateRelNode implements RelNode, Serializable {
+ private final RelNode input;
+ private final ExpressionNode generator;
+ private final List childOutput;
+ private final AdvancedExtensionNode extensionNode;
+ private final boolean outer;
+
+ GenerateRelNode(
+ RelNode input, ExpressionNode generator, List childOutput, boolean outer) {
+ this(input, generator, childOutput, null, outer);
+ }
+
+ GenerateRelNode(
+ RelNode input,
+ ExpressionNode generator,
+ List childOutput,
+ AdvancedExtensionNode extensionNode,
+ boolean outer) {
+ this.input = input;
+ this.generator = generator;
+ this.childOutput = childOutput;
+ this.extensionNode = extensionNode;
+ this.outer = outer;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ GenerateRel.Builder generateRelBuilder = GenerateRel.newBuilder();
+
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+ generateRelBuilder.setCommon(relCommonBuilder.build());
+
+ if (input != null) {
+ generateRelBuilder.setInput(input.toProtobuf());
+ }
+
+ if (generator != null) {
+ generateRelBuilder.setGenerator(generator.toProtobuf());
+ }
+
+ for (ExpressionNode node : childOutput) {
+ generateRelBuilder.addChildOutput(node.toProtobuf());
+ }
+
+ generateRelBuilder.setOuter(outer);
+
+ if (extensionNode != null) {
+ generateRelBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+
+ Rel.Builder relBuilder = Rel.newBuilder();
+ relBuilder.setGenerate(generateRelBuilder.build());
+ return relBuilder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/JoinRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/JoinRelNode.java
new file mode 100644
index 000000000000..714340cdf670
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/JoinRelNode.java
@@ -0,0 +1,77 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.JoinRel;
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+
+import java.io.Serializable;
+
+public class JoinRelNode implements RelNode, Serializable {
+ private final RelNode left;
+ private final RelNode right;
+ private final JoinRel.JoinType joinType;
+ private final ExpressionNode expression;
+ private final ExpressionNode postJoinFilter;
+ private final AdvancedExtensionNode extensionNode;
+
+ JoinRelNode(
+ RelNode left,
+ RelNode right,
+ JoinRel.JoinType joinType,
+ ExpressionNode expression,
+ ExpressionNode postJoinFilter,
+ AdvancedExtensionNode extensionNode) {
+ this.left = left;
+ this.right = right;
+ this.joinType = joinType;
+ this.expression = expression;
+ this.postJoinFilter = postJoinFilter;
+ this.extensionNode = extensionNode;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+ JoinRel.Builder joinBuilder = JoinRel.newBuilder();
+
+ joinBuilder.setType(joinType);
+
+ if (left != null) {
+ joinBuilder.setLeft(left.toProtobuf());
+ }
+ if (right != null) {
+ joinBuilder.setRight(right.toProtobuf());
+ }
+ if (expression != null) {
+ joinBuilder.setExpression(expression.toProtobuf());
+ }
+ if (postJoinFilter != null) {
+ joinBuilder.setPostJoinFilter(postJoinFilter.toProtobuf());
+ }
+ if (extensionNode != null) {
+ joinBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+
+ return Rel.newBuilder().setJoin(joinBuilder.build()).build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/ProjectRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/ProjectRelNode.java
new file mode 100644
index 000000000000..d1cccd159582
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/ProjectRelNode.java
@@ -0,0 +1,81 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.ProjectRel;
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ProjectRelNode implements RelNode, Serializable {
+ private final RelNode input;
+ private final List expressionNodes = new ArrayList<>();
+ private final AdvancedExtensionNode extensionNode;
+ private final int emitStartIndex;
+
+ ProjectRelNode(RelNode input, List expressionNodes, int emitStartIndex) {
+ this.input = input;
+ this.expressionNodes.addAll(expressionNodes);
+ this.extensionNode = null;
+ this.emitStartIndex = emitStartIndex;
+ }
+
+ ProjectRelNode(
+ RelNode input,
+ List expressionNodes,
+ AdvancedExtensionNode extensionNode,
+ int emitStartIndex) {
+ this.input = input;
+ this.expressionNodes.addAll(expressionNodes);
+ this.extensionNode = extensionNode;
+ this.emitStartIndex = emitStartIndex;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ if (emitStartIndex < 0) {
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+ } else {
+ RelCommon.Emit.Builder emitBuilder = RelCommon.Emit.newBuilder();
+ for (int i = 0; i < expressionNodes.size(); i++) {
+ emitBuilder.addOutputMapping(i + emitStartIndex);
+ }
+ relCommonBuilder.setEmit(emitBuilder.build());
+ }
+ ProjectRel.Builder projectBuilder = ProjectRel.newBuilder();
+ projectBuilder.setCommon(relCommonBuilder.build());
+ if (input != null) {
+ projectBuilder.setInput(input.toProtobuf());
+ }
+ for (ExpressionNode expressionNode : expressionNodes) {
+ projectBuilder.addExpressions(expressionNode.toProtobuf());
+ }
+ if (extensionNode != null) {
+ projectBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+ Rel.Builder builder = Rel.newBuilder();
+ builder.setProject(projectBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java
new file mode 100644
index 000000000000..6673337ffbcb
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java
@@ -0,0 +1,81 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.SubstraitContext;
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+import org.apache.gluten.substrait.type.ColumnTypeNode;
+import org.apache.gluten.substrait.type.TypeNode;
+
+import io.substrait.proto.CrossRel;
+import io.substrait.proto.JoinRel;
+import io.substrait.proto.SortField;
+
+import java.util.List;
+
+/** Contains helper functions for constructing substrait relations. */
+public class RelBuilder {
+ private RelBuilder() {}
+
+ public static RelNode makeFilterRel(
+ RelNode input, ExpressionNode condition, SubstraitContext context, Long operatorId) {
+ context.registerRelToOperator(operatorId);
+ return new FilterRelNode(input, condition);
+ }
+
+ public static RelNode makeFilterRel(
+ RelNode input,
+ ExpressionNode condition,
+ AdvancedExtensionNode extensionNode,
+ SubstraitContext context,
+ Long operatorId) {
+ context.registerRelToOperator(operatorId);
+ return new FilterRelNode(input, condition, extensionNode);
+ }
+
+ public static RelNode makeProjectRel(
+ RelNode input,
+ List expressionNodes,
+ SubstraitContext context,
+ Long operatorId) {
+ context.registerRelToOperator(operatorId);
+ return new ProjectRelNode(input, expressionNodes, -1);
+ }
+
+ public static RelNode makeProjectRel(
+ RelNode input,
+ List expressionNodes,
+ SubstraitContext context,
+ Long operatorId,
+ int emitStartIndex) {
+ context.registerRelToOperator(operatorId);
+ return new ProjectRelNode(input, expressionNodes, emitStartIndex);
+ }
+
+ public static RelNode makeProjectRel(
+ RelNode input,
+ List expressionNodes,
+ AdvancedExtensionNode extensionNode,
+ SubstraitContext context,
+ Long operatorId,
+ int emitStartIndex) {
+ context.registerRelToOperator(operatorId);
+ return new ProjectRelNode(input, expressionNodes, extensionNode, emitStartIndex);
+ }
+
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/RelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/RelNode.java
new file mode 100644
index 000000000000..3dacd947c0f4
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/RelNode.java
@@ -0,0 +1,31 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import io.substrait.proto.Rel;
+
+import java.io.Serializable;
+
+/** Contains helper functions for constructing substrait relations. */
+public interface RelNode extends Serializable {
+ /**
+ * Converts a Rel into a protobuf.
+ *
+ * @return A rel protobuf
+ */
+ Rel toProtobuf();
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/SortRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/SortRelNode.java
new file mode 100644
index 000000000000..90a541fa6f8b
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/SortRelNode.java
@@ -0,0 +1,71 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+import io.substrait.proto.SortField;
+import io.substrait.proto.SortRel;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class SortRelNode implements RelNode, Serializable {
+ private final RelNode input;
+ private final List sorts = new ArrayList<>();
+ private final AdvancedExtensionNode extensionNode;
+
+ public SortRelNode(RelNode input, List sorts, AdvancedExtensionNode extensionNode) {
+ this.input = input;
+ this.sorts.addAll(sorts);
+ this.extensionNode = extensionNode;
+ }
+
+ public SortRelNode(RelNode input, List sorts) {
+ this.input = input;
+ this.sorts.addAll(sorts);
+ this.extensionNode = null;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+
+ SortRel.Builder sortBuilder = SortRel.newBuilder();
+ sortBuilder.setCommon(relCommonBuilder.build());
+
+ if (input != null) {
+ sortBuilder.setInput(input.toProtobuf());
+ }
+
+ for (int i = 0; i < sorts.size(); i++) {
+ sortBuilder.addSorts(i, sorts.get(i));
+ }
+
+ if (extensionNode != null) {
+ sortBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+
+ Rel.Builder builder = Rel.newBuilder();
+ builder.setSort(sortBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/SplitInfo.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/SplitInfo.java
new file mode 100644
index 000000000000..6e2eeff7d8a4
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/SplitInfo.java
@@ -0,0 +1,34 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import com.google.protobuf.MessageOrBuilder;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * A serializable representation of a read split for native engine, including the file path and
+ * other information of the scan table. It is returned by {@link
+ * org.apache.gluten.execution.BasicScanExecTransformer#getSplitInfos()}.
+ */
+public interface SplitInfo extends Serializable {
+ /** The preferred locations where the table files returned by this read split can run faster. */
+ List preferredLocations();
+
+ MessageOrBuilder toProtobuf();
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/TopNNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/TopNNode.java
new file mode 100644
index 000000000000..a33d7aaca5b5
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/TopNNode.java
@@ -0,0 +1,76 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+import io.substrait.proto.SortField;
+import io.substrait.proto.TopNRel;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TopNNode implements RelNode, Serializable {
+ private final RelNode input;
+ private final Long count;
+ private final List sorts = new ArrayList<>();
+ private final AdvancedExtensionNode extensionNode;
+
+ public TopNNode(RelNode input, Long count, List sorts) {
+ this.input = input;
+ this.count = count;
+ this.sorts.addAll(sorts);
+ this.extensionNode = null;
+ }
+
+ public TopNNode(
+ RelNode input, Long count, List sorts, AdvancedExtensionNode extensionNode) {
+ this.input = input;
+ this.count = count;
+ this.sorts.addAll(sorts);
+ this.extensionNode = extensionNode;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+
+ TopNRel.Builder topNBuilder = TopNRel.newBuilder();
+
+ if (input != null) {
+ topNBuilder.setInput(input.toProtobuf());
+ }
+
+ topNBuilder.setN(count);
+
+ for (int i = 0; i < sorts.size(); i++) {
+ topNBuilder.addSorts(i, sorts.get(i));
+ }
+
+ if (extensionNode != null) {
+ topNBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+
+ Rel.Builder relBuilder = Rel.newBuilder();
+ relBuilder.setTopN(topNBuilder.build());
+ return relBuilder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/WindowGroupLimitRelNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/WindowGroupLimitRelNode.java
new file mode 100644
index 000000000000..a7a0ea62aa05
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/rel/WindowGroupLimitRelNode.java
@@ -0,0 +1,91 @@
+/*
+ * 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.gluten.substrait.rel;
+
+import org.apache.gluten.substrait.expression.ExpressionNode;
+import org.apache.gluten.substrait.extensions.AdvancedExtensionNode;
+
+import io.substrait.proto.Rel;
+import io.substrait.proto.RelCommon;
+import io.substrait.proto.SortField;
+import io.substrait.proto.WindowGroupLimitRel;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class WindowGroupLimitRelNode implements RelNode, Serializable {
+ private final RelNode input;
+ private final List partitionExpressions = new ArrayList<>();
+ private final List sorts = new ArrayList<>();
+ private final AdvancedExtensionNode extensionNode;
+ private final Integer limit;
+
+ public WindowGroupLimitRelNode(
+ RelNode input,
+ List partitionExpressions,
+ List sorts,
+ Integer limit) {
+ this.input = input;
+ this.partitionExpressions.addAll(partitionExpressions);
+ this.sorts.addAll(sorts);
+ this.limit = limit;
+ this.extensionNode = null;
+ }
+
+ public WindowGroupLimitRelNode(
+ RelNode input,
+ List partitionExpressions,
+ List sorts,
+ Integer limit,
+ AdvancedExtensionNode extensionNode) {
+ this.input = input;
+ this.partitionExpressions.addAll(partitionExpressions);
+ this.sorts.addAll(sorts);
+ this.limit = limit;
+ this.extensionNode = extensionNode;
+ }
+
+ @Override
+ public Rel toProtobuf() {
+ RelCommon.Builder relCommonBuilder = RelCommon.newBuilder();
+ relCommonBuilder.setDirect(RelCommon.Direct.newBuilder());
+
+ WindowGroupLimitRel.Builder windowBuilder = WindowGroupLimitRel.newBuilder();
+ windowBuilder.setCommon(relCommonBuilder.build());
+ if (input != null) {
+ windowBuilder.setInput(input.toProtobuf());
+ }
+
+ for (int i = 0; i < partitionExpressions.size(); i++) {
+ windowBuilder.addPartitionExpressions(i, partitionExpressions.get(i).toProtobuf());
+ }
+
+ for (int i = 0; i < sorts.size(); i++) {
+ windowBuilder.addSorts(i, sorts.get(i));
+ }
+
+ windowBuilder.setLimit(limit);
+
+ if (extensionNode != null) {
+ windowBuilder.setAdvancedExtension(extensionNode.toProtobuf());
+ }
+ Rel.Builder builder = Rel.newBuilder();
+ builder.setWindowGroupLimit(windowBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/BinaryTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/BinaryTypeNode.java
new file mode 100644
index 000000000000..bbd5cd154180
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/BinaryTypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class BinaryTypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public BinaryTypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.Binary.Builder binaryBuilder = Type.Binary.newBuilder();
+ if (nullable) {
+ binaryBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ binaryBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setBinary(binaryBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/BooleanTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/BooleanTypeNode.java
new file mode 100644
index 000000000000..6ae3bd50e08a
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/BooleanTypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class BooleanTypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public BooleanTypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.Boolean.Builder booleanBuilder = Type.Boolean.newBuilder();
+ if (nullable) {
+ booleanBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ booleanBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setBool(booleanBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/ColumnTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/ColumnTypeNode.java
new file mode 100644
index 000000000000..2e50c9b7ea89
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/ColumnTypeNode.java
@@ -0,0 +1,34 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.NamedStruct.ColumnType;
+
+import java.io.Serializable;
+
+public class ColumnTypeNode implements Serializable {
+
+ private final ColumnType columnType;
+
+ public ColumnTypeNode(ColumnType columnType) {
+ this.columnType = columnType;
+ }
+
+ public ColumnType toProtobuf() {
+ return columnType;
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/DateTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/DateTypeNode.java
new file mode 100644
index 000000000000..7ae6b9602cd5
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/DateTypeNode.java
@@ -0,0 +1,42 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class DateTypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public DateTypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.Date.Builder dateBuilder = Type.Date.newBuilder();
+ if (nullable) {
+ dateBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ dateBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+ Type.Builder builder = Type.newBuilder();
+ builder.setDate(dateBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/DecimalTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/DecimalTypeNode.java
new file mode 100644
index 000000000000..57f983fd598a
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/DecimalTypeNode.java
@@ -0,0 +1,49 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class DecimalTypeNode implements TypeNode, Serializable {
+ public final Boolean nullable;
+ public final int precision;
+ public final int scale;
+
+ public DecimalTypeNode(Boolean nullable, int precision, int scale) {
+ this.nullable = nullable;
+ this.precision = precision;
+ this.scale = scale;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.Decimal.Builder decimalBuilder = Type.Decimal.newBuilder();
+ decimalBuilder.setPrecision(precision);
+ decimalBuilder.setScale(scale);
+ if (nullable) {
+ decimalBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ decimalBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setDecimal(decimalBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FP32TypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FP32TypeNode.java
new file mode 100644
index 000000000000..c51b41096876
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FP32TypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class FP32TypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public FP32TypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.FP32.Builder doubleBuilder = Type.FP32.newBuilder();
+ if (nullable) {
+ doubleBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ doubleBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setFp32(doubleBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FP64TypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FP64TypeNode.java
new file mode 100644
index 000000000000..39bd7f6e8a86
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FP64TypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class FP64TypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public FP64TypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.FP64.Builder doubleBuilder = Type.FP64.newBuilder();
+ if (nullable) {
+ doubleBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ doubleBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setFp64(doubleBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FixedBinaryTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FixedBinaryTypeNode.java
new file mode 100644
index 000000000000..86889eee3d40
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FixedBinaryTypeNode.java
@@ -0,0 +1,46 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class FixedBinaryTypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+ private final int length;
+
+ public FixedBinaryTypeNode(Boolean nullable, int length) {
+ this.nullable = nullable;
+ this.length = length;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.FixedBinary.Builder fixedBinaryBuilder = Type.FixedBinary.newBuilder();
+ if (nullable) {
+ fixedBinaryBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ fixedBinaryBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+ fixedBinaryBuilder.setLength(length);
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setFixedBinary(fixedBinaryBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FixedCharTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FixedCharTypeNode.java
new file mode 100644
index 000000000000..1e671651bfda
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/FixedCharTypeNode.java
@@ -0,0 +1,46 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class FixedCharTypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+ private final int length;
+
+ public FixedCharTypeNode(Boolean nullable, int length) {
+ this.nullable = nullable;
+ this.length = length;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.FixedChar.Builder fixedCharBuilder = Type.FixedChar.newBuilder();
+ if (nullable) {
+ fixedCharBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ fixedCharBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+ fixedCharBuilder.setLength(length);
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setFixedChar(fixedCharBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I16TypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I16TypeNode.java
new file mode 100644
index 000000000000..7acaf9c6bfeb
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I16TypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class I16TypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public I16TypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.I16.Builder i16Builder = Type.I16.newBuilder();
+ if (nullable) {
+ i16Builder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ i16Builder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setI16(i16Builder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I32TypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I32TypeNode.java
new file mode 100644
index 000000000000..d8ae29d5397c
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I32TypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class I32TypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public I32TypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.I32.Builder i32Builder = Type.I32.newBuilder();
+ if (nullable) {
+ i32Builder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ i32Builder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setI32(i32Builder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I64TypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I64TypeNode.java
new file mode 100644
index 000000000000..0042772fd97d
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I64TypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class I64TypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public I64TypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.I64.Builder i64Builder = Type.I64.newBuilder();
+ if (nullable) {
+ i64Builder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ i64Builder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setI64(i64Builder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I8TypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I8TypeNode.java
new file mode 100644
index 000000000000..a3de677ee024
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/I8TypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class I8TypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public I8TypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.I8.Builder i8Builder = Type.I8.newBuilder();
+ if (nullable) {
+ i8Builder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ i8Builder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setI8(i8Builder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/IntervalYearTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/IntervalYearTypeNode.java
new file mode 100644
index 000000000000..384f9e77368a
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/IntervalYearTypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class IntervalYearTypeNode implements TypeNode, Serializable {
+
+ private final Boolean nullable;
+
+ public IntervalYearTypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.IntervalYear.Builder intervalYearBuilder = Type.IntervalYear.newBuilder();
+ if (nullable) {
+ intervalYearBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ intervalYearBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+ Type.Builder builder = Type.newBuilder();
+ builder.setIntervalYear(intervalYearBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/ListNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/ListNode.java
new file mode 100644
index 000000000000..4713e0daa83f
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/ListNode.java
@@ -0,0 +1,48 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class ListNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+ private final TypeNode nestedType;
+
+ public ListNode(Boolean nullable, TypeNode nestedType) {
+ this.nullable = nullable;
+ this.nestedType = nestedType;
+ }
+
+ // It's used in ExplodeTransformer to determine output datatype from children.
+ public TypeNode getNestedType() {
+ return nestedType;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.List.Builder listBuilder = Type.List.newBuilder();
+ listBuilder.setType(nestedType.toProtobuf());
+ listBuilder.setNullability(
+ nullable ? Type.Nullability.NULLABILITY_NULLABLE : Type.Nullability.NULLABILITY_REQUIRED);
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setList(listBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/MapNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/MapNode.java
new file mode 100644
index 000000000000..bdef11eeded5
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/MapNode.java
@@ -0,0 +1,64 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class MapNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+ private final TypeNode keyType;
+ private final TypeNode valType;
+
+ public MapNode(Boolean nullable, TypeNode keyType, TypeNode valType) {
+ this.nullable = nullable;
+ this.keyType = keyType;
+ this.valType = valType;
+ }
+
+ // It's used in ExplodeTransformer to determine output datatype from children.
+ public TypeNode getNestedType() {
+ List types = new ArrayList<>();
+ types.add(keyType);
+ types.add(valType);
+ return TypeBuilder.makeStruct(false, types);
+ }
+
+ public TypeNode getKeyType() {
+ return keyType;
+ }
+
+ public TypeNode getValueType() {
+ return valType;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.Map.Builder mapBuilder = Type.Map.newBuilder();
+ mapBuilder.setKey(keyType.toProtobuf());
+ mapBuilder.setValue(valType.toProtobuf());
+ mapBuilder.setNullability(
+ nullable ? Type.Nullability.NULLABILITY_NULLABLE : Type.Nullability.NULLABILITY_REQUIRED);
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setMap(mapBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/NothingNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/NothingNode.java
new file mode 100644
index 000000000000..7aa9953cb9eb
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/NothingNode.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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class NothingNode implements TypeNode, Serializable {
+ public NothingNode() {}
+
+ @Override
+ public Type toProtobuf() {
+ Type.Nothing.Builder nothingBuilder = Type.Nothing.newBuilder();
+ Type.Builder builder = Type.newBuilder();
+ builder.setNothing(nothingBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/StringTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/StringTypeNode.java
new file mode 100644
index 000000000000..83e029e73182
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/StringTypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class StringTypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public StringTypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.String.Builder stringBuilder = Type.String.newBuilder();
+ if (nullable) {
+ stringBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ stringBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setString(stringBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/StructNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/StructNode.java
new file mode 100644
index 000000000000..6faeb4d02b27
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/StructNode.java
@@ -0,0 +1,60 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+public class StructNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+ private final List types = new ArrayList<>();
+ private final List names = new ArrayList<>();
+
+ public StructNode(Boolean nullable, List types, List names) {
+ this.nullable = nullable;
+ this.types.addAll(types);
+ this.names.addAll(names);
+ }
+
+ public StructNode(Boolean nullable, List types) {
+ this.nullable = nullable;
+ this.types.addAll(types);
+ }
+
+ public List getFieldTypes() {
+ return types;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.Struct.Builder structBuilder = Type.Struct.newBuilder();
+ structBuilder.setNullability(
+ nullable ? Type.Nullability.NULLABILITY_NULLABLE : Type.Nullability.NULLABILITY_REQUIRED);
+ for (TypeNode typeNode : types) {
+ structBuilder.addTypes(typeNode.toProtobuf());
+ }
+ for (String name : names) {
+ structBuilder.addNames(name);
+ }
+ Type.Builder builder = Type.newBuilder();
+ builder.setStruct(structBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/TimestampTypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/TimestampTypeNode.java
new file mode 100644
index 000000000000..15919fcdee5f
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/TimestampTypeNode.java
@@ -0,0 +1,43 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+import java.io.Serializable;
+
+public class TimestampTypeNode implements TypeNode, Serializable {
+ private final Boolean nullable;
+
+ public TimestampTypeNode(Boolean nullable) {
+ this.nullable = nullable;
+ }
+
+ @Override
+ public Type toProtobuf() {
+ Type.Timestamp.Builder timestampBuilder = Type.Timestamp.newBuilder();
+ if (nullable) {
+ timestampBuilder.setNullability(Type.Nullability.NULLABILITY_NULLABLE);
+ } else {
+ timestampBuilder.setNullability(Type.Nullability.NULLABILITY_REQUIRED);
+ }
+
+ Type.Builder builder = Type.newBuilder();
+ builder.setTimestamp(timestampBuilder.build());
+ return builder.build();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/TypeBuilder.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/TypeBuilder.java
new file mode 100644
index 000000000000..28cb10be27d6
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/TypeBuilder.java
@@ -0,0 +1,103 @@
+/*
+ * 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.gluten.substrait.type;
+
+import java.util.List;
+
+public class TypeBuilder {
+ private TypeBuilder() {}
+
+ public static TypeNode makeFP32(Boolean nullable) {
+ return new FP32TypeNode(nullable);
+ }
+
+ public static TypeNode makeFP64(Boolean nullable) {
+ return new FP64TypeNode(nullable);
+ }
+
+ public static TypeNode makeBoolean(Boolean nullable) {
+ return new BooleanTypeNode(nullable);
+ }
+
+ public static TypeNode makeString(Boolean nullable) {
+ return new StringTypeNode(nullable);
+ }
+
+ public static TypeNode makeFixedChar(Boolean nullable, int length) {
+ return new FixedCharTypeNode(nullable, length);
+ }
+
+ public static TypeNode makeFixedBinary(Boolean nullable, int length) {
+ return new FixedBinaryTypeNode(nullable, length);
+ }
+
+ public static TypeNode makeBinary(Boolean nullable) {
+ return new BinaryTypeNode(nullable);
+ }
+
+ public static TypeNode makeI8(Boolean nullable) {
+ return new I8TypeNode(nullable);
+ }
+
+ public static TypeNode makeI16(Boolean nullable) {
+ return new I16TypeNode(nullable);
+ }
+
+ public static TypeNode makeI32(Boolean nullable) {
+ return new I32TypeNode(nullable);
+ }
+
+ public static TypeNode makeI64(Boolean nullable) {
+ return new I64TypeNode(nullable);
+ }
+
+ public static TypeNode makeDate(Boolean nullable) {
+ return new DateTypeNode(nullable);
+ }
+
+ public static TypeNode makeIntervalYear(Boolean nullable) {
+ return new IntervalYearTypeNode(nullable);
+ }
+
+ public static TypeNode makeDecimal(Boolean nullable, Integer precision, Integer scale) {
+ return new DecimalTypeNode(nullable, precision, scale);
+ }
+
+ public static TypeNode makeTimestamp(Boolean nullable) {
+ return new TimestampTypeNode(nullable);
+ }
+
+ public static TypeNode makeStruct(Boolean nullable, List types, List names) {
+ return new StructNode(nullable, types, names);
+ }
+
+ public static TypeNode makeStruct(Boolean nullable, List types) {
+ return new StructNode(nullable, types);
+ }
+
+ public static TypeNode makeMap(Boolean nullable, TypeNode keyType, TypeNode valType) {
+ return new MapNode(nullable, keyType, valType);
+ }
+
+ public static TypeNode makeList(Boolean nullable, TypeNode nestedType) {
+ return new ListNode(nullable, nestedType);
+ }
+
+ public static TypeNode makeNothing() {
+ return new NothingNode();
+ }
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/TypeNode.java b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/TypeNode.java
new file mode 100644
index 000000000000..a631122e0a56
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/substrait/type/TypeNode.java
@@ -0,0 +1,23 @@
+/*
+ * 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.gluten.substrait.type;
+
+import io.substrait.proto.Type;
+
+public interface TypeNode {
+ Type toProtobuf();
+}
diff --git a/gluten-flink/core/src/main/java/org/apache/gluten/vectorized/VLNativeRowVector.java b/gluten-flink/core/src/main/java/org/apache/gluten/vectorized/VLNativeRowVector.java
new file mode 100644
index 000000000000..0c47de358deb
--- /dev/null
+++ b/gluten-flink/core/src/main/java/org/apache/gluten/vectorized/VLNativeRowVector.java
@@ -0,0 +1,40 @@
+/*
+ * 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.gluten.vectorized;
+
+import org.apache.flink.table.data.RowData;
+
+/** Converter between velox RowVector and Flink RowData. */
+public class VLNativeRowVector {
+ private long rowAddress;
+
+ public VLNativeRowVector(long rowAddress) {
+ this.rowAddress = rowAddress;
+ }
+
+ public static VLNativeRowVector fromRowData(RowData row) {
+ return new VLNativeRowVector(0);
+ }
+
+ public long rowAddress() {
+ return rowAddress;
+ }
+
+ public RowData toRowData() {
+ return null;
+ }
+}
diff --git a/gluten-flink/core/src/main/resources/substrait/proto/substrait/algebra.proto b/gluten-flink/core/src/main/resources/substrait/proto/substrait/algebra.proto
new file mode 100644
index 000000000000..0abb50b323ac
--- /dev/null
+++ b/gluten-flink/core/src/main/resources/substrait/proto/substrait/algebra.proto
@@ -0,0 +1,1473 @@
+// SPDX-License-Identifier: Apache-2.0
+syntax = "proto3";
+
+package substrait;
+
+import "google/protobuf/any.proto";
+import "substrait/extensions/extensions.proto";
+import "substrait/type.proto";
+
+option csharp_namespace = "Substrait.Protobuf";
+option go_package = "github.com/substrait-io/substrait-go/proto";
+option java_multiple_files = true;
+option java_package = "io.substrait.proto";
+
+// Common fields for all relational operators
+message RelCommon {
+ oneof emit_kind {
+ // The underlying relation is output as is (no reordering or projection of columns)
+ Direct direct = 1;
+ // Allows to control for order and inclusion of fields
+ Emit emit = 2;
+ }
+
+ Hint hint = 3;
+ substrait.extensions.AdvancedExtension advanced_extension = 4;
+
+ // Direct indicates no change on presence and ordering of fields in the output
+ message Direct {}
+
+ // Remap which fields are output and in which order
+ message Emit {
+ repeated int32 output_mapping = 1;
+ }
+
+ // Changes to the operation that can influence efficiency/performance but
+ // should not impact correctness.
+ message Hint {
+ Stats stats = 1;
+ RuntimeConstraint constraint = 2;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+
+ // The statistics related to a hint (physical properties of records)
+ message Stats {
+ double row_count = 1;
+ double record_size = 2;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+ }
+
+ message RuntimeConstraint {
+ // TODO: nodes, cpu threads/%, memory, iops, etc.
+
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+ }
+ }
+}
+
+// The scan operator of base data (physical or virtual), including filtering and projection.
+message ReadRel {
+ RelCommon common = 1;
+ NamedStruct base_schema = 2;
+ Expression filter = 3;
+ Expression best_effort_filter = 11;
+ Expression.MaskExpression projection = 4;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+
+ // Definition of which type of scan operation is to be performed
+ oneof read_type {
+ VirtualTable virtual_table = 5;
+ LocalFiles local_files = 6;
+ NamedTable named_table = 7;
+ ExtensionTable extension_table = 8;
+ }
+
+ // A base table. The list of string is used to represent namespacing (e.g., mydb.mytable).
+ // This assumes shared catalog between systems exchanging a message.
+ message NamedTable {
+ repeated string names = 1;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+ }
+
+ // A table composed of literals.
+ message VirtualTable {
+ repeated Expression.Literal.Struct values = 1;
+ }
+
+ // A stub type that can be used to extend/introduce new table types outside
+ // the specification.
+ message ExtensionTable {
+ google.protobuf.Any detail = 1;
+ }
+
+ // Represents a list of files in input of a scan operation
+ message LocalFiles {
+ repeated FileOrFiles items = 1;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+
+ // Many files consist of indivisible chunks (e.g. parquet row groups
+ // or CSV rows). If a slice partially selects an indivisible chunk
+ // then the consumer should employ some rule to decide which slice to
+ // include the chunk in (e.g. include it in the slice that contains
+ // the midpoint of the chunk)
+ message FileOrFiles {
+ oneof path_type {
+ // A URI that can refer to either a single folder or a single file
+ string uri_path = 1;
+ // A URI where the path portion is a glob expression that can
+ // identify zero or more paths.
+ // Consumers should support the POSIX syntax. The recursive
+ // globstar (**) may not be supported.
+ string uri_path_glob = 2;
+ // A URI that refers to a single file
+ string uri_file = 3;
+ // A URI that refers to a single folder
+ string uri_folder = 4;
+ }
+
+ // Original file format enum, superseded by the file_format oneof.
+ reserved 5;
+ reserved "format";
+
+ // The index of the partition this item belongs to
+ uint64 partition_index = 6;
+
+
+
+ // The start position in byte to read from this item
+ uint64 start = 7;
+
+ // The length in byte to read from this item
+ uint64 length = 8;
+
+ message ParquetReadOptions {
+ bool enable_row_group_maxmin_index = 1;
+ }
+ message ArrowReadOptions {}
+ message OrcReadOptions {}
+ message DwrfReadOptions {}
+ message TextReadOptions {
+ string field_delimiter = 1;
+ uint64 max_block_size = 2;
+ NamedStruct schema = 3 [deprecated=true];
+ string quote = 4;
+ uint64 header = 5;
+ string escape = 6;
+ string null_value = 7;
+ bool empty_as_default = 8;
+ }
+ message JsonReadOptions {
+ uint64 max_block_size = 1;
+ NamedStruct schema = 2 [deprecated=true];
+ }
+ message IcebergReadOptions {
+ enum FileContent {
+ DATA = 0;
+ POSITION_DELETES = 1;
+ EQUALITY_DELETES = 2;
+ }
+ message DeleteFile {
+ FileContent fileContent = 1;
+ string filePath = 2;
+ uint64 fileSize = 3;
+ uint64 recordCount = 4;
+ oneof file_format {
+ ParquetReadOptions parquet = 5;
+ OrcReadOptions orc = 6;
+ }
+ }
+ oneof file_format {
+ ParquetReadOptions parquet = 1;
+ OrcReadOptions orc = 2;
+ }
+ repeated DeleteFile delete_files = 3;
+ }
+
+ // File reading options
+ oneof file_format {
+ ParquetReadOptions parquet = 9;
+ ArrowReadOptions arrow = 10;
+ OrcReadOptions orc = 11;
+ google.protobuf.Any extension = 12;
+ DwrfReadOptions dwrf = 13;
+ TextReadOptions text = 14;
+ JsonReadOptions json = 15;
+ IcebergReadOptions iceberg = 16;
+ }
+
+ message partitionColumn {
+ string key = 1;
+ string value = 2;
+ }
+ repeated partitionColumn partition_columns = 17;
+
+ /// File schema
+ NamedStruct schema = 18;
+
+ message metadataColumn {
+ string key = 1;
+ string value = 2;
+ }
+ repeated metadataColumn metadata_columns = 19;
+
+ // File properties contained in split
+ message fileProperties {
+ int64 fileSize = 1;
+ int64 modificationTime = 2;
+ }
+ fileProperties properties = 20;
+ }
+ }
+}
+
+// This operator allows to represent calculated expressions of fields (e.g., a+b). Direct/Emit are used to represent classical relational projections
+message ProjectRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ repeated Expression expressions = 3;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// The binary JOIN relational operator left-join-right, including various join types, a join condition and post_join_filter expression
+message JoinRel {
+ RelCommon common = 1;
+ Rel left = 2;
+ Rel right = 3;
+ Expression expression = 4;
+ Expression post_join_filter = 5;
+
+ JoinType type = 6;
+
+ enum JoinType {
+ JOIN_TYPE_UNSPECIFIED = 0;
+ JOIN_TYPE_INNER = 1;
+ JOIN_TYPE_OUTER = 2;
+ JOIN_TYPE_LEFT = 3;
+ JOIN_TYPE_RIGHT = 4;
+ JOIN_TYPE_LEFT_SEMI = 5;
+ JOIN_TYPE_LEFT_ANTI = 6;
+ JOIN_TYPE_LEFT_SINGLE = 7;
+ JOIN_TYPE_RIGHT_SEMI = 8;
+ JOIN_TYPE_RIGHT_ANTI = 9;
+ JOIN_TYPE_RIGHT_SINGLE = 10;
+ }
+
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// Cartesian product relational operator of two tables (left and right)
+message CrossRel {
+ RelCommon common = 1;
+ Rel left = 2;
+ Rel right = 3;
+ Expression expression = 4;
+
+ JoinType type = 5;
+
+ // TODO -- Remove this unnecessary type.
+ enum JoinType {
+ JOIN_TYPE_UNSPECIFIED = 0;
+ JOIN_TYPE_INNER = 1;
+ JOIN_TYPE_OUTER = 2;
+ JOIN_TYPE_LEFT = 3;
+ JOIN_TYPE_RIGHT = 4;
+ JOIN_TYPE_LEFT_SEMI = 5;
+ JOIN_TYPE_LEFT_ANTI = 6;
+ JOIN_TYPE_LEFT_SINGLE = 7;
+ JOIN_TYPE_RIGHT_SEMI = 8;
+ JOIN_TYPE_RIGHT_ANTI = 9;
+ JOIN_TYPE_RIGHT_SINGLE = 10;
+ }
+
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// The relational operator representing LIMIT/OFFSET or TOP type semantics.
+message FetchRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ // the offset expressed in number of records
+ int64 offset = 3;
+ // the amount of records to return
+ int64 count = 4;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// The relational operator representing TOP N calculation
+message TopNRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ int64 n = 3;
+ repeated SortField sorts = 4;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// The relational operator representing a GROUP BY Aggregate
+message AggregateRel {
+ RelCommon common = 1;
+
+ // Input of the aggregation
+ Rel input = 2;
+
+ // A list of expression grouping that the aggregation measured should be calculated for.
+ repeated Grouping groupings = 3;
+
+ // A list of one or more aggregate expressions along with an optional filter.
+ repeated Measure measures = 4;
+
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+
+ message Grouping {
+ repeated Expression grouping_expressions = 1;
+ }
+
+ message Measure {
+ AggregateFunction measure = 1;
+
+ // An optional boolean expression that acts to filter which records are
+ // included in the measure. True means include this record for calculation
+ // within the measure.
+ // Helps to support SUM() FILTER(WHERE...) syntax without masking opportunities for optimization
+ Expression filter = 2;
+ }
+}
+
+// The ORDERY BY (or sorting) relational operator. Beside describing a base relation, it includes a list of fields to sort on
+message SortRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ repeated SortField sorts = 3;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+message WindowRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ repeated Measure measures = 3;
+ repeated Expression partition_expressions = 4;
+ repeated SortField sorts = 5;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+
+ message Measure {
+ Expression.WindowFunction measure = 1;
+ }
+}
+
+message WindowGroupLimitRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ repeated Expression partition_expressions = 3;
+ repeated SortField sorts = 4;
+ int32 limit = 5;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// The relational operator capturing simple FILTERs (as in the WHERE clause of SQL)
+message FilterRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ Expression condition = 3;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// The relational set operators (intersection/union/etc..)
+message SetRel {
+ RelCommon common = 1;
+ // The first input is the primary input, the remaining are secondary
+ // inputs. There must be at least two inputs.
+ repeated Rel inputs = 2;
+ SetOp op = 3;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+
+ enum SetOp {
+ SET_OP_UNSPECIFIED = 0;
+ SET_OP_MINUS_PRIMARY = 1;
+ SET_OP_MINUS_MULTISET = 2;
+ SET_OP_INTERSECTION_PRIMARY = 3;
+ SET_OP_INTERSECTION_MULTISET = 4;
+ SET_OP_UNION_DISTINCT = 5;
+ SET_OP_UNION_ALL = 6;
+ }
+}
+
+// Stub to support extension with a single input
+message ExtensionSingleRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ google.protobuf.Any detail = 3;
+}
+
+// Stub to support extension with a zero inputs
+message ExtensionLeafRel {
+ RelCommon common = 1;
+ google.protobuf.Any detail = 2;
+}
+
+// Stub to support extension with multiple inputs
+message ExtensionMultiRel {
+ RelCommon common = 1;
+ repeated Rel inputs = 2;
+ google.protobuf.Any detail = 3;
+}
+
+// A redistribution operation
+message ExchangeRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ int32 partition_count = 3;
+ repeated ExchangeTarget targets = 4;
+
+ // the type of exchange used
+ oneof exchange_kind {
+ ScatterFields scatter_by_fields = 5;
+ SingleBucketExpression single_target = 6;
+ MultiBucketExpression multi_target = 7;
+ RoundRobin round_robin = 8;
+ Broadcast broadcast = 9;
+ }
+
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+
+ message ScatterFields {
+ repeated Expression.FieldReference fields = 1;
+ }
+
+ // Returns a single bucket number per record.
+ message SingleBucketExpression {
+ Expression expression = 1;
+ }
+
+ // Returns zero or more bucket numbers per record
+ message MultiBucketExpression {
+ Expression expression = 1;
+ bool constrained_to_count = 2;
+ }
+
+ // Send all data to every target.
+ message Broadcast {}
+
+ // Route approximately
+ message RoundRobin {
+ // whether the round robin behavior is required to exact (per record) or
+ // approximate. Defaults to approximate.
+ bool exact = 1;
+ }
+
+ // The message to describe partition targets of an exchange
+ message ExchangeTarget {
+ // Describes the partition id(s) to send. If this is empty, all data is sent
+ // to this target.
+ repeated int32 partition_id = 1;
+
+ oneof target_type {
+ string uri = 2;
+ google.protobuf.Any extended = 3;
+ }
+ }
+}
+
+// Duplicates records, possibly switching output expressions between each duplicate.
+// Default output is all of the fields declared followed by one int64 field that contains the
+// duplicate_id which is a zero-index ordinal of which duplicate of the original record this
+// corresponds to.
+message ExpandRel {
+ RelCommon common = 1;
+ Rel input = 2;
+ repeated ExpandField fields = 4;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+
+ message ExpandField {
+ oneof field_type {
+ // Field that switches output based on which duplicate_id we're outputting
+ SwitchingField switching_field = 2;
+
+ // Field that outputs the same value no matter which duplicate_id we're on.
+ Expression consistent_field = 3;
+ }
+ }
+
+ message SwitchingField {
+ // Array that contains an expression to output per duplicate_id
+ // each `switching_field` must have the same number of expressions
+ // all expressions within a switching field be the same type class but can differ in nullability.
+ // this column will be nullable if any of the expressions are nullable.
+ repeated Expression duplicates = 1;
+ }
+}
+
+// A relation with output field names.
+//
+// This is for use at the root of a `Rel` tree.
+message RelRoot {
+ // A relation
+ Rel input = 1;
+ // Field names in depth-first order
+ repeated string names = 2;
+ Type.Struct output_schema = 3;
+}
+
+// A relation (used internally in a plan)
+message Rel {
+ oneof rel_type {
+ ReadRel read = 1;
+ FilterRel filter = 2;
+ FetchRel fetch = 3;
+ AggregateRel aggregate = 4;
+ SortRel sort = 5;
+ JoinRel join = 6;
+ ProjectRel project = 7;
+ SetRel set = 8;
+ ExtensionSingleRel extension_single = 9;
+ ExtensionMultiRel extension_multi = 10;
+ ExtensionLeafRel extension_leaf = 11;
+ CrossRel cross = 12;
+ //Physical relations
+ HashJoinRel hash_join = 13;
+ MergeJoinRel merge_join = 14;
+ ExpandRel expand = 15;
+ WindowRel window = 16;
+ GenerateRel generate = 17;
+ WriteRel write = 18;
+ TopNRel top_n = 19;
+ WindowGroupLimitRel windowGroupLimit = 20;
+ }
+}
+
+// A base object for writing (e.g., a table or a view).
+message NamedObjectWrite {
+ // The list of string is used to represent namespacing (e.g., mydb.mytable).
+ // This assumes shared catalog between systems exchanging a message.
+ repeated string names = 1;
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// A stub type that can be used to extend/introduce new table types outside
+// the specification.
+message ExtensionObject {
+ google.protobuf.Any detail = 1;
+}
+
+message DdlRel {
+ // Definition of which type of object we are operating on
+ oneof write_type {
+ NamedObjectWrite named_object = 1;
+ ExtensionObject extension_object = 2;
+ }
+
+ // The columns that will be modified (representing after-image of a schema change)
+ NamedStruct table_schema = 3;
+ // The default values for the columns (representing after-image of a schema change)
+ // E.g., in case of an ALTER TABLE that changes some of the column default values, we expect
+ // the table_defaults Struct to report a full list of default values reflecting the result of applying
+ // the ALTER TABLE operator successfully
+ Expression.Literal.Struct table_defaults = 4;
+
+ // Which type of object we operate on
+ DdlObject object = 5;
+
+ // The type of operation to perform
+ DdlOp op = 6;
+
+ // The body of the CREATE VIEW
+ Rel view_definition = 7;
+
+ enum DdlObject {
+ DDL_OBJECT_UNSPECIFIED = 0;
+ // A Table object in the system
+ DDL_OBJECT_TABLE = 1;
+ // A View object in the system
+ DDL_OBJECT_VIEW = 2;
+ }
+
+ enum DdlOp {
+ DDL_OP_UNSPECIFIED = 0;
+ // A create operation (for any object)
+ DDL_OP_CREATE = 1;
+ // A create operation if the object does not exist, or replaces it (equivalent to a DROP + CREATE) if the object already exists
+ DDL_OP_CREATE_OR_REPLACE = 2;
+ // An operation that modifies the schema (e.g., column names, types, default values) for the target object
+ DDL_OP_ALTER = 3;
+ // An operation that removes an object from the system
+ DDL_OP_DROP = 4;
+ // An operation that removes an object from the system (without throwing an exception if the object did not exist)
+ DDL_OP_DROP_IF_EXIST = 5;
+ }
+ //TODO add PK/constraints/indexes/etc..?
+}
+
+// The operator that modifies the content of a database (operates on 1 table at a time, but tuple-selection/source can be
+// based on joining of multiple tables).
+message WriteRel {
+ // Definition of which TABLE we are operating on
+ oneof write_type {
+ NamedObjectWrite named_table = 1;
+ ExtensionObject extension_table = 2;
+ }
+
+ // The schema of the table (must align with Rel input (e.g., number of leaf fields must match))
+ NamedStruct table_schema = 3;
+
+ // The type of operation to perform
+ WriteOp op = 4;
+
+ // The relation that determines the tuples to add/remove/modify
+ // the schema must match with table_schema. Default values must be explicitly stated
+ // in a ProjectRel at the top of the input. The match must also
+ // occur in case of DELETE to ensure multi-engine plans are unequivocal.
+ Rel input = 5;
+
+ // Output mode determines what is the output of executing this rel
+ OutputMode output = 6;
+
+ enum WriteOp {
+ WRITE_OP_UNSPECIFIED = 0;
+ // The insert of new tuples in a table
+ WRITE_OP_INSERT = 1;
+ // The removal of tuples from a table
+ WRITE_OP_DELETE = 2;
+ // The modification of existing tuples within a table
+ WRITE_OP_UPDATE = 3;
+ // The Creation of a new table, and the insert of new tuples in the table
+ WRITE_OP_CTAS = 4;
+ }
+
+ enum OutputMode {
+ OUTPUT_MODE_UNSPECIFIED = 0;
+ // return no tuples at all
+ OUTPUT_MODE_NO_OUTPUT = 1;
+ // this mode makes the operator return all the tuple INSERTED/DELETED/UPDATED by the operator.
+ // The operator returns the AFTER-image of any change. This can be further manipulated by operators upstreams
+ // (e.g., retunring the typical "count of modified tuples").
+ // For scenarios in which the BEFORE image is required, the user must implement a spool (via references to
+ // subplans in the body of the Rel input) and return those with anounter PlanRel.relations.
+ OUTPUT_MODE_MODIFIED_TUPLES = 2;
+ }
+}
+
+// The hash equijoin join operator will build a hash table out of the right input based on a set of join keys.
+// It will then probe that hash table for incoming inputs, finding matches.
+message HashJoinRel {
+ RelCommon common = 1;
+ Rel left = 2;
+ Rel right = 3;
+ repeated Expression.FieldReference left_keys = 4;
+ repeated Expression.FieldReference right_keys = 5;
+ Expression post_join_filter = 6;
+
+ JoinType type = 7;
+
+ enum JoinType {
+ JOIN_TYPE_UNSPECIFIED = 0;
+ JOIN_TYPE_INNER = 1;
+ JOIN_TYPE_OUTER = 2;
+ JOIN_TYPE_LEFT = 3;
+ JOIN_TYPE_RIGHT = 4;
+ JOIN_TYPE_LEFT_SEMI = 5;
+ JOIN_TYPE_RIGHT_SEMI = 6;
+ JOIN_TYPE_LEFT_ANTI = 7;
+ JOIN_TYPE_RIGHT_ANTI = 8;
+ JOIN_TYPE_LEFT_SINGLE = 9;
+ JOIN_TYPE_RIGHT_SINGLE = 10;
+ }
+
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// The merge equijoin does a join by taking advantage of two sets that are sorted on the join keys.
+// This allows the join operation to be done in a streaming fashion.
+message MergeJoinRel {
+ RelCommon common = 1;
+ Rel left = 2;
+ Rel right = 3;
+ repeated Expression.FieldReference left_keys = 4;
+ repeated Expression.FieldReference right_keys = 5;
+ Expression post_join_filter = 6;
+
+ JoinType type = 7;
+
+ enum JoinType {
+ JOIN_TYPE_UNSPECIFIED = 0;
+ JOIN_TYPE_INNER = 1;
+ JOIN_TYPE_OUTER = 2;
+ JOIN_TYPE_LEFT = 3;
+ JOIN_TYPE_RIGHT = 4;
+ JOIN_TYPE_LEFT_SEMI = 5;
+ JOIN_TYPE_RIGHT_SEMI = 6;
+ JOIN_TYPE_LEFT_ANTI = 7;
+ JOIN_TYPE_RIGHT_ANTI = 8;
+ }
+
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// The argument of a function
+message FunctionArgument {
+ oneof arg_type {
+ string enum = 1;
+ Type type = 2;
+ Expression value = 3;
+ }
+}
+
+// An optional function argument. Typically used for specifying behavior in
+// invalid or corner cases.
+message FunctionOption {
+ // Name of the option to set. If the consumer does not recognize the
+ // option, it must reject the plan. The name is matched case-insensitively
+ // with option names defined for the function.
+ string name = 1;
+
+ // List of behavior options allowed by the producer. At least one must be
+ // specified; to leave an option unspecified, simply don't add an entry to
+ // `options`. The consumer must use the first option from the list that it
+ // supports. If the consumer supports none of the specified options, it
+ // must reject the plan. The name is matched case-insensitively and must
+ // match one of the option values defined for the option.
+ repeated string preference = 2;
+}
+
+message Expression {
+ oneof rex_type {
+ Literal literal = 1;
+ FieldReference selection = 2;
+ ScalarFunction scalar_function = 3;
+ WindowFunction window_function = 5;
+ IfThen if_then = 6;
+ SwitchExpression switch_expression = 7;
+ SingularOrList singular_or_list = 8;
+ MultiOrList multi_or_list = 9;
+ Cast cast = 11;
+ Subquery subquery = 12;
+ Nested nested = 13;
+
+ // deprecated: enum literals are only sensible in the context of
+ // function arguments, for which FunctionArgument should now be
+ // used
+ Enum enum = 10 [deprecated = true];
+ }
+
+ message Enum {
+ option deprecated = true;
+
+ oneof enum_kind {
+ string specified = 1;
+ Empty unspecified = 2;
+ }
+
+ message Empty {
+ option deprecated = true;
+ }
+ }
+
+ message Literal {
+ oneof literal_type {
+ bool boolean = 1;
+ int32 i8 = 2;
+ int32 i16 = 3;
+ int32 i32 = 5;
+ int64 i64 = 7;
+ float fp32 = 10;
+ double fp64 = 11;
+ string string = 12;
+ bytes binary = 13;
+ // Timestamp in units of microseconds since the UNIX epoch.
+ int64 timestamp = 14;
+ // Date in units of days since the UNIX epoch.
+ int32 date = 16;
+ // Time in units of microseconds past midnight
+ int64 time = 17;
+ IntervalYearToMonth interval_year_to_month = 19;
+ IntervalDayToSecond interval_day_to_second = 20;
+ string fixed_char = 21;
+ VarChar var_char = 22;
+ bytes fixed_binary = 23;
+ Decimal decimal = 24;
+ Struct struct = 25;
+ Map map = 26;
+ // Timestamp in units of microseconds since the UNIX epoch.
+ int64 timestamp_tz = 27;
+ bytes uuid = 28;
+ Type null = 29; // a typed null literal
+ List list = 30;
+ Type.List empty_list = 31;
+ Type.Map empty_map = 32;
+ UserDefined user_defined = 33;
+ }
+
+ // whether the literal type should be treated as a nullable type. Applies to
+ // all members of union other than the Typed null (which should directly
+ // declare nullability).
+ bool nullable = 50;
+
+ // optionally points to a type_variation_anchor defined in this plan.
+ // Applies to all members of union other than the Typed null (which should
+ // directly declare the type variation).
+ uint32 type_variation_reference = 51;
+
+ message VarChar {
+ string value = 1;
+ uint32 length = 2;
+ }
+
+ message Decimal {
+ // little-endian twos-complement integer representation of complete value
+ // (ignoring precision) Always 16 bytes in length
+ bytes value = 1;
+ // The maximum number of digits allowed in the value.
+ // the maximum precision is 38.
+ int32 precision = 2;
+ // declared scale of decimal literal
+ int32 scale = 3;
+ }
+
+ message Map {
+ message KeyValue {
+ Literal key = 1;
+ Literal value = 2;
+ }
+
+ repeated KeyValue key_values = 1;
+ }
+
+ message IntervalYearToMonth {
+ int32 years = 1;
+ int32 months = 2;
+ }
+
+ message IntervalDayToSecond {
+ int32 days = 1;
+ int32 seconds = 2;
+ int32 microseconds = 3;
+ }
+
+ message Struct {
+ // A possibly heterogeneously typed list of literals
+ repeated Literal fields = 1;
+ }
+
+ message List {
+ // A homogeneously typed list of literals
+ repeated Literal values = 1;
+ }
+
+ message UserDefined {
+ // points to a type_anchor defined in this plan
+ uint32 type_reference = 1;
+
+ // The parameters to be bound to the type class, if the type class is
+ // parameterizable.
+ repeated Type.Parameter type_parameters = 3;
+
+ // the value of the literal, serialized using some type-specific
+ // protobuf message
+ google.protobuf.Any value = 2;
+ }
+ }
+
+ // Expression to dynamically construct nested types.
+ message Nested {
+ // Whether the returned nested type is nullable.
+ bool nullable = 1;
+
+ // Optionally points to a type_variation_anchor defined in this plan for
+ // the returned nested type.
+ uint32 type_variation_reference = 2;
+
+ oneof nested_type {
+ Struct struct = 3;
+ List list = 4;
+ Map map = 5;
+ }
+
+ message Map {
+ message KeyValue {
+ // Mandatory key/value expressions.
+ Expression key = 1;
+ Expression value = 2;
+ }
+
+ // One or more key-value pairs. To specify an empty map, use
+ // Literal.empty_map (otherwise type information would be missing).
+ repeated KeyValue key_values = 1;
+ }
+
+ message Struct {
+ // Zero or more possibly heterogeneously-typed list of expressions that
+ // form the struct fields.
+ repeated Expression fields = 1;
+ }
+
+ message List {
+ // A homogeneously-typed list of one or more expressions that form the
+ // list entries. To specify an empty list, use Literal.empty_list
+ // (otherwise type information would be missing).
+ repeated Expression values = 1;
+ }
+ }
+
+ // A scalar function call.
+ message ScalarFunction {
+ // Points to a function_anchor defined in this plan, which must refer
+ // to a scalar function in the associated YAML file. Required; avoid
+ // using anchor/reference zero.
+ uint32 function_reference = 1;
+
+ // The arguments to be bound to the function. This must have exactly the
+ // number of arguments specified in the function definition, and the
+ // argument types must also match exactly:
+ //
+ // - Value arguments must be bound using FunctionArgument.value, and
+ // the expression in that must yield a value of a type that a function
+ // overload is defined for.
+ // - Type arguments must be bound using FunctionArgument.type.
+ // - Enum arguments must be bound using FunctionArgument.enum
+ // followed by Enum.specified, with a string that case-insensitively
+ // matches one of the allowed options.
+ repeated FunctionArgument arguments = 4;
+
+ // Options to specify behavior for corner cases, or leave behavior
+ // unspecified if the consumer does not need specific behavior in these
+ // cases.
+ repeated FunctionOption options = 5;
+
+ // Must be set to the return type of the function, exactly as derived
+ // using the declaration in the extension.
+ Type output_type = 3;
+
+ // Deprecated; use arguments instead.
+ repeated Expression args = 2 [deprecated = true];
+ }
+
+ // A window function call.
+ message WindowFunction {
+ // Points to a function_anchor defined in this plan, which must refer
+ // to a window function in the associated YAML file. Required; 0 is
+ // considered to be a valid anchor/reference.
+ uint32 function_reference = 1;
+
+ // The arguments to be bound to the function. This must have exactly the
+ // number of arguments specified in the function definition, and the
+ // argument types must also match exactly:
+ //
+ // - Value arguments must be bound using FunctionArgument.value, and
+ // the expression in that must yield a value of a type that a function
+ // overload is defined for.
+ // - Type arguments must be bound using FunctionArgument.type, and a
+ // function overload must be defined for that type.
+ // - Enum arguments must be bound using FunctionArgument.enum
+ // followed by Enum.specified, with a string that case-insensitively
+ // matches one of the allowed options.
+ repeated FunctionArgument arguments = 9;
+
+ // Options to specify behavior for corner cases, or leave behavior
+ // unspecified if the consumer does not need specific behavior in these
+ // cases.
+ repeated FunctionOption options = 11;
+
+ // Must be set to the return type of the function, exactly as derived
+ // using the declaration in the extension.
+ Type output_type = 7;
+
+ // Describes which part of the window function to perform within the
+ // context of distributed algorithms. Required. Must be set to
+ // INITIAL_TO_RESULT for window functions that are not decomposable.
+ AggregationPhase phase = 6;
+
+ // If specified, the records that are part of the window defined by
+ // upper_bound and lower_bound are ordered according to this list
+ // before they are aggregated. The first sort field has the highest
+ // priority; only if a sort field determines two records to be equivalent
+ // is the next field queried. This field is optional, and is only allowed
+ // if the window function is defined to support sorting.
+ repeated SortField sorts = 3;
+
+ // Specifies whether equivalent records are merged before being aggregated.
+ // Optional, defaults to AGGREGATION_INVOCATION_ALL.
+ AggregateFunction.AggregationInvocation invocation = 10;
+
+ // When one or more partition expressions are specified, two records are
+ // considered to be in the same partition if and only if these expressions
+ // yield an equal tuple of values for both. When computing the window
+ // function, only the subset of records within the bounds that are also in
+ // the same partition as the current record are aggregated.
+ repeated Expression partitions = 2;
+
+ // Defines the record relative to the current record from which the window
+ // extends. The bound is inclusive. If the lower bound indexes a record
+ // greater than the upper bound, TODO (null range/no records passed?
+ // wrapping around as if lower/upper were swapped? error? null?).
+ // Optional; defaults to the start of the partition.
+ Bound lower_bound = 5;
+
+ string column_name = 12;
+ WindowType window_type = 13;
+
+ // Defines the record relative to the current record up to which the window
+ // extends. The bound is inclusive. If the upper bound indexes a record
+ // less than the lower bound, TODO (null range/no records passed?
+ // wrapping around as if lower/upper were swapped? error? null?).
+ // Optional; defaults to the end of the partition.
+ Bound upper_bound = 4;
+
+ // Deprecated; use arguments instead.
+ repeated Expression args = 8 [deprecated = true];
+
+ // Defines one of the two boundaries for the window of a window function.
+ message Bound {
+ // Defines that the bound extends this far back from the current record.
+ message Preceding {
+ oneof kind {
+ // A strictly positive integer specifying the number of records that
+ // the window extends back from the current record. Use
+ // CurrentRow for offset zero and Following for negative offsets.
+ int64 offset = 1;
+
+ // the reference to pre-project range frame boundary.
+ Expression ref = 2;
+ }
+ }
+
+ // Defines that the bound extends this far ahead of the current record.
+ message Following {
+ oneof kind {
+ // A strictly positive integer specifying the number of records that
+ // the window extends ahead of the current record. Use
+ // CurrentRow for offset zero and Preceding for negative offsets.
+ int64 offset = 1;
+
+ // the reference to pre-project range frame boundary.
+ Expression ref = 2;
+ }
+ }
+
+ // Defines that the bound extends to or from the current record.
+ message CurrentRow {}
+
+ message Unbounded_Preceding {}
+
+ message Unbounded_Following {}
+
+ oneof kind {
+ // The bound extends some number of records behind the current record.
+ Preceding preceding = 1;
+
+ // The bound extends some number of records ahead of the current
+ // record.
+ Following following = 2;
+
+ // The bound extends to the current record.
+ CurrentRow current_row = 3;
+
+ Unbounded_Preceding unbounded_preceding = 4;
+ Unbounded_Following unbounded_following = 5;
+ }
+ }
+ }
+
+ message IfThen {
+ repeated IfClause ifs = 1;
+ Expression else = 2;
+
+ message IfClause {
+ Expression if = 1;
+ Expression then = 2;
+ }
+ }
+
+ message Cast {
+ Type type = 1;
+ Expression input = 2;
+ FailureBehavior failure_behavior = 3;
+
+ enum FailureBehavior {
+ FAILURE_BEHAVIOR_UNSPECIFIED = 0;
+ FAILURE_BEHAVIOR_RETURN_NULL = 1;
+ FAILURE_BEHAVIOR_THROW_EXCEPTION = 2;
+ }
+ }
+
+ message SwitchExpression {
+ Expression match = 3;
+ repeated IfValue ifs = 1;
+ Expression else = 2;
+
+ message IfValue {
+ Literal if = 1;
+ Expression then = 2;
+ }
+ }
+
+ message SingularOrList {
+ Expression value = 1;
+ repeated Expression options = 2;
+ }
+
+ message MultiOrList {
+ repeated Expression value = 1;
+ repeated Record options = 2;
+
+ message Record {
+ repeated Expression fields = 1;
+ }
+ }
+
+ message EmbeddedFunction {
+ repeated Expression arguments = 1;
+ Type output_type = 2;
+ oneof kind {
+ PythonPickleFunction python_pickle_function = 3;
+ WebAssemblyFunction web_assembly_function = 4;
+ }
+
+ message PythonPickleFunction {
+ bytes function = 1;
+ repeated string prerequisite = 2;
+ }
+
+ message WebAssemblyFunction {
+ bytes script = 1;
+ repeated string prerequisite = 2;
+ }
+ }
+
+ // A way to reference the inner property of a complex record. Can reference
+ // either a map key by literal, a struct field by the ordinal position of
+ // the desired field or a particular element in an array. Supports
+ // expressions that would roughly translate to something similar to:
+ // a.b[2].c['my_map_key'].x where a,b,c and x are struct field references
+ // (ordinalized in the internal representation here), [2] is a list offset
+ // and ['my_map_key'] is a reference into a map field.
+ message ReferenceSegment {
+ oneof reference_type {
+ MapKey map_key = 1;
+ StructField struct_field = 2;
+ ListElement list_element = 3;
+ }
+
+ message MapKey {
+ // literal based reference to specific possible value in map.
+ Literal map_key = 1;
+
+ // Optional child segment
+ ReferenceSegment child = 2;
+ }
+
+ message StructField {
+ // zero-indexed ordinal position of field in struct
+ int32 field = 1;
+
+ // Optional child segment
+ ReferenceSegment child = 2;
+ }
+
+ message ListElement {
+ // zero-indexed ordinal position of element in list
+ int32 offset = 1;
+
+ // Optional child segment
+ ReferenceSegment child = 2;
+ }
+ }
+
+ // A reference that takes an existing subtype and selectively removes fields
+ // from it. For example, one might initially have an inner struct with 100
+ // fields but a a particular operation only needs to interact with only 2 of
+ // those 100 fields. In this situation, one would use a mask expression to
+ // eliminate the 98 fields that are not relevant to the rest of the operation
+ // pipeline.
+ //
+ // Note that this does not fundamentally alter the structure of data beyond
+ // the elimination of unecessary elements.
+ message MaskExpression {
+ StructSelect select = 1;
+ bool maintain_singular_struct = 2;
+
+ message Select {
+ oneof type {
+ StructSelect struct = 1;
+ ListSelect list = 2;
+ MapSelect map = 3;
+ }
+ }
+
+ message StructSelect {
+ repeated StructItem struct_items = 1;
+ }
+
+ message StructItem {
+ int32 field = 1;
+ Select child = 2;
+ }
+
+ message ListSelect {
+ repeated ListSelectItem selection = 1;
+ Select child = 2;
+
+ message ListSelectItem {
+ oneof type {
+ ListElement item = 1;
+ ListSlice slice = 2;
+ }
+
+ message ListElement {
+ int32 field = 1;
+ }
+
+ message ListSlice {
+ int32 start = 1;
+ int32 end = 2;
+ }
+ }
+ }
+
+ message MapSelect {
+ oneof select {
+ MapKey key = 1;
+ MapKeyExpression expression = 2;
+ }
+
+ Select child = 3;
+
+ message MapKey {
+ string map_key = 1;
+ }
+
+ message MapKeyExpression {
+ string map_key_expression = 1;
+ }
+ }
+ }
+
+ // A reference to an inner part of a complex object. Can reference reference a
+ // single element or a masked version of elements
+ message FieldReference {
+ // Whether this is composed of a single element reference or a masked
+ // element subtree
+ oneof reference_type {
+ ReferenceSegment direct_reference = 1;
+ MaskExpression masked_reference = 2;
+ }
+
+ // Whether this reference has an origin of a root struct or is based on the
+ // ouput of an expression. When this is a RootReference and direct_reference
+ // above is used, the direct_reference must be of a type StructField.
+ oneof root_type {
+ Expression expression = 3;
+ RootReference root_reference = 4;
+ OuterReference outer_reference = 5;
+ }
+
+ // Singleton that expresses this FieldReference is rooted off the root
+ // incoming record type
+ message RootReference {}
+
+ // A root reference for the outer relation's subquery
+ message OuterReference {
+ // number of subquery boundaries to traverse up for this field's reference
+ //
+ // This value must be >= 1
+ uint32 steps_out = 1;
+ }
+ }
+
+ // Subquery relation expression
+ message Subquery {
+ oneof subquery_type {
+ // Scalar subquery
+ Scalar scalar = 1;
+ // x IN y predicate
+ InPredicate in_predicate = 2;
+ // EXISTS/UNIQUE predicate
+ SetPredicate set_predicate = 3;
+ // ANY/ALL predicate
+ SetComparison set_comparison = 4;
+ }
+
+ // A subquery with one row and one column. This is often an aggregate
+ // though not required to be.
+ message Scalar {
+ Rel input = 1;
+ }
+
+ // Predicate checking that the left expression is contained in the right
+ // subquery
+ //
+ // Examples:
+ //
+ // x IN (SELECT * FROM t)
+ // (x, y) IN (SELECT a, b FROM t)
+ message InPredicate {
+ repeated Expression needles = 1;
+ Rel haystack = 2;
+ }
+
+ // A predicate over a set of rows in the form of a subquery
+ // EXISTS and UNIQUE are common SQL forms of this operation.
+ message SetPredicate {
+ enum PredicateOp {
+ PREDICATE_OP_UNSPECIFIED = 0;
+ PREDICATE_OP_EXISTS = 1;
+ PREDICATE_OP_UNIQUE = 2;
+ }
+ // TODO: should allow expressions
+ PredicateOp predicate_op = 1;
+ Rel tuples = 2;
+ }
+
+ // A subquery comparison using ANY or ALL.
+ // Examples:
+ //
+ // SELECT *
+ // FROM t1
+ // WHERE x < ANY(SELECT y from t2)
+ message SetComparison {
+ enum ComparisonOp {
+ COMPARISON_OP_UNSPECIFIED = 0;
+ COMPARISON_OP_EQ = 1;
+ COMPARISON_OP_NE = 2;
+ COMPARISON_OP_LT = 3;
+ COMPARISON_OP_GT = 4;
+ COMPARISON_OP_LE = 5;
+ COMPARISON_OP_GE = 6;
+ }
+
+ enum ReductionOp {
+ REDUCTION_OP_UNSPECIFIED = 0;
+ REDUCTION_OP_ANY = 1;
+ REDUCTION_OP_ALL = 2;
+ }
+
+ // ANY or ALL
+ ReductionOp reduction_op = 1;
+ // A comparison operator
+ ComparisonOp comparison_op = 2;
+ // left side of the expression
+ Expression left = 3;
+ // right side of the expression
+ Rel right = 4;
+ }
+ }
+}
+
+message GenerateRel {
+ RelCommon common = 1;
+ Rel input = 2;
+
+ Expression generator = 3;
+ repeated Expression child_output = 4;
+ bool outer = 5;
+
+ substrait.extensions.AdvancedExtension advanced_extension = 10;
+}
+
+// The description of a field to sort on (including the direction of sorting and null semantics)
+message SortField {
+ Expression expr = 1;
+
+ oneof sort_kind {
+ SortDirection direction = 2;
+ uint32 comparison_function_reference = 3;
+ }
+ enum SortDirection {
+ SORT_DIRECTION_UNSPECIFIED = 0;
+ SORT_DIRECTION_ASC_NULLS_FIRST = 1;
+ SORT_DIRECTION_ASC_NULLS_LAST = 2;
+ SORT_DIRECTION_DESC_NULLS_FIRST = 3;
+ SORT_DIRECTION_DESC_NULLS_LAST = 4;
+ SORT_DIRECTION_CLUSTERED = 5;
+ }
+}
+
+// Describes which part of an aggregation or window function to perform within
+// the context of distributed algorithms.
+enum AggregationPhase {
+ // Implies `INTERMEDIATE_TO_RESULT`.
+ AGGREGATION_PHASE_UNSPECIFIED = 0;
+
+ // Specifies that the function should be run only up to the point of
+ // generating an intermediate value, to be further aggregated later using
+ // INTERMEDIATE_TO_INTERMEDIATE or INTERMEDIATE_TO_RESULT.
+ AGGREGATION_PHASE_INITIAL_TO_INTERMEDIATE = 1;
+
+ // Specifies that the inputs of the aggregate or window function are the
+ // intermediate values of the function, and that the output should also be
+ // an intermediate value, to be further aggregated later using
+ // INTERMEDIATE_TO_INTERMEDIATE or INTERMEDIATE_TO_RESULT.
+ AGGREGATION_PHASE_INTERMEDIATE_TO_INTERMEDIATE = 2;
+
+ // A complete invocation: the function should aggregate the given set of
+ // inputs to yield a single return value. This style must be used for
+ // aggregate or window functions that are not decomposable.
+ AGGREGATION_PHASE_INITIAL_TO_RESULT = 3;
+
+ // Specifies that the inputs of the aggregate or window function are the
+ // intermediate values of the function, generated previously using
+ // INITIAL_TO_INTERMEDIATE and possibly INTERMEDIATE_TO_INTERMEDIATE calls.
+ // This call should combine the intermediate values to yield the final
+ // return value.
+ AGGREGATION_PHASE_INTERMEDIATE_TO_RESULT = 4;
+}
+
+enum WindowType {
+ ROWS = 0;
+ RANGE = 1;
+}
+
+// An aggregate function.
+message AggregateFunction {
+ // Points to a function_anchor defined in this plan, which must refer
+ // to an aggregate function in the associated YAML file. Required; 0 is
+ // considered to be a valid anchor/reference.
+ uint32 function_reference = 1;
+
+ // The arguments to be bound to the function. This must have exactly the
+ // number of arguments specified in the function definition, and the
+ // argument types must also match exactly:
+ //
+ // - Value arguments must be bound using FunctionArgument.value, and
+ // the expression in that must yield a value of a type that a function
+ // overload is defined for.
+ // - Type arguments must be bound using FunctionArgument.type, and a
+ // function overload must be defined for that type.
+ // - Enum arguments must be bound using FunctionArgument.enum
+ // followed by Enum.specified, with a string that case-insensitively
+ // matches one of the allowed options.
+ // - Optional enum arguments must be bound using FunctionArgument.enum
+ // followed by either Enum.specified or Enum.unspecified. If specified,
+ // the string must case-insensitively match one of the allowed options.
+ repeated FunctionArgument arguments = 7;
+
+ // Options to specify behavior for corner cases, or leave behavior
+ // unspecified if the consumer does not need specific behavior in these
+ // cases.
+ repeated FunctionOption options = 8;
+
+ // Must be set to the return type of the function, exactly as derived
+ // using the declaration in the extension.
+ Type output_type = 5;
+
+ // Describes which part of the aggregation to perform within the context of
+ // distributed algorithms. Required. Must be set to INITIAL_TO_RESULT for
+ // aggregate functions that are not decomposable.
+ AggregationPhase phase = 4;
+
+ // If specified, the aggregated records are ordered according to this list
+ // before they are aggregated. The first sort field has the highest
+ // priority; only if a sort field determines two records to be equivalent is
+ // the next field queried. This field is optional.
+ repeated SortField sorts = 3;
+
+ // Specifies whether equivalent records are merged before being aggregated.
+ // Optional, defaults to AGGREGATION_INVOCATION_ALL.
+ AggregationInvocation invocation = 6;
+
+ // deprecated; use arguments instead
+ repeated Expression args = 2 [deprecated = true];
+
+ // Method in which equivalent records are merged before being aggregated.
+ enum AggregationInvocation {
+ // This default value implies AGGREGATION_INVOCATION_ALL.
+ AGGREGATION_INVOCATION_UNSPECIFIED = 0;
+
+ // Use all values in the aggregation calculation.
+ AGGREGATION_INVOCATION_ALL = 1;
+
+ // Use only distinct values in the aggregation calculation.
+ AGGREGATION_INVOCATION_DISTINCT = 2;
+ }
+
+ // This rel is used to create references,
+ // in case we refer to a RelRoot field names will be ignored
+ message ReferenceRel {
+ int32 subtree_ordinal = 1;
+ }
+}
diff --git a/gluten-flink/core/src/main/resources/substrait/proto/substrait/capabilities.proto b/gluten-flink/core/src/main/resources/substrait/proto/substrait/capabilities.proto
new file mode 100644
index 000000000000..351427189a7d
--- /dev/null
+++ b/gluten-flink/core/src/main/resources/substrait/proto/substrait/capabilities.proto
@@ -0,0 +1,29 @@
+// SPDX-License-Identifier: Apache-2.0
+syntax = "proto3";
+
+package substrait;
+
+option csharp_namespace = "Substrait.Protobuf";
+option go_package = "github.com/substrait-io/substrait-go/proto";
+option java_multiple_files = true;
+option java_package = "io.substrait.proto";
+
+// Defines a set of Capabilities that a system (producer or consumer) supports.
+message Capabilities {
+ // List of Substrait versions this system supports
+ repeated string substrait_versions = 1;
+
+ // list of com.google.Any message types this system supports for advanced
+ // extensions.
+ repeated string advanced_extension_type_urls = 2;
+
+ // list of simple extensions this system supports.
+ repeated SimpleExtension simple_extensions = 3;
+
+ message SimpleExtension {
+ string uri = 1;
+ repeated string function_keys = 2;
+ repeated string type_keys = 3;
+ repeated string type_variation_keys = 4;
+ }
+}
diff --git a/gluten-flink/core/src/main/resources/substrait/proto/substrait/extended_expression.proto b/gluten-flink/core/src/main/resources/substrait/proto/substrait/extended_expression.proto
new file mode 100755
index 000000000000..5d1152055930
--- /dev/null
+++ b/gluten-flink/core/src/main/resources/substrait/proto/substrait/extended_expression.proto
@@ -0,0 +1,51 @@
+// SPDX-License-Identifier: Apache-2.0
+syntax = "proto3";
+
+package substrait;
+
+import "substrait/algebra.proto";
+import "substrait/extensions/extensions.proto";
+import "substrait/plan.proto";
+import "substrait/type.proto";
+
+option csharp_namespace = "Substrait.Protobuf";
+option go_package = "github.com/substrait-io/substrait-go/proto";
+option java_multiple_files = true;
+option java_package = "io.substrait.proto";
+
+message ExpressionReference {
+ oneof expr_type {
+ Expression expression = 1;
+ AggregateFunction measure = 2;
+ }
+ // Field names in depth-first order
+ repeated string output_names = 3;
+}
+
+// Describe a set of operations to complete.
+// For compactness sake, identifiers are normalized at the plan level.
+message ExtendedExpression {
+ // Substrait version of the expression. Optional up to 0.17.0, required for later
+ // versions.
+ Version version = 7;
+
+ // a list of yaml specifications this expression may depend on
+ repeated substrait.extensions.SimpleExtensionURI extension_uris = 1;
+
+ // a list of extensions this expression may depend on
+ repeated substrait.extensions.SimpleExtensionDeclaration extensions = 2;
+
+ // one or more expression trees with same order in plan rel
+ repeated ExpressionReference referred_expr = 3;
+
+ NamedStruct base_schema = 4;
+ // additional extensions associated with this expression.
+ substrait.extensions.AdvancedExtension advanced_extensions = 5;
+
+ // A list of com.google.Any entities that this plan may use. Can be used to
+ // warn if some embedded message types are unknown. Note that this list may
+ // include message types that are ignorable (optimizations) or that are
+ // unused. In many cases, a consumer may be able to work with a plan even if
+ // one or more message types defined here are unknown.
+ repeated string expected_type_urls = 6;
+}
diff --git a/gluten-flink/core/src/main/resources/substrait/proto/substrait/extensions/extensions.proto b/gluten-flink/core/src/main/resources/substrait/proto/substrait/extensions/extensions.proto
new file mode 100644
index 000000000000..d3af5f3d9ed1
--- /dev/null
+++ b/gluten-flink/core/src/main/resources/substrait/proto/substrait/extensions/extensions.proto
@@ -0,0 +1,81 @@
+// SPDX-License-Identifier: Apache-2.0
+syntax = "proto3";
+
+package substrait.extensions;
+
+import "google/protobuf/any.proto";
+
+option csharp_namespace = "Substrait.Protobuf";
+option go_package = "github.com/substrait-io/substrait-go/proto/extensions";
+option java_multiple_files = true;
+option java_package = "io.substrait.proto";
+
+message SimpleExtensionURI {
+ // A surrogate key used in the context of a single plan used to reference the
+ // URI associated with an extension.
+ uint32 extension_uri_anchor = 1;
+
+ // The URI where this extension YAML can be retrieved. This is the "namespace"
+ // of this extension.
+ string uri = 2;
+}
+
+// Describes a mapping between a specific extension entity and the uri where
+// that extension can be found.
+message SimpleExtensionDeclaration {
+ oneof mapping_type {
+ ExtensionType extension_type = 1;
+ ExtensionTypeVariation extension_type_variation = 2;
+ ExtensionFunction extension_function = 3;
+ }
+
+ // Describes a Type
+ message ExtensionType {
+ // references the extension_uri_anchor defined for a specific extension URI.
+ uint32 extension_uri_reference = 1;
+
+ // A surrogate key used in the context of a single plan to reference a
+ // specific extension type
+ uint32 type_anchor = 2;
+
+ // the name of the type in the defined extension YAML.
+ string name = 3;
+ }
+
+ message ExtensionTypeVariation {
+ // references the extension_uri_anchor defined for a specific extension URI.
+ uint32 extension_uri_reference = 1;
+
+ // A surrogate key used in the context of a single plan to reference a
+ // specific type variation
+ uint32 type_variation_anchor = 2;
+
+ // the name of the type in the defined extension YAML.
+ string name = 3;
+ }
+
+ message ExtensionFunction {
+ // references the extension_uri_anchor defined for a specific extension URI.
+ uint32 extension_uri_reference = 1;
+
+ // A surrogate key used in the context of a single plan to reference a
+ // specific function
+ uint32 function_anchor = 2;
+
+ // A simple name if there is only one impl for the function within the YAML.
+ // A compound name, referencing that includes type short names if there is
+ // more than one impl per name in the YAML.
+ string name = 3;
+ }
+}
+
+// A generic object that can be used to embed additional extension information
+// into the serialized substrait plan.
+message AdvancedExtension {
+ // An optimization is helpful information that don't influence semantics. May
+ // be ignored by a consumer.
+ google.protobuf.Any optimization = 1;
+
+ // An enhancement alter semantics. Cannot be ignored by a consumer.
+ google.protobuf.Any enhancement = 2;
+}
diff --git a/gluten-flink/core/src/main/resources/substrait/proto/substrait/function.proto b/gluten-flink/core/src/main/resources/substrait/proto/substrait/function.proto
new file mode 100644
index 000000000000..123f4a1bf749
--- /dev/null
+++ b/gluten-flink/core/src/main/resources/substrait/proto/substrait/function.proto
@@ -0,0 +1,148 @@
+// SPDX-License-Identifier: Apache-2.0
+syntax = "proto3";
+
+package substrait;
+
+import "substrait/parameterized_types.proto";
+import "substrait/type.proto";
+import "substrait/type_expressions.proto";
+
+option csharp_namespace = "Substrait.Protobuf";
+option go_package = "github.com/substrait-io/substrait-go/proto";
+option java_multiple_files = true;
+option java_package = "io.substrait.proto";
+
+// List of function signatures available.
+message FunctionSignature {
+ message FinalArgVariadic {
+ // the minimum number of arguments allowed for the list of final arguments
+ // (inclusive).
+ int64 min_args = 1;
+
+ // the maximum number of arguments allowed for the list of final arguments
+ // (exclusive)
+ int64 max_args = 2;
+
+ // the type of parameterized type consistency
+ ParameterConsistency consistency = 3;
+
+ enum ParameterConsistency {
+ PARAMETER_CONSISTENCY_UNSPECIFIED = 0;
+
+ // All argument must be the same concrete type.
+ PARAMETER_CONSISTENCY_CONSISTENT = 1;
+
+ // Each argument can be any possible concrete type afforded by the bounds
+ // of any parameter defined in the arguments specification.
+ PARAMETER_CONSISTENCY_INCONSISTENT = 2;
+ }
+ }
+
+ message FinalArgNormal {}
+
+ message Scalar {
+ repeated Argument arguments = 2;
+ repeated string name = 3;
+ Description description = 4;
+
+ bool deterministic = 7;
+ bool session_dependent = 8;
+
+ DerivationExpression output_type = 9;
+
+ oneof final_variable_behavior {
+ FinalArgVariadic variadic = 10;
+ FinalArgNormal normal = 11;
+ }
+
+ repeated Implementation implementations = 12;
+ }
+
+ message Aggregate {
+ repeated Argument arguments = 2;
+ string name = 3;
+ Description description = 4;
+
+ bool deterministic = 7;
+ bool session_dependent = 8;
+
+ DerivationExpression output_type = 9;
+
+ oneof final_variable_behavior {
+ FinalArgVariadic variadic = 10;
+ FinalArgNormal normal = 11;
+ }
+
+ bool ordered = 14;
+ uint64 max_set = 12;
+ Type intermediate_type = 13;
+
+ repeated Implementation implementations = 15;
+ }
+
+ message Window {
+ repeated Argument arguments = 2;
+ repeated string name = 3;
+ Description description = 4;
+
+ bool deterministic = 7;
+ bool session_dependent = 8;
+
+ DerivationExpression intermediate_type = 9;
+ DerivationExpression output_type = 10;
+ oneof final_variable_behavior {
+ FinalArgVariadic variadic = 16;
+ FinalArgNormal normal = 17;
+ }
+ bool ordered = 11;
+ uint64 max_set = 12;
+ WindowType window_type = 14;
+ repeated Implementation implementations = 15;
+
+ enum WindowType {
+ WINDOW_TYPE_UNSPECIFIED = 0;
+ WINDOW_TYPE_STREAMING = 1;
+ WINDOW_TYPE_PARTITION = 2;
+ }
+ }
+
+ message Description {
+ string language = 1;
+ string body = 2;
+ }
+
+ message Implementation {
+ Type type = 1;
+ string uri = 2;
+
+ enum Type {
+ TYPE_UNSPECIFIED = 0;
+ TYPE_WEB_ASSEMBLY = 1;
+ TYPE_TRINO_JAR = 2;
+ }
+ }
+
+ message Argument {
+ string name = 1;
+
+ oneof argument_kind {
+ ValueArgument value = 2;
+ TypeArgument type = 3;
+ EnumArgument enum = 4;
+ }
+
+ message ValueArgument {
+ ParameterizedType type = 1;
+ bool constant = 2;
+ }
+
+ message TypeArgument {
+ ParameterizedType type = 1;
+ }
+
+ message EnumArgument {
+ repeated string options = 1;
+ bool optional = 2;
+ }
+ }
+}
diff --git a/gluten-flink/core/src/main/resources/substrait/proto/substrait/parameterized_types.proto b/gluten-flink/core/src/main/resources/substrait/proto/substrait/parameterized_types.proto
new file mode 100644
index 000000000000..db0669354fa2
--- /dev/null
+++ b/gluten-flink/core/src/main/resources/substrait/proto/substrait/parameterized_types.proto
@@ -0,0 +1,128 @@
+// SPDX-License-Identifier: Apache-2.0
+syntax = "proto3";
+
+package substrait;
+
+import "substrait/type.proto";
+
+option csharp_namespace = "Substrait.Protobuf";
+option go_package = "github.com/substrait-io/substrait-go/proto";
+option java_multiple_files = true;
+option java_package = "io.substrait.proto";
+
+message ParameterizedType {
+ oneof kind {
+ Type.Boolean bool = 1;
+ Type.I8 i8 = 2;
+ Type.I16 i16 = 3;
+ Type.I32 i32 = 5;
+ Type.I64 i64 = 7;
+ Type.FP32 fp32 = 10;
+ Type.FP64 fp64 = 11;
+ Type.String string = 12;
+ Type.Binary binary = 13;
+ Type.Timestamp timestamp = 14;
+ Type.Date date = 16;
+ Type.Time time = 17;
+ Type.IntervalYear interval_year = 19;
+ Type.IntervalDay interval_day = 20;
+ Type.TimestampTZ timestamp_tz = 29;
+ Type.UUID uuid = 32;
+
+ ParameterizedFixedChar fixed_char = 21;
+ ParameterizedVarChar varchar = 22;
+ ParameterizedFixedBinary fixed_binary = 23;
+ ParameterizedDecimal decimal = 24;
+
+ ParameterizedStruct struct = 25;
+ ParameterizedList list = 27;
+ ParameterizedMap map = 28;
+
+ ParameterizedUserDefined user_defined = 30;
+
+ // Deprecated in favor of user_defined, which allows nullability and
+ // variations to be specified. If user_defined_pointer is encountered,
+ // treat it as being non-nullable and having the default variation.
+ uint32 user_defined_pointer = 31 [deprecated = true];
+
+ TypeParameter type_parameter = 33;
+ }
+
+ message TypeParameter {
+ string name = 1;
+ repeated ParameterizedType bounds = 2;
+ }
+
+ message IntegerParameter {
+ string name = 1;
+ NullableInteger range_start_inclusive = 2;
+ NullableInteger range_end_exclusive = 3;
+ }
+
+ message NullableInteger {
+ int64 value = 1;
+ }
+
+ message ParameterizedFixedChar {
+ IntegerOption length = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ParameterizedVarChar {
+ IntegerOption length = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ParameterizedFixedBinary {
+ IntegerOption length = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ParameterizedDecimal {
+ IntegerOption scale = 1;
+ IntegerOption precision = 2;
+ uint32 variation_pointer = 3;
+ Type.Nullability nullability = 4;
+ }
+
+ message ParameterizedStruct {
+ repeated ParameterizedType types = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ParameterizedNamedStruct {
+ // list of names in dfs order
+ repeated string names = 1;
+ ParameterizedStruct struct = 2;
+ }
+
+ message ParameterizedList {
+ ParameterizedType type = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ParameterizedMap {
+ ParameterizedType key = 1;
+ ParameterizedType value = 2;
+ uint32 variation_pointer = 3;
+ Type.Nullability nullability = 4;
+ }
+
+ message ParameterizedUserDefined {
+ uint32 type_pointer = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message IntegerOption {
+ oneof integer_type {
+ int32 literal = 1;
+ IntegerParameter parameter = 2;
+ }
+ }
+}
diff --git a/gluten-flink/core/src/main/resources/substrait/proto/substrait/plan.proto b/gluten-flink/core/src/main/resources/substrait/proto/substrait/plan.proto
new file mode 100644
index 000000000000..e5657fb8f1ef
--- /dev/null
+++ b/gluten-flink/core/src/main/resources/substrait/proto/substrait/plan.proto
@@ -0,0 +1,82 @@
+// SPDX-License-Identifier: Apache-2.0
+syntax = "proto3";
+
+package substrait;
+
+import "substrait/algebra.proto";
+import "substrait/extensions/extensions.proto";
+
+option csharp_namespace = "Substrait.Protobuf";
+option go_package = "github.com/substrait-io/substrait-go/proto";
+option java_multiple_files = true;
+option java_package = "io.substrait.proto";
+
+// Either a relation or root relation
+message PlanRel {
+ oneof rel_type {
+ // Any relation (used for references and CTEs)
+ Rel rel = 1;
+ // The root of a relation tree
+ RelRoot root = 2;
+ }
+}
+
+// Describe a set of operations to complete.
+// For compactness sake, identifiers are normalized at the plan level.
+message Plan {
+ // Substrait version of the plan. Optional up to 0.17.0, required for later
+ // versions.
+ Version version = 6;
+
+ // a list of yaml specifications this plan may depend on
+ repeated substrait.extensions.SimpleExtensionURI extension_uris = 1;
+
+ // a list of extensions this plan may depend on
+ repeated substrait.extensions.SimpleExtensionDeclaration extensions = 2;
+
+ // one or more relation trees that are associated with this plan.
+ repeated PlanRel relations = 3;
+
+ // additional extensions associated with this plan.
+ substrait.extensions.AdvancedExtension advanced_extensions = 4;
+
+ // A list of com.google.Any entities that this plan may use. Can be used to
+ // warn if some embedded message types are unknown. Note that this list may
+ // include message types that are ignorable (optimizations) or that are
+ // unused. In many cases, a consumer may be able to work with a plan even if
+ // one or more message types defined here are unknown.
+ repeated string expected_type_urls = 5;
+}
+
+// This message type can be used to deserialize only the version of a Substrait
+// Plan message. This prevents deserialization errors when there were breaking
+// changes between the Substrait version of the tool that produced the plan and
+// the Substrait version used to deserialize it, such that a consumer can emit
+// a more helpful error message in this case.
+message PlanVersion {
+ Version version = 6;
+}
+
+message Version {
+ // Substrait version number.
+ uint32 major_number = 1;
+ uint32 minor_number = 2;
+ uint32 patch_number = 3;
+
+ // If a particular version of Substrait is used that does not correspond to
+ // a version number exactly (for example when using an unofficial fork or
+ // using a version that is not yet released or is between versions), set this
+ // to the full git hash of the utilized commit of
+ // https://github.com/substrait-io/substrait (or fork thereof), represented
+ // using a lowercase hex ASCII string 40 characters in length. The version
+ // number above should be set to the most recent version tag in the history
+ // of that commit.
+ string git_hash = 4;
+
+ // Identifying information for the producer that created this plan. Under
+ // ideal circumstances, consumers should not need this information. However,
+ // it is foreseen that consumers may need to work around bugs in particular
+ // producers in practice, and therefore may need to know which producer
+ // created the plan.
+ string producer = 5;
+}
diff --git a/gluten-flink/core/src/main/resources/substrait/proto/substrait/type.proto b/gluten-flink/core/src/main/resources/substrait/proto/substrait/type.proto
new file mode 100644
index 000000000000..b5fcb95623ac
--- /dev/null
+++ b/gluten-flink/core/src/main/resources/substrait/proto/substrait/type.proto
@@ -0,0 +1,243 @@
+// SPDX-License-Identifier: Apache-2.0
+syntax = "proto3";
+
+package substrait;
+
+import "google/protobuf/empty.proto";
+
+option csharp_namespace = "Substrait.Protobuf";
+option go_package = "github.com/substrait-io/substrait-go/proto";
+option java_multiple_files = true;
+option java_package = "io.substrait.proto";
+
+message Type {
+ oneof kind {
+ Boolean bool = 1;
+ I8 i8 = 2;
+ I16 i16 = 3;
+ I32 i32 = 5;
+ I64 i64 = 7;
+ FP32 fp32 = 10;
+ FP64 fp64 = 11;
+ String string = 12;
+ Binary binary = 13;
+ Timestamp timestamp = 14;
+ Date date = 16;
+ Time time = 17;
+ IntervalYear interval_year = 19;
+ IntervalDay interval_day = 20;
+ TimestampTZ timestamp_tz = 29;
+ UUID uuid = 32;
+
+ FixedChar fixed_char = 21;
+ VarChar varchar = 22;
+ FixedBinary fixed_binary = 23;
+ Decimal decimal = 24;
+
+ Struct struct = 25;
+ List list = 27;
+ Map map = 28;
+
+ UserDefined user_defined = 30;
+
+ // Deprecated in favor of user_defined, which allows nullability and
+ // variations to be specified. If user_defined_type_reference is
+ // encountered, treat it as being non-nullable and having the default
+ // variation.
+ uint32 user_defined_type_reference = 31 [deprecated = true];
+
+ Nothing nothing = 33;
+ }
+
+ enum Nullability {
+ NULLABILITY_UNSPECIFIED = 0;
+ NULLABILITY_NULLABLE = 1;
+ NULLABILITY_REQUIRED = 2;
+ }
+
+ message Nothing {
+ uint32 type_variation_reference = 1;
+ }
+
+ message Boolean {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message I8 {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message I16 {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message I32 {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message I64 {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message FP32 {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message FP64 {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message String {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message Binary {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message Timestamp {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message Date {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message Time {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message TimestampTZ {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message IntervalYear {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message IntervalDay {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ message UUID {
+ uint32 type_variation_reference = 1;
+ Nullability nullability = 2;
+ }
+
+ // Start compound types.
+ message FixedChar {
+ int32 length = 1;
+ uint32 type_variation_reference = 2;
+ Nullability nullability = 3;
+ }
+
+ message VarChar {
+ int32 length = 1;
+ uint32 type_variation_reference = 2;
+ Nullability nullability = 3;
+ }
+
+ message FixedBinary {
+ int32 length = 1;
+ uint32 type_variation_reference = 2;
+ Nullability nullability = 3;
+ }
+
+ message Decimal {
+ int32 scale = 1;
+ int32 precision = 2;
+ uint32 type_variation_reference = 3;
+ Nullability nullability = 4;
+ }
+
+ message Struct {
+ repeated Type types = 1;
+ uint32 type_variation_reference = 2;
+ Nullability nullability = 3;
+ repeated string names = 4;
+ }
+
+ message List {
+ Type type = 1;
+ uint32 type_variation_reference = 2;
+ Nullability nullability = 3;
+ }
+
+ message Map {
+ Type key = 1;
+ Type value = 2;
+ uint32 type_variation_reference = 3;
+ Nullability nullability = 4;
+ }
+
+ message UserDefined {
+ uint32 type_reference = 1;
+ uint32 type_variation_reference = 2;
+ Nullability nullability = 3;
+ repeated Parameter type_parameters = 4;
+ }
+
+ message Parameter {
+ oneof parameter {
+ // Explicitly null/unspecified parameter, to select the default value (if
+ // any).
+ google.protobuf.Empty null = 1;
+
+ // Data type parameters, like the i32 in LIST.
+ Type data_type = 2;
+
+ // Value parameters, like the 10 in VARCHAR<10>.
+ bool boolean = 3;
+ int64 integer = 4;
+ string enum = 5;
+ string string = 6;
+ }
+ }
+}
+
+// A message for modeling name/type pairs.
+//
+// Useful for representing relation schemas.
+//
+// Notes:
+//
+// * The names field is in depth-first order.
+//
+// For example a schema such as:
+//
+// a: int64
+// b: struct
+//
+// would have a `names` field that looks like:
+//
+// ["a", "b", "c", "d"]
+//
+// * Only struct fields are contained in this field's elements,
+// * Map keys should be traversed first, then values when producing/consuming
+message NamedStruct {
+ // list of names in dfs order
+ repeated string names = 1;
+ Type.Struct struct = 2;
+ repeated ColumnType column_types = 3;
+ enum ColumnType {
+ NORMAL_COL = 0;
+ PARTITION_COL = 1;
+ METADATA_COL = 2;
+ ROWINDEX_COL = 3;
+ }
+}
diff --git a/gluten-flink/core/src/main/resources/substrait/proto/substrait/type_expressions.proto b/gluten-flink/core/src/main/resources/substrait/proto/substrait/type_expressions.proto
new file mode 100644
index 000000000000..4be4aab47d40
--- /dev/null
+++ b/gluten-flink/core/src/main/resources/substrait/proto/substrait/type_expressions.proto
@@ -0,0 +1,160 @@
+// SPDX-License-Identifier: Apache-2.0
+syntax = "proto3";
+
+package substrait;
+
+import "substrait/type.proto";
+
+option csharp_namespace = "Substrait.Protobuf";
+option go_package = "github.com/substrait-io/substrait-go/proto";
+option java_multiple_files = true;
+option java_package = "io.substrait.proto";
+
+message DerivationExpression {
+ oneof kind {
+ Type.Boolean bool = 1;
+ Type.I8 i8 = 2;
+ Type.I16 i16 = 3;
+ Type.I32 i32 = 5;
+ Type.I64 i64 = 7;
+ Type.FP32 fp32 = 10;
+ Type.FP64 fp64 = 11;
+ Type.String string = 12;
+ Type.Binary binary = 13;
+ Type.Timestamp timestamp = 14;
+ Type.Date date = 16;
+ Type.Time time = 17;
+ Type.IntervalYear interval_year = 19;
+ Type.IntervalDay interval_day = 20;
+ Type.TimestampTZ timestamp_tz = 29;
+ Type.UUID uuid = 32;
+
+ ExpressionFixedChar fixed_char = 21;
+ ExpressionVarChar varchar = 22;
+ ExpressionFixedBinary fixed_binary = 23;
+ ExpressionDecimal decimal = 24;
+
+ ExpressionStruct struct = 25;
+ ExpressionList list = 27;
+ ExpressionMap map = 28;
+
+ ExpressionUserDefined user_defined = 30;
+
+ // Deprecated in favor of user_defined, which allows nullability and
+ // variations to be specified. If user_defined_pointer is encountered,
+ // treat it as being non-nullable and having the default variation.
+ uint32 user_defined_pointer = 31 [deprecated = true];
+
+ string type_parameter_name = 33;
+ string integer_parameter_name = 34;
+
+ int32 integer_literal = 35;
+ UnaryOp unary_op = 36;
+ BinaryOp binary_op = 37;
+ IfElse if_else = 38;
+ ReturnProgram return_program = 39;
+ }
+
+ message ExpressionFixedChar {
+ DerivationExpression length = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ExpressionVarChar {
+ DerivationExpression length = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ExpressionFixedBinary {
+ DerivationExpression length = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ExpressionDecimal {
+ DerivationExpression scale = 1;
+ DerivationExpression precision = 2;
+ uint32 variation_pointer = 3;
+ Type.Nullability nullability = 4;
+ }
+
+ message ExpressionStruct {
+ repeated DerivationExpression types = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ExpressionNamedStruct {
+ repeated string names = 1;
+ ExpressionStruct struct = 2;
+ }
+
+ message ExpressionList {
+ DerivationExpression type = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message ExpressionMap {
+ DerivationExpression key = 1;
+ DerivationExpression value = 2;
+ uint32 variation_pointer = 3;
+ Type.Nullability nullability = 4;
+ }
+
+ message ExpressionUserDefined {
+ uint32 type_pointer = 1;
+ uint32 variation_pointer = 2;
+ Type.Nullability nullability = 3;
+ }
+
+ message IfElse {
+ DerivationExpression if_condition = 1;
+ DerivationExpression if_return = 2;
+ DerivationExpression else_return = 3;
+ }
+
+ message UnaryOp {
+ UnaryOpType op_type = 1;
+ DerivationExpression arg = 2;
+
+ enum UnaryOpType {
+ UNARY_OP_TYPE_UNSPECIFIED = 0;
+ UNARY_OP_TYPE_BOOLEAN_NOT = 1;
+ }
+ }
+
+ message BinaryOp {
+ BinaryOpType op_type = 1;
+ DerivationExpression arg1 = 2;
+ DerivationExpression arg2 = 3;
+
+ enum BinaryOpType {
+ BINARY_OP_TYPE_UNSPECIFIED = 0;
+ BINARY_OP_TYPE_PLUS = 1;
+ BINARY_OP_TYPE_MINUS = 2;
+ BINARY_OP_TYPE_MULTIPLY = 3;
+ BINARY_OP_TYPE_DIVIDE = 4;
+ BINARY_OP_TYPE_MIN = 5;
+ BINARY_OP_TYPE_MAX = 6;
+ BINARY_OP_TYPE_GREATER_THAN = 7;
+ BINARY_OP_TYPE_LESS_THAN = 8;
+ BINARY_OP_TYPE_AND = 9;
+ BINARY_OP_TYPE_OR = 10;
+ BINARY_OP_TYPE_EQUALS = 11;
+ BINARY_OP_TYPE_COVERS = 12;
+ }
+ }
+
+ message ReturnProgram {
+ message Assignment {
+ string name = 1;
+ DerivationExpression expression = 2;
+ }
+
+ repeated Assignment assignments = 1;
+ DerivationExpression final_expression = 2;
+ }
+}
diff --git a/gluten-flink/pom.xml b/gluten-flink/pom.xml
new file mode 100644
index 000000000000..b2fbaf9e3ae6
--- /dev/null
+++ b/gluten-flink/pom.xml
@@ -0,0 +1,38 @@
+
+
+
+ 4.0.0
+
+
+ org.apache.gluten
+ gluten-parent
+ 1.4.0-SNAPSHOT
+ ../pom.xml
+
+
+ gluten-flink
+ Gluten Flink
+ pom
+
+
+ core
+
+
+
+ 1.20.0
+
+