diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index a175dd37cb51c5..1404677d70b940 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -99,7 +99,7 @@ #include "service/backend_options.h" #include "service/backend_service.h" #include "service/point_query_executor.h" -#include "udf/python/python_udf_server.h" +#include "udf/python/python_server.h" #include "util/bfd_parser.h" #include "util/bit_util.h" #include "util/brpc_client_cache.h" @@ -890,7 +890,7 @@ void ExecEnv::destroy() { _s_tracking_memory = false; clear_storage_resource(); - PythonUDFServerManager::instance().shutdown(); + PythonServerManager::instance().shutdown(); LOG(INFO) << "Doris exec envorinment is destoried."; } diff --git a/be/src/udf/python/python_client.cpp b/be/src/udf/python/python_client.cpp new file mode 100644 index 00000000000000..cdf50df143c1ab --- /dev/null +++ b/be/src/udf/python/python_client.cpp @@ -0,0 +1,161 @@ +// 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. + +#include "udf/python/python_client.h" + +#include "arrow/flight/client.h" +#include "arrow/flight/server.h" +#include "common/compiler_util.h" +#include "common/status.h" +#include "udf/python/python_udf_meta.h" +#include "udf/python/python_udf_runtime.h" +#include "util/arrow/utils.h" + +namespace doris { + +Status PythonClient::init(const PythonUDFMeta& func_meta, ProcessPtr process) { + if (_inited) { + return Status::InternalError("PythonClient has already been initialized"); + } + + // Set operation name based on client type + switch (func_meta.client_type) { + case PythonClientType::UDF: + _operation_name = "Python UDF"; + break; + case PythonClientType::UDAF: + _operation_name = "Python UDAF"; + break; + case PythonClientType::UDTF: + _operation_name = "Python UDTF"; + break; + default: + return Status::InternalError("Invalid Python client type"); + } + + // Parse and connect to Python server location + arrow::flight::Location location; + RETURN_DORIS_STATUS_IF_RESULT_ERROR(location, + arrow::flight::Location::Parse(process->get_uri())); + RETURN_DORIS_STATUS_IF_RESULT_ERROR(_arrow_client, FlightClient::Connect(location)); + + // Serialize function metadata to JSON command + std::string command; + RETURN_IF_ERROR(func_meta.serialize_to_json(&command)); + + // Create Flight descriptor and establish bidirectional streaming + FlightDescriptor descriptor = FlightDescriptor::Command(command); + arrow::flight::FlightClient::DoExchangeResult exchange_res; + RETURN_DORIS_STATUS_IF_RESULT_ERROR(exchange_res, _arrow_client->DoExchange(descriptor)); + + _reader = std::move(exchange_res.reader); + _writer = std::move(exchange_res.writer); + _process = std::move(process); + _inited = true; + + return Status::OK(); +} + +Status PythonClient::close() { + if (!_inited || !_writer) { + return Status::OK(); + } + + auto writer_res = _writer->Close(); + if (!writer_res.ok()) { + // Don't propagate error from close, just log it + LOG(WARNING) << "Error closing Python client writer: " << writer_res.message(); + } + + _inited = false; + _begin = false; + _arrow_client.reset(); + _writer.reset(); + _reader.reset(); + + // Return process to pool if available + if (auto* pool = _process->pool(); pool) { + pool->return_process(std::move(_process)); + } + + return Status::OK(); +} + +Status PythonClient::handle_error(arrow::Status status) { + DCHECK(!status.ok()); + + // Clean up resources + _writer.reset(); + _reader.reset(); + _process->shutdown(); + + // Extract and clean error message + std::string msg = status.message(); + LOG(ERROR) << _operation_name << " error: " << msg; + + // Remove Python traceback noise for cleaner error messages + size_t pos = msg.find("The above exception was the direct cause"); + if (pos != std::string::npos) { + msg = msg.substr(0, pos); + } + + return Status::RuntimeError(trim(msg)); +} + +Status PythonClient::check_process_alive() const { + if (UNLIKELY(!_process->is_alive())) { + return Status::RuntimeError("{} process is not alive", _operation_name); + } + return Status::OK(); +} + +Status PythonClient::begin_stream(const std::shared_ptr& schema) { + if (UNLIKELY(!_begin)) { + auto begin_res = _writer->Begin(schema); + if (!begin_res.ok()) { + return handle_error(begin_res); + } + _begin = true; + } + return Status::OK(); +} + +Status PythonClient::write_batch(const arrow::RecordBatch& input) { + auto write_res = _writer->WriteRecordBatch(input); + if (!write_res.ok()) { + return handle_error(write_res); + } + return Status::OK(); +} + +Status PythonClient::read_batch(std::shared_ptr* output) { + auto read_res = _reader->Next(); + if (!read_res.ok()) { + return handle_error(read_res.status()); + } + + arrow::flight::FlightStreamChunk chunk = std::move(*read_res); + if (!chunk.data) { + _process->shutdown(); + return Status::InternalError("Received null RecordBatch from {} server", _operation_name); + } + + *output = std::move(chunk.data); + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/udf/python/python_client.h b/be/src/udf/python/python_client.h new file mode 100644 index 00000000000000..d4193acff5f4fa --- /dev/null +++ b/be/src/udf/python/python_client.h @@ -0,0 +1,117 @@ +// 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. + +#pragma once + +#include + +#include "arrow/flight/client.h" +#include "common/status.h" +#include "udf/python/python_udf_meta.h" +#include "udf/python/python_udf_runtime.h" +#include "util/arrow/utils.h" + +namespace doris { + +/** + * Base class for Python UDF/UDAF/UDTF clients + * + * Provides common functionality for communicating with Python server via Arrow Flight: + * - Connection management + * - Stream initialization + * - Error handling + * - Process lifecycle management + */ +class PythonClient { +public: + using FlightDescriptor = arrow::flight::FlightDescriptor; + using FlightClient = arrow::flight::FlightClient; + using FlightStreamWriter = arrow::flight::FlightStreamWriter; + using FlightStreamReader = arrow::flight::FlightStreamReader; + + PythonClient() = default; + virtual ~PythonClient() = default; + + /** + * Initialize connection to Python server + * @param func_meta Function metadata (contains client_type for operation name) + * @param process Python process handle + * @return Status + */ + Status init(const PythonUDFMeta& func_meta, ProcessPtr process); + + /** + * Close connection and cleanup resources + * @return Status + */ + Status close(); + + /** + * Handle Arrow Flight error + * @param status Arrow status + * @return Doris Status with formatted error message + */ + Status handle_error(arrow::Status status); + + /** + * Get process information for debugging + * @return Process string representation + */ + std::string print_process() const { return _process->to_string(); } + +protected: + /** + * Check if process is alive + * @return Status + */ + Status check_process_alive() const; + + /** + * Begin Flight stream with schema (called only once per stream) + * @param schema Input schema + * @return Status + */ + Status begin_stream(const std::shared_ptr& schema); + + /** + * Write RecordBatch to server + * @param input Input RecordBatch + * @return Status + */ + Status write_batch(const arrow::RecordBatch& input); + + /** + * Read RecordBatch from server + * @param output Output RecordBatch + * @return Status + */ + Status read_batch(std::shared_ptr* output); + + // Common state + bool _inited = false; + bool _begin = false; // Track if Begin() has been called + std::string _operation_name; // Operation name for error messages + std::unique_ptr _arrow_client; + std::unique_ptr _writer; + std::unique_ptr _reader; + ProcessPtr _process; + +private: + DISALLOW_COPY_AND_ASSIGN(PythonClient); +}; + +} // namespace doris diff --git a/be/src/udf/python/python_env.cpp b/be/src/udf/python/python_env.cpp index 0b29be8728cd79..72f2c5d9c98770 100644 --- a/be/src/udf/python/python_env.cpp +++ b/be/src/udf/python/python_env.cpp @@ -25,7 +25,7 @@ #include #include "common/status.h" -#include "udf/python/python_udf_server.h" +#include "udf/python/python_server.h" #include "util/string_util.h" namespace doris { @@ -283,7 +283,7 @@ Status PythonVersionManager::init(PythonEnvType env_type, const fs::path& python std::vector versions; RETURN_IF_ERROR(_env_scanner->scan()); RETURN_IF_ERROR(_env_scanner->get_versions(&versions)); - RETURN_IF_ERROR(PythonUDFServerManager::instance().init(versions)); + RETURN_IF_ERROR(PythonServerManager::instance().init(versions)); return Status::OK(); } diff --git a/be/src/udf/python/python_udf_server.cpp b/be/src/udf/python/python_server.cpp similarity index 85% rename from be/src/udf/python/python_udf_server.cpp rename to be/src/udf/python/python_server.cpp index 542df1a3868d06..85ffac2a7319c4 100644 --- a/be/src/udf/python/python_udf_server.cpp +++ b/be/src/udf/python/python_server.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "udf/python/python_udf_server.h" +#include "udf/python/python_server.h" #include #include @@ -29,10 +29,11 @@ #include "common/config.h" #include "udf/python/python_udaf_client.h" #include "udf/python/python_udf_client.h" +#include "udf/python/python_udtf_client.h" namespace doris { -Status PythonUDFServerManager::init(const std::vector& versions) { +Status PythonServerManager::init(const std::vector& versions) { std::lock_guard lock(_pools_mutex); for (const auto& version : versions) { if (_pools.find(version) != _pools.end()) continue; @@ -45,9 +46,9 @@ Status PythonUDFServerManager::init(const std::vector& versions) } template -Status PythonUDFServerManager::get_client(const PythonUDFMeta& func_meta, - const PythonVersion& version, - std::shared_ptr* client) { +Status PythonServerManager::get_client(const PythonUDFMeta& func_meta, + const PythonVersion& version, + std::shared_ptr* client) { PythonUDFProcessPoolPtr* pool = nullptr; { std::lock_guard lock(_pools_mutex); @@ -65,12 +66,12 @@ Status PythonUDFServerManager::get_client(const PythonUDFMeta& func_meta, return Status::OK(); } -Status PythonUDFServerManager::fork(PythonUDFProcessPool* pool, ProcessPtr* process) { +Status PythonServerManager::fork(PythonUDFProcessPool* pool, ProcessPtr* process) { DCHECK(pool != nullptr); const PythonVersion& version = pool->get_python_version(); // e.g. /usr/local/python3.7/bin/python3 std::string python_executable_path = version.get_executable_path(); - // e.g. /{DORIS_HOME}/plugins/python_udf/python_udf_server.py + // e.g. /{DORIS_HOME}/plugins/python_udf/python_server.py std::string fight_server_path = get_fight_server_path(); // e.g. grpc+unix:///home/doris/output/be/lib/udf/python/python_udf std::string base_unix_socket_path = get_base_unix_socket_path(); @@ -131,7 +132,7 @@ Status PythonUDFServerManager::fork(PythonUDFProcessPool* pool, ProcessPtr* proc return Status::OK(); } -void PythonUDFServerManager::shutdown() { +void PythonServerManager::shutdown() { std::lock_guard lock(_pools_mutex); for (auto& pool : _pools) { pool.second->shutdown(); @@ -140,13 +141,17 @@ void PythonUDFServerManager::shutdown() { LOG(INFO) << "Python UDF server manager shutdown successfully"; } -// Explicit template instantiation for UDF and UDAF clients -template Status PythonUDFServerManager::get_client( +// Explicit template instantiation for UDF, UDAF and UDTF clients +template Status PythonServerManager::get_client( const PythonUDFMeta& func_meta, const PythonVersion& version, std::shared_ptr* client); -template Status PythonUDFServerManager::get_client( +template Status PythonServerManager::get_client( const PythonUDFMeta& func_meta, const PythonVersion& version, std::shared_ptr* client); +template Status PythonServerManager::get_client( + const PythonUDFMeta& func_meta, const PythonVersion& version, + std::shared_ptr* client); + } // namespace doris \ No newline at end of file diff --git a/be/src/udf/python/python_udf_server.h b/be/src/udf/python/python_server.h similarity index 83% rename from be/src/udf/python/python_udf_server.h rename to be/src/udf/python/python_server.h index 890379aef5c5a3..a78f570398d2bb 100644 --- a/be/src/udf/python/python_udf_server.h +++ b/be/src/udf/python/python_server.h @@ -20,23 +20,22 @@ #include #include "common/status.h" +#include "udf/python/python_udaf_client.h" #include "udf/python/python_udf_client.h" #include "udf/python/python_udf_meta.h" #include "udf/python/python_udf_runtime.h" +#include "udf/python/python_udtf_client.h" namespace doris { -class PythonUDAFClient; -using PythonUDAFClientPtr = std::shared_ptr; - -class PythonUDFServerManager { +class PythonServerManager { public: - PythonUDFServerManager() = default; + PythonServerManager() = default; - ~PythonUDFServerManager() = default; + ~PythonServerManager() = default; - static PythonUDFServerManager& instance() { - static PythonUDFServerManager instance; + static PythonServerManager& instance() { + static PythonServerManager instance; return instance; } diff --git a/be/src/udf/python/python_udf_server.py b/be/src/udf/python/python_server.py similarity index 89% rename from be/src/udf/python/python_udf_server.py rename to be/src/udf/python/python_server.py index 417b0597762e74..d11ba2272596a2 100644 --- a/be/src/udf/python/python_udf_server.py +++ b/be/src/udf/python/python_server.py @@ -33,6 +33,7 @@ from datetime import datetime from enum import Enum from pathlib import Path +from logging.handlers import RotatingFileHandler import pandas as pd import pyarrow as pa @@ -48,7 +49,8 @@ class ServerState: @staticmethod def setup_logging(): - """Setup logging configuration for the UDF server.""" + """Setup logging configuration for the UDF server with rotation.""" + doris_home = os.getenv("DORIS_HOME") if not doris_home: # Fallback to current directory if DORIS_HOME is not set @@ -56,17 +58,37 @@ def setup_logging(): log_dir = os.path.join(doris_home, "lib", "udf", "python") os.makedirs(log_dir, exist_ok=True) + + # Use shared log file with process ID in each log line log_file = os.path.join(log_dir, "python_udf_output.log") + max_bytes = 128 * 1024 * 1024 # 128MB + backup_count = 5 + + # Use RotatingFileHandler to automatically manage log file size + file_handler = RotatingFileHandler( + log_file, maxBytes=max_bytes, backupCount=backup_count, encoding="utf-8" + ) + + # Include process ID in log format + file_handler.setFormatter( + logging.Formatter( + "[%(asctime)s] [PID:%(process)d] [%(levelname)s] [%(filename)s:%(lineno)d] %(message)s" + ) + ) logging.basicConfig( level=logging.INFO, - format="[%(asctime)s] [%(levelname)s] [%(filename)s:%(lineno)d] %(message)s", handlers=[ - logging.FileHandler(log_file, mode="a", encoding="utf-8"), + file_handler, logging.StreamHandler(sys.stderr), # Also log to stderr for debugging ], ) - logging.info("Logging initialized. Log file: %s", log_file) + logging.info( + "Logging initialized. Log file: %s (max_size=%dMB, backups=%d)", + log_file, + max_bytes // (1024 * 1024), + backup_count, + ) @staticmethod def extract_base_unix_socket_path(unix_socket_uri: str) -> str: @@ -592,7 +614,6 @@ def load(self) -> AdaptivePythonUDF: ) raise ValueError(f"'{symbol}' is not a callable function.") - logging.info("Successfully loaded function '%s' from inline code", symbol) return AdaptivePythonUDF(self.python_udf_meta, func) @@ -981,7 +1002,7 @@ def load_from_module( @staticmethod def validate_udaf_class(udaf_class: type): """ - Validate that the UDAF class follows the required Snowflake pattern. + Validate that the UDAF class implements required methods. Args: udaf_class: The class to validate @@ -1042,7 +1063,7 @@ def set_udaf_class(self, udaf_class: type): Set the UDAF class to use for creating instances. Args: - udaf_class: The UDAF class (must follow Snowflake pattern) + udaf_class: The UDAF class Note: Validation is performed by UDAFClassLoader before calling this method. @@ -1285,18 +1306,14 @@ def reset(self, place_id: int) -> None: ) logging.error(error_msg) raise RuntimeError(error_msg) - + if self.udaf_class is None: - raise RuntimeError( - "UDAF class not set. Call set_udaf_class() first." - ) + raise RuntimeError("UDAF class not set. Call set_udaf_class() first.") try: self.states[place_id] = self.udaf_class() except Exception as e: - logging.error( - "Error resetting state for place_id %s: %s", place_id, e - ) + logging.error("Error resetting state for place_id %s: %s", place_id, e) raise RuntimeError(f"Error resetting state: {e}") from e def destroy(self, place_id: int) -> None: @@ -1318,7 +1335,7 @@ def destroy(self, place_id: int) -> None: ) logging.error(error_msg) raise RuntimeError(error_msg) - + del self.states[place_id] def clear_all(self) -> None: @@ -1330,7 +1347,7 @@ def clear_all(self) -> None: class FlightServer(flight.FlightServerBase): - """Arrow Flight server for executing Python UDFs and UDAFs.""" + """Arrow Flight server for executing Python UDFs, UDAFs, and UDTFs.""" def __init__(self, location: str): """ @@ -1719,7 +1736,7 @@ def _handle_udaf_destroy( [pa.array([success], type=pa.bool_())], ["success"] ) - def _do_exchange_udf( + def _handle_exchange_udf( self, python_udf_meta: PythonUDFMeta, reader: flight.MetadataRecordBatchReader, @@ -1761,7 +1778,7 @@ def _do_exchange_udf( started = True writer.write_batch(result_batch) - def _do_exchange_udaf( + def _handle_exchange_udaf( self, python_udaf_meta: PythonUDFMeta, reader: flight.MetadataRecordBatchReader, @@ -1853,6 +1870,164 @@ def _do_exchange_udaf( writer.write_batch(unified_response) + def _handle_exchange_udtf( + self, + python_udtf_meta: PythonUDFMeta, + reader: flight.MetadataRecordBatchReader, + writer: flight.MetadataRecordBatchWriter, + ) -> None: + """ + Handle bidirectional streaming for UDTF execution. + + Protocol (ListArray-based): + - Input: RecordBatch with input columns + - Output: RecordBatch with a single ListArray column + * ListArray automatically manages offsets internally + * Each list element contains the outputs for one input row + + Example: + Input: 3 rows + UDTF yields: Row 0 -> 5 outputs, Row 1 -> 2 outputs, Row 2 -> 3 outputs + Output: ListArray with 3 elements (one per input row) + - Element 0: List of 5 structs + - Element 1: List of 2 structs + - Element 2: List of 3 structs + """ + loader = UDFLoaderFactory.get_loader(python_udtf_meta) + adaptive_udtf = loader.load() + udtf_func = adaptive_udtf._eval_func + started = False + + for chunk in reader: + if not chunk.data: + logging.info("Empty chunk received, skipping") + continue + + input_batch = chunk.data + + # Validate input schema + check_schema_result, error_msg = self.check_schema( + input_batch, python_udtf_meta.input_types + ) + if not check_schema_result: + logging.error("Schema mismatch: %s", error_msg) + raise ValueError(f"Schema mismatch: {error_msg}") + + # Process all input rows and build ListArray + try: + response_batch = self._process_udtf_with_list_array( + udtf_func, input_batch, python_udtf_meta.output_type + ) + + # Send the response batch + if not started: + writer.begin(response_batch.schema) + started = True + + writer.write_batch(response_batch) + + except Exception as e: + logging.error( + "Error in UDTF execution: %s\nTraceback: %s", + e, + traceback.format_exc(), + ) + raise RuntimeError(f"Error in UDTF execution: {e}") from e + + def _process_udtf_with_list_array( + self, + udtf_func: Callable, + input_batch: pa.RecordBatch, + expected_output_type: pa.DataType, + ) -> pa.RecordBatch: + """ + Process UDTF function on all input rows and generate a ListArray. + + Args: + udtf_func: The UDTF function to call + input_batch: Input RecordBatch with N rows + expected_output_type: Expected Arrow type for output data + + Returns: + RecordBatch with a single ListArray column where each element + is a list of outputs for the corresponding input row + """ + all_results = [] # List of lists: one list per input row + + # Check if output is single-field or multi-field + # For single-field output, we allow yielding scalar values directly + is_single_field = not pa.types.is_struct(expected_output_type) + + # Process each input row + for row_idx in range(input_batch.num_rows): + # Extract row as tuple of arguments + row_args = tuple( + input_batch.column(col_idx)[row_idx].as_py() + for col_idx in range(input_batch.num_columns) + ) + + # Call UDTF function - it can yield tuples or scalar values (for single-field output) + result = udtf_func(*row_args) + + # Collect output rows for this input row + row_outputs = [] + if inspect.isgenerator(result): + for output_value in result: + if is_single_field: + # Single-field output: accept both scalar and tuple + if isinstance(output_value, tuple): + # User provided tuple (e.g., (value,)) - extract scalar + if len(output_value) != 1: + raise ValueError( + f"Single-field UDTF should yield 1-tuples or scalars, got {len(output_value)}-tuple" + ) + row_outputs.append(output_value[0]) # Extract scalar from tuple + else: + # User provided scalar - use directly + row_outputs.append(output_value) + else: + # Multi-field output: must be tuple + if not isinstance(output_value, tuple): + raise ValueError( + f"Multi-field UDTF must yield tuples, got {type(output_value)}" + ) + row_outputs.append(output_value) + elif result is not None: + # Function returned a single value instead of yielding + if is_single_field: + # Single-field: accept scalar or tuple + if isinstance(result, tuple): + if len(result) != 1: + raise ValueError( + f"Single-field UDTF should return 1-tuple or scalar, got {len(result)}-tuple" + ) + row_outputs.append(result[0]) # Extract scalar from tuple + else: + row_outputs.append(result) + else: + # Multi-field: must be tuple + if not isinstance(result, tuple): + raise ValueError( + f"Multi-field UDTF must return tuples, got {type(result)}" + ) + row_outputs.append(result) + + all_results.append(row_outputs) + + try: + list_array = pa.array(all_results, type=pa.list_(expected_output_type)) + except Exception as e: + logging.error( + "Failed to create ListArray: %s, element_type: %s", e, expected_output_type + ) + raise RuntimeError(f"Failed to create ListArray: {e}") from e + + # Create RecordBatch with single ListArray column + schema = pa.schema([pa.field("results", pa.list_(expected_output_type))]) + response_batch = pa.RecordBatch.from_arrays([list_array], schema=schema) + + return response_batch + def do_exchange( self, context: flight.ServerCallContext, @@ -1861,30 +2036,20 @@ def do_exchange( writer: flight.MetadataRecordBatchWriter, ) -> None: """ - Handle bidirectional streaming for both UDF and UDAF execution. + Handle bidirectional streaming for UDF, UDAF, and UDTF execution. Determines operation type (UDF vs UDAF vs UDTF) from descriptor metadata. """ - logging.info("Received exchange request: %s", descriptor) - python_udf_meta = self.parse_python_udf_meta(descriptor) if not python_udf_meta: raise ValueError("Invalid or missing metadata in descriptor") - # Route to appropriate handler based on client_type - logging.info( - "Handling %s operation for: %s", - python_udf_meta.client_type.name, - python_udf_meta.name, - ) - if python_udf_meta.is_udf(): - self._do_exchange_udf(python_udf_meta, reader, writer) + self._handle_exchange_udf(python_udf_meta, reader, writer) elif python_udf_meta.is_udaf(): - self._do_exchange_udaf(python_udf_meta, reader, writer) + self._handle_exchange_udaf(python_udf_meta, reader, writer) elif python_udf_meta.is_udtf(): - # TODO: Implement UDTF support - raise NotImplementedError("UDTF is not yet supported") + self._handle_exchange_udtf(python_udf_meta, reader, writer) else: raise ValueError(f"Unsupported client type: {python_udf_meta.client_type}") @@ -1939,11 +2104,13 @@ def main(unix_socket_path: str) -> None: current_pid = os.getpid() ServerState.unix_socket_path = f"{unix_socket_path}_{current_pid}.sock" - # Start unified server that handles both UDF and UDAF + # Start unified server that handles UDF, UDAF, and UDTF server = FlightServer(ServerState.unix_socket_path) print(ServerState.PYTHON_SERVER_START_SUCCESS_MSG, flush=True) - logging.info("##### PYTHON UDF/UDAF SERVER STARTED AT %s #####", datetime.now()) + logging.info( + "##### PYTHON UDF/UDAF/UDTF SERVER STARTED AT %s #####", datetime.now() + ) server.wait() except Exception as e: @@ -1959,8 +2126,8 @@ def main(unix_socket_path: str) -> None: if __name__ == "__main__": parser = argparse.ArgumentParser( - description="Run an Arrow Flight UDF/UDAF server over Unix socket. " - "The server handles both UDF and UDAF operations dynamically." + description="Run an Arrow Flight UDF/UDAF/UDTF server over Unix socket. " + "The server handles UDF, UDAF, and UDTF operations dynamically." ) parser.add_argument( "unix_socket_path", diff --git a/be/src/udf/python/python_udaf_client.cpp b/be/src/udf/python/python_udaf_client.cpp index 40bd8e3ba4fb6d..ab8a8c3e6af584 100644 --- a/be/src/udf/python/python_udaf_client.cpp +++ b/be/src/udf/python/python_udaf_client.cpp @@ -196,31 +196,6 @@ Status PythonUDAFClient::create(const PythonUDFMeta& func_meta, ProcessPtr proce return Status::OK(); } -Status PythonUDAFClient::init(const PythonUDFMeta& func_meta, ProcessPtr process) { - if (_inited) { - return Status::InternalError("PythonUDAFClient has already been initialized"); - } - - arrow::flight::Location location; - RETURN_DORIS_STATUS_IF_RESULT_ERROR(location, - arrow::flight::Location::Parse(process->get_uri())); - RETURN_DORIS_STATUS_IF_RESULT_ERROR(_arrow_client, FlightClient::Connect(location)); - - std::string command; - RETURN_IF_ERROR(func_meta.serialize_to_json(&command)); - - FlightDescriptor descriptor = FlightDescriptor::Command(command); - arrow::flight::FlightClient::DoExchangeResult exchange_res; - RETURN_DORIS_STATUS_IF_RESULT_ERROR(exchange_res, _arrow_client->DoExchange(descriptor)); - - _reader = std::move(exchange_res.reader); - _writer = std::move(exchange_res.writer); - _process = std::move(process); - _inited = true; - - return Status::OK(); -} - Status PythonUDAFClient::create(int64_t place_id) { RETURN_IF_ERROR( (_execute_operation(place_id, nullptr, nullptr, nullptr))); @@ -232,7 +207,7 @@ Status PythonUDAFClient::accumulate(int64_t place_id, bool is_single_place, const arrow::RecordBatch& input, int64_t row_start, int64_t row_end, const int64_t* places, int64_t place_offset) { if (UNLIKELY(!_process->is_alive())) { - return Status::RuntimeError("Python UDAF process is not alive"); + return Status::RuntimeError("{} process is not alive", _operation_name); } // Validate input parameters @@ -394,7 +369,7 @@ Status PythonUDAFClient::reset(int64_t place_id) { Status PythonUDAFClient::destroy(int64_t place_id) { if (UNLIKELY(!_process->is_alive())) { - return Status::RuntimeError("Python UDAF process is not alive"); + return Status::RuntimeError("{} process is not alive", _operation_name); } // Create unified batch for DESTROY operation @@ -438,22 +413,8 @@ Status PythonUDAFClient::close() { // Destroy all remaining states RETURN_IF_ERROR(destroy_all()); - auto writer_res = _writer->Close(); - if (!writer_res.ok()) { - return _handle_error(writer_res); - } - - _inited = false; - _begin = false; - _arrow_client.reset(); - _writer.reset(); - _reader.reset(); - - if (auto* pool = _process->pool(); pool) { - pool->return_process(std::move(_process)); - } - - return Status::OK(); + // Call base class close which handles cleanup + return PythonClient::close(); } std::string PythonUDAFClient::print_operation(UDAFOperation op) { @@ -483,7 +444,7 @@ Status PythonUDAFClient::_execute_operation(int64_t place_id, const std::shared_ptr& data, std::shared_ptr* output) { if (UNLIKELY(!_process->is_alive())) { - return Status::RuntimeError("Python UDAF process is not alive"); + return Status::RuntimeError("{} process is not alive", _operation_name); } // Create unified batch for the operation @@ -529,7 +490,7 @@ Status PythonUDAFClient::_send_operation(const arrow::RecordBatch* input, // Always use the unified schema for all operations auto begin_res = _writer->Begin(kUnifiedUDAFSchema); if (!begin_res.ok()) { - return _handle_error(begin_res); + return handle_error(begin_res); } _begin = true; } @@ -537,20 +498,21 @@ Status PythonUDAFClient::_send_operation(const arrow::RecordBatch* input, // Step 2: Write the record batch to server auto write_res = _writer->WriteRecordBatch(*input); if (!write_res.ok()) { - return _handle_error(write_res); + return handle_error(write_res); } // Step 3: Read response from server (if output is expected) if (output != nullptr) { auto read_res = _reader->Next(); if (!read_res.ok()) { - return _handle_error(read_res.status()); + return handle_error(read_res.status()); } arrow::flight::FlightStreamChunk chunk = std::move(*read_res); if (!chunk.data) { _process->shutdown(); - return Status::InternalError("Received empty RecordBatch from Python UDAF server"); + return Status::InternalError("Received empty RecordBatch from {} server", + _operation_name); } // The response is in unified format: [result_data: binary] @@ -581,19 +543,4 @@ Status PythonUDAFClient::_send_operation(const arrow::RecordBatch* input, return Status::OK(); } -Status PythonUDAFClient::_handle_error(arrow::Status status) { - DCHECK(!status.ok()); - _writer.reset(); - _reader.reset(); - _process->shutdown(); - - std::string msg = status.message(); - // Remove Python traceback noise - size_t pos = msg.find("The above exception was the direct cause"); - if (pos != std::string::npos) { - msg = msg.substr(0, pos); - } - return Status::RuntimeError(trim(msg)); -} - } // namespace doris diff --git a/be/src/udf/python/python_udaf_client.h b/be/src/udf/python/python_udaf_client.h index 91c9d0a610b340..170463c1f23291 100644 --- a/be/src/udf/python/python_udaf_client.h +++ b/be/src/udf/python/python_udaf_client.h @@ -19,14 +19,7 @@ #include -#include -#include - -#include "arrow/flight/client.h" -#include "common/status.h" -#include "udf/python/python_udf_meta.h" -#include "udf/python/python_udf_runtime.h" -#include "util/arrow/utils.h" +#include "udf/python/python_client.h" namespace doris { @@ -53,13 +46,8 @@ using PythonUDAFClientPtr = std::shared_ptr; * 6. RESET: Reset state to initial value * 7. DESTROY: Clean up resources */ -class PythonUDAFClient { +class PythonUDAFClient : public PythonClient { public: - using FlightDescriptor = arrow::flight::FlightDescriptor; - using FlightClient = arrow::flight::FlightClient; - using FlightStreamWriter = arrow::flight::FlightStreamWriter; - using FlightStreamReader = arrow::flight::FlightStreamReader; - // UDAF operation types enum class UDAFOperation : uint8_t { CREATE = 0, // Create new aggregate state @@ -72,13 +60,11 @@ class PythonUDAFClient { }; PythonUDAFClient() = default; - ~PythonUDAFClient() = default; + ~PythonUDAFClient() override = default; static Status create(const PythonUDFMeta& func_meta, ProcessPtr process, PythonUDAFClientPtr* client); - Status init(const PythonUDFMeta& func_meta, ProcessPtr process); - /** * Create aggregate state for a place * @param place_id Unique identifier for the aggregate state @@ -145,17 +131,18 @@ class PythonUDAFClient { */ Status destroy_all(); + /** + * Close client connection and cleanup + * Overrides base class to destroy all states first + * @return Status + */ Status close(); - std::string print_process() const { return _process->to_string(); } - static std::string print_operation(UDAFOperation op); private: DISALLOW_COPY_AND_ASSIGN(PythonUDAFClient); - Status _handle_error(arrow::Status status); - /** * Helper to execute a UDAF operation (CREATE, RESET, DESTROY, etc.) * This consolidates the common pattern: @@ -187,13 +174,6 @@ class PythonUDAFClient { Status _send_operation(const arrow::RecordBatch* input, std::shared_ptr* output); - bool _inited = false; - bool _begin = false; // Track if Begin() has been called - std::unique_ptr _arrow_client; - std::unique_ptr _writer; - std::unique_ptr _reader; - ProcessPtr _process; - // Track created states for cleanup std::unordered_set _created_states; diff --git a/be/src/udf/python/python_udf_client.cpp b/be/src/udf/python/python_udf_client.cpp index 8164d9ff7e7a23..8e7aa38b33420e 100644 --- a/be/src/udf/python/python_udf_client.cpp +++ b/be/src/udf/python/python_udf_client.cpp @@ -17,14 +17,7 @@ #include "udf/python/python_udf_client.h" -#include - -#include "arrow/flight/client.h" -#include "arrow/flight/server.h" #include "common/status.h" -#include "udf/python/python_udf_meta.h" -#include "udf/python/python_udf_runtime.h" -#include "util/arrow/utils.h" namespace doris { @@ -36,89 +29,12 @@ Status PythonUDFClient::create(const PythonUDFMeta& func_meta, ProcessPtr proces return Status::OK(); } -Status PythonUDFClient::init(const PythonUDFMeta& func_meta, ProcessPtr process) { - if (_inited) { - return Status::InternalError("PythonUDFClient has already been initialized"); - } - arrow::flight::Location location; - RETURN_DORIS_STATUS_IF_RESULT_ERROR(location, - arrow::flight::Location::Parse(process->get_uri())); - RETURN_DORIS_STATUS_IF_RESULT_ERROR(_arrow_client, FlightClient::Connect(location)); - std::string command; - RETURN_IF_ERROR(func_meta.serialize_to_json(&command)); - FlightDescriptor descriptor = FlightDescriptor::Command(command); - arrow::flight::FlightClient::DoExchangeResult exchange_res; - RETURN_DORIS_STATUS_IF_RESULT_ERROR(exchange_res, _arrow_client->DoExchange(descriptor)); - _reader = std::move(exchange_res.reader); - _writer = std::move(exchange_res.writer); - _process = std::move(process); - _inited = true; - return Status::OK(); -} - Status PythonUDFClient::evaluate(const arrow::RecordBatch& input, std::shared_ptr* output) { - if (!_process->is_alive()) { - return Status::RuntimeError("Python UDF process is not alive"); - } - - // Step 1: Begin exchange with schema (only once) - if (UNLIKELY(!_begin)) { - auto begin_res = _writer->Begin(input.schema()); - if (!begin_res.ok()) { - return handle_error(begin_res); - } - _begin = true; - } - - // Step 2: Write the record batch to server - auto write_res = _writer->WriteRecordBatch(input); - if (!write_res.ok()) { - return handle_error(write_res); - } - - // Step 3: Read response from server - auto read_res = _reader->Next(); - if (!read_res.ok()) { - return handle_error(read_res.status()); - } - - arrow::flight::FlightStreamChunk chunk = std::move(*read_res); - if (!chunk.data) { - _process->shutdown(); - return Status::InternalError("Received empty RecordBatch from Python UDF server"); - } - *output = std::move(chunk.data); - return Status::OK(); -} - -Status PythonUDFClient::handle_error(arrow::Status status) { - DCHECK(!status.ok()); - _writer.reset(); - _reader.reset(); - _process->shutdown(); - std::string msg = status.message(); - size_t pos = msg.find("The above exception was the direct cause"); - if (pos != std::string::npos) { - msg = msg.substr(0, pos); - } - return Status::RuntimeError(trim(msg)); -} - -Status PythonUDFClient::close() { - if (!_inited || !_writer) return Status::OK(); - auto writer_res = _writer->Close(); - if (!writer_res.ok()) { - return handle_error(writer_res); - } - _inited = false; - _begin = false; - _arrow_client.reset(); - _writer.reset(); - _reader.reset(); - if (auto* pool = _process->pool(); pool) { - pool->return_process(std::move(_process)); - } + RETURN_IF_ERROR(check_process_alive()); + RETURN_IF_ERROR(begin_stream(input.schema())); + RETURN_IF_ERROR(write_batch(input)); + RETURN_IF_ERROR(read_batch(output)); return Status::OK(); } diff --git a/be/src/udf/python/python_udf_client.h b/be/src/udf/python/python_udf_client.h index 9d88b79d31d9b0..969dd8d02d3cbd 100644 --- a/be/src/udf/python/python_udf_client.h +++ b/be/src/udf/python/python_udf_client.h @@ -19,52 +19,57 @@ #include -#include "arrow/flight/client.h" -#include "common/status.h" -#include "udf/python/python_udf_meta.h" -#include "udf/python/python_udf_runtime.h" -#include "util/arrow/utils.h" +#include "udf/python/python_client.h" namespace doris { class PythonUDFClient; -class PythonUDFProcessPool; using PythonUDFClientPtr = std::shared_ptr; -class PythonUDFClient { +/** + * Python UDF Client + * + * Implements standard UDF (User-Defined Function) pattern with a single evaluation function: + * - evaluate_func(*args): Process input arguments and return result + * + * UDF Characteristics: + * - Takes scalar or column inputs + * - Returns scalar or column outputs + * - Stateless evaluation (each call is independent) + * - Simple input-output transformation + * + * Example: + * ```python + * def evaluate_func(x, y): + * # Add two numbers + * return x + y + * ``` + * + * Communication protocol with Python server: + * 1. Send input batch (RecordBatch with N rows) + * 2. Python calls evaluate_func() for each row (or vectorized) + * 3. Receive output batch (RecordBatch with N rows) + */ +class PythonUDFClient : public PythonClient { public: - using FlightDescriptor = arrow::flight::FlightDescriptor; - using FlightClient = arrow::flight::FlightClient; - using FlightStreamWriter = arrow::flight::FlightStreamWriter; - using FlightStreamReader = arrow::flight::FlightStreamReader; - PythonUDFClient() = default; - - ~PythonUDFClient() = default; + ~PythonUDFClient() override = default; static Status create(const PythonUDFMeta& func_meta, ProcessPtr process, PythonUDFClientPtr* client); - Status init(const PythonUDFMeta& func_meta, ProcessPtr process); - + /** + * Evaluate UDF on input rows + * + * @param input Input row batch (columns = UDF function parameters) + * @param output Output row batch (single column = UDF return value) + * @return Status + */ Status evaluate(const arrow::RecordBatch& input, std::shared_ptr* output); - Status close(); - - Status handle_error(arrow::Status status); - - std::string print_process() const { return _process->to_string(); } - private: DISALLOW_COPY_AND_ASSIGN(PythonUDFClient); - - bool _inited = false; - bool _begin = false; - std::unique_ptr _arrow_client; - std::unique_ptr _writer; - std::unique_ptr _reader; - ProcessPtr _process; }; } // namespace doris \ No newline at end of file diff --git a/be/src/udf/python/python_udf_runtime.cpp b/be/src/udf/python/python_udf_runtime.cpp index 3683fc478f64f1..49058e7cea676b 100644 --- a/be/src/udf/python/python_udf_runtime.cpp +++ b/be/src/udf/python/python_udf_runtime.cpp @@ -26,7 +26,7 @@ #include "common/config.h" #include "common/logging.h" #include "common/status.h" -#include "udf/python/python_udf_server.h" +#include "udf/python/python_server.h" namespace doris { @@ -95,7 +95,7 @@ Status PythonUDFProcessPool::init() { std::lock_guard lock(_mtx); for (size_t i = 0; i < _init_pool_size; ++i) { ProcessPtr process; - RETURN_IF_ERROR(PythonUDFServerManager::instance().fork(this, &process)); + RETURN_IF_ERROR(PythonServerManager::instance().fork(this, &process)); _idle_processes.push(std::move(process)); ++_current_size; } @@ -121,7 +121,7 @@ Status PythonUDFProcessPool::borrow_process(ProcessPtr* process) { // If we can create a new process, do it if (_current_size < _max_pool_size) { - RETURN_IF_ERROR(PythonUDFServerManager::instance().fork(this, process)); + RETURN_IF_ERROR(PythonServerManager::instance().fork(this, process)); ++_current_size; return Status::OK(); } diff --git a/be/src/udf/python/python_udf_runtime.h b/be/src/udf/python/python_udf_runtime.h index b58ef66785e4ef..3cbb7ab0734a43 100644 --- a/be/src/udf/python/python_udf_runtime.h +++ b/be/src/udf/python/python_udf_runtime.h @@ -30,7 +30,7 @@ static const char* UNIX_SOCKET_PREFIX = "grpc+unix://"; static const char* BASE_UNIX_SOCKET_PATH_TEMPLATE = "{}{}/lib/udf/python/python_udf"; static const char* UNIX_SOCKET_PATH_TEMPLATE = "{}_{}.sock"; static const char* FLIGHT_SERVER_PATH_TEMPLATE = "{}/plugins/python_udf/{}"; -static const char* FLIGHT_SERVER_FILENAME = "python_udf_server.py"; +static const char* FLIGHT_SERVER_FILENAME = "python_server.py"; static const char* EXECUTABLE_PYTHON_FILENAME = "python"; static const char* PYTHON_SERVER_START_SUCCESS_MSG = "Start python server successfully"; diff --git a/be/src/udf/python/python_udtf_client.cpp b/be/src/udf/python/python_udtf_client.cpp new file mode 100644 index 00000000000000..15a13b6a279bed --- /dev/null +++ b/be/src/udf/python/python_udtf_client.cpp @@ -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. + +#include "udf/python/python_udtf_client.h" + +#include "arrow/array/array_nested.h" +#include "arrow/array/array_primitive.h" +#include "arrow/record_batch.h" +#include "arrow/type.h" +#include "common/status.h" + +namespace doris { + +Status PythonUDTFClient::create(const PythonUDFMeta& func_meta, ProcessPtr process, + PythonUDTFClientPtr* client) { + PythonUDTFClientPtr python_udtf_client = std::make_shared(); + RETURN_IF_ERROR(python_udtf_client->init(func_meta, std::move(process))); + *client = std::move(python_udtf_client); + return Status::OK(); +} + +Status PythonUDTFClient::evaluate(const arrow::RecordBatch& input, + std::shared_ptr* list_array) { + RETURN_IF_ERROR(check_process_alive()); + RETURN_IF_ERROR(begin_stream(input.schema())); + RETURN_IF_ERROR(write_batch(input)); + + // Read the response (ListArray-based) + std::shared_ptr response_batch; + RETURN_IF_ERROR(read_batch(&response_batch)); + + // Validate response structure: should have a single ListArray column + if (response_batch->num_columns() != 1) { + return Status::InternalError( + fmt::format("Invalid UDTF response: expected 1 column (ListArray), got {}", + response_batch->num_columns())); + } + + auto list_array_ptr = response_batch->column(0); + if (list_array_ptr->type_id() != arrow::Type::LIST) { + return Status::InternalError( + fmt::format("Invalid UDTF response: expected ListArray, got type {}", + list_array_ptr->type()->ToString())); + } + + *list_array = std::static_pointer_cast(list_array_ptr); + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/udf/python/python_udtf_client.h b/be/src/udf/python/python_udtf_client.h new file mode 100644 index 00000000000000..35ae29d1b7ae4d --- /dev/null +++ b/be/src/udf/python/python_udtf_client.h @@ -0,0 +1,89 @@ +// 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. + +#pragma once + +#include + +#include "udf/python/python_client.h" + +namespace doris { + +class PythonUDTFClient; + +using PythonUDTFClientPtr = std::shared_ptr; + +/** + * Python UDTF Client + * + * Implements simplified UDTF (User-Defined Table Function): + * + * Handler Function: + * - evaluate_func(*args): Process input arguments and yield output rows + * + * UDTF Characteristics: + * - Takes scalar or table inputs + * - Returns table (multiple rows) + * - Simple yield pattern + * + * Example: + * ```python + * def evaluate_func(text, delimiter): + * # Split string by delimiter and return multiple results + * for item in text.split(delimiter): + * # or yield (item, ) + * yield item + * ``` + * + * Communication protocol with Python server: + * 1. Send input row batch to Python + * 2. Python calls evaluate_func() for each input row + * 3. Collect all output rows and return + */ +class PythonUDTFClient : public PythonClient { +public: + PythonUDTFClient() = default; + ~PythonUDTFClient() override = default; + + static Status create(const PythonUDFMeta& func_meta, ProcessPtr process, + PythonUDTFClientPtr* client); + + /** + * Evaluate UDTF on input rows + * + * Protocol (ListArray-based): + * Python server returns a RecordBatch with 1 column: + * - Column 0: ListArray where each list element corresponds to one input row's outputs + * + * Example: + * Input: 3 rows + * Output ListArray: + * [0]: [val1, val2, val3] (3 elements for input row 0) + * [1]: [] (0 elements for input row 1) + * [2]: [val4, val5, val6, val7] (4 elements for input row 2) + * + * @param input Input row batch (columns = UDTF function parameters) + * @param list_array Output ListArray (length = num_input_rows) + * @return Status + */ + Status evaluate(const arrow::RecordBatch& input, std::shared_ptr* list_array); + +private: + DISALLOW_COPY_AND_ASSIGN(PythonUDTFClient); +}; + +} // namespace doris diff --git a/be/src/vec/aggregate_functions/aggregate_function_python_udaf.cpp b/be/src/vec/aggregate_functions/aggregate_function_python_udaf.cpp index ede72e97633990..da90e0d521ab1b 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_python_udaf.cpp +++ b/be/src/vec/aggregate_functions/aggregate_function_python_udaf.cpp @@ -28,7 +28,7 @@ #include "runtime/user_function_cache.h" #include "udf/python/python_env.h" #include "udf/python/python_udf_runtime.h" -#include "udf/python/python_udf_server.h" +#include "udf/python/python_server.h" #include "util/arrow/block_convertor.h" #include "util/arrow/row_batch.h" #include "util/timezone_utils.h" @@ -342,8 +342,8 @@ Status AggregatePythonUDAF::_init_shared_client() const { return Status::OK(); } - RETURN_IF_ERROR(PythonUDFServerManager::instance().get_client(_func_meta, _python_version, - &_shared_client)); + RETURN_IF_ERROR(PythonServerManager::instance().get_client(_func_meta, _python_version, + &_shared_client)); LOG(INFO) << "Initialized shared Python UDAF client for function: " << _func_meta.name; return Status::OK(); } diff --git a/be/src/vec/exprs/table_function/python_udtf_function.cpp b/be/src/vec/exprs/table_function/python_udtf_function.cpp new file mode 100644 index 00000000000000..60a08bf3aded7f --- /dev/null +++ b/be/src/vec/exprs/table_function/python_udtf_function.cpp @@ -0,0 +1,283 @@ +// 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. + +#include "vec/exprs/table_function/python_udtf_function.h" + +#include +#include +#include +#include +#include + +#include "runtime/runtime_state.h" +#include "runtime/user_function_cache.h" +#include "udf/python/python_env.h" +#include "udf/python/python_udf_meta.h" +#include "udf/python/python_server.h" +#include "util/arrow/block_convertor.h" +#include "util/arrow/row_batch.h" +#include "util/arrow/utils.h" +#include "util/timezone_utils.h" +#include "vec/columns/column.h" +#include "vec/columns/column_array.h" +#include "vec/columns/column_nullable.h" +#include "vec/common/assert_cast.h" +#include "vec/core/block.h" +#include "vec/core/column_numbers.h" +#include "vec/data_types/data_type_array.h" +#include "vec/data_types/data_type_factory.hpp" +#include "vec/data_types/serde/data_type_array_serde.h" +#include "vec/exprs/vexpr.h" +#include "vec/exprs/vexpr_context.h" +#include "vec/functions/array/function_array_utils.h" + +namespace doris::vectorized { +#include "common/compile_check_begin.h" + +PythonUDTFFunction::PythonUDTFFunction(const TFunction& t_fn) : TableFunction(), _t_fn(t_fn) { + _fn_name = _t_fn.name.function_name; + TimezoneUtils::find_cctz_time_zone(TimezoneUtils::default_time_zone, _timezone_obj); + + // Like Java UDTF, FE passes the element type T, and we wrap it into array here + // This makes the behavior consistent with Java UDTF + DataTypePtr element_type = DataTypeFactory::instance().create_data_type(t_fn.ret_type); + _return_type = make_nullable(std::make_shared(make_nullable(element_type))); +} + +Status PythonUDTFFunction::open() { + PythonUDFMeta python_udf_meta; + python_udf_meta.id = _t_fn.id; + python_udf_meta.name = _t_fn.name.function_name; + python_udf_meta.symbol = _t_fn.scalar_fn.symbol; + + if (!_t_fn.function_code.empty()) { + python_udf_meta.type = PythonUDFLoadType::INLINE; + python_udf_meta.location = "inline"; + python_udf_meta.inline_code = _t_fn.function_code; + } else if (!_t_fn.hdfs_location.empty()) { + python_udf_meta.type = PythonUDFLoadType::MODULE; + python_udf_meta.location = _t_fn.hdfs_location; + python_udf_meta.checksum = _t_fn.checksum; + } else { + python_udf_meta.type = PythonUDFLoadType::UNKNOWN; + python_udf_meta.location = "unknown"; + } + + python_udf_meta.client_type = PythonClientType::UDTF; + + if (python_udf_meta.type == PythonUDFLoadType::MODULE) { + RETURN_IF_ERROR(UserFunctionCache::instance()->get_pypath( + python_udf_meta.id, python_udf_meta.location, python_udf_meta.checksum, + &python_udf_meta.location)); + } + + PythonVersion version; + if (_t_fn.__isset.runtime_version && !_t_fn.runtime_version.empty()) { + RETURN_IF_ERROR( + PythonVersionManager::instance().get_version(_t_fn.runtime_version, &version)); + python_udf_meta.runtime_version = version.full_version; + } else { + return Status::InvalidArgument("Python UDTF runtime version is not set"); + } + + for (const auto& arg_type : _t_fn.arg_types) { + DataTypePtr doris_type = DataTypeFactory::instance().create_data_type(arg_type); + python_udf_meta.input_types.push_back(doris_type); + } + + // For Python UDTF, FE passes the element type T (like Java UDTF) + // Use it directly as the UDF's return type for Python metadata + python_udf_meta.return_type = DataTypeFactory::instance().create_data_type(_t_fn.ret_type); + python_udf_meta.always_nullable = python_udf_meta.return_type->is_nullable(); + RETURN_IF_ERROR(python_udf_meta.check()); + + RETURN_IF_ERROR( + PythonServerManager::instance().get_client(python_udf_meta, version, &_udtf_client)); + + if (!_udtf_client) { + return Status::InternalError("Failed to create Python UDTF client"); + } + + return Status::OK(); +} + +Status PythonUDTFFunction::process_init(Block* block, RuntimeState* state) { + // Step 1: Extract input columns from child expressions + auto child_size = _expr_context->root()->children().size(); + ColumnNumbers child_column_idxs; + child_column_idxs.resize(child_size); + for (int i = 0; i < child_size; ++i) { + int result_id = -1; + RETURN_IF_ERROR(_expr_context->root()->children()[i]->execute(_expr_context.get(), block, + &result_id)); + DCHECK_NE(result_id, -1); + child_column_idxs[i] = result_id; + } + + // Step 2: Build input block and convert to Arrow format + vectorized::Block input_block; + for (uint32_t i = 0; i < child_column_idxs.size(); ++i) { + input_block.insert(block->get_by_position(child_column_idxs[i])); + } + std::shared_ptr input_schema; + std::shared_ptr input_batch; + RETURN_IF_ERROR(get_arrow_schema_from_block(input_block, &input_schema, + TimezoneUtils::default_time_zone)); + RETURN_IF_ERROR(convert_to_arrow_batch(input_block, input_schema, arrow::default_memory_pool(), + &input_batch, _timezone_obj)); + + // Step 3: Call Python UDTF to evaluate all rows at once (similar to Java UDTF's JNI call) + // Python returns a ListArray where each element contains outputs for one input row + std::shared_ptr list_array; + RETURN_IF_ERROR(_udtf_client->evaluate(*input_batch, &list_array)); + + // Step 4: Convert Python server output (ListArray) to Doris array column + RETURN_IF_ERROR(_convert_list_array_to_array_column(list_array)); + + // Step 5: Extract array column metadata using extract_column_array_info + if (!extract_column_array_info(*_array_result_column, _array_column_detail)) { + return Status::NotSupported("column type {} not supported now", + _array_result_column->get_name()); + } + + return Status::OK(); +} + +void PythonUDTFFunction::process_row(size_t row_idx) { + TableFunction::process_row(row_idx); + + // Check if array is null for this row + if (!_array_column_detail.array_nullmap_data || + !_array_column_detail.array_nullmap_data[row_idx]) { + _array_offset = (*_array_column_detail.offsets_ptr)[row_idx - 1]; + _cur_size = (*_array_column_detail.offsets_ptr)[row_idx] - _array_offset; + } + // When it's NULL at row_idx, _cur_size stays 0, meaning current_empty() + // If outer function: will continue with insert_default + // If not outer function: will not insert any value +} + +void PythonUDTFFunction::process_close() { + _array_result_column = nullptr; + _array_column_detail.reset(); + _array_offset = 0; +} + +void PythonUDTFFunction::get_same_many_values(MutableColumnPtr& column, int length) { + size_t pos = _array_offset + _cur_offset; + if (current_empty() || (_array_column_detail.nested_nullmap_data && + _array_column_detail.nested_nullmap_data[pos])) { + column->insert_many_defaults(length); + } else { + if (_is_nullable) { + auto* nullable_column = assert_cast(column.get()); + auto nested_column = nullable_column->get_nested_column_ptr(); + auto nullmap_column = nullable_column->get_null_map_column_ptr(); + nested_column->insert_many_from(*_array_column_detail.nested_col, pos, length); + assert_cast(nullmap_column.get())->insert_many_defaults(length); + } else { + column->insert_many_from(*_array_column_detail.nested_col, pos, length); + } + } +} + +int PythonUDTFFunction::get_value(MutableColumnPtr& column, int max_step) { + max_step = std::min(max_step, (int)(_cur_size - _cur_offset)); + size_t pos = _array_offset + _cur_offset; + + if (current_empty()) { + column->insert_default(); + max_step = 1; + } else { + if (_is_nullable) { + auto* nullable_column = assert_cast(column.get()); + auto nested_column = nullable_column->get_nested_column_ptr(); + auto* nullmap_column = + assert_cast(nullable_column->get_null_map_column_ptr().get()); + + nested_column->insert_range_from(*_array_column_detail.nested_col, pos, max_step); + size_t old_size = nullmap_column->size(); + nullmap_column->resize(old_size + max_step); + memcpy(nullmap_column->get_data().data() + old_size, + _array_column_detail.nested_nullmap_data + pos * sizeof(UInt8), + max_step * sizeof(UInt8)); + } else { + column->insert_range_from(*_array_column_detail.nested_col, pos, max_step); + } + } + forward(max_step); + return max_step; +} + +Status PythonUDTFFunction::close() { + // Close UDTF client + if (_udtf_client) { + Status status = _udtf_client->close(); + if (!status.ok()) { + LOG(WARNING) << "Failed to close UDTF client: " << status.to_string(); + } + _udtf_client.reset(); + } + + return TableFunction::close(); +} + +Status PythonUDTFFunction::_convert_list_array_to_array_column( + const std::shared_ptr& list_array) { + if (!list_array) { + return Status::InternalError("Received null ListArray from Python UDTF"); + } + + size_t num_input_rows = list_array->length(); + + // Handle nullable array column + MutableColumnPtr array_col_ptr = _return_type->create_column(); + ColumnNullable* nullable_col = nullptr; + ColumnArray* array_col = nullptr; + + if (_return_type->is_nullable()) { + nullable_col = assert_cast(array_col_ptr.get()); + array_col = assert_cast( + nullable_col->get_nested_column_ptr()->assume_mutable().get()); + } else { + array_col = assert_cast(array_col_ptr.get()); + } + + // Create DataTypeArraySerDe for direct Arrow conversion + DataTypePtr element_type = DataTypeFactory::instance().create_data_type(_t_fn.ret_type); + DataTypePtr array_type = std::make_shared(make_nullable(element_type)); + auto array_serde = array_type->get_serde(); + + // Use read_column_from_arrow for optimized conversion + // This directly converts Arrow ListArray to Doris ColumnArray + // No struct unwrapping needed - Python server sends the correct format! + RETURN_IF_ERROR(array_serde->read_column_from_arrow( + array_col->assume_mutable_ref(), list_array.get(), 0, num_input_rows, _timezone_obj)); + + // Handle nullable wrapper: all array elements are non-null + // (empty arrays [] are non-null, different from NULL) + if (nullable_col) { + auto& null_map = nullable_col->get_null_map_data(); + null_map.resize_fill(num_input_rows, 0); // All non-null + } + + _array_result_column = std::move(array_col_ptr); + return Status::OK(); +} + +#include "common/compile_check_end.h" +} // namespace doris::vectorized diff --git a/be/src/vec/exprs/table_function/python_udtf_function.h b/be/src/vec/exprs/table_function/python_udtf_function.h new file mode 100644 index 00000000000000..c4a62aa9864948 --- /dev/null +++ b/be/src/vec/exprs/table_function/python_udtf_function.h @@ -0,0 +1,97 @@ +// 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. + +#pragma once + +#include +#include + +#include "common/status.h" +#include "udf/python/python_udtf_client.h" +#include "vec/columns/column.h" +#include "vec/data_types/data_type.h" +#include "vec/exprs/table_function/table_function.h" +#include "vec/functions/array/function_array_utils.h" + +namespace doris::vectorized { +#include "common/compile_check_begin.h" + +/** + * PythonUDTFFunction - Python User-Defined Table Function + * + * Execution Flow: + * 1. open() - Create Python UDTF client and establish RPC connection + * 2. process_init(block) - Batch evaluate all rows: + * - Convert input block to Arrow RecordBatch + * - Call Python UDTF server via RPC (evaluates all rows in one call) + * - Receive Arrow ListArray (one list per input row) + * - Convert to array column using DataTypeArraySerDe + * 3. process_row(row_idx) - Set array offset for current row + * 4. get_value()/get_same_many_values() - Extract values from array column + * 5. process_close() - Clean up batch state (array column, offsets) + * 6. close() - Close Python UDTF client and RPC connection + */ +class PythonUDTFFunction final : public TableFunction { + ENABLE_FACTORY_CREATOR(PythonUDTFFunction); + +public: + PythonUDTFFunction(const TFunction& t_fn); + ~PythonUDTFFunction() override = default; + + Status open() override; + Status process_init(Block* block, RuntimeState* state) override; + void process_row(size_t row_idx) override; + void process_close() override; + void get_same_many_values(MutableColumnPtr& column, int length) override; + int get_value(MutableColumnPtr& column, int max_step) override; + Status close() override; + +private: + /** + * Convert Python UDTF output (Arrow ListArray) to Doris array column + * + * Input from Python server (via Arrow RPC): + * - list_array: Arrow ListArray where each element corresponds to one input row's outputs + * + * Format: + * - Single-column output: List (e.g., List, List) + * - Multi-column output: List> + * + * Example: 3 input rows producing variable output rows + * ListArray structure: + * [0]: [val1, val2, val3] (3 elements) + * [1]: [] (0 elements - empty array) + * [2]: [val4, val5, val6, val7] (4 elements) + * + * @param list_array Arrow ListArray containing UDTF output (length = num_input_rows) + * @return Status indicating success or validation/conversion errors + */ + Status _convert_list_array_to_array_column(const std::shared_ptr& list_array); + + const TFunction& _t_fn; + DataTypePtr _return_type; + PythonUDTFClientPtr _udtf_client; + cctz::time_zone _timezone_obj; + + // Result storage (similar to Java UDTF) + ColumnPtr _array_result_column; // Array column storing all results + ColumnArrayExecutionData _array_column_detail; // Array metadata for efficient access + int64_t _array_offset = 0; // Offset into array for current row +}; + +#include "common/compile_check_end.h" +} // namespace doris::vectorized diff --git a/be/src/vec/exprs/table_function/table_function_factory.cpp b/be/src/vec/exprs/table_function/table_function_factory.cpp index 9b4aaa29ea763b..0990b7754ed385 100644 --- a/be/src/vec/exprs/table_function/table_function_factory.cpp +++ b/be/src/vec/exprs/table_function/table_function_factory.cpp @@ -24,6 +24,7 @@ #include "agent/be_exec_version_manager.h" #include "common/object_pool.h" +#include "vec/exprs/table_function/python_udtf_function.h" #include "vec/exprs/table_function/table_function.h" #include "vec/exprs/table_function/udf_table_function.h" #include "vec/exprs/table_function/vexplode.h" @@ -66,6 +67,12 @@ Status TableFunctionFactory::get_fn(const TFunction& t_fn, ObjectPool* pool, Tab (*fn)->set_outer(); } return Status::OK(); + } else if (t_fn.binary_type == TFunctionBinaryType::PYTHON_UDF) { + *fn = pool->add(PythonUDTFFunction::create_unique(t_fn).release()); + if (is_outer) { + (*fn)->set_outer(); + } + return Status::OK(); } else { const std::string& fn_name_raw = t_fn.name.function_name; const std::string& fn_name_real = diff --git a/be/src/vec/exprs/vectorized_fn_call.cpp b/be/src/vec/exprs/vectorized_fn_call.cpp index 553cb768ad20be..8dd4d69fd3d9d8 100644 --- a/be/src/vec/exprs/vectorized_fn_call.cpp +++ b/be/src/vec/exprs/vectorized_fn_call.cpp @@ -119,10 +119,18 @@ Status VectorizedFnCall::prepare(RuntimeState* state, const RowDescriptor& desc, } } else if (_fn.binary_type == TFunctionBinaryType::PYTHON_UDF) { if (config::enable_python_udf_support) { - _function = PythonFunctionCall::create(_fn, argument_template, _data_type); - LOG(INFO) << fmt::format( - "create python function call: {}, runtime version: {}, function code: {}", - _fn.name.function_name, _fn.runtime_version, _fn.function_code); + if (_fn.is_udtf_function) { + // fake function. it's no use and can't execute. + // Python UDTF is executed via PythonUDTFFunction in table function path + auto builder = + std::make_shared(FunctionFake::create()); + _function = builder->build(argument_template, std::make_shared()); + } else { + _function = PythonFunctionCall::create(_fn, argument_template, _data_type); + LOG(INFO) << fmt::format( + "create python function call: {}, runtime version: {}, function code: {}", + _fn.name.function_name, _fn.runtime_version, _fn.function_code); + } } else { return Status::InternalError( "Python UDF is not enabled, you can change be config enable_python_udf_support " diff --git a/be/src/vec/functions/function_python_udf.cpp b/be/src/vec/functions/function_python_udf.cpp index 4bb6f259a9d3d9..b25eca26452216 100644 --- a/be/src/vec/functions/function_python_udf.cpp +++ b/be/src/vec/functions/function_python_udf.cpp @@ -30,7 +30,7 @@ #include "common/status.h" #include "runtime/user_function_cache.h" #include "udf/python/python_udf_meta.h" -#include "udf/python/python_udf_server.h" +#include "udf/python/python_server.h" #include "util/arrow/block_convertor.h" #include "util/arrow/row_batch.h" #include "util/timezone_utils.h" @@ -91,7 +91,7 @@ Status PythonFunctionCall::open(FunctionContext* context, } PythonUDFClientPtr client = nullptr; - RETURN_IF_ERROR(PythonUDFServerManager::instance().get_client(func_meta, version, &client)); + RETURN_IF_ERROR(PythonServerManager::instance().get_client(func_meta, version, &client)); if (!client) { return Status::InternalError("Python UDF client is null"); diff --git a/build.sh b/build.sh index 516fc2a6614f10..a7def39472023b 100755 --- a/build.sh +++ b/build.sh @@ -941,7 +941,7 @@ EOF mkdir -p "${DORIS_OUTPUT}/be/plugins/connectors/" mkdir -p "${DORIS_OUTPUT}/be/plugins/hadoop_conf/" mkdir -p "${DORIS_OUTPUT}/be/plugins/java_extensions/" - cp -r -p "${DORIS_HOME}/be/src/udf/python/python_udf_server.py" "${DORIS_OUTPUT}/be/plugins/python_udf/" + cp -r -p "${DORIS_HOME}/be/src/udf/python/python_server.py" "${DORIS_OUTPUT}/be/plugins/python_udf/" fi if [[ "${BUILD_BROKER}" -eq 1 ]]; then diff --git a/conf/be.conf b/conf/be.conf index 7e351912795521..6ff2f5a19e58c1 100644 --- a/conf/be.conf +++ b/conf/be.conf @@ -94,4 +94,7 @@ aws_log_level = 2 # Error = 4 azure_log_level = 4 ## If you are not running in aws cloud, you can disable EC2 metadata -AWS_EC2_METADATA_DISABLED=true \ No newline at end of file +AWS_EC2_METADATA_DISABLED=true + +python_env_mode = conda +python_conda_root_path = /usr/local/miniconda3 \ No newline at end of file diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java index 71a0a1d72afa48..3727577ac64a0d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Function.java @@ -221,6 +221,8 @@ public Function(Function other) { this.isUDTFunction = other.isUDTFunction; this.isStaticLoad = other.isStaticLoad; this.expirationTime = other.expirationTime; + this.runtimeVersion = other.runtimeVersion; + this.functionCode = other.functionCode; } public void setNestedFunction(Function nestedFunction) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionUtil.java index 320a06125a5211..7ac98257364984 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionUtil.java @@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdtf; import org.apache.doris.nereids.trees.expressions.functions.udf.PythonUdaf; import org.apache.doris.nereids.trees.expressions.functions.udf.PythonUdf; +import org.apache.doris.nereids.trees.expressions.functions.udf.PythonUdtf; import org.apache.doris.nereids.types.DataType; import org.apache.doris.thrift.TFunctionBinaryType; @@ -183,7 +184,11 @@ public static boolean translateToNereids(String dbName, Function function) { AliasUdf.translateToNereidsFunction(dbName, ((AliasFunction) function)); } else if (function instanceof ScalarFunction) { if (function.isUDTFunction()) { - JavaUdtf.translateToNereidsFunction(dbName, ((ScalarFunction) function)); + if (function.getBinaryType() == TFunctionBinaryType.JAVA_UDF) { + JavaUdtf.translateToNereidsFunction(dbName, ((ScalarFunction) function)); + } else if (function.getBinaryType() == TFunctionBinaryType.PYTHON_UDF) { + PythonUdtf.translateToNereidsFunction(dbName, ((ScalarFunction) function)); + } } else { if (function.getBinaryType() == TFunctionBinaryType.JAVA_UDF) { JavaUdf.translateToNereidsFunction(dbName, ((ScalarFunction) function)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java index 560891df9ea512..c559f688beb08d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java @@ -104,6 +104,7 @@ import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdtf; import org.apache.doris.nereids.trees.expressions.functions.udf.PythonUdaf; import org.apache.doris.nereids.trees.expressions.functions.udf.PythonUdf; +import org.apache.doris.nereids.trees.expressions.functions.udf.PythonUdtf; import org.apache.doris.nereids.trees.expressions.functions.window.WindowFunction; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; @@ -897,6 +898,16 @@ public Expr visitPythonUdaf(PythonUdaf udaf, PlanTranslatorContext context) { return functionCallExpr; } + @Override + public Expr visitPythonUdtf(PythonUdtf udtf, PlanTranslatorContext context) { + FunctionParams exprs = new FunctionParams(udtf.children().stream() + .map(expression -> expression.accept(this, context)) + .collect(Collectors.toList())); + FunctionCallExpr functionCallExpr = new FunctionCallExpr(udtf.getCatalogFunction(), exprs); + functionCallExpr.setNullableFromNereids(udtf.nullable()); + return functionCallExpr; + } + // TODO: Supports for `distinct` private Expr translateAggregateFunction(AggregateFunction function, List currentPhaseArguments, List aggFnArguments, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/PythonUdtf.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/PythonUdtf.java new file mode 100644 index 00000000000000..e662e43212a20d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/PythonUdtf.java @@ -0,0 +1,189 @@ +// 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.doris.nereids.trees.expressions.functions.udf; + +import org.apache.doris.analysis.FunctionName; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.Function; +import org.apache.doris.catalog.Function.NullableMode; +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.catalog.Type; +import org.apache.doris.common.util.URI; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.VirtualSlotReference; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Udf; +import org.apache.doris.nereids.trees.expressions.functions.generator.TableGeneratingFunction; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.thrift.TFunctionBinaryType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * Python UDTF for Nereids + */ +public class PythonUdtf extends TableGeneratingFunction implements ExplicitlyCastableSignature, Udf { + private final String dbName; + private final long functionId; + private final TFunctionBinaryType binaryType; + private final FunctionSignature signature; + private final NullableMode nullableMode; + private final String objectFile; + private final String symbol; + private final String prepareFn; + private final String closeFn; + private final String checkSum; + private final boolean isStaticLoad; + private final long expirationTime; + private final String runtimeVersion; + private final String functionCode; + + /** + * Constructor of Python UDTF + */ + public PythonUdtf(String name, long functionId, String dbName, TFunctionBinaryType binaryType, + FunctionSignature signature, + NullableMode nullableMode, String objectFile, String symbol, String prepareFn, String closeFn, + String checkSum, boolean isStaticLoad, long expirationTime, + String runtimeVersion, String functionCode, Expression... args) { + super(name, args); + this.dbName = dbName; + this.functionId = functionId; + this.binaryType = binaryType; + this.signature = signature; + this.nullableMode = nullableMode; + this.objectFile = objectFile; + this.symbol = symbol; + this.prepareFn = prepareFn; + this.closeFn = closeFn; + this.checkSum = checkSum; + this.isStaticLoad = isStaticLoad; + this.expirationTime = expirationTime; + this.runtimeVersion = runtimeVersion; + this.functionCode = functionCode; + } + + /** + * withChildren. + */ + @Override + public PythonUdtf withChildren(List children) { + Preconditions.checkArgument(children.size() == this.children.size()); + return new PythonUdtf(getName(), functionId, dbName, binaryType, signature, nullableMode, + objectFile, symbol, prepareFn, closeFn, checkSum, isStaticLoad, expirationTime, + runtimeVersion, functionCode, children.toArray(new Expression[0])); + } + + @Override + public List getSignatures() { + return ImmutableList.of(signature); + } + + @Override + public boolean hasVarArguments() { + return signature.hasVarArgs; + } + + @Override + public int arity() { + return signature.argumentsTypes.size(); + } + + @Override + public Function getCatalogFunction() { + try { + org.apache.doris.catalog.ScalarFunction expr = org.apache.doris.catalog.ScalarFunction.createUdf( + binaryType, + new FunctionName(dbName, getName()), + signature.argumentsTypes.stream().map(DataType::toCatalogDataType).toArray(Type[]::new), + signature.returnType.toCatalogDataType(), + signature.hasVarArgs, + objectFile == null ? null : URI.create(objectFile), + symbol, + prepareFn, + closeFn + ); + expr.setNullableMode(nullableMode); + expr.setChecksum(checkSum); + expr.setId(functionId); + expr.setStaticLoad(isStaticLoad); + expr.setExpirationTime(expirationTime); + expr.setUDTFunction(true); + expr.setRuntimeVersion(runtimeVersion); + expr.setFunctionCode(functionCode); + return expr; + } catch (Exception e) { + throw new AnalysisException(e.getMessage(), e.getCause()); + } + } + + /** + * translate catalog python udtf to nereids python udtf + */ + public static void translateToNereidsFunction(String dbName, org.apache.doris.catalog.ScalarFunction scalar) { + String fnName = scalar.functionName(); + DataType retType = DataType.fromCatalogType(scalar.getReturnType()); + List argTypes = Arrays.stream(scalar.getArgs()) + .map(DataType::fromCatalogType) + .collect(Collectors.toList()); + + FunctionSignature.FuncSigBuilder sigBuilder = FunctionSignature.ret(retType); + FunctionSignature sig = scalar.hasVarArgs() + ? sigBuilder.varArgs(argTypes.toArray(new DataType[0])) + : sigBuilder.args(argTypes.toArray(new DataType[0])); + + VirtualSlotReference[] virtualSlots = argTypes.stream() + .map(type -> new VirtualSlotReference(type.toString(), type, Optional.empty(), + (shape) -> ImmutableList.of())) + .toArray(VirtualSlotReference[]::new); + + PythonUdtf udtf = new PythonUdtf(fnName, scalar.getId(), dbName, scalar.getBinaryType(), sig, + scalar.getNullableMode(), + scalar.getLocation() == null ? null : scalar.getLocation().getLocation(), + scalar.getSymbolName(), + scalar.getPrepareFnSymbol(), + scalar.getCloseFnSymbol(), + scalar.getChecksum(), + scalar.isStaticLoad(), + scalar.getExpirationTime(), + scalar.getRuntimeVersion(), + scalar.getFunctionCode(), + virtualSlots); + + PythonUdtfBuilder builder = new PythonUdtfBuilder(udtf); + Env.getCurrentEnv().getFunctionRegistry().addUdf(dbName, fnName, builder); + } + + @Override + public NullableMode getNullableMode() { + return nullableMode; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitPythonUdtf(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/PythonUdtfBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/PythonUdtfBuilder.java new file mode 100644 index 00000000000000..3c032ba18abe9d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/PythonUdtfBuilder.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.doris.nereids.trees.expressions.functions.udf; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.common.Pair; +import org.apache.doris.common.util.ReflectionUtils; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.util.TypeCoercionUtils; + +import com.google.common.base.Suppliers; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * function builder for python udtf + */ +public class PythonUdtfBuilder extends UdfBuilder { + private final PythonUdtf udtf; + private final int arity; + private final boolean isVarArgs; + + public PythonUdtfBuilder(PythonUdtf udtf) { + this.udtf = udtf; + this.isVarArgs = udtf.hasVarArguments(); + this.arity = udtf.arity(); + } + + @Override + public List getArgTypes() { + return Suppliers.memoize(() -> udtf.getSignatures().get(0).argumentsTypes.stream() + .map(DataType.class::cast) + .collect(Collectors.toList())).get(); + } + + @Override + public List getSignatures() { + return udtf.getSignatures(); + } + + @Override + public Class functionClass() { + return PythonUdtf.class; + } + + @Override + public boolean canApply(List arguments) { + if ((isVarArgs && arity > arguments.size() + 1) || (!isVarArgs && arguments.size() != arity)) { + return false; + } + for (Object argument : arguments) { + if (!(argument instanceof Expression)) { + Optional primitiveType = ReflectionUtils.getPrimitiveType(argument.getClass()); + if (!primitiveType.isPresent() || !Expression.class.isAssignableFrom(primitiveType.get())) { + return false; + } + } + } + return true; + } + + @Override + public Pair build(String name, List arguments) { + List exprs = arguments.stream().map(Expression.class::cast).collect(Collectors.toList()); + List argTypes = udtf.getSignatures().get(0).argumentsTypes; + + List processedExprs = Lists.newArrayList(); + for (int i = 0; i < exprs.size(); ++i) { + processedExprs.add(TypeCoercionUtils.castIfNotSameType(exprs.get(i), argTypes.get(i))); + } + return Pair.ofSame(udtf.withChildren(processedExprs)); + } + + @Override + public String parameterDisplayString() { + StringBuilder string = new StringBuilder("("); + for (int i = 0; i < udtf.getArgumentsTypes().size(); ++i) { + if (i > 0) { + string.append(", "); + } + string.append(udtf.getArgumentsTypes().get(i)); + if (isVarArgs && i + 1 == udtf.getArgumentsTypes().size()) { + string.append("..."); + } + } + return string.append(")").toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableGeneratingFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableGeneratingFunctionVisitor.java index 9fae7c397cada9..4192ca61465171 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableGeneratingFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableGeneratingFunctionVisitor.java @@ -42,6 +42,7 @@ import org.apache.doris.nereids.trees.expressions.functions.generator.PosExplodeOuter; import org.apache.doris.nereids.trees.expressions.functions.generator.TableGeneratingFunction; import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdtf; +import org.apache.doris.nereids.trees.expressions.functions.udf.PythonUdtf; /** * visitor function for all table generating function. @@ -137,6 +138,10 @@ default R visitJavaUdtf(JavaUdtf udtf, C context) { return visitTableGeneratingFunction(udtf, context); } + default R visitPythonUdtf(PythonUdtf udtf, C context) { + return visitTableGeneratingFunction(udtf, context); + } + default R visitPosExplode(PosExplode posExplode, C context) { return visitTableGeneratingFunction(posExplode, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateFunctionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateFunctionCommand.java index c9bd2666f3d7de..45b043703b4ca5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateFunctionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateFunctionCommand.java @@ -454,9 +454,13 @@ private void analyzeUdtf() throws AnalysisException { throw new AnalysisException("No 'symbol' in properties"); } if (!returnType.isArrayType()) { - throw new AnalysisException("JAVA_UDF OF UDTF return type must be array type"); + throw new AnalysisException("JAVA_UDTF OR PYTHON_UDTF return type must be array type"); + } + if (binaryType == TFunctionBinaryType.JAVA_UDF) { + analyzeJavaUdf(symbol); + } else if (binaryType == TFunctionBinaryType.PYTHON_UDF) { + analyzePythonUdtf(symbol); } - analyzeJavaUdf(symbol); URI location; if (!Strings.isNullOrEmpty(originalUserFile)) { location = URI.create(originalUserFile); @@ -470,6 +474,8 @@ private void analyzeUdtf() throws AnalysisException { function.setChecksum(checksum); function.setNullableMode(returnNullMode); function.setUDTFunction(true); + function.setRuntimeVersion(runtimeVersion); + function.setFunctionCode(functionCode); // Todo: maybe in create tables function, need register two function, one is // normal and one is outer as those have different result when result is NULL. } @@ -893,6 +899,26 @@ private void checkUdfType(Class clazz, Method method, Type expType, Class pType, } } + private void analyzePythonUdtf(String clazz) throws AnalysisException { + if (Strings.isNullOrEmpty(clazz)) { + throw new AnalysisException("No symbol class name provided for Python UDTF"); + } + + if (Strings.isNullOrEmpty(this.functionCode)) { + return; + } + + this.functionCode = this.functionCode.trim(); + if (!(this.functionCode.startsWith("$$") && this.functionCode.endsWith("$$"))) { + throw new AnalysisException("Inline Python UDTF code must be start with $$ and end with $$"); + } + + this.functionCode = this.functionCode.substring(2, this.functionCode.length() - 2); + if (this.functionCode.isEmpty()) { + throw new AnalysisException("Inline Python UDTF is empty"); + } + } + private void checkRPCUdf(String symbol) throws AnalysisException { // TODO(yangzhg) support check function in FE when function service behind load balancer // the format for load balance can ref https://github.com/apache/incubator-brpc/blob/master/docs/en/client.md#connect-to-a-cluster diff --git a/regression-test/conf/regression-conf.groovy b/regression-test/conf/regression-conf.groovy index e1d1bfd7561d0c..22f92669126ace 100644 --- a/regression-test/conf/regression-conf.groovy +++ b/regression-test/conf/regression-conf.groovy @@ -292,5 +292,4 @@ enableExternalHudiTest = false // The properties string of hudi catalog hudiEmrCatalog = "" icebergS3TablesCatalog="" -icebergS3TablesCatalogGlueRest="" - +icebergS3TablesCatalogGlueRest="" \ No newline at end of file diff --git a/regression-test/data/pythonudaf_p0/test_pythonudaf_complex_state_objects_inline.out b/regression-test/data/pythonudaf_p0/test_pythonudaf_complex_state_objects_inline.out index c45eadc04def2b..a54c688a128cf2 100644 --- a/regression-test/data/pythonudaf_p0/test_pythonudaf_complex_state_objects_inline.out +++ b/regression-test/data/pythonudaf_p0/test_pythonudaf_complex_state_objects_inline.out @@ -23,15 +23,6 @@ Home {"unique_users": 1, "unique_products": 1, "payment_methods": ["Debit"]} -- !test_hierarchical_agg -- {"East": {"categories": {"Electronics": {"products": 3, "revenue": 1479.95}, "Furniture": {"products": 1, "revenue": 299.99}}, "total_revenue": 1779.94}, "North": {"categories": {"Electronics": {"products": 5, "revenue": 2069.93}}, "total_revenue": 2069.93}, "South": {"categories": {"Accessories": {"products": 1, "revenue": 29.97}, "Electronics": {"products": 2, "revenue": 879.97}, "Home": {"products": 1, "revenue": 45.99}}, "total_revenue": 955.93}, "West": {"categories": {"Accessories": {"products": 1, "revenue": 31.98}, "Electronics": {"products": 1, "revenue": 29.99}}, "total_revenue": 61.97}} --- !test_recent_window -- -101 {"count": 4, "values": [1299.99, 399.99, 59.98, 149.99], "avg": 477.49, "max": 1299.99, "min": 59.98} -102 {"count": 3, "values": [79.99, 45.99, 29.97], "avg": 51.98, "max": 79.99, "min": 29.97} -103 {"count": 3, "values": [89.99, 1299.99, 89.97], "avg": 493.32, "max": 1299.99, "min": 89.97} -104 {"count": 2, "values": [29.99, 31.98], "avg": 30.98, "max": 31.98, "min": 29.99} -105 {"count": 1, "values": [159.98], "avg": 159.98, "max": 159.98, "min": 159.98} -106 {"count": 1, "values": [799.98], "avg": 799.98, "max": 799.98, "min": 799.98} -107 {"count": 1, "values": [299.99], "avg": 299.99, "max": 299.99, "min": 299.99} - -- !test_complex_window -- 101 Laptop Pro 1299.99 {"101": {"item_count": 1, "total_spent": 1299.99, "unique_categories": 1}} 101 Mouse 29.99 {"101": {"item_count": 2, "total_spent": 1359.97, "unique_categories": 1}} diff --git a/regression-test/data/pythonudaf_p0/test_pythonudaf_complex_state_objects_module.out b/regression-test/data/pythonudaf_p0/test_pythonudaf_complex_state_objects_module.out index 499552c8753814..03796e9a118a6d 100644 --- a/regression-test/data/pythonudaf_p0/test_pythonudaf_complex_state_objects_module.out +++ b/regression-test/data/pythonudaf_p0/test_pythonudaf_complex_state_objects_module.out @@ -23,15 +23,6 @@ Home {"unique_users": 1, "unique_products": 1, "payment_methods": ["Debit"]} -- !test_hierarchical_agg -- {"East": {"categories": {"Electronics": {"products": 3, "revenue": 1479.95}, "Furniture": {"products": 1, "revenue": 299.99}}, "total_revenue": 1779.94}, "North": {"categories": {"Electronics": {"products": 5, "revenue": 2069.93}}, "total_revenue": 2069.93}, "South": {"categories": {"Accessories": {"products": 1, "revenue": 29.97}, "Electronics": {"products": 2, "revenue": 879.97}, "Home": {"products": 1, "revenue": 45.99}}, "total_revenue": 955.93}, "West": {"categories": {"Accessories": {"products": 1, "revenue": 31.98}, "Electronics": {"products": 1, "revenue": 29.99}}, "total_revenue": 61.97}} --- !test_recent_window -- -101 {"count": 4, "values": [1299.99, 399.99, 59.98, 149.99], "avg": 477.49, "max": 1299.99, "min": 59.98} -102 {"count": 3, "values": [79.99, 45.99, 29.97], "avg": 51.98, "max": 79.99, "min": 29.97} -103 {"count": 3, "values": [89.99, 1299.99, 89.97], "avg": 493.32, "max": 1299.99, "min": 89.97} -104 {"count": 2, "values": [29.99, 31.98], "avg": 30.98, "max": 31.98, "min": 29.99} -105 {"count": 1, "values": [159.98], "avg": 159.98, "max": 159.98, "min": 159.98} -106 {"count": 1, "values": [799.98], "avg": 799.98, "max": 799.98, "min": 799.98} -107 {"count": 1, "values": [299.99], "avg": 299.99, "max": 299.99, "min": 299.99} - -- !test_complex_window -- 101 Laptop Pro 1299.99 {"101": {"item_count": 1, "total_spent": 1299.99, "unique_categories": 1}} 101 Mouse 29.99 {"101": {"item_count": 2, "total_spent": 1359.97, "unique_categories": 1}} diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_basic_inline.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_basic_inline.out new file mode 100644 index 00000000000000..8fff208efbbe19 --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_basic_inline.out @@ -0,0 +1,245 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !split_string -- +apple +banana +cherry + +-- !generate_series -- +1 +2 +3 +4 +5 +10 +11 +12 + +-- !generate_series_multiple -- +1 +2 +3 +4 +5 +10 +11 +12 + +-- !running_sum -- +10 10 +20 20 +30 30 +40 40 + +-- !explode_json -- +apple +banana +cherry + +-- !top_n -- +A 100 1 +A 90 1 +A 80 1 +A 70 1 +B 200 1 +B 190 1 + +-- !duplicate -- +Hello 1 +Hello 2 +Hello 3 + +-- !filter_positive -- +1 +3 +7 + +-- !cartesian -- +A X +A Y +A Z +B X +B Y +B Z + +-- !all_filtered -- + +-- !mixed_filter -- +2 -5 +4 -3 + +-- !empty_input -- + +-- !nullable_true -- +1 1 +2 \N +3 3 +4 \N +5 5 + +-- !non_nullable_false -- +1 20 +2 40 +3 60 + +-- !nullable_with_nulls -- +1 \N +2 \N +3 \N +4 \N + +-- !non_nullable_with_nulls -- +1 0 +2 20 +3 0 +4 40 + +-- !default_nullable -- +1 HELLO +2 \N +3 WORLD +4 \N + +-- !multi_nullable -- +1 a +1 b +1 c +2 \N +2 x +2 z +3 \N +3 \N +3 \N + +-- !scalar_int -- +1 +2 +3 +4 +5 + +-- !scalar_string -- +apple +banana +cherry + +-- !mixed_style -- +10 +20 +30 + +-- !return_scalar -- +HELLO + +-- !multi_field_check -- +42 84 + +-- !outer_without -- +1 20 +5 30 + +-- !outer_with -- +1 20 +2 \N +3 \N +4 \N +5 30 + +-- !outer_string_without -- +1 hello +1 world +5 single + +-- !outer_string_with -- +1 hello +1 world +2 \N +3 \N +4 \N +5 single + +-- !outer_range_without -- +1 1 +1 2 +1 3 +5 1 + +-- !outer_range_with -- +1 1 +1 2 +1 3 +2 \N +3 \N +4 \N +5 1 + +-- !outer_multifield_without -- +1 Alice 30 +5 Charlie 25 + +-- !outer_multifield_with -- +1 Alice 30 +2 \N \N +3 \N \N +4 \N \N +5 Charlie 25 + +-- !outer_mixed_functions -- +1 1 1 +1 1 2 +1 1 3 +1 1 4 +1 1 5 +1 2 1 +1 2 2 +1 2 3 +1 2 4 +1 2 5 +1 3 1 +1 3 2 +1 3 3 +1 3 4 +1 3 5 +1 4 1 +1 4 2 +1 4 3 +1 4 4 +1 4 5 +1 5 1 +1 5 2 +1 5 3 +1 5 4 +1 5 5 +3 1 1 +3 1 2 +3 1 3 +3 2 1 +3 2 2 +3 2 3 +3 3 1 +3 3 2 +3 3 3 + +-- !outer_builtin_explode -- +1 1 +1 2 +1 3 +4 5 + +-- !outer_builtin_explode_outer -- +1 1 +1 2 +1 3 +2 \N +3 \N +4 5 + +-- !outer_doc_inner -- +1 apple,banana +4 cherry + +-- !outer_doc_outer -- +1 apple,banana +2 \N +3 \N +4 cherry + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_basic_module.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_basic_module.out new file mode 100644 index 00000000000000..03c1fbd5947ff0 --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_basic_module.out @@ -0,0 +1,71 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !split_string -- +apple +banana +cherry + +-- !generate_series -- +1 +2 +3 +4 +5 +10 +11 +12 + +-- !generate_series_multiple -- +1 +2 +3 +4 +5 +10 +11 +12 + +-- !running_sum -- +10 10 +20 20 +30 30 +40 40 + +-- !explode_json -- +apple +banana +cherry + +-- !top_n -- +A 100 1 +A 90 1 +A 80 1 +A 70 1 +B 200 1 +B 190 1 + +-- !duplicate -- +Hello 1 +Hello 2 +Hello 3 + +-- !filter_positive -- +1 +3 +7 + +-- !cartesian -- +A X +A Y +A Z +B X +B Y +B Z + +-- !all_filtered -- + +-- !mixed_filter -- +2 -5 +4 -3 + +-- !empty_input -- + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_data_types_inline.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_data_types_inline.out new file mode 100644 index 00000000000000..6fc24ef99dea3f --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_data_types_inline.out @@ -0,0 +1,112 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !tinyint -- +-60 -120 +0 0 +63 126 + +-- !smallint -- +-1000 1000000 +0 0 +1000 1000000 + +-- !bigint -- +-1000000000000 -999999999999 +0 1 +1000000000000 1000000000001 + +-- !float -- +-3.14 -1.57 +0.0 0.0 +2.718 1.359 + +-- !double -- +0.0 0.0 +4.0 2.0 +16.0 4.0 +100.0 10.0 + +-- !boolean -- +false true FALSE +true false TRUE + +-- !string -- +DoRiS 5 DORIS doris +Hello 5 HELLO hello +WORLD 5 WORLD world + +-- !date -- +2024-01-01 2024 1 1 +2024-06-15 2024 6 15 +2024-12-31 2024 12 31 + +-- !datetime -- +2024-01-01T08:30 8 30 +2024-06-15T12:00 12 0 +2024-12-31T23:59 23 59 + +-- !array_int -- +1 0 1 2 +1 1 2 4 +1 2 3 6 +2 0 10 20 +2 1 20 40 +3 0 100 200 + +-- !array_string -- +1 apple 5 +1 banana 6 +2 bird 4 +2 cat 3 +2 dog 3 + +-- !struct -- +Alice 25 adult +Bob 15 child +Charlie 30 adult + +-- !multi_types -- +100 apple apple_100 +200 banana banana_200 + +-- !decimal -- +123.45 246.90 +678.90 1357.80 +999.99 1999.98 + +-- !map_like -- +1 age 25 +1 score 90 +2 age 30 +2 level 3 +2 score 85 + +-- !nested_array -- +1 0 10 +1 0 20 +1 1 30 +1 1 40 +2 0 50 +2 1 60 +2 1 70 +2 1 80 + +-- !array_structs -- +1 Alice 25 90 +1 Bob 30 85 +2 Charlie 28 88 + +-- !struct_array -- +1 Alice 3 sports,music,reading +2 Bob 2 coding,gaming + +-- !json_extract -- +1 age 25 +1 city NYC +1 name Alice +2 age 30 +2 name Bob + +-- !complex_struct -- +1 101 Alice NYC 10001 +2 102 Bob LA 90001 + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_data_types_module.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_data_types_module.out new file mode 100644 index 00000000000000..6fc24ef99dea3f --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_data_types_module.out @@ -0,0 +1,112 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !tinyint -- +-60 -120 +0 0 +63 126 + +-- !smallint -- +-1000 1000000 +0 0 +1000 1000000 + +-- !bigint -- +-1000000000000 -999999999999 +0 1 +1000000000000 1000000000001 + +-- !float -- +-3.14 -1.57 +0.0 0.0 +2.718 1.359 + +-- !double -- +0.0 0.0 +4.0 2.0 +16.0 4.0 +100.0 10.0 + +-- !boolean -- +false true FALSE +true false TRUE + +-- !string -- +DoRiS 5 DORIS doris +Hello 5 HELLO hello +WORLD 5 WORLD world + +-- !date -- +2024-01-01 2024 1 1 +2024-06-15 2024 6 15 +2024-12-31 2024 12 31 + +-- !datetime -- +2024-01-01T08:30 8 30 +2024-06-15T12:00 12 0 +2024-12-31T23:59 23 59 + +-- !array_int -- +1 0 1 2 +1 1 2 4 +1 2 3 6 +2 0 10 20 +2 1 20 40 +3 0 100 200 + +-- !array_string -- +1 apple 5 +1 banana 6 +2 bird 4 +2 cat 3 +2 dog 3 + +-- !struct -- +Alice 25 adult +Bob 15 child +Charlie 30 adult + +-- !multi_types -- +100 apple apple_100 +200 banana banana_200 + +-- !decimal -- +123.45 246.90 +678.90 1357.80 +999.99 1999.98 + +-- !map_like -- +1 age 25 +1 score 90 +2 age 30 +2 level 3 +2 score 85 + +-- !nested_array -- +1 0 10 +1 0 20 +1 1 30 +1 1 40 +2 0 50 +2 1 60 +2 1 70 +2 1 80 + +-- !array_structs -- +1 Alice 25 90 +1 Bob 30 85 +2 Charlie 28 88 + +-- !struct_array -- +1 Alice 3 sports,music,reading +2 Bob 2 coding,gaming + +-- !json_extract -- +1 age 25 +1 city NYC +1 name Alice +2 age 30 +2 name Bob + +-- !complex_struct -- +1 101 Alice NYC 10001 +2 102 Bob LA 90001 + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_edge_cases_inline.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_edge_cases_inline.out new file mode 100644 index 00000000000000..0fd24efbe00d2e --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_edge_cases_inline.out @@ -0,0 +1,70 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !null_int -- +1 \N true -1 +2 0 false 0 +3 10 false 20 +4 \N true -1 + +-- !null_string -- +1 NULL -1 +2 EMPTY 0 +3 NORMAL 5 +4 NULL -1 + +-- !empty_array -- +1 NULL -1 +2 EMPTY 0 +3 NORMAL 3 + +-- !null_struct -- +1 true true Alice_25 +2 true false Bob_no_age +3 false true no_name_30 +4 false false all_fields_null + +-- !empty_table -- + +-- !single_row -- +100 100 +100 101 +100 102 + +-- !long_string -- +1 1000 AAAAAAAAAA AAAAAAAAAA +2 5000 BBBBBBBBBB BBBBBBBBBB + +-- !large_array -- +1 100 100 1 1 +2 50 250 5 5 + +-- !output_explosion -- +1 10 0 9 +2 50 0 49 + +-- !special_numbers -- +1 -2147483648 NEGATIVE true +2 -1 NEGATIVE false +3 0 ZERO false +4 1 POSITIVE false +5 2147483647 POSITIVE true +6 \N NULL false + +-- !special_doubles -- +1 0.0 ZERO +2 1.0E-15 VERY_SMALL +3 1.0E15 VERY_LARGE +4 -1.0E15 VERY_LARGE +5 3.14159 NORMAL + +-- !special_strings -- +1 11 false NORMAL +2 15 false HAS_SYMBOLS +3 8 false HAS_WHITESPACE +4 4 true HAS_UNICODE +5 0 false EMPTY + +-- !boundary_dates -- +1 1970-01-01 1970 true +2 2024-06-15 2024 false +3 9999-12-31 9999 true + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_edge_cases_module.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_edge_cases_module.out new file mode 100644 index 00000000000000..e5238713750663 --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_edge_cases_module.out @@ -0,0 +1,70 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !null_int -- +1 \N true -1 +2 0 false 0 +3 10 false 20 +4 \N true -1 + +-- !null_string -- +1 NULL -1 +2 EMPTY 0 +3 NORMAL 5 +4 NULL -1 + +-- !empty_array -- +1 NULL -1 +2 EMPTY 0 +3 NORMAL 3 + +-- !null_struct -- +1 true true Alice_25 +2 true false Bob_no_age +3 false true no_name_30 +4 false false all_fields_null + +-- !empty_table -- + +-- !single_row -- +100 100 +100 101 +100 102 + +-- !long_string -- +1 1000 AAAAAAAAAA AAAAAAAAAA +2 5000 BBBBBBBBBB BBBBBBBBBB + +-- !large_array -- +1 100 100 1 1 +2 50 250 5 5 + +-- !output_explosion -- +1 10 0 9 +2 50 0 49 + +-- !special_numbers -- +1 -2147483648 NEGATIVE true +2 -1 NEGATIVE false +3 0 ZERO false +4 1 POSITIVE false +5 2147483647 POSITIVE true +6 \N NULL false + +-- !special_doubles -- +1 0 ZERO +2 1e-15 VERY_SMALL +3 1000000000000000 VERY_LARGE +4 -1000000000000000 VERY_LARGE +5 3.14159 NORMAL + +-- !special_strings -- +1 11 false NORMAL +2 15 false HAS_SYMBOLS +3 8 false HAS_WHITESPACE +4 4 true HAS_UNICODE +5 0 false EMPTY + +-- !boundary_dates -- +1 1970-01-01 1970 true +2 2024-06-15 2024 false +3 9999-12-31 9999 true + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_exceptions_inline.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_exceptions_inline.out new file mode 100644 index 00000000000000..afd3588411dfe5 --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_exceptions_inline.out @@ -0,0 +1,86 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !safe_divide -- +1 10 2 5.0 success +2 10 0 \N division_by_zero +3 0 5 0.0 success +4 -8 4 -2.0 success + +-- !overflow_check -- +1 100 200 safe +2 5000000000000 10000000000000 safe +3 -5000000000000 -10000000000000 safe +4 \N \N null_input + +-- !parse_number -- +1 123 123.0 true +2 45.67 45.67 true +3 abc \N false +4 12.34.56 \N false +5 \N false +6 \N \N false + +-- !type_check -- +1 hello str 5 +2 str 0 +3 12345 str 5 +4 \N NoneType 0 + +-- !safe_index -- +1 3 1 20 success +2 3 5 \N out_of_bounds +3 3 -1 \N out_of_bounds +4 0 0 \N empty_array +5 0 0 \N null_array + +-- !collection_stats -- +1 5 15 3.0 computed +2 0 0 0.0 empty_array +3 0 0 0.0 null_array +4 2 30 15.0 computed + +-- !safe_struct_access -- +1 true true Alice 30 +2 true false Bob \N +3 false true \N 25 +4 false false \N \N + +-- !string_slice -- +1 hello world 0 5 hello success +2 hello world 6 11 world success +3 hello world 20 30 empty_slice +4 hello world 5 2 empty_slice +5 \N 0 5 \N null_string + +-- !check_encoding -- +1 hello 5 5 false +2 你好世界 12 4 true +3 café 5 4 true +4 0 0 false +5 \N 0 0 false + +-- !conditional_process -- +1 -10 negative 10 +2 0 zero 1 +3 50 small_positive 100 +4 200 large_positive 200 +5 \N null 0 + +-- !filter_yield -- +1 10 +5 22 + +-- !number_range -- +1 0.0 zero true +2 1.0E-150 extremely_small true +3 1.0E150 extremely_large true +4 0.5 small true +5 123.456 normal true +6 \N null true + +-- !date_validation -- +1 2024-01-01 2024 true normal +2 2000-02-29 2000 true normal +3 1970-01-01 1970 false normal +4 9999-12-31 9999 false far_future +5 \N 0 false null_date + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_exceptions_module.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_exceptions_module.out new file mode 100644 index 00000000000000..bf33b0a48658ab --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_exceptions_module.out @@ -0,0 +1,86 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !safe_divide -- +1 10 2 5 success +2 10 0 \N division_by_zero +3 0 5 0 success +4 -8 4 -2 success + +-- !overflow_check -- +1 100 200 safe +2 5000000000000 10000000000000 safe +3 -5000000000000 -10000000000000 safe +4 \N \N null_input + +-- !parse_number -- +1 123 123 true +2 45.67 45.67 true +3 abc \N false +4 12.34.56 \N false +5 \N false +6 \N \N false + +-- !type_check -- +1 hello str 5 +2 str 0 +3 12345 str 5 +4 \N NoneType 0 + +-- !safe_index -- +1 3 1 20 success +2 3 5 \N out_of_bounds +3 3 -1 \N out_of_bounds +4 0 0 \N empty_array +5 0 0 \N null_array + +-- !collection_stats -- +1 5 15 3 computed +2 0 0 0 empty_array +3 0 0 0 null_array +4 2 30 15 computed + +-- !safe_struct_access -- +1 true true Alice 30 +2 true false Bob \N +3 false true \N 25 +4 false false \N \N + +-- !string_slice -- +1 hello world 0 5 hello success +2 hello world 6 11 world success +3 hello world 20 30 empty_slice +4 hello world 5 2 empty_slice +5 \N 0 5 \N null_string + +-- !check_encoding -- +1 hello 5 5 false +2 你好世界 12 4 true +3 café 5 4 true +4 0 0 false +5 \N 0 0 false + +-- !conditional_process -- +1 -10 negative 10 +2 0 zero 1 +3 50 small_positive 100 +4 200 large_positive 200 +5 \N null 0 + +-- !filter_yield -- +1 10 +5 22 + +-- !number_range -- +1 0 zero true +2 1e-150 extremely_small true +3 1e+150 extremely_large true +4 0.5 small true +5 123.456 normal true +6 \N null true + +-- !date_validation -- +1 2024-01-01 2024 true normal +2 2000-02-29 2000 true normal +3 1970-01-01 1970 false normal +4 9999-12-31 9999 false far_future +5 \N 0 false null_date + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_io_patterns_inline.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_io_patterns_inline.out new file mode 100644 index 00000000000000..5b2e8d612694d3 --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_io_patterns_inline.out @@ -0,0 +1,72 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !one_to_one -- +10 20 +20 40 +30 60 + +-- !one_to_many -- +1 1 +1 2 +1 3 +2 1 +2 2 +3 1 +3 2 +3 3 +3 4 + +-- !one_to_zero -- +2 +4 +6 + +-- !one_to_variable -- +1 hello +2 hello +2 world +4 a +4 b +4 c + +-- !aggregate_pattern -- +large 2 +medium 2 +small 2 + +-- !explosive -- +0 0 +0 1 +0 2 +1 0 +1 1 +1 2 + +-- !conditional -- +0 neutral +0 zero +5 negative +7 positive +10 positive + +-- !all_or_nothing -- +1 h 0 +1 e 1 +1 l 2 +1 l 3 +1 o 4 +3 w 0 +3 o 1 +3 r 2 +3 l 3 +3 d 4 + +-- !empty_input -- + +-- !batch_process -- +10 2 20 +10 3 30 +10 5 50 +20 2 40 +20 3 60 +20 5 100 + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_io_patterns_module.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_io_patterns_module.out new file mode 100644 index 00000000000000..5b2e8d612694d3 --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_io_patterns_module.out @@ -0,0 +1,72 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !one_to_one -- +10 20 +20 40 +30 60 + +-- !one_to_many -- +1 1 +1 2 +1 3 +2 1 +2 2 +3 1 +3 2 +3 3 +3 4 + +-- !one_to_zero -- +2 +4 +6 + +-- !one_to_variable -- +1 hello +2 hello +2 world +4 a +4 b +4 c + +-- !aggregate_pattern -- +large 2 +medium 2 +small 2 + +-- !explosive -- +0 0 +0 1 +0 2 +1 0 +1 1 +1 2 + +-- !conditional -- +0 neutral +0 zero +5 negative +7 positive +10 positive + +-- !all_or_nothing -- +1 h 0 +1 e 1 +1 l 2 +1 l 3 +1 o 4 +3 w 0 +3 o 1 +3 r 2 +3 l 3 +3 d 4 + +-- !empty_input -- + +-- !batch_process -- +10 2 20 +10 3 30 +10 5 50 +20 2 40 +20 3 60 +20 5 100 + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_sql_integration_inline.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_sql_integration_inline.out new file mode 100644 index 00000000000000..d452805258fee1 --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_sql_integration_inline.out @@ -0,0 +1,213 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !where_before -- +1 A 0 apple +1 A 1 banana +3 A 0 red +3 A 1 green +3 A 2 blue + +-- !where_after -- +1 0 apple +3 0 red +3 1 green +3 2 blue +4 0 one + +-- !where_combined -- +1 A apple +2 B cat +3 A red + +-- !join_inner -- +1 1 one false +1 2 two true +1 3 three false +2 2 two true +2 3 three false +2 4 four true + +-- !join_left -- +1 1 one +1 2 two +2 5 five +2 6 \N +2 7 \N + +-- !join_self -- +1 20 2 20 +1 30 2 30 + +-- !group_by_udtf -- +apple 2 +banana 2 +cat 2 +cherry 1 +dog 1 + +-- !group_by_mixed -- +animal cat 2 +animal dog 1 +fruit apple 2 +fruit banana 2 +fruit cherry 1 + +-- !group_by_having -- +apple 2 +banana 2 +cat 2 + +-- !group_by_multi_agg -- +1 5 1 5 15 3.0 +2 5 3 7 25 5.0 +3 3 10 12 33 11.0 + +-- !order_by_udtf -- +2 Bob 95 +1 Alice 92 +2 Bob 90 +2 Bob 88 +3 Charlie 88 + +-- !order_by_mixed -- +1 Alice 2 78 +1 Alice 1 92 +1 Alice 0 85 +2 Bob 2 95 +2 Bob 1 88 +2 Bob 0 90 +3 Charlie 2 88 +3 Charlie 1 82 +3 Charlie 0 70 + +-- !limit_only -- +1 85 +1 92 +1 78 + +-- !top_n_pattern -- +1 Alice 92 +1 Alice 85 +2 Bob 95 +2 Bob 90 +3 Charlie 88 +3 Charlie 82 + +-- !subquery_in -- +1 A,B,C +3 A,C,E + +-- !subquery_from -- +C 3 +A 2 +B 2 +D 1 +E 1 + +-- !subquery_nested -- +C 3 +A 2 +B 2 + +-- !distinct_udtf -- +blue +green +red +yellow + +-- !count_distinct -- +4 + +-- !union_all -- +1 X +1 Y +2 Y +2 Z + +-- !union_distinct -- +X +Y +Z + +-- !array_filter -- +1 3 +2 3 +2 4 +2 5 +3 3 +3 4 + +-- !array_aggregate -- +1 3 6 2.0 +2 4 14 3.5 +3 2 7 3.5 + +-- !window_function -- +1 A 10 1 100 +2 A 15 2 100 +1 A 20 3 100 +2 A 25 4 100 +1 A 30 5 100 +3 B 5 1 30 +3 B 10 2 30 +3 B 15 3 30 + +-- !case_when -- +1 5 small +1 15 medium +1 25 large +2 10 medium +2 20 large +2 30 large + +-- !nested_2level -- +1 1,2 1 +1 1,2 2 +1 3 3 +2 4,5 4 +2 4,5 5 + +-- !parallel_lateral -- +1 A X +1 A Y +1 B X +1 B Y +2 C Z + +-- !nested_join -- +1 soccer 10 +1 tennis 8 +1 pizza 5 +2 rock 9 + +-- !nested_groupby -- +1 3 450 +2 3 650 + +-- !nested_3level -- +1 0 A +1 0 B +1 0 C +1 1 D +1 1 E +1 2 F + +-- !nested_array_expansion -- +1 1 10 +1 1 20 +1 2 30 +2 1 40 +2 1 50 + +-- !nested_multifilter -- +1 \N 20 +1 \N 30 +1 \N 40 +2 \N 50 +2 \N 60 + +-- !nested_distinct -- +blue +green +red +yellow + diff --git a/regression-test/data/pythonudtf_p0/test_pythonudtf_sql_integration_module.out b/regression-test/data/pythonudtf_p0/test_pythonudtf_sql_integration_module.out new file mode 100644 index 00000000000000..c1107085c7c45a --- /dev/null +++ b/regression-test/data/pythonudtf_p0/test_pythonudtf_sql_integration_module.out @@ -0,0 +1,213 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !where_before -- +1 A 0 apple +1 A 1 banana +3 A 0 red +3 A 1 green +3 A 2 blue + +-- !where_after -- +1 0 apple +3 0 red +3 1 green +3 2 blue +4 0 one + +-- !where_combined -- +1 A apple +2 B cat +3 A red + +-- !join_inner -- +1 1 one false +1 2 two true +1 3 three false +2 2 two true +2 3 three false +2 4 four true + +-- !join_left -- +1 1 one +1 2 two +2 5 five +2 6 \N +2 7 \N + +-- !join_self -- +1 20 2 20 +1 30 2 30 + +-- !group_by_udtf -- +apple 2 +banana 2 +cat 2 +cherry 1 +dog 1 + +-- !group_by_mixed -- +animal cat 2 +animal dog 1 +fruit apple 2 +fruit banana 2 +fruit cherry 1 + +-- !group_by_having -- +apple 2 +banana 2 +cat 2 + +-- !group_by_multi_agg -- +1 5 1 5 15 3 +2 5 3 7 25 5 +3 3 10 12 33 11 + +-- !order_by_udtf -- +2 Bob 95 +1 Alice 92 +2 Bob 90 +2 Bob 88 +3 Charlie 88 + +-- !order_by_mixed -- +1 Alice 2 78 +1 Alice 1 92 +1 Alice 0 85 +2 Bob 2 95 +2 Bob 1 88 +2 Bob 0 90 +3 Charlie 2 88 +3 Charlie 1 82 +3 Charlie 0 70 + +-- !limit_only -- +1 85 +1 92 +1 78 + +-- !top_n_pattern -- +1 Alice 92 +1 Alice 85 +2 Bob 95 +2 Bob 90 +3 Charlie 88 +3 Charlie 82 + +-- !subquery_in -- +1 A,B,C +3 A,C,E + +-- !subquery_from -- +C 3 +A 2 +B 2 +D 1 +E 1 + +-- !subquery_nested -- +C 3 +A 2 +B 2 + +-- !distinct_udtf -- +blue +green +red +yellow + +-- !count_distinct -- +4 + +-- !union_all -- +1 X +1 Y +2 Y +2 Z + +-- !union_distinct -- +X +Y +Z + +-- !array_filter -- +1 3 +2 3 +2 4 +2 5 +3 3 +3 4 + +-- !array_aggregate -- +1 3 6 2 +2 4 14 3.5 +3 2 7 3.5 + +-- !window_function -- +1 A 10 1 100 +2 A 15 2 100 +1 A 20 3 100 +2 A 25 4 100 +1 A 30 5 100 +3 B 5 1 30 +3 B 10 2 30 +3 B 15 3 30 + +-- !case_when -- +1 5 small +1 15 medium +1 25 large +2 10 medium +2 20 large +2 30 large + +-- !nested_2level -- +1 1,2 1 +1 1,2 2 +1 3 3 +2 4,5 4 +2 4,5 5 + +-- !parallel_lateral -- +1 A X +1 A Y +1 B X +1 B Y +2 C Z + +-- !nested_join -- +1 soccer 10 +1 tennis 8 +1 pizza 5 +2 rock 9 + +-- !nested_groupby -- +1 3 450 +2 3 650 + +-- !nested_3level -- +1 0 A +1 0 B +1 0 C +1 1 D +1 1 E +1 2 F + +-- !nested_array_expansion -- +1 1 10 +1 1 20 +1 2 30 +2 1 40 +2 1 50 + +-- !nested_multifilter -- +1 \N 20 +1 \N 30 +1 \N 40 +2 \N 50 +2 \N 60 + +-- !nested_distinct -- +blue +green +red +yellow + diff --git a/regression-test/suites/pythonudaf_p0/test_pythonudaf_complex_state_objects_inline.groovy b/regression-test/suites/pythonudaf_p0/test_pythonudaf_complex_state_objects_inline.groovy index b255fef92cf0f1..88cbdd476e9265 100644 --- a/regression-test/suites/pythonudaf_p0/test_pythonudaf_complex_state_objects_inline.groovy +++ b/regression-test/suites/pythonudaf_p0/test_pythonudaf_complex_state_objects_inline.groovy @@ -512,57 +512,6 @@ class HierarchicalAggUDAF: \$\$; """ - // ======================================== - // UDAF 7: Deque-based State - Recent Transactions Window - // Uses collections.deque for efficient FIFO operations - // ======================================== - sql """ DROP FUNCTION IF EXISTS py_recent_window(DECIMAL, INT); """ - sql """ - CREATE AGGREGATE FUNCTION py_recent_window(DECIMAL, INT) - RETURNS VARCHAR - PROPERTIES ( - "type" = "PYTHON_UDF", - "symbol" = "RecentWindowUDAF", - "runtime_version" = "${runtime_version}" - ) - AS \$\$ -from collections import deque -import json - -class RecentWindowUDAF: - def __init__(self): - # Keep last 5 transactions using deque - self.window = deque(maxlen=5) - - @property - def aggregate_state(self): - # Convert deque to list for pickle - return list(self.window) - - def accumulate(self, price, quantity): - if price is not None and quantity is not None: - revenue = float(price) * int(quantity) - self.window.append(revenue) - - def merge(self, other_state): - # Merge by extending window (keeps last 5) - for item in other_state: - self.window.append(item) - - def finish(self): - if not self.window: - return json.dumps({'count': 0}) - - return json.dumps({ - 'count': len(self.window), - 'values': [round(v, 2) for v in self.window], - 'avg': round(sum(self.window) / len(self.window), 2), - 'max': round(max(self.window), 2), - 'min': round(min(self.window), 2) - }) -\$\$; - """ - // ======================================== // Test Cases // ======================================== @@ -615,17 +564,7 @@ class RecentWindowUDAF: FROM complex_transactions; """ - // Test 7: Recent Window (Deque) - qt_test_recent_window """ - SELECT - user_id, - py_recent_window(price, quantity) as recent_purchases - FROM complex_transactions - GROUP BY user_id - ORDER BY user_id; - """ - - // Test 8: Complex State with Window Function + // Test 7: Complex State with Window Function qt_test_complex_window """ SELECT user_id, @@ -637,7 +576,7 @@ class RecentWindowUDAF: ORDER BY user_id, transaction_id; """ - // Test 9: Multiple Complex UDAFs in Single Query + // Test 8: Multiple Complex UDAFs in Single Query qt_test_multi_complex """ SELECT region, @@ -648,7 +587,7 @@ class RecentWindowUDAF: ORDER BY region; """ - // Test 10: Nested Query with Complex State + // Test 9: Nested Query with Complex State qt_test_nested_complex """ SELECT region, @@ -664,7 +603,7 @@ class RecentWindowUDAF: ORDER BY region; """ - // Test 11: Complex State Serialization in Shuffle (GROUP BY multiple columns) + // Test 10: Complex State Serialization in Shuffle (GROUP BY multiple columns) qt_test_complex_shuffle """ SELECT region, @@ -675,7 +614,7 @@ class RecentWindowUDAF: ORDER BY region, category; """ - // Test 12: Edge Case - Empty Groups + // Test 11: Edge Case - Empty Groups qt_test_empty_groups """ SELECT region, @@ -685,7 +624,7 @@ class RecentWindowUDAF: GROUP BY region; """ - // Test 13: Edge Case - NULL Values + // Test 12: Edge Case - NULL Values sql """ DROP TABLE IF EXISTS complex_nulls; """ sql """ CREATE TABLE complex_nulls ( @@ -715,7 +654,7 @@ class RecentWindowUDAF: FROM complex_nulls; """ - // Test 14: Performance - Large Complex State + // Test 13: Performance - Large Complex State qt_test_large_state """ SELECT COUNT(*) as total_transactions, @@ -735,6 +674,5 @@ class RecentWindowUDAF: sql """ DROP FUNCTION IF EXISTS py_unique_tracker(INT, INT, VARCHAR); """ sql """ DROP FUNCTION IF EXISTS py_category_summary(VARCHAR, DECIMAL, INT); """ sql """ DROP FUNCTION IF EXISTS py_hierarchical_agg(VARCHAR, VARCHAR, VARCHAR, DECIMAL, INT); """ - sql """ DROP FUNCTION IF EXISTS py_recent_window(DECIMAL, INT); """ } } diff --git a/regression-test/suites/pythonudaf_p0/test_pythonudaf_complex_state_objects_module.groovy b/regression-test/suites/pythonudaf_p0/test_pythonudaf_complex_state_objects_module.groovy index 765f5633955195..d53c90c48d9f10 100644 --- a/regression-test/suites/pythonudaf_p0/test_pythonudaf_complex_state_objects_module.groovy +++ b/regression-test/suites/pythonudaf_p0/test_pythonudaf_complex_state_objects_module.groovy @@ -165,22 +165,6 @@ suite("test_pythonudaf_complex_state_objects_module") { ); """ - // ======================================== - // UDAF 7: Deque-based State - Recent Transactions Window - // ======================================== - sql """ DROP FUNCTION IF EXISTS py_recent_window_mod(DECIMAL, INT); """ - sql """ - CREATE AGGREGATE FUNCTION py_recent_window_mod(DECIMAL, INT) - RETURNS VARCHAR - PROPERTIES ( - "file" = "file://${pyPath}", - "symbol" = "complex_state_udaf.RecentWindowUDAF", - "type" = "PYTHON_UDF", - "always_nullable" = "true", - "runtime_version" = "${runtime_version}" - ); - """ - // ======================================== // Test Cases // ======================================== @@ -233,17 +217,7 @@ suite("test_pythonudaf_complex_state_objects_module") { FROM complex_transactions_mod; """ - // Test 7: Recent Window (Deque) - qt_test_recent_window """ - SELECT - user_id, - py_recent_window_mod(price, quantity) as recent_purchases - FROM complex_transactions_mod - GROUP BY user_id - ORDER BY user_id; - """ - - // Test 8: Complex State with Window Function + // Test 7: Complex State with Window Function qt_test_complex_window """ SELECT user_id, @@ -255,7 +229,7 @@ suite("test_pythonudaf_complex_state_objects_module") { ORDER BY user_id, transaction_id; """ - // Test 9: Multiple Complex UDAFs in Single Query + // Test 8: Multiple Complex UDAFs in Single Query qt_test_multi_complex """ SELECT region, @@ -266,7 +240,7 @@ suite("test_pythonudaf_complex_state_objects_module") { ORDER BY region; """ - // Test 10: Nested Query with Complex State + // Test 9: Nested Query with Complex State qt_test_nested_complex """ SELECT region, @@ -282,7 +256,7 @@ suite("test_pythonudaf_complex_state_objects_module") { ORDER BY region; """ - // Test 11: Complex State Serialization in Shuffle (GROUP BY multiple columns) + // Test 10: Complex State Serialization in Shuffle (GROUP BY multiple columns) qt_test_complex_shuffle """ SELECT region, @@ -293,7 +267,7 @@ suite("test_pythonudaf_complex_state_objects_module") { ORDER BY region, category; """ - // Test 12: Edge Case - Empty Groups + // Test 11: Edge Case - Empty Groups qt_test_empty_groups """ SELECT region, @@ -303,7 +277,7 @@ suite("test_pythonudaf_complex_state_objects_module") { GROUP BY region; """ - // Test 13: Edge Case - NULL Values + // Test 12: Edge Case - NULL Values sql """ DROP TABLE IF EXISTS complex_nulls_mod; """ sql """ CREATE TABLE complex_nulls_mod ( @@ -333,7 +307,7 @@ suite("test_pythonudaf_complex_state_objects_module") { FROM complex_nulls_mod; """ - // Test 14: Performance - Large Complex State + // Test 13: Performance - Large Complex State qt_test_large_state """ SELECT COUNT(*) as total_transactions, @@ -342,7 +316,7 @@ suite("test_pythonudaf_complex_state_objects_module") { FROM complex_transactions_mod; """ - // Test 15: Module Reusability - Create another function from same module + // Test 14: Module Reusability - Create another function from same module sql """ DROP FUNCTION IF EXISTS py_user_profile_mod2(INT, VARCHAR, VARCHAR, DECIMAL, INT); """ sql """ CREATE AGGREGATE FUNCTION py_user_profile_mod2(INT, VARCHAR, VARCHAR, DECIMAL, INT) @@ -363,7 +337,7 @@ suite("test_pythonudaf_complex_state_objects_module") { FROM complex_transactions_mod; """ - // Test 16: Global Functions + // Test 15: Global Functions sql """ DROP GLOBAL FUNCTION IF EXISTS py_user_profile_global(INT, VARCHAR, VARCHAR, DECIMAL, INT); """ sql """ CREATE GLOBAL AGGREGATE FUNCTION py_user_profile_global(INT, VARCHAR, VARCHAR, DECIMAL, INT) @@ -392,7 +366,6 @@ suite("test_pythonudaf_complex_state_objects_module") { try_sql("DROP FUNCTION IF EXISTS py_unique_tracker_mod(INT, INT, VARCHAR);") try_sql("DROP FUNCTION IF EXISTS py_category_summary_mod(VARCHAR, DECIMAL, INT);") try_sql("DROP FUNCTION IF EXISTS py_hierarchical_agg_mod(VARCHAR, VARCHAR, VARCHAR, DECIMAL, INT);") - try_sql("DROP FUNCTION IF EXISTS py_recent_window_mod(DECIMAL, INT);") try_sql("DROP FUNCTION IF EXISTS py_user_profile_mod2(INT, VARCHAR, VARCHAR, DECIMAL, INT);") try_sql("DROP TABLE IF EXISTS complex_transactions_mod;") try_sql("DROP TABLE IF EXISTS complex_nulls_mod;") diff --git a/regression-test/suites/pythonudaf_p0/udaf_scripts/complex_state_udaf.py b/regression-test/suites/pythonudaf_p0/udaf_scripts/complex_state_udaf.py index 72a56bf05a56fc..2d87b524fb59aa 100644 --- a/regression-test/suites/pythonudaf_p0/udaf_scripts/complex_state_udaf.py +++ b/regression-test/suites/pythonudaf_p0/udaf_scripts/complex_state_udaf.py @@ -398,37 +398,41 @@ def finish(self): # ======================================== # UDAF 7: Deque-based State - Recent Transactions Window +# Modified to use sorted aggregation for deterministic results # ======================================== class RecentWindowUDAF: - """Uses collections.deque for efficient FIFO operations""" + """Aggregates transactions with deterministic sorting""" def __init__(self): - # Keep last 5 transactions using deque - self.window = deque(maxlen=5) + # Keep all transactions for deterministic ordering + self.all_transactions = [] @property def aggregate_state(self): - # Convert deque to list for pickle - return list(self.window) + # Return all transactions for merging + return self.all_transactions def accumulate(self, price, quantity): if price is not None and quantity is not None: revenue = float(price) * int(quantity) - self.window.append(revenue) + self.all_transactions.append(revenue) def merge(self, other_state): - # Merge by extending window (keeps last 5) - for item in other_state: - self.window.append(item) + # Merge all transactions + self.all_transactions.extend(other_state) def finish(self): - if not self.window: + if not self.all_transactions: return json.dumps({'count': 0}) + # Sort for deterministic results, then take last 5 + sorted_trans = sorted(self.all_transactions) + window = sorted_trans[-5:] if len(sorted_trans) > 5 else sorted_trans + return json.dumps({ - 'count': len(self.window), - 'values': [round(v, 2) for v in self.window], - 'avg': round(sum(self.window) / len(self.window), 2), - 'max': round(max(self.window), 2), - 'min': round(min(self.window), 2) + 'count': len(window), + 'values': [round(v, 2) for v in window], + 'avg': round(sum(window) / len(window), 2), + 'max': round(max(window), 2), + 'min': round(min(window), 2) }) diff --git a/regression-test/suites/pythonudaf_p0/udaf_scripts/pyudaf.zip b/regression-test/suites/pythonudaf_p0/udaf_scripts/pyudaf.zip index 4982af8b96df56..1dc76099d43326 100644 Binary files a/regression-test/suites/pythonudaf_p0/udaf_scripts/pyudaf.zip and b/regression-test/suites/pythonudaf_p0/udaf_scripts/pyudaf.zip differ diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_basic_inline.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_basic_inline.groovy new file mode 100644 index 00000000000000..0352de160860a8 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_basic_inline.groovy @@ -0,0 +1,1247 @@ +// 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. + +suite("test_pythonudtf_basic_inline") { + // Basic Python UDTF tests following Snowflake syntax + // UDTF (User-Defined Table Function) returns table (multiple rows) from scalar/table input + + def runtime_version = "3.10.12" + + try { + // ======================================== + // Test 1: Simple String Split UDTF + // Input: Single string + // Output: Multiple rows (one per split part) + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_split_string(STRING); """ + sql """ + CREATE TABLES FUNCTION py_split_string(STRING) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "split_string_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def split_string_udtf(input_str): + '''Split comma-separated string into rows''' + if input_str: + parts = input_str.split(',') + for part in parts: + yield (part.strip(),) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS temp_input; """ + sql """ + CREATE TABLE temp_input ( + id INT, + input_str STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_input VALUES (1, 'apple,banana,cherry'); + """ + + qt_split_string """ + SELECT part + FROM temp_input + LATERAL VIEW py_split_string(input_str) tmp AS part; + """ + + // ======================================== + // Test 2: Generate Series UDTF + // Input: start, end integers + // Output: Sequence of integers + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_generate_series(INT, INT); """ + sql """ + CREATE TABLES FUNCTION py_generate_series(INT, INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "generate_series_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def generate_series_udtf(start, end): + '''Generate integer series from start to end''' + if start is not None and end is not None: + for i in range(start, end + 1): + yield (i,) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS temp_series; """ + sql """ + CREATE TABLE temp_series ( + id INT, + start_val INT, + end_val INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_series VALUES (1, 1, 5), (2, 10, 12); + """ + + qt_generate_series """ + SELECT tmp.value + FROM temp_series + LATERAL VIEW py_generate_series(start_val, end_val) tmp AS value; + """ + + qt_generate_series_multiple """ + SELECT tmp.value + FROM temp_series + LATERAL VIEW py_generate_series(start_val, end_val) tmp AS value + ORDER BY tmp.value; + """ + + // ======================================== + // Test 3: Running Sum UDTF (without state management) + // Note: Function-based UDTFs cannot maintain state across calls + // Each row is processed independently + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_running_sum(INT); """ + sql """ + CREATE TABLES FUNCTION py_running_sum(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "running_sum_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def running_sum_udtf(value): + '''Return value with itself as cumulative sum (stateless)''' + # Note: Function-based UDTF cannot maintain state + # This is simplified to return (value, value) + if value is not None: + yield (value, value) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS numbers_table; """ + sql """ + CREATE TABLE numbers_table ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO numbers_table VALUES + (1, 10), + (2, 20), + (3, 30), + (4, 40); + """ + + qt_running_sum """ + SELECT original_value, cumulative_sum + FROM numbers_table + LATERAL VIEW py_running_sum(value) tmp AS original_value, cumulative_sum + ORDER BY original_value; + """ + + // ======================================== + // Test 4: Explode Array UDTF + // Similar to LATERAL VIEW explode in Hive + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_explode_json_array(STRING); """ + sql """ + CREATE TABLES FUNCTION py_explode_json_array(STRING) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "explode_json_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +import json + +def explode_json_udtf(json_str): + '''Explode JSON ARRAY into rows''' + if json_str: + try: + data = json.loads(json_str) + if isinstance(data, list): + for item in data: + yield (str(item),) + except: + pass # Skip invalid JSON +\$\$; + """ + + sql """ DROP TABLE IF EXISTS temp_json; """ + sql """ + CREATE TABLE temp_json ( + id INT, + json_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_json VALUES (1, '["apple", "banana", "cherry"]'); + """ + + qt_explode_json """ + SELECT element + FROM temp_json + LATERAL VIEW py_explode_json_array(json_data) tmp AS element; + """ + + // ======================================== + // Test 5: Top-N UDTF (stateless version) + // Note: Without state, this simply returns first n values per row + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_top_n(INT, INT); """ + sql """ + CREATE TABLES FUNCTION py_top_n(INT, INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "top_n_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def top_n_udtf(value, n): + '''Return single value with rank 1 (stateless)''' + # Without state, each row is independent + if value is not None and n is not None and n > 0: + yield (value, 1) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS ranked_data; """ + sql """ + CREATE TABLE ranked_data ( + id INT, + category STRING, + value INT, + top_n INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO ranked_data VALUES + (1, 'A', 100, 2), + (2, 'A', 90, 2), + (3, 'A', 80, 2), + (4, 'A', 70, 2), + (5, 'B', 200, 2), + (6, 'B', 190, 2); + """ + + qt_top_n """ + SELECT category, tmp.value, tmp.rank + FROM ranked_data + LATERAL VIEW py_top_n(value, top_n) tmp AS value, rank + ORDER BY category, tmp.rank; + """ + + // ======================================== + // Test 6: Multiple Outputs per Input + // One input row can generate multiple output rows + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_duplicate_n_times(STRING, INT); """ + sql """ + CREATE TABLES FUNCTION py_duplicate_n_times(STRING, INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "duplicate_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def duplicate_udtf(text, n): + '''Duplicate input text N times''' + if text and n: + for i in range(n): + yield (text, i + 1) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS temp_dup; """ + sql """ + CREATE TABLE temp_dup ( + id INT, + text STRING, + times INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_dup VALUES (1, 'Hello', 3); + """ + + qt_duplicate """ + SELECT output, idx + FROM temp_dup + LATERAL VIEW py_duplicate_n_times(text, times) tmp AS output, idx; + """ + + // ======================================== + // Test 7: Conditional Output (Skip Rows) + // UDTF can skip rows by not yielding + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_filter_positive(INT); """ + sql """ + CREATE TABLES FUNCTION py_filter_positive(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "filter_positive_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def filter_positive_udtf(value): + '''Only output positive values''' + if value is not None and value > 0: + yield (value,) + # If value <= 0, don't yield (skip this row) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS mixed_numbers; """ + sql """ + CREATE TABLE mixed_numbers ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO mixed_numbers VALUES (1, -5), (2, 0), (3, 3), (4, -2), (5, 7), (6, 1); + """ + + qt_filter_positive """ + SELECT positive_value + FROM mixed_numbers + LATERAL VIEW py_filter_positive(value) tmp AS positive_value + ORDER BY positive_value; + """ + + // ======================================== + // Test 8: Cartesian Product UDTF + // Generate all combinations + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_cartesian(STRING, STRING); """ + sql """ + CREATE TABLES FUNCTION py_cartesian(STRING, STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "cartesian_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def cartesian_udtf(list1, list2): + '''Generate cartesian product of two comma-separated lists''' + if list1 and list2: + items1 = [x.strip() for x in list1.split(',')] + items2 = [y.strip() for y in list2.split(',')] + + for x in items1: + for y in items2: + yield (x, y) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS temp_cart; """ + sql """ + CREATE TABLE temp_cart ( + id INT, + list1 STRING, + list2 STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_cart VALUES (1, 'A,B', 'X,Y,Z'); + """ + + qt_cartesian """ + SELECT item1, item2 + FROM temp_cart + LATERAL VIEW py_cartesian(list1, list2) tmp AS item1, item2 + ORDER BY item1, item2; + """ + + // ======================================== + // Test 9: All Rows Filtered (Empty Output) + // Tests data_batch = None case + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_filter_negative(INT); """ + sql """ + CREATE TABLES FUNCTION py_filter_negative(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "filter_negative_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def filter_negative_udtf(value): + '''Only output negative values (filter all positive numbers)''' + if value is not None and value < 0: + yield (value,) + # For positive numbers, don't yield anything +\$\$; + """ + + sql """ DROP TABLE IF EXISTS temp_all_positive; """ + sql """ + CREATE TABLE temp_all_positive ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // Insert only positive numbers - all should be filtered + sql """ + INSERT INTO temp_all_positive VALUES (1, 10), (2, 20), (3, 30); + """ + + // Expected: No output rows (all filtered), but should not crash + qt_all_filtered """ + SELECT id, neg_value + FROM temp_all_positive + LATERAL VIEW py_filter_negative(value) tmp AS neg_value + ORDER BY id; + """ + + // ======================================== + // Test 10: Mixed - Some Filtered, Some Not + // ======================================== + sql """ DROP TABLE IF EXISTS temp_mixed; """ + sql """ + CREATE TABLE temp_mixed ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // Mix of positive and negative - only negative should pass + sql """ + INSERT INTO temp_mixed VALUES (1, 10), (2, -5), (3, 20), (4, -3); + """ + + qt_mixed_filter """ + SELECT id, neg_value + FROM temp_mixed + LATERAL VIEW py_filter_negative(value) tmp AS neg_value + ORDER BY id, neg_value; + """ + + // ======================================== + // Test 11: Empty Input Table + // Tests empty batch case + // ======================================== + sql """ DROP TABLE IF EXISTS temp_empty; """ + sql """ + CREATE TABLE temp_empty ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // No data inserted - empty table + qt_empty_input """ + SELECT id, neg_value + FROM temp_empty + LATERAL VIEW py_filter_negative(value) tmp AS neg_value; + """ + + // ======================================== + // Test 12: always_nullable = true (default) + // Function can return NULL even with NOT NULL input + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_nullable_processor(INT); """ + sql """ + CREATE TABLES FUNCTION py_nullable_processor(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "nullable_processor_udtf", + "runtime_version" = "3.10.12", + "always_nullable" = "true" + ) + AS \$\$ +def nullable_processor_udtf(value): + '''Return NULL for even numbers, value for odd numbers''' + if value is None: + yield (None,) + elif value % 2 == 0: + yield (None,) # Return NULL for even numbers + else: + yield (value,) # Return original value for odd numbers +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_nullable; """ + sql """ + CREATE TABLE test_nullable ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nullable VALUES (1, 1), (2, 2), (3, 3), (4, 4), (5, 5); + """ + + // Should return NULL for even values, original value for odd + qt_nullable_true """ + SELECT id, result + FROM test_nullable + LATERAL VIEW py_nullable_processor(value) tmp AS result + ORDER BY id; + """ + + // ======================================== + // Test 13: always_nullable = false + // Function guarantees NOT NULL output with NOT NULL input + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_non_nullable_processor(INT); """ + sql """ + CREATE TABLES FUNCTION py_non_nullable_processor(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "non_nullable_processor_udtf", + "runtime_version" = "3.10.12", + "always_nullable" = "false" + ) + AS \$\$ +def non_nullable_processor_udtf(value): + '''Always return non-NULL value, double the input''' + if value is None: + yield (0,) # Return 0 for NULL input + else: + yield (value * 2,) # Always return non-NULL +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_non_nullable; """ + sql """ + CREATE TABLE test_non_nullable ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_non_nullable VALUES (1, 10), (2, 20), (3, 30); + """ + + // Should return doubled values, all NOT NULL + qt_non_nullable_false """ + SELECT id, result + FROM test_non_nullable + LATERAL VIEW py_non_nullable_processor(value) tmp AS result + ORDER BY id; + """ + + // ======================================== + // Test 14: always_nullable with NULL inputs + // Test how both modes handle NULL inputs + // ======================================== + sql """ DROP TABLE IF EXISTS test_null_inputs; """ + sql """ + CREATE TABLE test_null_inputs ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_null_inputs VALUES (1, NULL), (2, 10), (3, NULL), (4, 20); + """ + + // Test with always_nullable = true (can return NULL) + qt_nullable_with_nulls """ + SELECT id, result + FROM test_null_inputs + LATERAL VIEW py_nullable_processor(value) tmp AS result + ORDER BY id; + """ + + // Test with always_nullable = false (converts NULL to 0) + qt_non_nullable_with_nulls """ + SELECT id, result + FROM test_null_inputs + LATERAL VIEW py_non_nullable_processor(value) tmp AS result + ORDER BY id; + """ + + // ======================================== + // Test 15: always_nullable default behavior + // If not specified, should default to true + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_default_nullable(STRING); """ + sql """ + CREATE TABLES FUNCTION py_default_nullable(STRING) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "default_nullable_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def default_nullable_udtf(text): + '''Return NULL for empty strings, uppercase for non-empty''' + if not text or text.strip() == '': + yield (None,) + else: + yield (text.upper(),) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_default_nullable; """ + sql """ + CREATE TABLE test_default_nullable ( + id INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_default_nullable VALUES (1, 'hello'), (2, ''), (3, 'world'), (4, ' '); + """ + + // Should return NULL for empty/blank strings (default nullable behavior) + qt_default_nullable """ + SELECT id, result + FROM test_default_nullable + LATERAL VIEW py_default_nullable(text) tmp AS result + ORDER BY id; + """ + + // ======================================== + // Test 16: always_nullable with multiple outputs + // Test nullable behavior with functions returning multiple rows + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_nullable_explode(STRING); """ + sql """ + CREATE TABLES FUNCTION py_nullable_explode(STRING) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "nullable_explode_udtf", + "runtime_version" = "3.10.12", + "always_nullable" = "true" + ) + AS \$\$ +def nullable_explode_udtf(csv_string): + '''Split CSV and return NULL for empty parts''' + if not csv_string: + return + parts = csv_string.split(',') + for part in parts: + stripped = part.strip() + if stripped: + yield (stripped,) + else: + yield (None,) # Return NULL for empty parts +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_multi_nullable; """ + sql """ + CREATE TABLE test_multi_nullable ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_multi_nullable VALUES + (1, 'a,b,c'), + (2, 'x,,z'), + (3, ',,'); + """ + + // Should return NULL for empty parts in CSV + qt_multi_nullable """ + SELECT id, part + FROM test_multi_nullable + LATERAL VIEW py_nullable_explode(data) tmp AS part + ORDER BY id, part; + """ + + // ======================================== + // Test: Scalar Value Support (New Feature) + // Single-field UDTF can yield scalar values directly + // ======================================== + + // Test Case 1: yield scalar (int) + sql """ DROP FUNCTION IF EXISTS py_scalar_int(INT, INT); """ + sql """ + CREATE TABLES FUNCTION py_scalar_int(INT, INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "scalar_int_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def scalar_int_udtf(start, end): + '''Yield scalar integers directly (no tuple wrapping)''' + if start is not None and end is not None: + for i in range(start, end + 1): + yield i # Direct scalar, not (i,) +\$\$; + """ + + qt_scalar_int """ + SELECT tmp.value + FROM (SELECT 1 as start_val, 5 as end_val) t + LATERAL VIEW py_scalar_int(start_val, end_val) tmp AS value + ORDER BY tmp.value; + """ + + // Test Case 2: yield scalar (string) + sql """ DROP FUNCTION IF EXISTS py_scalar_string(STRING); """ + sql """ + CREATE TABLES FUNCTION py_scalar_string(STRING) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "scalar_string_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def scalar_string_udtf(text): + '''Split string and yield parts directly as scalars''' + if text: + for part in text.split(','): + yield part.strip() # Direct string, not (part.strip(),) +\$\$; + """ + + qt_scalar_string """ + SELECT tmp.value + FROM (SELECT 'apple,banana,cherry' as text) t + LATERAL VIEW py_scalar_string(text) tmp AS value + ORDER BY tmp.value; + """ + + // Test Case 3: Mixed - both scalar and tuple should work + sql """ DROP FUNCTION IF EXISTS py_mixed_style(INT); """ + sql """ + CREATE TABLES FUNCTION py_mixed_style(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "mixed_style_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def mixed_style_udtf(n): + '''Test mixing scalar and tuple yields (should both work)''' + if n is not None and n > 0: + # First yield as scalar + yield n + # Then yield as tuple + yield (n * 2,) + # Then scalar again + yield n * 3 +\$\$; + """ + + qt_mixed_style """ + SELECT tmp.value + FROM (SELECT 10 as n) t + LATERAL VIEW py_mixed_style(n) tmp AS value + ORDER BY tmp.value; + """ + + // Test Case 4: return scalar (not yield) + sql """ DROP FUNCTION IF EXISTS py_return_scalar(STRING); """ + sql """ + CREATE TABLES FUNCTION py_return_scalar(STRING) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "return_scalar_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def return_scalar_udtf(text): + '''Return scalar value instead of yielding''' + if text: + return text.upper() # Direct return, not (text.upper(),) +\$\$; + """ + + qt_return_scalar """ + SELECT tmp.value + FROM (SELECT 'hello' as text) t + LATERAL VIEW py_return_scalar(text) tmp AS value; + """ + + // Test Case 5: Verify multi-field still requires tuples + sql """ DROP FUNCTION IF EXISTS py_multi_field_check(INT); """ + sql """ + CREATE TABLES FUNCTION py_multi_field_check(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "multi_field_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def multi_field_udtf(n): + '''Multi-field UDTF still requires tuples''' + if n is not None: + yield (n, n * 2) # Must be tuple for multi-field +\$\$; + """ + + qt_multi_field_check """ + SELECT tmp.original, tmp.doubled + FROM (SELECT 42 as n) t + LATERAL VIEW py_multi_field_check(n) tmp AS original, doubled; + """ + + // ======================================== + // Test: OUTER Semantics + // When function is registered as func, both func and func_outer are available + // func: skips NULL/empty results (no output row) + // func_outer: outputs NULL for NULL/empty results (guaranteed output row) + // ======================================== + + // Test Case 1: Simple UDTF with NULL handling + sql """ DROP FUNCTION IF EXISTS py_process_value(INT); """ + sql """ + CREATE TABLES FUNCTION py_process_value(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_value_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_value_udtf(value): + '''Process value: if positive, yield doubled value; otherwise yield nothing''' + if value is not None and value > 0: + yield (value * 2,) + # If value is None or <= 0, don't yield anything +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_outer_basic; """ + sql """ + CREATE TABLE test_outer_basic ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_outer_basic VALUES + (1, 10), -- positive: should output 20 + (2, NULL), -- NULL: func skips, func_outer outputs NULL + (3, 0), -- zero: func skips, func_outer outputs NULL + (4, -5), -- negative: func skips, func_outer outputs NULL + (5, 15); -- positive: should output 30 + """ + + // Test without _outer: NULL/non-positive values are skipped (no output row) + qt_outer_without """ + SELECT id, result + FROM test_outer_basic + LATERAL VIEW py_process_value(value) tmp AS result + ORDER BY id; + """ + + // Test with _outer: NULL/non-positive values output NULL (guaranteed row per input) + qt_outer_with """ + SELECT id, result + FROM test_outer_basic + LATERAL VIEW py_process_value_outer(value) tmp AS result + ORDER BY id; + """ + + // Test Case 2: String split with empty/NULL strings + sql """ DROP FUNCTION IF EXISTS py_split_words(STRING); """ + sql """ + CREATE TABLES FUNCTION py_split_words(STRING) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "split_words_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def split_words_udtf(text): + '''Split text by spaces. Empty/NULL strings yield nothing.''' + if text and text.strip(): + words = text.strip().split() + for word in words: + yield (word,) + # Empty or NULL strings: no output +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_outer_strings; """ + sql """ + CREATE TABLE test_outer_strings ( + id INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_outer_strings VALUES + (1, 'hello world'), -- should split into 2 rows + (2, NULL), -- NULL + (3, ''), -- empty string + (4, ' '), -- whitespace only + (5, 'single'); -- single word + """ + + // Without _outer: only rows 1 and 5 produce output + qt_outer_string_without """ + SELECT id, word + FROM test_outer_strings + LATERAL VIEW py_split_words(text) tmp AS word + ORDER BY id, word; + """ + + // With _outer: all rows produce at least one output (NULL for rows 2,3,4) + qt_outer_string_with """ + SELECT id, word + FROM test_outer_strings + LATERAL VIEW py_split_words_outer(text) tmp AS word + ORDER BY id, word; + """ + + // Test Case 3: Array expansion with empty arrays + sql """ DROP FUNCTION IF EXISTS py_expand_range(INT); """ + sql """ + CREATE TABLES FUNCTION py_expand_range(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "expand_range_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def expand_range_udtf(n): + '''Generate numbers from 1 to n. If n <= 0 or NULL, yield nothing.''' + if n is not None and n > 0: + for i in range(1, n + 1): + yield (i,) + # If n is None or <= 0, no output +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_outer_range; """ + sql """ + CREATE TABLE test_outer_range ( + id INT, + count INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_outer_range VALUES + (1, 3), -- should generate 1,2,3 + (2, NULL), -- NULL + (3, 0), -- zero (no range) + (4, -2), -- negative (no range) + (5, 1); -- should generate 1 + """ + + // Without _outer: only rows 1 and 5 produce output + qt_outer_range_without """ + SELECT id, num + FROM test_outer_range + LATERAL VIEW py_expand_range(count) tmp AS num + ORDER BY id, num; + """ + + // With _outer: all rows produce output (NULL for rows 2,3,4) + qt_outer_range_with """ + SELECT id, num + FROM test_outer_range + LATERAL VIEW py_expand_range_outer(count) tmp AS num + ORDER BY id, num; + """ + + // Test Case 4: Multiple column output with OUTER + sql """ DROP FUNCTION IF EXISTS py_parse_csv(STRING); """ + sql """ + CREATE TABLES FUNCTION py_parse_csv(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "parse_csv_udtf", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def parse_csv_udtf(csv_line): + '''Parse CSV line into field1,field2. Empty/NULL yields nothing.''' + if csv_line and csv_line.strip(): + parts = csv_line.split(',') + if len(parts) >= 2: + yield (parts[0].strip(), parts[1].strip()) + # Empty or invalid CSV: no output +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_outer_multifield; """ + sql """ + CREATE TABLE test_outer_multifield ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_outer_multifield VALUES + (1, 'Alice,30'), -- valid CSV + (2, NULL), -- NULL + (3, ''), -- empty + (4, 'Bob'), -- incomplete CSV (only 1 field) + (5, 'Charlie,25'); -- valid CSV + """ + + // Without _outer: only rows 1 and 5 produce output + qt_outer_multifield_without """ + SELECT id, field1, field2 + FROM test_outer_multifield + LATERAL VIEW py_parse_csv(data) tmp AS field1, field2 + ORDER BY id; + """ + + // With _outer: all rows produce output (NULL,NULL for rows 2,3,4) + qt_outer_multifield_with """ + SELECT id, field1, field2 + FROM test_outer_multifield + LATERAL VIEW py_parse_csv_outer(data) tmp AS field1, field2 + ORDER BY id; + """ + + // Test Case 5: Combining regular and outer table functions + sql """ DROP TABLE IF EXISTS test_outer_combined; """ + sql """ + CREATE TABLE test_outer_combined ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_outer_combined VALUES + (1, 5), + (2, NULL), + (3, 3); + """ + + // Mix regular and outer: regular filters, outer preserves + qt_outer_mixed_functions """ + SELECT id, r1.num as regular_num, r2.num as outer_num + FROM test_outer_combined + LATERAL VIEW py_expand_range(value) r1 AS num + LATERAL VIEW py_expand_range_outer(value) r2 AS num + ORDER BY id, regular_num, outer_num; + """ + + // Test Case 6: Verify outer behavior with built-in functions + sql """ DROP TABLE IF EXISTS test_outer_builtin; """ + sql """ + CREATE TABLE test_outer_builtin ( + id INT, + arr ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_outer_builtin VALUES + (1, [1, 2, 3]), -- normal array + (2, NULL), -- NULL array + (3, []), -- empty array + (4, [5]); -- single element + """ + + // Built-in explode (no outer): skips NULL and empty + qt_outer_builtin_explode """ + SELECT id, elem + FROM test_outer_builtin + LATERAL VIEW explode(arr) tmp AS elem + ORDER BY id, elem; + """ + + // Built-in explode_outer: preserves NULL and empty rows + qt_outer_builtin_explode_outer """ + SELECT id, elem + FROM test_outer_builtin + LATERAL VIEW explode_outer(arr) tmp AS elem + ORDER BY id, elem; + """ + + // Test Case 7: Documentation example - LEFT OUTER JOIN semantics + sql """ DROP TABLE IF EXISTS orders; """ + sql """ + CREATE TABLE orders ( + order_id INT, + items STRING + ) ENGINE=OLAP + DUPLICATE KEY(order_id) + DISTRIBUTED BY HASH(order_id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO orders VALUES + (1, 'apple,banana'), -- order with items + (2, NULL), -- order with NULL items + (3, ''), -- order with empty items + (4, 'cherry'); -- order with one item + """ + + // Without outer: orders 2 and 3 disappear (like INNER JOIN) + qt_outer_doc_inner """ + SELECT order_id, item + FROM orders + LATERAL VIEW py_split_words(items) tmp AS item + ORDER BY order_id, item; + """ + + // With outer: all orders preserved (like LEFT OUTER JOIN) + qt_outer_doc_outer """ + SELECT order_id, item + FROM orders + LATERAL VIEW py_split_words_outer(items) tmp AS item + ORDER BY order_id, item; + """ + + } finally { + try_sql("DROP FUNCTION IF EXISTS py_split_string(STRING);") + try_sql("DROP FUNCTION IF EXISTS py_generate_series(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS py_running_sum(INT);") + try_sql("DROP FUNCTION IF EXISTS py_explode_json_array(STRING);") + try_sql("DROP FUNCTION IF EXISTS py_top_n(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS py_duplicate_n_times(STRING, INT);") + try_sql("DROP FUNCTION IF EXISTS py_filter_positive(INT);") + try_sql("DROP FUNCTION IF EXISTS py_cartesian(STRING, STRING);") + try_sql("DROP FUNCTION IF EXISTS py_filter_negative(INT);") + try_sql("DROP FUNCTION IF EXISTS py_nullable_processor(INT);") + try_sql("DROP FUNCTION IF EXISTS py_non_nullable_processor(INT);") + try_sql("DROP FUNCTION IF EXISTS py_default_nullable(STRING);") + try_sql("DROP FUNCTION IF EXISTS py_nullable_explode(STRING);") + try_sql("DROP FUNCTION IF EXISTS py_scalar_int(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS py_scalar_string(STRING);") + try_sql("DROP FUNCTION IF EXISTS py_mixed_style(INT);") + try_sql("DROP FUNCTION IF EXISTS py_return_scalar(STRING);") + try_sql("DROP FUNCTION IF EXISTS py_multi_field_check(INT);") + try_sql("DROP FUNCTION IF EXISTS py_process_value(INT);") + try_sql("DROP FUNCTION IF EXISTS py_split_words(STRING);") + try_sql("DROP FUNCTION IF EXISTS py_expand_range(INT);") + try_sql("DROP FUNCTION IF EXISTS py_parse_csv(STRING);") + try_sql("DROP TABLE IF EXISTS temp_input;") + try_sql("DROP TABLE IF EXISTS numbers_table;") + try_sql("DROP TABLE IF EXISTS ranked_data;") + try_sql("DROP TABLE IF EXISTS mixed_numbers;") + try_sql("DROP TABLE IF EXISTS temp_all_positive;") + try_sql("DROP TABLE IF EXISTS temp_mixed;") + try_sql("DROP TABLE IF EXISTS temp_empty;") + try_sql("DROP TABLE IF EXISTS test_nullable;") + try_sql("DROP TABLE IF EXISTS test_non_nullable;") + try_sql("DROP TABLE IF EXISTS test_null_inputs;") + try_sql("DROP TABLE IF EXISTS test_default_nullable;") + try_sql("DROP TABLE IF EXISTS test_multi_nullable;") + try_sql("DROP TABLE IF EXISTS test_outer_basic;") + try_sql("DROP TABLE IF EXISTS test_outer_strings;") + try_sql("DROP TABLE IF EXISTS test_outer_range;") + try_sql("DROP TABLE IF EXISTS test_outer_multifield;") + try_sql("DROP TABLE IF EXISTS test_outer_combined;") + try_sql("DROP TABLE IF EXISTS test_outer_builtin;") + try_sql("DROP TABLE IF EXISTS orders;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_basic_module.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_basic_module.groovy new file mode 100644 index 00000000000000..5f586c8f71c220 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_basic_module.groovy @@ -0,0 +1,475 @@ +// 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. + +suite("test_pythonudtf_basic_module") { + // Basic Python UDTF tests using module-based deployment + + def pyPath = """${context.file.parent}/udtf_scripts/pyudtf.zip""" + scp_udf_file_to_all_be(pyPath) + def runtime_version = "3.10.12" + log.info("Python zip path: ${pyPath}".toString()) + + try { + // ======================================== + // Test 1: Simple String Split UDTF + // Input: Single string + // Output: Multiple rows (one per split part) + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_split_string_module(STRING); """ + sql """ + CREATE TABLES FUNCTION py_split_string_module(STRING) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.split_string_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS temp_input_module; """ + sql """ + CREATE TABLE temp_input_module ( + id INT, + input_str STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_input_module VALUES (1, 'apple,banana,cherry'); + """ + + qt_split_string """ + SELECT part + FROM temp_input_module + LATERAL VIEW py_split_string_module(input_str) tmp AS part; + """ + + // ======================================== + // Test 2: Generate Series UDTF + // Input: start, end integers + // Output: Sequence of integers + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_generate_series_module(INT, INT); """ + sql """ + CREATE TABLES FUNCTION py_generate_series_module(INT, INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.generate_series_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS temp_series_module; """ + sql """ + CREATE TABLE temp_series_module ( + id INT, + start_val INT, + end_val INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_series_module VALUES (1, 1, 5), (2, 10, 12); + """ + + qt_generate_series """ + SELECT tmp.value + FROM temp_series_module + LATERAL VIEW py_generate_series_module(start_val, end_val) tmp AS value; + """ + + qt_generate_series_multiple """ + SELECT tmp.value + FROM temp_series_module + LATERAL VIEW py_generate_series_module(start_val, end_val) tmp AS value + ORDER BY tmp.value; + """ + + // ======================================== + // Test 3: Running Sum UDTF (without state management) + // Note: Function-based UDTFs cannot maintain state across calls + // Each row is processed independently + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_running_sum_module(INT); """ + sql """ + CREATE TABLES FUNCTION py_running_sum_module(INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.running_sum_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS numbers_table_module; """ + sql """ + CREATE TABLE numbers_table_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO numbers_table_module VALUES + (1, 10), + (2, 20), + (3, 30), + (4, 40); + """ + + qt_running_sum """ + SELECT original_value, cumulative_sum + FROM numbers_table_module + LATERAL VIEW py_running_sum_module(value) tmp AS original_value, cumulative_sum + ORDER BY original_value; + """ + + // ======================================== + // Test 4: Explode Array UDTF + // Similar to LATERAL VIEW explode in Hive + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_explode_json_array_module(STRING); """ + sql """ + CREATE TABLES FUNCTION py_explode_json_array_module(STRING) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.explode_json_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS temp_json_module; """ + sql """ + CREATE TABLE temp_json_module ( + id INT, + json_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_json_module VALUES (1, '["apple", "banana", "cherry"]'); + """ + + qt_explode_json """ + SELECT element + FROM temp_json_module + LATERAL VIEW py_explode_json_array_module(json_data) tmp AS element; + """ + + // ======================================== + // Test 5: Top-N UDTF (stateless version) + // Note: Without state, this simply returns first n values per row + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_top_n_module(INT, INT); """ + sql """ + CREATE TABLES FUNCTION py_top_n_module(INT, INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.top_n_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS ranked_data_module; """ + sql """ + CREATE TABLE ranked_data_module ( + id INT, + category STRING, + value INT, + top_n INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO ranked_data_module VALUES + (1, 'A', 100, 2), + (2, 'A', 90, 2), + (3, 'A', 80, 2), + (4, 'A', 70, 2), + (5, 'B', 200, 2), + (6, 'B', 190, 2); + """ + + qt_top_n """ + SELECT category, tmp.value, tmp.rank + FROM ranked_data_module + LATERAL VIEW py_top_n_module(value, top_n) tmp AS value, rank + ORDER BY category, tmp.rank; + """ + + // ======================================== + // Test 6: Multiple Outputs per Input + // One input row can generate multiple output rows + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_duplicate_n_times_module(STRING, INT); """ + sql """ + CREATE TABLES FUNCTION py_duplicate_n_times_module(STRING, INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.duplicate_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS temp_dup_module; """ + sql """ + CREATE TABLE temp_dup_module ( + id INT, + text STRING, + times INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_dup_module VALUES (1, 'Hello', 3); + """ + + qt_duplicate """ + SELECT output, idx + FROM temp_dup_module + LATERAL VIEW py_duplicate_n_times_module(text, times) tmp AS output, idx; + """ + + // ======================================== + // Test 7: Conditional Output (Skip Rows) + // UDTF can skip rows by not yielding + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_filter_positive_module(INT); """ + sql """ + CREATE TABLES FUNCTION py_filter_positive_module(INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.filter_positive_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS mixed_numbers_module; """ + sql """ + CREATE TABLE mixed_numbers_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO mixed_numbers_module VALUES (1, -5), (2, 0), (3, 3), (4, -2), (5, 7), (6, 1); + """ + + qt_filter_positive """ + SELECT positive_value + FROM mixed_numbers_module + LATERAL VIEW py_filter_positive_module(value) tmp AS positive_value + ORDER BY positive_value; + """ + + // ======================================== + // Test 8: Cartesian Product UDTF + // Generate all combinations + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_cartesian_module(STRING, STRING); """ + sql """ + CREATE TABLES FUNCTION py_cartesian_module(STRING, STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.cartesian_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS temp_cart_module; """ + sql """ + CREATE TABLE temp_cart_module ( + id INT, + list1 STRING, + list2 STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO temp_cart_module VALUES (1, 'A,B', 'X,Y,Z'); + """ + + qt_cartesian """ + SELECT item1, item2 + FROM temp_cart_module + LATERAL VIEW py_cartesian_module(list1, list2) tmp AS item1, item2 + ORDER BY item1, item2; + """ + + // ======================================== + // Test 9: All Rows Filtered (Empty Output) + // Tests data_batch = None case + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_filter_negative_module(INT); """ + sql """ + CREATE TABLES FUNCTION py_filter_negative_module(INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.filter_negative_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS temp_all_positive_module; """ + sql """ + CREATE TABLE temp_all_positive_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // Insert only positive numbers - all should be filtered + sql """ + INSERT INTO temp_all_positive_module VALUES (1, 10), (2, 20), (3, 30); + """ + + // Expected: No output rows (all filtered), but should not crash + qt_all_filtered """ + SELECT id, neg_value + FROM temp_all_positive_module + LATERAL VIEW py_filter_negative_module(value) tmp AS neg_value + ORDER BY id; + """ + + // ======================================== + // Test 10: Mixed - Some Filtered, Some Not + // ======================================== + sql """ DROP TABLE IF EXISTS temp_mixed_module; """ + sql """ + CREATE TABLE temp_mixed_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // Mix of positive and negative - only negative should pass + sql """ + INSERT INTO temp_mixed_module VALUES (1, 10), (2, -5), (3, 20), (4, -3); + """ + + qt_mixed_filter """ + SELECT id, neg_value + FROM temp_mixed_module + LATERAL VIEW py_filter_negative_module(value) tmp AS neg_value + ORDER BY id, neg_value; + """ + + // ======================================== + // Test 11: Empty Input Table + // Tests empty batch case + // ======================================== + sql """ DROP TABLE IF EXISTS temp_empty_module; """ + sql """ + CREATE TABLE temp_empty_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // No data inserted - empty table + qt_empty_input """ + SELECT id, neg_value + FROM temp_empty_module + LATERAL VIEW py_filter_negative_module(value) tmp AS neg_value; + """ + + // ======================================== + // Test 12: always_nullable = true (default) + // Function can return NULL even with NOT NULL input + // ======================================== + sql """ DROP FUNCTION IF EXISTS py_nullable_processor_module(INT); """ + sql """ + CREATE TABLES FUNCTION py_nullable_processor_module(INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.basic_udtf.nullable_processor_udtf", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}", + "always_nullable" = "true" + ); + """ + + } finally { + try_sql("DROP FUNCTION IF EXISTS py_split_string_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS py_generate_series_module(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS py_running_sum_module(INT);") + try_sql("DROP FUNCTION IF EXISTS py_explode_json_array_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS py_top_n_module(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS py_duplicate_n_times_module(STRING, INT);") + try_sql("DROP FUNCTION IF EXISTS py_filter_positive_module(INT);") + try_sql("DROP FUNCTION IF EXISTS py_cartesian_module(STRING, STRING);") + try_sql("DROP FUNCTION IF EXISTS py_filter_negative_module(INT);") + try_sql("DROP TABLE IF EXISTS temp_input_module;") + try_sql("DROP TABLE IF EXISTS numbers_table_module;") + try_sql("DROP TABLE IF EXISTS ranked_data_module;") + try_sql("DROP TABLE IF EXISTS mixed_numbers_module;") + try_sql("DROP TABLE IF EXISTS temp_all_positive_module;") + try_sql("DROP TABLE IF EXISTS temp_mixed_module;") + try_sql("DROP TABLE IF EXISTS temp_empty_module;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_data_types_inline.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_data_types_inline.groovy new file mode 100644 index 00000000000000..4c05ecc841d444 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_data_types_inline.groovy @@ -0,0 +1,979 @@ +// 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. + +suite("test_pythonudtf_data_types_inline") { + // Test Python UDTF with Various Data Types + // Coverage: Basic types, numeric types, date/time types, complex types + + def runtime_version = "3.10.12" + + try { + // ======================================== + // Type 1: TINYINT (1-byte integer: -128 to 127) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_tinyint(TINYINT); """ + sql """ + CREATE TABLES FUNCTION udtf_tinyint(TINYINT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_tinyint", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_tinyint(v): + '''Process TINYINT: test small integer range''' + if v is not None: + yield (v, v * 2) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_tinyint; """ + sql """ + CREATE TABLE test_tinyint ( + id INT, + v TINYINT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_tinyint VALUES (1, -60), (2, 0), (3, 63); + """ + + qt_tinyint """ + SELECT tmp.original, tmp.doubled + FROM test_tinyint + LATERAL VIEW udtf_tinyint(v) tmp AS original, doubled + ORDER BY tmp.original; + """ + + // ======================================== + // Type 2: SMALLINT (2-byte integer: -32768 to 32767) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_smallint(SMALLINT); """ + sql """ + CREATE TABLES FUNCTION udtf_smallint(SMALLINT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_smallint", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_smallint(v): + '''Process SMALLINT: test medium integer range''' + if v is not None: + yield (v, v * v) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_smallint; """ + sql """ + CREATE TABLE test_smallint ( + id INT, + v SMALLINT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_smallint VALUES (1, -1000), (2, 0), (3, 1000); + """ + + qt_smallint """ + SELECT tmp.original, tmp.squared + FROM test_smallint + LATERAL VIEW udtf_smallint(v) tmp AS original, squared + ORDER BY tmp.original; + """ + + // ======================================== + // Type 3: BIGINT (8-byte integer) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_bigint(BIGINT); """ + sql """ + CREATE TABLES FUNCTION udtf_bigint(BIGINT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_bigint", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_bigint(v): + '''Process BIGINT: test large integer range''' + if v is not None: + yield (v, v + 1) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_bigint; """ + sql """ + CREATE TABLE test_bigint ( + id INT, + v BIGINT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_bigint VALUES (1, -1000000000000), (2, 0), (3, 1000000000000); + """ + + qt_bigint """ + SELECT tmp.original, tmp.incremented + FROM test_bigint + LATERAL VIEW udtf_bigint(v) tmp AS original, incremented + ORDER BY tmp.original; + """ + + // ======================================== + // Type 4: FLOAT (4-byte floating point) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_float(FLOAT); """ + sql """ + CREATE TABLES FUNCTION udtf_float(FLOAT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_float", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_float(v): + '''Process FLOAT: test floating point numbers''' + if v is not None: + yield (v, v / 2.0) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_float; """ + sql """ + CREATE TABLE test_float ( + id INT, + v FLOAT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_float VALUES (1, -3.14), (2, 0.0), (3, 2.718); + """ + + qt_float """ + SELECT tmp.original, tmp.halved + FROM test_float + LATERAL VIEW udtf_float(v) tmp AS original, halved + ORDER BY tmp.original; + """ + + // ======================================== + // Type 5: DOUBLE (8-byte floating point) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_double(DOUBLE); """ + sql """ + CREATE TABLES FUNCTION udtf_double(DOUBLE) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_double", + "runtime_version" = "3.10.12" + ) + AS \$\$ +import math + +def process_double(v): + '''Process DOUBLE: test high precision floating point''' + if v is not None and v >= 0: + yield (v, math.sqrt(v)) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_double; """ + sql """ + CREATE TABLE test_double ( + id INT, + v DOUBLE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_double VALUES (1, 0.0), (2, 4.0), (3, 16.0), (4, 100.0); + """ + + qt_double """ + SELECT tmp.original, tmp.sqrt_value + FROM test_double + LATERAL VIEW udtf_double(v) tmp AS original, sqrt_value + ORDER BY tmp.original; + """ + + // ======================================== + // Type 6: BOOLEAN + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_boolean(BOOLEAN); """ + sql """ + CREATE TABLES FUNCTION udtf_boolean(BOOLEAN) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_boolean", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_boolean(v): + '''Process BOOLEAN: test true/false values''' + if v is not None: + yield (v, not v, 'TRUE' if v else 'FALSE') +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_boolean; """ + sql """ + CREATE TABLE test_boolean ( + id INT, + v BOOLEAN + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_boolean VALUES (1, true), (2, false); + """ + + qt_boolean """ + SELECT tmp.original, tmp.negated, tmp.as_string + FROM test_boolean + LATERAL VIEW udtf_boolean(v) tmp AS original, negated, as_string + ORDER BY tmp.original; + """ + + // ======================================== + // Type 7: STRING (Variable length text) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_string(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_string(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_string", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_string(v): + '''Process STRING: test text manipulation''' + if v is not None: + yield (v, len(v), v.upper(), v.lower()) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_string; """ + sql """ + CREATE TABLE test_string ( + id INT, + v STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_string VALUES (1, 'Hello'), (2, 'WORLD'), (3, 'DoRiS'); + """ + + qt_string """ + SELECT tmp.original, tmp.length, tmp.upper, tmp.lower + FROM test_string + LATERAL VIEW udtf_string(v) tmp AS original, length, upper, lower + ORDER BY tmp.original; + """ + + // ======================================== + // Type 8: DATE (Date without time) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_date(DATE); """ + sql """ + CREATE TABLES FUNCTION udtf_date(DATE) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_date", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_date(v): + '''Process DATE: extract date components''' + if v is not None: + # v is a datetime.date object + yield (v, v.year, v.month, v.day) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_date; """ + sql """ + CREATE TABLE test_date ( + id INT, + v DATE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_date VALUES (1, '2024-01-01'), (2, '2024-06-15'), (3, '2024-12-31'); + """ + + qt_date """ + SELECT tmp.original, tmp.year, tmp.month, tmp.day + FROM test_date + LATERAL VIEW udtf_date(v) tmp AS original, year, month, day + ORDER BY tmp.original; + """ + + // ======================================== + // Type 9: DATETIME (Date with time) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_datetime(DATETIME); """ + sql """ + CREATE TABLES FUNCTION udtf_datetime(DATETIME) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_datetime", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_datetime(v): + '''Process DATETIME: extract time components''' + if v is not None: + # v is a datetime.datetime object + yield (v, v.hour, v.minute) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_datetime; """ + sql """ + CREATE TABLE test_datetime ( + id INT, + v DATETIME + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_datetime VALUES + (1, '2024-01-01 08:30:00'), + (2, '2024-06-15 12:00:00'), + (3, '2024-12-31 23:59:00'); + """ + + qt_datetime """ + SELECT tmp.original, tmp.hour, tmp.minute + FROM test_datetime + LATERAL VIEW udtf_datetime(v) tmp AS original, hour, minute + ORDER BY tmp.original; + """ + + // ======================================== + // Type 10: ARRAY (Array of integers) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_array_int(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_array_int(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_array_int", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_array_int(arr): + '''Process ARRAY: explode array and process each element''' + if arr is not None: + for i, elem in enumerate(arr): + if elem is not None: + yield (i, elem, elem * 2) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_array_int; """ + sql """ + CREATE TABLE test_array_int ( + id INT, + v ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_int VALUES + (1, [1, 2, 3]), + (2, [10, 20]), + (3, [100]); + """ + + qt_array_int """ + SELECT id, tmp.arr_pos, tmp.element, tmp.doubled + FROM test_array_int + LATERAL VIEW udtf_array_int(v) tmp AS arr_pos, element, doubled + ORDER BY id, tmp.arr_pos; + """ + + // ======================================== + // Type 11: ARRAY (Array of strings) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_array_string(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_array_string(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_array_string", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_array_string(arr): + '''Process ARRAY: explode and get string lengths''' + if arr is not None: + for elem in arr: + if elem is not None: + yield (elem, len(elem)) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_array_string; """ + sql """ + CREATE TABLE test_array_string ( + id INT, + v ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_string VALUES + (1, ['apple', 'banana']), + (2, ['cat', 'dog', 'bird']); + """ + + qt_array_string """ + SELECT id, tmp.element, tmp.length + FROM test_array_string + LATERAL VIEW udtf_array_string(v) tmp AS element, length + ORDER BY id, tmp.element; + """ + + // ======================================== + // Type 12: STRUCT (Structured data) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_struct(STRUCT); """ + sql """ + CREATE TABLES FUNCTION udtf_struct(STRUCT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_struct", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_struct(person): + '''Process STRUCT: access struct fields''' + if person is not None: + name = person['name'] if 'name' in person else None + age = person['age'] if 'age' in person else None + + if name is not None and age is not None: + category = 'child' if age < 18 else 'adult' + yield (name, age, category) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_struct; """ + sql """ + CREATE TABLE test_struct ( + id INT, + person STRUCT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_struct VALUES + (1, named_struct('name', 'Alice', 'age', 25)), + (2, named_struct('name', 'Bob', 'age', 15)), + (3, named_struct('name', 'Charlie', 'age', 30)); + """ + + qt_struct """ + SELECT tmp.name, tmp.age, tmp.category + FROM test_struct + LATERAL VIEW udtf_struct(person) tmp AS name, age, category + ORDER BY tmp.name; + """ + + // ======================================== + // Type 13: Multiple Input Types (INT, STRING) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_multi_types(INT, STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_multi_types(INT, STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_multi_types", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_multi_types(num, text): + '''Process multiple input types''' + if num is not None and text is not None: + yield (num, text, f"{text}_{num}") +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_multi_types; """ + sql """ + CREATE TABLE test_multi_types ( + id INT, + num INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_multi_types VALUES (1, 100, 'apple'), (2, 200, 'banana'); + """ + + qt_multi_types """ + SELECT tmp.number, tmp.text, tmp.combined + FROM test_multi_types + LATERAL VIEW udtf_multi_types(num, text) tmp AS number, text, combined + ORDER BY tmp.number; + """ + + // ======================================== + // Type 14: DECIMAL (High precision decimal) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_decimal(DECIMAL(10,2)); """ + sql """ + CREATE TABLES FUNCTION udtf_decimal(DECIMAL(10,2)) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_decimal", + "runtime_version" = "3.10.12" + ) + AS \$\$ +from decimal import Decimal + +def process_decimal(v): + '''Process DECIMAL: high precision arithmetic''' + if v is not None: + doubled = v * 2 + yield (v, doubled) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_decimal; """ + sql """ + CREATE TABLE test_decimal ( + id INT, + v DECIMAL(10,2) + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_decimal VALUES (1, 123.45), (2, 678.90), (3, 999.99); + """ + + qt_decimal """ + SELECT tmp.original, tmp.doubled + FROM test_decimal + LATERAL VIEW udtf_decimal(v) tmp AS original, doubled + ORDER BY tmp.original; + """ + + // ======================================== + // Section: P1 - Complex Data Types + // ======================================== + + // Test P1.1: MAP type (if supported) + // Note: Doris may not fully support MAP in UDTF, test with workaround + sql """ DROP FUNCTION IF EXISTS udtf_map_processor(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_map_processor(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_map_string", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_map_string(map_str): + '''Process map-like string (key1:val1,key2:val2)''' + if map_str: + pairs = map_str.split(',') + for pair in pairs: + if ':' in pair: + k, val = pair.split(':', 1) + try: + yield (k.strip(), int(val.strip())) + except ValueError: + pass +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_map_like; """ + sql """ + CREATE TABLE test_map_like ( + id INT, + map_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_map_like VALUES + (1, 'age:25,score:90'), + (2, 'age:30,score:85,level:3'); + """ + + qt_map_like """ + SELECT id, tmp.k, tmp.v + FROM test_map_like + LATERAL VIEW udtf_map_processor(map_data) tmp AS k, v + ORDER BY id, tmp.k; + """ + + // Test P1.2: Nested ARRAY (ARRAY> simulated) + sql """ DROP FUNCTION IF EXISTS udtf_nested_array(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_nested_array(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_nested_array", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_nested_array(nested_str): + '''Process nested array string ([[1,2],[3,4]])''' + if nested_str: + # Remove brackets and split by ],[ + nested_str = nested_str.strip('[]') + groups = nested_str.split('],[') + + for group_idx, group in enumerate(groups): + elements = group.strip('[]').split(',') + for elem in elements: + try: + yield (group_idx, int(elem.strip())) + except ValueError: + pass +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_nested_array; """ + sql """ + CREATE TABLE test_nested_array ( + id INT, + nested_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_array VALUES + (1, '[[10,20],[30,40]]'), + (2, '[[50],[60,70,80]]'); + """ + + qt_nested_array """ + SELECT id, tmp.group_idx, tmp.element + FROM test_nested_array + LATERAL VIEW udtf_nested_array(nested_data) tmp AS group_idx, element + ORDER BY id, tmp.group_idx, tmp.element; + """ + + // Test P1.3: ARRAY> + sql """ DROP FUNCTION IF EXISTS udtf_array_of_structs(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_array_of_structs(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_array_structs", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_array_structs(data): + '''Process array of structs (name:age:score|name:age:score)''' + if data: + items = data.split('|') + for item in items: + parts = item.split(':') + if len(parts) == 3: + try: + yield (parts[0], int(parts[1]), int(parts[2])) + except ValueError: + pass +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_array_structs; """ + sql """ + CREATE TABLE test_array_structs ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_structs VALUES + (1, 'Alice:25:90|Bob:30:85'), + (2, 'Charlie:28:88'); + """ + + qt_array_structs """ + SELECT id, tmp.name, tmp.age, tmp.score + FROM test_array_structs + LATERAL VIEW udtf_array_of_structs(data) tmp AS name, age, score + ORDER BY id, tmp.name; + """ + + // Test P1.4: STRUCT with nested ARRAY + sql """ DROP FUNCTION IF EXISTS udtf_struct_with_array(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_struct_with_array(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_struct_array", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_struct_array(data): + '''Process struct with array (name:tag1,tag2,tag3)''' + if data and ':' in data: + name, tags = data.split(':', 1) + tag_list = tags.split(',') + yield (name, len(tag_list), ','.join(tag_list)) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_struct_array; """ + sql """ + CREATE TABLE test_struct_array ( + id INT, + person_tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_struct_array VALUES + (1, 'Alice:sports,music,reading'), + (2, 'Bob:coding,gaming'); + """ + + qt_struct_array """ + SELECT id, tmp.person_name, tmp.tag_count, tmp.tags + FROM test_struct_array + LATERAL VIEW udtf_struct_with_array(person_tags) tmp AS person_name, tag_count, tags + ORDER BY id; + """ + + // Test P1.5: JSON-like data processing + sql """ DROP FUNCTION IF EXISTS udtf_json_extract(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_json_extract(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "extract_json_fields", + "runtime_version" = "3.10.12" + ) + AS \$\$ +import json + +def extract_json_fields(json_str): + '''Extract JSON fields''' + if json_str: + try: + data = json.loads(json_str) + if isinstance(data, dict): + for k, v in data.items(): + yield (k, str(v)) + except: + pass +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_json_data; """ + sql """ + CREATE TABLE test_json_data ( + id INT, + json_content STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_json_data VALUES + (1, '{"name":"Alice","age":25,"city":"NYC"}'), + (2, '{"name":"Bob","age":30}'); + """ + + qt_json_extract """ + SELECT id, tmp.field, tmp.v + FROM test_json_data + LATERAL VIEW udtf_json_extract(json_content) tmp AS field, v + ORDER BY id, tmp.field; + """ + + // Test P1.6: Complex nested STRUCT + sql """ DROP FUNCTION IF EXISTS udtf_complex_struct(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_complex_struct(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_complex_struct", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_complex_struct(data): + '''Process complex struct (id:name:city:zip)''' + if data: + parts = data.split(':') + if len(parts) == 4: + try: + yield (int(parts[0]), parts[1], parts[2], parts[3]) + except ValueError: + pass +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_complex_struct; """ + sql """ + CREATE TABLE test_complex_struct ( + id INT, + user_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_complex_struct VALUES + (1, '101:Alice:NYC:10001'), + (2, '102:Bob:LA:90001'); + """ + + qt_complex_struct """ + SELECT id, tmp.user_id, tmp.user_name, tmp.address_city, tmp.address_zip + FROM test_complex_struct + LATERAL VIEW udtf_complex_struct(user_data) tmp AS user_id, user_name, address_city, address_zip + ORDER BY id; + """ + + } finally { + try_sql("DROP FUNCTION IF EXISTS udtf_tinyint(TINYINT);") + try_sql("DROP FUNCTION IF EXISTS udtf_smallint(SMALLINT);") + try_sql("DROP FUNCTION IF EXISTS udtf_bigint(BIGINT);") + try_sql("DROP FUNCTION IF EXISTS udtf_float(FLOAT);") + try_sql("DROP FUNCTION IF EXISTS udtf_double(DOUBLE);") + try_sql("DROP FUNCTION IF EXISTS udtf_boolean(BOOLEAN);") + try_sql("DROP FUNCTION IF EXISTS udtf_string(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_date(DATE);") + try_sql("DROP FUNCTION IF EXISTS udtf_datetime(DATETIME);") + try_sql("DROP FUNCTION IF EXISTS udtf_array_int(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_array_string(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_struct(STRUCT);") + try_sql("DROP FUNCTION IF EXISTS udtf_multi_types(INT, STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_decimal(DECIMAL(10,2));") + try_sql("DROP FUNCTION IF EXISTS udtf_map_processor(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_nested_array(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_array_of_structs(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_struct_with_array(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_json_extract(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_complex_struct(STRING);") + try_sql("DROP TABLE IF EXISTS test_tinyint;") + try_sql("DROP TABLE IF EXISTS test_smallint;") + try_sql("DROP TABLE IF EXISTS test_bigint;") + try_sql("DROP TABLE IF EXISTS test_float;") + try_sql("DROP TABLE IF EXISTS test_double;") + try_sql("DROP TABLE IF EXISTS test_boolean;") + try_sql("DROP TABLE IF EXISTS test_string;") + try_sql("DROP TABLE IF EXISTS test_date;") + try_sql("DROP TABLE IF EXISTS test_datetime;") + try_sql("DROP TABLE IF EXISTS test_array_int;") + try_sql("DROP TABLE IF EXISTS test_array_string;") + try_sql("DROP TABLE IF EXISTS test_struct;") + try_sql("DROP TABLE IF EXISTS test_multi_types;") + try_sql("DROP TABLE IF EXISTS test_decimal;") + try_sql("DROP TABLE IF EXISTS test_map_like;") + try_sql("DROP TABLE IF EXISTS test_nested_array;") + try_sql("DROP TABLE IF EXISTS test_array_structs;") + try_sql("DROP TABLE IF EXISTS test_struct_array;") + try_sql("DROP TABLE IF EXISTS test_json_data;") + try_sql("DROP TABLE IF EXISTS test_complex_struct;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_data_types_module.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_data_types_module.groovy new file mode 100644 index 00000000000000..b987b11732c50c --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_data_types_module.groovy @@ -0,0 +1,827 @@ +// 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. + +suite("test_pythonudtf_data_types_module") { + // Test Python UDTF with Various Data Types using module-based deployment + // UDTFs are loaded from pyudtf.zip file + + def pyPath = """${context.file.parent}/udtf_scripts/pyudtf.zip""" + scp_udf_file_to_all_be(pyPath) + def runtime_version = "3.10.12" + log.info("Python zip path: ${pyPath}".toString()) + + try { + // ======================================== + // Type 1: TINYINT (1-byte integer: -128 to 127) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_tinyint_module(TINYINT); """ + sql """ + CREATE TABLES FUNCTION udtf_tinyint_module(TINYINT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_tinyint", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_tinyint_module; """ + sql """ + CREATE TABLE test_tinyint_module ( + id INT, + v TINYINT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_tinyint_module VALUES (1, -60), (2, 0), (3, 63); + """ + + qt_tinyint """ + SELECT tmp.original, tmp.doubled + FROM test_tinyint_module + LATERAL VIEW udtf_tinyint_module(v) tmp AS original, doubled + ORDER BY tmp.original; + """ + + // ======================================== + // Type 2: SMALLINT (2-byte integer: -32768 to 32767) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_smallint_module(SMALLINT); """ + sql """ + CREATE TABLES FUNCTION udtf_smallint_module(SMALLINT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_smallint", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_smallint_module; """ + sql """ + CREATE TABLE test_smallint_module ( + id INT, + v SMALLINT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_smallint_module VALUES (1, -1000), (2, 0), (3, 1000); + """ + + qt_smallint """ + SELECT tmp.original, tmp.squared + FROM test_smallint_module + LATERAL VIEW udtf_smallint_module(v) tmp AS original, squared + ORDER BY tmp.original; + """ + + // ======================================== + // Type 3: BIGINT (8-byte integer) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_bigint_module(BIGINT); """ + sql """ + CREATE TABLES FUNCTION udtf_bigint_module(BIGINT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_bigint", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_bigint_module; """ + sql """ + CREATE TABLE test_bigint_module ( + id INT, + v BIGINT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_bigint_module VALUES (1, -1000000000000), (2, 0), (3, 1000000000000); + """ + + qt_bigint """ + SELECT tmp.original, tmp.incremented + FROM test_bigint_module + LATERAL VIEW udtf_bigint_module(v) tmp AS original, incremented + ORDER BY tmp.original; + """ + + // ======================================== + // Type 4: FLOAT (4-byte floating point) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_float_module(FLOAT); """ + sql """ + CREATE TABLES FUNCTION udtf_float_module(FLOAT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_float", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_float_module; """ + sql """ + CREATE TABLE test_float_module ( + id INT, + v FLOAT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_float_module VALUES (1, -3.14), (2, 0.0), (3, 2.718); + """ + + qt_float """ + SELECT tmp.original, tmp.halved + FROM test_float_module + LATERAL VIEW udtf_float_module(v) tmp AS original, halved + ORDER BY tmp.original; + """ + + // ======================================== + // Type 5: DOUBLE (8-byte floating point) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_double_module(DOUBLE); """ + sql """ + CREATE TABLES FUNCTION udtf_double_module(DOUBLE) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_double", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_double_module; """ + sql """ + CREATE TABLE test_double_module ( + id INT, + v DOUBLE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_double_module VALUES (1, 0.0), (2, 4.0), (3, 16.0), (4, 100.0); + """ + + qt_double """ + SELECT tmp.original, tmp.sqrt_value + FROM test_double_module + LATERAL VIEW udtf_double_module(v) tmp AS original, sqrt_value + ORDER BY tmp.original; + """ + + // ======================================== + // Type 6: BOOLEAN + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_boolean_module(BOOLEAN); """ + sql """ + CREATE TABLES FUNCTION udtf_boolean_module(BOOLEAN) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_boolean", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_boolean_module; """ + sql """ + CREATE TABLE test_boolean_module ( + id INT, + v BOOLEAN + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_boolean_module VALUES (1, true), (2, false); + """ + + qt_boolean """ + SELECT tmp.original, tmp.negated, tmp.as_string + FROM test_boolean_module + LATERAL VIEW udtf_boolean_module(v) tmp AS original, negated, as_string + ORDER BY tmp.original; + """ + + // ======================================== + // Type 7: STRING (Variable length text) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_string_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_string_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_string", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_string_module; """ + sql """ + CREATE TABLE test_string_module ( + id INT, + v STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_string_module VALUES (1, 'Hello'), (2, 'WORLD'), (3, 'DoRiS'); + """ + + qt_string """ + SELECT tmp.original, tmp.length, tmp.upper, tmp.lower + FROM test_string_module + LATERAL VIEW udtf_string_module(v) tmp AS original, length, upper, lower + ORDER BY tmp.original; + """ + + // ======================================== + // Type 8: DATE (Date without time) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_date_module(DATE); """ + sql """ + CREATE TABLES FUNCTION udtf_date_module(DATE) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_date", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_date_module; """ + sql """ + CREATE TABLE test_date_module ( + id INT, + v DATE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_date_module VALUES (1, '2024-01-01'), (2, '2024-06-15'), (3, '2024-12-31'); + """ + + qt_date """ + SELECT tmp.original, tmp.year, tmp.month, tmp.day + FROM test_date_module + LATERAL VIEW udtf_date_module(v) tmp AS original, year, month, day + ORDER BY tmp.original; + """ + + // ======================================== + // Type 9: DATETIME (Date with time) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_datetime_module(DATETIME); """ + sql """ + CREATE TABLES FUNCTION udtf_datetime_module(DATETIME) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_datetime", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_datetime_module; """ + sql """ + CREATE TABLE test_datetime_module ( + id INT, + v DATETIME + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_datetime_module VALUES + (1, '2024-01-01 08:30:00'), + (2, '2024-06-15 12:00:00'), + (3, '2024-12-31 23:59:00'); + """ + + qt_datetime """ + SELECT tmp.original, tmp.hour, tmp.minute + FROM test_datetime_module + LATERAL VIEW udtf_datetime_module(v) tmp AS original, hour, minute + ORDER BY tmp.original; + """ + + // ======================================== + // Type 10: ARRAY (Array of integers) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_array_int_module(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_array_int_module(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_array_int", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_array_int_module; """ + sql """ + CREATE TABLE test_array_int_module ( + id INT, + v ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_int_module VALUES + (1, [1, 2, 3]), + (2, [10, 20]), + (3, [100]); + """ + + qt_array_int """ + SELECT id, tmp.arr_pos, tmp.element, tmp.doubled + FROM test_array_int_module + LATERAL VIEW udtf_array_int_module(v) tmp AS arr_pos, element, doubled + ORDER BY id, tmp.arr_pos; + """ + + // ======================================== + // Type 11: ARRAY (Array of strings) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_array_string_module(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_array_string_module(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_array_string", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_array_string_module; """ + sql """ + CREATE TABLE test_array_string_module ( + id INT, + v ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_string_module VALUES + (1, ['apple', 'banana']), + (2, ['cat', 'dog', 'bird']); + """ + + qt_array_string """ + SELECT id, tmp.element, tmp.length + FROM test_array_string_module + LATERAL VIEW udtf_array_string_module(v) tmp AS element, length + ORDER BY id, tmp.element; + """ + + // ======================================== + // Type 12: STRUCT (Structured data) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_struct_module(STRUCT); """ + sql """ + CREATE TABLES FUNCTION udtf_struct_module(STRUCT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_struct", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_struct_module; """ + sql """ + CREATE TABLE test_struct_module ( + id INT, + person STRUCT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_struct_module VALUES + (1, named_struct('name', 'Alice', 'age', 25)), + (2, named_struct('name', 'Bob', 'age', 15)), + (3, named_struct('name', 'Charlie', 'age', 30)); + """ + + qt_struct """ + SELECT tmp.name, tmp.age, tmp.category + FROM test_struct_module + LATERAL VIEW udtf_struct_module(person) tmp AS name, age, category + ORDER BY tmp.name; + """ + + // ======================================== + // Type 13: Multiple Input Types (INT, STRING) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_multi_types_module(INT, STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_multi_types_module(INT, STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_multi_types", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_multi_types_module; """ + sql """ + CREATE TABLE test_multi_types_module ( + id INT, + num INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_multi_types_module VALUES (1, 100, 'apple'), (2, 200, 'banana'); + """ + + qt_multi_types """ + SELECT tmp.number, tmp.text, tmp.combined + FROM test_multi_types_module + LATERAL VIEW udtf_multi_types_module(num, text) tmp AS number, text, combined + ORDER BY tmp.number; + """ + + // ======================================== + // Type 14: DECIMAL (High precision decimal) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_decimal_module(DECIMAL(10,2)); """ + sql """ + CREATE TABLES FUNCTION udtf_decimal_module(DECIMAL(10,2)) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_decimal", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_decimal_module; """ + sql """ + CREATE TABLE test_decimal_module ( + id INT, + v DECIMAL(10,2) + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_decimal_module VALUES (1, 123.45), (2, 678.90), (3, 999.99); + """ + + qt_decimal """ + SELECT tmp.original, tmp.doubled + FROM test_decimal_module + LATERAL VIEW udtf_decimal_module(v) tmp AS original, doubled + ORDER BY tmp.original; + """ + + // ======================================== + // Section: P1 - Complex Data Types + // ======================================== + + // Test P1.1: MAP type (if supported) + // Note: Doris may not fully support MAP in UDTF, test with workaround + sql """ DROP FUNCTION IF EXISTS udtf_map_processor_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_map_processor_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_map_string", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_map_like_module; """ + sql """ + CREATE TABLE test_map_like_module ( + id INT, + map_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_map_like_module VALUES + (1, 'age:25,score:90'), + (2, 'age:30,score:85,level:3'); + """ + + qt_map_like """ + SELECT id, tmp.k, tmp.v + FROM test_map_like_module + LATERAL VIEW udtf_map_processor_module(map_data) tmp AS k, v + ORDER BY id, tmp.k; + """ + + // Test P1.2: Nested ARRAY (ARRAY> simulated) + sql """ DROP FUNCTION IF EXISTS udtf_nested_array_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_nested_array_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_nested_array", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_nested_array_module; """ + sql """ + CREATE TABLE test_nested_array_module ( + id INT, + nested_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_array_module VALUES + (1, '[[10,20],[30,40]]'), + (2, '[[50],[60,70,80]]'); + """ + + qt_nested_array """ + SELECT id, tmp.group_idx, tmp.element + FROM test_nested_array_module + LATERAL VIEW udtf_nested_array_module(nested_data) tmp AS group_idx, element + ORDER BY id, tmp.group_idx, tmp.element; + """ + + // Test P1.3: ARRAY> + sql """ DROP FUNCTION IF EXISTS udtf_array_of_structs_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_array_of_structs_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_array_structs", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_array_structs_module; """ + sql """ + CREATE TABLE test_array_structs_module ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_structs_module VALUES + (1, 'Alice:25:90|Bob:30:85'), + (2, 'Charlie:28:88'); + """ + + qt_array_structs """ + SELECT id, tmp.name, tmp.age, tmp.score + FROM test_array_structs_module + LATERAL VIEW udtf_array_of_structs_module(data) tmp AS name, age, score + ORDER BY id, tmp.name; + """ + + // Test P1.4: STRUCT with nested ARRAY + sql """ DROP FUNCTION IF EXISTS udtf_struct_with_array_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_struct_with_array_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_struct_array", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_struct_array_module; """ + sql """ + CREATE TABLE test_struct_array_module ( + id INT, + person_tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_struct_array_module VALUES + (1, 'Alice:sports,music,reading'), + (2, 'Bob:coding,gaming'); + """ + + qt_struct_array """ + SELECT id, tmp.person_name, tmp.tag_count, tmp.tags + FROM test_struct_array_module + LATERAL VIEW udtf_struct_with_array_module(person_tags) tmp AS person_name, tag_count, tags + ORDER BY id; + """ + + // Test P1.5: JSON-like data processing + sql """ DROP FUNCTION IF EXISTS udtf_json_extract_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_json_extract_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.extract_json_fields", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_json_data_module; """ + sql """ + CREATE TABLE test_json_data_module ( + id INT, + json_content STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_json_data_module VALUES + (1, '{"name":"Alice","age":25,"city":"NYC"}'), + (2, '{"name":"Bob","age":30}'); + """ + + qt_json_extract """ + SELECT id, tmp.field, tmp.v + FROM test_json_data_module + LATERAL VIEW udtf_json_extract_module(json_content) tmp AS field, v + ORDER BY id, tmp.field; + """ + + // Test P1.6: Complex nested STRUCT + sql """ DROP FUNCTION IF EXISTS udtf_complex_struct_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_complex_struct_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.data_types_udtf.process_complex_struct", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_complex_struct_module; """ + sql """ + CREATE TABLE test_complex_struct_module ( + id INT, + user_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_complex_struct_module VALUES + (1, '101:Alice:NYC:10001'), + (2, '102:Bob:LA:90001'); + """ + + qt_complex_struct """ + SELECT id, tmp.user_id, tmp.user_name, tmp.address_city, tmp.address_zip + FROM test_complex_struct_module + LATERAL VIEW udtf_complex_struct_module(user_data) tmp AS user_id, user_name, address_city, address_zip + ORDER BY id; + """ + + } finally { + try_sql("DROP FUNCTION IF EXISTS udtf_tinyint_module(TINYINT);") + try_sql("DROP FUNCTION IF EXISTS udtf_smallint_module(SMALLINT);") + try_sql("DROP FUNCTION IF EXISTS udtf_bigint_module(BIGINT);") + try_sql("DROP FUNCTION IF EXISTS udtf_float_module(FLOAT);") + try_sql("DROP FUNCTION IF EXISTS udtf_double_module(DOUBLE);") + try_sql("DROP FUNCTION IF EXISTS udtf_boolean_module(BOOLEAN);") + try_sql("DROP FUNCTION IF EXISTS udtf_string_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_date_module(DATE);") + try_sql("DROP FUNCTION IF EXISTS udtf_datetime_module(DATETIME);") + try_sql("DROP FUNCTION IF EXISTS udtf_array_int_module(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_array_string_module(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_struct_module(STRUCT);") + try_sql("DROP FUNCTION IF EXISTS udtf_multi_types_module(INT, STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_decimal_module(DECIMAL(10,2));") + try_sql("DROP FUNCTION IF EXISTS udtf_map_processor_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_nested_array_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_array_of_structs_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_struct_with_array_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_json_extract_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_complex_struct_module(STRING);") + try_sql("DROP TABLE IF EXISTS test_tinyint_module;") + try_sql("DROP TABLE IF EXISTS test_smallint_module;") + try_sql("DROP TABLE IF EXISTS test_bigint_module;") + try_sql("DROP TABLE IF EXISTS test_float_module;") + try_sql("DROP TABLE IF EXISTS test_double_module;") + try_sql("DROP TABLE IF EXISTS test_boolean_module;") + try_sql("DROP TABLE IF EXISTS test_string_module;") + try_sql("DROP TABLE IF EXISTS test_date_module;") + try_sql("DROP TABLE IF EXISTS test_datetime_module;") + try_sql("DROP TABLE IF EXISTS test_array_int_module;") + try_sql("DROP TABLE IF EXISTS test_array_string_module;") + try_sql("DROP TABLE IF EXISTS test_struct_module;") + try_sql("DROP TABLE IF EXISTS test_multi_types_module;") + try_sql("DROP TABLE IF EXISTS test_decimal_module;") + try_sql("DROP TABLE IF EXISTS test_map_like_module;") + try_sql("DROP TABLE IF EXISTS test_nested_array_module;") + try_sql("DROP TABLE IF EXISTS test_array_structs_module;") + try_sql("DROP TABLE IF EXISTS test_struct_array_module;") + try_sql("DROP TABLE IF EXISTS test_json_data_module;") + try_sql("DROP TABLE IF EXISTS test_complex_struct_module;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_edge_cases_inline.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_edge_cases_inline.groovy new file mode 100644 index 00000000000000..f0cbacfb7859a2 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_edge_cases_inline.groovy @@ -0,0 +1,701 @@ +// 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. + +suite("test_pythonudtf_edge_cases_inline") { + // Test Python UDTF Edge Cases and Boundary Conditions + // Coverage: NULL handling, extreme cases, special values + + def runtime_version = "3.10.12" + + try { + // ======================================== + // Section 1: NULL Value Handling + // ======================================== + + // Test 1.1: NULL Integer Input + sql """ DROP FUNCTION IF EXISTS udtf_null_int(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_null_int(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "handle_null_int", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def handle_null_int(value): + '''Handle NULL integer values''' + if value is None: + yield (None, True, -1) # NULL indicator + else: + yield (value, False, value * 2) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_null_int; """ + sql """ + CREATE TABLE test_null_int ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_null_int VALUES (1, NULL), (2, 0), (3, 10), (4, NULL); + """ + + qt_null_int """ + SELECT id, tmp.input_value, tmp.is_null, tmp.result + FROM test_null_int + LATERAL VIEW udtf_null_int(value) tmp AS input_value, is_null, result + ORDER BY id; + """ + + // Test 1.2: Empty String vs NULL String + sql """ DROP FUNCTION IF EXISTS udtf_null_string(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_null_string(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "handle_null_string", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def handle_null_string(value): + '''Distinguish NULL from empty string''' + if value is None: + yield ('NULL', -1) + elif value == '': + yield ('EMPTY', 0) + else: + yield ('NORMAL', len(value)) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_null_string; """ + sql """ + CREATE TABLE test_null_string ( + id INT, + value STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_null_string VALUES (1, NULL), (2, ''), (3, 'hello'), (4, NULL); + """ + + qt_null_string """ + SELECT id, tmp.value_type, tmp.length + FROM test_null_string + LATERAL VIEW udtf_null_string(value) tmp AS value_type, length + ORDER BY id; + """ + + // Test 1.3: Empty Array + sql """ DROP FUNCTION IF EXISTS udtf_empty_array(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_empty_array(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "handle_empty_array", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def handle_empty_array(arr): + '''Handle NULL vs empty array''' + if arr is None: + yield ('NULL', -1) + elif len(arr) == 0: + yield ('EMPTY', 0) + else: + yield ('NORMAL', len(arr)) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_empty_array; """ + sql """ + CREATE TABLE test_empty_array ( + id INT, + value ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_empty_array VALUES + (1, NULL), + (2, []), + (3, [1, 2, 3]); + """ + + qt_empty_array """ + SELECT id, tmp.array_type, tmp.size + FROM test_empty_array + LATERAL VIEW udtf_empty_array(value) tmp AS array_type, size + ORDER BY id; + """ + + // Test 1.4: NULL Fields in STRUCT + sql """ DROP FUNCTION IF EXISTS udtf_null_struct(STRUCT); """ + sql """ + CREATE TABLES FUNCTION udtf_null_struct(STRUCT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "handle_null_struct", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def handle_null_struct(person): + '''Handle NULL fields in STRUCT''' + if person is None: + yield (False, False, 'struct_is_null') + else: + name = person.get('name') + age = person.get('age') + has_name = name is not None + has_age = age is not None + + if has_name and has_age: + summary = f"{name}_{age}" + elif has_name: + summary = f"{name}_no_age" + elif has_age: + summary = f"no_name_{age}" + else: + summary = "all_fields_null" + + yield (has_name, has_age, summary) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_null_struct; """ + sql """ + CREATE TABLE test_null_struct ( + id INT, + person STRUCT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_null_struct VALUES + (1, named_struct('name', 'Alice', 'age', 25)), + (2, named_struct('name', 'Bob', 'age', NULL)), + (3, named_struct('name', NULL, 'age', 30)), + (4, named_struct('name', NULL, 'age', NULL)); + """ + + qt_null_struct """ + SELECT id, tmp.has_name, tmp.has_age, tmp.summary + FROM test_null_struct + LATERAL VIEW udtf_null_struct(person) tmp AS has_name, has_age, summary + ORDER BY id; + """ + + // ======================================== + // Section 2: Extreme Cases + // ======================================== + + // Test 2.1: Empty Table (0 rows) + sql """ DROP FUNCTION IF EXISTS udtf_empty_table(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_empty_table(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_empty_table", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_empty_table(value): + '''This should never be called for empty table''' + if value is not None: + yield (value * 2,) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_empty_table; """ + sql """ + CREATE TABLE test_empty_table ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // No INSERT - table remains empty + + qt_empty_table """ + SELECT tmp.value + FROM test_empty_table + LATERAL VIEW udtf_empty_table(value) tmp AS value; + """ + + // Test 2.2: Single Row Table + sql """ DROP FUNCTION IF EXISTS udtf_single_row(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_single_row(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_single_row", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_single_row(value): + '''Process single row input''' + if value is not None: + for i in range(3): + yield (value, value + i) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_single_row; """ + sql """ + CREATE TABLE test_single_row ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_single_row VALUES (1, 100); + """ + + qt_single_row """ + SELECT tmp.original, tmp.generated + FROM test_single_row + LATERAL VIEW udtf_single_row(value) tmp AS original, generated + ORDER BY tmp.generated; + """ + + // Test 2.3: Large Field - Long String + sql """ DROP FUNCTION IF EXISTS udtf_long_string(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_long_string(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_long_string", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_long_string(text): + '''Process very long string''' + if text is not None: + length = len(text) + first_10 = text[:10] if length >= 10 else text + last_10 = text[-10:] if length >= 10 else text + yield (length, first_10, last_10) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_long_string; """ + sql """ + CREATE TABLE test_long_string ( + id INT, + value STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_long_string VALUES + (1, REPEAT('A', 1000)), + (2, REPEAT('B', 5000)); + """ + + qt_long_string """ + SELECT id, tmp.length, tmp.first_10, tmp.last_10 + FROM test_long_string + LATERAL VIEW udtf_long_string(value) tmp AS length, first_10, last_10 + ORDER BY id; + """ + + // Test 2.4: Large Array + sql """ DROP FUNCTION IF EXISTS udtf_large_array(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_large_array(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_large_array", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_large_array(arr): + '''Process large array - compute statistics instead of exploding''' + if arr is not None and len(arr) > 0: + total = len(arr) + total_sum = sum(x for x in arr if x is not None) + first = arr[0] if len(arr) > 0 else None + last = arr[-1] if len(arr) > 0 else None + yield (total, total_sum, first, last) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_large_array; """ + sql """ + CREATE TABLE test_large_array ( + id INT, + value ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_large_array VALUES + (1, ARRAY_REPEAT(1, 100)), + (2, ARRAY_REPEAT(5, 50)); + """ + + qt_large_array """ + SELECT id, tmp.total_elements, tmp.sum_value, tmp.first_elem, tmp.last_elem + FROM test_large_array + LATERAL VIEW udtf_large_array(value) tmp AS total_elements, sum_value, first_elem, last_elem + ORDER BY id; + """ + + // Test 2.5: Output Explosion - Controlled + sql """ DROP FUNCTION IF EXISTS udtf_output_explosion(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_output_explosion(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "output_explosion", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def output_explosion(n): + '''Generate many outputs from single input (controlled explosion)''' + if n is not None and 0 < n <= 100: # Safety limit + for i in range(n): + yield (i,) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_output_explosion; """ + sql """ + CREATE TABLE test_output_explosion ( + id INT, + multiplier INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_output_explosion VALUES (1, 10), (2, 50); + """ + + qt_output_explosion """ + SELECT id, COUNT(*) as output_count, MIN(tmp.value) as min_val, MAX(tmp.value) as max_val + FROM test_output_explosion + LATERAL VIEW udtf_output_explosion(multiplier) tmp AS value + GROUP BY id + ORDER BY id; + """ + + // ======================================== + // Section 3: Special Values + // ======================================== + + // Test 3.1: Special Numeric Values (0, negative, boundary) + sql """ DROP FUNCTION IF EXISTS udtf_special_numbers(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_special_numbers(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_special_numbers", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_special_numbers(value): + '''Categorize special numeric values''' + INT_MIN = -2147483648 + INT_MAX = 2147483647 + + if value is None: + yield (None, 'NULL', False) + elif value == 0: + yield (value, 'ZERO', False) + elif value == INT_MIN or value == INT_MAX: + category = 'POSITIVE' if value > 0 else 'NEGATIVE' + yield (value, category, True) # is_boundary = True + elif value > 0: + yield (value, 'POSITIVE', False) + else: + yield (value, 'NEGATIVE', False) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_special_numbers; """ + sql """ + CREATE TABLE test_special_numbers ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_special_numbers VALUES + (1, -2147483648), -- INT MIN + (2, -1), + (3, 0), + (4, 1), + (5, 2147483647), -- INT MAX + (6, NULL); + """ + + qt_special_numbers """ + SELECT id, tmp.original, tmp.category, tmp.is_boundary + FROM test_special_numbers + LATERAL VIEW udtf_special_numbers(value) tmp AS original, category, is_boundary + ORDER BY id; + """ + + // Test 3.2: Special Double Values (infinity, very small numbers) + sql """ DROP FUNCTION IF EXISTS udtf_special_doubles(DOUBLE); """ + sql """ + CREATE TABLES FUNCTION udtf_special_doubles(DOUBLE) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_special_doubles", + "runtime_version" = "3.10.12" + ) + AS \$\$ +import math + +def process_special_doubles(value): + '''Classify special double values''' + if value is None: + yield (None, 'NULL') + elif math.isnan(value): + yield (value, 'NAN') + elif math.isinf(value): + if value > 0: + yield (value, 'POSITIVE_INF') + else: + yield (value, 'NEGATIVE_INF') + elif value == 0.0: + yield (value, 'ZERO') + elif abs(value) < 1e-10: + yield (value, 'VERY_SMALL') + elif abs(value) > 1e10: + yield (value, 'VERY_LARGE') + else: + yield (value, 'NORMAL') +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_special_doubles; """ + sql """ + CREATE TABLE test_special_doubles ( + id INT, + value DOUBLE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_special_doubles VALUES + (1, 0.0), + (2, 1e-15), + (3, 1e15), + (4, -1e15), + (5, 3.14159); + """ + + qt_special_doubles """ + SELECT id, tmp.original, tmp.classification + FROM test_special_doubles + LATERAL VIEW udtf_special_doubles(value) tmp AS original, classification + ORDER BY id; + """ + + // Test 3.3: Special String Values (special characters, Unicode) + sql """ DROP FUNCTION IF EXISTS udtf_special_strings(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_special_strings(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_special_strings", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_special_strings(text): + '''Process strings with special characters''' + if text is None: + yield (0, False, 'NULL') + elif text == '': + yield (0, False, 'EMPTY') + else: + length = len(text) + has_special = any(ord(c) > 127 for c in text) + + if has_special: + desc = 'HAS_UNICODE' + elif any(c in text for c in ['\\n', '\\t', '\\r']): + desc = 'HAS_WHITESPACE' + elif any(c in text for c in ['!', '@', '#', '\$', '%']): + desc = 'HAS_SYMBOLS' + else: + desc = 'NORMAL' + + yield (length, has_special, desc) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_special_strings; """ + sql """ + CREATE TABLE test_special_strings ( + id INT, + value STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_special_strings VALUES + (1, 'normal text'), + (2, 'hello@world.com'), + (3, 'tab\\there'), + (4, '你好世界'), + (5, ''); + """ + + qt_special_strings """ + SELECT id, tmp.length, tmp.has_special, tmp.description + FROM test_special_strings + LATERAL VIEW udtf_special_strings(value) tmp AS length, has_special, description + ORDER BY id; + """ + + // Test 3.4: Boundary Dates + sql """ DROP FUNCTION IF EXISTS udtf_boundary_dates(DATE); """ + sql """ + CREATE TABLES FUNCTION udtf_boundary_dates(DATE) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_boundary_dates", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_boundary_dates(dt): + '''Process boundary date values''' + if dt is None: + yield (None, 0, False) + else: + year = dt.year + # Check if it's a boundary date + is_boundary = year in [1970, 9999] or (year == 1970 and dt.month == 1 and dt.day == 1) + yield (dt, year, is_boundary) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_boundary_dates; """ + sql """ + CREATE TABLE test_boundary_dates ( + id INT, + value DATE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_boundary_dates VALUES + (1, '1970-01-01'), + (2, '2024-06-15'), + (3, '9999-12-31'); + """ + + qt_boundary_dates """ + SELECT id, tmp.original, tmp.year, tmp.is_boundary + FROM test_boundary_dates + LATERAL VIEW udtf_boundary_dates(value) tmp AS original, year, is_boundary + ORDER BY id; + """ + + } finally { + // Cleanup functions + try_sql("DROP FUNCTION IF EXISTS udtf_null_int(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_null_string(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_empty_array(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_null_struct(STRUCT);") + try_sql("DROP FUNCTION IF EXISTS udtf_empty_table(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_single_row(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_long_string(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_large_array(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_output_explosion(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_special_numbers(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_special_doubles(DOUBLE);") + try_sql("DROP FUNCTION IF EXISTS udtf_special_strings(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_boundary_dates(DATE);") + + // Cleanup tables + try_sql("DROP TABLE IF EXISTS test_null_int;") + try_sql("DROP TABLE IF EXISTS test_null_string;") + try_sql("DROP TABLE IF EXISTS test_empty_array;") + try_sql("DROP TABLE IF EXISTS test_null_struct;") + try_sql("DROP TABLE IF EXISTS test_empty_table;") + try_sql("DROP TABLE IF EXISTS test_single_row;") + try_sql("DROP TABLE IF EXISTS test_long_string;") + try_sql("DROP TABLE IF EXISTS test_large_array;") + try_sql("DROP TABLE IF EXISTS test_output_explosion;") + try_sql("DROP TABLE IF EXISTS test_special_numbers;") + try_sql("DROP TABLE IF EXISTS test_special_doubles;") + try_sql("DROP TABLE IF EXISTS test_special_strings;") + try_sql("DROP TABLE IF EXISTS test_boundary_dates;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_edge_cases_module.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_edge_cases_module.groovy new file mode 100644 index 00000000000000..3bd4bf4a29ebbd --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_edge_cases_module.groovy @@ -0,0 +1,554 @@ +// 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. + +suite("test_pythonudtf_edge_cases_module") { + // Test Python UDTF Edge Cases and Boundary Conditions + // Coverage: NULL handling, extreme cases, special values + + def pyPath = """${context.file.parent}/udtf_scripts/pyudtf.zip""" + scp_udf_file_to_all_be(pyPath) + def runtime_version = "3.10.12" + log.info("Python zip path: ${pyPath}".toString()) + + try { + // ======================================== + // Section 1: NULL Value Handling + // ======================================== + + // Test 1.1: NULL Integer Input + sql """ DROP FUNCTION IF EXISTS udtf_null_int_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_null_int_module(INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.handle_null_int", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_null_int_module; """ + sql """ + CREATE TABLE test_null_int_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_null_int_module VALUES (1, NULL), (2, 0), (3, 10), (4, NULL); + """ + + qt_null_int """ + SELECT id, tmp.input_value, tmp.is_null, tmp.result + FROM test_null_int_module + LATERAL VIEW udtf_null_int_module(value) tmp AS input_value, is_null, result + ORDER BY id; + """ + + // Test 1.2: Empty String vs NULL String + sql """ DROP FUNCTION IF EXISTS udtf_null_string_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_null_string_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.handle_null_string", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_null_string_module; """ + sql """ + CREATE TABLE test_null_string_module ( + id INT, + value STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_null_string_module VALUES (1, NULL), (2, ''), (3, 'hello'), (4, NULL); + """ + + qt_null_string """ + SELECT id, tmp.value_type, tmp.length + FROM test_null_string_module + LATERAL VIEW udtf_null_string_module(value) tmp AS value_type, length + ORDER BY id; + """ + + // Test 1.3: Empty Array + sql """ DROP FUNCTION IF EXISTS udtf_empty_array_module(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_empty_array_module(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.handle_empty_array", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_empty_array_module; """ + sql """ + CREATE TABLE test_empty_array_module ( + id INT, + value ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_empty_array_module VALUES + (1, NULL), + (2, []), + (3, [1, 2, 3]); + """ + + qt_empty_array """ + SELECT id, tmp.array_type, tmp.size + FROM test_empty_array_module + LATERAL VIEW udtf_empty_array_module(value) tmp AS array_type, size + ORDER BY id; + """ + + // Test 1.4: NULL Fields in STRUCT + sql """ DROP FUNCTION IF EXISTS udtf_null_struct_module(STRUCT); """ + sql """ + CREATE TABLES FUNCTION udtf_null_struct_module(STRUCT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.handle_null_struct", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_null_struct_module; """ + sql """ + CREATE TABLE test_null_struct_module ( + id INT, + person STRUCT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_null_struct_module VALUES + (1, named_struct('name', 'Alice', 'age', 25)), + (2, named_struct('name', 'Bob', 'age', NULL)), + (3, named_struct('name', NULL, 'age', 30)), + (4, named_struct('name', NULL, 'age', NULL)); + """ + + qt_null_struct """ + SELECT id, tmp.has_name, tmp.has_age, tmp.summary + FROM test_null_struct_module + LATERAL VIEW udtf_null_struct_module(person) tmp AS has_name, has_age, summary + ORDER BY id; + """ + + // ======================================== + // Section 2: Extreme Cases + // ======================================== + + // Test 2.1: Empty Table (0 rows) + sql """ DROP FUNCTION IF EXISTS udtf_empty_table_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_empty_table_module(INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.process_empty_table", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_empty_table_module; """ + sql """ + CREATE TABLE test_empty_table_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // No INSERT - table remains empty + + qt_empty_table """ + SELECT tmp.value + FROM test_empty_table_module + LATERAL VIEW udtf_empty_table_module(value) tmp AS value; + """ + + // Test 2.2: Single Row Table + sql """ DROP FUNCTION IF EXISTS udtf_single_row_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_single_row_module(INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.process_single_row", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_single_row_module; """ + sql """ + CREATE TABLE test_single_row_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_single_row_module VALUES (1, 100); + """ + + qt_single_row """ + SELECT tmp.original, tmp.generated + FROM test_single_row_module + LATERAL VIEW udtf_single_row_module(value) tmp AS original, generated + ORDER BY tmp.generated; + """ + + // Test 2.3: Large Field - Long String + sql """ DROP FUNCTION IF EXISTS udtf_long_string_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_long_string_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.process_long_string", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_long_string_module; """ + sql """ + CREATE TABLE test_long_string_module ( + id INT, + value STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_long_string_module VALUES + (1, REPEAT('A', 1000)), + (2, REPEAT('B', 5000)); + """ + + qt_long_string """ + SELECT id, tmp.length, tmp.first_10, tmp.last_10 + FROM test_long_string_module + LATERAL VIEW udtf_long_string_module(value) tmp AS length, first_10, last_10 + ORDER BY id; + """ + + // Test 2.4: Large Array + sql """ DROP FUNCTION IF EXISTS udtf_large_array_module(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_large_array_module(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.process_large_array", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_large_array_module; """ + sql """ + CREATE TABLE test_large_array_module ( + id INT, + value ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_large_array_module VALUES + (1, ARRAY_REPEAT(1, 100)), + (2, ARRAY_REPEAT(5, 50)); + """ + + qt_large_array """ + SELECT id, tmp.total_elements, tmp.sum_value, tmp.first_elem, tmp.last_elem + FROM test_large_array_module + LATERAL VIEW udtf_large_array_module(value) tmp AS total_elements, sum_value, first_elem, last_elem + ORDER BY id; + """ + + // Test 2.5: Output Explosion - Controlled + sql """ DROP FUNCTION IF EXISTS udtf_output_explosion_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_output_explosion_module(INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.output_explosion", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_output_explosion_module; """ + sql """ + CREATE TABLE test_output_explosion_module ( + id INT, + multiplier INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_output_explosion_module VALUES (1, 10), (2, 50); + """ + + qt_output_explosion """ + SELECT id, COUNT(*) as output_count, MIN(tmp.value) as min_val, MAX(tmp.value) as max_val + FROM test_output_explosion_module + LATERAL VIEW udtf_output_explosion_module(multiplier) tmp AS value + GROUP BY id + ORDER BY id; + """ + + // ======================================== + // Section 3: Special Values + // ======================================== + + // Test 3.1: Special Numeric Values (0, negative, boundary) + sql """ DROP FUNCTION IF EXISTS udtf_special_numbers_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_special_numbers_module(INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.process_special_numbers", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_special_numbers_module; """ + sql """ + CREATE TABLE test_special_numbers_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_special_numbers_module VALUES + (1, -2147483648), -- INT MIN + (2, -1), + (3, 0), + (4, 1), + (5, 2147483647), -- INT MAX + (6, NULL); + """ + + qt_special_numbers """ + SELECT id, tmp.original, tmp.category, tmp.is_boundary + FROM test_special_numbers_module + LATERAL VIEW udtf_special_numbers_module(value) tmp AS original, category, is_boundary + ORDER BY id; + """ + + // Test 3.2: Special Double Values (infinity, very small numbers) + sql """ DROP FUNCTION IF EXISTS udtf_special_doubles_module(DOUBLE); """ + sql """ + CREATE TABLES FUNCTION udtf_special_doubles_module(DOUBLE) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.process_special_doubles", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_special_doubles_module; """ + sql """ + CREATE TABLE test_special_doubles_module ( + id INT, + value DOUBLE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_special_doubles_module VALUES + (1, 0.0), + (2, 1e-15), + (3, 1e15), + (4, -1e15), + (5, 3.14159); + """ + + qt_special_doubles """ + SELECT id, tmp.original, tmp.classification + FROM test_special_doubles_module + LATERAL VIEW udtf_special_doubles_module(value) tmp AS original, classification + ORDER BY id; + """ + + // Test 3.3: Special String Values (special characters, Unicode) + sql """ DROP FUNCTION IF EXISTS udtf_special_strings_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_special_strings_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.process_special_strings", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_special_strings_module; """ + sql """ + CREATE TABLE test_special_strings_module ( + id INT, + value STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_special_strings_module VALUES + (1, 'normal text'), + (2, 'hello@world.com'), + (3, 'tab\\there'), + (4, '你好世界'), + (5, ''); + """ + + qt_special_strings """ + SELECT id, tmp.length, tmp.has_special, tmp.description + FROM test_special_strings_module + LATERAL VIEW udtf_special_strings_module(value) tmp AS length, has_special, description + ORDER BY id; + """ + + // Test 3.4: Boundary Dates + sql """ DROP FUNCTION IF EXISTS udtf_boundary_dates_module(DATE); """ + sql """ + CREATE TABLES FUNCTION udtf_boundary_dates_module(DATE) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.edge_cases_udtf.process_boundary_dates", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_boundary_dates_module; """ + sql """ + CREATE TABLE test_boundary_dates_module ( + id INT, + value DATE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_boundary_dates_module VALUES + (1, '1970-01-01'), + (2, '2024-06-15'), + (3, '9999-12-31'); + """ + + qt_boundary_dates """ + SELECT id, tmp.original, tmp.year, tmp.is_boundary + FROM test_boundary_dates_module + LATERAL VIEW udtf_boundary_dates_module(value) tmp AS original, year, is_boundary + ORDER BY id; + """ + + } finally { + // Cleanup functions + try_sql("DROP FUNCTION IF EXISTS udtf_null_int_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_null_string_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_empty_array_module(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_null_struct_module(STRUCT);") + try_sql("DROP FUNCTION IF EXISTS udtf_empty_table_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_single_row_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_long_string_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_large_array_module(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_output_explosion_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_special_numbers_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_special_doubles_module(DOUBLE);") + try_sql("DROP FUNCTION IF EXISTS udtf_special_strings_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_boundary_dates_module(DATE);") + + // Cleanup tables + try_sql("DROP TABLE IF EXISTS test_null_int_module;") + try_sql("DROP TABLE IF EXISTS test_null_string_module;") + try_sql("DROP TABLE IF EXISTS test_empty_array_module;") + try_sql("DROP TABLE IF EXISTS test_null_struct_module;") + try_sql("DROP TABLE IF EXISTS test_empty_table_module;") + try_sql("DROP TABLE IF EXISTS test_single_row_module;") + try_sql("DROP TABLE IF EXISTS test_long_string_module;") + try_sql("DROP TABLE IF EXISTS test_large_array_module;") + try_sql("DROP TABLE IF EXISTS test_output_explosion_module;") + try_sql("DROP TABLE IF EXISTS test_special_numbers_module;") + try_sql("DROP TABLE IF EXISTS test_special_doubles_module;") + try_sql("DROP TABLE IF EXISTS test_special_strings_module;") + try_sql("DROP TABLE IF EXISTS test_boundary_dates_module;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_exceptions_inline.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_exceptions_inline.groovy new file mode 100644 index 00000000000000..832a7b1e6bf70b --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_exceptions_inline.groovy @@ -0,0 +1,788 @@ +// 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. + +suite("test_pythonudtf_exceptions_inline") { + // Test Python UDTF Exception Handling + // Coverage: Runtime errors, type errors, logic errors, edge cases + + def runtime_version = "3.10.12" + + try { + // ======================================== + // Section 1: Arithmetic Exceptions + // ======================================== + + // Test 1.1: Division by Zero - Handled + sql """ DROP FUNCTION IF EXISTS udtf_safe_divide(INT, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_safe_divide(INT, INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "safe_divide", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def safe_divide(a, b): + '''Safe division with error handling''' + try: + if b == 0: + yield (a, b, None, 'division_by_zero') + else: + result = a / b + yield (a, b, result, 'success') + except Exception as e: + yield (a, b, None, f'error_{type(e).__name__}') +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_division; """ + sql """ + CREATE TABLE test_division ( + id INT, + num INT, + denom INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_division VALUES + (1, 10, 2), + (2, 10, 0), + (3, 0, 5), + (4, -8, 4); + """ + + qt_safe_divide """ + SELECT id, tmp.numerator, tmp.denominator, tmp.result, tmp.error_msg + FROM test_division + LATERAL VIEW udtf_safe_divide(num, denom) tmp AS numerator, denominator, result, error_msg + ORDER BY id; + """ + + // Test 1.2: Integer Overflow Detection + sql """ DROP FUNCTION IF EXISTS udtf_overflow_check(BIGINT); """ + sql """ + CREATE TABLES FUNCTION udtf_overflow_check(BIGINT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "check_overflow", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def check_overflow(value): + '''Check for potential overflow in operations''' + if value is None: + yield (None, None, 'null_input') + else: + # BIGINT range: -2^63 to 2^63-1 + MAX_BIGINT = 9223372036854775807 + MIN_BIGINT = -9223372036854775808 + + doubled = value * 2 + + # Check if doubled value is within safe range + if doubled > MAX_BIGINT or doubled < MIN_BIGINT: + yield (value, None, 'would_overflow') + else: + yield (value, doubled, 'safe') +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_overflow; """ + sql """ + CREATE TABLE test_overflow ( + id INT, + big_val BIGINT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_overflow VALUES + (1, 100), + (2, 5000000000000), + (3, -5000000000000), + (4, NULL); + """ + + qt_overflow_check """ + SELECT id, tmp.original, tmp.doubled, tmp.status + FROM test_overflow + LATERAL VIEW udtf_overflow_check(big_val) tmp AS original, doubled, status + ORDER BY id; + """ + + // ======================================== + // Section 2: Type Conversion Errors + // ======================================== + + // Test 2.1: String to Number Conversion + sql """ DROP FUNCTION IF EXISTS udtf_parse_number(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_parse_number(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "parse_number", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def parse_number(text): + '''Parse string to number with error handling''' + if text is None: + yield (None, None, False) + else: + try: + num = float(text) + yield (text, num, True) + except ValueError: + yield (text, None, False) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_parse; """ + sql """ + CREATE TABLE test_parse ( + id INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_parse VALUES + (1, '123'), + (2, '45.67'), + (3, 'abc'), + (4, '12.34.56'), + (5, ''), + (6, NULL); + """ + + qt_parse_number """ + SELECT id, tmp.input, tmp.parsed, tmp.is_valid + FROM test_parse + LATERAL VIEW udtf_parse_number(text) tmp AS input, parsed, is_valid + ORDER BY id; + """ + + // Test 2.2: Type Mismatch Handling + sql """ DROP FUNCTION IF EXISTS udtf_type_check(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_type_check(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "check_type", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def check_type(value): + '''Check and report value type''' + type_name = type(value).__name__ + + if value is None: + yield (None, 'NoneType', 0) + elif isinstance(value, str): + yield (value, type_name, len(value)) + else: + # Unexpected type - convert to string + yield (str(value), type_name, len(str(value))) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_types; """ + sql """ + CREATE TABLE test_types ( + id INT, + val STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_types VALUES + (1, 'hello'), + (2, ''), + (3, '12345'), + (4, NULL); + """ + + qt_type_check """ + SELECT id, tmp.value, tmp.type_name, tmp.length + FROM test_types + LATERAL VIEW udtf_type_check(val) tmp AS value, type_name, length + ORDER BY id; + """ + + // ======================================== + // Section 3: Collection/Array Errors + // ======================================== + + // Test 3.1: Array Index Out of Bounds + sql """ DROP FUNCTION IF EXISTS udtf_safe_index(ARRAY, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_safe_index(ARRAY, INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "safe_array_access", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def safe_array_access(arr, position): + '''Safe array element access''' + if arr is None: + yield (0, position, None, 'null_array') + elif len(arr) == 0: + yield (0, position, None, 'empty_array') + elif position < 0 or position >= len(arr): + yield (len(arr), position, None, 'out_of_bounds') + else: + yield (len(arr), position, arr[position], 'success') +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_array_access; """ + sql """ + CREATE TABLE test_array_access ( + id INT, + arr ARRAY, + pos INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_access VALUES + (1, [10, 20, 30], 1), + (2, [10, 20, 30], 5), + (3, [10, 20, 30], -1), + (4, [], 0), + (5, NULL, 0); + """ + + qt_safe_index """ + SELECT id, tmp.arr_size, tmp.target_pos, tmp.value, tmp.status + FROM test_array_access + LATERAL VIEW udtf_safe_index(arr, pos) tmp AS arr_size, target_pos, value, status + ORDER BY id; + """ + + // Test 3.2: Empty Collection Handling + sql """ DROP FUNCTION IF EXISTS udtf_collection_stats(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_collection_stats(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "compute_stats", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def compute_stats(arr): + '''Compute statistics with empty array handling''' + if arr is None: + yield (0, 0, 0.0, 'null_array') + elif len(arr) == 0: + yield (0, 0, 0.0, 'empty_array') + else: + count = len(arr) + total = sum(x for x in arr if x is not None) + avg = total / count if count > 0 else 0.0 + yield (count, total, avg, 'computed') +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_collection_stats; """ + sql """ + CREATE TABLE test_collection_stats ( + id INT, + data ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_collection_stats VALUES + (1, [1, 2, 3, 4, 5]), + (2, []), + (3, NULL), + (4, [10, 20]); + """ + + qt_collection_stats """ + SELECT id, tmp.count, tmp.total, tmp.avg, tmp.status + FROM test_collection_stats + LATERAL VIEW udtf_collection_stats(data) tmp AS count, total, avg, status + ORDER BY id; + """ + + // ======================================== + // Section 4: Dictionary/STRUCT Errors + // ======================================== + + // Test 4.1: Missing Dictionary Keys + sql """ DROP FUNCTION IF EXISTS udtf_safe_struct_access(STRUCT); """ + sql """ + CREATE TABLES FUNCTION udtf_safe_struct_access(STRUCT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "access_struct_fields", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def access_struct_fields(person): + '''Safe STRUCT field access''' + if person is None: + yield (False, False, None, None) + else: + # Use .get() to safely access dictionary keys + name = person.get('name') + age = person.get('age') + + has_name = name is not None + has_age = age is not None + + yield (has_name, has_age, name, age) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_struct_access; """ + sql """ + CREATE TABLE test_struct_access ( + id INT, + person STRUCT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_struct_access VALUES + (1, named_struct('name', 'Alice', 'age', 30)), + (2, named_struct('name', 'Bob', 'age', NULL)), + (3, named_struct('name', NULL, 'age', 25)), + (4, NULL); + """ + + qt_safe_struct_access """ + SELECT id, tmp.has_name, tmp.has_age, tmp.name_val, tmp.age_val + FROM test_struct_access + LATERAL VIEW udtf_safe_struct_access(person) tmp AS has_name, has_age, name_val, age_val + ORDER BY id; + """ + + // ======================================== + // Section 5: String Processing Errors + // ======================================== + + // Test 5.1: Invalid String Operations + sql """ DROP FUNCTION IF EXISTS udtf_string_slice(STRING, INT, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_string_slice(STRING, INT, INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "slice_string", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def slice_string(text, start, end): + '''Safe string slicing''' + if text is None: + yield (None, start, end, None, 'null_string') + elif start is None or end is None: + yield (text, start, end, None, 'null_index') + else: + length = len(text) + + # Clamp indices to valid range + safe_start = max(0, min(start, length)) + safe_end = max(0, min(end, length)) + + if safe_start >= safe_end: + yield (text, start, end, '', 'empty_slice') + else: + result = text[safe_start:safe_end] + yield (text, start, end, result, 'success') +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_string_slice; """ + sql """ + CREATE TABLE test_string_slice ( + id INT, + text STRING, + start_pos INT, + end_pos INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_string_slice VALUES + (1, 'hello world', 0, 5), + (2, 'hello world', 6, 11), + (3, 'hello world', 20, 30), + (4, 'hello world', 5, 2), + (5, NULL, 0, 5); + """ + + qt_string_slice """ + SELECT id, tmp.original, tmp.start_pos, tmp.end_pos, tmp.slice, tmp.status + FROM test_string_slice + LATERAL VIEW udtf_string_slice(text, start_pos, end_pos) tmp AS original, start_pos, end_pos, slice, status + ORDER BY id; + """ + + // Test 5.2: Encoding/Decoding Errors + sql """ DROP FUNCTION IF EXISTS udtf_check_encoding(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_check_encoding(STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "check_text_encoding", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def check_text_encoding(text): + '''Check string encoding properties''' + if text is None: + yield (None, 0, 0, False) + else: + byte_len = len(text.encode('utf-8')) + char_len = len(text) + has_unicode = byte_len > char_len + + yield (text, byte_len, char_len, has_unicode) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_encoding; """ + sql """ + CREATE TABLE test_encoding ( + id INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_encoding VALUES + (1, 'hello'), + (2, '你好世界'), + (3, 'café'), + (4, ''), + (5, NULL); + """ + + qt_check_encoding """ + SELECT id, tmp.text, tmp.byte_length, tmp.char_length, tmp.has_unicode + FROM test_encoding + LATERAL VIEW udtf_check_encoding(text) tmp AS text, byte_length, char_length, has_unicode + ORDER BY id; + """ + + // ======================================== + // Section 6: Logic and State Errors + // ======================================== + + // Test 6.1: Conditional Logic Errors + sql """ DROP FUNCTION IF EXISTS udtf_conditional_process(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_conditional_process(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "process_conditional", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def process_conditional(value): + '''Process value based on multiple conditions''' + if value is None: + yield (None, 'null', 0) + elif value < 0: + # For negative: take absolute value + yield (value, 'negative', abs(value)) + elif value == 0: + # Zero case: return 1 + yield (value, 'zero', 1) + elif value > 0 and value <= 100: + # Small positive: double it + yield (value, 'small_positive', value * 2) + else: + # Large positive: return as-is + yield (value, 'large_positive', value) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_conditional; """ + sql """ + CREATE TABLE test_conditional ( + id INT, + val INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_conditional VALUES + (1, -10), + (2, 0), + (3, 50), + (4, 200), + (5, NULL); + """ + + qt_conditional_process """ + SELECT id, tmp.input, tmp.category, tmp.result + FROM test_conditional + LATERAL VIEW udtf_conditional_process(val) tmp AS input, category, result + ORDER BY id; + """ + + // Test 6.2: Yield Control - No Output Case + sql """ DROP FUNCTION IF EXISTS udtf_filter_yield(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_filter_yield(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "conditional_yield", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def conditional_yield(value): + '''Only yield for even positive numbers''' + if value is not None and value > 0 and value % 2 == 0: + yield (value,) + # For other cases, yield nothing (filter out) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_filter_yield; """ + sql """ + CREATE TABLE test_filter_yield ( + id INT, + val INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_filter_yield VALUES + (1, 10), + (2, 15), + (3, -4), + (4, 0), + (5, 22), + (6, NULL); + """ + + qt_filter_yield """ + SELECT id, tmp.value + FROM test_filter_yield + LATERAL VIEW udtf_filter_yield(val) tmp AS value + ORDER BY id; + """ + + // ======================================== + // Section 7: Edge Cases in Computation + // ======================================== + + // Test 7.1: Very Small and Very Large Numbers + sql """ DROP FUNCTION IF EXISTS udtf_number_range(DOUBLE); """ + sql """ + CREATE TABLES FUNCTION udtf_number_range(DOUBLE) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "classify_number_range", + "runtime_version" = "3.10.12" + ) + AS \$\$ +import math + +def classify_number_range(value): + '''Classify number by magnitude''' + if value is None: + yield (None, 'null', True) + elif math.isnan(value): + yield (value, 'nan', False) + elif math.isinf(value): + yield (value, 'infinity', False) + elif value == 0.0: + yield (value, 'zero', True) + elif abs(value) < 1e-100: + yield (value, 'extremely_small', True) + elif abs(value) > 1e100: + yield (value, 'extremely_large', True) + elif abs(value) < 1.0: + yield (value, 'small', True) + else: + yield (value, 'normal', True) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_number_range; """ + sql """ + CREATE TABLE test_number_range ( + id INT, + val DOUBLE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_number_range VALUES + (1, 0.0), + (2, 1e-150), + (3, 1e150), + (4, 0.5), + (5, 123.456), + (6, NULL); + """ + + qt_number_range """ + SELECT id, tmp.value, tmp.magnitude, tmp.is_finite + FROM test_number_range + LATERAL VIEW udtf_number_range(val) tmp AS value, magnitude, is_finite + ORDER BY id; + """ + + // Test 7.2: Date/Time Edge Cases + sql """ DROP FUNCTION IF EXISTS udtf_date_validation(DATE); """ + sql """ + CREATE TABLES FUNCTION udtf_date_validation(DATE) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "validate_date", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def validate_date(dt): + '''Validate and classify dates''' + if dt is None: + yield (None, 0, False, 'null_date') + else: + year = dt.year + + # Check if leap year + is_leap = (year % 4 == 0 and year % 100 != 0) or (year % 400 == 0) + + # Classify date + if year < 1900: + status = 'very_old' + elif year > 2100: + status = 'far_future' + else: + status = 'normal' + + yield (dt, year, is_leap, status) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_date_validation; """ + sql """ + CREATE TABLE test_date_validation ( + id INT, + dt DATE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_date_validation VALUES + (1, '2024-01-01'), + (2, '2000-02-29'), + (3, '1970-01-01'), + (4, '9999-12-31'), + (5, NULL); + """ + + qt_date_validation """ + SELECT id, tmp.input_date, tmp.year, tmp.is_leap_year, tmp.status + FROM test_date_validation + LATERAL VIEW udtf_date_validation(dt) tmp AS input_date, year, is_leap_year, status + ORDER BY id; + """ + + } finally { + // Cleanup functions + try_sql("DROP FUNCTION IF EXISTS udtf_safe_divide(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_overflow_check(BIGINT);") + try_sql("DROP FUNCTION IF EXISTS udtf_parse_number(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_type_check(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_safe_index(ARRAY, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_collection_stats(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_safe_struct_access(STRUCT);") + try_sql("DROP FUNCTION IF EXISTS udtf_string_slice(STRING, INT, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_check_encoding(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_conditional_process(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_filter_yield(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_number_range(DOUBLE);") + try_sql("DROP FUNCTION IF EXISTS udtf_date_validation(DATE);") + + // Cleanup tables + try_sql("DROP TABLE IF EXISTS test_division;") + try_sql("DROP TABLE IF EXISTS test_overflow;") + try_sql("DROP TABLE IF EXISTS test_parse;") + try_sql("DROP TABLE IF EXISTS test_types;") + try_sql("DROP TABLE IF EXISTS test_array_access;") + try_sql("DROP TABLE IF EXISTS test_collection_stats;") + try_sql("DROP TABLE IF EXISTS test_struct_access;") + try_sql("DROP TABLE IF EXISTS test_string_slice;") + try_sql("DROP TABLE IF EXISTS test_encoding;") + try_sql("DROP TABLE IF EXISTS test_conditional;") + try_sql("DROP TABLE IF EXISTS test_filter_yield;") + try_sql("DROP TABLE IF EXISTS test_number_range;") + try_sql("DROP TABLE IF EXISTS test_date_validation;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_exceptions_module.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_exceptions_module.groovy new file mode 100644 index 00000000000000..f2b9f800ceba95 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_exceptions_module.groovy @@ -0,0 +1,609 @@ +// 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. + +suite("test_pythonudtf_exceptions_module") { + // Test Python UDTF Exception Handling + // Coverage: Runtime errors, type errors, logic errors, edge cases + + def pyPath = """${context.file.parent}/udtf_scripts/pyudtf.zip""" + scp_udf_file_to_all_be(pyPath) + def runtime_version = "3.10.12" + log.info("Python zip path: ${pyPath}".toString()) + + try { + // ======================================== + // Section 1: Arithmetic Exceptions + // ======================================== + + // Test 1.1: Division by Zero - Handled + sql """ DROP FUNCTION IF EXISTS udtf_safe_divide_module(INT, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_safe_divide_module(INT, INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.safe_divide", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_division_module; """ + sql """ + CREATE TABLE test_division_module ( + id INT, + num INT, + denom INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_division_module VALUES + (1, 10, 2), + (2, 10, 0), + (3, 0, 5), + (4, -8, 4); + """ + + qt_safe_divide """ + SELECT id, tmp.numerator, tmp.denominator, tmp.result, tmp.error_msg + FROM test_division_module + LATERAL VIEW udtf_safe_divide_module(num, denom) tmp AS numerator, denominator, result, error_msg + ORDER BY id; + """ + + // Test 1.2: Integer Overflow Detection + sql """ DROP FUNCTION IF EXISTS udtf_overflow_check_module(BIGINT); """ + sql """ + CREATE TABLES FUNCTION udtf_overflow_check_module(BIGINT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.check_overflow", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_overflow_module; """ + sql """ + CREATE TABLE test_overflow_module ( + id INT, + big_val BIGINT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_overflow_module VALUES + (1, 100), + (2, 5000000000000), + (3, -5000000000000), + (4, NULL); + """ + + qt_overflow_check """ + SELECT id, tmp.original, tmp.doubled, tmp.status + FROM test_overflow_module + LATERAL VIEW udtf_overflow_check_module(big_val) tmp AS original, doubled, status + ORDER BY id; + """ + + // ======================================== + // Section 2: Type Conversion Errors + // ======================================== + + // Test 2.1: String to Number Conversion + sql """ DROP FUNCTION IF EXISTS udtf_parse_number_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_parse_number_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.parse_number", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_parse_module; """ + sql """ + CREATE TABLE test_parse_module ( + id INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_parse_module VALUES + (1, '123'), + (2, '45.67'), + (3, 'abc'), + (4, '12.34.56'), + (5, ''), + (6, NULL); + """ + + qt_parse_number """ + SELECT id, tmp.input, tmp.parsed, tmp.is_valid + FROM test_parse_module + LATERAL VIEW udtf_parse_number_module(text) tmp AS input, parsed, is_valid + ORDER BY id; + """ + + // Test 2.2: Type Mismatch Handling + sql """ DROP FUNCTION IF EXISTS udtf_type_check_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_type_check_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.check_type", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_types_module; """ + sql """ + CREATE TABLE test_types_module ( + id INT, + val STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_types_module VALUES + (1, 'hello'), + (2, ''), + (3, '12345'), + (4, NULL); + """ + + qt_type_check """ + SELECT id, tmp.value, tmp.type_name, tmp.length + FROM test_types_module + LATERAL VIEW udtf_type_check_module(val) tmp AS value, type_name, length + ORDER BY id; + """ + + // ======================================== + // Section 3: Collection/Array Errors + // ======================================== + + // Test 3.1: Array Index Out of Bounds + sql """ DROP FUNCTION IF EXISTS udtf_safe_index_module(ARRAY, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_safe_index_module(ARRAY, INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.safe_array_access", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_array_access_module; """ + sql """ + CREATE TABLE test_array_access_module ( + id INT, + arr ARRAY, + pos INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_access_module VALUES + (1, [10, 20, 30], 1), + (2, [10, 20, 30], 5), + (3, [10, 20, 30], -1), + (4, [], 0), + (5, NULL, 0); + """ + + qt_safe_index """ + SELECT id, tmp.arr_size, tmp.target_pos, tmp.value, tmp.status + FROM test_array_access_module + LATERAL VIEW udtf_safe_index_module(arr, pos) tmp AS arr_size, target_pos, value, status + ORDER BY id; + """ + + // Test 3.2: Empty Collection Handling + sql """ DROP FUNCTION IF EXISTS udtf_collection_stats_module(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_collection_stats_module(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.compute_stats", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_collection_stats_module; """ + sql """ + CREATE TABLE test_collection_stats_module ( + id INT, + data ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_collection_stats_module VALUES + (1, [1, 2, 3, 4, 5]), + (2, []), + (3, NULL), + (4, [10, 20]); + """ + + qt_collection_stats """ + SELECT id, tmp.count, tmp.total, tmp.avg, tmp.status + FROM test_collection_stats_module + LATERAL VIEW udtf_collection_stats_module(data) tmp AS count, total, avg, status + ORDER BY id; + """ + + // ======================================== + // Section 4: Dictionary/STRUCT Errors + // ======================================== + + // Test 4.1: Missing Dictionary Keys + sql """ DROP FUNCTION IF EXISTS udtf_safe_struct_access_module(STRUCT); """ + sql """ + CREATE TABLES FUNCTION udtf_safe_struct_access_module(STRUCT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.access_struct_fields", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_struct_access_module; """ + sql """ + CREATE TABLE test_struct_access_module ( + id INT, + person STRUCT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_struct_access_module VALUES + (1, named_struct('name', 'Alice', 'age', 30)), + (2, named_struct('name', 'Bob', 'age', NULL)), + (3, named_struct('name', NULL, 'age', 25)), + (4, NULL); + """ + + qt_safe_struct_access """ + SELECT id, tmp.has_name, tmp.has_age, tmp.name_val, tmp.age_val + FROM test_struct_access_module + LATERAL VIEW udtf_safe_struct_access_module(person) tmp AS has_name, has_age, name_val, age_val + ORDER BY id; + """ + + // ======================================== + // Section 5: String Processing Errors + // ======================================== + + // Test 5.1: Invalid String Operations + sql """ DROP FUNCTION IF EXISTS udtf_string_slice_module(STRING, INT, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_string_slice_module(STRING, INT, INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.slice_string", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_string_slice_module; """ + sql """ + CREATE TABLE test_string_slice_module ( + id INT, + text STRING, + start_pos INT, + end_pos INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_string_slice_module VALUES + (1, 'hello world', 0, 5), + (2, 'hello world', 6, 11), + (3, 'hello world', 20, 30), + (4, 'hello world', 5, 2), + (5, NULL, 0, 5); + """ + + qt_string_slice """ + SELECT id, tmp.original, tmp.start_pos, tmp.end_pos, tmp.slice, tmp.status + FROM test_string_slice_module + LATERAL VIEW udtf_string_slice_module(text, start_pos, end_pos) tmp AS original, start_pos, end_pos, slice, status + ORDER BY id; + """ + + // Test 5.2: Encoding/Decoding Errors + sql """ DROP FUNCTION IF EXISTS udtf_check_encoding_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_check_encoding_module(STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.check_text_encoding", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_encoding_module; """ + sql """ + CREATE TABLE test_encoding_module ( + id INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_encoding_module VALUES + (1, 'hello'), + (2, '你好世界'), + (3, 'café'), + (4, ''), + (5, NULL); + """ + + qt_check_encoding """ + SELECT id, tmp.text, tmp.byte_length, tmp.char_length, tmp.has_unicode + FROM test_encoding_module + LATERAL VIEW udtf_check_encoding_module(text) tmp AS text, byte_length, char_length, has_unicode + ORDER BY id; + """ + + // ======================================== + // Section 6: Logic and State Errors + // ======================================== + + // Test 6.1: Conditional Logic Errors + sql """ DROP FUNCTION IF EXISTS udtf_conditional_process_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_conditional_process_module(INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.process_conditional", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_conditional_module; """ + sql """ + CREATE TABLE test_conditional_module ( + id INT, + val INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_conditional_module VALUES + (1, -10), + (2, 0), + (3, 50), + (4, 200), + (5, NULL); + """ + + qt_conditional_process """ + SELECT id, tmp.input, tmp.category, tmp.result + FROM test_conditional_module + LATERAL VIEW udtf_conditional_process_module(val) tmp AS input, category, result + ORDER BY id; + """ + + // Test 6.2: Yield Control - No Output Case + sql """ DROP FUNCTION IF EXISTS udtf_filter_yield_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_filter_yield_module(INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.conditional_yield", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_filter_yield_module; """ + sql """ + CREATE TABLE test_filter_yield_module ( + id INT, + val INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_filter_yield_module VALUES + (1, 10), + (2, 15), + (3, -4), + (4, 0), + (5, 22), + (6, NULL); + """ + + qt_filter_yield """ + SELECT id, tmp.value + FROM test_filter_yield_module + LATERAL VIEW udtf_filter_yield_module(val) tmp AS value + ORDER BY id; + """ + + // ======================================== + // Section 7: Edge Cases in Computation + // ======================================== + + // Test 7.1: Very Small and Very Large Numbers + sql """ DROP FUNCTION IF EXISTS udtf_number_range_module(DOUBLE); """ + sql """ + CREATE TABLES FUNCTION udtf_number_range_module(DOUBLE) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.classify_number_range", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_number_range_module; """ + sql """ + CREATE TABLE test_number_range_module ( + id INT, + val DOUBLE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_number_range_module VALUES + (1, 0.0), + (2, 1e-150), + (3, 1e150), + (4, 0.5), + (5, 123.456), + (6, NULL); + """ + + qt_number_range """ + SELECT id, tmp.value, tmp.magnitude, tmp.is_finite + FROM test_number_range_module + LATERAL VIEW udtf_number_range_module(val) tmp AS value, magnitude, is_finite + ORDER BY id; + """ + + // Test 7.2: Date/Time Edge Cases + sql """ DROP FUNCTION IF EXISTS udtf_date_validation_module(DATE); """ + sql """ + CREATE TABLES FUNCTION udtf_date_validation_module(DATE) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.exceptions_udtf.validate_date", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_date_validation_module; """ + sql """ + CREATE TABLE test_date_validation_module ( + id INT, + dt DATE + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_date_validation_module VALUES + (1, '2024-01-01'), + (2, '2000-02-29'), + (3, '1970-01-01'), + (4, '9999-12-31'), + (5, NULL); + """ + + qt_date_validation """ + SELECT id, tmp.input_date, tmp.year, tmp.is_leap_year, tmp.status + FROM test_date_validation_module + LATERAL VIEW udtf_date_validation_module(dt) tmp AS input_date, year, is_leap_year, status + ORDER BY id; + """ + + } finally { + // Cleanup functions + try_sql("DROP FUNCTION IF EXISTS udtf_safe_divide_module(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_overflow_check_module(BIGINT);") + try_sql("DROP FUNCTION IF EXISTS udtf_parse_number_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_type_check_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_safe_index_module(ARRAY, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_collection_stats_module(ARRAY);") + try_sql("DROP FUNCTION IF EXISTS udtf_safe_struct_access_module(STRUCT);") + try_sql("DROP FUNCTION IF EXISTS udtf_string_slice_module(STRING, INT, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_check_encoding_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_conditional_process_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_filter_yield_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_number_range_module(DOUBLE);") + try_sql("DROP FUNCTION IF EXISTS udtf_date_validation_module(DATE);") + + // Cleanup tables + try_sql("DROP TABLE IF EXISTS test_division_module;") + try_sql("DROP TABLE IF EXISTS test_overflow_module;") + try_sql("DROP TABLE IF EXISTS test_parse_module;") + try_sql("DROP TABLE IF EXISTS test_types_module;") + try_sql("DROP TABLE IF EXISTS test_array_access_module;") + try_sql("DROP TABLE IF EXISTS test_collection_stats_module;") + try_sql("DROP TABLE IF EXISTS test_struct_access_module;") + try_sql("DROP TABLE IF EXISTS test_string_slice_module;") + try_sql("DROP TABLE IF EXISTS test_encoding_module;") + try_sql("DROP TABLE IF EXISTS test_conditional_module;") + try_sql("DROP TABLE IF EXISTS test_filter_yield_module;") + try_sql("DROP TABLE IF EXISTS test_number_range_module;") + try_sql("DROP TABLE IF EXISTS test_date_validation_module;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_io_patterns_inline.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_io_patterns_inline.groovy new file mode 100644 index 00000000000000..24cb13f82ac90d --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_io_patterns_inline.groovy @@ -0,0 +1,522 @@ +// 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. + +suite("test_pythonudtf_io_patterns_inline") { + // Test Python UDTF Input/Output Patterns + // Testing different cardinality patterns: 1-to-1, 1-to-N, 1-to-0, N-to-M + + def runtime_version = "3.10.12" + + try { + // ======================================== + // Pattern 1: One-to-One (1 input row → 1 output row) + // Each input row produces exactly one output row + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_one_to_one(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_one_to_one(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "one_to_one", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def one_to_one(value): + '''Each input row produces exactly one output row''' + if value is not None: + yield (value, value * 2) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_one_to_one; """ + sql """ + CREATE TABLE test_one_to_one ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_one_to_one VALUES (1, 10), (2, 20), (3, 30); + """ + + qt_one_to_one """ + SELECT tmp.input, tmp.doubled + FROM test_one_to_one + LATERAL VIEW udtf_one_to_one(value) tmp AS input, doubled + ORDER BY tmp.input; + """ + + // ======================================== + // Pattern 2: One-to-Many (1 input row → N output all_rows) + // Each input row produces multiple output all_rows + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_one_to_many(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_one_to_many(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "one_to_many", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def one_to_many(n): + '''Each input row produces N output all_rows (1 to n)''' + if n is not None and n > 0: + for i in range(1, n + 1): + yield (i,) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_one_to_many; """ + sql """ + CREATE TABLE test_one_to_many ( + id INT, + count INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_one_to_many VALUES (1, 3), (2, 2), (3, 4); + """ + + qt_one_to_many """ + SELECT id, tmp.value + FROM test_one_to_many + LATERAL VIEW udtf_one_to_many(count) tmp AS value + ORDER BY id, tmp.value; + """ + + // ======================================== + // Pattern 3: One-to-Zero (1 input row → 0 output all_rows) + // Some input all_rows produce no output (filtering) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_one_to_zero(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_one_to_zero(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "one_to_zero", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def one_to_zero(value): + '''Only output even numbers, skip odd numbers (zero output)''' + if value is not None and value % 2 == 0: + yield (value,) + # Odd numbers: no yield, zero output all_rows +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_one_to_zero; """ + sql """ + CREATE TABLE test_one_to_zero ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_one_to_zero VALUES (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6); + """ + + qt_one_to_zero """ + SELECT tmp.value + FROM test_one_to_zero + LATERAL VIEW udtf_one_to_zero(value) tmp AS value + ORDER BY tmp.value; + """ + + // ======================================== + // Pattern 4: One-to-Variable (1 input row → 0/1/N output all_rows) + // Different input all_rows produce different numbers of output all_rows + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_one_to_variable(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_one_to_variable(STRING) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "one_to_variable", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def one_to_variable(text): + ''' + - Empty string → 0 all_rows + - Single word → 1 row + - Multiple words → N all_rows + ''' + if text: + words = text.split() + for word in words: + yield (word,) + # Empty or None: no yield, zero output +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_one_to_variable; """ + sql """ + CREATE TABLE test_one_to_variable ( + id INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_one_to_variable VALUES + (1, 'hello'), -- 1 output + (2, 'hello world'), -- 2 outputs + (3, ''), -- 0 outputs + (4, 'a b c'); -- 3 outputs + """ + + qt_one_to_variable """ + SELECT id, tmp.word + FROM test_one_to_variable + LATERAL VIEW udtf_one_to_variable(text) tmp AS word + ORDER BY id, tmp.word; + """ + + // ======================================== + // Pattern 5: Many-to-One (N input all_rows → aggregate to fewer all_rows) + // Note: This simulates batch processing where each row independently + // produces output, but conceptually represents aggregation pattern + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_aggregate_pattern(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_aggregate_pattern(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "aggregate_pattern", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def aggregate_pattern(value): + '''Categorize numbers into ranges''' + if value is not None: + if value < 10: + category = 'small' + elif value < 100: + category = 'medium' + else: + category = 'large' + yield (value, category) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_aggregate_pattern; """ + sql """ + CREATE TABLE test_aggregate_pattern ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_aggregate_pattern VALUES + (1, 5), (2, 50), (3, 500), (4, 8), (5, 80), (6, 800); + """ + + qt_aggregate_pattern """ + SELECT tmp.category, COUNT(*) as count + FROM test_aggregate_pattern + LATERAL VIEW udtf_aggregate_pattern(value) tmp AS value, category + GROUP BY tmp.category + ORDER BY tmp.category; + """ + + // ======================================== + // Pattern 6: Explosive Growth (1 input row → many output all_rows) + // Testing large multiplication factor + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_explosive(INT, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_explosive(INT, INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "explosive", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def explosive(all_rows, all_cols): + '''Generate all_rows * all_cols output all_rows (cartesian product)''' + if all_rows is not None and all_cols is not None and all_rows > 0 and all_cols > 0: + for r in range(all_rows): + for c in range(all_cols): + yield (r, c) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_explosive; """ + sql """ + CREATE TABLE test_explosive ( + id INT, + all_rows INT, + all_cols INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_explosive VALUES (1, 2, 3); + """ + + qt_explosive """ + SELECT tmp.row_id, tmp.col_id + FROM test_explosive + LATERAL VIEW udtf_explosive(all_rows, all_cols) tmp AS row_id, col_id + ORDER BY tmp.row_id, tmp.col_id; + """ + + // ======================================== + // Pattern 7: Conditional Branching (different logic paths) + // Same function but different output counts based on condition + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_conditional(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_conditional(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "conditional", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def conditional(value): + ''' + - Positive: output (value, 'positive') + - Negative: output (abs(value), 'negative') + - Zero: output both (0, 'zero') and (0, 'neutral') + ''' + if value is not None: + if value > 0: + yield (value, 'positive') + elif value < 0: + yield (abs(value), 'negative') + else: + yield (0, 'zero') + yield (0, 'neutral') +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_conditional; """ + sql """ + CREATE TABLE test_conditional ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_conditional VALUES (1, 10), (2, -5), (3, 0), (4, 7); + """ + + qt_conditional """ + SELECT tmp.value, tmp.type + FROM test_conditional + LATERAL VIEW udtf_conditional(value) tmp AS value, type + ORDER BY tmp.value, tmp.type; + """ + + // ======================================== + // Pattern 8: All-or-Nothing (either all all_rows or no all_rows) + // Based on a condition, output all or nothing + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_all_or_nothing(STRING, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_all_or_nothing(STRING, INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "all_or_nothing", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def all_or_nothing(text, min_length): + ''' + If text length >= min_length: output each character with position + Otherwise: output nothing + ''' + if text and len(text) >= min_length: + for i, char in enumerate(text): + yield (char, i) + # If condition not met: no yield +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_all_or_nothing; """ + sql """ + CREATE TABLE test_all_or_nothing ( + id INT, + text STRING, + min_len INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_all_or_nothing VALUES + (1, 'hello', 3), -- 5 outputs (length=5 >= 3) + (2, 'hi', 5), -- 0 outputs (length=2 < 5) + (3, 'world', 4); -- 5 outputs (length=5 >= 4) + """ + + qt_all_or_nothing """ + SELECT id, tmp.char, tmp.pos + FROM test_all_or_nothing + LATERAL VIEW udtf_all_or_nothing(text, min_len) tmp AS char, pos + ORDER BY id, tmp.pos; + """ + + // ======================================== + // Pattern 9: Empty Input Table (0 input all_rows) + // Test behavior with no input data + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_empty_input(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_empty_input(INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "empty_input", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def empty_input(value): + '''Simple identity function''' + if value is not None: + yield (value,) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_empty_input; """ + sql """ + CREATE TABLE test_empty_input ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // No INSERT - table is empty + + qt_empty_input """ + SELECT tmp.value + FROM test_empty_input + LATERAL VIEW udtf_empty_input(value) tmp AS value; + """ + + // ======================================== + // Pattern 10: Batch Processing Simulation + // Multiple input all_rows, each producing variable outputs + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_batch_process(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_batch_process(INT) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "batch_process", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def batch_process(value): + '''For each input, generate multiples (2x, 3x, 5x)''' + if value is not None and value > 0: + for factor in [2, 3, 5]: + yield (value, factor, value * factor) +\$\$; + """ + + sql """ DROP TABLE IF EXISTS test_batch_process; """ + sql """ + CREATE TABLE test_batch_process ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_batch_process VALUES (1, 10), (2, 20); + """ + + qt_batch_process """ + SELECT tmp.original, tmp.factor, tmp.result + FROM test_batch_process + LATERAL VIEW udtf_batch_process(value) tmp AS original, factor, result + ORDER BY tmp.original, tmp.factor; + """ + + } finally { + // Cleanup functions + try_sql("DROP FUNCTION IF EXISTS udtf_one_to_one(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_one_to_many(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_one_to_zero(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_one_to_variable(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_aggregate_pattern(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_explosive(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_conditional(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_all_or_nothing(STRING, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_empty_input(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_batch_process(INT);") + + // Cleanup tables + try_sql("DROP TABLE IF EXISTS test_one_to_one;") + try_sql("DROP TABLE IF EXISTS test_one_to_many;") + try_sql("DROP TABLE IF EXISTS test_one_to_zero;") + try_sql("DROP TABLE IF EXISTS test_one_to_variable;") + try_sql("DROP TABLE IF EXISTS test_aggregate_pattern;") + try_sql("DROP TABLE IF EXISTS test_explosive;") + try_sql("DROP TABLE IF EXISTS test_conditional;") + try_sql("DROP TABLE IF EXISTS test_all_or_nothing;") + try_sql("DROP TABLE IF EXISTS test_empty_input;") + try_sql("DROP TABLE IF EXISTS test_batch_process;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_io_patterns_module.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_io_patterns_module.groovy new file mode 100644 index 00000000000000..1289e48f68937c --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_io_patterns_module.groovy @@ -0,0 +1,442 @@ +// 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. + +suite("test_pythonudtf_io_patterns_module") { + // Test Python UDTF Input/Output Patterns + // Testing different cardinality patterns: 1-to-1, 1-to-N, 1-to-0, N-to-M + + def pyPath = """${context.file.parent}/udtf_scripts/pyudtf.zip""" + scp_udf_file_to_all_be(pyPath) + def runtime_version = "3.10.12" + log.info("Python zip path: ${pyPath}".toString()) + + try { + // ======================================== + // Pattern 1: One-to-One (1 input row → 1 output row) + // Each input row produces exactly one output row + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_one_to_one_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_one_to_one_module(INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.one_to_one", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_one_to_one_module; """ + sql """ + CREATE TABLE test_one_to_one_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_one_to_one_module VALUES (1, 10), (2, 20), (3, 30); + """ + + qt_one_to_one """ + SELECT tmp.input, tmp.doubled + FROM test_one_to_one_module + LATERAL VIEW udtf_one_to_one_module(value) tmp AS input, doubled + ORDER BY tmp.input; + """ + + // ======================================== + // Pattern 2: One-to-Many (1 input row → N output all_rows) + // Each input row produces multiple output all_rows + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_one_to_many_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_one_to_many_module(INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.one_to_many", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_one_to_many_module; """ + sql """ + CREATE TABLE test_one_to_many_module ( + id INT, + count INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_one_to_many_module VALUES (1, 3), (2, 2), (3, 4); + """ + + qt_one_to_many """ + SELECT id, tmp.value + FROM test_one_to_many_module + LATERAL VIEW udtf_one_to_many_module(count) tmp AS value + ORDER BY id, tmp.value; + """ + + // ======================================== + // Pattern 3: One-to-Zero (1 input row → 0 output all_rows) + // Some input all_rows produce no output (filtering) + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_one_to_zero_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_one_to_zero_module(INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.one_to_zero", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_one_to_zero_module; """ + sql """ + CREATE TABLE test_one_to_zero_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_one_to_zero_module VALUES (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6); + """ + + qt_one_to_zero """ + SELECT tmp.value + FROM test_one_to_zero_module + LATERAL VIEW udtf_one_to_zero_module(value) tmp AS value + ORDER BY tmp.value; + """ + + // ======================================== + // Pattern 4: One-to-Variable (1 input row → 0/1/N output all_rows) + // Different input all_rows produce different numbers of output all_rows + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_one_to_variable_module(STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_one_to_variable_module(STRING) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.one_to_variable", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_one_to_variable_module; """ + sql """ + CREATE TABLE test_one_to_variable_module ( + id INT, + text STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_one_to_variable_module VALUES + (1, 'hello'), -- 1 output + (2, 'hello world'), -- 2 outputs + (3, ''), -- 0 outputs + (4, 'a b c'); -- 3 outputs + """ + + qt_one_to_variable """ + SELECT id, tmp.word + FROM test_one_to_variable_module + LATERAL VIEW udtf_one_to_variable_module(text) tmp AS word + ORDER BY id, tmp.word; + """ + + // ======================================== + // Pattern 5: Many-to-One (N input all_rows → aggregate to fewer all_rows) + // Note: This simulates batch processing where each row independently + // produces output, but conceptually represents aggregation pattern + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_aggregate_pattern_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_aggregate_pattern_module(INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.aggregate_pattern", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_aggregate_pattern_module; """ + sql """ + CREATE TABLE test_aggregate_pattern_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_aggregate_pattern_module VALUES + (1, 5), (2, 50), (3, 500), (4, 8), (5, 80), (6, 800); + """ + + qt_aggregate_pattern """ + SELECT tmp.category, COUNT(*) as count + FROM test_aggregate_pattern_module + LATERAL VIEW udtf_aggregate_pattern_module(value) tmp AS value, category + GROUP BY tmp.category + ORDER BY tmp.category; + """ + + // ======================================== + // Pattern 6: Explosive Growth (1 input row → many output all_rows) + // Testing large multiplication factor + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_explosive_module(INT, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_explosive_module(INT, INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.explosive", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_explosive_module; """ + sql """ + CREATE TABLE test_explosive_module ( + id INT, + all_rows INT, + all_cols INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_explosive_module VALUES (1, 2, 3); + """ + + qt_explosive """ + SELECT tmp.row_id, tmp.col_id + FROM test_explosive_module + LATERAL VIEW udtf_explosive_module(all_rows, all_cols) tmp AS row_id, col_id + ORDER BY tmp.row_id, tmp.col_id; + """ + + // ======================================== + // Pattern 7: Conditional Branching (different logic paths) + // Same function but different output counts based on condition + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_conditional_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_conditional_module(INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.conditional", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_conditional_module; """ + sql """ + CREATE TABLE test_conditional_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_conditional_module VALUES (1, 10), (2, -5), (3, 0), (4, 7); + """ + + qt_conditional """ + SELECT tmp.value, tmp.type + FROM test_conditional_module + LATERAL VIEW udtf_conditional_module(value) tmp AS value, type + ORDER BY tmp.value, tmp.type; + """ + + // ======================================== + // Pattern 8: All-or-Nothing (either all all_rows or no all_rows) + // Based on a condition, output all or nothing + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_all_or_nothing_module(STRING, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_all_or_nothing_module(STRING, INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.all_or_nothing", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_all_or_nothing_module; """ + sql """ + CREATE TABLE test_all_or_nothing_module ( + id INT, + text STRING, + min_len INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_all_or_nothing_module VALUES + (1, 'hello', 3), -- 5 outputs (length=5 >= 3) + (2, 'hi', 5), -- 0 outputs (length=2 < 5) + (3, 'world', 4); -- 5 outputs (length=5 >= 4) + """ + + qt_all_or_nothing """ + SELECT id, tmp.char, tmp.pos + FROM test_all_or_nothing_module + LATERAL VIEW udtf_all_or_nothing_module(text, min_len) tmp AS char, pos + ORDER BY id, tmp.pos; + """ + + // ======================================== + // Pattern 9: Empty Input Table (0 input all_rows) + // Test behavior with no input data + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_empty_input_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_empty_input_module(INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.empty_input", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_empty_input_module; """ + sql """ + CREATE TABLE test_empty_input_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + // No INSERT - table is empty + + qt_empty_input """ + SELECT tmp.value + FROM test_empty_input_module + LATERAL VIEW udtf_empty_input_module(value) tmp AS value; + """ + + // ======================================== + // Pattern 10: Batch Processing Simulation + // Multiple input all_rows, each producing variable outputs + // ======================================== + sql """ DROP FUNCTION IF EXISTS udtf_batch_process_module(INT); """ + sql """ + CREATE TABLES FUNCTION udtf_batch_process_module(INT) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.io_patterns_udtf.batch_process", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + sql """ DROP TABLE IF EXISTS test_batch_process_module; """ + sql """ + CREATE TABLE test_batch_process_module ( + id INT, + value INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_batch_process_module VALUES (1, 10), (2, 20); + """ + + qt_batch_process """ + SELECT tmp.original, tmp.factor, tmp.result + FROM test_batch_process_module + LATERAL VIEW udtf_batch_process_module(value) tmp AS original, factor, result + ORDER BY tmp.original, tmp.factor; + """ + + } finally { + // Cleanup functions + try_sql("DROP FUNCTION IF EXISTS udtf_one_to_one_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_one_to_many_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_one_to_zero_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_one_to_variable_module(STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_aggregate_pattern_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_explosive_module(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_conditional_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_all_or_nothing_module(STRING, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_empty_input_module(INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_batch_process_module(INT);") + + // Cleanup tables + try_sql("DROP TABLE IF EXISTS test_one_to_one_module;") + try_sql("DROP TABLE IF EXISTS test_one_to_many_module;") + try_sql("DROP TABLE IF EXISTS test_one_to_zero_module;") + try_sql("DROP TABLE IF EXISTS test_one_to_variable_module;") + try_sql("DROP TABLE IF EXISTS test_aggregate_pattern_module;") + try_sql("DROP TABLE IF EXISTS test_explosive_module;") + try_sql("DROP TABLE IF EXISTS test_conditional_module;") + try_sql("DROP TABLE IF EXISTS test_all_or_nothing_module;") + try_sql("DROP TABLE IF EXISTS test_empty_input_module;") + try_sql("DROP TABLE IF EXISTS test_batch_process_module;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_sql_integration_inline.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_sql_integration_inline.groovy new file mode 100644 index 00000000000000..080b862825e409 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_sql_integration_inline.groovy @@ -0,0 +1,1001 @@ +// 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. + +suite("test_pythonudtf_sql_integration_inline") { + // Test Python UDTF Integration with SQL Operations + // Coverage: WHERE, JOIN, GROUP BY, ORDER BY, LIMIT, Subqueries, CTEs + + def runtime_version = "3.10.12" + + try { + // ======================================== + // Prepare Common UDTF Functions + // ======================================== + + // Helper UDTF: Split string into multiple records + sql """ DROP FUNCTION IF EXISTS udtf_split(STRING, STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_split(STRING, STRING) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "split_with_position", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def split_with_position(text, delimiter): + '''Split string and return with position''' + if text and delimiter: + parts = text.split(delimiter) + for i, part in enumerate(parts): + yield (i, part.strip()) +\$\$; + """ + + // Helper UDTF: Generate number sequence + sql """ DROP FUNCTION IF EXISTS udtf_range(INT, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_range(INT, INT) + RETURNS ARRAY + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "generate_range", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def generate_range(start, end): + '''Generate integer range''' + if start is not None and end is not None: + for i in range(start, end + 1): + yield (i,) +\$\$; + """ + + // Helper UDTF: Expand array elements + sql """ DROP FUNCTION IF EXISTS udtf_explode_array(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_explode_array(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "type" = "PYTHON_UDF", + "symbol" = "explode_with_index", + "runtime_version" = "3.10.12" + ) + AS \$\$ +def explode_with_index(arr): + '''Explode array with index''' + if arr: + for i, elem in enumerate(arr): + if elem is not None: + yield (elem, i) +\$\$; + """ + + // ======================================== + // Section 1: UDTF with WHERE Clause + // ======================================== + + // Test 1.1: Filter BEFORE UDTF (reduce input) + sql """ DROP TABLE IF EXISTS test_where_before; """ + sql """ + CREATE TABLE test_where_before ( + id INT, + category STRING, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_where_before VALUES + (1, 'A', 'apple,banana'), + (2, 'B', 'cat,dog'), + (3, 'A', 'red,green,blue'), + (4, 'C', 'one,two'); + """ + + qt_where_before """ + SELECT id, category, tmp.position, tmp.value + FROM test_where_before + LATERAL VIEW udtf_split(data, ',') tmp AS position, value + WHERE category = 'A' + ORDER BY id, tmp.position; + """ + + // Test 1.2: Filter AFTER UDTF (filter expanded results) + qt_where_after """ + SELECT id, tmp.position, tmp.value + FROM test_where_before + LATERAL VIEW udtf_split(data, ',') tmp AS position, value + WHERE tmp.value LIKE '%e%' + ORDER BY id, tmp.position; + """ + + // Test 1.3: Combined Filter (before and after UDTF) + qt_where_combined """ + SELECT id, category, tmp.value + FROM test_where_before + LATERAL VIEW udtf_split(data, ',') tmp AS position, value + WHERE category IN ('A', 'B') AND tmp.position = 0 + ORDER BY id; + """ + + // ======================================== + // Section 2: UDTF with JOIN Operations + // ======================================== + + // Prepare dimension table + sql """ DROP TABLE IF EXISTS dim_numbers; """ + sql """ + CREATE TABLE dim_numbers ( + num INT, + num_name STRING, + is_even BOOLEAN + ) ENGINE=OLAP + DUPLICATE KEY(num) + DISTRIBUTED BY HASH(num) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO dim_numbers VALUES + (1, 'one', false), + (2, 'two', true), + (3, 'three', false), + (4, 'four', true), + (5, 'five', false); + """ + + // Prepare fact table + sql """ DROP TABLE IF EXISTS fact_ranges; """ + sql """ + CREATE TABLE fact_ranges ( + id INT, + start_num INT, + end_num INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO fact_ranges VALUES + (1, 1, 3), + (2, 2, 4); + """ + + // Test 2.1: INNER JOIN with UDTF + qt_join_inner """ + SELECT + f.id, + tmp.num, + d.num_name, + d.is_even + FROM fact_ranges f + LATERAL VIEW udtf_range(f.start_num, f.end_num) tmp AS num + INNER JOIN dim_numbers d ON tmp.num = d.num + ORDER BY f.id, tmp.num; + """ + + // Test 2.2: LEFT JOIN with UDTF (some generated values may not match) + sql """ DROP TABLE IF EXISTS fact_ranges_extended; """ + sql """ + CREATE TABLE fact_ranges_extended ( + id INT, + start_num INT, + end_num INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO fact_ranges_extended VALUES + (1, 1, 2), + (2, 5, 7); + """ + + qt_join_left """ + SELECT + f.id, + tmp.num, + d.num_name + FROM fact_ranges_extended f + LATERAL VIEW udtf_range(f.start_num, f.end_num) tmp AS num + LEFT JOIN dim_numbers d ON tmp.num = d.num + ORDER BY f.id, tmp.num; + """ + + // Test 2.3: Self-JOIN through UDTF + sql """ DROP TABLE IF EXISTS test_self_join; """ + sql """ + CREATE TABLE test_self_join ( + id INT, + value_list STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_self_join VALUES + (1, '10,20,30'), + (2, '20,30,40'); + """ + + qt_join_self """ + SELECT + t1.id AS id1, + value1, + t2.id AS id2, + value2 + FROM test_self_join t1 + LATERAL VIEW udtf_split(t1.value_list, ',') tmp1 AS pos1, value1 + INNER JOIN test_self_join t2 + LATERAL VIEW udtf_split(t2.value_list, ',') tmp2 AS pos2, value2 + ON value1 = value2 AND t1.id < t2.id + ORDER BY t1.id, value1, t2.id; + """ + + // ======================================== + // Section 3: UDTF with GROUP BY and Aggregation + // ======================================== + + sql """ DROP TABLE IF EXISTS test_group_by; """ + sql """ + CREATE TABLE test_group_by ( + id INT, + category STRING, + tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_group_by VALUES + (1, 'fruit', 'apple,banana,apple'), + (2, 'fruit', 'banana,cherry'), + (3, 'animal', 'cat,dog,cat'); + """ + + // Test 3.1: GROUP BY after UDTF expansion + qt_group_by_udtf """ + SELECT + tmp.value AS tag, + COUNT(*) AS occurrence_count + FROM test_group_by + LATERAL VIEW udtf_split(tags, ',') tmp AS position, value + GROUP BY tmp.value + ORDER BY occurrence_count DESC, tag; + """ + + // Test 3.2: GROUP BY with original table columns + qt_group_by_mixed """ + SELECT + category, + tmp.value AS tag, + COUNT(*) AS tag_count + FROM test_group_by + LATERAL VIEW udtf_split(tags, ',') tmp AS position, value + GROUP BY category, tmp.value + ORDER BY category, tag_count DESC; + """ + + // Test 3.3: Aggregation with HAVING clause + qt_group_by_having """ + SELECT + tmp.value AS tag, + COUNT(*) AS cnt + FROM test_group_by + LATERAL VIEW udtf_split(tags, ',') tmp AS position, value + GROUP BY tmp.value + HAVING COUNT(*) > 1 + ORDER BY cnt DESC, tag; + """ + + // Test 3.4: Multiple aggregation functions + sql """ DROP TABLE IF EXISTS test_agg_numbers; """ + sql """ + CREATE TABLE test_agg_numbers ( + id INT, + start_val INT, + end_val INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_agg_numbers VALUES + (1, 1, 5), + (2, 3, 7), + (3, 10, 12); + """ + + qt_group_by_multi_agg """ + SELECT + id, + COUNT(*) AS total_count, + MIN(tmp.num) AS min_num, + MAX(tmp.num) AS max_num, + SUM(tmp.num) AS sum_num, + AVG(tmp.num) AS avg_num + FROM test_agg_numbers + LATERAL VIEW udtf_range(start_val, end_val) tmp AS num + GROUP BY id + ORDER BY id; + """ + + // ======================================== + // Section 4: UDTF with ORDER BY and LIMIT + // ======================================== + + sql """ DROP TABLE IF EXISTS test_order_limit; """ + sql """ + CREATE TABLE test_order_limit ( + id INT, + name STRING, + scores STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_order_limit VALUES + (1, 'Alice', '85,92,78'), + (2, 'Bob', '90,88,95'), + (3, 'Charlie', '70,82,88'); + """ + + // Test 4.1: ORDER BY UDTF output + qt_order_by_udtf """ + SELECT + id, + name, + tmp.value AS score + FROM test_order_limit + LATERAL VIEW udtf_split(scores, ',') tmp AS position, value + ORDER BY CAST(tmp.value AS INT) DESC, name + LIMIT 5; + """ + + // Test 4.2: ORDER BY original and UDTF columns + qt_order_by_mixed """ + SELECT + id, + name, + tmp.position, + tmp.value AS score + FROM test_order_limit + LATERAL VIEW udtf_split(scores, ',') tmp AS position, value + ORDER BY id ASC, tmp.position DESC; + """ + + // Test 4.3: LIMIT without ORDER BY + qt_limit_only """ + SELECT + id, + tmp.value + FROM test_order_limit + LATERAL VIEW udtf_split(scores, ',') tmp AS position, value + LIMIT 3; + """ + + // Test 4.4: TOP-N pattern (ORDER BY + LIMIT per group) + qt_top_n_pattern """ + SELECT id, name, score + FROM ( + SELECT + id, + name, + CAST(tmp.value AS INT) AS score, + ROW_NUMBER() OVER (PARTITION BY id ORDER BY CAST(tmp.value AS INT) DESC) AS rn + FROM test_order_limit + LATERAL VIEW udtf_split(scores, ',') tmp AS position, value + ) ranked + WHERE rn <= 2 + ORDER BY id, score DESC; + """ + + // ======================================== + // Section 5: UDTF in Subqueries + // ======================================== + + sql """ DROP TABLE IF EXISTS test_subquery; """ + sql """ + CREATE TABLE test_subquery ( + id INT, + item_list STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_subquery VALUES + (1, 'A,B,C'), + (2, 'B,C,D'), + (3, 'A,C,E'); + """ + + // Test 5.1: UDTF in WHERE IN subquery + qt_subquery_in """ + SELECT id, item_list + FROM test_subquery + WHERE id IN ( + SELECT DISTINCT id + FROM test_subquery + LATERAL VIEW udtf_split(item_list, ',') tmp AS position, value + WHERE tmp.value = 'A' + ) + ORDER BY id; + """ + + // Test 5.2: UDTF in FROM subquery + qt_subquery_from """ + SELECT + item, + COUNT(DISTINCT source_id) AS source_count + FROM ( + SELECT id AS source_id, tmp.value AS item + FROM test_subquery + LATERAL VIEW udtf_split(item_list, ',') tmp AS position, value + ) expanded + GROUP BY item + ORDER BY source_count DESC, item; + """ + + // Test 5.3: Nested subqueries with UDTF + qt_subquery_nested """ + SELECT item, total_occurrences + FROM ( + SELECT item, COUNT(*) AS total_occurrences + FROM ( + SELECT id, tmp.value AS item + FROM test_subquery + LATERAL VIEW udtf_split(item_list, ',') tmp AS position, value + ) level1 + GROUP BY item + ) level2 + WHERE total_occurrences >= 2 + ORDER BY total_occurrences DESC, item; + """ + + // ======================================== + // Section 6: UDTF with DISTINCT + // ======================================== + + sql """ DROP TABLE IF EXISTS test_distinct; """ + sql """ + CREATE TABLE test_distinct ( + id INT, + tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_distinct VALUES + (1, 'red,blue,red'), + (2, 'blue,green'), + (3, 'red,yellow'); + """ + + // Test 6.1: DISTINCT on UDTF output + qt_distinct_udtf """ + SELECT DISTINCT tmp.value AS tag + FROM test_distinct + LATERAL VIEW udtf_split(tags, ',') tmp AS position, value + ORDER BY tag; + """ + + // Test 6.2: COUNT DISTINCT + qt_count_distinct """ + SELECT COUNT(DISTINCT tmp.value) AS unique_tag_count + FROM test_distinct + LATERAL VIEW udtf_split(tags, ',') tmp AS position, value; + """ + + // ======================================== + // Section 7: UDTF with UNION + // ======================================== + + sql """ DROP TABLE IF EXISTS test_union_a; """ + sql """ + CREATE TABLE test_union_a ( + id INT, + items STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_union_a VALUES (1, 'X,Y'); + """ + + sql """ DROP TABLE IF EXISTS test_union_b; """ + sql """ + CREATE TABLE test_union_b ( + id INT, + items STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_union_b VALUES (2, 'Y,Z'); + """ + + // Test 7.1: UNION ALL with UDTF + qt_union_all """ + SELECT id, tmp.value AS item + FROM test_union_a + LATERAL VIEW udtf_split(items, ',') tmp AS position, value + UNION ALL + SELECT id, tmp.value AS item + FROM test_union_b + LATERAL VIEW udtf_split(items, ',') tmp AS position, value + ORDER BY id, item; + """ + + // Test 7.2: UNION (removes duplicates) + qt_union_distinct """ + SELECT tmp.value AS item + FROM test_union_a + LATERAL VIEW udtf_split(items, ',') tmp AS position, value + UNION + SELECT tmp.value AS item + FROM test_union_b + LATERAL VIEW udtf_split(items, ',') tmp AS position, value + ORDER BY item; + """ + + // ======================================== + // Section 8: UDTF with Complex Array Operations + // ======================================== + + sql """ DROP TABLE IF EXISTS test_array_ops; """ + sql """ + CREATE TABLE test_array_ops ( + id INT, + numbers ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_ops VALUES + (1, [1, 2, 3]), + (2, [2, 3, 4, 5]), + (3, [3, 4]); + """ + + // Test 8.1: Filter array elements through UDTF + qt_array_filter """ + SELECT + id, + tmp.element + FROM test_array_ops + LATERAL VIEW udtf_explode_array(numbers) tmp AS element, element_index + WHERE tmp.element > 2 + ORDER BY id, tmp.element; + """ + + // Test 8.2: Aggregate array elements + qt_array_aggregate """ + SELECT + id, + COUNT(*) AS element_count, + SUM(tmp.element) AS element_sum, + AVG(tmp.element) AS element_avg + FROM test_array_ops + LATERAL VIEW udtf_explode_array(numbers) tmp AS element, element_index + GROUP BY id + ORDER BY id; + """ + + // ======================================== + // Section 9: UDTF with Window Functions + // ======================================== + + sql """ DROP TABLE IF EXISTS test_window; """ + sql """ + CREATE TABLE test_window ( + id INT, + category STRING, + value_list STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_window VALUES + (1, 'A', '10,20,30'), + (2, 'A', '15,25'), + (3, 'B', '5,10,15'); + """ + + // Test 9.1: Window function over UDTF results + qt_window_function """ + SELECT + id, + category, + CAST(tmp.value AS INT) AS val, + ROW_NUMBER() OVER (PARTITION BY category ORDER BY CAST(tmp.value AS INT)) AS rn, + SUM(CAST(tmp.value AS INT)) OVER (PARTITION BY category) AS category_total + FROM test_window + LATERAL VIEW udtf_split(value_list, ',') tmp AS position, value + ORDER BY category, val; + """ + + // ======================================== + // Section 10: UDTF with CASE WHEN + // ======================================== + + sql """ DROP TABLE IF EXISTS test_case_when; """ + sql """ + CREATE TABLE test_case_when ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_case_when VALUES + (1, '5,15,25'), + (2, '10,20,30'); + """ + + // Test 10.1: CASE WHEN on UDTF results + qt_case_when """ + SELECT + id, + tmp.value, + CASE + WHEN CAST(tmp.value AS INT) < 10 THEN 'small' + WHEN CAST(tmp.value AS INT) < 20 THEN 'medium' + ELSE 'large' + END AS size_category + FROM test_case_when + LATERAL VIEW udtf_split(data, ',') tmp AS position, value + ORDER BY id, CAST(tmp.value AS INT); + """ + + // ======================================== + // Section 11: - Multiple LATERAL VIEW Nesting + // ======================================== + + // Test 11.1: Two-level LATERAL VIEW nesting (sequential) + sql """ DROP TABLE IF EXISTS test_nested_2level; """ + sql """ + CREATE TABLE test_nested_2level ( + id INT, + categories STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_2level VALUES + (1, 'A:1,2|B:3'), + (2, 'C:4,5'); + """ + + qt_nested_2level """ + SELECT + id, + cat, + CAST(num AS INT) as num + FROM test_nested_2level + LATERAL VIEW udtf_split(categories, '|') t1 AS p1, cat_nums + LATERAL VIEW udtf_split(cat_nums, ':') t2 AS p2, cat + LATERAL VIEW udtf_split(cat, ',') t3 AS p3, num + WHERE p2 = 1 + ORDER BY id, cat, num; + """ + + // Test 11.2: Parallel LATERAL VIEWs (cartesian product) + sql """ DROP TABLE IF EXISTS test_parallel_lateral; """ + sql """ + CREATE TABLE test_parallel_lateral ( + id INT, + list1 STRING, + list2 STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_parallel_lateral VALUES + (1, 'A,B', 'X,Y'), + (2, 'C', 'Z'); + """ + + qt_parallel_lateral """ + SELECT + id, + item1, + item2 + FROM test_parallel_lateral + LATERAL VIEW udtf_split(list1, ',') t1 AS p1, item1 + LATERAL VIEW udtf_split(list2, ',') t2 AS p2, item2 + ORDER BY id, item1, item2; + """ + + // Test 11.3: Nested LATERAL VIEW with JOIN + sql """ DROP TABLE IF EXISTS test_nested_join_base; """ + sql """ + CREATE TABLE test_nested_join_base ( + user_id INT, + tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(user_id) + DISTRIBUTED BY HASH(user_id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_join_base VALUES + (1, 'sports:soccer,tennis|food:pizza'), + (2, 'music:rock'); + """ + + sql """ DROP TABLE IF EXISTS dim_tag_info; """ + sql """ + CREATE TABLE dim_tag_info ( + tag VARCHAR(50), + score INT + ) ENGINE=OLAP + DUPLICATE KEY(tag) + DISTRIBUTED BY HASH(tag) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO dim_tag_info VALUES + ('soccer', 10), + ('tennis', 8), + ('pizza', 5), + ('rock', 9); + """ + + qt_nested_join """ + SELECT + u.user_id, + tag_name, + d.score + FROM test_nested_join_base u + LATERAL VIEW udtf_split(u.tags, '|') t1 AS p1, cat_tags + LATERAL VIEW udtf_split(cat_tags, ':') t2 AS p2, part + LATERAL VIEW udtf_split(part, ',') t3 AS p3, tag_name + INNER JOIN dim_tag_info d ON d.tag = tag_name + WHERE p2 = 1 + ORDER BY u.user_id, d.score DESC; + """ + + // Test 11.4: Nested LATERAL VIEW with GROUP BY aggregation + sql """ DROP TABLE IF EXISTS test_nested_groupby; """ + sql """ + CREATE TABLE test_nested_groupby ( + store_id INT, + sales_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(store_id) + DISTRIBUTED BY HASH(store_id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_groupby VALUES + (1, 'day1:100,200|day2:150'), + (2, 'day1:300|day2:250,100'); + """ + + qt_nested_groupby """ + SELECT + store_id, + COUNT(*) as sale_count, + SUM(CAST(amount AS INT)) as total_amount + FROM test_nested_groupby + LATERAL VIEW udtf_split(sales_data, '|') t1 AS p1, day_amounts + LATERAL VIEW udtf_split(day_amounts, ':') t2 AS p2, part + LATERAL VIEW udtf_split(part, ',') t3 AS p3, amount + WHERE p2 = 1 + GROUP BY store_id + ORDER BY store_id; + """ + + // Test 11.5: Three-level deep nesting + sql """ DROP TABLE IF EXISTS test_nested_3level; """ + sql """ + CREATE TABLE test_nested_3level ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_3level VALUES + (1, 'A,B,C|D,E|F'); + """ + + qt_nested_3level """ + SELECT + id, + grp_pos, + item + FROM test_nested_3level + LATERAL VIEW udtf_split(data, '|') t1 AS grp_pos, group_items + LATERAL VIEW udtf_split(group_items, ',') t2 AS item_pos, item + ORDER BY id, grp_pos, item_pos; + """ + + // Test 11.6: Nested with array expansion + sql """ DROP TABLE IF EXISTS test_nested_array_expansion; """ + sql """ + CREATE TABLE test_nested_array_expansion ( + id INT, + group_id INT, + numbers ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_array_expansion VALUES + (1, 1, [10, 20]), + (1, 2, [30]), + (2, 1, [40, 50]); + """ + + qt_nested_array_expansion """ + SELECT + id, + group_id, + element + FROM test_nested_array_expansion + LATERAL VIEW udtf_explode_array(numbers) t1 AS element, idx + ORDER BY id, group_id, element; + """ + + // Test 11.7: Nested with WHERE filtering at multiple levels + sql """ DROP TABLE IF EXISTS test_nested_multifilter; """ + sql """ + CREATE TABLE test_nested_multifilter ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_multifilter VALUES + (1, 'A:10,20,30|B:40'), + (2, 'C:50,60'); + """ + + qt_nested_multifilter """ + SELECT + id, + cat_name, + CAST(num AS INT) as num + FROM ( + SELECT + id, + p1, + CASE WHEN p2 = 0 THEN part END AS cat_name, + CASE WHEN p2 = 1 THEN part END AS nums + FROM test_nested_multifilter + LATERAL VIEW udtf_split(data, '|') t1 AS p1, cat_nums + LATERAL VIEW udtf_split(cat_nums, ':') t2 AS p2, part + ) t + LATERAL VIEW udtf_split(nums, ',') t3 AS p3, num + WHERE nums IS NOT NULL AND CAST(num AS INT) >= 20 + ORDER BY id, p1, num; + """ + + // Test 11.8: Nested with DISTINCT across levels + sql """ DROP TABLE IF EXISTS test_nested_distinct; """ + sql """ + CREATE TABLE test_nested_distinct ( + id INT, + tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_distinct VALUES + (1, 'red,blue|red,green'), + (2, 'blue,yellow'); + """ + + qt_nested_distinct """ + SELECT DISTINCT color + FROM test_nested_distinct + LATERAL VIEW udtf_split(tags, '|') t1 AS p1, color_list + LATERAL VIEW udtf_split(color_list, ',') t2 AS p2, color + ORDER BY color; + """ + + } finally { + try_sql("DROP FUNCTION IF EXISTS udtf_split(STRING, STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_range(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_explode_array(ARRAY);") + try_sql("DROP TABLE IF EXISTS test_where_before;") + try_sql("DROP TABLE IF EXISTS dim_numbers;") + try_sql("DROP TABLE IF EXISTS fact_ranges;") + try_sql("DROP TABLE IF EXISTS fact_ranges_extended;") + try_sql("DROP TABLE IF EXISTS test_self_join;") + try_sql("DROP TABLE IF EXISTS test_group_by;") + try_sql("DROP TABLE IF EXISTS test_agg_numbers;") + try_sql("DROP TABLE IF EXISTS test_order_limit;") + try_sql("DROP TABLE IF EXISTS test_subquery;") + try_sql("DROP TABLE IF EXISTS test_distinct;") + try_sql("DROP TABLE IF EXISTS test_union_a;") + try_sql("DROP TABLE IF EXISTS test_union_b;") + try_sql("DROP TABLE IF EXISTS test_array_ops;") + try_sql("DROP TABLE IF EXISTS test_window;") + try_sql("DROP TABLE IF EXISTS test_case_when;") + try_sql("DROP TABLE IF EXISTS test_nested_2level;") + try_sql("DROP TABLE IF EXISTS test_parallel_lateral;") + try_sql("DROP TABLE IF EXISTS test_nested_join_base;") + try_sql("DROP TABLE IF EXISTS dim_tag_info;") + try_sql("DROP TABLE IF EXISTS test_nested_groupby;") + try_sql("DROP TABLE IF EXISTS test_nested_3level;") + try_sql("DROP TABLE IF EXISTS test_nested_array_expansion;") + try_sql("DROP TABLE IF EXISTS test_nested_multifilter;") + try_sql("DROP TABLE IF EXISTS test_nested_distinct;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/test_pythonudtf_sql_integration_module.groovy b/regression-test/suites/pythonudtf_p0/test_pythonudtf_sql_integration_module.groovy new file mode 100644 index 00000000000000..3928c2560e2f00 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/test_pythonudtf_sql_integration_module.groovy @@ -0,0 +1,984 @@ +// 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. + +suite("test_pythonudtf_sql_integration_module") { + // Test Python UDTF Integration with SQL Operations + // Coverage: WHERE, JOIN, GROUP BY, ORDER BY, LIMIT, Subqueries, CTEs + + def pyPath = """${context.file.parent}/udtf_scripts/pyudtf.zip""" + scp_udf_file_to_all_be(pyPath) + def runtime_version = "3.10.12" + log.info("Python zip path: ${pyPath}".toString()) + + try { + // ======================================== + // Prepare Common UDTF Functions + // ======================================== + + // Helper UDTF: Split string into multiple records + sql """ DROP FUNCTION IF EXISTS udtf_split_module(STRING, STRING); """ + sql """ + CREATE TABLES FUNCTION udtf_split_module(STRING, STRING) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.sql_integration_udtf.split_with_position", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + // Helper UDTF: Generate number sequence + sql """ DROP FUNCTION IF EXISTS udtf_range_module(INT, INT); """ + sql """ + CREATE TABLES FUNCTION udtf_range_module(INT, INT) + RETURNS ARRAY + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.sql_integration_udtf.generate_range", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + // Helper UDTF: Expand array elements + sql """ DROP FUNCTION IF EXISTS udtf_explode_array_module(ARRAY); """ + sql """ + CREATE TABLES FUNCTION udtf_explode_array_module(ARRAY) + RETURNS ARRAY> + PROPERTIES ( + "file" = "file://${pyPath}", + "symbol" = "pyudtf_module.sql_integration_udtf.explode_with_index", + "type" = "PYTHON_UDF", + "runtime_version" = "${runtime_version}" + ); + """ + + // ======================================== + // Section 1: UDTF with WHERE Clause + // ======================================== + + // Test 1.1: Filter BEFORE UDTF (reduce input) + sql """ DROP TABLE IF EXISTS test_where_before_module; """ + sql """ + CREATE TABLE test_where_before_module ( + id INT, + category STRING, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_where_before_module VALUES + (1, 'A', 'apple,banana'), + (2, 'B', 'cat,dog'), + (3, 'A', 'red,green,blue'), + (4, 'C', 'one,two'); + """ + + qt_where_before """ + SELECT id, category, tmp.position, tmp.value + FROM test_where_before_module + LATERAL VIEW udtf_split_module(data, ',') tmp AS position, value + WHERE category = 'A' + ORDER BY id, tmp.position; + """ + + // Test 1.2: Filter AFTER UDTF (filter expanded results) + qt_where_after """ + SELECT id, tmp.position, tmp.value + FROM test_where_before_module + LATERAL VIEW udtf_split_module(data, ',') tmp AS position, value + WHERE tmp.value LIKE '%e%' + ORDER BY id, tmp.position; + """ + + // Test 1.3: Combined Filter (before and after UDTF) + qt_where_combined """ + SELECT id, category, tmp.value + FROM test_where_before_module + LATERAL VIEW udtf_split_module(data, ',') tmp AS position, value + WHERE category IN ('A', 'B') AND tmp.position = 0 + ORDER BY id; + """ + + // ======================================== + // Section 2: UDTF with JOIN Operations + // ======================================== + + // Prepare dimension table + sql """ DROP TABLE IF EXISTS dim_numbers_module; """ + sql """ + CREATE TABLE dim_numbers_module ( + num INT, + num_name STRING, + is_even BOOLEAN + ) ENGINE=OLAP + DUPLICATE KEY(num) + DISTRIBUTED BY HASH(num) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO dim_numbers_module VALUES + (1, 'one', false), + (2, 'two', true), + (3, 'three', false), + (4, 'four', true), + (5, 'five', false); + """ + + // Prepare fact table + sql """ DROP TABLE IF EXISTS fact_ranges_module; """ + sql """ + CREATE TABLE fact_ranges_module ( + id INT, + start_num INT, + end_num INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO fact_ranges_module VALUES + (1, 1, 3), + (2, 2, 4); + """ + + // Test 2.1: INNER JOIN with UDTF + qt_join_inner """ + SELECT + f.id, + tmp.num, + d.num_name, + d.is_even + FROM fact_ranges_module f + LATERAL VIEW udtf_range_module(f.start_num, f.end_num) tmp AS num + INNER JOIN dim_numbers_module d ON tmp.num = d.num + ORDER BY f.id, tmp.num; + """ + + // Test 2.2: LEFT JOIN with UDTF (some generated values may not match) + sql """ DROP TABLE IF EXISTS fact_ranges_extended_module; """ + sql """ + CREATE TABLE fact_ranges_extended_module ( + id INT, + start_num INT, + end_num INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO fact_ranges_extended_module VALUES + (1, 1, 2), + (2, 5, 7); + """ + + qt_join_left """ + SELECT + f.id, + tmp.num, + d.num_name + FROM fact_ranges_extended_module f + LATERAL VIEW udtf_range_module(f.start_num, f.end_num) tmp AS num + LEFT JOIN dim_numbers_module d ON tmp.num = d.num + ORDER BY f.id, tmp.num; + """ + + // Test 2.3: Self-JOIN through UDTF + sql """ DROP TABLE IF EXISTS test_self_join_module; """ + sql """ + CREATE TABLE test_self_join_module ( + id INT, + value_list STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_self_join_module VALUES + (1, '10,20,30'), + (2, '20,30,40'); + """ + + qt_join_self """ + SELECT + t1.id AS id1, + value1, + t2.id AS id2, + value2 + FROM test_self_join_module t1 + LATERAL VIEW udtf_split_module(t1.value_list, ',') tmp1 AS pos1, value1 + INNER JOIN test_self_join_module t2 + LATERAL VIEW udtf_split_module(t2.value_list, ',') tmp2 AS pos2, value2 + ON value1 = value2 AND t1.id < t2.id + ORDER BY t1.id, value1, t2.id; + """ + + // ======================================== + // Section 3: UDTF with GROUP BY and Aggregation + // ======================================== + + sql """ DROP TABLE IF EXISTS test_group_by_module; """ + sql """ + CREATE TABLE test_group_by_module ( + id INT, + category STRING, + tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_group_by_module VALUES + (1, 'fruit', 'apple,banana,apple'), + (2, 'fruit', 'banana,cherry'), + (3, 'animal', 'cat,dog,cat'); + """ + + // Test 3.1: GROUP BY after UDTF expansion + qt_group_by_udtf """ + SELECT + tmp.value AS tag, + COUNT(*) AS occurrence_count + FROM test_group_by_module + LATERAL VIEW udtf_split_module(tags, ',') tmp AS position, value + GROUP BY tmp.value + ORDER BY occurrence_count DESC, tag; + """ + + // Test 3.2: GROUP BY with original table columns + qt_group_by_mixed """ + SELECT + category, + tmp.value AS tag, + COUNT(*) AS tag_count + FROM test_group_by_module + LATERAL VIEW udtf_split_module(tags, ',') tmp AS position, value + GROUP BY category, tmp.value + ORDER BY category, tag_count DESC; + """ + + // Test 3.3: Aggregation with HAVING clause + qt_group_by_having """ + SELECT + tmp.value AS tag, + COUNT(*) AS cnt + FROM test_group_by_module + LATERAL VIEW udtf_split_module(tags, ',') tmp AS position, value + GROUP BY tmp.value + HAVING COUNT(*) > 1 + ORDER BY cnt DESC, tag; + """ + + // Test 3.4: Multiple aggregation functions + sql """ DROP TABLE IF EXISTS test_agg_numbers_module; """ + sql """ + CREATE TABLE test_agg_numbers_module ( + id INT, + start_val INT, + end_val INT + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_agg_numbers_module VALUES + (1, 1, 5), + (2, 3, 7), + (3, 10, 12); + """ + + qt_group_by_multi_agg """ + SELECT + id, + COUNT(*) AS total_count, + MIN(tmp.num) AS min_num, + MAX(tmp.num) AS max_num, + SUM(tmp.num) AS sum_num, + AVG(tmp.num) AS avg_num + FROM test_agg_numbers_module + LATERAL VIEW udtf_range_module(start_val, end_val) tmp AS num + GROUP BY id + ORDER BY id; + """ + + // ======================================== + // Section 4: UDTF with ORDER BY and LIMIT + // ======================================== + + sql """ DROP TABLE IF EXISTS test_order_limit_module; """ + sql """ + CREATE TABLE test_order_limit_module ( + id INT, + name STRING, + scores STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_order_limit_module VALUES + (1, 'Alice', '85,92,78'), + (2, 'Bob', '90,88,95'), + (3, 'Charlie', '70,82,88'); + """ + + // Test 4.1: ORDER BY UDTF output + qt_order_by_udtf """ + SELECT + id, + name, + tmp.value AS score + FROM test_order_limit_module + LATERAL VIEW udtf_split_module(scores, ',') tmp AS position, value + ORDER BY CAST(tmp.value AS INT) DESC, name + LIMIT 5; + """ + + // Test 4.2: ORDER BY original and UDTF columns + qt_order_by_mixed """ + SELECT + id, + name, + tmp.position, + tmp.value AS score + FROM test_order_limit_module + LATERAL VIEW udtf_split_module(scores, ',') tmp AS position, value + ORDER BY id ASC, tmp.position DESC; + """ + + // Test 4.3: LIMIT without ORDER BY + qt_limit_only """ + SELECT + id, + tmp.value + FROM test_order_limit_module + LATERAL VIEW udtf_split_module(scores, ',') tmp AS position, value + LIMIT 3; + """ + + // Test 4.4: TOP-N pattern (ORDER BY + LIMIT per group) + qt_top_n_pattern """ + SELECT id, name, score + FROM ( + SELECT + id, + name, + CAST(tmp.value AS INT) AS score, + ROW_NUMBER() OVER (PARTITION BY id ORDER BY CAST(tmp.value AS INT) DESC) AS rn + FROM test_order_limit_module + LATERAL VIEW udtf_split_module(scores, ',') tmp AS position, value + ) ranked + WHERE rn <= 2 + ORDER BY id, score DESC; + """ + + // ======================================== + // Section 5: UDTF in Subqueries + // ======================================== + + sql """ DROP TABLE IF EXISTS test_subquery_module; """ + sql """ + CREATE TABLE test_subquery_module ( + id INT, + item_list STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_subquery_module VALUES + (1, 'A,B,C'), + (2, 'B,C,D'), + (3, 'A,C,E'); + """ + + // Test 5.1: UDTF in WHERE IN subquery + qt_subquery_in """ + SELECT id, item_list + FROM test_subquery_module + WHERE id IN ( + SELECT DISTINCT id + FROM test_subquery_module + LATERAL VIEW udtf_split_module(item_list, ',') tmp AS position, value + WHERE tmp.value = 'A' + ) + ORDER BY id; + """ + + // Test 5.2: UDTF in FROM subquery + qt_subquery_from """ + SELECT + item, + COUNT(DISTINCT source_id) AS source_count + FROM ( + SELECT id AS source_id, tmp.value AS item + FROM test_subquery_module + LATERAL VIEW udtf_split_module(item_list, ',') tmp AS position, value + ) expanded + GROUP BY item + ORDER BY source_count DESC, item; + """ + + // Test 5.3: Nested subqueries with UDTF + qt_subquery_nested """ + SELECT item, total_occurrences + FROM ( + SELECT item, COUNT(*) AS total_occurrences + FROM ( + SELECT id, tmp.value AS item + FROM test_subquery_module + LATERAL VIEW udtf_split_module(item_list, ',') tmp AS position, value + ) level1 + GROUP BY item + ) level2 + WHERE total_occurrences >= 2 + ORDER BY total_occurrences DESC, item; + """ + + // ======================================== + // Section 6: UDTF with DISTINCT + // ======================================== + + sql """ DROP TABLE IF EXISTS test_distinct_module; """ + sql """ + CREATE TABLE test_distinct_module ( + id INT, + tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_distinct_module VALUES + (1, 'red,blue,red'), + (2, 'blue,green'), + (3, 'red,yellow'); + """ + + // Test 6.1: DISTINCT on UDTF output + qt_distinct_udtf """ + SELECT DISTINCT tmp.value AS tag + FROM test_distinct_module + LATERAL VIEW udtf_split_module(tags, ',') tmp AS position, value + ORDER BY tag; + """ + + // Test 6.2: COUNT DISTINCT + qt_count_distinct """ + SELECT COUNT(DISTINCT tmp.value) AS unique_tag_count + FROM test_distinct_module + LATERAL VIEW udtf_split_module(tags, ',') tmp AS position, value; + """ + + // ======================================== + // Section 7: UDTF with UNION + // ======================================== + + sql """ DROP TABLE IF EXISTS test_union_a_module; """ + sql """ + CREATE TABLE test_union_a_module ( + id INT, + items STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_union_a_module VALUES (1, 'X,Y'); + """ + + sql """ DROP TABLE IF EXISTS test_union_b_module; """ + sql """ + CREATE TABLE test_union_b_module ( + id INT, + items STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_union_b_module VALUES (2, 'Y,Z'); + """ + + // Test 7.1: UNION ALL with UDTF + qt_union_all """ + SELECT id, tmp.value AS item + FROM test_union_a_module + LATERAL VIEW udtf_split_module(items, ',') tmp AS position, value + UNION ALL + SELECT id, tmp.value AS item + FROM test_union_b_module + LATERAL VIEW udtf_split_module(items, ',') tmp AS position, value + ORDER BY id, item; + """ + + // Test 7.2: UNION (removes duplicates) + qt_union_distinct """ + SELECT tmp.value AS item + FROM test_union_a_module + LATERAL VIEW udtf_split_module(items, ',') tmp AS position, value + UNION + SELECT tmp.value AS item + FROM test_union_b_module + LATERAL VIEW udtf_split_module(items, ',') tmp AS position, value + ORDER BY item; + """ + + // ======================================== + // Section 8: UDTF with Complex Array Operations + // ======================================== + + sql """ DROP TABLE IF EXISTS test_array_ops_module; """ + sql """ + CREATE TABLE test_array_ops_module ( + id INT, + numbers ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_array_ops_module VALUES + (1, [1, 2, 3]), + (2, [2, 3, 4, 5]), + (3, [3, 4]); + """ + + // Test 8.1: Filter array elements through UDTF + qt_array_filter """ + SELECT + id, + tmp.element + FROM test_array_ops_module + LATERAL VIEW udtf_explode_array_module(numbers) tmp AS element, element_index + WHERE tmp.element > 2 + ORDER BY id, tmp.element; + """ + + // Test 8.2: Aggregate array elements + qt_array_aggregate """ + SELECT + id, + COUNT(*) AS element_count, + SUM(tmp.element) AS element_sum, + AVG(tmp.element) AS element_avg + FROM test_array_ops_module + LATERAL VIEW udtf_explode_array_module(numbers) tmp AS element, element_index + GROUP BY id + ORDER BY id; + """ + + // ======================================== + // Section 9: UDTF with Window Functions + // ======================================== + + sql """ DROP TABLE IF EXISTS test_window_module; """ + sql """ + CREATE TABLE test_window_module ( + id INT, + category STRING, + value_list STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_window_module VALUES + (1, 'A', '10,20,30'), + (2, 'A', '15,25'), + (3, 'B', '5,10,15'); + """ + + // Test 9.1: Window function over UDTF results + qt_window_function """ + SELECT + id, + category, + CAST(tmp.value AS INT) AS val, + ROW_NUMBER() OVER (PARTITION BY category ORDER BY CAST(tmp.value AS INT)) AS rn, + SUM(CAST(tmp.value AS INT)) OVER (PARTITION BY category) AS category_total + FROM test_window_module + LATERAL VIEW udtf_split_module(value_list, ',') tmp AS position, value + ORDER BY category, val; + """ + + // ======================================== + // Section 10: UDTF with CASE WHEN + // ======================================== + + sql """ DROP TABLE IF EXISTS test_case_when_module; """ + sql """ + CREATE TABLE test_case_when_module ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_case_when_module VALUES + (1, '5,15,25'), + (2, '10,20,30'); + """ + + // Test 10.1: CASE WHEN on UDTF results + qt_case_when """ + SELECT + id, + tmp.value, + CASE + WHEN CAST(tmp.value AS INT) < 10 THEN 'small' + WHEN CAST(tmp.value AS INT) < 20 THEN 'medium' + ELSE 'large' + END AS size_category + FROM test_case_when_module + LATERAL VIEW udtf_split_module(data, ',') tmp AS position, value + ORDER BY id, CAST(tmp.value AS INT); + """ + + // ======================================== + // Section 11: - Multiple LATERAL VIEW Nesting + // ======================================== + + // Test 11.1: Two-level LATERAL VIEW nesting (sequential) + sql """ DROP TABLE IF EXISTS test_nested_2level_module; """ + sql """ + CREATE TABLE test_nested_2level_module ( + id INT, + categories STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_2level_module VALUES + (1, 'A:1,2|B:3'), + (2, 'C:4,5'); + """ + + qt_nested_2level """ + SELECT + id, + cat, + CAST(num AS INT) as num + FROM test_nested_2level_module + LATERAL VIEW udtf_split_module(categories, '|') t1 AS p1, cat_nums + LATERAL VIEW udtf_split_module(cat_nums, ':') t2 AS p2, cat + LATERAL VIEW udtf_split_module(cat, ',') t3 AS p3, num + WHERE p2 = 1 + ORDER BY id, cat, num; + """ + + // Test 11.2: Parallel LATERAL VIEWs (cartesian product) + sql """ DROP TABLE IF EXISTS test_parallel_lateral_module; """ + sql """ + CREATE TABLE test_parallel_lateral_module ( + id INT, + list1 STRING, + list2 STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_parallel_lateral_module VALUES + (1, 'A,B', 'X,Y'), + (2, 'C', 'Z'); + """ + + qt_parallel_lateral """ + SELECT + id, + item1, + item2 + FROM test_parallel_lateral_module + LATERAL VIEW udtf_split_module(list1, ',') t1 AS p1, item1 + LATERAL VIEW udtf_split_module(list2, ',') t2 AS p2, item2 + ORDER BY id, item1, item2; + """ + + // Test 11.3: Nested LATERAL VIEW with JOIN + sql """ DROP TABLE IF EXISTS test_nested_join_base_module; """ + sql """ + CREATE TABLE test_nested_join_base_module ( + user_id INT, + tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(user_id) + DISTRIBUTED BY HASH(user_id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_join_base_module VALUES + (1, 'sports:soccer,tennis|food:pizza'), + (2, 'music:rock'); + """ + + sql """ DROP TABLE IF EXISTS dim_tag_info_module; """ + sql """ + CREATE TABLE dim_tag_info_module ( + tag VARCHAR(50), + score INT + ) ENGINE=OLAP + DUPLICATE KEY(tag) + DISTRIBUTED BY HASH(tag) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO dim_tag_info_module VALUES + ('soccer', 10), + ('tennis', 8), + ('pizza', 5), + ('rock', 9); + """ + + qt_nested_join """ + SELECT + u.user_id, + tag_name, + d.score + FROM test_nested_join_base_module u + LATERAL VIEW udtf_split_module(u.tags, '|') t1 AS p1, cat_tags + LATERAL VIEW udtf_split_module(cat_tags, ':') t2 AS p2, part + LATERAL VIEW udtf_split_module(part, ',') t3 AS p3, tag_name + INNER JOIN dim_tag_info_module d ON d.tag = tag_name + WHERE p2 = 1 + ORDER BY u.user_id, d.score DESC; + """ + + // Test 11.4: Nested LATERAL VIEW with GROUP BY aggregation + sql """ DROP TABLE IF EXISTS test_nested_groupby_module; """ + sql """ + CREATE TABLE test_nested_groupby_module ( + store_id INT, + sales_data STRING + ) ENGINE=OLAP + DUPLICATE KEY(store_id) + DISTRIBUTED BY HASH(store_id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_groupby_module VALUES + (1, 'day1:100,200|day2:150'), + (2, 'day1:300|day2:250,100'); + """ + + qt_nested_groupby """ + SELECT + store_id, + COUNT(*) as sale_count, + SUM(CAST(amount AS INT)) as total_amount + FROM test_nested_groupby_module + LATERAL VIEW udtf_split_module(sales_data, '|') t1 AS p1, day_amounts + LATERAL VIEW udtf_split_module(day_amounts, ':') t2 AS p2, part + LATERAL VIEW udtf_split_module(part, ',') t3 AS p3, amount + WHERE p2 = 1 + GROUP BY store_id + ORDER BY store_id; + """ + + // Test 11.5: Three-level deep nesting + sql """ DROP TABLE IF EXISTS test_nested_3level_module; """ + sql """ + CREATE TABLE test_nested_3level_module ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_3level_module VALUES + (1, 'A,B,C|D,E|F'); + """ + + qt_nested_3level """ + SELECT + id, + grp_pos, + item + FROM test_nested_3level_module + LATERAL VIEW udtf_split_module(data, '|') t1 AS grp_pos, group_items + LATERAL VIEW udtf_split_module(group_items, ',') t2 AS item_pos, item + ORDER BY id, grp_pos, item_pos; + """ + + // Test 11.6: Nested with array expansion + sql """ DROP TABLE IF EXISTS test_nested_array_expansion_module; """ + sql """ + CREATE TABLE test_nested_array_expansion_module ( + id INT, + group_id INT, + numbers ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_array_expansion_module VALUES + (1, 1, [10, 20]), + (1, 2, [30]), + (2, 1, [40, 50]); + """ + + qt_nested_array_expansion """ + SELECT + id, + group_id, + element + FROM test_nested_array_expansion_module + LATERAL VIEW udtf_explode_array_module(numbers) t1 AS element, idx + ORDER BY id, group_id, element; + """ + + // Test 11.7: Nested with WHERE filtering at multiple levels + sql """ DROP TABLE IF EXISTS test_nested_multifilter_module; """ + sql """ + CREATE TABLE test_nested_multifilter_module ( + id INT, + data STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_multifilter_module VALUES + (1, 'A:10,20,30|B:40'), + (2, 'C:50,60'); + """ + + qt_nested_multifilter """ + SELECT + id, + cat_name, + CAST(num AS INT) as num + FROM ( + SELECT + id, + p1, + CASE WHEN p2 = 0 THEN part END AS cat_name, + CASE WHEN p2 = 1 THEN part END AS nums + FROM test_nested_multifilter_module + LATERAL VIEW udtf_split_module(data, '|') t1 AS p1, cat_nums + LATERAL VIEW udtf_split_module(cat_nums, ':') t2 AS p2, part + ) t + LATERAL VIEW udtf_split_module(nums, ',') t3 AS p3, num + WHERE nums IS NOT NULL AND CAST(num AS INT) >= 20 + ORDER BY id, p1, num; + """ + + // Test 11.8: Nested with DISTINCT across levels + sql """ DROP TABLE IF EXISTS test_nested_distinct_module; """ + sql """ + CREATE TABLE test_nested_distinct_module ( + id INT, + tags STRING + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES("replication_num" = "1"); + """ + + sql """ + INSERT INTO test_nested_distinct_module VALUES + (1, 'red,blue|red,green'), + (2, 'blue,yellow'); + """ + + qt_nested_distinct """ + SELECT DISTINCT color + FROM test_nested_distinct_module + LATERAL VIEW udtf_split_module(tags, '|') t1 AS p1, color_list + LATERAL VIEW udtf_split_module(color_list, ',') t2 AS p2, color + ORDER BY color; + """ + + } finally { + try_sql("DROP FUNCTION IF EXISTS udtf_split_module(STRING, STRING);") + try_sql("DROP FUNCTION IF EXISTS udtf_range_module(INT, INT);") + try_sql("DROP FUNCTION IF EXISTS udtf_explode_array_module(ARRAY);") + try_sql("DROP TABLE IF EXISTS test_where_before_module;") + try_sql("DROP TABLE IF EXISTS dim_numbers_module;") + try_sql("DROP TABLE IF EXISTS fact_ranges_module;") + try_sql("DROP TABLE IF EXISTS fact_ranges_extended_module;") + try_sql("DROP TABLE IF EXISTS test_self_join_module;") + try_sql("DROP TABLE IF EXISTS test_group_by_module;") + try_sql("DROP TABLE IF EXISTS test_agg_numbers_module;") + try_sql("DROP TABLE IF EXISTS test_order_limit_module;") + try_sql("DROP TABLE IF EXISTS test_subquery_module;") + try_sql("DROP TABLE IF EXISTS test_distinct_module;") + try_sql("DROP TABLE IF EXISTS test_union_a_module;") + try_sql("DROP TABLE IF EXISTS test_union_b_module;") + try_sql("DROP TABLE IF EXISTS test_array_ops_module;") + try_sql("DROP TABLE IF EXISTS test_window_module;") + try_sql("DROP TABLE IF EXISTS test_case_when_module;") + try_sql("DROP TABLE IF EXISTS test_nested_2level_module;") + try_sql("DROP TABLE IF EXISTS test_parallel_lateral_module;") + try_sql("DROP TABLE IF EXISTS test_nested_join_base_module;") + try_sql("DROP TABLE IF EXISTS dim_tag_info_module;") + try_sql("DROP TABLE IF EXISTS test_nested_groupby_module;") + try_sql("DROP TABLE IF EXISTS test_nested_3level_module;") + try_sql("DROP TABLE IF EXISTS test_nested_array_expansion_module;") + try_sql("DROP TABLE IF EXISTS test_nested_multifilter_module;") + try_sql("DROP TABLE IF EXISTS test_nested_distinct_module;") + } +} diff --git a/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf.zip b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf.zip new file mode 100644 index 00000000000000..f04942c97849c9 Binary files /dev/null and b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf.zip differ diff --git a/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/basic_udtf.py b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/basic_udtf.py new file mode 100644 index 00000000000000..c4ece6f1b71e38 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/basic_udtf.py @@ -0,0 +1,95 @@ +# 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. + +"""Basic UDTF implementations - copied from inline tests""" + +import json + + +def split_string_udtf(input_str): + '''Split comma-separated string into rows''' + if input_str: + parts = input_str.split(',') + for part in parts: + yield (part.strip(),) + + +def generate_series_udtf(start, end): + '''Generate integer series from start to end''' + if start is not None and end is not None: + for i in range(start, end + 1): + yield (i,) + + +def running_sum_udtf(value): + '''Return value with itself as cumulative sum (stateless)''' + # Note: Function-based UDTF cannot maintain state + # This is simplified to return (value, value) + if value is not None: + yield (value, value) + + +def explode_json_udtf(json_str): + '''Explode JSON ARRAY into rows''' + if json_str: + try: + data = json.loads(json_str) + if isinstance(data, list): + for item in data: + yield (str(item),) + except: + pass # Skip invalid JSON + + +def top_n_udtf(value, n): + '''Return single value with rank 1 (stateless)''' + # Without state, each row is independent + if value is not None and n is not None and n > 0: + yield (value, 1) + + +def duplicate_udtf(text, n): + '''Duplicate input text N times''' + if text and n: + for i in range(n): + yield (text, i + 1) + + +def filter_positive_udtf(value): + '''Only output positive values''' + if value is not None and value > 0: + yield (value,) + # If value <= 0, don't yield (skip this row) + + +def cartesian_udtf(list1, list2): + '''Generate cartesian product of two comma-separated lists''' + if list1 and list2: + items1 = [x.strip() for x in list1.split(',')] + items2 = [y.strip() for y in list2.split(',')] + + for x in items1: + for y in items2: + yield (x, y) + + +def filter_negative_udtf(value): + '''Only output negative values (filter all positive numbers)''' + if value is not None and value < 0: + yield (value,) + # For positive numbers, don't yield anything + diff --git a/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/data_types_udtf.py b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/data_types_udtf.py new file mode 100644 index 00000000000000..8a32317c44bb7f --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/data_types_udtf.py @@ -0,0 +1,191 @@ +# 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. + +"""Data type handling UDTF implementations - copied from inline tests""" + +import math +import json +from decimal import Decimal + + +def process_tinyint(v): + '''Process TINYINT: test small integer range''' + if v is not None: + yield (v, v * 2) + + +def process_smallint(v): + '''Process SMALLINT: test medium integer range''' + if v is not None: + yield (v, v * v) + + +def process_bigint(v): + '''Process BIGINT: test large integer range''' + if v is not None: + yield (v, v + 1) + + +def process_float(v): + '''Process FLOAT: test floating point numbers''' + if v is not None: + yield (v, v / 2.0) + + +def process_double(v): + '''Process DOUBLE: test high precision floating point''' + if v is not None and v >= 0: + yield (v, math.sqrt(v)) + + +def process_boolean(v): + '''Process BOOLEAN: test true/false values''' + if v is not None: + yield (v, not v, 'TRUE' if v else 'FALSE') + + +def process_string(v): + '''Process STRING: test text manipulation''' + if v is not None: + yield (v, len(v), v.upper(), v.lower()) + + +def process_date(v): + '''Process DATE: extract date components''' + if v is not None: + # v is a datetime.date object + yield (v, v.year, v.month, v.day) + + +def process_datetime(v): + '''Process DATETIME: extract time components''' + if v is not None: + # v is a datetime.datetime object + yield (v, v.hour, v.minute) + + +def process_array_int(arr): + '''Process ARRAY: explode array and process each element''' + if arr is not None: + for i, elem in enumerate(arr): + if elem is not None: + yield (i, elem, elem * 2) + + +def process_array_string(arr): + '''Process ARRAY: explode and get string lengths''' + if arr is not None: + for elem in arr: + if elem is not None: + yield (elem, len(elem)) + + +def process_struct(person): + '''Process STRUCT: access struct fields''' + if person is not None: + name = person['name'] if 'name' in person else None + age = person['age'] if 'age' in person else None + + if name is not None and age is not None: + category = 'child' if age < 18 else 'adult' + yield (name, age, category) + + +def process_multi_types(num, text): + '''Process multiple input types''' + if num is not None and text is not None: + yield (num, text, f"{text}_{num}") + + +def process_decimal(v): + '''Process DECIMAL: high precision arithmetic''' + if v is not None: + doubled = v * 2 + yield (v, doubled) + + +def process_map_string(map_str): + '''Process map-like string (key1:val1,key2:val2)''' + if map_str: + pairs = map_str.split(',') + for pair in pairs: + if ':' in pair: + k, val = pair.split(':', 1) + try: + yield (k.strip(), int(val.strip())) + except ValueError: + pass + + +def process_nested_array(nested_str): + '''Process nested array string ([[1,2],[3,4]])''' + if nested_str: + # Remove brackets and split by ],[ + nested_str = nested_str.strip('[]') + groups = nested_str.split('],[') + + for group_idx, group in enumerate(groups): + elements = group.strip('[]').split(',') + for elem in elements: + try: + yield (group_idx, int(elem.strip())) + except ValueError: + pass + + +def process_array_structs(data): + '''Process array of structs (name:age:score|name:age:score)''' + if data: + items = data.split('|') + for item in items: + parts = item.split(':') + if len(parts) == 3: + try: + yield (parts[0], int(parts[1]), int(parts[2])) + except ValueError: + pass + + +def process_struct_array(data): + '''Process struct with array (name:tag1,tag2,tag3)''' + if data and ':' in data: + name, tags = data.split(':', 1) + tag_list = tags.split(',') + yield (name, len(tag_list), ','.join(tag_list)) + + +def extract_json_fields(json_str): + '''Extract JSON fields''' + if json_str: + try: + data = json.loads(json_str) + if isinstance(data, dict): + for k, v in data.items(): + yield (k, str(v)) + except: + pass + + +def process_complex_struct(data): + '''Process complex struct (id:name:city:zip)''' + if data: + parts = data.split(':') + if len(parts) == 4: + try: + yield (int(parts[0]), parts[1], parts[2], parts[3]) + except ValueError: + pass diff --git a/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/edge_cases_udtf.py b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/edge_cases_udtf.py new file mode 100644 index 00000000000000..63515790f45c79 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/edge_cases_udtf.py @@ -0,0 +1,181 @@ +# 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. + +"""Edge cases UDTF implementations""" + +import math + + +def handle_null_int(value): + '''Handle NULL integer values''' + if value is None: + yield (None, True, -1) # NULL indicator + else: + yield (value, False, value * 2) + + +def handle_null_string(value): + '''Distinguish NULL from empty string''' + if value is None: + yield ('NULL', -1) + elif value == '': + yield ('EMPTY', 0) + else: + yield ('NORMAL', len(value)) + + +def handle_empty_array(arr): + '''Handle NULL vs empty array''' + if arr is None: + yield ('NULL', -1) + elif len(arr) == 0: + yield ('EMPTY', 0) + else: + yield ('NORMAL', len(arr)) + + +def handle_null_struct(person): + '''Handle NULL fields in STRUCT''' + if person is None: + yield (False, False, 'struct_is_null') + else: + name = person.get('name') + age = person.get('age') + has_name = name is not None + has_age = age is not None + + if has_name and has_age: + summary = f"{name}_{age}" + elif has_name: + summary = f"{name}_no_age" + elif has_age: + summary = f"no_name_{age}" + else: + summary = "all_fields_null" + + yield (has_name, has_age, summary) + + +def process_empty_table(value): + '''This should never be called for empty table''' + if value is not None: + yield (value * 2,) + + +def process_single_row(value): + '''Process single row input''' + if value is not None: + for i in range(3): + yield (value, value + i) + + +def process_long_string(text): + '''Process very long string''' + if text is not None: + length = len(text) + first_10 = text[:10] if length >= 10 else text + last_10 = text[-10:] if length >= 10 else text + yield (length, first_10, last_10) + + +def process_large_array(arr): + '''Process large array - compute statistics instead of exploding''' + if arr is not None and len(arr) > 0: + total = len(arr) + total_sum = sum(x for x in arr if x is not None) + first = arr[0] if len(arr) > 0 else None + last = arr[-1] if len(arr) > 0 else None + yield (total, total_sum, first, last) + + +def output_explosion(n): + '''Generate many outputs from single input (controlled explosion)''' + if n is not None and 0 < n <= 100: # Safety limit + for i in range(n): + yield (i,) + + +def process_special_numbers(value): + '''Categorize special numeric values''' + INT_MIN = -2147483648 + INT_MAX = 2147483647 + + if value is None: + yield (None, 'NULL', False) + elif value == 0: + yield (value, 'ZERO', False) + elif value == INT_MIN or value == INT_MAX: + category = 'POSITIVE' if value > 0 else 'NEGATIVE' + yield (value, category, True) # is_boundary = True + elif value > 0: + yield (value, 'POSITIVE', False) + else: + yield (value, 'NEGATIVE', False) + + +def process_special_doubles(value): + '''Classify special double values''' + if value is None: + yield (None, 'NULL') + elif math.isnan(value): + yield (value, 'NAN') + elif math.isinf(value): + if value > 0: + yield (value, 'POSITIVE_INF') + else: + yield (value, 'NEGATIVE_INF') + elif value == 0.0: + yield (value, 'ZERO') + elif abs(value) < 1e-10: + yield (value, 'VERY_SMALL') + elif abs(value) > 1e10: + yield (value, 'VERY_LARGE') + else: + yield (value, 'NORMAL') + + +def process_special_strings(text): + '''Process strings with special characters''' + if text is None: + yield (0, False, 'NULL') + elif text == '': + yield (0, False, 'EMPTY') + else: + length = len(text) + has_special = any(ord(c) > 127 for c in text) + + if has_special: + desc = 'HAS_UNICODE' + elif any(c in text for c in ['\n', '\t', '\r']): + desc = 'HAS_WHITESPACE' + elif any(c in text for c in ['!', '@', '#', '$', '%']): + desc = 'HAS_SYMBOLS' + else: + desc = 'NORMAL' + + yield (length, has_special, desc) + + +def process_boundary_dates(dt): + '''Process boundary date values''' + if dt is None: + yield (None, 0, False) + else: + year = dt.year + # Check if it's a boundary date + is_boundary = year in [1970, 9999] or (year == 1970 and dt.month == 1 and dt.day == 1) + yield (dt, year, is_boundary) diff --git a/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/exceptions_udtf.py b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/exceptions_udtf.py new file mode 100644 index 00000000000000..b663c7aa878dc7 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/exceptions_udtf.py @@ -0,0 +1,213 @@ +# 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. + +"""Exception handling UDTF implementations""" + +import math + + +def safe_divide(a, b): + '''Safe division with error handling''' + try: + if b == 0: + yield (a, b, None, 'division_by_zero') + else: + result = a / b + yield (a, b, result, 'success') + except Exception as e: + yield (a, b, None, f'error_{type(e).__name__}') + + +def check_overflow(value): + '''Check for potential overflow in operations''' + if value is None: + yield (None, None, 'null_input') + else: + # BIGINT range: -2^63 to 2^63-1 + MAX_BIGINT = 9223372036854775807 + MIN_BIGINT = -9223372036854775808 + + doubled = value * 2 + + # Check if doubled value is within safe range + if doubled > MAX_BIGINT or doubled < MIN_BIGINT: + yield (value, None, 'would_overflow') + else: + yield (value, doubled, 'safe') + + +def parse_number(text): + '''Parse string to number with error handling''' + if text is None: + yield (None, None, False) + else: + try: + num = float(text) + yield (text, num, True) + except ValueError: + yield (text, None, False) + + +def check_type(value): + '''Check and report value type''' + type_name = type(value).__name__ + + if value is None: + yield (None, 'NoneType', 0) + elif isinstance(value, str): + yield (value, type_name, len(value)) + else: + # Unexpected type - convert to string + yield (str(value), type_name, len(str(value))) + + +def safe_array_access(arr, position): + '''Safe array element access''' + if arr is None: + yield (0, position, None, 'null_array') + elif len(arr) == 0: + yield (0, position, None, 'empty_array') + elif position < 0 or position >= len(arr): + yield (len(arr), position, None, 'out_of_bounds') + else: + yield (len(arr), position, arr[position], 'success') + + +def compute_stats(arr): + '''Compute statistics with empty array handling''' + if arr is None: + yield (0, 0, 0.0, 'null_array') + elif len(arr) == 0: + yield (0, 0, 0.0, 'empty_array') + else: + count = len(arr) + total = sum(x for x in arr if x is not None) + avg = total / count if count > 0 else 0.0 + yield (count, total, avg, 'computed') + + +def access_struct_fields(person): + '''Safe STRUCT field access''' + if person is None: + yield (False, False, None, None) + else: + # Use .get() to safely access dictionary keys + name = person.get('name') + age = person.get('age') + + has_name = name is not None + has_age = age is not None + + yield (has_name, has_age, name, age) + + +def slice_string(text, start, end): + '''Safe string slicing''' + if text is None: + yield (None, start, end, None, 'null_string') + elif start is None or end is None: + yield (text, start, end, None, 'null_index') + else: + length = len(text) + + # Clamp indices to valid range + safe_start = max(0, min(start, length)) + safe_end = max(0, min(end, length)) + + if safe_start >= safe_end: + yield (text, start, end, '', 'empty_slice') + else: + result = text[safe_start:safe_end] + yield (text, start, end, result, 'success') + + +def check_text_encoding(text): + '''Check string encoding properties''' + if text is None: + yield (None, 0, 0, False) + else: + byte_len = len(text.encode('utf-8')) + char_len = len(text) + has_unicode = byte_len > char_len + + yield (text, byte_len, char_len, has_unicode) + + +def process_conditional(value): + '''Process value based on multiple conditions''' + if value is None: + yield (None, 'null', 0) + elif value < 0: + # For negative: take absolute value + yield (value, 'negative', abs(value)) + elif value == 0: + # Zero case: return 1 + yield (value, 'zero', 1) + elif value > 0 and value <= 100: + # Small positive: double it + yield (value, 'small_positive', value * 2) + else: + # Large positive: return as-is + yield (value, 'large_positive', value) + + +def conditional_yield(value): + '''Only yield for even positive numbers''' + if value is not None and value > 0 and value % 2 == 0: + yield (value,) + # For other cases, yield nothing (filter out) + + +def classify_number_range(value): + '''Classify number by magnitude''' + if value is None: + yield (None, 'null', True) + elif math.isnan(value): + yield (value, 'nan', False) + elif math.isinf(value): + yield (value, 'infinity', False) + elif value == 0.0: + yield (value, 'zero', True) + elif abs(value) < 1e-100: + yield (value, 'extremely_small', True) + elif abs(value) > 1e100: + yield (value, 'extremely_large', True) + elif abs(value) < 1.0: + yield (value, 'small', True) + else: + yield (value, 'normal', True) + + +def validate_date(dt): + '''Validate and classify dates''' + if dt is None: + yield (None, 0, False, 'null_date') + else: + year = dt.year + + # Check if leap year + is_leap = (year % 4 == 0 and year % 100 != 0) or (year % 400 == 0) + + # Classify date + if year < 1900: + status = 'very_old' + elif year > 2100: + status = 'far_future' + else: + status = 'normal' + + yield (dt, year, is_leap, status) diff --git a/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/io_patterns_udtf.py b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/io_patterns_udtf.py new file mode 100644 index 00000000000000..b275bfd618dd31 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/io_patterns_udtf.py @@ -0,0 +1,111 @@ +# 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. + +"""I/O pattern UDTF implementations - testing various cardinality patterns""" + + +def one_to_one(value): + '''Each input row produces exactly one output row''' + if value is not None: + yield (value, value * 2) + + +def one_to_many(n): + '''Each input row produces N output all_rows (1 to n)''' + if n is not None and n > 0: + for i in range(1, n + 1): + yield (i,) + + +def one_to_zero(value): + '''Only output even numbers, skip odd numbers (zero output)''' + if value is not None and value % 2 == 0: + yield (value,) + # Odd numbers: no yield, zero output all_rows + + +def one_to_variable(text): + ''' + - Empty string → 0 all_rows + - Single word → 1 row + - Multiple words → N all_rows + ''' + if text: + words = text.split() + for word in words: + yield (word,) + # Empty or None: no yield, zero output + + +def aggregate_pattern(value): + '''Categorize numbers into ranges''' + if value is not None: + if value < 10: + category = 'small' + elif value < 100: + category = 'medium' + else: + category = 'large' + yield (value, category) + + +def explosive(all_rows, all_cols): + '''Generate all_rows * all_cols output all_rows (cartesian product)''' + if all_rows is not None and all_cols is not None and all_rows > 0 and all_cols > 0: + for r in range(all_rows): + for c in range(all_cols): + yield (r, c) + + +def conditional(value): + ''' + - Positive: output (value, 'positive') + - Negative: output (abs(value), 'negative') + - Zero: output both (0, 'zero') and (0, 'neutral') + ''' + if value is not None: + if value > 0: + yield (value, 'positive') + elif value < 0: + yield (abs(value), 'negative') + else: + yield (0, 'zero') + yield (0, 'neutral') + + +def all_or_nothing(text, min_length): + ''' + If text length >= min_length: output each character with position + Otherwise: output nothing + ''' + if text and len(text) >= min_length: + for i, char in enumerate(text): + yield (char, i) + # If condition not met: no yield + + +def empty_input(value): + '''Simple identity function''' + if value is not None: + yield (value,) + + +def batch_process(value): + '''For each input, generate multiples (2x, 3x, 5x)''' + if value is not None and value > 0: + for factor in [2, 3, 5]: + yield (value, factor, value * factor) diff --git a/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/sql_integration_udtf.py b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/sql_integration_udtf.py new file mode 100644 index 00000000000000..63027562918e64 --- /dev/null +++ b/regression-test/suites/pythonudtf_p0/udtf_scripts/pyudtf_module/sql_integration_udtf.py @@ -0,0 +1,41 @@ +# 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. + +"""SQL integration UDTF implementations for complex query patterns""" + + +def split_with_position(text, delimiter): + '''Split string and return with position''' + if text and delimiter: + parts = text.split(delimiter) + for i, part in enumerate(parts): + yield (i, part.strip()) + + +def generate_range(start, end): + '''Generate integer range''' + if start is not None and end is not None: + for i in range(start, end + 1): + yield (i,) + + +def explode_with_index(arr): + '''Explode array with index''' + if arr: + for i, elem in enumerate(arr): + if elem is not None: + yield (elem, i)