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 + +