diff --git a/GNUmakefile.in b/GNUmakefile.in index d836f7ba525..623074305ce 100644 --- a/GNUmakefile.in +++ b/GNUmakefile.in @@ -34,6 +34,9 @@ ifeq ($(with_openssl), yes) endif ifeq ($(enable_pax), yes) $(MAKE) -C contrib/pax_storage all +endif +ifeq ($(enable_ic_udp2),yes) + $(MAKE) -C contrib/udp2 all endif $(MAKE) -C gpMgmt all $(MAKE) -C gpcontrib all @@ -81,6 +84,9 @@ ifeq ($(enable_pax), yes) endif ifeq ($(with_openssl), yes) $(MAKE) -C contrib/sslinfo $@ +endif +ifeq ($(enable_ic_udp2),yes) + $(MAKE) -C contrib/udp2 $@ endif $(MAKE) -C gpMgmt $@ $(MAKE) -C gpcontrib $@ diff --git a/configure b/configure index 44d3fae95b6..26516737df6 100755 --- a/configure +++ b/configure @@ -761,6 +761,7 @@ PROTOBUF_LIBS PROTOBUF_CFLAGS enable_preload_ic_module enable_ic_proxy +enable_ic_udp2 enable_external_fts HAVE_CXX14 enable_gpcloud @@ -909,6 +910,7 @@ enable_shared_postgres_backend enable_link_postgres_with_shared enable_gpcloud enable_external_fts +enable_ic_udp2 enable_ic_proxy enable_preload_ic_module enable_pax @@ -1629,6 +1631,7 @@ Optional Features: libpostgres.so --enable-gpcloud enable gpcloud support --enable-external-fts enable external fts support + --enable-ic-udp2 enable interconnect udp2 implement --enable-ic-proxy enable interconnect proxy mode (requires libuv library) --disable-preload-ic-module @@ -9181,6 +9184,38 @@ $as_echo "#define USE_INTERNAL_FTS 1" >>confdefs.h CFLAGS="$CFLAGS -DUSE_INTERNAL_FTS=1" fi +# +# ic-udp2 +# + + +# Check whether --enable-ic-udp2 was given. +if test "${enable_ic_udp2+set}" = set; then : + enableval=$enable_ic_udp2; + case $enableval in + yes) + +$as_echo "#define ENABLE_IC_UDP2 1" >>confdefs.h + + ;; + no) + : + ;; + *) + as_fn_error $? "no argument expected for --enable-ic-udp2 option" "$LINENO" 5 + ;; + esac + +else + enable_ic_udp2=no + +fi + + +{ $as_echo "$as_me:${as_lineno-$LINENO}: result: checking whether to build with interconnect udp2 support ... $enable_ic_udp2" >&5 +$as_echo "checking whether to build with interconnect udp2 support ... $enable_ic_udp2" >&6; } + + # # ic-proxy # diff --git a/configure.ac b/configure.ac index 0d0529fc35f..1c8662f71b6 100644 --- a/configure.ac +++ b/configure.ac @@ -923,6 +923,16 @@ if test "$enable_external_fts" = no; then CFLAGS="$CFLAGS -DUSE_INTERNAL_FTS=1" fi +# +# ic-udp2 +# +PGAC_ARG_BOOL(enable, ic-udp2, no, + [enable interconnect udp2 implement], + [AC_DEFINE(ENABLE_IC_UDP2, 1, + [Define to 1 to build with interconnect udp2 support (--enable-ic-udp2)])]) +AC_MSG_RESULT([checking whether to build with interconnect udp2 support ... $enable_ic_udp2]) +AC_SUBST(enable_ic_udp2) + # # ic-proxy # diff --git a/contrib/Makefile b/contrib/Makefile index 2292adb88f2..b14600e3557 100644 --- a/contrib/Makefile +++ b/contrib/Makefile @@ -104,6 +104,12 @@ else ALWAYS_SUBDIRS += pax_storage endif +ifeq ($(enable_ic_udp2),yes) +SUBDIRS += udp2 +else +ALWAYS_SUBDIRS += udp2 +endif + # Missing: # start-scripts \ (does not have a makefile) diff --git a/contrib/udp2/CMakeLists.txt b/contrib/udp2/CMakeLists.txt new file mode 100644 index 00000000000..9a4835a1b53 --- /dev/null +++ b/contrib/udp2/CMakeLists.txt @@ -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. + +cmake_minimum_required(VERSION 3.11.0) +project(udp2) + +set(CMAKE_CXX_STANDARD 14) +set(CMAKE_CXX_STANDARD_REQUIRED True) + +# Get the top-level project directory +set(TOP_DIR ${PROJECT_SOURCE_DIR}/../..) +set(CBDB_INCLUDE_DIR ${TOP_DIR}/src/include) + +# CMAKE_INSTALL_PREFIX should be set by the calling Makefile +# If not set, we'll use a reasonable default but warn about it +if(NOT DEFINED CMAKE_INSTALL_PREFIX OR CMAKE_INSTALL_PREFIX STREQUAL "/usr/local") + message(WARNING "CMAKE_INSTALL_PREFIX not set by parent build system, using default") + set(CMAKE_INSTALL_PREFIX "/usr/local" CACHE PATH "Install prefix" FORCE) +endif() + +# Check for debug/release configuration from main project +include(CheckSymbolExists) +set(PG_CONFIG_HEADER_FILE "${CBDB_INCLUDE_DIR}/pg_config.h") +if(EXISTS "${PG_CONFIG_HEADER_FILE}") + CHECK_SYMBOL_EXISTS(USE_ASSERT_CHECKING "${PG_CONFIG_HEADER_FILE}" UDP2_USE_DEBUG) + if(UDP2_USE_DEBUG) + set(CMAKE_BUILD_TYPE "Debug" CACHE STRING "Choose the type of build." FORCE) + message(STATUS "Setting CMAKE_BUILD_TYPE to 'Debug' based on main project configuration") + else() + set(CMAKE_BUILD_TYPE "Release" CACHE STRING "Choose the type of build." FORCE) + message(STATUS "Setting CMAKE_BUILD_TYPE to 'Release' based on main project configuration") + endif() +else() + # Fallback to Release if pg_config.h is not found + if(NOT CMAKE_BUILD_TYPE) + set(CMAKE_BUILD_TYPE Release CACHE STRING "Choose the type of build." FORCE) + message(STATUS "Setting default CMAKE_BUILD_TYPE to 'Release'") + endif() +endif() + +# First, build and install ic_common as a subdirectory +add_subdirectory(ic_common) + +# Set up include directories +include_directories(${CMAKE_CURRENT_SOURCE_DIR}) +include_directories(${CMAKE_INSTALL_PREFIX}/include/postgresql/) +include_directories(${CMAKE_INSTALL_PREFIX}/include/postgresql/udp2/) +include_directories(${CBDB_INCLUDE_DIR}) +include_directories(${CBDB_INCLUDE_DIR}/server) + +# Set up library directories +link_directories(${CMAKE_INSTALL_PREFIX}/lib/postgresql/) + +# Source files for udp2 module +set(UDP2_SOURCES + ic_udp2.c + ic_modules.c +) + +# Create the udp2 shared library +add_library(udp2 SHARED ${UDP2_SOURCES}) + +# Set compiler flags consistent with main project +set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_GNU_SOURCE -Wall -Wpointer-arith -Wendif-labels -Wmissing-format-attribute -Wimplicit-fallthrough=3 -Wcast-function-type -Wformat-security -fno-strict-aliasing -fwrapv") + +if (${CMAKE_BUILD_TYPE} STREQUAL "Debug") + set(CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS} -g -ggdb") + message(STATUS "Building udp2 in Debug mode") +elseif (${CMAKE_BUILD_TYPE} STREQUAL "Release") + set(CMAKE_C_FLAGS_RELEASE "${CMAKE_C_FLAGS} -O2 -DNDEBUG") + message(STATUS "Building udp2 in Release mode") +endif() + +# Link against ic_common library +target_link_libraries(udp2 ic_common) + +# Make sure ic_common is built before udp2 +add_dependencies(udp2 ic_common) + +# Set output name and remove lib prefix +set_target_properties(udp2 PROPERTIES + OUTPUT_NAME "udp2" + PREFIX "" +) + +# Install the udp2 library +install(TARGETS udp2 + LIBRARY DESTINATION "${CMAKE_INSTALL_PREFIX}/lib/postgresql/" + ARCHIVE DESTINATION "${CMAKE_INSTALL_PREFIX}/lib/postgresql/" +) + +# Install udp2 headers +install(FILES + "${CMAKE_CURRENT_SOURCE_DIR}/ic_udp2.h" + "${CMAKE_CURRENT_SOURCE_DIR}/ic_modules.h" + DESTINATION "${CMAKE_INSTALL_PREFIX}/include/postgresql/" +) diff --git a/contrib/udp2/Makefile b/contrib/udp2/Makefile new file mode 100644 index 00000000000..e4891e9a528 --- /dev/null +++ b/contrib/udp2/Makefile @@ -0,0 +1,70 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# contrib/udp2/Makefile + +top_builddir = ../.. + +# Include the standard PostgreSQL build system to get variables like prefix, DESTDIR, etc. +ifdef USE_PGXS +PG_CONFIG = pg_config +PGXS := $(shell $(PG_CONFIG) --pgxs) +include $(PGXS) +else +subdir = contrib/udp2 +include $(top_builddir)/src/Makefile.global +include $(top_srcdir)/contrib/contrib-global.mk +endif + +# Use CMake for building this module +CMAKE_BUILD_DIR = build + +# Default target +all: $(CMAKE_BUILD_DIR)/udp2.so + +# Create build directory and configure with CMake +$(CMAKE_BUILD_DIR)/Makefile: + @echo "Configuring udp2 with CMake..." + @mkdir -p $(CMAKE_BUILD_DIR) + @cd $(CMAKE_BUILD_DIR) && cmake -DCMAKE_INSTALL_PREFIX=$(DESTDIR)$(prefix) .. + +# Build the project using CMake +$(CMAKE_BUILD_DIR)/udp2.so: $(CMAKE_BUILD_DIR)/Makefile + @echo "Building udp2 with CMake..." + @cd $(CMAKE_BUILD_DIR) && $(MAKE) + @cp $(CMAKE_BUILD_DIR)/udp2.so udp2.so + +# Install target +install: $(CMAKE_BUILD_DIR)/udp2.so + @echo "Installing udp2..." + @cd $(CMAKE_BUILD_DIR) && $(MAKE) install + +# Clean target +clean: + @echo "Cleaning udp2..." + @rm -rf $(CMAKE_BUILD_DIR) + @rm -f udp2.so + +# Ensure ic_common is built first +ic_common: + @echo "Building ic_common..." + @mkdir -p ic_common/build + @cd ic_common/build && cmake -DCMAKE_INSTALL_PREFIX=$(DESTDIR)$(prefix) .. && $(MAKE) && $(MAKE) install + +# Make sure ic_common is built before udp2 +$(CMAKE_BUILD_DIR)/udp2.so: ic_common + +.PHONY: all install clean ic_common diff --git a/contrib/udp2/README.md b/contrib/udp2/README.md new file mode 100644 index 00000000000..9cfe9f9c989 --- /dev/null +++ b/contrib/udp2/README.md @@ -0,0 +1,247 @@ + + +# UDP2 Interconnect Protocol Implementation + +## Project Background + +UDP2 is a next-generation interconnect protocol implementation based on the original UDP protocol, located in the `contrib/udp2` directory. In CloudBerry Database, the interconnect is responsible for data transmission and synchronization between nodes, serving as a core component for distributed query execution. + +Currently, the database supports three interconnect protocol implementations: +- **TCP** (`contrib/interconnect/tcp`) - Reliable transmission based on TCP protocol +- **UDP** (`contrib/interconnect/udp`) - High-performance transmission based on UDP protocol +- **Proxy** (`contrib/interconnect/proxy`) - Proxy-based transmission + +UDP2 is an architectural refactoring based on the original UDP protocol implementation, aimed at achieving complete separation between interconnect and the database kernel. + +## Project Goals + +The core objectives of the UDP2 protocol implementation are: + +1. **Architecture Decoupling**: Completely separate the interconnect protocol implementation from the database kernel, enabling independent development and evolution +2. **Independent Testing**: Enable end-to-end functional and performance testing of interconnect without depending on the database kernel +3. **Rapid Diagnosis**: Quickly identify whether issues are at the interconnect level or database kernel level +4. **Modular Design**: Provide clear interface boundaries for easier extension and maintenance + +## Current Project Implementation Architecture + +### Overall Architecture Design + +UDP2 adopts a layered architecture design, primarily divided into two layers: + +``` +┌─────────────────────────────────────────────────────────┐ +│ Database Kernel Layer │ +│ ┌────────────────────────────────────────────────────┐ │ +│ │ contrib/udp2/ │ │ +│ │ ┌─────────────────┐ ┌────────────────────────┐ │ │ +│ │ │ ic_modules.c │ │ ic_udp2.c │ │ │ +│ │ │ ic_modules.h │ │ ic_udp2.h │ │ │ +│ │ └─────────────────┘ └────────────────────────┘ │ │ +│ │ Adapter Layer (Database Adapter) │ │ +│ └────────────────────────────────────────────────────┘ │ +└─────────────────────────────────────────────────────────┘ + │ + │ C/C++ Interface + ▼ +┌──────────────────────────────────────────────────────────┐ +│ Independent IC Communication Library │ +│ ┌─────────────────────────────────────────────────────┐ │ +│ │ contrib/udp2/ic_common/ │ │ +│ │ ┌─────────────────┐ ┌─────────────────────────┐ │ │ +│ │ │ ic_types.h │ │ ic_utility.hpp │ │ │ +│ │ │ ic_except.hpp │ │ ic_faultinjection.h │ │ │ +│ │ └─────────────────┘ └─────────────────────────┘ │ │ +│ │ ┌────────────────────────────────────────────────┐ │ │ +│ │ │ contrib/udp2/ic_common/udp2/ │ │ │ +│ │ │ ┌─────────────────┐ ┌─────────────────────┐ │ │ │ +│ │ │ │ ic_udp2.h │ │ ic_udp2.hpp │ │ │ │ +│ │ │ │ ic_udp2.cpp │ │ic_udp2_internal.hpp │ │ │ │ +│ │ │ └─────────────────┘ └─────────────────────┘ │ │ │ +│ │ └────────────────────────────────────────────────┘ │ │ +│ │ Core Communication Library │ │ +│ └─────────────────────────────────────────────────────┘ │ +└──────────────────────────────────────────────────────────┘ +``` + +### Core Component Description + +#### 1. Adapter Layer (`contrib/udp2/`) +- **ic_modules.c/h**: Module registration and initialization, implementing the `MotionIPCLayer` interface +- **ic_udp2.c/h**: Adapter layer between database kernel and ic_common library +- Responsible for converting database kernel data structures to ic_common library standard interfaces + +#### 2. Core Communication Library (`contrib/udp2/ic_common/`) +- **ic_types.h**: Defines core data types and interfaces, decoupled from database kernel +- **ic_utility.hpp**: Common utility functions and logging system +- **ic_except.hpp**: Exception handling mechanism +- **ic_faultinjection.h**: Fault injection testing support + +#### 3. UDP2 Protocol Implementation (`contrib/udp2/ic_common/udp2/`) +- **ic_udp2.h**: C language interface definition +- **ic_udp2.hpp**: C++ interface definition +- **ic_udp2.cpp**: Core protocol implementation +- **ic_udp2_internal.hpp**: Internal implementation details + +### Build System + +UDP2 uses CMake build system with support for independent compilation: + +``` +contrib/udp2/ +├── CMakeLists.txt # Main build configuration +├── Makefile # PostgreSQL-compatible Makefile +└── ic_common/ + ├── CMakeLists.txt # ic_common library build configuration + └── build/ # Build output directory +``` + +Build process: +1. First build the `ic_common` dynamic library (`libic_common.so`) +2. Then build the `udp2` module (`udp2.so`), linking against the `ic_common` library + +## How to Switch Database to This Protocol Implementation + +### Enable UDP2 Support at Compile Time + +1. **Configure compilation options**: +```bash +./configure --enable-ic-udp2 [other options] +make && make install +``` + +2. **Verify compilation results**: +```bash +# Check if udp2.so is generated +ls -la $GPHOME/lib/postgresql/udp2.so + +# Check if ic_common library is installed +ls -la $GPHOME/lib/libic_common.so +``` + +### Runtime Configuration + +```bash +# Set cluster to use UDP2 by default +gpconfig -c gp_interconnect_type -v udp2 + +# Reload configuration +gpstop -air +``` + +```sql +-- Check current interconnect type +SHOW gp_interconnect_type; +``` + +## Technical Details + +### Interface Design + +UDP2 achieves decoupling between database kernel and communication library through standardized C interfaces: + +```c +// Core interface functions (ic_common/udp2/ic_udp2.h) +extern ICChunkTransportState* UDP2_SetupUDP(ICSliceTable *sliceTable, + SessionMotionLayerIPCParam *param); +extern void UDP2_TeardownUDP(ICChunkTransportState *transportStates, bool hasErrors); + +// Data send/receive interfaces +extern bool UDP2_SendData(ICChunkTransportState *transportStates, + int16 motNodeID, int16 targetRoute, + DataBlock *pblocks, int num, bool broadcast); +extern void UDP2_RecvAny(ICChunkTransportState *transportStates, + int16 motNodeID, int16 *srcRoute, + GetDataLenInPacket getLen, DataBlock *data); +``` + +### Data Structure Mapping + +UDP2 defines lightweight data structures to replace complex database kernel structures: + +```c +// Lightweight process information (replaces CdbProcess) +typedef struct ICCdbProcess { + bool valid; + char *listenerAddr; + int listenerPort; + int pid; + int contentid; + int dbid; +} ICCdbProcess; + +// Lightweight slice information (replaces ExecSlice) +typedef struct ICExecSlice { + int sliceIndex; + int parentIndex; + int numChildren; + int *children; + int numSegments; + int numPrimaryProcesses; + ICCdbProcess *primaryProcesses; +} ICExecSlice; +``` + +### Error Handling Mechanism + +UDP2 implements a unified error handling mechanism: + +```c +typedef enum ErrorLevel { + LEVEL_OK, + LEVEL_ERROR, + LEVEL_FATAL, +} ErrorLevel; + +// Error handling interfaces +extern void SetLastError(ErrorLevel level, const char *msg); +extern ICError* GetLastError(); +extern void ResetLastError(); +``` + +## Development and Debugging + +### Independent Compilation Testing + +```bash +# Enter ic_common directory +cd contrib/udp2/ic_common + +# Create build directory +mkdir build && cd build + +# Configure and compile +cmake -DCMAKE_BUILD_TYPE=Debug .. +make -j +``` + +### Debug Configuration + +Enable verbose logging in development environment: + +```sql +-- Enable interconnect debug logging +SET gp_log_interconnect = 'debug'; + +-- Set log level +SET log_min_messages = 'debug1'; + +-- Enable detailed error information +SET gp_interconnect_log_stats = on; +``` diff --git a/contrib/udp2/ic_common/CMakeLists.txt b/contrib/udp2/ic_common/CMakeLists.txt new file mode 100644 index 00000000000..2c26e002b84 --- /dev/null +++ b/contrib/udp2/ic_common/CMakeLists.txt @@ -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. + +cmake_minimum_required(VERSION 3.11.0) +project(ic_common) + +set(CMAKE_CXX_STANDARD 14) +set(CMAKE_CXX_STANDARD_REQUIRED True) + +# Get the top-level project directory +set(TOP_DIR ${PROJECT_SOURCE_DIR}/../../..) +set(CBDB_INCLUDE_DIR ${TOP_DIR}/src/include) + +# CMAKE_INSTALL_PREFIX should be set by the calling Makefile +# If not set, we'll use a reasonable default but warn about it +if(NOT DEFINED CMAKE_INSTALL_PREFIX OR CMAKE_INSTALL_PREFIX STREQUAL "/usr/local") + message(WARNING "CMAKE_INSTALL_PREFIX not set by parent build system, using default") + set(CMAKE_INSTALL_PREFIX "/usr/local" CACHE PATH "Install prefix" FORCE) +endif() + +# Check for debug/release configuration from main project +include(CheckSymbolExists) +set(PG_CONFIG_HEADER_FILE "${CBDB_INCLUDE_DIR}/pg_config.h") +if(EXISTS "${PG_CONFIG_HEADER_FILE}") + CHECK_SYMBOL_EXISTS(USE_ASSERT_CHECKING "${PG_CONFIG_HEADER_FILE}" IC_USE_DEBUG) + if(IC_USE_DEBUG) + set(CMAKE_BUILD_TYPE "Debug" CACHE STRING "Choose the type of build." FORCE) + message(STATUS "Setting CMAKE_BUILD_TYPE to 'Debug' based on main project configuration") + else() + set(CMAKE_BUILD_TYPE "Release" CACHE STRING "Choose the type of build." FORCE) + message(STATUS "Setting CMAKE_BUILD_TYPE to 'Release' based on main project configuration") + endif() +else() + # Fallback to Release if pg_config.h is not found + if(NOT CMAKE_BUILD_TYPE) + set(CMAKE_BUILD_TYPE Release CACHE STRING "Choose the type of build." FORCE) + message(STATUS "Setting default CMAKE_BUILD_TYPE to 'Release'") + endif() +endif() + +file(GLOB SOURCES "${CMAKE_CURRENT_SOURCE_DIR}/*.cpp" "${CMAKE_CURRENT_SOURCE_DIR}/udp2/*.cpp") + +include_directories(${CMAKE_CURRENT_SOURCE_DIR}) +include_directories(${CMAKE_CURRENT_SOURCE_DIR}/udp2) + +add_library(ic_common SHARED ${SOURCES}) + +set_target_properties(ic_common PROPERTIES OUTPUT_NAME "ic_common") + +# Set compiler flags consistent with main project +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -D_GNU_SOURCE -Wall -Wpointer-arith -Wendif-labels -Wmissing-format-attribute -Wimplicit-fallthrough=3 -Wcast-function-type -Wformat-security -fno-strict-aliasing -fwrapv") + +if (${CMAKE_BUILD_TYPE} STREQUAL "Debug") + set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS} -g -ggdb") + message(STATUS "Building ic_common in Debug mode") +elseif (${CMAKE_BUILD_TYPE} STREQUAL "Release") + set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS} -O2 -DNDEBUG") + message(STATUS "Building ic_common in Release mode") +endif() + +# Install headers to the main project's include directory +install(FILES + "${CMAKE_CURRENT_SOURCE_DIR}/ic_types.h" + "${CMAKE_CURRENT_SOURCE_DIR}/ic_except.hpp" + "${CMAKE_CURRENT_SOURCE_DIR}/ic_utility.hpp" + "${CMAKE_CURRENT_SOURCE_DIR}/ic_faultinjection.h" + DESTINATION "${CMAKE_INSTALL_PREFIX}/include/postgresql/" +) + +install(FILES + "${CMAKE_CURRENT_SOURCE_DIR}/udp2/ic_udp2.h" + "${CMAKE_CURRENT_SOURCE_DIR}/udp2/ic_udp2.hpp" + "${CMAKE_CURRENT_SOURCE_DIR}/udp2/ic_udp2_internal.hpp" + DESTINATION "${CMAKE_INSTALL_PREFIX}/include/postgresql/udp2/" +) + +# Install library to the main project's lib directory +install(TARGETS ic_common + LIBRARY DESTINATION "${CMAKE_INSTALL_PREFIX}/lib/" + ARCHIVE DESTINATION "${CMAKE_INSTALL_PREFIX}/lib/" +) diff --git a/contrib/udp2/ic_common/ic_except.hpp b/contrib/udp2/ic_common/ic_except.hpp new file mode 100644 index 00000000000..9761d499487 --- /dev/null +++ b/contrib/udp2/ic_common/ic_except.hpp @@ -0,0 +1,75 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_except.hpp + * + * IDENTIFICATION + * contrib/udp2/ic_common/ic_except.hpp + * + *------------------------------------------------------------------------- + */ +#ifndef IC_EXCEPT_HPP +#define IC_EXCEPT_HPP + +class ICException: public std::runtime_error { +public: + ICException(const std::string & arg, const char *file, int line): std::runtime_error(arg) { + std::stringstream ss; + ss << arg << " from " << file << ":" << line << std::endl; + detail = ss.str(); + } + ~ICException() throw() {} + + virtual const char *msg() const { + return detail.c_str(); + } + +protected: + std::string detail; +}; + +class ICFatalException: public ICException { +public: + ICFatalException(const std::string & arg, const char *file, int line): + ICException(arg, file, line) {} + ~ICFatalException() throw () {} +}; + +class ICInvalidIndex: public ICException { +public: + ICInvalidIndex(const std::string & arg, const char *file, int line): + ICException(arg, file, line) {} + ~ICInvalidIndex() throw () {} +}; + +class ICNetworkException: public ICException { +public: + ICNetworkException(const std::string & arg, const char *file, int line): + ICException(arg, file, line) {} + ~ICNetworkException() throw () {} +}; + +class ICReceiveThreadException: public ICException { +public: + ICReceiveThreadException(const std::string & arg, const char *file, int line): + ICException(arg, file, line) {} + ~ICReceiveThreadException() throw () {} +}; + +#endif // IC_EXCEPT_HPP \ No newline at end of file diff --git a/contrib/udp2/ic_common/ic_faultinjection.h b/contrib/udp2/ic_common/ic_faultinjection.h new file mode 100644 index 00000000000..b79cfed28dd --- /dev/null +++ b/contrib/udp2/ic_common/ic_faultinjection.h @@ -0,0 +1,620 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_faultinjection.h + * + * IDENTIFICATION + * contrib/udp2/ic_common/ic_faultinjection.h + * + *------------------------------------------------------------------------- + */ + +#ifndef IC_FAULTINJECTION_H +#define IC_FAULTINJECTION_H + +#ifdef HAVE_POLL_H +#include +#endif + +#ifdef HAVE_SYS_POLL_H +#include +#endif + +#ifdef USE_ASSERT_CHECKING + +static bool udp_testmode = false; + +static inline void +set_test_mode() +{ + if (session_param.gp_udpic_dropseg != UNDEF_SEGMENT + || session_param.gp_udpic_dropacks_percent != 0 + || session_param.gp_udpic_dropxmit_percent != 0 + || session_param.gp_udpic_fault_inject_percent != 0) + udp_testmode = true; + else + udp_testmode = false; +} + +/* + * testmode_inject_fault + * Return whether we inject a fault given a probability. + * + */ +static inline bool +testmode_inject_fault(int percent) +{ + if (udp_testmode && + (session_param.gp_udpic_dropseg == UNDEF_SEGMENT || session_param.gp_udpic_dropseg == global_param.segindex)) + { + if (random() % 100 < percent) + return true; + } + + return false; +} + +/* Track the malloc times */ +static pthread_mutex_t icudp_malloc_tracking_lock = PTHREAD_MUTEX_INITIALIZER; +static int64 icudp_malloc_times = 0; + +/* Fault type enumeration. */ +typedef enum { + /* These are used to inject packet content corruption. */ + FINC_PKT_HEADER_SHORTEN = 0, + FINC_PKT_PKT_SHORTEN = 1, + FINC_PKT_CRC_CORRUPT = 2, + FINC_PKT_HEADER_LEN_ZERO = 3, + FINC_PKT_HEADER_LEN_NEGATIVE = 4, + FINC_PKT_MISMATCH = 5, + + /* These are used to inject query cancel and process die. */ + FINC_INTR_QUERY_CANCEL = 12, + FINC_INTR_PROC_DIE = 13, + + /* These are used to inject OS API errors. */ + FINC_OS_EAGAIN = 16, + FINC_OS_EINTR = 17, + FINC_OS_EWOULDBLOCK = 18, + FINC_OS_NET_INTERFACE = 19, + FINC_OS_MEM_INTERFACE = 20, + FINC_OS_CREATE_THREAD = 21, + FINC_PKT_TOO_LONG = 22, + + /* These are used to inject network faults. */ + FINC_NET_RECV_ERROR = 23, + FINC_NET_PKT_DUP = 24, + FINC_NET_RECV_ZERO = 25, + + /* This is a fault which is used to introduce a specific null return of malloc in bg thread */ + FINC_RX_BUF_NULL = 29, + + /* The last guard item, don't put anything behind this one. */ + FINC_MAX_LIMITATION = 31, +} FAULT_INJECTION_TYPE; + +#define FINC_HAS_FAULT(type) (session_param.gp_udpic_fault_inject_bitmap & (1U << (type))) + +/* +* FAULT_TYPE_FUNCTION +* define the all test function name which uses fault injection test. +*/ +typedef enum { + FUNC_TESTMODE_RECVFROM, + FUNC_TESTMODE_MAX +} FAULT_TYPE_FUNCTION; + +/* +* fault_type_array_length +* record acutal fault type number for every test function which uses fault injection test. +*/ +static unsigned int fault_type_array_length[FUNC_TESTMODE_MAX][1] = { 0 }; + +/* +* fault_type_array +* define the fault type for for every test function which uses fault injection test. +*/ +static const unsigned int fault_type_array[][FINC_MAX_LIMITATION + 1] = { + { FINC_OS_EAGAIN, FINC_OS_EINTR, FINC_OS_EWOULDBLOCK, FINC_NET_RECV_ZERO, FINC_OS_NET_INTERFACE, FINC_NET_RECV_ERROR, FINC_MAX_LIMITATION } +}; + +/* + * testmode_check_interrupts + * ML_CHECK_FOR_INTERRUPTS in test mode with interrupts injected. + */ +static inline void +testmode_check_interrupts(const char *caller_name, bool teardownActive) +{ +} + +/* We needs a private copy to corrupt the packet. */ +#define FAULT_INJECT_BACKUP_PKT() \ +do { \ + pktModified = true; \ + memcpy(&hdrbk, (void *) buffer, sizeof(icpkthdr)); \ +} while (0) + +/* + * get_fault_type_array_length + * Retrive test function fault type size recorded in array fault_type_array_length. + */ +static inline unsigned int +get_fault_type_array_length(unsigned int func_name) +{ + if (!fault_type_array_length[func_name][0]) { + for (unsigned int i = 0; i < FINC_MAX_LIMITATION + 1; i++) { + if (fault_type_array[func_name][i] == FINC_MAX_LIMITATION) { + fault_type_array_length[func_name][0] = i; + break; + } + } + } + return fault_type_array_length[func_name][0]; +} + +/* + * random_with_range + * Generate the random value predefined in fault_type_array. + */ +static inline int +random_with_array(unsigned int func_name) +{ + Assert((func_name < FUNC_TESTMODE_MAX) && (fault_type_array!= NULL)); + unsigned int length = get_fault_type_array_length(func_name); + Assert((length != 0) && (length < FINC_MAX_LIMITATION)); + return fault_type_array[func_name][(rand() % length)]; +} + +/* + * testmode_sendto + * Many kinds of fault packets are injected in this function. + */ +static ssize_t +testmode_sendto(const char *caller_name, int socket, const void *buffer, + size_t length, int flags, const struct sockaddr *dest_addr, + socklen_t dest_len) +{ + int n; + int testmode_length = length; + size_t icpkthdr_size = sizeof(icpkthdr); + bool is_pkt = false; + bool pktModified = false; + int fault_type; + icpkthdr hdrbk; + icpkthdr *msg = (icpkthdr *) buffer; + + if (!testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + goto no_fault_inject; + + /* + * Generate a fault type. + */ + fault_type = random() % FINC_MAX_LIMITATION; + + /* Make sure we are modifying a packet. */ + if (length >= icpkthdr_size) + is_pkt = true; + + /* Inject a fault */ + switch (fault_type) + { + case FINC_PKT_HEADER_SHORTEN: + if (!FINC_HAS_FAULT(fault_type) || !is_pkt) + break; + testmode_length = icpkthdr_size - 1; + LOG(INFO, "inject fault to sendto: FINC_PKT_HEADER_SHORTEN"); + break; + + case FINC_PKT_PKT_SHORTEN: + if (!FINC_HAS_FAULT(fault_type) || !is_pkt) + break; + if (length > icpkthdr_size) + testmode_length--; + LOG(INFO, "inject fault to sendto: FINC_PKT_PKT_SHORTEN"); + break; + + case FINC_PKT_CRC_CORRUPT: + if (!FINC_HAS_FAULT(fault_type) || !is_pkt) + break; + FAULT_INJECT_BACKUP_PKT(); + if (!session_param.gp_interconnect_full_crc) + break; + msg->crc++; + LOG(INFO, "inject fault to sendto: FINC_PKT_CRC_CORRUPT"); + break; + + case FINC_PKT_HEADER_LEN_ZERO: + if (!FINC_HAS_FAULT(fault_type) || !is_pkt) + break; + FAULT_INJECT_BACKUP_PKT(); + msg->len = 0; + LOG(INFO, "inject fault to sendto: FINC_PKT_HEADER_LEN_ZERO"); + break; + + case FINC_PKT_HEADER_LEN_NEGATIVE: + if (!FINC_HAS_FAULT(fault_type) || !is_pkt) + break; + FAULT_INJECT_BACKUP_PKT(); + msg->len = -1; + LOG(INFO, "inject fault to sendto: FINC_PKT_HEADER_LEN_NEGATIVE"); + break; + + case FINC_PKT_MISMATCH: + if (!FINC_HAS_FAULT(fault_type) || !is_pkt) + break; + FAULT_INJECT_BACKUP_PKT(); + msg->srcPid = -1; /* There is no such pid. */ + msg->icId = 0; + msg->seq = 1; + LOG(INFO, "inject fault to sendto: FINC_PKT_MISMATCH"); + break; + + case FINC_OS_EAGAIN: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to sendto: FINC_OS_EAGAIN"); + errno = EAGAIN; + return -1; + + case FINC_OS_EINTR: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to sendto: FINC_OS_EINTR"); + errno = EINTR; + return -1; + + case FINC_NET_PKT_DUP: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to sendto: FINC_NET_PKT_DUP"); + if ((n = sendto(socket, buffer, testmode_length, flags, dest_addr, dest_len)) != (int)length) + return n; + break; + + case FINC_OS_NET_INTERFACE: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to sendto: FINC_OS_NET_INTERFACE"); + errno = EFAULT; + return -1; + + case FINC_PKT_TOO_LONG: + if (!FINC_HAS_FAULT(fault_type) || !is_pkt) + break; + LOG(INFO, "inject fault to sendto: FINC_PKT_TOO_LONG"); + errno = EMSGSIZE; + return -1; + + default: + break; + } + +no_fault_inject: + n = sendto(socket, buffer, testmode_length, flags, dest_addr, dest_len); + + if (pktModified) + memcpy((void *) buffer, &hdrbk, sizeof(icpkthdr)); + return n; +} + +/* + * testmode_recvfrom + * recvfrom function with faults injected. + */ +static ssize_t +testmode_recvfrom(const char *caller_name, int socket, void *buffer, + size_t length, int flags, struct sockaddr *address, + socklen_t *address_len) +{ + int fault_type; + + if (!testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + goto no_fault_inject; + + fault_type = random_with_array(FUNC_TESTMODE_RECVFROM); + + switch (fault_type) + { + case FINC_OS_EAGAIN: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to recvfrom: FINC_OS_EAGAIN"); + errno = EAGAIN; + return -1; + + case FINC_OS_EINTR: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to recvfrom: FINC_OS_EINTR"); + errno = EINTR; + return -1; + + case FINC_OS_EWOULDBLOCK: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to recvfrom: FINC_OS_EWOULDBLOCK"); + errno = EWOULDBLOCK; + return -1; + + case FINC_NET_RECV_ZERO: + if (!FINC_HAS_FAULT(fault_type)) + break; + + memset(buffer, 0, length); + LOG(INFO, "inject fault to recvfrom: FINC_NET_RECV_ZERO"); + return 0; + + case FINC_OS_NET_INTERFACE: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to recvfrom: FINC_OS_NET_INTERFACE"); + errno = EFAULT; + return -1; + + case FINC_NET_RECV_ERROR: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to recvfrom: FINC_NET_RECV_ERROR"); + errno = EFAULT; + return -1; + + default: + break; + } + +no_fault_inject: + return recvfrom(socket, buffer, length, flags, address, address_len); +} + +/* + * testmode_poll + * poll function with faults injected. + */ +static int +testmode_poll(const char *caller_name, struct pollfd fds[], nfds_t nfds, + int timeout) +{ + int fault_type; + + if (!testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + goto no_fault_inject; + + fault_type = random() % FINC_MAX_LIMITATION; + + switch (fault_type) + { + case FINC_OS_EINTR: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to poll: FINC_OS_EINTR"); + errno = EINTR; + return -1; + + case FINC_OS_NET_INTERFACE: + if (!FINC_HAS_FAULT(fault_type)) + break; + LOG(INFO, "inject fault to poll: FINC_OS_NET_INTERFACE"); + errno = EFAULT; + return -1; + + default: + break; + } + +no_fault_inject: + return poll(fds, nfds, timeout); +} + +/* + * testmode_socket + * socket function with faults injected. + * + */ +static int +testmode_socket(const char *caller_name, int domain, int type, int protocol) +{ + if (FINC_HAS_FAULT(FINC_OS_NET_INTERFACE) && + testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + { + LOG(INFO, "inject fault to socket: FINC_OS_NET_INTERFACE"); + errno = ENOMEM; + return -1; + } + + return socket(domain, type, protocol); +} + +/* + * testmode_bind + * bind function with fault injected. + * + */ +static int +testmode_bind(const char *caller_name, int socket, + const struct sockaddr *address, socklen_t address_len) +{ + if (FINC_HAS_FAULT(FINC_OS_NET_INTERFACE) && + testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + { + LOG(INFO, "inject fault to bind: FINC_OS_NET_INTERFACE"); + errno = EFAULT; + return -1; + } + + return bind(socket, address, address_len); +} + +/* + * testmode_getsockname + * getsockname function with faults injected. + * + */ +static int +testmode_getsockname(const char *caller_name, int socket, + struct sockaddr *address, + socklen_t *address_len) +{ + if (FINC_HAS_FAULT(FINC_OS_NET_INTERFACE) && + testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + { + LOG(INFO, "inject fault to getsockname: FINC_OS_NET_INTERFACE"); + errno = EFAULT; + return -1; + } + + return getsockname(socket, address, address_len); +} + +/* + * testmode_setsockopt + * setsockopt with faults injected. + */ +static int +testmode_setsockopt(const char *caller_name, int socket, int level, + int option_name, const void *option_value, + socklen_t option_len) +{ + if (FINC_HAS_FAULT(FINC_OS_NET_INTERFACE) && + testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + { + LOG(INFO, "inject fault to setsockopt: FINC_OS_NET_INTERFACE"); + errno = ENOMEM; + return -1; + } + + return setsockopt(socket, level, option_name, option_value, option_len); +} + +/* + * testmode_pg_getaddrinfo_all + * pg_getaddrinfo_all with faults injected. + */ +static int +testmode_pg_getaddrinfo_all(const char *caller_name, const char *hostname, + const char *servname, const struct addrinfo *hints, + struct addrinfo **res) +{ + if (FINC_HAS_FAULT(FINC_OS_NET_INTERFACE) && + testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + { + LOG(INFO, "inject fault to pg_getaddrinfo_all: FINC_OS_NET_INTERFACE"); + return -1; + } + + //return pg_getaddrinfo_all(hostname, servname, hints, res); + return getaddrinfo(hostname, servname, hints, res); +} + +/* + * testmode_free + * free function with free time tracking added. + */ +static void +testmode_free(const char *caller_name, void *ptr) +{ + if (ptr == NULL) + return; + + pthread_mutex_lock(&icudp_malloc_tracking_lock); + icudp_malloc_times--; + pthread_mutex_unlock(&icudp_malloc_tracking_lock); + free(ptr); +} + +/* + * testmode_pthread_create + * pthread_create with faults injected. + */ +static int +testmode_pthread_create(const char *caller_name, pthread_t *thread, + const pthread_attr_t *attr, void *(*start_routine) (void *), void *arg) +{ + if (FINC_HAS_FAULT(FINC_OS_CREATE_THREAD) && + testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + { + LOG(INFO, "inject fault to pthread_create: FINC_OS_CREATE_THREAD"); + return ENOMEM; + } + + return pthread_create(thread, attr, start_routine, arg); +} + +#ifdef ML_CHECK_FOR_INTERRUPTS +#undef ML_CHECK_FOR_INTERRUPTS +#endif +#undef sendto +#undef recvfrom +#undef poll +#undef socket +#undef bind +#undef getsockname +#undef getsockopt +#undef setsockopt +#undef pg_getaddrinfo_all +#undef malloc +#undef free +#undef palloc0 +#undef pthread_create + +#define ML_CHECK_FOR_INTERRUPTS(teardownActive) \ + testmode_check_interrupts(__FUNCTION__, teardownActive) + +#define sendto(socket, buffer, length, flags, dest_addr, dest_len) \ + testmode_sendto(__FUNCTION__, socket, buffer, length, flags, dest_addr, dest_len) + +#define recvfrom(socket, buffer, length, flags, address, address_len) \ + testmode_recvfrom(__FUNCTION__, socket, buffer, length, flags, address, address_len) + +#define poll(fds, nfds, timeout) \ + testmode_poll(__FUNCTION__, fds, nfds, timeout) + +#define socket(domain, type, protocol) \ + testmode_socket(__FUNCTION__, domain, type, protocol) + +#define bind(socket, address, address_len) \ + testmode_bind(__FUNCTION__, socket, address, address_len) + +#define getsockname(socket, address, address_len) \ + testmode_getsockname(__FUNCTION__, socket, address, address_len) + +#define getsockopt(socket, level, option_name, option_value, option_len) \ + testmode_getsockopt(__FUNCTION__, socket, level, option_name, option_value, option_len) + +#define setsockopt(socket, level, option_name, option_value, option_len) \ + testmode_setsockopt(__FUNCTION__, socket, level, option_name, option_value, option_len) + +#define getaddrinfo(hostname, servname, hints, res) \ + testmode_pg_getaddrinfo_all(__FUNCTION__, hostname, servname, hints, res) + +#define malloc(size) \ + testmode_malloc(__FUNCTION__, size) + +#define free(ptr) \ + testmode_free(__FUNCTION__, ptr) + +/* +#define ic_malloc0(size) \ + testmode_malloc0(__FUNCTION__, size) +*/ + +#define pthread_create(thread, attr, start_routine, arg) \ + testmode_pthread_create(__FUNCTION__, thread, attr, start_routine, arg) +#endif + +#endif \ No newline at end of file diff --git a/contrib/udp2/ic_common/ic_types.h b/contrib/udp2/ic_common/ic_types.h new file mode 100644 index 00000000000..e31d02e493d --- /dev/null +++ b/contrib/udp2/ic_common/ic_types.h @@ -0,0 +1,212 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_types.h + * + * IDENTIFICATION + * contrib/udp2/ic_common/ic_types.h + * + *------------------------------------------------------------------------- + */ +#ifndef IC_TYPES_H +#define IC_TYPES_H + +#ifdef __cplusplus +extern "C" { +#endif + +/* + * data type + */ +typedef signed char int8; /* == 8 bits */ +typedef signed short int16; /* == 16 bits */ +typedef signed int int32; /* == 32 bits */ +typedef unsigned char uint8; /* == 8 bits */ +typedef unsigned short uint16; /* == 16 bits */ +typedef unsigned int uint32; /* == 32 bits */ +typedef long int int64; +typedef unsigned long int uint64; + +typedef uint64 DistributedTransactionId; + +/* the lite version of CdbProcess */ +typedef struct ICCdbProcess +{ + bool valid; + char *listenerAddr; /* Interconnect listener IPv4 address, a C-string */ + int listenerPort; /* Interconnect listener port */ + int pid; /* Backend PID of the process. */ + int contentid; + int dbid; + +} ICCdbProcess; + +/* the lite version of ExecSlice */ +typedef struct ICExecSlice +{ + int sliceIndex; + int parentIndex; + + int numChildren; + int *children; + + int numSegments; + + int numPrimaryProcesses; + ICCdbProcess *primaryProcesses; + +} ICExecSlice; + +/* the lite version of SliceTable */ +typedef struct ICSliceTable +{ + int localSlice; /* Index of the slice to execute. */ + int numSlices; + ICExecSlice *slices; /* Array of ICExecSlice, indexed by SliceIndex */ + uint32 ic_instance_id; + +} ICSliceTable; + +typedef struct ICChunkTransportState +{ + /* keeps track of if we've "activated" connections via SetupInterconnect(). */ + bool activated; + bool teardownActive; + + /* slice table stuff. */ + struct ICSliceTable *sliceTable; + int sliceId; + int icInstanceId; /* the same as sliceTable->ic_instance_id */ + + /* whether we've logged when network timeout happens */ + bool networkTimeoutIsLogged; + + /* save client's state */ + void *clientState; + +} ICChunkTransportState; + +struct MemoryBlock +{ + unsigned char *pos; + int len; +}; + +typedef struct MemoryBlock BufferBlock; +typedef struct MemoryBlock DataBlock; + +typedef int (*GetDataLenInPacket)(unsigned char *msg, int msg_size); + +/* + * GlobalMotionLayerIPCParam and SessionMotionLayerIPCParam + */ +typedef bool (*CheckPostmasterIsAliveCallback)(void); +typedef void (*CheckInterruptsCallback)(int teardownActive); +typedef void (*SimpleFaultInjectorCallback)(const char *faultname); +typedef void *(*CreateOpaqueDataWithConn)(void); +typedef void (*DestroyOpaqueDataInConn)(void **); +typedef void (*CheckCancelOnQDCallback)(ICChunkTransportState *pTransportStates); + +typedef struct GlobalMotionLayerIPCParam +{ + char *interconnect_address; /* postmaster.h */ + int Gp_role; /* Gp_role */ + int ic_htab_size; /* cdbgang.h */ + int segment_number; /* getgpsegmentCount() */ + int MyProcPid; /* miscadmin.h */ + int dbid; /* GpIdentity */ + int segindex; /* GpIdentity */ + bool MyProcPort; /* miscadmin.h */ + int myprocport_sock; /* MyProcPort->sock */ + int Gp_max_packet_size; /* default: 8192 */ + int Gp_udp_bufsize_k; /* default: 0 */ + int Gp_interconnect_address_type; /* default: INTERCONNECT_ADDRESS_TYPE_UNICAST_IC */ + + CheckPostmasterIsAliveCallback checkPostmasterIsAliveCallback; + CheckInterruptsCallback checkInterruptsCallback; + SimpleFaultInjectorCallback simpleFaultInjectorCallback; + + CreateOpaqueDataWithConn createOpaqueDataCallback; + DestroyOpaqueDataInConn destroyOpaqueDataCallback; + + CheckCancelOnQDCallback checkCancelOnQDCallback; + +} GlobalMotionLayerIPCParam; + +typedef struct SessionMotionLayerIPCParam +{ + int Gp_interconnect_queue_depth; /* default: 4 */ + int Gp_interconnect_snd_queue_depth; /* default: 2 */ + int Gp_interconnect_cursor_ic_table_size; /* default: 128 */ + int Gp_interconnect_timer_period; /* default: 5 */ + int Gp_interconnect_timer_checking_period; /* default: 20 */ + int Gp_interconnect_default_rtt; /* default: 20 */ + int Gp_interconnect_min_rto; /* default: 20 */ + int Gp_interconnect_transmit_timeout; /* default: 3600 */ + int Gp_interconnect_min_retries_before_timeout; /* default: 100 */ + int Gp_interconnect_debug_retry_interval; /* default: 10 */ + bool gp_interconnect_full_crc; /* default: false */ + bool gp_interconnect_aggressive_retry; /* default: true */ + bool gp_interconnect_cache_future_packets; /* default: true */ + bool gp_interconnect_log_stats; /* default: false */ + int interconnect_setup_timeout; /* default: 7200 */ + int gp_log_interconnect; /* default: terse */ + int gp_session_id; /* global unique id for session. */ + int Gp_interconnect_fc_method; /* default: INTERCONNECT_FC_METHOD_LOSS */ + int gp_command_count; + uint32 gp_interconnect_id; + int log_min_messages; /* default: IC_WARNING */ + DistributedTransactionId distTransId; /* default: 0 */ + + int gp_udpic_dropseg; /* default: -2 */ + int gp_udpic_dropacks_percent; /* default: 0 */ + int gp_udpic_dropxmit_percent; /* default: 0 */ + int gp_udpic_fault_inject_percent; /* default: 0 */ + int gp_udpic_fault_inject_bitmap; /* default: 0 */ + int gp_udpic_network_disable_ipv6; /* default: 0 */ + +} SessionMotionLayerIPCParam; + +/* + * handle error + */ +#define MSGLEN 1024 + +typedef enum ErrorLevel +{ + LEVEL_OK, + LEVEL_ERROR, + LEVEL_FATAL, +} ErrorLevel; + +typedef struct ICError +{ + ErrorLevel level; + char msg[MSGLEN]; +} ICError; + +extern void ResetLastError(); +extern ICError* GetLastError(); +extern void SetLastError(ErrorLevel level, const char *msg); + +#ifdef __cplusplus +} +#endif + +#endif // IC_TYPES_H \ No newline at end of file diff --git a/contrib/udp2/ic_common/ic_utility.cpp b/contrib/udp2/ic_common/ic_utility.cpp new file mode 100644 index 00000000000..7a5a0747238 --- /dev/null +++ b/contrib/udp2/ic_common/ic_utility.cpp @@ -0,0 +1,1371 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_utility.cpp + * + * IDENTIFICATION + * contrib/udp2/ic_common/ic_utility.cpp + * + *------------------------------------------------------------------------- + */ + +#include + +#include "ic_types.h" +#include "ic_utility.hpp" + +/* + * global_param and session_param; + */ +GlobalMotionLayerIPCParam global_param = + {NULL, /* interconnect_address */ + -1, /* Gp_role */ + 0, /* ic_htab_size */ + 0, /* segment_number */ + -1, /* MyProcPid */ + -1, /* dbid */ + -1, /* segindex */ + false,/* MyProcPort */ + -1, /* myprocport_sock */ + 8192, /* Gp_max_packet_size */ + 0, /* Gp_udp_bufsize_k */ + INTERCONNECT_ADDRESS_TYPE_UNICAST_IC, /* Gp_interconnect_address_type */ + NULL, /* CheckPostmasterIsAliveCallback */ + NULL, /* checkInterruptsCallback */ + NULL, /* simpleFaultInjectorCallback */ + NULL, /* createOpaqueDataCallback */ + NULL, /* destroyOpaqueDataCallback */ + NULL, /* checkCancelOnQDCallback */ + }; + +SessionMotionLayerIPCParam session_param = + {4, /* Gp_interconnect_queue_depth */ + 2, /* Gp_interconnect_snd_queue_depth */ + 128, /* Gp_interconnect_cursor_ic_table_size */ + 5, /* Gp_interconnect_timer_period */ + 20, /* Gp_interconnect_timer_checking_period */ + 20, /* Gp_interconnect_default_rtt */ + 20, /* Gp_interconnect_min_rto */ + 3600, /* Gp_interconnect_transmit_timeout */ + 100, /* Gp_interconnect_min_retries_before_timeout */ + 10, /* Gp_interconnect_debug_retry_interval */ + false,/* gp_interconnect_full_crc */ + true, /* gp_interconnect_aggressive_retry */ + true, /* gp_interconnect_cache_future_packets */ + false,/* gp_interconnect_log_stats */ + 7200, /* interconnect_setup_timeout */ + GPVARS_VERBOSITY_TERSE_IC, /* gp_log_interconnect */ + -1, /* gp_session_id */ + INTERCONNECT_FC_METHOD_LOSS_IC, /* Gp_interconnect_fc_method */ + -1, /* gp_command_count */ + 0, /* gp_interconnect_id */ + IC_WARNING, /* log_min_messages */ + 0, /* distTransId */ + + UNDEF_SEGMENT, /* gp_udpic_dropseg */ + 0, /* gp_udpic_dropacks_percent */ + 0, /* gp_udpic_dropxmit_percent */ + 0, /* gp_udpic_fault_inject_percent */ + 0, /* gp_udpic_fault_inject_bitmap */ + 0, /* gp_udpic_network_disable_ipv6 */ + }; + + +/* + * logger stuff + */ +Logger RootLogger; +static std::mutex LoggerMutex; +static thread_local std::once_flag Once; +static thread_local char ProcessId[64]; + +const char * SeverityName[] = { "FATAL", "ERROR", "WARNING", "INFO", "DEBUG1", + "DEBUG2", "DEBUG3", "DEBUG4", "DEBUG5" + }; + +static void InitProcessId() { + std::stringstream ss; + ss.imbue(std::locale::classic()); + ss << "p" << getpid() << ", th" << pthread_self(); + snprintf(ProcessId, sizeof(ProcessId), "%s", ss.str().c_str()); +} + +Logger::Logger() : + fd(STDERR_FILENO), severity(DEFAULT_LOG_LEVEL) { +} + +Logger::~Logger() { +} + +void Logger::setOutputFd(int f) { + fd = f; +} + +void Logger::setLogSeverity(LogSeverity l) { + severity.store(l, std::memory_order_relaxed); +} + +void Logger::printf(LogSeverity s, const char * fmt, ...) { + va_list ap; + + if (s > severity || fd < 0) { + return; + } + + try { + call_once(Once, InitProcessId); + std::vector buffer; + struct tm tm_time; + struct timeval tval; + memset(&tval, 0, sizeof(tval)); + gettimeofday(&tval, NULL); + localtime_r(&tval.tv_sec, &tm_time); + //determine buffer size + va_start(ap, fmt); + int size = vsnprintf(nullptr, 0, fmt, ap); + va_end(ap); + //100 is enough for prefix + buffer.resize(size + 100); + size = snprintf(buffer.data(), buffer.size(), "%04d-%02d-%02d %02d:%02d:%02d.%06ld, %s, %s ", tm_time.tm_year + 1900, + 1 + tm_time.tm_mon, tm_time.tm_mday, tm_time.tm_hour, + tm_time.tm_min, tm_time.tm_sec, static_cast(tval.tv_usec), ProcessId, SeverityName[s]); + va_start(ap, fmt); + size += vsnprintf(buffer.data() + size, buffer.size() - size, fmt, ap); + va_end(ap); + { + std::lock_guard lock(LoggerMutex); + dprintf(fd, "%s\n", buffer.data()); + } + return; + } catch (const std::exception & e) { + dprintf(fd, "%s:%d %s %s", __FILE__, __LINE__, + "FATAL: get an unexpected exception:", e.what()); + throw; + } +} + +/* + * crc32 + */ +static uint32 comp_crc32c_sb8(uint32 crc, const void *data, size_t len); + +/* + * Use slicing-by-8 algorithm. + * + * On big-endian systems, the intermediate value is kept in reverse byte + * order, to avoid byte-swapping during the calculation. FIN_CRC32C reverses + * the bytes to the final order. + */ +#define INIT_CRC32C(crc) ((crc) = 0xFFFFFFFF) +#define COMP_CRC32C(crc, data, len) \ + ((crc) = comp_crc32c_sb8((crc), (data), (len))) +#ifdef WORDS_BIGENDIAN +#define FIN_CRC32C(crc) ((crc) = ic_bswap32(crc) ^ 0xFFFFFFFF) +#else +#define FIN_CRC32C(crc) ((crc) ^= 0xFFFFFFFF) +#endif + +uint32 +ComputeCRC(const void *data, size_t len) +{ + uint32 local_crc; + + INIT_CRC32C(local_crc); + COMP_CRC32C(local_crc, data, len); + FIN_CRC32C(local_crc); + + return local_crc; +} + +/* + * Lookup tables for the slicing-by-8 algorithm, for the so-called Castagnoli + * polynomial (the same that is used e.g. in iSCSI), 0x1EDC6F41. Using + * Williams' terms, this is the "normal", not "reflected" version. However, on + * big-endian systems the values in the tables are stored in byte-reversed + * order (IOW, the tables are stored in little-endian order even on big-endian + * systems). + */ +static const uint32 crc32c_table[8][256] = { +#ifndef WORDS_BIGENDIAN + { + 0x00000000, 0xF26B8303, 0xE13B70F7, 0x1350F3F4, + 0xC79A971F, 0x35F1141C, 0x26A1E7E8, 0xD4CA64EB, + 0x8AD958CF, 0x78B2DBCC, 0x6BE22838, 0x9989AB3B, + 0x4D43CFD0, 0xBF284CD3, 0xAC78BF27, 0x5E133C24, + 0x105EC76F, 0xE235446C, 0xF165B798, 0x030E349B, + 0xD7C45070, 0x25AFD373, 0x36FF2087, 0xC494A384, + 0x9A879FA0, 0x68EC1CA3, 0x7BBCEF57, 0x89D76C54, + 0x5D1D08BF, 0xAF768BBC, 0xBC267848, 0x4E4DFB4B, + 0x20BD8EDE, 0xD2D60DDD, 0xC186FE29, 0x33ED7D2A, + 0xE72719C1, 0x154C9AC2, 0x061C6936, 0xF477EA35, + 0xAA64D611, 0x580F5512, 0x4B5FA6E6, 0xB93425E5, + 0x6DFE410E, 0x9F95C20D, 0x8CC531F9, 0x7EAEB2FA, + 0x30E349B1, 0xC288CAB2, 0xD1D83946, 0x23B3BA45, + 0xF779DEAE, 0x05125DAD, 0x1642AE59, 0xE4292D5A, + 0xBA3A117E, 0x4851927D, 0x5B016189, 0xA96AE28A, + 0x7DA08661, 0x8FCB0562, 0x9C9BF696, 0x6EF07595, + 0x417B1DBC, 0xB3109EBF, 0xA0406D4B, 0x522BEE48, + 0x86E18AA3, 0x748A09A0, 0x67DAFA54, 0x95B17957, + 0xCBA24573, 0x39C9C670, 0x2A993584, 0xD8F2B687, + 0x0C38D26C, 0xFE53516F, 0xED03A29B, 0x1F682198, + 0x5125DAD3, 0xA34E59D0, 0xB01EAA24, 0x42752927, + 0x96BF4DCC, 0x64D4CECF, 0x77843D3B, 0x85EFBE38, + 0xDBFC821C, 0x2997011F, 0x3AC7F2EB, 0xC8AC71E8, + 0x1C661503, 0xEE0D9600, 0xFD5D65F4, 0x0F36E6F7, + 0x61C69362, 0x93AD1061, 0x80FDE395, 0x72966096, + 0xA65C047D, 0x5437877E, 0x4767748A, 0xB50CF789, + 0xEB1FCBAD, 0x197448AE, 0x0A24BB5A, 0xF84F3859, + 0x2C855CB2, 0xDEEEDFB1, 0xCDBE2C45, 0x3FD5AF46, + 0x7198540D, 0x83F3D70E, 0x90A324FA, 0x62C8A7F9, + 0xB602C312, 0x44694011, 0x5739B3E5, 0xA55230E6, + 0xFB410CC2, 0x092A8FC1, 0x1A7A7C35, 0xE811FF36, + 0x3CDB9BDD, 0xCEB018DE, 0xDDE0EB2A, 0x2F8B6829, + 0x82F63B78, 0x709DB87B, 0x63CD4B8F, 0x91A6C88C, + 0x456CAC67, 0xB7072F64, 0xA457DC90, 0x563C5F93, + 0x082F63B7, 0xFA44E0B4, 0xE9141340, 0x1B7F9043, + 0xCFB5F4A8, 0x3DDE77AB, 0x2E8E845F, 0xDCE5075C, + 0x92A8FC17, 0x60C37F14, 0x73938CE0, 0x81F80FE3, + 0x55326B08, 0xA759E80B, 0xB4091BFF, 0x466298FC, + 0x1871A4D8, 0xEA1A27DB, 0xF94AD42F, 0x0B21572C, + 0xDFEB33C7, 0x2D80B0C4, 0x3ED04330, 0xCCBBC033, + 0xA24BB5A6, 0x502036A5, 0x4370C551, 0xB11B4652, + 0x65D122B9, 0x97BAA1BA, 0x84EA524E, 0x7681D14D, + 0x2892ED69, 0xDAF96E6A, 0xC9A99D9E, 0x3BC21E9D, + 0xEF087A76, 0x1D63F975, 0x0E330A81, 0xFC588982, + 0xB21572C9, 0x407EF1CA, 0x532E023E, 0xA145813D, + 0x758FE5D6, 0x87E466D5, 0x94B49521, 0x66DF1622, + 0x38CC2A06, 0xCAA7A905, 0xD9F75AF1, 0x2B9CD9F2, + 0xFF56BD19, 0x0D3D3E1A, 0x1E6DCDEE, 0xEC064EED, + 0xC38D26C4, 0x31E6A5C7, 0x22B65633, 0xD0DDD530, + 0x0417B1DB, 0xF67C32D8, 0xE52CC12C, 0x1747422F, + 0x49547E0B, 0xBB3FFD08, 0xA86F0EFC, 0x5A048DFF, + 0x8ECEE914, 0x7CA56A17, 0x6FF599E3, 0x9D9E1AE0, + 0xD3D3E1AB, 0x21B862A8, 0x32E8915C, 0xC083125F, + 0x144976B4, 0xE622F5B7, 0xF5720643, 0x07198540, + 0x590AB964, 0xAB613A67, 0xB831C993, 0x4A5A4A90, + 0x9E902E7B, 0x6CFBAD78, 0x7FAB5E8C, 0x8DC0DD8F, + 0xE330A81A, 0x115B2B19, 0x020BD8ED, 0xF0605BEE, + 0x24AA3F05, 0xD6C1BC06, 0xC5914FF2, 0x37FACCF1, + 0x69E9F0D5, 0x9B8273D6, 0x88D28022, 0x7AB90321, + 0xAE7367CA, 0x5C18E4C9, 0x4F48173D, 0xBD23943E, + 0xF36E6F75, 0x0105EC76, 0x12551F82, 0xE03E9C81, + 0x34F4F86A, 0xC69F7B69, 0xD5CF889D, 0x27A40B9E, + 0x79B737BA, 0x8BDCB4B9, 0x988C474D, 0x6AE7C44E, + 0xBE2DA0A5, 0x4C4623A6, 0x5F16D052, 0xAD7D5351 + }, + { + 0x00000000, 0x13A29877, 0x274530EE, 0x34E7A899, + 0x4E8A61DC, 0x5D28F9AB, 0x69CF5132, 0x7A6DC945, + 0x9D14C3B8, 0x8EB65BCF, 0xBA51F356, 0xA9F36B21, + 0xD39EA264, 0xC03C3A13, 0xF4DB928A, 0xE7790AFD, + 0x3FC5F181, 0x2C6769F6, 0x1880C16F, 0x0B225918, + 0x714F905D, 0x62ED082A, 0x560AA0B3, 0x45A838C4, + 0xA2D13239, 0xB173AA4E, 0x859402D7, 0x96369AA0, + 0xEC5B53E5, 0xFFF9CB92, 0xCB1E630B, 0xD8BCFB7C, + 0x7F8BE302, 0x6C297B75, 0x58CED3EC, 0x4B6C4B9B, + 0x310182DE, 0x22A31AA9, 0x1644B230, 0x05E62A47, + 0xE29F20BA, 0xF13DB8CD, 0xC5DA1054, 0xD6788823, + 0xAC154166, 0xBFB7D911, 0x8B507188, 0x98F2E9FF, + 0x404E1283, 0x53EC8AF4, 0x670B226D, 0x74A9BA1A, + 0x0EC4735F, 0x1D66EB28, 0x298143B1, 0x3A23DBC6, + 0xDD5AD13B, 0xCEF8494C, 0xFA1FE1D5, 0xE9BD79A2, + 0x93D0B0E7, 0x80722890, 0xB4958009, 0xA737187E, + 0xFF17C604, 0xECB55E73, 0xD852F6EA, 0xCBF06E9D, + 0xB19DA7D8, 0xA23F3FAF, 0x96D89736, 0x857A0F41, + 0x620305BC, 0x71A19DCB, 0x45463552, 0x56E4AD25, + 0x2C896460, 0x3F2BFC17, 0x0BCC548E, 0x186ECCF9, + 0xC0D23785, 0xD370AFF2, 0xE797076B, 0xF4359F1C, + 0x8E585659, 0x9DFACE2E, 0xA91D66B7, 0xBABFFEC0, + 0x5DC6F43D, 0x4E646C4A, 0x7A83C4D3, 0x69215CA4, + 0x134C95E1, 0x00EE0D96, 0x3409A50F, 0x27AB3D78, + 0x809C2506, 0x933EBD71, 0xA7D915E8, 0xB47B8D9F, + 0xCE1644DA, 0xDDB4DCAD, 0xE9537434, 0xFAF1EC43, + 0x1D88E6BE, 0x0E2A7EC9, 0x3ACDD650, 0x296F4E27, + 0x53028762, 0x40A01F15, 0x7447B78C, 0x67E52FFB, + 0xBF59D487, 0xACFB4CF0, 0x981CE469, 0x8BBE7C1E, + 0xF1D3B55B, 0xE2712D2C, 0xD69685B5, 0xC5341DC2, + 0x224D173F, 0x31EF8F48, 0x050827D1, 0x16AABFA6, + 0x6CC776E3, 0x7F65EE94, 0x4B82460D, 0x5820DE7A, + 0xFBC3FAF9, 0xE861628E, 0xDC86CA17, 0xCF245260, + 0xB5499B25, 0xA6EB0352, 0x920CABCB, 0x81AE33BC, + 0x66D73941, 0x7575A136, 0x419209AF, 0x523091D8, + 0x285D589D, 0x3BFFC0EA, 0x0F186873, 0x1CBAF004, + 0xC4060B78, 0xD7A4930F, 0xE3433B96, 0xF0E1A3E1, + 0x8A8C6AA4, 0x992EF2D3, 0xADC95A4A, 0xBE6BC23D, + 0x5912C8C0, 0x4AB050B7, 0x7E57F82E, 0x6DF56059, + 0x1798A91C, 0x043A316B, 0x30DD99F2, 0x237F0185, + 0x844819FB, 0x97EA818C, 0xA30D2915, 0xB0AFB162, + 0xCAC27827, 0xD960E050, 0xED8748C9, 0xFE25D0BE, + 0x195CDA43, 0x0AFE4234, 0x3E19EAAD, 0x2DBB72DA, + 0x57D6BB9F, 0x447423E8, 0x70938B71, 0x63311306, + 0xBB8DE87A, 0xA82F700D, 0x9CC8D894, 0x8F6A40E3, + 0xF50789A6, 0xE6A511D1, 0xD242B948, 0xC1E0213F, + 0x26992BC2, 0x353BB3B5, 0x01DC1B2C, 0x127E835B, + 0x68134A1E, 0x7BB1D269, 0x4F567AF0, 0x5CF4E287, + 0x04D43CFD, 0x1776A48A, 0x23910C13, 0x30339464, + 0x4A5E5D21, 0x59FCC556, 0x6D1B6DCF, 0x7EB9F5B8, + 0x99C0FF45, 0x8A626732, 0xBE85CFAB, 0xAD2757DC, + 0xD74A9E99, 0xC4E806EE, 0xF00FAE77, 0xE3AD3600, + 0x3B11CD7C, 0x28B3550B, 0x1C54FD92, 0x0FF665E5, + 0x759BACA0, 0x663934D7, 0x52DE9C4E, 0x417C0439, + 0xA6050EC4, 0xB5A796B3, 0x81403E2A, 0x92E2A65D, + 0xE88F6F18, 0xFB2DF76F, 0xCFCA5FF6, 0xDC68C781, + 0x7B5FDFFF, 0x68FD4788, 0x5C1AEF11, 0x4FB87766, + 0x35D5BE23, 0x26772654, 0x12908ECD, 0x013216BA, + 0xE64B1C47, 0xF5E98430, 0xC10E2CA9, 0xD2ACB4DE, + 0xA8C17D9B, 0xBB63E5EC, 0x8F844D75, 0x9C26D502, + 0x449A2E7E, 0x5738B609, 0x63DF1E90, 0x707D86E7, + 0x0A104FA2, 0x19B2D7D5, 0x2D557F4C, 0x3EF7E73B, + 0xD98EEDC6, 0xCA2C75B1, 0xFECBDD28, 0xED69455F, + 0x97048C1A, 0x84A6146D, 0xB041BCF4, 0xA3E32483 + }, + { + 0x00000000, 0xA541927E, 0x4F6F520D, 0xEA2EC073, + 0x9EDEA41A, 0x3B9F3664, 0xD1B1F617, 0x74F06469, + 0x38513EC5, 0x9D10ACBB, 0x773E6CC8, 0xD27FFEB6, + 0xA68F9ADF, 0x03CE08A1, 0xE9E0C8D2, 0x4CA15AAC, + 0x70A27D8A, 0xD5E3EFF4, 0x3FCD2F87, 0x9A8CBDF9, + 0xEE7CD990, 0x4B3D4BEE, 0xA1138B9D, 0x045219E3, + 0x48F3434F, 0xEDB2D131, 0x079C1142, 0xA2DD833C, + 0xD62DE755, 0x736C752B, 0x9942B558, 0x3C032726, + 0xE144FB14, 0x4405696A, 0xAE2BA919, 0x0B6A3B67, + 0x7F9A5F0E, 0xDADBCD70, 0x30F50D03, 0x95B49F7D, + 0xD915C5D1, 0x7C5457AF, 0x967A97DC, 0x333B05A2, + 0x47CB61CB, 0xE28AF3B5, 0x08A433C6, 0xADE5A1B8, + 0x91E6869E, 0x34A714E0, 0xDE89D493, 0x7BC846ED, + 0x0F382284, 0xAA79B0FA, 0x40577089, 0xE516E2F7, + 0xA9B7B85B, 0x0CF62A25, 0xE6D8EA56, 0x43997828, + 0x37691C41, 0x92288E3F, 0x78064E4C, 0xDD47DC32, + 0xC76580D9, 0x622412A7, 0x880AD2D4, 0x2D4B40AA, + 0x59BB24C3, 0xFCFAB6BD, 0x16D476CE, 0xB395E4B0, + 0xFF34BE1C, 0x5A752C62, 0xB05BEC11, 0x151A7E6F, + 0x61EA1A06, 0xC4AB8878, 0x2E85480B, 0x8BC4DA75, + 0xB7C7FD53, 0x12866F2D, 0xF8A8AF5E, 0x5DE93D20, + 0x29195949, 0x8C58CB37, 0x66760B44, 0xC337993A, + 0x8F96C396, 0x2AD751E8, 0xC0F9919B, 0x65B803E5, + 0x1148678C, 0xB409F5F2, 0x5E273581, 0xFB66A7FF, + 0x26217BCD, 0x8360E9B3, 0x694E29C0, 0xCC0FBBBE, + 0xB8FFDFD7, 0x1DBE4DA9, 0xF7908DDA, 0x52D11FA4, + 0x1E704508, 0xBB31D776, 0x511F1705, 0xF45E857B, + 0x80AEE112, 0x25EF736C, 0xCFC1B31F, 0x6A802161, + 0x56830647, 0xF3C29439, 0x19EC544A, 0xBCADC634, + 0xC85DA25D, 0x6D1C3023, 0x8732F050, 0x2273622E, + 0x6ED23882, 0xCB93AAFC, 0x21BD6A8F, 0x84FCF8F1, + 0xF00C9C98, 0x554D0EE6, 0xBF63CE95, 0x1A225CEB, + 0x8B277743, 0x2E66E53D, 0xC448254E, 0x6109B730, + 0x15F9D359, 0xB0B84127, 0x5A968154, 0xFFD7132A, + 0xB3764986, 0x1637DBF8, 0xFC191B8B, 0x595889F5, + 0x2DA8ED9C, 0x88E97FE2, 0x62C7BF91, 0xC7862DEF, + 0xFB850AC9, 0x5EC498B7, 0xB4EA58C4, 0x11ABCABA, + 0x655BAED3, 0xC01A3CAD, 0x2A34FCDE, 0x8F756EA0, + 0xC3D4340C, 0x6695A672, 0x8CBB6601, 0x29FAF47F, + 0x5D0A9016, 0xF84B0268, 0x1265C21B, 0xB7245065, + 0x6A638C57, 0xCF221E29, 0x250CDE5A, 0x804D4C24, + 0xF4BD284D, 0x51FCBA33, 0xBBD27A40, 0x1E93E83E, + 0x5232B292, 0xF77320EC, 0x1D5DE09F, 0xB81C72E1, + 0xCCEC1688, 0x69AD84F6, 0x83834485, 0x26C2D6FB, + 0x1AC1F1DD, 0xBF8063A3, 0x55AEA3D0, 0xF0EF31AE, + 0x841F55C7, 0x215EC7B9, 0xCB7007CA, 0x6E3195B4, + 0x2290CF18, 0x87D15D66, 0x6DFF9D15, 0xC8BE0F6B, + 0xBC4E6B02, 0x190FF97C, 0xF321390F, 0x5660AB71, + 0x4C42F79A, 0xE90365E4, 0x032DA597, 0xA66C37E9, + 0xD29C5380, 0x77DDC1FE, 0x9DF3018D, 0x38B293F3, + 0x7413C95F, 0xD1525B21, 0x3B7C9B52, 0x9E3D092C, + 0xEACD6D45, 0x4F8CFF3B, 0xA5A23F48, 0x00E3AD36, + 0x3CE08A10, 0x99A1186E, 0x738FD81D, 0xD6CE4A63, + 0xA23E2E0A, 0x077FBC74, 0xED517C07, 0x4810EE79, + 0x04B1B4D5, 0xA1F026AB, 0x4BDEE6D8, 0xEE9F74A6, + 0x9A6F10CF, 0x3F2E82B1, 0xD50042C2, 0x7041D0BC, + 0xAD060C8E, 0x08479EF0, 0xE2695E83, 0x4728CCFD, + 0x33D8A894, 0x96993AEA, 0x7CB7FA99, 0xD9F668E7, + 0x9557324B, 0x3016A035, 0xDA386046, 0x7F79F238, + 0x0B899651, 0xAEC8042F, 0x44E6C45C, 0xE1A75622, + 0xDDA47104, 0x78E5E37A, 0x92CB2309, 0x378AB177, + 0x437AD51E, 0xE63B4760, 0x0C158713, 0xA954156D, + 0xE5F54FC1, 0x40B4DDBF, 0xAA9A1DCC, 0x0FDB8FB2, + 0x7B2BEBDB, 0xDE6A79A5, 0x3444B9D6, 0x91052BA8 + }, + { + 0x00000000, 0xDD45AAB8, 0xBF672381, 0x62228939, + 0x7B2231F3, 0xA6679B4B, 0xC4451272, 0x1900B8CA, + 0xF64463E6, 0x2B01C95E, 0x49234067, 0x9466EADF, + 0x8D665215, 0x5023F8AD, 0x32017194, 0xEF44DB2C, + 0xE964B13D, 0x34211B85, 0x560392BC, 0x8B463804, + 0x924680CE, 0x4F032A76, 0x2D21A34F, 0xF06409F7, + 0x1F20D2DB, 0xC2657863, 0xA047F15A, 0x7D025BE2, + 0x6402E328, 0xB9474990, 0xDB65C0A9, 0x06206A11, + 0xD725148B, 0x0A60BE33, 0x6842370A, 0xB5079DB2, + 0xAC072578, 0x71428FC0, 0x136006F9, 0xCE25AC41, + 0x2161776D, 0xFC24DDD5, 0x9E0654EC, 0x4343FE54, + 0x5A43469E, 0x8706EC26, 0xE524651F, 0x3861CFA7, + 0x3E41A5B6, 0xE3040F0E, 0x81268637, 0x5C632C8F, + 0x45639445, 0x98263EFD, 0xFA04B7C4, 0x27411D7C, + 0xC805C650, 0x15406CE8, 0x7762E5D1, 0xAA274F69, + 0xB327F7A3, 0x6E625D1B, 0x0C40D422, 0xD1057E9A, + 0xABA65FE7, 0x76E3F55F, 0x14C17C66, 0xC984D6DE, + 0xD0846E14, 0x0DC1C4AC, 0x6FE34D95, 0xB2A6E72D, + 0x5DE23C01, 0x80A796B9, 0xE2851F80, 0x3FC0B538, + 0x26C00DF2, 0xFB85A74A, 0x99A72E73, 0x44E284CB, + 0x42C2EEDA, 0x9F874462, 0xFDA5CD5B, 0x20E067E3, + 0x39E0DF29, 0xE4A57591, 0x8687FCA8, 0x5BC25610, + 0xB4868D3C, 0x69C32784, 0x0BE1AEBD, 0xD6A40405, + 0xCFA4BCCF, 0x12E11677, 0x70C39F4E, 0xAD8635F6, + 0x7C834B6C, 0xA1C6E1D4, 0xC3E468ED, 0x1EA1C255, + 0x07A17A9F, 0xDAE4D027, 0xB8C6591E, 0x6583F3A6, + 0x8AC7288A, 0x57828232, 0x35A00B0B, 0xE8E5A1B3, + 0xF1E51979, 0x2CA0B3C1, 0x4E823AF8, 0x93C79040, + 0x95E7FA51, 0x48A250E9, 0x2A80D9D0, 0xF7C57368, + 0xEEC5CBA2, 0x3380611A, 0x51A2E823, 0x8CE7429B, + 0x63A399B7, 0xBEE6330F, 0xDCC4BA36, 0x0181108E, + 0x1881A844, 0xC5C402FC, 0xA7E68BC5, 0x7AA3217D, + 0x52A0C93F, 0x8FE56387, 0xEDC7EABE, 0x30824006, + 0x2982F8CC, 0xF4C75274, 0x96E5DB4D, 0x4BA071F5, + 0xA4E4AAD9, 0x79A10061, 0x1B838958, 0xC6C623E0, + 0xDFC69B2A, 0x02833192, 0x60A1B8AB, 0xBDE41213, + 0xBBC47802, 0x6681D2BA, 0x04A35B83, 0xD9E6F13B, + 0xC0E649F1, 0x1DA3E349, 0x7F816A70, 0xA2C4C0C8, + 0x4D801BE4, 0x90C5B15C, 0xF2E73865, 0x2FA292DD, + 0x36A22A17, 0xEBE780AF, 0x89C50996, 0x5480A32E, + 0x8585DDB4, 0x58C0770C, 0x3AE2FE35, 0xE7A7548D, + 0xFEA7EC47, 0x23E246FF, 0x41C0CFC6, 0x9C85657E, + 0x73C1BE52, 0xAE8414EA, 0xCCA69DD3, 0x11E3376B, + 0x08E38FA1, 0xD5A62519, 0xB784AC20, 0x6AC10698, + 0x6CE16C89, 0xB1A4C631, 0xD3864F08, 0x0EC3E5B0, + 0x17C35D7A, 0xCA86F7C2, 0xA8A47EFB, 0x75E1D443, + 0x9AA50F6F, 0x47E0A5D7, 0x25C22CEE, 0xF8878656, + 0xE1873E9C, 0x3CC29424, 0x5EE01D1D, 0x83A5B7A5, + 0xF90696D8, 0x24433C60, 0x4661B559, 0x9B241FE1, + 0x8224A72B, 0x5F610D93, 0x3D4384AA, 0xE0062E12, + 0x0F42F53E, 0xD2075F86, 0xB025D6BF, 0x6D607C07, + 0x7460C4CD, 0xA9256E75, 0xCB07E74C, 0x16424DF4, + 0x106227E5, 0xCD278D5D, 0xAF050464, 0x7240AEDC, + 0x6B401616, 0xB605BCAE, 0xD4273597, 0x09629F2F, + 0xE6264403, 0x3B63EEBB, 0x59416782, 0x8404CD3A, + 0x9D0475F0, 0x4041DF48, 0x22635671, 0xFF26FCC9, + 0x2E238253, 0xF36628EB, 0x9144A1D2, 0x4C010B6A, + 0x5501B3A0, 0x88441918, 0xEA669021, 0x37233A99, + 0xD867E1B5, 0x05224B0D, 0x6700C234, 0xBA45688C, + 0xA345D046, 0x7E007AFE, 0x1C22F3C7, 0xC167597F, + 0xC747336E, 0x1A0299D6, 0x782010EF, 0xA565BA57, + 0xBC65029D, 0x6120A825, 0x0302211C, 0xDE478BA4, + 0x31035088, 0xEC46FA30, 0x8E647309, 0x5321D9B1, + 0x4A21617B, 0x9764CBC3, 0xF54642FA, 0x2803E842 + }, + { + 0x00000000, 0x38116FAC, 0x7022DF58, 0x4833B0F4, + 0xE045BEB0, 0xD854D11C, 0x906761E8, 0xA8760E44, + 0xC5670B91, 0xFD76643D, 0xB545D4C9, 0x8D54BB65, + 0x2522B521, 0x1D33DA8D, 0x55006A79, 0x6D1105D5, + 0x8F2261D3, 0xB7330E7F, 0xFF00BE8B, 0xC711D127, + 0x6F67DF63, 0x5776B0CF, 0x1F45003B, 0x27546F97, + 0x4A456A42, 0x725405EE, 0x3A67B51A, 0x0276DAB6, + 0xAA00D4F2, 0x9211BB5E, 0xDA220BAA, 0xE2336406, + 0x1BA8B557, 0x23B9DAFB, 0x6B8A6A0F, 0x539B05A3, + 0xFBED0BE7, 0xC3FC644B, 0x8BCFD4BF, 0xB3DEBB13, + 0xDECFBEC6, 0xE6DED16A, 0xAEED619E, 0x96FC0E32, + 0x3E8A0076, 0x069B6FDA, 0x4EA8DF2E, 0x76B9B082, + 0x948AD484, 0xAC9BBB28, 0xE4A80BDC, 0xDCB96470, + 0x74CF6A34, 0x4CDE0598, 0x04EDB56C, 0x3CFCDAC0, + 0x51EDDF15, 0x69FCB0B9, 0x21CF004D, 0x19DE6FE1, + 0xB1A861A5, 0x89B90E09, 0xC18ABEFD, 0xF99BD151, + 0x37516AAE, 0x0F400502, 0x4773B5F6, 0x7F62DA5A, + 0xD714D41E, 0xEF05BBB2, 0xA7360B46, 0x9F2764EA, + 0xF236613F, 0xCA270E93, 0x8214BE67, 0xBA05D1CB, + 0x1273DF8F, 0x2A62B023, 0x625100D7, 0x5A406F7B, + 0xB8730B7D, 0x806264D1, 0xC851D425, 0xF040BB89, + 0x5836B5CD, 0x6027DA61, 0x28146A95, 0x10050539, + 0x7D1400EC, 0x45056F40, 0x0D36DFB4, 0x3527B018, + 0x9D51BE5C, 0xA540D1F0, 0xED736104, 0xD5620EA8, + 0x2CF9DFF9, 0x14E8B055, 0x5CDB00A1, 0x64CA6F0D, + 0xCCBC6149, 0xF4AD0EE5, 0xBC9EBE11, 0x848FD1BD, + 0xE99ED468, 0xD18FBBC4, 0x99BC0B30, 0xA1AD649C, + 0x09DB6AD8, 0x31CA0574, 0x79F9B580, 0x41E8DA2C, + 0xA3DBBE2A, 0x9BCAD186, 0xD3F96172, 0xEBE80EDE, + 0x439E009A, 0x7B8F6F36, 0x33BCDFC2, 0x0BADB06E, + 0x66BCB5BB, 0x5EADDA17, 0x169E6AE3, 0x2E8F054F, + 0x86F90B0B, 0xBEE864A7, 0xF6DBD453, 0xCECABBFF, + 0x6EA2D55C, 0x56B3BAF0, 0x1E800A04, 0x269165A8, + 0x8EE76BEC, 0xB6F60440, 0xFEC5B4B4, 0xC6D4DB18, + 0xABC5DECD, 0x93D4B161, 0xDBE70195, 0xE3F66E39, + 0x4B80607D, 0x73910FD1, 0x3BA2BF25, 0x03B3D089, + 0xE180B48F, 0xD991DB23, 0x91A26BD7, 0xA9B3047B, + 0x01C50A3F, 0x39D46593, 0x71E7D567, 0x49F6BACB, + 0x24E7BF1E, 0x1CF6D0B2, 0x54C56046, 0x6CD40FEA, + 0xC4A201AE, 0xFCB36E02, 0xB480DEF6, 0x8C91B15A, + 0x750A600B, 0x4D1B0FA7, 0x0528BF53, 0x3D39D0FF, + 0x954FDEBB, 0xAD5EB117, 0xE56D01E3, 0xDD7C6E4F, + 0xB06D6B9A, 0x887C0436, 0xC04FB4C2, 0xF85EDB6E, + 0x5028D52A, 0x6839BA86, 0x200A0A72, 0x181B65DE, + 0xFA2801D8, 0xC2396E74, 0x8A0ADE80, 0xB21BB12C, + 0x1A6DBF68, 0x227CD0C4, 0x6A4F6030, 0x525E0F9C, + 0x3F4F0A49, 0x075E65E5, 0x4F6DD511, 0x777CBABD, + 0xDF0AB4F9, 0xE71BDB55, 0xAF286BA1, 0x9739040D, + 0x59F3BFF2, 0x61E2D05E, 0x29D160AA, 0x11C00F06, + 0xB9B60142, 0x81A76EEE, 0xC994DE1A, 0xF185B1B6, + 0x9C94B463, 0xA485DBCF, 0xECB66B3B, 0xD4A70497, + 0x7CD10AD3, 0x44C0657F, 0x0CF3D58B, 0x34E2BA27, + 0xD6D1DE21, 0xEEC0B18D, 0xA6F30179, 0x9EE26ED5, + 0x36946091, 0x0E850F3D, 0x46B6BFC9, 0x7EA7D065, + 0x13B6D5B0, 0x2BA7BA1C, 0x63940AE8, 0x5B856544, + 0xF3F36B00, 0xCBE204AC, 0x83D1B458, 0xBBC0DBF4, + 0x425B0AA5, 0x7A4A6509, 0x3279D5FD, 0x0A68BA51, + 0xA21EB415, 0x9A0FDBB9, 0xD23C6B4D, 0xEA2D04E1, + 0x873C0134, 0xBF2D6E98, 0xF71EDE6C, 0xCF0FB1C0, + 0x6779BF84, 0x5F68D028, 0x175B60DC, 0x2F4A0F70, + 0xCD796B76, 0xF56804DA, 0xBD5BB42E, 0x854ADB82, + 0x2D3CD5C6, 0x152DBA6A, 0x5D1E0A9E, 0x650F6532, + 0x081E60E7, 0x300F0F4B, 0x783CBFBF, 0x402DD013, + 0xE85BDE57, 0xD04AB1FB, 0x9879010F, 0xA0686EA3 + }, + { + 0x00000000, 0xEF306B19, 0xDB8CA0C3, 0x34BCCBDA, + 0xB2F53777, 0x5DC55C6E, 0x697997B4, 0x8649FCAD, + 0x6006181F, 0x8F367306, 0xBB8AB8DC, 0x54BAD3C5, + 0xD2F32F68, 0x3DC34471, 0x097F8FAB, 0xE64FE4B2, + 0xC00C303E, 0x2F3C5B27, 0x1B8090FD, 0xF4B0FBE4, + 0x72F90749, 0x9DC96C50, 0xA975A78A, 0x4645CC93, + 0xA00A2821, 0x4F3A4338, 0x7B8688E2, 0x94B6E3FB, + 0x12FF1F56, 0xFDCF744F, 0xC973BF95, 0x2643D48C, + 0x85F4168D, 0x6AC47D94, 0x5E78B64E, 0xB148DD57, + 0x370121FA, 0xD8314AE3, 0xEC8D8139, 0x03BDEA20, + 0xE5F20E92, 0x0AC2658B, 0x3E7EAE51, 0xD14EC548, + 0x570739E5, 0xB83752FC, 0x8C8B9926, 0x63BBF23F, + 0x45F826B3, 0xAAC84DAA, 0x9E748670, 0x7144ED69, + 0xF70D11C4, 0x183D7ADD, 0x2C81B107, 0xC3B1DA1E, + 0x25FE3EAC, 0xCACE55B5, 0xFE729E6F, 0x1142F576, + 0x970B09DB, 0x783B62C2, 0x4C87A918, 0xA3B7C201, + 0x0E045BEB, 0xE13430F2, 0xD588FB28, 0x3AB89031, + 0xBCF16C9C, 0x53C10785, 0x677DCC5F, 0x884DA746, + 0x6E0243F4, 0x813228ED, 0xB58EE337, 0x5ABE882E, + 0xDCF77483, 0x33C71F9A, 0x077BD440, 0xE84BBF59, + 0xCE086BD5, 0x213800CC, 0x1584CB16, 0xFAB4A00F, + 0x7CFD5CA2, 0x93CD37BB, 0xA771FC61, 0x48419778, + 0xAE0E73CA, 0x413E18D3, 0x7582D309, 0x9AB2B810, + 0x1CFB44BD, 0xF3CB2FA4, 0xC777E47E, 0x28478F67, + 0x8BF04D66, 0x64C0267F, 0x507CEDA5, 0xBF4C86BC, + 0x39057A11, 0xD6351108, 0xE289DAD2, 0x0DB9B1CB, + 0xEBF65579, 0x04C63E60, 0x307AF5BA, 0xDF4A9EA3, + 0x5903620E, 0xB6330917, 0x828FC2CD, 0x6DBFA9D4, + 0x4BFC7D58, 0xA4CC1641, 0x9070DD9B, 0x7F40B682, + 0xF9094A2F, 0x16392136, 0x2285EAEC, 0xCDB581F5, + 0x2BFA6547, 0xC4CA0E5E, 0xF076C584, 0x1F46AE9D, + 0x990F5230, 0x763F3929, 0x4283F2F3, 0xADB399EA, + 0x1C08B7D6, 0xF338DCCF, 0xC7841715, 0x28B47C0C, + 0xAEFD80A1, 0x41CDEBB8, 0x75712062, 0x9A414B7B, + 0x7C0EAFC9, 0x933EC4D0, 0xA7820F0A, 0x48B26413, + 0xCEFB98BE, 0x21CBF3A7, 0x1577387D, 0xFA475364, + 0xDC0487E8, 0x3334ECF1, 0x0788272B, 0xE8B84C32, + 0x6EF1B09F, 0x81C1DB86, 0xB57D105C, 0x5A4D7B45, + 0xBC029FF7, 0x5332F4EE, 0x678E3F34, 0x88BE542D, + 0x0EF7A880, 0xE1C7C399, 0xD57B0843, 0x3A4B635A, + 0x99FCA15B, 0x76CCCA42, 0x42700198, 0xAD406A81, + 0x2B09962C, 0xC439FD35, 0xF08536EF, 0x1FB55DF6, + 0xF9FAB944, 0x16CAD25D, 0x22761987, 0xCD46729E, + 0x4B0F8E33, 0xA43FE52A, 0x90832EF0, 0x7FB345E9, + 0x59F09165, 0xB6C0FA7C, 0x827C31A6, 0x6D4C5ABF, + 0xEB05A612, 0x0435CD0B, 0x308906D1, 0xDFB96DC8, + 0x39F6897A, 0xD6C6E263, 0xE27A29B9, 0x0D4A42A0, + 0x8B03BE0D, 0x6433D514, 0x508F1ECE, 0xBFBF75D7, + 0x120CEC3D, 0xFD3C8724, 0xC9804CFE, 0x26B027E7, + 0xA0F9DB4A, 0x4FC9B053, 0x7B757B89, 0x94451090, + 0x720AF422, 0x9D3A9F3B, 0xA98654E1, 0x46B63FF8, + 0xC0FFC355, 0x2FCFA84C, 0x1B736396, 0xF443088F, + 0xD200DC03, 0x3D30B71A, 0x098C7CC0, 0xE6BC17D9, + 0x60F5EB74, 0x8FC5806D, 0xBB794BB7, 0x544920AE, + 0xB206C41C, 0x5D36AF05, 0x698A64DF, 0x86BA0FC6, + 0x00F3F36B, 0xEFC39872, 0xDB7F53A8, 0x344F38B1, + 0x97F8FAB0, 0x78C891A9, 0x4C745A73, 0xA344316A, + 0x250DCDC7, 0xCA3DA6DE, 0xFE816D04, 0x11B1061D, + 0xF7FEE2AF, 0x18CE89B6, 0x2C72426C, 0xC3422975, + 0x450BD5D8, 0xAA3BBEC1, 0x9E87751B, 0x71B71E02, + 0x57F4CA8E, 0xB8C4A197, 0x8C786A4D, 0x63480154, + 0xE501FDF9, 0x0A3196E0, 0x3E8D5D3A, 0xD1BD3623, + 0x37F2D291, 0xD8C2B988, 0xEC7E7252, 0x034E194B, + 0x8507E5E6, 0x6A378EFF, 0x5E8B4525, 0xB1BB2E3C + }, + { + 0x00000000, 0x68032CC8, 0xD0065990, 0xB8057558, + 0xA5E0C5D1, 0xCDE3E919, 0x75E69C41, 0x1DE5B089, + 0x4E2DFD53, 0x262ED19B, 0x9E2BA4C3, 0xF628880B, + 0xEBCD3882, 0x83CE144A, 0x3BCB6112, 0x53C84DDA, + 0x9C5BFAA6, 0xF458D66E, 0x4C5DA336, 0x245E8FFE, + 0x39BB3F77, 0x51B813BF, 0xE9BD66E7, 0x81BE4A2F, + 0xD27607F5, 0xBA752B3D, 0x02705E65, 0x6A7372AD, + 0x7796C224, 0x1F95EEEC, 0xA7909BB4, 0xCF93B77C, + 0x3D5B83BD, 0x5558AF75, 0xED5DDA2D, 0x855EF6E5, + 0x98BB466C, 0xF0B86AA4, 0x48BD1FFC, 0x20BE3334, + 0x73767EEE, 0x1B755226, 0xA370277E, 0xCB730BB6, + 0xD696BB3F, 0xBE9597F7, 0x0690E2AF, 0x6E93CE67, + 0xA100791B, 0xC90355D3, 0x7106208B, 0x19050C43, + 0x04E0BCCA, 0x6CE39002, 0xD4E6E55A, 0xBCE5C992, + 0xEF2D8448, 0x872EA880, 0x3F2BDDD8, 0x5728F110, + 0x4ACD4199, 0x22CE6D51, 0x9ACB1809, 0xF2C834C1, + 0x7AB7077A, 0x12B42BB2, 0xAAB15EEA, 0xC2B27222, + 0xDF57C2AB, 0xB754EE63, 0x0F519B3B, 0x6752B7F3, + 0x349AFA29, 0x5C99D6E1, 0xE49CA3B9, 0x8C9F8F71, + 0x917A3FF8, 0xF9791330, 0x417C6668, 0x297F4AA0, + 0xE6ECFDDC, 0x8EEFD114, 0x36EAA44C, 0x5EE98884, + 0x430C380D, 0x2B0F14C5, 0x930A619D, 0xFB094D55, + 0xA8C1008F, 0xC0C22C47, 0x78C7591F, 0x10C475D7, + 0x0D21C55E, 0x6522E996, 0xDD279CCE, 0xB524B006, + 0x47EC84C7, 0x2FEFA80F, 0x97EADD57, 0xFFE9F19F, + 0xE20C4116, 0x8A0F6DDE, 0x320A1886, 0x5A09344E, + 0x09C17994, 0x61C2555C, 0xD9C72004, 0xB1C40CCC, + 0xAC21BC45, 0xC422908D, 0x7C27E5D5, 0x1424C91D, + 0xDBB77E61, 0xB3B452A9, 0x0BB127F1, 0x63B20B39, + 0x7E57BBB0, 0x16549778, 0xAE51E220, 0xC652CEE8, + 0x959A8332, 0xFD99AFFA, 0x459CDAA2, 0x2D9FF66A, + 0x307A46E3, 0x58796A2B, 0xE07C1F73, 0x887F33BB, + 0xF56E0EF4, 0x9D6D223C, 0x25685764, 0x4D6B7BAC, + 0x508ECB25, 0x388DE7ED, 0x808892B5, 0xE88BBE7D, + 0xBB43F3A7, 0xD340DF6F, 0x6B45AA37, 0x034686FF, + 0x1EA33676, 0x76A01ABE, 0xCEA56FE6, 0xA6A6432E, + 0x6935F452, 0x0136D89A, 0xB933ADC2, 0xD130810A, + 0xCCD53183, 0xA4D61D4B, 0x1CD36813, 0x74D044DB, + 0x27180901, 0x4F1B25C9, 0xF71E5091, 0x9F1D7C59, + 0x82F8CCD0, 0xEAFBE018, 0x52FE9540, 0x3AFDB988, + 0xC8358D49, 0xA036A181, 0x1833D4D9, 0x7030F811, + 0x6DD54898, 0x05D66450, 0xBDD31108, 0xD5D03DC0, + 0x8618701A, 0xEE1B5CD2, 0x561E298A, 0x3E1D0542, + 0x23F8B5CB, 0x4BFB9903, 0xF3FEEC5B, 0x9BFDC093, + 0x546E77EF, 0x3C6D5B27, 0x84682E7F, 0xEC6B02B7, + 0xF18EB23E, 0x998D9EF6, 0x2188EBAE, 0x498BC766, + 0x1A438ABC, 0x7240A674, 0xCA45D32C, 0xA246FFE4, + 0xBFA34F6D, 0xD7A063A5, 0x6FA516FD, 0x07A63A35, + 0x8FD9098E, 0xE7DA2546, 0x5FDF501E, 0x37DC7CD6, + 0x2A39CC5F, 0x423AE097, 0xFA3F95CF, 0x923CB907, + 0xC1F4F4DD, 0xA9F7D815, 0x11F2AD4D, 0x79F18185, + 0x6414310C, 0x0C171DC4, 0xB412689C, 0xDC114454, + 0x1382F328, 0x7B81DFE0, 0xC384AAB8, 0xAB878670, + 0xB66236F9, 0xDE611A31, 0x66646F69, 0x0E6743A1, + 0x5DAF0E7B, 0x35AC22B3, 0x8DA957EB, 0xE5AA7B23, + 0xF84FCBAA, 0x904CE762, 0x2849923A, 0x404ABEF2, + 0xB2828A33, 0xDA81A6FB, 0x6284D3A3, 0x0A87FF6B, + 0x17624FE2, 0x7F61632A, 0xC7641672, 0xAF673ABA, + 0xFCAF7760, 0x94AC5BA8, 0x2CA92EF0, 0x44AA0238, + 0x594FB2B1, 0x314C9E79, 0x8949EB21, 0xE14AC7E9, + 0x2ED97095, 0x46DA5C5D, 0xFEDF2905, 0x96DC05CD, + 0x8B39B544, 0xE33A998C, 0x5B3FECD4, 0x333CC01C, + 0x60F48DC6, 0x08F7A10E, 0xB0F2D456, 0xD8F1F89E, + 0xC5144817, 0xAD1764DF, 0x15121187, 0x7D113D4F + }, + { + 0x00000000, 0x493C7D27, 0x9278FA4E, 0xDB448769, + 0x211D826D, 0x6821FF4A, 0xB3657823, 0xFA590504, + 0x423B04DA, 0x0B0779FD, 0xD043FE94, 0x997F83B3, + 0x632686B7, 0x2A1AFB90, 0xF15E7CF9, 0xB86201DE, + 0x847609B4, 0xCD4A7493, 0x160EF3FA, 0x5F328EDD, + 0xA56B8BD9, 0xEC57F6FE, 0x37137197, 0x7E2F0CB0, + 0xC64D0D6E, 0x8F717049, 0x5435F720, 0x1D098A07, + 0xE7508F03, 0xAE6CF224, 0x7528754D, 0x3C14086A, + 0x0D006599, 0x443C18BE, 0x9F789FD7, 0xD644E2F0, + 0x2C1DE7F4, 0x65219AD3, 0xBE651DBA, 0xF759609D, + 0x4F3B6143, 0x06071C64, 0xDD439B0D, 0x947FE62A, + 0x6E26E32E, 0x271A9E09, 0xFC5E1960, 0xB5626447, + 0x89766C2D, 0xC04A110A, 0x1B0E9663, 0x5232EB44, + 0xA86BEE40, 0xE1579367, 0x3A13140E, 0x732F6929, + 0xCB4D68F7, 0x827115D0, 0x593592B9, 0x1009EF9E, + 0xEA50EA9A, 0xA36C97BD, 0x782810D4, 0x31146DF3, + 0x1A00CB32, 0x533CB615, 0x8878317C, 0xC1444C5B, + 0x3B1D495F, 0x72213478, 0xA965B311, 0xE059CE36, + 0x583BCFE8, 0x1107B2CF, 0xCA4335A6, 0x837F4881, + 0x79264D85, 0x301A30A2, 0xEB5EB7CB, 0xA262CAEC, + 0x9E76C286, 0xD74ABFA1, 0x0C0E38C8, 0x453245EF, + 0xBF6B40EB, 0xF6573DCC, 0x2D13BAA5, 0x642FC782, + 0xDC4DC65C, 0x9571BB7B, 0x4E353C12, 0x07094135, + 0xFD504431, 0xB46C3916, 0x6F28BE7F, 0x2614C358, + 0x1700AEAB, 0x5E3CD38C, 0x857854E5, 0xCC4429C2, + 0x361D2CC6, 0x7F2151E1, 0xA465D688, 0xED59ABAF, + 0x553BAA71, 0x1C07D756, 0xC743503F, 0x8E7F2D18, + 0x7426281C, 0x3D1A553B, 0xE65ED252, 0xAF62AF75, + 0x9376A71F, 0xDA4ADA38, 0x010E5D51, 0x48322076, + 0xB26B2572, 0xFB575855, 0x2013DF3C, 0x692FA21B, + 0xD14DA3C5, 0x9871DEE2, 0x4335598B, 0x0A0924AC, + 0xF05021A8, 0xB96C5C8F, 0x6228DBE6, 0x2B14A6C1, + 0x34019664, 0x7D3DEB43, 0xA6796C2A, 0xEF45110D, + 0x151C1409, 0x5C20692E, 0x8764EE47, 0xCE589360, + 0x763A92BE, 0x3F06EF99, 0xE44268F0, 0xAD7E15D7, + 0x572710D3, 0x1E1B6DF4, 0xC55FEA9D, 0x8C6397BA, + 0xB0779FD0, 0xF94BE2F7, 0x220F659E, 0x6B3318B9, + 0x916A1DBD, 0xD856609A, 0x0312E7F3, 0x4A2E9AD4, + 0xF24C9B0A, 0xBB70E62D, 0x60346144, 0x29081C63, + 0xD3511967, 0x9A6D6440, 0x4129E329, 0x08159E0E, + 0x3901F3FD, 0x703D8EDA, 0xAB7909B3, 0xE2457494, + 0x181C7190, 0x51200CB7, 0x8A648BDE, 0xC358F6F9, + 0x7B3AF727, 0x32068A00, 0xE9420D69, 0xA07E704E, + 0x5A27754A, 0x131B086D, 0xC85F8F04, 0x8163F223, + 0xBD77FA49, 0xF44B876E, 0x2F0F0007, 0x66337D20, + 0x9C6A7824, 0xD5560503, 0x0E12826A, 0x472EFF4D, + 0xFF4CFE93, 0xB67083B4, 0x6D3404DD, 0x240879FA, + 0xDE517CFE, 0x976D01D9, 0x4C2986B0, 0x0515FB97, + 0x2E015D56, 0x673D2071, 0xBC79A718, 0xF545DA3F, + 0x0F1CDF3B, 0x4620A21C, 0x9D642575, 0xD4585852, + 0x6C3A598C, 0x250624AB, 0xFE42A3C2, 0xB77EDEE5, + 0x4D27DBE1, 0x041BA6C6, 0xDF5F21AF, 0x96635C88, + 0xAA7754E2, 0xE34B29C5, 0x380FAEAC, 0x7133D38B, + 0x8B6AD68F, 0xC256ABA8, 0x19122CC1, 0x502E51E6, + 0xE84C5038, 0xA1702D1F, 0x7A34AA76, 0x3308D751, + 0xC951D255, 0x806DAF72, 0x5B29281B, 0x1215553C, + 0x230138CF, 0x6A3D45E8, 0xB179C281, 0xF845BFA6, + 0x021CBAA2, 0x4B20C785, 0x906440EC, 0xD9583DCB, + 0x613A3C15, 0x28064132, 0xF342C65B, 0xBA7EBB7C, + 0x4027BE78, 0x091BC35F, 0xD25F4436, 0x9B633911, + 0xA777317B, 0xEE4B4C5C, 0x350FCB35, 0x7C33B612, + 0x866AB316, 0xCF56CE31, 0x14124958, 0x5D2E347F, + 0xE54C35A1, 0xAC704886, 0x7734CFEF, 0x3E08B2C8, + 0xC451B7CC, 0x8D6DCAEB, 0x56294D82, 0x1F1530A5 + } +#else /* !WORDS_BIGENDIAN */ + { + 0x00000000, 0x03836BF2, 0xF7703BE1, 0xF4F35013, + 0x1F979AC7, 0x1C14F135, 0xE8E7A126, 0xEB64CAD4, + 0xCF58D98A, 0xCCDBB278, 0x3828E26B, 0x3BAB8999, + 0xD0CF434D, 0xD34C28BF, 0x27BF78AC, 0x243C135E, + 0x6FC75E10, 0x6C4435E2, 0x98B765F1, 0x9B340E03, + 0x7050C4D7, 0x73D3AF25, 0x8720FF36, 0x84A394C4, + 0xA09F879A, 0xA31CEC68, 0x57EFBC7B, 0x546CD789, + 0xBF081D5D, 0xBC8B76AF, 0x487826BC, 0x4BFB4D4E, + 0xDE8EBD20, 0xDD0DD6D2, 0x29FE86C1, 0x2A7DED33, + 0xC11927E7, 0xC29A4C15, 0x36691C06, 0x35EA77F4, + 0x11D664AA, 0x12550F58, 0xE6A65F4B, 0xE52534B9, + 0x0E41FE6D, 0x0DC2959F, 0xF931C58C, 0xFAB2AE7E, + 0xB149E330, 0xB2CA88C2, 0x4639D8D1, 0x45BAB323, + 0xAEDE79F7, 0xAD5D1205, 0x59AE4216, 0x5A2D29E4, + 0x7E113ABA, 0x7D925148, 0x8961015B, 0x8AE26AA9, + 0x6186A07D, 0x6205CB8F, 0x96F69B9C, 0x9575F06E, + 0xBC1D7B41, 0xBF9E10B3, 0x4B6D40A0, 0x48EE2B52, + 0xA38AE186, 0xA0098A74, 0x54FADA67, 0x5779B195, + 0x7345A2CB, 0x70C6C939, 0x8435992A, 0x87B6F2D8, + 0x6CD2380C, 0x6F5153FE, 0x9BA203ED, 0x9821681F, + 0xD3DA2551, 0xD0594EA3, 0x24AA1EB0, 0x27297542, + 0xCC4DBF96, 0xCFCED464, 0x3B3D8477, 0x38BEEF85, + 0x1C82FCDB, 0x1F019729, 0xEBF2C73A, 0xE871ACC8, + 0x0315661C, 0x00960DEE, 0xF4655DFD, 0xF7E6360F, + 0x6293C661, 0x6110AD93, 0x95E3FD80, 0x96609672, + 0x7D045CA6, 0x7E873754, 0x8A746747, 0x89F70CB5, + 0xADCB1FEB, 0xAE487419, 0x5ABB240A, 0x59384FF8, + 0xB25C852C, 0xB1DFEEDE, 0x452CBECD, 0x46AFD53F, + 0x0D549871, 0x0ED7F383, 0xFA24A390, 0xF9A7C862, + 0x12C302B6, 0x11406944, 0xE5B33957, 0xE63052A5, + 0xC20C41FB, 0xC18F2A09, 0x357C7A1A, 0x36FF11E8, + 0xDD9BDB3C, 0xDE18B0CE, 0x2AEBE0DD, 0x29688B2F, + 0x783BF682, 0x7BB89D70, 0x8F4BCD63, 0x8CC8A691, + 0x67AC6C45, 0x642F07B7, 0x90DC57A4, 0x935F3C56, + 0xB7632F08, 0xB4E044FA, 0x401314E9, 0x43907F1B, + 0xA8F4B5CF, 0xAB77DE3D, 0x5F848E2E, 0x5C07E5DC, + 0x17FCA892, 0x147FC360, 0xE08C9373, 0xE30FF881, + 0x086B3255, 0x0BE859A7, 0xFF1B09B4, 0xFC986246, + 0xD8A47118, 0xDB271AEA, 0x2FD44AF9, 0x2C57210B, + 0xC733EBDF, 0xC4B0802D, 0x3043D03E, 0x33C0BBCC, + 0xA6B54BA2, 0xA5362050, 0x51C57043, 0x52461BB1, + 0xB922D165, 0xBAA1BA97, 0x4E52EA84, 0x4DD18176, + 0x69ED9228, 0x6A6EF9DA, 0x9E9DA9C9, 0x9D1EC23B, + 0x767A08EF, 0x75F9631D, 0x810A330E, 0x828958FC, + 0xC97215B2, 0xCAF17E40, 0x3E022E53, 0x3D8145A1, + 0xD6E58F75, 0xD566E487, 0x2195B494, 0x2216DF66, + 0x062ACC38, 0x05A9A7CA, 0xF15AF7D9, 0xF2D99C2B, + 0x19BD56FF, 0x1A3E3D0D, 0xEECD6D1E, 0xED4E06EC, + 0xC4268DC3, 0xC7A5E631, 0x3356B622, 0x30D5DDD0, + 0xDBB11704, 0xD8327CF6, 0x2CC12CE5, 0x2F424717, + 0x0B7E5449, 0x08FD3FBB, 0xFC0E6FA8, 0xFF8D045A, + 0x14E9CE8E, 0x176AA57C, 0xE399F56F, 0xE01A9E9D, + 0xABE1D3D3, 0xA862B821, 0x5C91E832, 0x5F1283C0, + 0xB4764914, 0xB7F522E6, 0x430672F5, 0x40851907, + 0x64B90A59, 0x673A61AB, 0x93C931B8, 0x904A5A4A, + 0x7B2E909E, 0x78ADFB6C, 0x8C5EAB7F, 0x8FDDC08D, + 0x1AA830E3, 0x192B5B11, 0xEDD80B02, 0xEE5B60F0, + 0x053FAA24, 0x06BCC1D6, 0xF24F91C5, 0xF1CCFA37, + 0xD5F0E969, 0xD673829B, 0x2280D288, 0x2103B97A, + 0xCA6773AE, 0xC9E4185C, 0x3D17484F, 0x3E9423BD, + 0x756F6EF3, 0x76EC0501, 0x821F5512, 0x819C3EE0, + 0x6AF8F434, 0x697B9FC6, 0x9D88CFD5, 0x9E0BA427, + 0xBA37B779, 0xB9B4DC8B, 0x4D478C98, 0x4EC4E76A, + 0xA5A02DBE, 0xA623464C, 0x52D0165F, 0x51537DAD, + }, + { + 0x00000000, 0x7798A213, 0xEE304527, 0x99A8E734, + 0xDC618A4E, 0xABF9285D, 0x3251CF69, 0x45C96D7A, + 0xB8C3149D, 0xCF5BB68E, 0x56F351BA, 0x216BF3A9, + 0x64A29ED3, 0x133A3CC0, 0x8A92DBF4, 0xFD0A79E7, + 0x81F1C53F, 0xF669672C, 0x6FC18018, 0x1859220B, + 0x5D904F71, 0x2A08ED62, 0xB3A00A56, 0xC438A845, + 0x3932D1A2, 0x4EAA73B1, 0xD7029485, 0xA09A3696, + 0xE5535BEC, 0x92CBF9FF, 0x0B631ECB, 0x7CFBBCD8, + 0x02E38B7F, 0x757B296C, 0xECD3CE58, 0x9B4B6C4B, + 0xDE820131, 0xA91AA322, 0x30B24416, 0x472AE605, + 0xBA209FE2, 0xCDB83DF1, 0x5410DAC5, 0x238878D6, + 0x664115AC, 0x11D9B7BF, 0x8871508B, 0xFFE9F298, + 0x83124E40, 0xF48AEC53, 0x6D220B67, 0x1ABAA974, + 0x5F73C40E, 0x28EB661D, 0xB1438129, 0xC6DB233A, + 0x3BD15ADD, 0x4C49F8CE, 0xD5E11FFA, 0xA279BDE9, + 0xE7B0D093, 0x90287280, 0x098095B4, 0x7E1837A7, + 0x04C617FF, 0x735EB5EC, 0xEAF652D8, 0x9D6EF0CB, + 0xD8A79DB1, 0xAF3F3FA2, 0x3697D896, 0x410F7A85, + 0xBC050362, 0xCB9DA171, 0x52354645, 0x25ADE456, + 0x6064892C, 0x17FC2B3F, 0x8E54CC0B, 0xF9CC6E18, + 0x8537D2C0, 0xF2AF70D3, 0x6B0797E7, 0x1C9F35F4, + 0x5956588E, 0x2ECEFA9D, 0xB7661DA9, 0xC0FEBFBA, + 0x3DF4C65D, 0x4A6C644E, 0xD3C4837A, 0xA45C2169, + 0xE1954C13, 0x960DEE00, 0x0FA50934, 0x783DAB27, + 0x06259C80, 0x71BD3E93, 0xE815D9A7, 0x9F8D7BB4, + 0xDA4416CE, 0xADDCB4DD, 0x347453E9, 0x43ECF1FA, + 0xBEE6881D, 0xC97E2A0E, 0x50D6CD3A, 0x274E6F29, + 0x62870253, 0x151FA040, 0x8CB74774, 0xFB2FE567, + 0x87D459BF, 0xF04CFBAC, 0x69E41C98, 0x1E7CBE8B, + 0x5BB5D3F1, 0x2C2D71E2, 0xB58596D6, 0xC21D34C5, + 0x3F174D22, 0x488FEF31, 0xD1270805, 0xA6BFAA16, + 0xE376C76C, 0x94EE657F, 0x0D46824B, 0x7ADE2058, + 0xF9FAC3FB, 0x8E6261E8, 0x17CA86DC, 0x605224CF, + 0x259B49B5, 0x5203EBA6, 0xCBAB0C92, 0xBC33AE81, + 0x4139D766, 0x36A17575, 0xAF099241, 0xD8913052, + 0x9D585D28, 0xEAC0FF3B, 0x7368180F, 0x04F0BA1C, + 0x780B06C4, 0x0F93A4D7, 0x963B43E3, 0xE1A3E1F0, + 0xA46A8C8A, 0xD3F22E99, 0x4A5AC9AD, 0x3DC26BBE, + 0xC0C81259, 0xB750B04A, 0x2EF8577E, 0x5960F56D, + 0x1CA99817, 0x6B313A04, 0xF299DD30, 0x85017F23, + 0xFB194884, 0x8C81EA97, 0x15290DA3, 0x62B1AFB0, + 0x2778C2CA, 0x50E060D9, 0xC94887ED, 0xBED025FE, + 0x43DA5C19, 0x3442FE0A, 0xADEA193E, 0xDA72BB2D, + 0x9FBBD657, 0xE8237444, 0x718B9370, 0x06133163, + 0x7AE88DBB, 0x0D702FA8, 0x94D8C89C, 0xE3406A8F, + 0xA68907F5, 0xD111A5E6, 0x48B942D2, 0x3F21E0C1, + 0xC22B9926, 0xB5B33B35, 0x2C1BDC01, 0x5B837E12, + 0x1E4A1368, 0x69D2B17B, 0xF07A564F, 0x87E2F45C, + 0xFD3CD404, 0x8AA47617, 0x130C9123, 0x64943330, + 0x215D5E4A, 0x56C5FC59, 0xCF6D1B6D, 0xB8F5B97E, + 0x45FFC099, 0x3267628A, 0xABCF85BE, 0xDC5727AD, + 0x999E4AD7, 0xEE06E8C4, 0x77AE0FF0, 0x0036ADE3, + 0x7CCD113B, 0x0B55B328, 0x92FD541C, 0xE565F60F, + 0xA0AC9B75, 0xD7343966, 0x4E9CDE52, 0x39047C41, + 0xC40E05A6, 0xB396A7B5, 0x2A3E4081, 0x5DA6E292, + 0x186F8FE8, 0x6FF72DFB, 0xF65FCACF, 0x81C768DC, + 0xFFDF5F7B, 0x8847FD68, 0x11EF1A5C, 0x6677B84F, + 0x23BED535, 0x54267726, 0xCD8E9012, 0xBA163201, + 0x471C4BE6, 0x3084E9F5, 0xA92C0EC1, 0xDEB4ACD2, + 0x9B7DC1A8, 0xECE563BB, 0x754D848F, 0x02D5269C, + 0x7E2E9A44, 0x09B63857, 0x901EDF63, 0xE7867D70, + 0xA24F100A, 0xD5D7B219, 0x4C7F552D, 0x3BE7F73E, + 0xC6ED8ED9, 0xB1752CCA, 0x28DDCBFE, 0x5F4569ED, + 0x1A8C0497, 0x6D14A684, 0xF4BC41B0, 0x8324E3A3, + }, + { + 0x00000000, 0x7E9241A5, 0x0D526F4F, 0x73C02EEA, + 0x1AA4DE9E, 0x64369F3B, 0x17F6B1D1, 0x6964F074, + 0xC53E5138, 0xBBAC109D, 0xC86C3E77, 0xB6FE7FD2, + 0xDF9A8FA6, 0xA108CE03, 0xD2C8E0E9, 0xAC5AA14C, + 0x8A7DA270, 0xF4EFE3D5, 0x872FCD3F, 0xF9BD8C9A, + 0x90D97CEE, 0xEE4B3D4B, 0x9D8B13A1, 0xE3195204, + 0x4F43F348, 0x31D1B2ED, 0x42119C07, 0x3C83DDA2, + 0x55E72DD6, 0x2B756C73, 0x58B54299, 0x2627033C, + 0x14FB44E1, 0x6A690544, 0x19A92BAE, 0x673B6A0B, + 0x0E5F9A7F, 0x70CDDBDA, 0x030DF530, 0x7D9FB495, + 0xD1C515D9, 0xAF57547C, 0xDC977A96, 0xA2053B33, + 0xCB61CB47, 0xB5F38AE2, 0xC633A408, 0xB8A1E5AD, + 0x9E86E691, 0xE014A734, 0x93D489DE, 0xED46C87B, + 0x8422380F, 0xFAB079AA, 0x89705740, 0xF7E216E5, + 0x5BB8B7A9, 0x252AF60C, 0x56EAD8E6, 0x28789943, + 0x411C6937, 0x3F8E2892, 0x4C4E0678, 0x32DC47DD, + 0xD98065C7, 0xA7122462, 0xD4D20A88, 0xAA404B2D, + 0xC324BB59, 0xBDB6FAFC, 0xCE76D416, 0xB0E495B3, + 0x1CBE34FF, 0x622C755A, 0x11EC5BB0, 0x6F7E1A15, + 0x061AEA61, 0x7888ABC4, 0x0B48852E, 0x75DAC48B, + 0x53FDC7B7, 0x2D6F8612, 0x5EAFA8F8, 0x203DE95D, + 0x49591929, 0x37CB588C, 0x440B7666, 0x3A9937C3, + 0x96C3968F, 0xE851D72A, 0x9B91F9C0, 0xE503B865, + 0x8C674811, 0xF2F509B4, 0x8135275E, 0xFFA766FB, + 0xCD7B2126, 0xB3E96083, 0xC0294E69, 0xBEBB0FCC, + 0xD7DFFFB8, 0xA94DBE1D, 0xDA8D90F7, 0xA41FD152, + 0x0845701E, 0x76D731BB, 0x05171F51, 0x7B855EF4, + 0x12E1AE80, 0x6C73EF25, 0x1FB3C1CF, 0x6121806A, + 0x47068356, 0x3994C2F3, 0x4A54EC19, 0x34C6ADBC, + 0x5DA25DC8, 0x23301C6D, 0x50F03287, 0x2E627322, + 0x8238D26E, 0xFCAA93CB, 0x8F6ABD21, 0xF1F8FC84, + 0x989C0CF0, 0xE60E4D55, 0x95CE63BF, 0xEB5C221A, + 0x4377278B, 0x3DE5662E, 0x4E2548C4, 0x30B70961, + 0x59D3F915, 0x2741B8B0, 0x5481965A, 0x2A13D7FF, + 0x864976B3, 0xF8DB3716, 0x8B1B19FC, 0xF5895859, + 0x9CEDA82D, 0xE27FE988, 0x91BFC762, 0xEF2D86C7, + 0xC90A85FB, 0xB798C45E, 0xC458EAB4, 0xBACAAB11, + 0xD3AE5B65, 0xAD3C1AC0, 0xDEFC342A, 0xA06E758F, + 0x0C34D4C3, 0x72A69566, 0x0166BB8C, 0x7FF4FA29, + 0x16900A5D, 0x68024BF8, 0x1BC26512, 0x655024B7, + 0x578C636A, 0x291E22CF, 0x5ADE0C25, 0x244C4D80, + 0x4D28BDF4, 0x33BAFC51, 0x407AD2BB, 0x3EE8931E, + 0x92B23252, 0xEC2073F7, 0x9FE05D1D, 0xE1721CB8, + 0x8816ECCC, 0xF684AD69, 0x85448383, 0xFBD6C226, + 0xDDF1C11A, 0xA36380BF, 0xD0A3AE55, 0xAE31EFF0, + 0xC7551F84, 0xB9C75E21, 0xCA0770CB, 0xB495316E, + 0x18CF9022, 0x665DD187, 0x159DFF6D, 0x6B0FBEC8, + 0x026B4EBC, 0x7CF90F19, 0x0F3921F3, 0x71AB6056, + 0x9AF7424C, 0xE46503E9, 0x97A52D03, 0xE9376CA6, + 0x80539CD2, 0xFEC1DD77, 0x8D01F39D, 0xF393B238, + 0x5FC91374, 0x215B52D1, 0x529B7C3B, 0x2C093D9E, + 0x456DCDEA, 0x3BFF8C4F, 0x483FA2A5, 0x36ADE300, + 0x108AE03C, 0x6E18A199, 0x1DD88F73, 0x634ACED6, + 0x0A2E3EA2, 0x74BC7F07, 0x077C51ED, 0x79EE1048, + 0xD5B4B104, 0xAB26F0A1, 0xD8E6DE4B, 0xA6749FEE, + 0xCF106F9A, 0xB1822E3F, 0xC24200D5, 0xBCD04170, + 0x8E0C06AD, 0xF09E4708, 0x835E69E2, 0xFDCC2847, + 0x94A8D833, 0xEA3A9996, 0x99FAB77C, 0xE768F6D9, + 0x4B325795, 0x35A01630, 0x466038DA, 0x38F2797F, + 0x5196890B, 0x2F04C8AE, 0x5CC4E644, 0x2256A7E1, + 0x0471A4DD, 0x7AE3E578, 0x0923CB92, 0x77B18A37, + 0x1ED57A43, 0x60473BE6, 0x1387150C, 0x6D1554A9, + 0xC14FF5E5, 0xBFDDB440, 0xCC1D9AAA, 0xB28FDB0F, + 0xDBEB2B7B, 0xA5796ADE, 0xD6B94434, 0xA82B0591, + }, + { + 0x00000000, 0xB8AA45DD, 0x812367BF, 0x39892262, + 0xF331227B, 0x4B9B67A6, 0x721245C4, 0xCAB80019, + 0xE66344F6, 0x5EC9012B, 0x67402349, 0xDFEA6694, + 0x1552668D, 0xADF82350, 0x94710132, 0x2CDB44EF, + 0x3DB164E9, 0x851B2134, 0xBC920356, 0x0438468B, + 0xCE804692, 0x762A034F, 0x4FA3212D, 0xF70964F0, + 0xDBD2201F, 0x637865C2, 0x5AF147A0, 0xE25B027D, + 0x28E30264, 0x904947B9, 0xA9C065DB, 0x116A2006, + 0x8B1425D7, 0x33BE600A, 0x0A374268, 0xB29D07B5, + 0x782507AC, 0xC08F4271, 0xF9066013, 0x41AC25CE, + 0x6D776121, 0xD5DD24FC, 0xEC54069E, 0x54FE4343, + 0x9E46435A, 0x26EC0687, 0x1F6524E5, 0xA7CF6138, + 0xB6A5413E, 0x0E0F04E3, 0x37862681, 0x8F2C635C, + 0x45946345, 0xFD3E2698, 0xC4B704FA, 0x7C1D4127, + 0x50C605C8, 0xE86C4015, 0xD1E56277, 0x694F27AA, + 0xA3F727B3, 0x1B5D626E, 0x22D4400C, 0x9A7E05D1, + 0xE75FA6AB, 0x5FF5E376, 0x667CC114, 0xDED684C9, + 0x146E84D0, 0xACC4C10D, 0x954DE36F, 0x2DE7A6B2, + 0x013CE25D, 0xB996A780, 0x801F85E2, 0x38B5C03F, + 0xF20DC026, 0x4AA785FB, 0x732EA799, 0xCB84E244, + 0xDAEEC242, 0x6244879F, 0x5BCDA5FD, 0xE367E020, + 0x29DFE039, 0x9175A5E4, 0xA8FC8786, 0x1056C25B, + 0x3C8D86B4, 0x8427C369, 0xBDAEE10B, 0x0504A4D6, + 0xCFBCA4CF, 0x7716E112, 0x4E9FC370, 0xF63586AD, + 0x6C4B837C, 0xD4E1C6A1, 0xED68E4C3, 0x55C2A11E, + 0x9F7AA107, 0x27D0E4DA, 0x1E59C6B8, 0xA6F38365, + 0x8A28C78A, 0x32828257, 0x0B0BA035, 0xB3A1E5E8, + 0x7919E5F1, 0xC1B3A02C, 0xF83A824E, 0x4090C793, + 0x51FAE795, 0xE950A248, 0xD0D9802A, 0x6873C5F7, + 0xA2CBC5EE, 0x1A618033, 0x23E8A251, 0x9B42E78C, + 0xB799A363, 0x0F33E6BE, 0x36BAC4DC, 0x8E108101, + 0x44A88118, 0xFC02C4C5, 0xC58BE6A7, 0x7D21A37A, + 0x3FC9A052, 0x8763E58F, 0xBEEAC7ED, 0x06408230, + 0xCCF88229, 0x7452C7F4, 0x4DDBE596, 0xF571A04B, + 0xD9AAE4A4, 0x6100A179, 0x5889831B, 0xE023C6C6, + 0x2A9BC6DF, 0x92318302, 0xABB8A160, 0x1312E4BD, + 0x0278C4BB, 0xBAD28166, 0x835BA304, 0x3BF1E6D9, + 0xF149E6C0, 0x49E3A31D, 0x706A817F, 0xC8C0C4A2, + 0xE41B804D, 0x5CB1C590, 0x6538E7F2, 0xDD92A22F, + 0x172AA236, 0xAF80E7EB, 0x9609C589, 0x2EA38054, + 0xB4DD8585, 0x0C77C058, 0x35FEE23A, 0x8D54A7E7, + 0x47ECA7FE, 0xFF46E223, 0xC6CFC041, 0x7E65859C, + 0x52BEC173, 0xEA1484AE, 0xD39DA6CC, 0x6B37E311, + 0xA18FE308, 0x1925A6D5, 0x20AC84B7, 0x9806C16A, + 0x896CE16C, 0x31C6A4B1, 0x084F86D3, 0xB0E5C30E, + 0x7A5DC317, 0xC2F786CA, 0xFB7EA4A8, 0x43D4E175, + 0x6F0FA59A, 0xD7A5E047, 0xEE2CC225, 0x568687F8, + 0x9C3E87E1, 0x2494C23C, 0x1D1DE05E, 0xA5B7A583, + 0xD89606F9, 0x603C4324, 0x59B56146, 0xE11F249B, + 0x2BA72482, 0x930D615F, 0xAA84433D, 0x122E06E0, + 0x3EF5420F, 0x865F07D2, 0xBFD625B0, 0x077C606D, + 0xCDC46074, 0x756E25A9, 0x4CE707CB, 0xF44D4216, + 0xE5276210, 0x5D8D27CD, 0x640405AF, 0xDCAE4072, + 0x1616406B, 0xAEBC05B6, 0x973527D4, 0x2F9F6209, + 0x034426E6, 0xBBEE633B, 0x82674159, 0x3ACD0484, + 0xF075049D, 0x48DF4140, 0x71566322, 0xC9FC26FF, + 0x5382232E, 0xEB2866F3, 0xD2A14491, 0x6A0B014C, + 0xA0B30155, 0x18194488, 0x219066EA, 0x993A2337, + 0xB5E167D8, 0x0D4B2205, 0x34C20067, 0x8C6845BA, + 0x46D045A3, 0xFE7A007E, 0xC7F3221C, 0x7F5967C1, + 0x6E3347C7, 0xD699021A, 0xEF102078, 0x57BA65A5, + 0x9D0265BC, 0x25A82061, 0x1C210203, 0xA48B47DE, + 0x88500331, 0x30FA46EC, 0x0973648E, 0xB1D92153, + 0x7B61214A, 0xC3CB6497, 0xFA4246F5, 0x42E80328, + }, + { + 0x00000000, 0xAC6F1138, 0x58DF2270, 0xF4B03348, + 0xB0BE45E0, 0x1CD154D8, 0xE8616790, 0x440E76A8, + 0x910B67C5, 0x3D6476FD, 0xC9D445B5, 0x65BB548D, + 0x21B52225, 0x8DDA331D, 0x796A0055, 0xD505116D, + 0xD361228F, 0x7F0E33B7, 0x8BBE00FF, 0x27D111C7, + 0x63DF676F, 0xCFB07657, 0x3B00451F, 0x976F5427, + 0x426A454A, 0xEE055472, 0x1AB5673A, 0xB6DA7602, + 0xF2D400AA, 0x5EBB1192, 0xAA0B22DA, 0x066433E2, + 0x57B5A81B, 0xFBDAB923, 0x0F6A8A6B, 0xA3059B53, + 0xE70BEDFB, 0x4B64FCC3, 0xBFD4CF8B, 0x13BBDEB3, + 0xC6BECFDE, 0x6AD1DEE6, 0x9E61EDAE, 0x320EFC96, + 0x76008A3E, 0xDA6F9B06, 0x2EDFA84E, 0x82B0B976, + 0x84D48A94, 0x28BB9BAC, 0xDC0BA8E4, 0x7064B9DC, + 0x346ACF74, 0x9805DE4C, 0x6CB5ED04, 0xC0DAFC3C, + 0x15DFED51, 0xB9B0FC69, 0x4D00CF21, 0xE16FDE19, + 0xA561A8B1, 0x090EB989, 0xFDBE8AC1, 0x51D19BF9, + 0xAE6A5137, 0x0205400F, 0xF6B57347, 0x5ADA627F, + 0x1ED414D7, 0xB2BB05EF, 0x460B36A7, 0xEA64279F, + 0x3F6136F2, 0x930E27CA, 0x67BE1482, 0xCBD105BA, + 0x8FDF7312, 0x23B0622A, 0xD7005162, 0x7B6F405A, + 0x7D0B73B8, 0xD1646280, 0x25D451C8, 0x89BB40F0, + 0xCDB53658, 0x61DA2760, 0x956A1428, 0x39050510, + 0xEC00147D, 0x406F0545, 0xB4DF360D, 0x18B02735, + 0x5CBE519D, 0xF0D140A5, 0x046173ED, 0xA80E62D5, + 0xF9DFF92C, 0x55B0E814, 0xA100DB5C, 0x0D6FCA64, + 0x4961BCCC, 0xE50EADF4, 0x11BE9EBC, 0xBDD18F84, + 0x68D49EE9, 0xC4BB8FD1, 0x300BBC99, 0x9C64ADA1, + 0xD86ADB09, 0x7405CA31, 0x80B5F979, 0x2CDAE841, + 0x2ABEDBA3, 0x86D1CA9B, 0x7261F9D3, 0xDE0EE8EB, + 0x9A009E43, 0x366F8F7B, 0xC2DFBC33, 0x6EB0AD0B, + 0xBBB5BC66, 0x17DAAD5E, 0xE36A9E16, 0x4F058F2E, + 0x0B0BF986, 0xA764E8BE, 0x53D4DBF6, 0xFFBBCACE, + 0x5CD5A26E, 0xF0BAB356, 0x040A801E, 0xA8659126, + 0xEC6BE78E, 0x4004F6B6, 0xB4B4C5FE, 0x18DBD4C6, + 0xCDDEC5AB, 0x61B1D493, 0x9501E7DB, 0x396EF6E3, + 0x7D60804B, 0xD10F9173, 0x25BFA23B, 0x89D0B303, + 0x8FB480E1, 0x23DB91D9, 0xD76BA291, 0x7B04B3A9, + 0x3F0AC501, 0x9365D439, 0x67D5E771, 0xCBBAF649, + 0x1EBFE724, 0xB2D0F61C, 0x4660C554, 0xEA0FD46C, + 0xAE01A2C4, 0x026EB3FC, 0xF6DE80B4, 0x5AB1918C, + 0x0B600A75, 0xA70F1B4D, 0x53BF2805, 0xFFD0393D, + 0xBBDE4F95, 0x17B15EAD, 0xE3016DE5, 0x4F6E7CDD, + 0x9A6B6DB0, 0x36047C88, 0xC2B44FC0, 0x6EDB5EF8, + 0x2AD52850, 0x86BA3968, 0x720A0A20, 0xDE651B18, + 0xD80128FA, 0x746E39C2, 0x80DE0A8A, 0x2CB11BB2, + 0x68BF6D1A, 0xC4D07C22, 0x30604F6A, 0x9C0F5E52, + 0x490A4F3F, 0xE5655E07, 0x11D56D4F, 0xBDBA7C77, + 0xF9B40ADF, 0x55DB1BE7, 0xA16B28AF, 0x0D043997, + 0xF2BFF359, 0x5ED0E261, 0xAA60D129, 0x060FC011, + 0x4201B6B9, 0xEE6EA781, 0x1ADE94C9, 0xB6B185F1, + 0x63B4949C, 0xCFDB85A4, 0x3B6BB6EC, 0x9704A7D4, + 0xD30AD17C, 0x7F65C044, 0x8BD5F30C, 0x27BAE234, + 0x21DED1D6, 0x8DB1C0EE, 0x7901F3A6, 0xD56EE29E, + 0x91609436, 0x3D0F850E, 0xC9BFB646, 0x65D0A77E, + 0xB0D5B613, 0x1CBAA72B, 0xE80A9463, 0x4465855B, + 0x006BF3F3, 0xAC04E2CB, 0x58B4D183, 0xF4DBC0BB, + 0xA50A5B42, 0x09654A7A, 0xFDD57932, 0x51BA680A, + 0x15B41EA2, 0xB9DB0F9A, 0x4D6B3CD2, 0xE1042DEA, + 0x34013C87, 0x986E2DBF, 0x6CDE1EF7, 0xC0B10FCF, + 0x84BF7967, 0x28D0685F, 0xDC605B17, 0x700F4A2F, + 0x766B79CD, 0xDA0468F5, 0x2EB45BBD, 0x82DB4A85, + 0xC6D53C2D, 0x6ABA2D15, 0x9E0A1E5D, 0x32650F65, + 0xE7601E08, 0x4B0F0F30, 0xBFBF3C78, 0x13D02D40, + 0x57DE5BE8, 0xFBB14AD0, 0x0F017998, 0xA36E68A0, + }, + { + 0x00000000, 0x196B30EF, 0xC3A08CDB, 0xDACBBC34, + 0x7737F5B2, 0x6E5CC55D, 0xB4977969, 0xADFC4986, + 0x1F180660, 0x0673368F, 0xDCB88ABB, 0xC5D3BA54, + 0x682FF3D2, 0x7144C33D, 0xAB8F7F09, 0xB2E44FE6, + 0x3E300CC0, 0x275B3C2F, 0xFD90801B, 0xE4FBB0F4, + 0x4907F972, 0x506CC99D, 0x8AA775A9, 0x93CC4546, + 0x21280AA0, 0x38433A4F, 0xE288867B, 0xFBE3B694, + 0x561FFF12, 0x4F74CFFD, 0x95BF73C9, 0x8CD44326, + 0x8D16F485, 0x947DC46A, 0x4EB6785E, 0x57DD48B1, + 0xFA210137, 0xE34A31D8, 0x39818DEC, 0x20EABD03, + 0x920EF2E5, 0x8B65C20A, 0x51AE7E3E, 0x48C54ED1, + 0xE5390757, 0xFC5237B8, 0x26998B8C, 0x3FF2BB63, + 0xB326F845, 0xAA4DC8AA, 0x7086749E, 0x69ED4471, + 0xC4110DF7, 0xDD7A3D18, 0x07B1812C, 0x1EDAB1C3, + 0xAC3EFE25, 0xB555CECA, 0x6F9E72FE, 0x76F54211, + 0xDB090B97, 0xC2623B78, 0x18A9874C, 0x01C2B7A3, + 0xEB5B040E, 0xF23034E1, 0x28FB88D5, 0x3190B83A, + 0x9C6CF1BC, 0x8507C153, 0x5FCC7D67, 0x46A74D88, + 0xF443026E, 0xED283281, 0x37E38EB5, 0x2E88BE5A, + 0x8374F7DC, 0x9A1FC733, 0x40D47B07, 0x59BF4BE8, + 0xD56B08CE, 0xCC003821, 0x16CB8415, 0x0FA0B4FA, + 0xA25CFD7C, 0xBB37CD93, 0x61FC71A7, 0x78974148, + 0xCA730EAE, 0xD3183E41, 0x09D38275, 0x10B8B29A, + 0xBD44FB1C, 0xA42FCBF3, 0x7EE477C7, 0x678F4728, + 0x664DF08B, 0x7F26C064, 0xA5ED7C50, 0xBC864CBF, + 0x117A0539, 0x081135D6, 0xD2DA89E2, 0xCBB1B90D, + 0x7955F6EB, 0x603EC604, 0xBAF57A30, 0xA39E4ADF, + 0x0E620359, 0x170933B6, 0xCDC28F82, 0xD4A9BF6D, + 0x587DFC4B, 0x4116CCA4, 0x9BDD7090, 0x82B6407F, + 0x2F4A09F9, 0x36213916, 0xECEA8522, 0xF581B5CD, + 0x4765FA2B, 0x5E0ECAC4, 0x84C576F0, 0x9DAE461F, + 0x30520F99, 0x29393F76, 0xF3F28342, 0xEA99B3AD, + 0xD6B7081C, 0xCFDC38F3, 0x151784C7, 0x0C7CB428, + 0xA180FDAE, 0xB8EBCD41, 0x62207175, 0x7B4B419A, + 0xC9AF0E7C, 0xD0C43E93, 0x0A0F82A7, 0x1364B248, + 0xBE98FBCE, 0xA7F3CB21, 0x7D387715, 0x645347FA, + 0xE88704DC, 0xF1EC3433, 0x2B278807, 0x324CB8E8, + 0x9FB0F16E, 0x86DBC181, 0x5C107DB5, 0x457B4D5A, + 0xF79F02BC, 0xEEF43253, 0x343F8E67, 0x2D54BE88, + 0x80A8F70E, 0x99C3C7E1, 0x43087BD5, 0x5A634B3A, + 0x5BA1FC99, 0x42CACC76, 0x98017042, 0x816A40AD, + 0x2C96092B, 0x35FD39C4, 0xEF3685F0, 0xF65DB51F, + 0x44B9FAF9, 0x5DD2CA16, 0x87197622, 0x9E7246CD, + 0x338E0F4B, 0x2AE53FA4, 0xF02E8390, 0xE945B37F, + 0x6591F059, 0x7CFAC0B6, 0xA6317C82, 0xBF5A4C6D, + 0x12A605EB, 0x0BCD3504, 0xD1068930, 0xC86DB9DF, + 0x7A89F639, 0x63E2C6D6, 0xB9297AE2, 0xA0424A0D, + 0x0DBE038B, 0x14D53364, 0xCE1E8F50, 0xD775BFBF, + 0x3DEC0C12, 0x24873CFD, 0xFE4C80C9, 0xE727B026, + 0x4ADBF9A0, 0x53B0C94F, 0x897B757B, 0x90104594, + 0x22F40A72, 0x3B9F3A9D, 0xE15486A9, 0xF83FB646, + 0x55C3FFC0, 0x4CA8CF2F, 0x9663731B, 0x8F0843F4, + 0x03DC00D2, 0x1AB7303D, 0xC07C8C09, 0xD917BCE6, + 0x74EBF560, 0x6D80C58F, 0xB74B79BB, 0xAE204954, + 0x1CC406B2, 0x05AF365D, 0xDF648A69, 0xC60FBA86, + 0x6BF3F300, 0x7298C3EF, 0xA8537FDB, 0xB1384F34, + 0xB0FAF897, 0xA991C878, 0x735A744C, 0x6A3144A3, + 0xC7CD0D25, 0xDEA63DCA, 0x046D81FE, 0x1D06B111, + 0xAFE2FEF7, 0xB689CE18, 0x6C42722C, 0x752942C3, + 0xD8D50B45, 0xC1BE3BAA, 0x1B75879E, 0x021EB771, + 0x8ECAF457, 0x97A1C4B8, 0x4D6A788C, 0x54014863, + 0xF9FD01E5, 0xE096310A, 0x3A5D8D3E, 0x2336BDD1, + 0x91D2F237, 0x88B9C2D8, 0x52727EEC, 0x4B194E03, + 0xE6E50785, 0xFF8E376A, 0x25458B5E, 0x3C2EBBB1, + }, + { + 0x00000000, 0xC82C0368, 0x905906D0, 0x587505B8, + 0xD1C5E0A5, 0x19E9E3CD, 0x419CE675, 0x89B0E51D, + 0x53FD2D4E, 0x9BD12E26, 0xC3A42B9E, 0x0B8828F6, + 0x8238CDEB, 0x4A14CE83, 0x1261CB3B, 0xDA4DC853, + 0xA6FA5B9C, 0x6ED658F4, 0x36A35D4C, 0xFE8F5E24, + 0x773FBB39, 0xBF13B851, 0xE766BDE9, 0x2F4ABE81, + 0xF50776D2, 0x3D2B75BA, 0x655E7002, 0xAD72736A, + 0x24C29677, 0xECEE951F, 0xB49B90A7, 0x7CB793CF, + 0xBD835B3D, 0x75AF5855, 0x2DDA5DED, 0xE5F65E85, + 0x6C46BB98, 0xA46AB8F0, 0xFC1FBD48, 0x3433BE20, + 0xEE7E7673, 0x2652751B, 0x7E2770A3, 0xB60B73CB, + 0x3FBB96D6, 0xF79795BE, 0xAFE29006, 0x67CE936E, + 0x1B7900A1, 0xD35503C9, 0x8B200671, 0x430C0519, + 0xCABCE004, 0x0290E36C, 0x5AE5E6D4, 0x92C9E5BC, + 0x48842DEF, 0x80A82E87, 0xD8DD2B3F, 0x10F12857, + 0x9941CD4A, 0x516DCE22, 0x0918CB9A, 0xC134C8F2, + 0x7A07B77A, 0xB22BB412, 0xEA5EB1AA, 0x2272B2C2, + 0xABC257DF, 0x63EE54B7, 0x3B9B510F, 0xF3B75267, + 0x29FA9A34, 0xE1D6995C, 0xB9A39CE4, 0x718F9F8C, + 0xF83F7A91, 0x301379F9, 0x68667C41, 0xA04A7F29, + 0xDCFDECE6, 0x14D1EF8E, 0x4CA4EA36, 0x8488E95E, + 0x0D380C43, 0xC5140F2B, 0x9D610A93, 0x554D09FB, + 0x8F00C1A8, 0x472CC2C0, 0x1F59C778, 0xD775C410, + 0x5EC5210D, 0x96E92265, 0xCE9C27DD, 0x06B024B5, + 0xC784EC47, 0x0FA8EF2F, 0x57DDEA97, 0x9FF1E9FF, + 0x16410CE2, 0xDE6D0F8A, 0x86180A32, 0x4E34095A, + 0x9479C109, 0x5C55C261, 0x0420C7D9, 0xCC0CC4B1, + 0x45BC21AC, 0x8D9022C4, 0xD5E5277C, 0x1DC92414, + 0x617EB7DB, 0xA952B4B3, 0xF127B10B, 0x390BB263, + 0xB0BB577E, 0x78975416, 0x20E251AE, 0xE8CE52C6, + 0x32839A95, 0xFAAF99FD, 0xA2DA9C45, 0x6AF69F2D, + 0xE3467A30, 0x2B6A7958, 0x731F7CE0, 0xBB337F88, + 0xF40E6EF5, 0x3C226D9D, 0x64576825, 0xAC7B6B4D, + 0x25CB8E50, 0xEDE78D38, 0xB5928880, 0x7DBE8BE8, + 0xA7F343BB, 0x6FDF40D3, 0x37AA456B, 0xFF864603, + 0x7636A31E, 0xBE1AA076, 0xE66FA5CE, 0x2E43A6A6, + 0x52F43569, 0x9AD83601, 0xC2AD33B9, 0x0A8130D1, + 0x8331D5CC, 0x4B1DD6A4, 0x1368D31C, 0xDB44D074, + 0x01091827, 0xC9251B4F, 0x91501EF7, 0x597C1D9F, + 0xD0CCF882, 0x18E0FBEA, 0x4095FE52, 0x88B9FD3A, + 0x498D35C8, 0x81A136A0, 0xD9D43318, 0x11F83070, + 0x9848D56D, 0x5064D605, 0x0811D3BD, 0xC03DD0D5, + 0x1A701886, 0xD25C1BEE, 0x8A291E56, 0x42051D3E, + 0xCBB5F823, 0x0399FB4B, 0x5BECFEF3, 0x93C0FD9B, + 0xEF776E54, 0x275B6D3C, 0x7F2E6884, 0xB7026BEC, + 0x3EB28EF1, 0xF69E8D99, 0xAEEB8821, 0x66C78B49, + 0xBC8A431A, 0x74A64072, 0x2CD345CA, 0xE4FF46A2, + 0x6D4FA3BF, 0xA563A0D7, 0xFD16A56F, 0x353AA607, + 0x8E09D98F, 0x4625DAE7, 0x1E50DF5F, 0xD67CDC37, + 0x5FCC392A, 0x97E03A42, 0xCF953FFA, 0x07B93C92, + 0xDDF4F4C1, 0x15D8F7A9, 0x4DADF211, 0x8581F179, + 0x0C311464, 0xC41D170C, 0x9C6812B4, 0x544411DC, + 0x28F38213, 0xE0DF817B, 0xB8AA84C3, 0x708687AB, + 0xF93662B6, 0x311A61DE, 0x696F6466, 0xA143670E, + 0x7B0EAF5D, 0xB322AC35, 0xEB57A98D, 0x237BAAE5, + 0xAACB4FF8, 0x62E74C90, 0x3A924928, 0xF2BE4A40, + 0x338A82B2, 0xFBA681DA, 0xA3D38462, 0x6BFF870A, + 0xE24F6217, 0x2A63617F, 0x721664C7, 0xBA3A67AF, + 0x6077AFFC, 0xA85BAC94, 0xF02EA92C, 0x3802AA44, + 0xB1B24F59, 0x799E4C31, 0x21EB4989, 0xE9C74AE1, + 0x9570D92E, 0x5D5CDA46, 0x0529DFFE, 0xCD05DC96, + 0x44B5398B, 0x8C993AE3, 0xD4EC3F5B, 0x1CC03C33, + 0xC68DF460, 0x0EA1F708, 0x56D4F2B0, 0x9EF8F1D8, + 0x174814C5, 0xDF6417AD, 0x87111215, 0x4F3D117D, + }, + { + 0x00000000, 0x277D3C49, 0x4EFA7892, 0x698744DB, + 0x6D821D21, 0x4AFF2168, 0x237865B3, 0x040559FA, + 0xDA043B42, 0xFD79070B, 0x94FE43D0, 0xB3837F99, + 0xB7862663, 0x90FB1A2A, 0xF97C5EF1, 0xDE0162B8, + 0xB4097684, 0x93744ACD, 0xFAF30E16, 0xDD8E325F, + 0xD98B6BA5, 0xFEF657EC, 0x97711337, 0xB00C2F7E, + 0x6E0D4DC6, 0x4970718F, 0x20F73554, 0x078A091D, + 0x038F50E7, 0x24F26CAE, 0x4D752875, 0x6A08143C, + 0x9965000D, 0xBE183C44, 0xD79F789F, 0xF0E244D6, + 0xF4E71D2C, 0xD39A2165, 0xBA1D65BE, 0x9D6059F7, + 0x43613B4F, 0x641C0706, 0x0D9B43DD, 0x2AE67F94, + 0x2EE3266E, 0x099E1A27, 0x60195EFC, 0x476462B5, + 0x2D6C7689, 0x0A114AC0, 0x63960E1B, 0x44EB3252, + 0x40EE6BA8, 0x679357E1, 0x0E14133A, 0x29692F73, + 0xF7684DCB, 0xD0157182, 0xB9923559, 0x9EEF0910, + 0x9AEA50EA, 0xBD976CA3, 0xD4102878, 0xF36D1431, + 0x32CB001A, 0x15B63C53, 0x7C317888, 0x5B4C44C1, + 0x5F491D3B, 0x78342172, 0x11B365A9, 0x36CE59E0, + 0xE8CF3B58, 0xCFB20711, 0xA63543CA, 0x81487F83, + 0x854D2679, 0xA2301A30, 0xCBB75EEB, 0xECCA62A2, + 0x86C2769E, 0xA1BF4AD7, 0xC8380E0C, 0xEF453245, + 0xEB406BBF, 0xCC3D57F6, 0xA5BA132D, 0x82C72F64, + 0x5CC64DDC, 0x7BBB7195, 0x123C354E, 0x35410907, + 0x314450FD, 0x16396CB4, 0x7FBE286F, 0x58C31426, + 0xABAE0017, 0x8CD33C5E, 0xE5547885, 0xC22944CC, + 0xC62C1D36, 0xE151217F, 0x88D665A4, 0xAFAB59ED, + 0x71AA3B55, 0x56D7071C, 0x3F5043C7, 0x182D7F8E, + 0x1C282674, 0x3B551A3D, 0x52D25EE6, 0x75AF62AF, + 0x1FA77693, 0x38DA4ADA, 0x515D0E01, 0x76203248, + 0x72256BB2, 0x555857FB, 0x3CDF1320, 0x1BA22F69, + 0xC5A34DD1, 0xE2DE7198, 0x8B593543, 0xAC24090A, + 0xA82150F0, 0x8F5C6CB9, 0xE6DB2862, 0xC1A6142B, + 0x64960134, 0x43EB3D7D, 0x2A6C79A6, 0x0D1145EF, + 0x09141C15, 0x2E69205C, 0x47EE6487, 0x609358CE, + 0xBE923A76, 0x99EF063F, 0xF06842E4, 0xD7157EAD, + 0xD3102757, 0xF46D1B1E, 0x9DEA5FC5, 0xBA97638C, + 0xD09F77B0, 0xF7E24BF9, 0x9E650F22, 0xB918336B, + 0xBD1D6A91, 0x9A6056D8, 0xF3E71203, 0xD49A2E4A, + 0x0A9B4CF2, 0x2DE670BB, 0x44613460, 0x631C0829, + 0x671951D3, 0x40646D9A, 0x29E32941, 0x0E9E1508, + 0xFDF30139, 0xDA8E3D70, 0xB30979AB, 0x947445E2, + 0x90711C18, 0xB70C2051, 0xDE8B648A, 0xF9F658C3, + 0x27F73A7B, 0x008A0632, 0x690D42E9, 0x4E707EA0, + 0x4A75275A, 0x6D081B13, 0x048F5FC8, 0x23F26381, + 0x49FA77BD, 0x6E874BF4, 0x07000F2F, 0x207D3366, + 0x24786A9C, 0x030556D5, 0x6A82120E, 0x4DFF2E47, + 0x93FE4CFF, 0xB48370B6, 0xDD04346D, 0xFA790824, + 0xFE7C51DE, 0xD9016D97, 0xB086294C, 0x97FB1505, + 0x565D012E, 0x71203D67, 0x18A779BC, 0x3FDA45F5, + 0x3BDF1C0F, 0x1CA22046, 0x7525649D, 0x525858D4, + 0x8C593A6C, 0xAB240625, 0xC2A342FE, 0xE5DE7EB7, + 0xE1DB274D, 0xC6A61B04, 0xAF215FDF, 0x885C6396, + 0xE25477AA, 0xC5294BE3, 0xACAE0F38, 0x8BD33371, + 0x8FD66A8B, 0xA8AB56C2, 0xC12C1219, 0xE6512E50, + 0x38504CE8, 0x1F2D70A1, 0x76AA347A, 0x51D70833, + 0x55D251C9, 0x72AF6D80, 0x1B28295B, 0x3C551512, + 0xCF380123, 0xE8453D6A, 0x81C279B1, 0xA6BF45F8, + 0xA2BA1C02, 0x85C7204B, 0xEC406490, 0xCB3D58D9, + 0x153C3A61, 0x32410628, 0x5BC642F3, 0x7CBB7EBA, + 0x78BE2740, 0x5FC31B09, 0x36445FD2, 0x1139639B, + 0x7B3177A7, 0x5C4C4BEE, 0x35CB0F35, 0x12B6337C, + 0x16B36A86, 0x31CE56CF, 0x58491214, 0x7F342E5D, + 0xA1354CE5, 0x864870AC, 0xEFCF3477, 0xC8B2083E, + 0xCCB751C4, 0xEBCA6D8D, 0x824D2956, 0xA530151F + } +#endif /* WORDS_BIGENDIAN */ +}; + +/* Accumulate one input byte */ +#ifdef WORDS_BIGENDIAN +#define CRC8(x) crc32c_table[0][((crc >> 24) ^ (x)) & 0xFF] ^ (crc << 8) +#else +#define CRC8(x) crc32c_table[0][(crc ^ (x)) & 0xFF] ^ (crc >> 8) +#endif + +uint32 +comp_crc32c_sb8(uint32 crc, const void *data, size_t len) +{ + const unsigned char *p = (const unsigned char *)data; + const uint32 *p4; + + /* + * Handle 0-3 initial bytes one at a time, so that the loop below starts + * with a pointer aligned to four bytes. + */ + while (len > 0 && ((uintptr_t) p & 3)) + { + crc = CRC8(*p++); + len--; + } + + /* + * Process eight bytes of data at a time. + */ + p4 = (const uint32 *) p; + while (len >= 8) + { + uint32 a = *p4++ ^ crc; + uint32 b = *p4++; + +#ifdef WORDS_BIGENDIAN + const uint8 c0 = b; + const uint8 c1 = b >> 8; + const uint8 c2 = b >> 16; + const uint8 c3 = b >> 24; + const uint8 c4 = a; + const uint8 c5 = a >> 8; + const uint8 c6 = a >> 16; + const uint8 c7 = a >> 24; +#else + const uint8 c0 = b >> 24; + const uint8 c1 = b >> 16; + const uint8 c2 = b >> 8; + const uint8 c3 = b; + const uint8 c4 = a >> 24; + const uint8 c5 = a >> 16; + const uint8 c6 = a >> 8; + const uint8 c7 = a; +#endif + + crc = + crc32c_table[0][c0] ^ crc32c_table[1][c1] ^ + crc32c_table[2][c2] ^ crc32c_table[3][c3] ^ + crc32c_table[4][c4] ^ crc32c_table[5][c5] ^ + crc32c_table[6][c6] ^ crc32c_table[7][c7]; + + len -= 8; + } + + /* + * Handle any remaining bytes one at a time. + */ + p = (const unsigned char *) p4; + while (len > 0) + { + crc = CRC8(*p++); + len--; + } + + return crc; +} + +/* + * error handle + */ +#ifdef __cplusplus +extern "C" { +#endif + +static thread_local ICError ic_error = {LEVEL_ERROR}; + +ICError* +GetLastError() +{ + return &ic_error; +} + +void +ResetLastError() +{ + ic_error.level = LEVEL_OK; +} + +void +SetLastError(ErrorLevel level, const char *msg) +{ + ic_error.level = level; + strncpy(ic_error.msg, msg, sizeof(ic_error.msg) - 1); + ic_error.msg[sizeof(ic_error.msg) - 1] = 0; +} + +#ifdef __cplusplus +} // extern "C" +#endif \ No newline at end of file diff --git a/contrib/udp2/ic_common/ic_utility.hpp b/contrib/udp2/ic_common/ic_utility.hpp new file mode 100644 index 00000000000..169c7b3cc18 --- /dev/null +++ b/contrib/udp2/ic_common/ic_utility.hpp @@ -0,0 +1,165 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_utility.hpp + * + * IDENTIFICATION + * contrib/udp2/ic_common/ic_utility.hpp + * + *------------------------------------------------------------------------- + */ +#ifndef IC_UTILITY_HPP +#define IC_UTILITY_HPP + +#include +#include +#include +#include + +#include +#include +#include + +/* Define this if you want tons of logs! */ +#undef AMS_VERBOSE_LOGGING + +#ifndef NDEBUG +#define USE_ASSERT_CHECKING 1 +#endif + +#define Assert(p) assert(p) +#define closesocket close + +#define IC_DEBUG5 10 +#define IC_DEBUG4 11 +#define IC_DEBUG3 12 +#define IC_DEBUG2 13 +#define IC_DEBUG1 14 +#define IC_LOG 15 +#define IC_WARNING 19 + +/* tupchunk.h */ +#define ANY_ROUTE -100 + +/* cdbvars.h */ +#define DEFAULT_PACKET_SIZE 8192 +#define MIN_PACKET_SIZE 512 +#define MAX_PACKET_SIZE 65507 /* Max payload for IPv4/UDP (subtract 20 more for IPv6 without extensions) */ +#define UNDEF_SEGMENT -2 + +/* c.h */ +#define Max(x, y) ((x) > (y) ? (x) : (y)) +#define Min(x, y) ((x) < (y) ? (x) : (y)) + +/* cdbinterconnect.h */ +#define CTS_INITIAL_SIZE (10) + +/* pg_list.h */ +#define NIL ((List *) NULL) + +/* transam.h */ +#define InvalidTransactionId (0) + +#define INVALID_SOCKET (-1) + +/* + * CONTAINER_OF + */ +#define CONTAINER_OF(ptr, type, member) \ + ( \ + reinterpret_cast(reinterpret_cast(ptr) - offsetof(type, member)) \ + ) + + +typedef enum GpVars_Verbosity_IC +{ + GPVARS_VERBOSITY_UNDEFINED_IC = 0, + GPVARS_VERBOSITY_OFF_IC, + GPVARS_VERBOSITY_TERSE_IC, + GPVARS_VERBOSITY_VERBOSE_IC, + GPVARS_VERBOSITY_DEBUG_IC, +} GpVars_Verbosity_IC; + +typedef enum GpVars_Interconnect_Method_IC +{ + INTERCONNECT_FC_METHOD_CAPACITY_IC = 0, + INTERCONNECT_FC_METHOD_LOSS_IC = 2, +} GpVars_Interconnect_Method_IC; + +typedef enum +{ + GP_ROLE_UNDEFINED_IC = 0, /* Should never see this role in use */ + GP_ROLE_UTILITY_IC, /* Operating as a simple database engine */ + GP_ROLE_DISPATCH_IC, /* Operating as the parallel query dispatcher */ + GP_ROLE_EXECUTE_IC, /* Operating as a parallel query executor */ +} GpRoleValue_IC; + +typedef enum GpVars_Interconnect_Address_Type_IC +{ + INTERCONNECT_ADDRESS_TYPE_UNICAST_IC = 0, + INTERCONNECT_ADDRESS_TYPE_WILDCARD_IC +} GpVars_Interconnect_Address_Type_IC; + + +/* + * global_param and session_param; + */ +extern GlobalMotionLayerIPCParam global_param; +extern SessionMotionLayerIPCParam session_param; + +/* + * logger stuff + */ +#define DEFAULT_LOG_LEVEL INFO +extern const char * SeverityName[9]; + +enum LogSeverity { + FATAL, LOG_ERROR, WARNING, INFO, DEBUG1, DEBUG2, DEBUG3, DEBUG4, DEBUG5 +}; + +class Logger; + +class Logger { +public: + Logger(); + + ~Logger(); + + void setOutputFd(int f); + + void setLogSeverity(LogSeverity l); + + void printf(LogSeverity s, const char * fmt, ...) __attribute__((format(printf, 3, 4))); + +private: + int fd; + std::atomic severity; +}; + +extern Logger RootLogger; + +#define LOG(s, fmt, ...) \ + RootLogger.printf(s, fmt, ##__VA_ARGS__) + +/* + * crc32 + */ +extern uint32 ComputeCRC(const void *data, size_t len); + +#endif // IC_UTILITY_HPP \ No newline at end of file diff --git a/contrib/udp2/ic_common/udp2/ic_udp2.cpp b/contrib/udp2/ic_common/udp2/ic_udp2.cpp new file mode 100644 index 00000000000..c09ea4e8434 --- /dev/null +++ b/contrib/udp2/ic_common/udp2/ic_udp2.cpp @@ -0,0 +1,6939 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_udp2.cpp + * + * IDENTIFICATION + * contrib/udp2/ic_common/udp2/ic_udp2.cpp + * + * +--------------+ + * | ic_types.h | + * +--------------+ + * / \ + * +--------------+ +---------------+ + * | C interface | | C++ interface | + * | ic_udp2.h | | ic_udp2.hpp | + * +--------------+ +---------------+ + * \ / + * +----------------------+ + * | C++ implement | + * | ic_udp2_internal.hpp| + * | ic_faultinjection.h | + * | ic_udp2.cpp | + * +----------------------+ + *------------------------------------------------------------------------- + */ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +/* + * interface header files + */ +#ifdef __cplusplus +extern "C" { +#endif + +#include "ic_types.h" +#include "ic_udp2.h" + +#ifdef __cplusplus +} +#endif + +#include "ic_udp2.hpp" + +/* + * internal header files + */ +#include "ic_utility.hpp" +#include "ic_udp2_internal.hpp" +#include "ic_faultinjection.h" + +static int timeoutArray[] = +{ + 1, + 1, + 2, + 4, + 8, + 16, + 32, + 64, + 128, + 256, + 512, + 512 /* MAX_TRY */ +}; + +/* + * Main thread (Receiver) and background thread use the information in + * this data structure to handle data packets. + */ +static ReceiveControlInfo rx_control_info; + +/* + * The buffer pool used for keeping data packets. + * + * maxCount is set to 1 to make sure there is always a buffer + * for picking packets from OS buffer. + */ +static RxBufferPool rx_buffer_pool = {1, 0, NULL}; + +/* + * The sender side buffer pool. + */ +static SendBufferPool snd_buffer_pool; + +/* + * Main thread use the information in this data structure to do ack handling + * and congestion control. + */ +static SendControlInfo snd_control_info; + +/* + * Shared control information that is used by senders, receivers and background thread. + */ +static ICGlobalControlInfo ic_control_info; + +/* + * All connections in a process share this unack queue ring instance. + */ +static UnackQueueRing unack_queue_ring = {0, 0, 0}; + +static int ICSenderSocket = -1; +static int32 ICSenderPort = 0; +static int ICSenderFamily = 0; + +/* Statistics for UDP interconnect. */ +static ICStatistics ic_statistics; + +/* Cached sockaddr of the listening udp socket */ +static struct sockaddr_storage udp_dummy_packet_sockaddr; + +/* UDP listen fd */ +static int UDP_listenerFd = -1; + +/* UDP listen port */ +static int32 udp_listener_port = 0; + +static std::mutex mtx; +static std::condition_variable cv; + +CChunkTransportState *CChunkTransportStateImpl::state_ = nullptr; + +/* + * Identity the user of ic module by vector_engine_is_user: + * "false" means PG executor, "true" means Arrow executor. + */ +static thread_local bool vector_engine_is_user = false; +static thread_local bool thread_quit = false; + +#define CHECK_QUIT_FLAG() \ + do { \ + if (thread_quit) { \ + throw ICException("received thread quit flag.", __FILE__, __LINE__); \ + } \ + } while(0) + +#define CHECK_INTERRUPTS(state) \ + do { \ + if (vector_engine_is_user) { \ + CHECK_QUIT_FLAG(); \ + } else if (global_param.checkInterruptsCallback) { \ + global_param.checkInterruptsCallback((state)->teardownActive); \ + } \ + } while(0) + +#define CHECK_CANCEL(state) \ + do { \ + if (vector_engine_is_user) { \ + CHECK_QUIT_FLAG(); \ + } else if (global_param.checkCancelOnQDCallback) { \ + global_param.checkCancelOnQDCallback(state); \ + } \ + } while(0) + +#define CHECK_POSTMASTER_ALIVE() \ + do { \ + if (vector_engine_is_user) { \ + CHECK_QUIT_FLAG(); \ + } else if (global_param.checkPostmasterIsAliveCallback && !global_param.checkPostmasterIsAliveCallback()) { \ + throw ICFatalException("FATAL, interconnect failed to send chunks, Postmaster is not alive.", __FILE__, __LINE__); \ + } \ + } while(0) + +/*========================================================================= + * STATIC FUNCTIONS declarations + */ + +/* Background thread error handling functions. */ +static void checkRxThreadError(void); +static void setRxThreadError(int eno); +static void resetRxThreadError(void); + +static uint32 setUDPSocketBufferSize(int ic_socket, int buffer_type); +static void setupOutgoingUDPConnection(int icid, TransportEntry *pChunkEntry, UDPConn *conn); + +/* ICBufferList functions. */ +static inline void icBufferListInitHeadLink(ICBufferLink *link); + +static inline void InitMotionUDPIFC(int *listenerSocketFd, int32 *listenerPort); +static inline void CleanupMotionUDPIFC(void); + +static bool dispatcherAYT(void); +static void checkQDConnectionAlive(void); + +static void *rxThreadFunc(void *arg); + +static void putIntoUnackQueueRing(UnackQueueRing *uqr, ICBuffer *buf, uint64 expTime, uint64 now); + +static bool cacheFuturePacket(icpkthdr *pkt, struct sockaddr_storage *peer, int peer_len); +static void cleanupStartupCache(void); +static void handleCachedPackets(void); + +static uint64 getCurrentTime(void); +static void initMutex(pthread_mutex_t *mutex); + +static inline void logPkt(const char *prefix, icpkthdr *pkt); + +static void ConvertToIPv4MappedAddr(struct sockaddr_storage *sockaddr, socklen_t *o_len); +#if defined(__darwin__) +#define s6_addr32 __u6_addr.__u6_addr32 +static void ConvertIPv6WildcardToLoopback(struct sockaddr_storage* dest); +#endif + +static void setupUDPListeningSocket(int *listenerSocketFd, int32 *listenerPort, + int *txFamily, struct sockaddr_storage *listenerSockaddr); +static void getSockAddr(struct sockaddr_storage *peer, socklen_t *peer_len, const char *listenerAddr, int listenerPort); +static void SendDummyPacket(void); +static bool handleDataPacket(UDPConn *conn, icpkthdr *pkt, struct sockaddr_storage *peer, socklen_t *peerlen, AckSendParam *param, bool *wakeup_mainthread); +static bool handleMismatch(icpkthdr *pkt, struct sockaddr_storage *peer, int peer_len); +static void initUnackQueueRing(UnackQueueRing *uqr); + +static ssize_t sendtoWithRetry(int socket, const void *message, size_t length, int flags, const struct sockaddr *dest_addr, socklen_t dest_len, int retry, const char *errDetail); + +static char *format_sockaddr_udp(struct sockaddr_storage *sa, char *buf, size_t len); + +static char* flags2txt(uint32 pkt_flags); + +static const char* flags_text[] = + {"recv2send", "ack", "stop", "eos", "nak", "disorder", "duplicate", "capacity"}; + +static char* +flags2txt(uint32 pkt_flags) +{ + thread_local static char flags[64]; + + char *p = flags; + *p = '\0'; + int bytes = 0; + for (size_t i = 0; i < sizeof(flags_text)/sizeof(flags_text[0]); ++i) { + if (pkt_flags & (1 << i)) + bytes += snprintf(p + bytes, 64, "%s | ", flags_text[i]); + } + + if (bytes > 0) + bytes -= 3; + *(p + bytes) = '\0'; + + return flags; +} + +/* + * CursorICHistoryTable::prune + * Prune entries in the hash table. + */ +void +CursorICHistoryTable::prune(uint32 icId) { + for (uint32 index = 0; index < size; index++) { + CursorICHistoryEntry *p = table[index], *q = NULL; + while (p) { + /* remove an entry if it is older than the prune-point */ + if (p->icId < icId) { + if (!q) + table[index] = p->next; + else + q->next = p->next; + + /* set up next loop */ + CursorICHistoryEntry *trash = p; + p = trash->next; + ic_free(trash); + + count--; + } else { + q = p; + p = p->next; + } + } + } +} + +#ifdef TRANSFER_PROTOCOL_STATS +typedef enum TransProtoEvent +{ + TPE_DATA_PKT_SEND, + TPE_ACK_PKT_QUERY +} TransProtoEvent; + +typedef struct TransProtoStatEntry TransProtoStatEntry; +struct TransProtoStatEntry +{ + TransProtoStatEntry *next; + + /* Basic information */ + uint32 time; + TransProtoEvent event; + int dstPid; + uint32 seq; + + /* more attributes can be added on demand. */ + + /* + * float cwnd; + * int capacity; + */ +}; + +typedef struct TransProtoStats TransProtoStats; +struct TransProtoStats +{ + std::mutex lock; + TransProtoStatEntry *head; + TransProtoStatEntry *tail; + uint64 count; + uint64 startTime; + + void init(); + void update(TransProtoEvent event, icpkthdr *pkt); + void dump(); +}; + +static TransProtoStats trans_proto_stats = +{ + {}, NULL, NULL, 0 +}; + +/* + * init + * Initialize the transport protocol states data structures. + */ +void +TransProtoStats::init() +{ + std::lock_guard guard(this->lock); + + while (this->head) { + TransProtoStatEntry *cur = this->head; + this->head = this->head->next; + ic_free(cur); + this->count--; + } + + this->head = NULL; + this->tail = NULL; + this->count = 0; + this->startTime = getCurrentTime(); +} + +void +TransProtoStats::update(TransProtoEvent event, icpkthdr *pkt) +{ + /* Add to list */ + TransProtoStatEntry *entry = (TransProtoStatEntry *) ic_malloc(sizeof(TransProtoStatEntry)); + if (!entry) + return; + + memset(entry, 0, sizeof(*entry)); + + /* change the list */ + std::lock_guard guard(this->lock); + if (this->count == 0) { + /* 1st element */ + this->head = entry; + this->tail = entry; + } else { + this->tail->next = entry; + this->tail = entry; + } + this->count++; + + entry->time = getCurrentTime() - this->startTime; + entry->event = event; + entry->dstPid = pkt->dstPid; + entry->seq = pkt->seq; + + /* + * Other attributes can be added on demand new->cwnd = + * snd_control_info.cwnd; new->capacity = conn->capacity; + */ +} + +void +TransProtoStats::dump() +{ + char tmpbuf[32]; + + snprintf(tmpbuf, 32, "%d.%lu.txt", global_param.MyProcPid, getCurrentTime()); + FILE *ofile = fopen(tmpbuf, "w+"); + + std::lock_guard guard(this->lock); + while (this->head) + { + TransProtoStatEntry *cur = NULL; + + cur = this->head; + this->head = this->head->next; + + fprintf(ofile, "time %d event %d seq %d destpid %d\n", cur->time, cur->event, cur->seq, cur->dstPid); + ic_free(cur); + this->count--; + } + + this->tail = NULL; + fclose(ofile); +} + +#endif /* TRANSFER_PROTOCOL_STATS */ + +/* + * initConnHashTable + * Initialize a connection hash table. + */ +bool +ConnHashTable::init() +{ + this->size = global_param.Gp_role == GP_ROLE_DISPATCH_IC ? + (global_param.segment_number * 2) : global_param.ic_htab_size; + Assert(this->size > 0); + + this->table = (struct ConnHtabBin **) ic_malloc(this->size * sizeof(struct ConnHtabBin *)); + if (this->table == NULL) + return false; + + for (int i = 0; i < this->size; i++) + this->table[i] = NULL; + + return true; +} + +/* + * connAddHash + * Add a connection to the hash table + * + * Note: we want to add a connection to the hashtable if it isn't + * already there ... so we just have to check the pointer values -- no + * need to use CONN_HASH_MATCH() at all! + */ +bool +ConnHashTable::add(UDPConn *conn) +{ + uint32 hashcode = CONN_HASH_VALUE(&conn->conn_info) % this->size; + + /* + * check for collision -- if we already have an entry for this connection, + * don't add another one. + */ + for (struct ConnHtabBin *bin = this->table[hashcode]; bin != NULL; bin = bin->next) + { + if (bin->conn == conn) + { + LOG(DEBUG5, "ConnHashTable::add(): duplicate ?! node %d route %d", conn->conn_info.motNodeId, conn->route); + return true; /* false *only* indicates memory-alloc + * failure. */ + } + } + + struct ConnHtabBin *newbin = (struct ConnHtabBin *) ic_malloc(sizeof(struct ConnHtabBin)); + if (newbin == NULL) + return false; + + newbin->conn = conn; + newbin->next = this->table[hashcode]; + this->table[hashcode] = newbin; + + ic_statistics.activeConnectionsNum++; + + return true; +} + +/* + * remove + * Delete a connection from the hash table + * + * Note: we want to remove a connection from the hashtable if it is + * there ... so we just have to check the pointer values -- no need to + * use CONN_HASH_MATCH() at all! + */ +void +ConnHashTable::remove(UDPConn *conn) +{ + uint32 hashcode; + struct ConnHtabBin *c, + *p, + *trash; + + hashcode = CONN_HASH_VALUE(&conn->conn_info) % this->size; + + c = this->table[hashcode]; + + /* find entry */ + p = NULL; + while (c != NULL) + { + /* found ? */ + if (c->conn == conn) + break; + + p = c; + c = c->next; + } + + /* not found ? */ + if (c == NULL) + { + return; + } + + /* found the connection, remove from the chain. */ + trash = c; + + if (p == NULL) + this->table[hashcode] = c->next; + else + p->next = c->next; + + ic_free(trash); + + ic_statistics.activeConnectionsNum--; + + return; +} + +/* + * findConnByHeader + * Find the corresponding connection given a pkt header information. + * + * With the new mirroring scheme, the interconnect is no longer involved: + * we don't have to disambiguate anymore. + * + * NOTE: the icpkthdr field dstListenerPort is used for disambiguation. + * on receivers it may not match the actual port (it may have an extra bit + * set (1<<31)). + */ +UDPConn * +ConnHashTable::find(icpkthdr *hdr) { + + uint32 hashcode = CONN_HASH_VALUE(hdr) % this->size; + for (struct ConnHtabBin *bin = this->table[hashcode]; bin != NULL; bin = bin->next) { + UDPConn *conn = bin->conn; + + if (CONN_HASH_MATCH(&conn->conn_info, hdr)) { + UDPConn *ret = conn; + if (IC_DEBUG5 >= session_param.log_min_messages) + LOG(DEBUG5, "ConnHashTable::find: found. route %d state %d hashcode %d conn %p", + conn->route, ret->state, hashcode, ret); + + return ret; + } + } + + if (IC_DEBUG5 >= session_param.log_min_messages) + LOG(DEBUG5, "ConnHashTable::find: not found! (hdr->srcPid %d hdr->srcContentId %d " + "hdr->dstContentId %d hdr->dstPid %d sess(%d:%d) cmd(%d:%d)) hashcode %d", + hdr->srcPid, hdr->srcContentId, hdr->dstContentId, hdr->dstPid, hdr->sessionId, + session_param.gp_session_id, hdr->icId, ic_control_info.ic_instance_id, hashcode); + + return NULL; +} + +void +ConnHashTable::destroy() { + for (int i = 0; i < this->size; i++) { + while (this->table[i] != NULL) { + struct ConnHtabBin *trash = this->table[i]; + this->table[i] = trash->next; + ic_free(trash); + } + } + + ic_free(this->table); + this->table = NULL; + this->size = 0; +} + +/* + * icBufferListInitHeadLink + * Initialize the pointers in the head link to point to itself. + */ +static inline void +icBufferListInitHeadLink(ICBufferLink *link) +{ + link->next = link->prev = link; +} + + +#if defined(USE_ASSERT_CHECKING) || defined(AMS_VERBOSE_LOGGING) + +/* + * icBufferListLog + * Log the buffer list. + */ +void +ICBufferList::icBufferListLog() +{ + LOG(INFO, "Length %d, type %d headptr %p", this->len, this->type, &this->head); + + ICBufferLink *bufLink = this->head.next; + + int len = this->len; + int i = 0; + + while (bufLink != &this->head && len > 0) + { + ICBuffer *buf = (this->type == ICBufferListType_Primary ? GET_ICBUFFER_FROM_PRIMARY(bufLink) + : GET_ICBUFFER_FROM_SECONDARY(bufLink)); + + LOG(INFO, "Node %d, linkptr %p", i++, bufLink); + + logPkt("from list", buf->pkt); + bufLink = bufLink->next; + len--; + } +} +#endif + +#ifdef USE_ASSERT_CHECKING +/* + * icBufferListCheck + * Buffer list sanity check. + */ +void +ICBufferList::icBufferListCheck(const char *prefix) +{ + int len = this->len; + ICBufferLink *link = this->head.next; + + if (len < 0) + { + LOG(LOG_ERROR, "ICBufferList ERROR %s: list length %d < 0 ", prefix, this->length()); + goto error; + } + + if (len == 0 && (this->head.prev != this->head.next && this->head.prev != &this->head)) + { + LOG(LOG_ERROR, "ICBufferList ERROR %s: length is 0, &list->head %p, prev %p, next %p", + prefix, &this->head, this->head.prev, this->head.next); + this->icBufferListLog(); + goto error; + } + + while (len > 0) + { + link = link->next; + len--; + } + + if (link != &this->head) + { + LOG(LOG_ERROR, "ICBufferList ERROR: %s len %d", prefix, this->len); + this->icBufferListLog(); + goto error; + } + + return; + +error: + LOG(INFO, "wait for 120s and then abort."); + ic_usleep(120000000); + abort(); +} +#endif + +/* + * ICBufferList::init + * Initialize the buffer list with the given type. + */ +void +ICBufferList::init(ICBufferListType atype) +{ + Assert(atype == ICBufferListType_Primary|| atype == ICBufferListType_Secondary); + + type = atype; + len = 0; + + icBufferListInitHeadLink(&head); + +#ifdef USE_ASSERT_CHECKING + this->icBufferListCheck("ICBufferList::init"); +#endif +} + +/* + * ICBufferList::is_head + * Return whether the given link is the head link of the list. + * + * This function is often used as the end condition of an iteration of the list. + */ +bool +ICBufferList::is_head(ICBufferLink *link) +{ +#ifdef USE_ASSERT_CHECKING + this->icBufferListCheck("ICBufferList::is_head"); +#endif + return (link == &head); +} + +/* + * ICBufferList::first + * Return the first link after the head link. + * + * Note that the head link is a pseudo link used to only to ease the operations of the link list. + * If the list only contains the head link, this function will return the head link. + */ +ICBufferLink * +ICBufferList::first() +{ +#ifdef USE_ASSERT_CHECKING + this->icBufferListCheck("ICBufferList::first"); +#endif + return head.next; +} + +/* + * ICBufferList::length + * Get the list length. + */ +int +ICBufferList::length() +{ + return len; +} + +/* + * ICBufferList::delete + * Remove an buffer from the buffer list and return the buffer. + */ +ICBuffer * +ICBufferList::remove(ICBuffer *buf) +{ +#ifdef USE_ASSERT_CHECKING + this->icBufferListCheck("ICBufferList::delete"); +#endif + + ICBufferLink *bufLink = NULL; + + bufLink = (this->type == ICBufferListType_Primary ? &buf->primary : &buf->secondary); + + bufLink->prev->next = bufLink->next; + bufLink->next->prev = bufLink->prev; + + len--; + + return buf; +} + +/* + * ICBufferList::pop + * Remove the head buffer from the list. + */ +ICBuffer * +ICBufferList::pop() +{ + ICBuffer *buf = NULL; + ICBufferLink *bufLink = NULL; + +#ifdef USE_ASSERT_CHECKING + this->icBufferListCheck("ICBufferList::pop"); +#endif + + if (this->len == 0) + return NULL; + + bufLink = this->first(); + buf = (this->type == ICBufferListType_Primary ? GET_ICBUFFER_FROM_PRIMARY(bufLink) + : GET_ICBUFFER_FROM_SECONDARY(bufLink)); + + bufLink->prev->next = bufLink->next; + bufLink->next->prev = bufLink->prev; + + this->len--; + + return buf; +} + +/* + * ICBufferList::free + * Free all the buffers in the list. + */ +void +ICBufferList::destroy() +{ + ICBuffer *buf = NULL; + +#ifdef USE_ASSERT_CHECKING + this->icBufferListCheck("ICBufferList::free"); +#endif + + while ((buf = this->pop()) != NULL) + ic_free(buf); +} + +/* + * ICBufferList::append + * Append a buffer to the tail of a double-link list. + */ +ICBuffer * +ICBufferList::append(ICBuffer *buf) +{ + Assert(buf); + +#ifdef USE_ASSERT_CHECKING + this->icBufferListCheck("ICBufferList::append"); +#endif + + ICBufferLink *bufLink = NULL; + + bufLink = (this->type == ICBufferListType_Primary ? &buf->primary : &buf->secondary); + + bufLink->prev = this->head.prev; + bufLink->next = &this->head; + + this->head.prev->next = bufLink; + this->head.prev = bufLink; + + this->len++; + + return buf; +} + +/* + * ICBufferList::return + * Return the buffers in the list to the free buffer list. + * + * If the buf is also in an expiration queue, we also need to remove it from the expiration queue. + * + */ +void +ICBufferList::release(bool inExpirationQueue) +{ +#ifdef USE_ASSERT_CHECKING + this->icBufferListCheck("ICBufferList::return"); +#endif + ICBuffer *buf = NULL; + + while ((buf = this->pop()) != NULL) + { + if (inExpirationQueue) /* the buf is in also in the expiration queue */ + { + ICBufferList *alist = &unack_queue_ring.slots[buf->unackQueueRingSlot]; + buf = alist->remove(buf); + unack_queue_ring.numOutStanding--; + if (this->length() >= 1) + unack_queue_ring.numSharedOutStanding--; + } + + snd_buffer_pool.freeList.append(buf); + } +} + +#ifdef USE_ASSERT_CHECKING +/* + * ICBufferList::dump_to_file + * Dump a buffer list. + */ +void +ICBufferList::dump_to_file(FILE *ofile) +{ + this->icBufferListCheck("ICBufferList::dump_to_file"); + + ICBufferLink *bufLink = this->head.next; + + int len = this->len; + int i = 0; + + fprintf(ofile, "List Length %d\n", len); + while (bufLink != &this->head && len > 0) + { + ICBuffer *buf = (this->type == ICBufferListType_Primary ? GET_ICBUFFER_FROM_PRIMARY(bufLink) + : GET_ICBUFFER_FROM_SECONDARY(bufLink)); + + fprintf(ofile, "Node %d, linkptr %p ", i++, bufLink); + fprintf(ofile, "Packet Content [%s: seq %d extraSeq %d]: motNodeId %d, crc %d len %d " + "srcContentId %d dstDesContentId %d " + "srcPid %d dstPid %d " + "srcListenerPort %d dstListernerPort %d " + "sendSliceIndex %d recvSliceIndex %d " + "sessionId %d icId %d " + "flags %d\n", + buf->pkt->flags & UDPIC_FLAGS_RECEIVER_TO_SENDER ? "ACK" : "DATA", + buf->pkt->seq, buf->pkt->extraSeq, buf->pkt->motNodeId, buf->pkt->crc, buf->pkt->len, + buf->pkt->srcContentId, buf->pkt->dstContentId, + buf->pkt->srcPid, buf->pkt->dstPid, + buf->pkt->srcListenerPort, buf->pkt->dstListenerPort, + buf->pkt->sendSliceIndex, buf->pkt->recvSliceIndex, + buf->pkt->sessionId, buf->pkt->icId, + buf->pkt->flags); + bufLink = bufLink->next; + len--; + } +} +#endif + +/* + * initUnackQueueRing + * Initialize an unack queue ring. + * + * Align current time to a slot boundary and set current slot index (time pointer) to 0. + */ +static void +initUnackQueueRing(UnackQueueRing *uqr) +{ + int i = 0; + + uqr->currentTime = 0; + uqr->idx = 0; + uqr->numOutStanding = 0; + uqr->numSharedOutStanding = 0; + + for (; i < UNACK_QUEUE_RING_SLOTS_NUM; i++) + { + uqr->slots[i].init(ICBufferListType_Secondary); + } +} + +/* + * RxBufferPool::get + * Get a receive buffer. + * + * SHOULD BE CALLED WITH ic_control_info.lock *LOCKED* + * + * NOTE: This function MUST NOT contain elog or ereport statements. + * elog is NOT thread-safe. Developers should instead use something like: + * + * NOTE: In threads, we cannot use palloc/pfree, because it's not thread safe. + */ +icpkthdr * +RxBufferPool::get() +{ + icpkthdr *ret = NULL; + +#ifdef USE_ASSERT_CHECKING + if (FINC_HAS_FAULT(FINC_RX_BUF_NULL) && + testmode_inject_fault(session_param.gp_udpic_fault_inject_percent)) + return NULL; +#endif + + do + { + if (this->freeList == NULL) + { + if (this->count > this->maxCount) + { + if (IC_DEBUG3 >= session_param.log_min_messages) + LOG(DEBUG3, "Interconnect ran out of rx-buffers count/max %d/%d", this->count, this->maxCount); + break; + } + + /* malloc is used for thread safty. */ + ret = (icpkthdr *) ic_malloc(global_param.Gp_max_packet_size); + + /* + * Note: we return NULL if the malloc() fails -- and the + * background thread will set the error. Main thread will check + * the error, report it and start teardown. + */ + if (ret != NULL) + this->count++; + + break; + } + + /* we have buffers available in our freelist */ + ret = this->get_free(); + + } while (0); + + return ret; +} + +/* + * RxBufferPool::put + * Return a receive buffer to free list + * + * SHOULD BE CALLED WITH ic_control_info.lock *LOCKED* + */ +void +RxBufferPool::put(icpkthdr *buf) +{ + /* return the buffer into the free list. */ + *(char **) buf = this->freeList; + this->freeList = (char *)buf; +} + +/* + * RxBufferPool::get_free + * Get a receive buffer from free list + * + * SHOULD BE CALLED WITH ic_control_info.lock *LOCKED* + * + * NOTE: This function MUST NOT contain elog or ereport statements. + * elog is NOT thread-safe. Developers should instead use something like: + * + * NOTE: In threads, we cannot use palloc/pfree, because it's not thread safe. + */ +icpkthdr * +RxBufferPool::get_free() +{ + icpkthdr *buf = NULL; + + buf = (icpkthdr *) this->freeList; + this->freeList = *(char **) (this->freeList); + return buf; +} + +/* + * RxBufferPool::free + * Free a receive buffer. + * + * NOTE: This function MUST NOT contain elog or ereport statements. + * elog is NOT thread-safe. Developers should instead use something like: + * + * NOTE: In threads, we cannot use palloc/pfree, because it's not thread safe. + */ +void +RxBufferPool::release(icpkthdr *buf) +{ + ic_free(buf); + count--; +} + +/* + * init + * Initialize the send buffer pool. + * + * The initial maxCount is set to 1 for gp_interconnect_snd_queue_depth = 1 case, + * then there is at least an extra free buffer to send for that case. + */ +void +SendBufferPool::init() +{ + this->freeList.init(ICBufferListType_Primary); + this->count = 0; + this->maxCount = (session_param.Gp_interconnect_snd_queue_depth == 1 ? 1 : 0); +} + +/* + * clean + * Clean the send buffer pool. + */ +void +SendBufferPool::clean() +{ + this->freeList.destroy(); + this->count = 0; + this->maxCount = 0; +} + +/* + * get + * Get a send buffer for a connection. + * + * Different flow control mechanisms use different buffer management policies. + * Capacity based flow control uses per-connection buffer policy and Loss based + * flow control uses shared buffer policy. + * + * Return NULL when no free buffer available. + */ +ICBuffer * +SendBufferPool::get(UDPConn *conn) +{ + ICBuffer *ret = NULL; + + ic_statistics.totalBuffers += (this->freeList.length() + this->maxCount - this->count); + ic_statistics.bufferCountingTime++; + + /* Capacity based flow control does not use shared buffers */ + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_CAPACITY_IC) + { + Assert(conn->unackQueue.length() + conn->sndQueue.length() <= session_param.Gp_interconnect_snd_queue_depth); + if (conn->unackQueue.length() + conn->sndQueue.length() >= session_param.Gp_interconnect_snd_queue_depth) + return NULL; + } + + if (this->freeList.length() > 0) + { + return this->freeList.pop(); + } + else + { + if (this->count < this->maxCount) + { + ret = (ICBuffer *) ic_malloc0(global_param.Gp_max_packet_size + sizeof(ICBuffer)); + this->count++; + ret->conn = NULL; + ret->nRetry = 0; + icBufferListInitHeadLink(&ret->primary); + icBufferListInitHeadLink(&ret->secondary); + ret->unackQueueRingSlot = 0; + } + else + { + return NULL; + } + } + + return ret; +} + +/* + * addCRC + * add CRC field to the packet. + */ +static void +addCRC(icpkthdr *pkt) +{ + pkt->crc = ComputeCRC(pkt, pkt->len); +} + +/* + * checkCRC + * check the validity of the packet. + */ +static bool +checkCRC(icpkthdr *pkt) +{ + uint32 rx_crc, + local_crc; + + rx_crc = pkt->crc; + pkt->crc = 0; + local_crc = ComputeCRC(pkt, pkt->len); + if (rx_crc != local_crc) + { + return false; + } + + return true; +} + + +/* + * checkRxThreadError + * Check whether there was error in the background thread in main thread. + * + * If error found, report it. + */ +static void +checkRxThreadError() +{ + int eno; + + eno = ic_atomic_read_u32(&ic_control_info.eno); + if (eno != 0) + { + errno = eno; + + std::stringstream ss; + ss <<"ERROR, interconnect encountered an error, in receive background thread: "<= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG3, "UDP-IC: setsockopt %s failed to set buffer size = %d bytes: %m", + buffer_type == SO_SNDBUF ? "send": "receive", + curr_size); + + curr_size = curr_size >> 1; + if (curr_size < UDPIC_MIN_BUF_SIZE) + return -1; + } + + if(session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG3, "UDP-IC: socket %s current buffer size = %d bytes", + buffer_type == SO_SNDBUF ? "send": "receive", + curr_size); + + return curr_size; +} + + +/* + * setupUDPListeningSocket + * Setup udp listening socket. + */ +static void +setupUDPListeningSocket(int *listenerSocketFd, int32 *listenerPort, int *txFamily, struct sockaddr_storage *listenerSockaddr) +{ + struct addrinfo *addrs = NULL; + struct addrinfo *addr; + struct addrinfo hints; + int ret; + int ic_socket = INVALID_SOCKET; + struct sockaddr_storage ic_socket_addr; + int tries = 0; + struct sockaddr_storage listenerAddr; + socklen_t listenerAddrlen = sizeof(ic_socket_addr); + uint32 socketSendBufferSize; + uint32 socketRecvBufferSize; + + memset(&hints, 0, sizeof(struct addrinfo)); + hints.ai_family = AF_UNSPEC; /* Allow IPv4 or IPv6 */ + hints.ai_socktype = SOCK_DGRAM; /* Datagram socket */ + hints.ai_protocol = 0; + hints.ai_addrlen = 0; + hints.ai_addr = NULL; + hints.ai_canonname = NULL; + hints.ai_next = NULL; + hints.ai_flags |= AI_NUMERICHOST; + +#ifdef USE_ASSERT_CHECKING + if (session_param.gp_udpic_network_disable_ipv6) + hints.ai_family = AF_INET; +#endif + + if (global_param.Gp_interconnect_address_type == INTERCONNECT_ADDRESS_TYPE_UNICAST_IC) + { + Assert(global_param.interconnect_address && strlen(global_param.interconnect_address) > 0); + hints.ai_flags |= AI_NUMERICHOST; + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG3, "getaddrinfo called with unicast address: %s", global_param.interconnect_address); + } + else + { + Assert(global_param.interconnect_address == NULL); + hints.ai_flags |= AI_PASSIVE; + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG3, "getaddrinfo called with wildcard address"); + } + + /* + * Restrict what IP address we will listen on to just the one that was + * used to create this QE session. + */ + Assert(global_param.interconnect_address && strlen(global_param.interconnect_address) > 0); + ret = getaddrinfo((!global_param.interconnect_address || global_param.interconnect_address[0] == '\0') ? NULL : global_param.interconnect_address, + NULL, &hints, &addrs); + if (ret || !addrs) + { + LOG(INFO, "could not resolve address for UDP IC socket %s: %s", + global_param.interconnect_address, + gai_strerror(ret)); + goto startup_failed; + } + + /* + * On some platforms, pg_getaddrinfo_all() may return multiple addresses + * only one of which will actually work (eg, both IPv6 and IPv4 addresses + * when kernel will reject IPv6). Worse, the failure may occur at the + * bind() or perhaps even connect() stage. So we must loop through the + * results till we find a working combination. We will generate DEBUG + * messages, but no error, for bogus combinations. + */ + for (addr = addrs; addr != NULL; addr = addr->ai_next) + { +#ifdef HAVE_UNIX_SOCKETS + /* Ignore AF_UNIX sockets, if any are returned. */ + if (addr->ai_family == AF_UNIX) + continue; +#endif + + if (++tries > 1 && session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG3, "trying another address for UDP interconnect socket"); + + ic_socket = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol); + if (ic_socket == INVALID_SOCKET) + { + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG3, "could not create UDP interconnect socket: %m"); + continue; + } + + /* + * Bind the socket to a kernel assigned ephemeral port on the + * interconnect_address. + */ + if (bind(ic_socket, addr->ai_addr, addr->ai_addrlen) < 0) + { + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG3, "could not bind UDP interconnect socket: %m"); + closesocket(ic_socket); + ic_socket = INVALID_SOCKET; + continue; + } + + /* Call getsockname() to eventually obtain the assigned ephemeral port */ + if (getsockname(ic_socket, (struct sockaddr *) &listenerAddr, &listenerAddrlen) < 0) + { + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG3, "could not get address of UDP interconnect socket: %m"); + closesocket(ic_socket); + ic_socket = INVALID_SOCKET; + continue; + } + + /* If we get here, we have a working socket */ + break; + } + + if (!addr || ic_socket == INVALID_SOCKET) + goto startup_failed; + + /* Memorize the socket fd, kernel assigned port and address family */ + *listenerSocketFd = ic_socket; + if (listenerAddr.ss_family == AF_INET6) + { + *listenerPort = ntohs(((struct sockaddr_in6 *) &listenerAddr)->sin6_port); + *txFamily = AF_INET6; + } + else + { + *listenerPort = ntohs(((struct sockaddr_in *) &listenerAddr)->sin_port); + *txFamily = AF_INET; + } + + /* + * cache the successful sockaddr of the listening socket, so + * we can use this information to connect to the listening socket. + */ + if (listenerSockaddr != NULL) + memcpy(listenerSockaddr, &listenerAddr, sizeof(struct sockaddr_storage)); + + /* Set up socket non-blocking mode */ + if (!ic_set_noblock(ic_socket)) + { + LOG(INFO, "could not set UDP interconnect socket to nonblocking mode: %s", strerror(errno)); + goto startup_failed; + } + + /* Set up the socket's send and receive buffer sizes. */ + socketRecvBufferSize = setUDPSocketBufferSize(ic_socket, SO_RCVBUF); + if (socketRecvBufferSize == static_cast(-1)) + goto startup_failed; + ic_control_info.socketRecvBufferSize = socketRecvBufferSize; + + socketSendBufferSize = setUDPSocketBufferSize(ic_socket, SO_SNDBUF); + if (socketSendBufferSize == static_cast(-1)) + goto startup_failed; + ic_control_info.socketSendBufferSize = socketSendBufferSize; + + if (addrs != NULL) + freeaddrinfo(addrs); + return; + +startup_failed: + if (addrs) + freeaddrinfo(addrs); + if (ic_socket != INVALID_SOCKET) + { + closesocket(ic_socket); + } + + std::stringstream ss; + ss << "ERROR,interconnect error: Could not set up udp listener socket: " << strerror(errno); + throw ICNetworkException(ss.str(), __FILE__, __LINE__); +} + +/* + * InitMutex + * Initialize mutex. + */ +static void +initMutex(pthread_mutex_t *mutex) +{ + pthread_mutexattr_t m_atts; + + pthread_mutexattr_init(&m_atts); + pthread_mutexattr_settype(&m_atts, PTHREAD_MUTEX_ERRORCHECK); + + pthread_mutex_init(mutex, &m_atts); +} + +/* + * Set up the udp interconnect pthread signal mask, we don't want to run our signal handlers + */ +static void +ic_set_pthread_sigmasks(sigset_t *old_sigs) +{ +#ifndef WIN32 + sigset_t sigs; + int err; + + sigfillset(&sigs); + + err = pthread_sigmask(SIG_BLOCK, &sigs, old_sigs); + if (err != 0) + { + std::stringstream ss; + ss << "ERROR: Failed to get pthread signal masks with return value: "<= 0) + closesocket(ICSenderSocket); + ICSenderSocket = -1; + ICSenderPort = 0; + ICSenderFamily = 0; + + memset(&udp_dummy_packet_sockaddr, 0, sizeof(udp_dummy_packet_sockaddr)); + +#ifdef USE_ASSERT_CHECKING + + /* + * Check malloc times, in Interconnect part, memory are carefully released + * in tear down code (even when error occurred). But if a FATAL error is + * reported, tear down code will not be executed. Thus, it is still + * possible the malloc times and free times do not match when we reach + * here. The process will die in this case, the mismatch does not + * introduce issues. + */ + if (icudp_malloc_times != 0) + LOG(INFO, "WARNING: malloc times and free times do not match. remain alloc times: %ld", icudp_malloc_times); +#endif +} + +/* + * getSockAddr + * Convert IP addr and port to sockaddr + */ +static void +getSockAddr(struct sockaddr_storage *peer, socklen_t *peer_len, const char *listenerAddr, int listenerPort) +{ + int ret; + char portNumberStr[32]; + char *service; + struct addrinfo *addrs = NULL; + struct addrinfo hint; + + /* + * Get socketaddr to connect to. + */ + + /* Initialize hint structure */ + memset(&hint, 0, sizeof(hint)); + hint.ai_socktype = SOCK_DGRAM; /* UDP */ + hint.ai_family = AF_UNSPEC; /* Allow for any family (v4, v6, even unix in + * the future) */ +#ifdef AI_NUMERICSERV + hint.ai_flags = AI_NUMERICHOST | AI_NUMERICSERV; /* Never do name + * resolution */ +#else + hint.ai_flags = AI_NUMERICHOST; /* Never do name resolution */ +#endif + + snprintf(portNumberStr, sizeof(portNumberStr), "%d", listenerPort); + service = portNumberStr; + + addrs = NULL; + /* NULL has special meaning to getaddrinfo(). */ + ret = getaddrinfo((!listenerAddr || listenerAddr[0] == '\0') ? NULL : listenerAddr, + service, &hint, &addrs); + if (ret || !addrs) + { + if (addrs) + freeaddrinfo(addrs); + + std::stringstream ss; + ss<<"ERROR, interconnect error: Could not parse remote listener address: '"<ai_family, addrs->ai_socktype, addrs->ai_protocol, listenerAddr); + memset(peer, 0, sizeof(struct sockaddr_storage)); + memcpy(peer, addrs->ai_addr, addrs->ai_addrlen); + *peer_len = addrs->ai_addrlen; + + if (addrs) + freeaddrinfo(addrs); +} + +/* + * format_sockaddr + * Format a sockaddr to a human readable string + * + * This function must be kept threadsafe, elog/ereport/palloc etc are not + * allowed within this function. + */ +static char * +format_sockaddr_udp(struct sockaddr_storage *sa, char *buf, size_t len) +{ + int ret; + char remote_host[NI_MAXHOST]; + char remote_port[NI_MAXSERV]; + + ret = getnameinfo((const struct sockaddr *)sa, sizeof(struct sockaddr_storage), + remote_host, sizeof(remote_host), + remote_port, sizeof(remote_port), + NI_NUMERICHOST | NI_NUMERICSERV); + if (ret != 0) + { + strncpy(remote_host, "???", sizeof(remote_host)); + strncpy(remote_port, "???", sizeof(remote_port)); + } + + if (ret != 0) + snprintf(buf, len, "?host?:?port?"); + else + { +#ifdef HAVE_IPV6 + if (sa->ss_family == AF_INET6) + snprintf(buf, len, "[%s]:%s", remote_host, remote_port); + else +#endif + snprintf(buf, len, "%s:%s", remote_host, remote_port); + } + + return buf; +} + +/* + * setupOutgoingUDPConnection + * Setup outgoing UDP connection. + */ +static void +setupOutgoingUDPConnection(int icid, TransportEntry *pEntry, UDPConn *conn) +{ + ICCdbProcess *cdbProc = NULL; + + Assert(pEntry); + + cdbProc = conn->cdbProc; + Assert(conn->state == mcsSetupOutgoingConnection); + Assert(conn->cdbProc); + + conn->remoteContentId = cdbProc->contentid; + conn->stat_min_ack_time = ~((uint64) 0); + + /* Save the information for the error message if getaddrinfo fails */ + if (strchr(cdbProc->listenerAddr, ':') != 0) + snprintf(conn->remoteHostAndPort, sizeof(conn->remoteHostAndPort), + "[%s]:%d", cdbProc->listenerAddr, cdbProc->listenerPort); + else + snprintf(conn->remoteHostAndPort, sizeof(conn->remoteHostAndPort), + "%s:%d", cdbProc->listenerAddr, cdbProc->listenerPort); + + /* + * Get socketaddr to connect to. + */ + getSockAddr(&conn->peer, &conn->peer_len, cdbProc->listenerAddr, cdbProc->listenerPort); + + /* Save the destination IP address */ + format_sockaddr_udp(&conn->peer, conn->remoteHostAndPort, + sizeof(conn->remoteHostAndPort)); + + Assert(conn->peer.ss_family == AF_INET || conn->peer.ss_family == AF_INET6); + + { +#ifdef USE_ASSERT_CHECKING + { + struct sockaddr_storage source_addr; + socklen_t source_addr_len; + + memset(&source_addr, 0, sizeof(source_addr)); + source_addr_len = sizeof(source_addr); + + if (getsockname(pEntry->txfd, (struct sockaddr *) &source_addr, &source_addr_len) == -1) + { + throw ICNetworkException(std::string("ERROR, interconnect Error: Could not get port from socket, %m")+strerror(errno), __FILE__, __LINE__); + } + Assert(pEntry->txfd_family == source_addr.ss_family); + } +#endif + + /* + * If the socket was created with a different address family than the + * place we are sending to, we might need to do something special. + */ + if (pEntry->txfd_family != conn->peer.ss_family) + { + /* + * If the socket was created AF_INET6, but the address we want to + * send to is IPv4 (AF_INET), we might need to change the address + * format. On Linux, it isn't necessary: glibc automatically + * handles this. But on MAC OSX and Solaris, we need to convert + * the IPv4 address to an V4-MAPPED address in AF_INET6 format. + */ + if (pEntry->txfd_family == AF_INET6) + { + LOG(DEBUG1, "We are inet6, remote is inet. Converting to v4 mapped address."); + ConvertToIPv4MappedAddr(&conn->peer, &conn->peer_len); + } + else + { + /* + * If we get here, something is really wrong. We created the + * socket as IPv4-only (AF_INET), but the address we are + * trying to send to is IPv6. It's possible we could have a + * V4-mapped address that we could convert to an IPv4 address, + * but there is currently no code path where that could + * happen. So this must be an error. + */ + throw ICNetworkException("ERROR: Trying to use an IPv4 (AF_INET) socket to send to an IPv6 address", __FILE__, __LINE__); + } + } + } + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "Interconnect connecting to seg%d slice%d %s pid=%d sockfd=%d", + conn->remoteContentId, pEntry->recvSlice->sliceIndex, conn->remoteHostAndPort, conn->cdbProc->pid, conn->sockfd); + + /* send connection request */ + memset(&conn->conn_info, 0, sizeof(conn->conn_info)); + conn->conn_info.len = 0; + conn->conn_info.flags = 0; + conn->conn_info.motNodeId = pEntry->motNodeId; + + conn->conn_info.recvSliceIndex = pEntry->recvSlice->sliceIndex; + conn->conn_info.sendSliceIndex = pEntry->sendSlice->sliceIndex; + conn->conn_info.srcContentId = global_param.segindex; + conn->conn_info.dstContentId = conn->cdbProc->contentid; + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "setupOutgoingUDPConnection: node %d route %d srccontent %d dstcontent %d: %s", + pEntry->motNodeId, conn->route, global_param.segindex, conn->cdbProc->contentid, conn->remoteHostAndPort); + + conn->conn_info.srcListenerPort = UDP2_GetListenPortUDP(); + conn->conn_info.srcPid = global_param.MyProcPid; + conn->conn_info.dstPid = conn->cdbProc->pid; + conn->conn_info.dstListenerPort = conn->cdbProc->listenerPort; + + conn->conn_info.sessionId = session_param.gp_session_id; + conn->conn_info.icId = icid; + + ic_control_info.connHtab.add(conn); + + /* + * No need to get the connection lock here, since background rx thread + * will never access send connections. + */ + conn->msgPos = NULL; + conn->msgSize = sizeof(conn->conn_info); + conn->stillActive = true; + conn->conn_info.seq = 1; + Assert(conn->peer.ss_family == AF_INET || conn->peer.ss_family == AF_INET6); +} + +/* + * If the socket was created AF_INET6, but the address we want to + * send to is IPv4 (AF_INET), we need to change the address + * format. On Linux, this is not necessary: glibc automatically + * handles this. But on MAC OSX and Solaris, we need to convert + * the IPv4 address to IPv4-mapped IPv6 address in AF_INET6 format. + * + * The comment above relies on getaddrinfo() via function getSockAddr to get + * the correct V4-mapped address. We need to be careful here as we need to + * ensure that the platform we are using is POSIX 1003-2001 compliant. + * Just to be on the safeside, we'll be keeping this function for + * now to be used for all platforms and not rely on POSIX. + * + * Since this can be called in a signal handler, we avoid the use of + * async-signal unsafe functions such as memset/memcpy + */ +static void +ConvertToIPv4MappedAddr(struct sockaddr_storage *sockaddr, socklen_t *o_len) +{ + const struct sockaddr_in *in = (const struct sockaddr_in *)sockaddr; + struct sockaddr_storage temp = {0}; + struct sockaddr_in6 *in6_new = (struct sockaddr_in6 *)&temp; + + /* Construct a IPv4-to-IPv6 mapped address. */ + temp.ss_family = AF_INET6; + in6_new->sin6_family = AF_INET6; + in6_new->sin6_port = in->sin_port; + in6_new->sin6_flowinfo = 0; + + ((uint16 *)&in6_new->sin6_addr)[5] = 0xffff; + + in6_new->sin6_addr.s6_addr32[3] = in->sin_addr.s_addr; + in6_new->sin6_scope_id = 0; + + /* copy it back */ + *sockaddr = temp; + *o_len = sizeof(struct sockaddr_in6); +} + +#if defined(__darwin__) +/* macos does not accept :: as the destination, we will need to covert this to the IPv6 loopback */ +static void +ConvertIPv6WildcardToLoopback(struct sockaddr_storage *dest) +{ + char address[INET6_ADDRSTRLEN]; + /* we want to terminate our own process, so this should be local */ + const struct sockaddr_in6 *in6 = (const struct sockaddr_in6 *)&udp_dummy_packet_sockaddr; + inet_ntop(AF_INET6, &in6->sin6_addr, address, sizeof(address)); + if (strcmp("::", address) == 0) + ((struct sockaddr_in6 *)dest)->sin6_addr = in6addr_loopback; +} +#endif + +/* + * handleCachedPackets + * Deal with cached packets. + */ +static void +handleCachedPackets(void) +{ + UDPConn *cachedConn = NULL; + UDPConn *setupConn = NULL; + ConnHtabBin *bin = NULL; + icpkthdr *pkt = NULL; + AckSendParam param; + int i = 0; + uint32 j = 0; + bool dummy; + + for (i = 0; i < ic_control_info.startupCacheHtab.size; i++) + { + bin = ic_control_info.startupCacheHtab.table[i]; + + while (bin) + { + cachedConn = bin->conn; + setupConn = NULL; + + for (j = 0; j < cachedConn->pkt_q_size; j++) + { + pkt = (icpkthdr *) cachedConn->pkt_q[j]; + + if (pkt == NULL) + continue; + + rx_buffer_pool.maxCount--; + + /* look up this pkt's connection in connHtab */ + setupConn = ic_control_info.connHtab.find(pkt); + if (setupConn == NULL) + { + /* mismatch! */ + rx_buffer_pool.put(pkt); + cachedConn->pkt_q[j] = NULL; + continue; + } + + memset(¶m, 0, sizeof(param)); + if (!handleDataPacket(setupConn, pkt, &cachedConn->peer, &cachedConn->peer_len, ¶m, &dummy)) + { + /* no need to cache this packet */ + rx_buffer_pool.put(pkt); + } + + ic_statistics.recvPktNum++; + if (param.msg.len != 0) + UDPConn::sendAckWithParam(¶m); + + cachedConn->pkt_q[j] = NULL; + } + bin = bin->next; + ic_control_info.startupCacheHtab.remove(cachedConn); + + /* + * MPP-19981 free the cached connections; otherwise memory leak + * would be introduced. + */ + ic_free(cachedConn->pkt_q); + delete cachedConn; + } + } +} + +/* + * CChunkTransportStateImpl::setup + * Internal function for setting up UDP interconnect. + */ +ICChunkTransportState* +CChunkTransportStateImpl::setup(ICSliceTable *sliceTable) +{ + pthread_mutex_lock(&ic_control_info.lock); + + Assert(sliceTable->ic_instance_id > 0); + + if (global_param.Gp_role == GP_ROLE_DISPATCH_IC) + { + /* + * QD use cursorHistoryTable to handle mismatch packets, no + * need to update ic_control_info.ic_instance_id + */ + Assert(session_param.gp_interconnect_id == sliceTable->ic_instance_id); + } + else + { + /* + * update ic_control_info.ic_instance_id, it is mainly used + * by rx thread to handle mismatch packets + */ + ic_control_info.ic_instance_id = sliceTable->ic_instance_id; + } + + CChunkTransportStateImpl *state_impl = new CChunkTransportStateImpl(sliceTable); + ICChunkTransportState *interconnect_context = static_cast(state_impl); + CChunkTransportStateImpl::state_ = static_cast(state_impl); + +#ifdef USE_ASSERT_CHECKING + ICExecSlice *mySlice = &interconnect_context->sliceTable->slices[sliceTable->localSlice]; + Assert(mySlice && mySlice->sliceIndex == sliceTable->localSlice); +#endif + +#ifdef USE_ASSERT_CHECKING + set_test_mode(); +#endif + + if (global_param.Gp_role == GP_ROLE_DISPATCH_IC) + { + CursorICHistoryTable *ich_table = &rx_control_info.cursorHistoryTable; + //DistributedTransactionId distTransId = getDistributedTransactionId(); TODO: add callback; + DistributedTransactionId distTransId = InvalidTransactionId; + + if (ich_table->count > (2 * ich_table->size)) + { + /* + * distTransId != lastDXatId + * Means the last transaction is finished, it's ok to make a prune. + */ + if (distTransId != rx_control_info.lastDXatId) + { + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "prune cursor history table (count %d), icid %d, prune_id %d", + ich_table->count, sliceTable->ic_instance_id, sliceTable->ic_instance_id); + ich_table->prune(sliceTable->ic_instance_id); + } + /* + * distTransId == lastDXatId and they are not InvalidTransactionId(0) + * Means current (non Read-Only) transaction isn't finished, should not prune. + */ + else if (rx_control_info.lastDXatId != InvalidTransactionId) + { + ; + } + /* + * distTransId == lastDXatId and they are InvalidTransactionId(0) + * Means they are the same transaction or different Read-Only transactions. + * + * For the latter, it's hard to get a perfect timepoint to prune: prune eagerly may + * cause problems (pruned current Txn's Ic instances), but prune in low frequency + * causes memory leak. + * + * So, we choose a simple algorithm to prune it here. And if it mistakenly prune out + * the still-in-used Ic instance (with lower id), the query may hang forever. + * Then user have to set a bigger gp_interconnect_cursor_ic_table_size value and + * try the query again, it is a workaround. + * + * More backgrounds please see: https://github.com/greenplum-db/gpdb/pull/16458 + */ + else + { + if (sliceTable->ic_instance_id > ich_table->size) + { + uint32 prune_id = sliceTable->ic_instance_id - ich_table->size; + Assert(prune_id < sliceTable->ic_instance_id); + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "prune cursor history table (count %d), icid %d, prune_id %d", + ich_table->count, sliceTable->ic_instance_id, prune_id); + ich_table->prune(prune_id); + } + } + } + + ich_table->add(sliceTable->ic_instance_id, session_param.gp_command_count); + /* save the latest transaction id */ + rx_control_info.lastDXatId = distTransId; + } + + /* Initiate receiving connections. */ + state_impl->CreateRecvEntries(sliceTable); + + /* Initiate outgoing connections. */ + state_impl->CreateSendEntries(sliceTable); + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "SetupUDPInterconnect will activate Listening on ports=%d/%d sockfd=%d.", 0, UDP2_GetListenPortUDP(), UDP_listenerFd); + + /* + * If there are packets cached by background thread, add them to the + * connections. + */ + if (session_param.gp_interconnect_cache_future_packets) + handleCachedPackets(); + + interconnect_context->activated = true; + + pthread_mutex_unlock(&ic_control_info.lock); + + return interconnect_context; +} + +/* + * sendControlMessage + * Helper function to send a control message. + */ +void +UDPConn::sendControlMessage(icpkthdr *pkt, int fd, struct sockaddr *addr, socklen_t peerLen) +{ + int n; + +#ifdef USE_ASSERT_CHECKING + if (testmode_inject_fault(session_param.gp_udpic_dropacks_percent)) + { +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "THROW CONTROL MESSAGE with seq %d extraSeq %d srcpid %d despid %d", pkt->seq, pkt->extraSeq, pkt->srcPid, pkt->dstPid); +#endif + return; + } +#endif + + /* Add CRC for the control message. */ + if (session_param.gp_interconnect_full_crc) + addCRC(pkt); + + /* retry 10 times for sending control message */ + int counter = 0; + while (counter < 10) + { + counter++; + n = sendto(fd, (const char *)pkt, pkt->len, 0, addr, peerLen); + if (n < 0) + { + if (errno == EINTR || errno == EAGAIN || errno == EWOULDBLOCK) + continue; + else + { + LOG(INFO, "sendcontrolmessage: got errno %d", errno); + return; + } + } + break; + } + if (n < int(pkt->len)) + LOG(INFO, "sendcontrolmessage: got error %d errno %d seq %d", n, errno, pkt->seq); +} + +/* + * setAckParam + * Set the ack sending parameters. + */ +void +UDPConn::setAckParam(AckSendParam *param, int32 flags, uint32 seq, uint32 extraSeq) +{ + memcpy(¶m->msg, (char *) &this->conn_info, sizeof(icpkthdr)); + param->msg.flags = flags; + param->msg.seq = seq; + param->msg.extraSeq = extraSeq; + param->msg.len = sizeof(icpkthdr); + param->peer = this->peer; + param->peer_len = this->peer_len; +} + +/* + * sendAckWithParam + * Send acknowledgment to sender. + */ +void +UDPConn::sendAckWithParam(AckSendParam *param) +{ + sendControlMessage(¶m->msg, UDP_listenerFd, (struct sockaddr *) ¶m->peer, param->peer_len); +} + +/* + * sendAck + * Send acknowledgment to sender. + */ +void +UDPConn::sendAck(int32 flags, uint32 seq, uint32 extraSeq) +{ + icpkthdr msg; + + memcpy(&msg, (char *) &this->conn_info, sizeof(msg)); + msg.flags = flags; + msg.seq = seq; + msg.extraSeq = extraSeq; + msg.len = sizeof(icpkthdr); + + LOG(DEBUG1, "sendack: node %d route %d seq %d extraSeq %d, flags %s", msg.motNodeId, this->route, msg.seq, msg.extraSeq, flags2txt(msg.flags)); + + sendControlMessage(&msg, UDP_listenerFd, (struct sockaddr *) &this->peer, this->peer_len); +} + +/* + * sendDisorderAck + * Send a disorder message to the sender. + * + * Whenever the receiver detects a disorder packet, it will assemble a disorder message + * which contains the sequence numbers of the possibly lost packets. + * + */ +void +UDPConn::sendDisorderAck(uint32 seq, uint32 extraSeq, uint32 lostPktCnt) +{ + icpkthdr *disorderBuffer = rx_control_info.disorderBuffer; + + memcpy(disorderBuffer, (char *) &this->conn_info, sizeof(icpkthdr)); + + disorderBuffer->flags |= UDPIC_FLAGS_DISORDER; + disorderBuffer->seq = seq; + disorderBuffer->extraSeq = extraSeq; + disorderBuffer->len = lostPktCnt * sizeof(uint32) + sizeof(icpkthdr); + +#ifdef AMS_VERBOSE_LOGGING + if (!(this->peer.ss_family == AF_INET || this->peer.ss_family == AF_INET6)) + { + LOG(INFO, "UDP Interconnect bug (in sendDisorderAck): trying to send ack when we don't know where to send to %s", this->remoteHostAndPort); + } +#endif + + sendControlMessage(disorderBuffer, UDP_listenerFd, (struct sockaddr *) &this->peer, this->peer_len); +} + +/* + * sendStatusQueryMessage + * Used by senders to send a status query message for a connection to receivers. + * + * When receivers get such a message, they will respond with + * the connection status (consumed seq, received seq ...). + */ +void +UDPConn::sendStatusQueryMessage(uint32 seq) +{ + icpkthdr msg; + + memcpy(&msg, (char *) &this->conn_info, sizeof(msg)); + msg.flags = UDPIC_FLAGS_CAPACITY; + msg.seq = seq; + msg.extraSeq = 0; + msg.len = sizeof(msg); + +#ifdef TRANSFER_PROTOCOL_STATS + trans_proto_stats.update(TPE_ACK_PKT_QUERY, &msg); +#endif + + sendControlMessage(&msg, entry_->txfd, (struct sockaddr *) &this->peer, this->peer_len); +} + +/* + * ReleaseBuffer + * Return a buffer and send an acknowledgment. + * + * SHOULD BE CALLED WITH ic_control_info.lock *LOCKED* + */ +void +UDPConn::ReleaseBuffer(AckSendParam *param) +{ + icpkthdr *buf; + uint32 seq; + + buf = (icpkthdr *) this->pkt_q[this->pkt_q_head]; + if (buf == NULL) + { + pthread_mutex_unlock(&ic_control_info.lock); + throw ICFatalException("FATAL: ReleaseBuffer: buffer is NULL", __FILE__, __LINE__); + } + + seq = buf->seq; + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "LOG: ReleaseBuffer conn %p pkt [seq %d] for node %d route %d, [head seq] %d queue size %d, queue head %d queue tail %d", + this, seq, buf->motNodeId, this->route, this->conn_info.seq - this->pkt_q_size, this->pkt_q_size, this->pkt_q_head, this->pkt_q_tail); +#endif + + this->pkt_q[this->pkt_q_head] = NULL; + this->pBuff = NULL; + this->pkt_q_head = (this->pkt_q_head + 1) % this->pkt_q_capacity; + this->pkt_q_size--; + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "LOG: ReleaseBuffer conn %p pkt [seq %d] for node %d route %d, [head seq] %d queue size %d, queue head %d queue tail %d", + this, seq, buf->motNodeId, this->route, this->conn_info.seq - this->pkt_q_size, this->pkt_q_size, this->pkt_q_head, this->pkt_q_tail); +#endif + + rx_buffer_pool.put(buf); + this->conn_info.extraSeq = seq; + + /* Send an Ack to the sender. */ + if ((seq % 2 == 0) || (this->pkt_q_capacity == 1)) + { + if (param != NULL) + { + this->setAckParam(param, UDPIC_FLAGS_ACK | UDPIC_FLAGS_CAPACITY | this->conn_info.flags, this->conn_info.seq - 1, seq); + } + else + { + this->sendAck(UDPIC_FLAGS_ACK | UDPIC_FLAGS_CAPACITY | this->conn_info.flags, this->conn_info.seq - 1, seq); + } + } +} + +/* + * computeExpirationPeriod + * Compute expiration period according to the connection information. + * + * Considerations on expiration period computation: + * + * RTT is dynamically computed, and expiration period is based on RTT values. + * We cannot simply use RTT as the expiration value, since real workload does + * not always have a stable RTT. A small constant value is multiplied to the RTT value + * to make the resending logic insensitive to the frequent small changes of RTT. + * + */ +uint64 +UDPConn::computeExpirationPeriod(uint32 retry) +{ + /* + * In fault injection mode, we often use DEFAULT_RTT, because the + * intentional large percent of packet/ack losses will make the RTT too + * large. This will lead to a slow retransmit speed. In real hardware + * environment/workload, we do not expect such a packet loss pattern. + */ +#ifdef USE_ASSERT_CHECKING + if (udp_testmode) + { + return DEFAULT_RTT; + } + else +#endif + { + uint32 factor = (retry <= 12 ? retry : 12); + return Max(MIN_EXPIRATION_PERIOD, Min(MAX_EXPIRATION_PERIOD, (int)(this->rtt + (this->dev << 2)) << (factor))); + } +} + +/* + * freeDisorderedPackets + * Put the disordered packets into free buffer list. + */ +void +UDPConn::freeDisorderedPackets() +{ + uint32 k; + + if (this->pkt_q == NULL) + return; + + for (k = 0; k < this->pkt_q_capacity; k++) + { + icpkthdr *buf = (icpkthdr *)this->pkt_q[k]; + + if (buf != NULL) + { + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "CLEAR Out-of-order PKT: conn %p pkt [seq %d] for node %d route %d, " + "[head seq] %d queue size %d, queue head %d queue tail %d", + this, buf->seq, buf->motNodeId, this->route, this->conn_info.seq - this->pkt_q_size, + this->pkt_q_size, this->pkt_q_head, this->pkt_q_tail); + + /* return the buffer into the free list. */ + rx_buffer_pool.put(buf); + this->pkt_q[k] = NULL; + } + } +} + +/* + * prepareRxConnForRead + * Prepare the receive connection for reading. + * + * MUST BE CALLED WITH ic_control_info.lock LOCKED. + */ +void +UDPConn::prepareRxConnForRead() +{ + LOG(DEBUG3, "In prepareRxConnForRead: conn %p, q_head %d q_tail %d q_size %d", + this, this->pkt_q_head, this->pkt_q_tail, this->pkt_q_size); + + Assert(this->pkt_q[this->pkt_q_head] != NULL); + this->pBuff = this->pkt_q[this->pkt_q_head]; + this->msgPos = this->pBuff; + this->msgSize = ((icpkthdr *) this->pBuff)->len; + this->recvBytes = this->msgSize; +} + +/* + * DeactiveConn + * Mark the connection inactive. + */ +void +UDPConn::DeactiveConn() +{ + pthread_mutex_lock(&ic_control_info.lock); + this->stillActive = false; + pthread_mutex_unlock(&ic_control_info.lock); +} + +/* + * handleAckedPacket + * Called by sender to process acked packet. + * + * Remove it from unack queue and unack queue ring, change the rtt ... + * + * RTT (Round Trip Time) is computed as the time between we send the packet + * and receive the acknowledgement for the packet. When an acknowledgement + * is received, an estimated RTT value (called SRTT, smoothed RTT) is updated + * by using the following equation. And we also set a limitation of the max + * value and min value for SRTT. + * (1) SRTT = (1 - g) SRTT + g x RTT (0 < g < 1) + * where RTT is the measured round trip time of the packet. In implementation, + * g is set to 1/8. In order to compute expiration period, we also compute an + * estimated delay variance SDEV by using: + * (2) SDEV = (1 - h) x SDEV + h x |SERR| (0 < h < 1, In implementation, h is set to 1/4) + * where SERR is calculated by using: + * (3) SERR = RTT - SRTT + * Expiration period determines the timing we resend a packet. A long RTT means + * a long expiration period. Delay variance is used to incorporate the variance + * of workload/network variances at different time. When a packet is retransmitted, + * we back off exponentially the expiration period. + * (4) exp_period = (SRTT + y x SDEV) << retry + * Here y is a constant (In implementation, we use 4) and retry is the times the + * packet is retransmitted. + */ +void +UDPConn::handleAckedPacket(ICBuffer *buf, uint64 now) +{ + uint64 ackTime = 0; + bool bufIsHead = false; + UDPConn *bufConn = NULL; + + bufIsHead = (&buf->primary == this->unackQueue.first()); + + buf = this->unackQueue.remove(buf); + + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_LOSS_IC) + { + ICBufferList *alist = &unack_queue_ring.slots[buf->unackQueueRingSlot]; + buf = alist->remove(buf); + unack_queue_ring.numOutStanding--; + if (this->unackQueue.length() >= 1) + unack_queue_ring.numSharedOutStanding--; + + ackTime = now - buf->sentTime; + + /* + * In udp_testmode, we do not change rtt dynamically due to the large + * number of packet losses introduced by fault injection code. This + * can decrease the testing time. + */ +#ifdef USE_ASSERT_CHECKING + if (!udp_testmode) +#endif + { + uint64 newRTT = 0; + uint64 newDEV = 0; + + if (buf->nRetry == 0) + { + bufConn = static_cast(buf->conn); + newRTT = bufConn->rtt - (bufConn->rtt >> RTT_SHIFT_COEFFICIENT) + (ackTime >> RTT_SHIFT_COEFFICIENT); + newRTT = Min(MAX_RTT, Max(newRTT, MIN_RTT)); + bufConn->rtt = newRTT; + + newDEV = bufConn->dev - (bufConn->dev >> DEV_SHIFT_COEFFICIENT) + ((Max(ackTime, newRTT) - Min(ackTime, newRTT)) >> DEV_SHIFT_COEFFICIENT); + newDEV = Min(MAX_DEV, Max(newDEV, MIN_DEV)); + bufConn->dev = newDEV; + + /* adjust the congestion control window. */ + if (snd_control_info.cwnd < snd_control_info.ssthresh) + snd_control_info.cwnd += 1; + else + snd_control_info.cwnd += 1 / snd_control_info.cwnd; + snd_control_info.cwnd = Min(snd_control_info.cwnd, snd_buffer_pool.maxCount); + } + } + } + + bufConn = static_cast(buf->conn); + bufConn->stat_total_ack_time += ackTime; + bufConn->stat_max_ack_time = Max(ackTime, bufConn->stat_max_ack_time); + bufConn->stat_min_ack_time = Min(ackTime, bufConn->stat_min_ack_time); + + /* + * only change receivedAckSeq when it is the smallest pkt we sent and have + * not received ack for it. + */ + if (bufIsHead) + this->receivedAckSeq = buf->pkt->seq; + + /* The first packet acts like a connect setup packet */ + if (buf->pkt->seq == 1) + this->state = mcsStarted; + + snd_buffer_pool.freeList.append(buf); + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "REMOVEPKT %d from unack queue for route %d (retry %d) sndbufmaxcount %d sndbufcount %d " + "sndbuffreelistlen %d, sntSeq %d consumedSeq %d recvAckSeq %d capacity %d, sndQ %d, unackQ %d", + buf->pkt->seq, this->route, buf->nRetry, snd_buffer_pool.maxCount, snd_buffer_pool.count, + snd_buffer_pool.freeList.length(), bufConn->sentSeq, bufConn->consumedSeq, + bufConn->receivedAckSeq, bufConn->capacity, bufConn->sndQueue.length(), + bufConn->unackQueue.length()); + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + { + bufConn->unackQueue.icBufferListLog(); + bufConn->sndQueue.icBufferListLog(); + } +#endif +} + +/* + * dispatcherAYT + * Check the connection from the dispatcher to verify that it is still there. + * + * The connection is a struct Port, stored in the global MyProcPort. + * + * Return true if the dispatcher connection is still alive. + */ +static bool +dispatcherAYT(void) +{ + ssize_t ret; + char buf; + + /* + * For background worker or auxiliary process like gdd, there is no client. + * As a result, MyProcPort is NULL. We should skip dispatcherAYT check here. + */ + if (global_param.MyProcPort == false) + return true; + + if (global_param.myprocport_sock < 0) + return false; + +#ifndef WIN32 + ret = recv(global_param.myprocport_sock, &buf, 1, MSG_PEEK | MSG_DONTWAIT); +#else + ret = recv(global_param.myprocport_sock, &buf, 1, MSG_PEEK | MSG_PARTIAL); +#endif + + if (ret == 0) /* socket has been closed. EOF */ + return false; + + if (ret > 0) /* data waiting on socket, it must be OK. */ + return true; + + if (ret == -1) /* error, or would be block. */ + { + if (errno == EAGAIN || errno == EINPROGRESS) + return true; /* connection intact, no data available */ + else + return false; + } + /* not reached */ + + return true; +} + +/* + * checkQDConnectionAlive + * Check whether QD connection is still alive. If not, report error. + */ +static void +checkQDConnectionAlive(void) +{ + if (!dispatcherAYT()) + { + if (global_param.Gp_role == GP_ROLE_EXECUTE_IC) + throw ICNetworkException("interconnect error segment lost contact with master (recv)", __FILE__, __LINE__); + else + throw ICNetworkException("interconnect error master lost contact with client (recv)", __FILE__, __LINE__); + } +} + +/* + * getCurrentTime + * get current time + * + */ +static uint64 +getCurrentTime(void) +{ + struct timeval newTime; + int status = 1; + uint64 t = 0; + +#if HAVE_LIBRT + /* Use clock_gettime to return monotonic time value. */ + struct timespec ts; + + status = clock_gettime(CLOCK_MONOTONIC, &ts); + + newTime.tv_sec = ts.tv_sec; + newTime.tv_usec = ts.tv_nsec / 1000; + +#endif + + if (status != 0) + gettimeofday(&newTime, NULL); + + t = ((uint64) newTime.tv_sec) * USECS_PER_SECOND + newTime.tv_usec; + return t; +} + +/* + * putIntoUnackQueueRing + * Put the buffer into the ring. + * + * expTime - expiration time from now + * + */ +static void +putIntoUnackQueueRing(UnackQueueRing *uqr, ICBuffer *buf, uint64 expTime, uint64 now) +{ + uint64 diff = 0; + int idx = 0; + + /* The first packet, currentTime is not initialized */ + if (uqr->currentTime == 0) + uqr->currentTime = now - (now % TIMER_SPAN); + + diff = now + expTime - uqr->currentTime; + if (diff >= UNACK_QUEUE_RING_LENGTH) + { +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "putIntoUnackQueueRing:" "now %lu expTime %lu diff %lu uqr-currentTime %lu", now, expTime, diff, uqr->currentTime); +#endif + diff = UNACK_QUEUE_RING_LENGTH - 1; + } + else if (diff < TIMER_SPAN) + { + diff = TIMER_SPAN; + } + + idx = (uqr->idx + diff / TIMER_SPAN) % UNACK_QUEUE_RING_SLOTS_NUM; + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "PUTTW: curtime %lu now %lu (diff %lu) expTime %lu previdx %d, nowidx %d, nextidx %d", uqr->currentTime, now, diff, expTime, buf->unackQueueRingSlot, uqr->idx, idx); +#endif + + buf->unackQueueRingSlot = idx; + unack_queue_ring.slots[idx].append(buf); +} + +/* + * handleDataPacket + * Handling the data packet. + * + * On return, will set *wakeup_mainthread, if a packet was received successfully + * and the caller should wake up the main thread, after releasing the mutex. + */ +static bool +handleDataPacket(UDPConn *conn, icpkthdr *pkt, struct sockaddr_storage *peer, socklen_t *peerlen, + AckSendParam *param, bool *wakeup_mainthread) +{ + if ((pkt->len == sizeof(icpkthdr)) && (pkt->flags & UDPIC_FLAGS_CAPACITY)) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "status queuy message received, seq %d, srcpid %d, dstpid %d, icid %d, sid %d", + pkt->seq, pkt->srcPid, pkt->dstPid, pkt->icId, pkt->sessionId); + +#ifdef AMS_VERBOSE_LOGGING + logPkt("STATUS QUERY MESSAGE", pkt); +#endif + uint32 seq = conn->conn_info.seq > 0 ? conn->conn_info.seq - 1 : 0; + uint32 extraSeq = conn->stopRequested ? seq : conn->conn_info.extraSeq; + + conn->setAckParam(param, UDPIC_FLAGS_CAPACITY | UDPIC_FLAGS_ACK | conn->conn_info.flags, seq, extraSeq); + + return false; + } + + /* + * when we're not doing a full-setup on every statement, we've got to + * update the peer info -- full setups do this at setup-time. + */ + + /* + * Note the change here, for process start race and disordered message, if + * we do not fill in peer address, then we may send some acks to unknown + * address. Thus, the following condition is used. + * + */ + if (pkt->seq <= conn->pkt_q_capacity) + { + /* fill in the peer. Need to cast away "volatile". ugly */ + memset((void *) &conn->peer, 0, sizeof(conn->peer)); + memcpy((void *) &conn->peer, peer, *peerlen); + conn->peer_len = *peerlen; + + conn->conn_info.dstListenerPort = pkt->dstListenerPort; + if (IC_DEBUG2 >= session_param.log_min_messages) + LOG(DEBUG2, "received the head packets when eliding setup, pkt seq %d", pkt->seq); + } + + /* data packet */ + if (pkt->flags & UDPIC_FLAGS_EOS) + { + if (IC_DEBUG3 >= session_param.log_min_messages) + LOG(DEBUG3, "received packet with EOS motid %d route %d seq %d", + pkt->motNodeId, conn->route, pkt->seq); + } + + /* + * if we got a stop, but didn't request a stop -- ignore, this is a + * startup blip: we must have acked with a stop -- we don't want to do + * anything further with the stop-message if we didn't request a stop! + * + * this is especially important after eliding setup is enabled. + */ + if (!conn->stopRequested && (pkt->flags & UDPIC_FLAGS_STOP)) + { + if (pkt->flags & UDPIC_FLAGS_EOS) + { + LOG(INFO, "non-requested stop flag, EOS! seq %d, flags 0x%x", pkt->seq, pkt->flags); + } + return false; + } + + if (conn->stopRequested && conn->stillActive) + { + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC && IC_DEBUG5 >= session_param.log_min_messages) + LOG(DEBUG5, "rx_thread got packet on active connection marked stopRequested. " + "(flags 0x%x) node %d route %d pkt seq %d conn seq %d", + pkt->flags, pkt->motNodeId, conn->route, pkt->seq, conn->conn_info.seq); + + /* can we update stillActive ? */ + if (IC_DEBUG2 >= session_param.log_min_messages) + if (!(pkt->flags & UDPIC_FLAGS_STOP) && !(pkt->flags & UDPIC_FLAGS_EOS)) + LOG(DEBUG2, "stop requested but no stop flag on return packet ?!"); + + if (pkt->flags & UDPIC_FLAGS_EOS) + conn->conn_info.flags |= UDPIC_FLAGS_EOS; + + if (conn->conn_info.seq < pkt->seq) + conn->conn_info.seq = pkt->seq; /* note here */ + + conn->setAckParam(param, UDPIC_FLAGS_ACK | UDPIC_FLAGS_STOP | UDPIC_FLAGS_CAPACITY | conn->conn_info.flags, pkt->seq, pkt->seq); + + /* we only update stillActive if eos has been sent by peer. */ + if (pkt->flags & UDPIC_FLAGS_EOS) + { + if (IC_DEBUG2 >= session_param.log_min_messages) + LOG(DEBUG2, "stop requested and acknowledged by sending peer"); + conn->stillActive = false; + } + + return false; + } + + /* dropped ack or timeout */ + if (pkt->seq < conn->conn_info.seq) + { + ic_statistics.duplicatedPktNum++; + if (IC_DEBUG3 >= session_param.log_min_messages) + LOG(DEBUG3, "dropped ack ? ignored data packet w/ cmd %d conn->cmd %d node %d route %d seq %d expected %d flags 0x%x", + pkt->icId, conn->conn_info.icId, pkt->motNodeId, conn->route, pkt->seq, conn->conn_info.seq, pkt->flags); + + conn->setAckParam(param, UDPIC_FLAGS_ACK | UDPIC_FLAGS_CAPACITY | conn->conn_info.flags, conn->conn_info.seq - 1, conn->conn_info.extraSeq); + + return false; + } + + /* sequence number is correct */ + if (!conn->stillActive) + { + /* peer may have dropped ack */ + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_VERBOSE_IC && + IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "received on inactive connection node %d route %d (seq %d pkt->seq %d)", + pkt->motNodeId, conn->route, conn->conn_info.seq, pkt->seq); + + if (conn->conn_info.seq < pkt->seq) + conn->conn_info.seq = pkt->seq; + conn->setAckParam(param, UDPIC_FLAGS_ACK | UDPIC_FLAGS_STOP | UDPIC_FLAGS_CAPACITY | conn->conn_info.flags, pkt->seq, pkt->seq); + + return false; + } + + /* headSeq is the seq for the head packet. */ + uint32 headSeq = conn->conn_info.seq - conn->pkt_q_size; + + if ((conn->pkt_q_size == conn->pkt_q_capacity) || (pkt->seq - headSeq >= conn->pkt_q_capacity)) + { + /* + * Error case: NO RX SPACE or out of range pkt This indicates a bug. + */ + logPkt("Interconnect error: received a packet when the queue is full ", pkt); + ic_statistics.disorderedPktNum++; + conn->stat_count_dropped++; + return false; + } + + /* put the packet at the his position */ + bool toWakeup = false; + + int pos = (pkt->seq - 1) % conn->pkt_q_capacity; + + if (conn->pkt_q[pos] == NULL) + { + conn->pkt_q[pos] = (uint8 *) pkt; + if (pos == conn->pkt_q_head) + { +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "SAVE pkt at QUEUE HEAD [seq %d] for node %d route %d, queue head seq %d, queue size %d, queue head %d queue tail %d", + pkt->seq, pkt->motNodeId, conn->route, headSeq, conn->pkt_q_size, conn->pkt_q_head, conn->pkt_q_tail); +#endif + toWakeup = true; + } + + if (pos == conn->pkt_q_tail) + { + /* move the queue tail */ + for (; conn->pkt_q[conn->pkt_q_tail] != NULL && conn->pkt_q_size < conn->pkt_q_capacity;) + { + conn->pkt_q_size++; + conn->pkt_q_tail = (conn->pkt_q_tail + 1) % conn->pkt_q_capacity; + conn->conn_info.seq++; + } + + /* set the EOS flag */ + if (((icpkthdr *) (conn->pkt_q[(conn->pkt_q_tail + conn->pkt_q_capacity - 1) % conn->pkt_q_capacity]))->flags & UDPIC_FLAGS_EOS) + { + conn->conn_info.flags |= UDPIC_FLAGS_EOS; + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "RX_THREAD: the packet with EOS flag is available for access in the queue for route %d", conn->route); + } + + /* ack data packet */ + conn->setAckParam(param, UDPIC_FLAGS_CAPACITY | UDPIC_FLAGS_ACK | conn->conn_info.flags, conn->conn_info.seq - 1, conn->conn_info.extraSeq); + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "SAVE conn %p pkt at QUEUE TAIL [seq %d] at pos [%d] for node %d route %d, [head seq] %d, queue size %d, queue head %d queue tail %d", + conn, pkt->seq, pos, pkt->motNodeId, conn->route, headSeq, conn->pkt_q_size, conn->pkt_q_head, conn->pkt_q_tail); +#endif + } + else /* deal with out-of-order packet */ + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "SAVE conn %p OUT-OF-ORDER pkt [seq %d] at pos [%d] for node %d route %d, [head seq] %d, queue size %d, queue head %d queue tail %d", + conn, pkt->seq, pos, pkt->motNodeId, conn->route, headSeq, conn->pkt_q_size, conn->pkt_q_head, conn->pkt_q_tail); + + /* send an ack for out-of-order packet */ + ic_statistics.disorderedPktNum++; + conn->handleDisorderPacket(pos, headSeq + conn->pkt_q_size, pkt); + } + } + else /* duplicate pkt */ + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "DUPLICATE pkt [seq %d], [head seq] %d, queue size %d, queue head %d queue tail %d", + pkt->seq, headSeq, conn->pkt_q_size, conn->pkt_q_head, conn->pkt_q_tail); + + conn->setAckParam(param, UDPIC_FLAGS_DUPLICATE | conn->conn_info.flags, pkt->seq, conn->conn_info.seq - 1); + ic_statistics.duplicatedPktNum++; + return false; + } + + /* Was the main thread waiting for something ? */ + if (rx_control_info.mainWaitingState.waiting && + rx_control_info.mainWaitingState.waitingNode == pkt->motNodeId && + rx_control_info.mainWaitingState.waitingQuery == pkt->icId && toWakeup) + { + if (rx_control_info.mainWaitingState.waitingRoute == ANY_ROUTE) + { + if (rx_control_info.mainWaitingState.reachRoute == ANY_ROUTE) + rx_control_info.mainWaitingState.reachRoute = conn->route; + } + else if (rx_control_info.mainWaitingState.waitingRoute == conn->route) + { + if (IC_DEBUG2 >= session_param.log_min_messages) + LOG(DEBUG2, "rx thread: main_waiting waking it route %d", rx_control_info.mainWaitingState.waitingRoute); + rx_control_info.mainWaitingState.reachRoute = conn->route; + } + /* WAKE MAIN THREAD HERE */ + *wakeup_mainthread = true; + } + + return true; +} + +/* + * rxThreadFunc + * Main function of the receive background thread. + * + * NOTE: This function MUST NOT contain elog or ereport statements. + * elog is NOT thread-safe. Developers should instead use something like: + * + * NOTE: In threads, we cannot use palloc/pfree, because it's not thread safe. + */ +static void * +rxThreadFunc(void *arg) +{ + icpkthdr *pkt = NULL; + bool skip_poll = false; + + for (;;) + { + struct pollfd nfd; + int n; + + /* check shutdown condition */ + if (ic_atomic_read_u32(&ic_control_info.shutdown) == 1) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "udp-ic: rx-thread shutting down"); + break; + } + + /* Try to get a buffer */ + if (pkt == NULL) + { + pthread_mutex_lock(&ic_control_info.lock); + pkt = rx_buffer_pool.get(); + pthread_mutex_unlock(&ic_control_info.lock); + + if (pkt == NULL) + { + setRxThreadError(ENOMEM); + continue; + } + } + + if (!skip_poll) + { + /* Do we have inbound traffic to handle ? */ + nfd.fd = UDP_listenerFd; + nfd.events = POLLIN; + + n = poll(&nfd, 1, RX_THREAD_POLL_TIMEOUT); + + if (ic_atomic_read_u32(&ic_control_info.shutdown) == 1) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "udp-ic: rx-thread shutting down"); + break; + } + + if (n < 0) + { + if (errno == EINTR) + continue; + + /* + * ERROR case: if simply break out the loop here, there will + * be a hung here, since main thread will never be waken up, + * and senders will not get responses anymore. + * + * Thus, we set an error flag, and let main thread to report + * an error. + */ + setRxThreadError(errno); + continue; + } + + if (n == 0) + continue; + } + + if (skip_poll || (n == 1 && (nfd.events & POLLIN))) + { + /* we've got something interesting to read */ + /* handle incoming */ + /* ready to read on our socket */ + int read_count = 0; + + struct sockaddr_storage peer; + socklen_t peerlen; + + peerlen = sizeof(peer); + read_count = recvfrom(UDP_listenerFd, (char *) pkt, global_param.Gp_max_packet_size, 0, + (struct sockaddr *) &peer, &peerlen); + + if (ic_atomic_read_u32(&ic_control_info.shutdown) == 1) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "udp-ic: rx-thread shutting down"); + break; + } + + if (IC_DEBUG5 >= session_param.log_min_messages) + LOG(DEBUG5, "received inbound len %d", read_count); + + if (read_count < 0) + { + skip_poll = false; + + if (errno == EWOULDBLOCK || errno == EINTR) + continue; + + LOG(LOG_ERROR, "Interconnect error: recvfrom (%d)", errno); + + /* + * ERROR case: if simply break out the loop here, there will + * be a hung here, since main thread will never be waken up, + * and senders will not get responses anymore. + * + * Thus, we set an error flag, and let main thread to report + * an error. + */ + setRxThreadError(errno); + continue; + } + + if (static_cast(read_count) < sizeof(icpkthdr)) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "Interconnect error: short conn receive (%d)", read_count); + continue; + } + + /* + * when we get a "good" recvfrom() result, we can skip poll() + * until we get a bad one. + */ + skip_poll = true; + + /* length must be >= 0 */ + if (pkt->len < 0) + { + if (IC_DEBUG3 >= session_param.log_min_messages) + LOG(DEBUG3, "received inbound with negative length"); + continue; + } + + if (pkt->len != static_cast(read_count)) + { + if (IC_DEBUG3 >= session_param.log_min_messages) + LOG(DEBUG3, "received inbound packet [%d], short: read %d bytes, pkt->len %d", pkt->seq, read_count, pkt->len); + continue; + } + + /* + * check the CRC of the payload. + */ + if (session_param.gp_interconnect_full_crc) + { + if (!checkCRC(pkt)) + { + ic_atomic_add_fetch_u32((ic_atomic_uint32 *) &ic_statistics.crcErrors, 1); + if (IC_DEBUG2 >= session_param.log_min_messages) + LOG(DEBUG2, "received network data error, dropping bad packet, user data unaffected."); + continue; + } + } + +#ifdef AMS_VERBOSE_LOGGING + logPkt("GOT MESSAGE", pkt); +#endif + + bool wakeup_mainthread = false; + AckSendParam param; + + memset(¶m, 0, sizeof(AckSendParam)); + + /* + * Get the connection for the pkt. + * + * The connection hash table should be locked until finishing the + * processing of the packet to avoid the connection + * addition/removal from the hash table during the mean time. + */ + pthread_mutex_lock(&ic_control_info.lock); + UDPConn *conn = ic_control_info.connHtab.find(pkt); + if (conn != NULL) + { + /* Handling a regular packet */ + if (handleDataPacket(conn, pkt, &peer, &peerlen, ¶m, &wakeup_mainthread)) + pkt = NULL; + ic_statistics.recvPktNum++; + } + else + { + /* + * There may have two kinds of Mismatched packets: a) Past + * packets from previous command after I was torn down b) + * Future packets from current command before my connections + * are built. + * + * The handling logic is to "Ack the past and Nak the future". + */ + if ((pkt->flags & UDPIC_FLAGS_RECEIVER_TO_SENDER) == 0) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "mismatched packet received, seq %d, srcpid %d, dstpid %d, icid %d, sid %d", + pkt->seq, pkt->srcPid, pkt->dstPid, pkt->icId, pkt->sessionId); + +#ifdef AMS_VERBOSE_LOGGING + logPkt("Got a Mismatched Packet", pkt); +#endif + + if (handleMismatch(pkt, &peer, peerlen)) + pkt = NULL; + ic_statistics.mismatchNum++; + } + } + pthread_mutex_unlock(&ic_control_info.lock); + + if (wakeup_mainthread) { + cv.notify_one(); + } + + /* + * real ack sending is after lock release to decrease the lock + * holding time. + */ + if (param.msg.len != 0) + UDPConn::sendAckWithParam(¶m); + } + + /* pthread_yield(); */ + } + + /* Before return, we release the packet. */ + if (pkt) + { + pthread_mutex_lock(&ic_control_info.lock); + rx_buffer_pool.release(pkt); + pkt = NULL; + pthread_mutex_unlock(&ic_control_info.lock); + } + + /* nothing to return */ + return NULL; +} + +/* + * handleMismatch + * If the mismatched packet is from an old connection, we may need to + * send an acknowledgment. + * + * We are called with the receiver-lock held, and we never release it. + * + * For QD: + * 1) Not in hashtable : NAK it/Do nothing + * Causes: a) Start race + * b) Before the entry for the ic instance is inserted, an error happened. + * c) From past transactions: should no happen. + * 2) Active in hashtable : NAK it/Do nothing + * Causes: a) Error reported after the entry is inserted, and connections are + * not inserted to the hashtable yet, and before teardown is called. + * 3) Inactive in hashtable: ACK it (with stop) + * Causes: a) Normal execution: after teardown is called on current command. + * b) Error case, 2a) after teardown is called. + * c) Normal execution: from past history transactions (should not happen). + * + * For QE: + * 1) pkt->id > ic_control_info.ic_instance_id : NAK it/Do nothing + * Causes: a) Start race + * b) Before ic_control_info.ic_instance_id is assigned to correct value, an error happened. + * 2) lastTornIcId < pkt->id == ic_control_info.ic_instance_id: NAK it/Do nothing + * Causes: a) Error reported after ic_control_info.ic_instance_id is set, and connections are + * not inserted to the hashtable yet, and before teardown is called. + * 3) lastTornIcId == pkt->id == ic_control_info.ic_instance_id: ACK it (with stop) + * Causes: a) Normal execution: after teardown is called on current command + * 4) pkt->id < ic_control_info.ic_instance_id: NAK it/Do nothing/ACK it. + * Causes: a) Should not happen. + * + */ +static bool +handleMismatch(icpkthdr *pkt, struct sockaddr_storage *peer, int peer_len) +{ + bool cached = false; + + /* + * we want to ack old packets; but *must* avoid acking connection + * requests: + * + * "ACK the past, NAK the future" explicit NAKs aren't necessary, we just + * don't want to ACK future packets, that confuses everyone. + */ + if (pkt->seq > 0 && pkt->sessionId == session_param.gp_session_id) + { + bool need_ack = false; + uint8 ack_flags = 0; + + /* + * The QD-backends can't use a counter, they've potentially got + * multiple instances (one for each active cursor) + */ + if (global_param.Gp_role == GP_ROLE_DISPATCH_IC) + { + struct CursorICHistoryEntry *p; + + p = rx_control_info.cursorHistoryTable.get(pkt->icId); + if (p) + { + if (p->status == 0) + { + /* Torn down. Ack the past. */ + need_ack = true; + } + else /* p->status == 1 */ + { + /* + * Not torn down yet. It happens when an error + * (out-of-memory, network error...) occurred after the + * cursor entry is inserted into the table in interconnect + * setup process. The peer will be canceled. + */ + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "GOT A MISMATCH PACKET WITH ID %d HISTORY THINKS IT IS ACTIVE", pkt->icId); + return cached; /* ignore, no ack */ + } + } + else + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "GOT A MISMATCH PACKET WITH ID %d HISTORY HAS NO RECORD", pkt->icId); + + /* + * No record means that two possibilities. 1) It is from the + * future. It is due to startup race. We do not ack future + * packets 2) Before the entry for the ic instance is + * inserted, an error happened. We do not ack for this case + * too. The peer will be canceled. + */ + ack_flags = UDPIC_FLAGS_NAK; + need_ack = false; + + if (session_param.gp_interconnect_cache_future_packets) + { + cached = cacheFuturePacket(pkt, peer, peer_len); + } + } + } + /* The QEs get to use a simple counter. */ + else if (global_param.Gp_role == GP_ROLE_EXECUTE_IC) + { + if (ic_control_info.ic_instance_id >= pkt->icId) + { + need_ack = true; + + /* + * We want to "ACK the past, but NAK the future." + * + * handleAck() will retransmit. + */ + if (pkt->seq >= 1 && pkt->icId > rx_control_info.lastTornIcId) + { + ack_flags = UDPIC_FLAGS_NAK; + need_ack = false; + } + } + else + { + /* + * ic_control_info.ic_instance_id < pkt->icId, from the future + */ + if (session_param.gp_interconnect_cache_future_packets) + { + cached = cacheFuturePacket(pkt, peer, peer_len); + } + } + } + + if (need_ack) + { + UDPConn dummyconn(NULL); + char buf[128]; /* numeric IP addresses shouldn't exceed + * about 50 chars, but play it safe */ + + memcpy(&dummyconn.conn_info, pkt, sizeof(icpkthdr)); + dummyconn.peer = *peer; + dummyconn.peer_len = peer_len; + + dummyconn.conn_info.flags |= ack_flags; + + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "ACKING PACKET WITH FLAGS: pkt->seq %d 0x%x [pkt->icId %d last-teardown %d interconnect_id %d]", + pkt->seq, dummyconn.conn_info.flags, pkt->icId, rx_control_info.lastTornIcId, ic_control_info.ic_instance_id); + + format_sockaddr_udp(&dummyconn.peer, buf, sizeof(buf)); + + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "ACKING PACKET TO %s", buf); + + if ((ack_flags & UDPIC_FLAGS_NAK) == 0) + { + ack_flags |= UDPIC_FLAGS_STOP | UDPIC_FLAGS_ACK | UDPIC_FLAGS_CAPACITY | UDPIC_FLAGS_RECEIVER_TO_SENDER; + } + else + { + ack_flags |= UDPIC_FLAGS_RECEIVER_TO_SENDER; + } + + /* + * There are two cases, we may need to send a response to sender + * here. One is start race and the other is receiver becomes idle. + * + * ack_flags here can take two possible values 1) UDPIC_FLAGS_NAK + * | UDPIC_FLAGS_RECEIVER_TO_SENDER (for start race) 2) + * UDPIC_FLAGS_STOP | UDPIC_FLAGS_ACK | UDPIC_FLAGS_CAPACITY | + * UDPIC_FLAGS_RECEIVER_TO_SENDER (for idle receiver) + * + * The final flags in the packet may take some extra bits such as + * 1) UDPIC_FLAGS_STOP 2) UDPIC_FLAGS_EOS 3) UDPIC_FLAGS_CAPACITY + * which are from original packet + */ + dummyconn.sendAck(ack_flags | dummyconn.conn_info.flags, dummyconn.conn_info.seq, dummyconn.conn_info.seq); + } + } + else + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "dropping packet from command-id %d seq %d (my cmd %d)", pkt->icId, pkt->seq, ic_control_info.ic_instance_id); + } + + return cached; +} + +/* + * cacheFuturePacket + * Cache the future packets during the setupUDPIFCInterconnect. + * + * Return true if packet is cached, otherwise false + */ +static bool +cacheFuturePacket(icpkthdr *pkt, struct sockaddr_storage *peer, int peer_len) +{ + UDPConn *conn = ic_control_info.startupCacheHtab.find(pkt); + if (conn == NULL) + { + try { + conn = new UDPConn(NULL); + } catch (const std::bad_alloc & e) { + errno = ENOMEM; + setRxThreadError(errno); + return false; + } + + memset((void *) conn, 0, sizeof(UDPConn)); + memcpy(&conn->conn_info, pkt, sizeof(icpkthdr)); + + conn->pkt_q_capacity = session_param.Gp_interconnect_queue_depth; + conn->pkt_q_size = session_param.Gp_interconnect_queue_depth; + conn->pkt_q = (uint8 **) ic_malloc(session_param.Gp_interconnect_queue_depth * sizeof(uint8 *)); + + if (conn->pkt_q == NULL) + { + /* malloc failed. */ + delete conn; + setRxThreadError(errno); + return false; + } + + /* We only use the array to store cached packets. */ + memset(conn->pkt_q, 0, session_param.Gp_interconnect_queue_depth * sizeof(uint8 *)); + + /* Put connection to the hashtable. */ + if (!ic_control_info.startupCacheHtab.add(conn)) + { + ic_free(conn->pkt_q); + delete conn; + setRxThreadError(errno); + return false; + } + + /* Setup the peer sock information. */ + memcpy(&conn->peer, peer, peer_len); + conn->peer_len = peer_len; + } + + /* + * Reject packets with invalid sequence numbers and packets which have + * been cached before. + */ + if (pkt->seq > conn->pkt_q_size || pkt->seq == 0 || conn->pkt_q[pkt->seq - 1] != NULL) + return false; + + conn->pkt_q[pkt->seq - 1] = (uint8 *) pkt; + rx_buffer_pool.maxCount++; + ic_statistics.startupCachedPktNum++; + + return true; +} + +/* + * cleanupStartupCache + * Clean the startup cache. + */ +static void +cleanupStartupCache() +{ + ConnHtabBin *bin = NULL; + UDPConn *cachedConn = NULL; + icpkthdr *pkt = NULL; + int i = 0; + uint32 j = 0; + + for (i = 0; i < ic_control_info.startupCacheHtab.size; i++) + { + bin = ic_control_info.startupCacheHtab.table[i]; + + while (bin) + { + cachedConn = bin->conn; + + for (j = 0; j < cachedConn->pkt_q_size; j++) + { + pkt = (icpkthdr *) cachedConn->pkt_q[j]; + + if (pkt == NULL) + continue; + + rx_buffer_pool.maxCount--; + rx_buffer_pool.put(pkt); + cachedConn->pkt_q[j] = NULL; + } + bin = bin->next; + ic_control_info.startupCacheHtab.remove(cachedConn); + + /* + * MPP-19981 free the cached connections; otherwise memory leak + * would be introduced. + */ + ic_free(cachedConn->pkt_q); + ic_free(cachedConn); + } + } +} + + +#ifdef USE_ASSERT_CHECKING + +/* The following functions are facility methods for debugging. + * They are quite useful when there are a large number of connections. + * These functions can be called from gdb to output internal information to a file. + */ + +/* + * dumpUnackQueueRing + * Dump an unack queue ring. + */ +static void +dumpUnackQueueRing(const char *fname) +{ + FILE *ofile = fopen(fname, "w+"); + int i; + + fprintf(ofile, "UnackQueueRing: currentTime %lu, idx %d numOutstanding %d numSharedOutstanding %d\n", + unack_queue_ring.currentTime, unack_queue_ring.idx, + unack_queue_ring.numOutStanding, unack_queue_ring.numSharedOutStanding); + fprintf(ofile, "==================================\n"); + for (i = 0; i < UNACK_QUEUE_RING_SLOTS_NUM; i++) + { + if (unack_queue_ring.slots[i].length() > 0) + { + unack_queue_ring.slots[i].dump_to_file(ofile); + } + } + + fclose(ofile); +} + +/* + * dumpConnections + * Dump connections. + */ +void +TransportEntry::dumpConnections(const char *fname) +{ + int i; + uint32 j; + + return; + + FILE *ofile = fopen(fname, "w+"); + + fprintf(ofile, "Entry connections: conn num %d \n", this->numConns); + fprintf(ofile, "==================================\n"); + for (i = 0; i < this->numConns; i++) + { + UDPConn *conn = this->GetConn(i); + + fprintf(ofile, "conns[%d] motNodeId=%d: remoteContentId=%d pid=%d sockfd=%d remote=%s " + "capacity=%d sentSeq=%d receivedAckSeq=%d consumedSeq=%d rtt=%lu" + " dev=%lu deadlockCheckBeginTime=%lu route=%d msgSize=%d msgPos=%p" + " recvBytes=%d tupleCount=%d stillActive=%d stopRequested=%d " + "state=%d\n", + i, this->motNodeId, + conn->remoteContentId, + conn->cdbProc ? conn->cdbProc->pid : 0, + conn->sockfd, + conn->remoteHostAndPort, + conn->capacity, conn->sentSeq, conn->receivedAckSeq, conn->consumedSeq, + conn->rtt, conn->dev, conn->deadlockCheckBeginTime, conn->route, conn->msgSize, conn->msgPos, + conn->recvBytes, conn->tupleCount, conn->stillActive, conn->stopRequested, + conn->state); + fprintf(ofile, "conn_info [%s: seq %d extraSeq %d]: motNodeId %d, crc %d len %d " + "srcContentId %d dstDesContentId %d " + "srcPid %d dstPid %d " + "srcListenerPort %d dstListernerPort %d " + "sendSliceIndex %d recvSliceIndex %d " + "sessionId %d icId %d " + "flags %d\n", + conn->conn_info.flags & UDPIC_FLAGS_RECEIVER_TO_SENDER ? "ACK" : "DATA", + conn->conn_info.seq, conn->conn_info.extraSeq, conn->conn_info.motNodeId, conn->conn_info.crc, conn->conn_info.len, + conn->conn_info.srcContentId, conn->conn_info.dstContentId, + conn->conn_info.srcPid, conn->conn_info.dstPid, + conn->conn_info.srcListenerPort, conn->conn_info.dstListenerPort, + conn->conn_info.sendSliceIndex, conn->conn_info.recvSliceIndex, + conn->conn_info.sessionId, conn->conn_info.icId, + conn->conn_info.flags); + + if (!ic_control_info.isSender) + { + fprintf(ofile, "pkt_q_size=%d pkt_q_head=%d pkt_q_tail=%d pkt_q=%p\n", conn->pkt_q_size, conn->pkt_q_head, conn->pkt_q_tail, conn->pkt_q); + for (j = 0; j < conn->pkt_q_capacity; j++) + { + if (conn->pkt_q != NULL && conn->pkt_q[j] != NULL) + { + icpkthdr *pkt = (icpkthdr *) conn->pkt_q[j]; + + fprintf(ofile, "Packet (pos %d) Info [%s: seq %d extraSeq %d]: motNodeId %d, crc %d len %d " + "srcContentId %d dstDesContentId %d " + "srcPid %d dstPid %d " + "srcListenerPort %d dstListernerPort %d " + "sendSliceIndex %d recvSliceIndex %d " + "sessionId %d icId %d " + "flags %d\n", + j, + pkt->flags & UDPIC_FLAGS_RECEIVER_TO_SENDER ? "ACK" : "DATA", + pkt->seq, pkt->extraSeq, pkt->motNodeId, pkt->crc, pkt->len, + pkt->srcContentId, pkt->dstContentId, + pkt->srcPid, pkt->dstPid, + pkt->srcListenerPort, pkt->dstListenerPort, + pkt->sendSliceIndex, pkt->recvSliceIndex, + pkt->sessionId, pkt->icId, + pkt->flags); + } + } + } + if (ic_control_info.isSender) + { + fprintf(ofile, "sndQueue "); + conn->sndQueue.dump_to_file(ofile); + fprintf(ofile, "unackQueue "); + conn->unackQueue.dump_to_file(ofile); + + dumpUnackQueueRing("/tmp/dumpUnackQueueRing"); + } + fprintf(ofile, "\n"); + } + fclose(ofile); +} +#endif + +/* + * logPkt + * Log a packet. + * + */ +static inline void +logPkt(const char *prefix, icpkthdr *pkt) +{ + LOG(INFO, "%s [%s: seq %d extraSeq %d]: motNodeId %d, crc %d len %d " + "srcContentId %d dstDesContentId %d " + "srcPid %d dstPid %d " + "srcListenerPort %d dstListernerPort %d " + "sendSliceIndex %d recvSliceIndex %d " + "sessionId %d icId %d " + "flags %d ", + prefix, pkt->flags & UDPIC_FLAGS_RECEIVER_TO_SENDER ? "ACK" : "DATA", + pkt->seq, pkt->extraSeq, pkt->motNodeId, pkt->crc, pkt->len, + pkt->srcContentId, pkt->dstContentId, + pkt->srcPid, pkt->dstPid, + pkt->srcListenerPort, pkt->dstListenerPort, + pkt->sendSliceIndex, pkt->recvSliceIndex, + pkt->sessionId, pkt->icId, + pkt->flags); +} + +/* + * Send a dummy packet to interconnect thread to exit poll() immediately + */ +static void +SendDummyPacket(void) +{ + int ret; + const char *dummy_pkt = "stop it"; + int counter; + struct sockaddr_storage dest; + socklen_t dest_len; + + Assert(udp_dummy_packet_sockaddr.ss_family == AF_INET || udp_dummy_packet_sockaddr.ss_family == AF_INET6); + Assert(ICSenderFamily == AF_INET || ICSenderFamily == AF_INET6); + + dest = udp_dummy_packet_sockaddr; + dest_len = (ICSenderFamily == AF_INET) ? sizeof(struct sockaddr_in) : sizeof(struct sockaddr_in6); + + if (ICSenderFamily == AF_INET6) + { +#if defined(__darwin__) + if (udp_dummy_packet_sockaddr.ss_family == AF_INET6) + ConvertIPv6WildcardToLoopback(&dest); +#endif + if (udp_dummy_packet_sockaddr.ss_family == AF_INET) + ConvertToIPv4MappedAddr(&dest, &dest_len); + } + + if (ICSenderFamily == AF_INET && udp_dummy_packet_sockaddr.ss_family == AF_INET6) + { + /* the size of AF_INET6 is bigger than the side of IPv4, so + * converting from IPv6 to IPv4 may potentially not work. */ + LOG(INFO, "sending dummy packet failed: cannot send from AF_INET to receiving on AF_INET6"); + return; + } + + /* + * Send a dummy package to the interconnect listener, try 10 times. + * We don't want to close the socket at the end of this function, since + * the socket will eventually close during the motion layer cleanup. + */ + + counter = 0; + while (counter < 10) + { + counter++; + ret = sendto(ICSenderSocket, dummy_pkt, strlen(dummy_pkt), 0, (struct sockaddr *) &dest, dest_len); + if (ret < 0) + { + if (errno == EINTR || errno == EAGAIN || errno == EWOULDBLOCK) + continue; + else + { + LOG(INFO, "send dummy packet failed, sendto failed: %m"); + return; + } + } + break; + } + + if (counter >= 10) + { + LOG(INFO, "send dummy packet failed, sendto failed with 10 times: %m"); + } +} + +/* + * prepareXmit + * Prepare connection for transmit. + */ +void +UDPConn::prepareXmit() +{ + this->conn_info.len = this->msgSize; + this->conn_info.crc = 0; + + memcpy(this->pBuff, &this->conn_info, sizeof(this->conn_info)); + + /* increase the sequence no */ + this->conn_info.seq++; + + if (session_param.gp_interconnect_full_crc) + { + icpkthdr *pkt = (icpkthdr *)this->pBuff; + addCRC(pkt); + } +} + + /* + * sendtoWithRetry + * Retry sendto logic and send the packets. + */ +static ssize_t +sendtoWithRetry(int socket, const void *message, size_t length, + int flags, const struct sockaddr *dest_addr, + socklen_t dest_len, int retry, const char *errDetail) +{ + int32 n; + int count = 0; + +xmit_retry: + /* + * If given retry count is positive, retry up to the limited times. + * Otherwise, retry for unlimited times until succeed. + */ + if (retry > 0 && ++count > retry) + return n; + n = sendto(socket, message, length, flags, dest_addr, dest_len); + if (n < 0) + { + int save_errno = errno; + + if (errno == EINTR) + goto xmit_retry; + + /* + * EAGAIN: no space ? not an error. + * + * EFAULT: In Linux system call, it only happens when copying a socket + * address into kernel space failed, which is less likely to happen, + * but mocked heavily by our fault injection in regression tests. + */ + if (errno == EAGAIN || errno == EFAULT) + return n; + + /* + * If Linux iptables (nf_conntrack?) drops an outgoing packet, it may + * return an EPERM to the application. This might be simply because of + * traffic shaping or congestion, so ignore it. + */ + if (errno == EPERM) + { + LOG(LOG_ERROR, "Interconnect error writing an outgoing packet: %m, " + "error during sendto() %s", errDetail); + return n; + } + + /* + * If the OS can detect an MTU issue on the host network interfaces, we + * would get EMSGSIZE here. So, bail with a HINT about checking MTU. + */ + if (errno == EMSGSIZE) + { + std::stringstream ss; + ss << "ERROR, Interconnect error writing an outgoing packet: " << strerror(errno) << "error during sendto() call (error:" << save_errno << ", " << errDetail << ")." + << "check if interface MTU is equal across the cluster and lower than gp_max_packet_size" << "\n"; + throw ICNetworkException(ss.str(), __FILE__, __LINE__); + } + + std::stringstream ss; + ss <<"ERROR, Interconnect error writing an outgoing packet: "<seq, pkt->srcPid, pkt->dstPid); +#endif + return; + } +#endif + + Assert(pkt->srcContentId == global_param.segindex); + Assert(pkt->motNodeId == entry_->motNodeId); + LOG(DEBUG3, "UDPConn::sendOnce(): icid: %d, motNodeId: %d, srcSeg: %d, dstSeg: %d, srcPid: %d, dstPid: %d, seq: %d, len: %d, flags: %s", + pkt->icId, pkt->motNodeId, pkt->srcContentId, pkt->dstContentId, pkt->srcPid, pkt->dstPid, pkt->seq, pkt->len, flags2txt(pkt->flags)); + + char errDetail[256]; + snprintf(errDetail, sizeof(errDetail), "For Remote Connection: contentId=%d at %s", + this->remoteContentId, + this->remoteHostAndPort); + n = sendtoWithRetry(this->entry_->txfd, pkt, pkt->len, 0, + (struct sockaddr *) &this->peer, this->peer_len, -1, errDetail); + if (n != int(pkt->len)) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "Interconnect error writing an outgoing packet [seq %d]: short transmit (given %d sent %d) during sendto() call." + "For Remote Connection: contentId=%d at %s", pkt->seq, pkt->len, n, + this->remoteContentId, this->remoteHostAndPort); +#ifdef AMS_VERBOSE_LOGGING + logPkt("PKT DETAILS ", pkt); +#endif + } + return; +} + +void +UDPConn::handleStop() +{ + if (!this->stillActive || !this->stopRequested) + return; + + /* mark buffer empty */ + this->tupleCount = 0; + this->msgSize = sizeof(this->conn_info); + + /* now send our stop-ack EOS */ + this->conn_info.flags |= UDPIC_FLAGS_EOS; + + Assert(this->curBuff != NULL); + + this->pBuff[this->msgSize] = 'S'; + this->msgSize += 1; + + /* now ready to actually send */ + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "handleStopMsgs: node %d route %d, seq %d", + entry_->motNodeId, this->route, this->conn_info.seq); + + /* place it into the send queue */ + this->prepareXmit(); + this->sndQueue.append(this->curBuff); + this->curBuff = NULL; + this->pBuff = NULL; + + /* return all buffers */ + this->sndQueue.release(false); + this->unackQueue.release(session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_CAPACITY_IC ? false : true); + + this->tupleCount = 0; + this->msgSize = sizeof(this->conn_info); + + this->state = mcsEosSent; + this->stillActive = false; + this->stopRequested = false; +} + +/* + * sendBuffers + * Called by sender to send the buffers in the send queue. + * + * Send the buffers in the send queue of the connection if there is capacity left + * and the congestion control condition is satisfied. + * + * Here, we make sure that a connection can have at least one outstanding buffer. + * This is very important for two reasons: + * + * 1) The handling logic of the ack of the outstanding buffer can always send a buffer + * in the send queue. Otherwise, there may be a deadlock. + * 2) This makes sure that any connection can have a minimum bandwidth for data + * sending. + * + * After sending a buffer, the buffer will be placed into both the unack queue and + * the corresponding queue in the unack queue ring. + */ +void +UDPConn::sendBuffers() +{ + while (this->capacity > 0 && this->sndQueue.length() > 0) + { + ICBuffer *buf = NULL; + + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_LOSS_IC) + { + if (this->unackQueue.length() > 0 && + unack_queue_ring.numSharedOutStanding >= (snd_control_info.cwnd - snd_control_info.minCwnd)) + break; + } + + /* for connection setup, we only allow one outstanding packet. */ + if (this->state == mcsSetupOutgoingConnection && this->unackQueue.length() >= 1) + break; + + buf = this->sndQueue.pop(); + + uint64 now = getCurrentTime(); + + buf->sentTime = now; + buf->unackQueueRingSlot = -1; + buf->nRetry = 0; + buf->conn = this; + this->capacity--; + + this->unackQueue.append(buf); + + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_LOSS_IC) + { + unack_queue_ring.numOutStanding++; + if (this->unackQueue.length() > 1) + unack_queue_ring.numSharedOutStanding++; + + putIntoUnackQueueRing(&unack_queue_ring, + buf, + this->computeExpirationPeriod(buf->nRetry), + now); + } + + /* + * Note the place of sendOnce here. If we send before appending it to + * the unack queue and putting it into unack queue ring, and there is + * a network error occurred in the sendOnce function, error message + * will be output. In the time of error message output, interrupts is + * potentially checked, if there is a pending query cancel, it will + * lead to a dangled buffer (memory leak). + */ +#ifdef TRANSFER_PROTOCOL_STATS + trans_proto_stats.update(TPE_DATA_PKT_SEND, buf->pkt); +#endif + + this->sendOnce(buf->pkt); + + ic_statistics.sndPktNum++; + +#ifdef AMS_VERBOSE_LOGGING + logPkt("SEND PKT DETAIL", buf->pkt); +#endif + + this->sentSeq = buf->pkt->seq; + } +} + +/* + * handleDisorderPacket + * Called by rx thread to assemble and send a disorder message. + * + * In current implementation, we limit the number of lost packet sequence numbers + * in the disorder message by the MIN_PACKET_SIZE. There are two reasons here: + * + * 1) The maximal number of lost packet sequence numbers are actually bounded by the + * receive queue depth whose maximal value is very large. Since we share the packet + * receive and ack receive in the background thread, the size of disorder should be + * also limited by the max packet size. + * 2) We can use Gp_max_packet_size here to limit the number of lost packet sequence numbers. + * But considering we do not want to let senders send many packets when getting a lost + * message. Here we use MIN_PACKET_SIZE. + * + * + * the format of a disorder message: + * I) pkt header + * - seq -> packet sequence number that triggers the disorder message + * - extraSeq -> the largest seq of the received packets + * - flags -> UDPIC_FLAGS_DISORDER + * - len -> sizeof(icpkthdr) + sizeof(uint32) * (lost pkt count) + * II) content + * - an array of lost pkt sequence numbers (uint32) + * + */ +void +UDPConn::handleDisorderPacket(int pos, uint32 tailSeq, icpkthdr *pkt) +{ + int start = 0; + uint32 lostPktCnt = 0; + uint32 *curSeq = (uint32 *) &rx_control_info.disorderBuffer[1]; + uint32 maxSeqs = MAX_SEQS_IN_DISORDER_ACK; + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "PROCESS_DISORDER PKT BEGIN:"); +#endif + + start = this->pkt_q_tail; + + while (start != pos && lostPktCnt < maxSeqs) + { + if (this->pkt_q[start] == NULL) + { + *curSeq = tailSeq; + lostPktCnt++; + curSeq++; + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "PROCESS_DISORDER add seq [%d], lostPktCnt %d", *curSeq, lostPktCnt); +#endif + } + + tailSeq++; + start = (start + 1) % this->pkt_q_capacity; + } + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "PROCESS_DISORDER PKT END:"); +#endif + + /* when reaching here, cnt must not be 0 */ + this->sendDisorderAck(pkt->seq, this->conn_info.seq - 1, lostPktCnt); +} + +/* + * handleAckForDisorderPkt + * Called by sender to deal with acks for disorder packet. + */ +bool +UDPConn::handleAckForDisorderPkt(icpkthdr *pkt) +{ + ICBufferLink *link = NULL; + ICBuffer *buf = NULL; + ICBufferLink *next = NULL; + uint64 now = getCurrentTime(); + uint32 *curLostPktSeq = 0; + int lostPktCnt = 0; + static uint32 times = 0; + static uint32 lastSeq = 0; + bool shouldSendBuffers = false; + + if (pkt->extraSeq != lastSeq) + { + lastSeq = pkt->extraSeq; + times = 0; + return false; + } + else + { + times++; + if (times != 2) + return false; + } + + curLostPktSeq = (uint32 *) &pkt[1]; + lostPktCnt = (pkt->len - sizeof(icpkthdr)) / sizeof(uint32); + + /* + * Resend all the missed packets and remove received packets from queues + */ + + link = this->unackQueue.first(); + buf = GET_ICBUFFER_FROM_PRIMARY(link); + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "DISORDER: pktlen %d cnt %d pktseq %d first loss %d buf %p", + pkt->len, lostPktCnt, pkt->seq, *curLostPktSeq, buf); + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + { + this->unackQueue.icBufferListLog(); + this->sndQueue.icBufferListLog(); + } +#endif + + /* + * iterate the unack queue + */ + while (!this->unackQueue.is_head(link) && buf->pkt->seq <= pkt->seq && lostPktCnt > 0) + { +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "DISORDER: bufseq %d curlostpkt %d cnt %d buf %p pkt->seq %d", + buf->pkt->seq, *curLostPktSeq, lostPktCnt, buf, pkt->seq); +#endif + + if (buf->pkt->seq == pkt->seq) + { + this->handleAckedPacket(buf, now); + shouldSendBuffers = true; + break; + } + + if (buf->pkt->seq == *curLostPktSeq) + { + /* this is a lost packet, retransmit */ + + buf->nRetry++; + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_LOSS_IC) + { + ICBufferList *alist = &unack_queue_ring.slots[buf->unackQueueRingSlot]; + buf = alist->remove(buf); + putIntoUnackQueueRing(&unack_queue_ring, buf, + this->computeExpirationPeriod(buf->nRetry), now); + } +#ifdef TRANSFER_PROTOCOL_STATS + trans_proto_stats.update(TPE_DATA_PKT_SEND, buf->pkt); +#endif + + Assert(this == buf->conn); + this->sendOnce(buf->pkt); + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "RESEND a buffer for DISORDER: seq %d", buf->pkt->seq); + logPkt("DISORDER RESEND DETAIL ", buf->pkt); +#endif + + ic_statistics.retransmits++; + curLostPktSeq++; + lostPktCnt--; + + link = link->next; + buf = GET_ICBUFFER_FROM_PRIMARY(link); + } + else if (buf->pkt->seq < *curLostPktSeq) + { + /* remove packet already received. */ + + next = link->next; + this->handleAckedPacket(buf, now); + shouldSendBuffers = true; + link = next; + buf = GET_ICBUFFER_FROM_PRIMARY(link); + } + else /* buf->pkt->seq > *curPktSeq */ + { + /* + * this case is introduced when the disorder message tell you a + * pkt is lost. But when we handle this message, a message (for + * example, duplicate ack, or another disorder message) arriving + * before this message already removed the pkt. + */ + curLostPktSeq++; + lostPktCnt--; + } + } + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_LOSS_IC) + { + snd_control_info.ssthresh = Max(snd_control_info.cwnd / 2, snd_control_info.minCwnd); + snd_control_info.cwnd = snd_control_info.ssthresh; + } +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "After DISORDER: sndQ %d unackQ %d", this->sndQueue.length(), this->unackQueue.length()); + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + { + this->unackQueue.icBufferListLog(); + this->sndQueue.icBufferListLog(); + } +#endif + + return shouldSendBuffers; +} + +/* + * handleAckForDuplicatePkt + * Called by sender to deal with acks for duplicate packet. + * + */ +bool +UDPConn::handleAckForDuplicatePkt(icpkthdr *pkt) +{ + ICBufferLink *link = NULL; + ICBuffer *buf = NULL; + ICBufferLink *next = NULL; + uint64 now = getCurrentTime(); + bool shouldSendBuffers = false; + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "RESEND the unacked buffers in the queue due to %s", pkt->len == 0 ? "PROCESS_START_RACE" : "DISORDER"); +#endif + + if (pkt->seq <= pkt->extraSeq) + { + /* Indicate a bug here. */ + LOG(LOG_ERROR, "invalid duplicate message: seq %d extraSeq %d", pkt->seq, pkt->extraSeq); + return false; + } + + link = this->unackQueue.first(); + buf = GET_ICBUFFER_FROM_PRIMARY(link); + + /* deal with continuous pkts */ + while (!this->unackQueue.is_head(link) && (buf->pkt->seq <= pkt->extraSeq)) + { + next = link->next; + this->handleAckedPacket(buf, now); + shouldSendBuffers = true; + link = next; + buf = GET_ICBUFFER_FROM_PRIMARY(link); + } + + /* deal with the single duplicate packet */ + while (!this->unackQueue.is_head(link) && buf->pkt->seq <= pkt->seq) + { + next = link->next; + if (buf->pkt->seq == pkt->seq) + { + this->handleAckedPacket(buf, now); + shouldSendBuffers = true; + break; + } + link = next; + buf = GET_ICBUFFER_FROM_PRIMARY(link); + } + + return shouldSendBuffers; +} + +/* + * checkNetworkTimeout + * check network timeout case. + */ +void +UDPConn::checkNetworkTimeout(ICBuffer *buf, uint64 now, bool *networkTimeoutIsLogged) +{ + /* + * Using only the time to first sent time to decide timeout is not enough, + * since there is a possibility the sender process is not scheduled or + * blocked by OS for a long time. In this case, only a few times are + * tried. Thus, the GUC Gp_interconnect_min_retries_before_timeout is + * added here. + */ + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC && + buf->nRetry % session_param.Gp_interconnect_debug_retry_interval == 0) + { + LOG(INFO, "resending packet (seq %d) to %s (pid %d cid %d) with %d retries in %lu seconds", + buf->pkt->seq, buf->conn->remoteHostAndPort, buf->pkt->dstPid, buf->pkt->dstContentId, buf->nRetry, + (now - buf->sentTime) / 1000 / 1000); + } + + if ((buf->nRetry > session_param.Gp_interconnect_min_retries_before_timeout) && + (now - buf->sentTime) > ((uint64) session_param.Gp_interconnect_transmit_timeout * 1000 * 1000)) + { + std::stringstream ss; + ss <<"ERROR, interconnect encountered a network error, please check your network"<< + "Failed to send packet (seq "<pkt->seq<<") to "<conn->remoteHostAndPort<< + " (pid "<pkt->dstPid<<" cid "<pkt->dstContentId<<") after "<nRetry<< + " retries in "<nRetry >= session_param.Gp_interconnect_min_retries_before_timeout) && !(*networkTimeoutIsLogged)) + { + LOG(WARNING, "interconnect may encountered a network error, please check your network" + "Failed to send packet (seq %d) to %s (pid %d cid %d) after %d retries.", + buf->pkt->seq, buf->conn->remoteHostAndPort, buf->pkt->dstPid, buf->pkt->dstContentId, buf->nRetry); + *networkTimeoutIsLogged = true; + } +} + +/* + * checkExpiration + * Check whether packets expire. If a packet expires, resend the packet, + * and adjust its position in the unack queue ring. + * + */ +void +UDPConn::checkExpiration(ICChunkTransportState *transportStates, uint64 now) +{ + /* check for expiration */ + int count = 0; + int retransmits = 0; + UDPConn *currBuffConn = NULL; + + Assert(unack_queue_ring.currentTime != 0); + while (now >= (unack_queue_ring.currentTime + TIMER_SPAN) && count++ < UNACK_QUEUE_RING_SLOTS_NUM) + { + /* expired, need to resend them */ + ICBuffer *curBuf = NULL; + + while (true) + { + ICBufferList *alist = &unack_queue_ring.slots[unack_queue_ring.idx]; + curBuf = alist->pop(); + if (curBuf == NULL) + break; + UDPConn *conn = static_cast(curBuf->conn); + curBuf->nRetry++; + putIntoUnackQueueRing(&unack_queue_ring, + curBuf, + conn->computeExpirationPeriod(curBuf->nRetry), now); + +#ifdef TRANSFER_PROTOCOL_STATS + trans_proto_stats.update(TPE_DATA_PKT_SEND, curBuf->pkt); +#endif + + conn->sendOnce(curBuf->pkt); + + currBuffConn = conn; + + retransmits++; + ic_statistics.retransmits++; + currBuffConn->stat_count_resent++; + currBuffConn->stat_max_resent = Max(currBuffConn->stat_max_resent, + currBuffConn->stat_count_resent); + + UDPConn::checkNetworkTimeout(curBuf, now, &transportStates->networkTimeoutIsLogged); + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "RESEND pkt with seq %d (retry %d, rtt %lu) to route %d", + curBuf->pkt->seq, curBuf->nRetry, currBuffConn->rtt, currBuffConn->route); + logPkt("RESEND PKT in checkExpiration", curBuf->pkt); +#endif + } + + unack_queue_ring.currentTime += TIMER_SPAN; + unack_queue_ring.idx = (unack_queue_ring.idx + 1) % (UNACK_QUEUE_RING_SLOTS_NUM); + } + + /* + * deal with case when there is a long time this function is not called. + */ + unack_queue_ring.currentTime = now - (now % TIMER_SPAN); + if (retransmits > 0) + { + snd_control_info.ssthresh = Max(snd_control_info.cwnd / 2, snd_control_info.minCwnd); + snd_control_info.cwnd = snd_control_info.minCwnd; + } +} + +/* + * checkDeadlock + * Check whether deadlock occurs on a connection. + * + * What this function does is to send a status query message to rx thread when + * the connection has not received any acks for some time. This is to avoid + * potential deadlock when there are continuous ack losses. Packet resending + * logic does not help avoiding deadlock here since the packets in the unack + * queue may already been removed when the sender knows that they have been + * already buffered in the receiver side queue. + * + * Some considerations on deadlock check time period: + * + * Potential deadlock occurs rarely. According to our experiments on various + * workloads and hardware. It occurred only when fault injection is enabled + * and a large number packets and acknowledgments are discarded. Thus, here we + * use a relatively large deadlock check period. + * + */ +void +UDPConn::checkDeadlock() +{ + uint64 deadlockCheckTime; + + if (this->unackQueue.length() == 0 && this->capacity == 0 && this->sndQueue.length() > 0) + { + /* we must have received some acks before deadlock occurs. */ + Assert(this->deadlockCheckBeginTime > 0); + +#ifdef USE_ASSERT_CHECKING + if (udp_testmode) + { + deadlockCheckTime = 100000; + } + else +#endif + { + deadlockCheckTime = DEADLOCK_CHECKING_TIME; + } + + uint64 now = getCurrentTime(); + + /* request the capacity to avoid the deadlock case */ + if (((now - ic_control_info.lastDeadlockCheckTime) > deadlockCheckTime) && + ((now - this->deadlockCheckBeginTime) > deadlockCheckTime)) + { + this->sendStatusQueryMessage(this->conn_info.seq - 1); + ic_control_info.lastDeadlockCheckTime = now; + ic_statistics.statusQueryMsgNum++; + + /* check network error. */ + if ((now - this->deadlockCheckBeginTime) > ((uint64) session_param.Gp_interconnect_transmit_timeout * 1000 * 1000)) + { + std::stringstream ss; + ss <<"ERROR, interconnect encountered a network error, please check your network"<< + "Did not get any response from "<stat_count_resent++; + this->stat_max_resent = Max(this->stat_max_resent, this->stat_count_resent); +} + +/* + * checkExpirationCapacityFC + * Check expiration for capacity based flow control method. + */ +void +UDPConn::checkExpirationCapacityFC(int timeout) +{ + if (this->unackQueue.length() == 0) + return; + + uint64 now = getCurrentTime(); + uint64 elapsed = now - ic_control_info.lastPacketSendTime; + + if (elapsed >= ((uint64) timeout * 1000)) + { + ICBufferLink *bufLink = this->unackQueue.first(); + ICBuffer *buf = GET_ICBUFFER_FROM_PRIMARY(bufLink); + + Assert(this == buf->conn); + this->sendOnce(buf->pkt); + + buf->nRetry++; + ic_control_info.lastPacketSendTime = now; + + this->updateRetransmitStatistics(); + checkNetworkTimeout(buf, now, &entry_->state->networkTimeoutIsLogged); + } +} + +/* + * checkExceptions + * Check exceptions including packet expiration, deadlock, bg thread error, NIC failure... + * Caller should start from 0 with retry, so that the expensive check for deadlock and + * QD connection can be avoided in a healthy state. + */ +void +UDPConn::checkExceptions(int retry, int timeout) +{ + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_CAPACITY_IC + /* || conn->state == mcsSetupOutgoingConnection */ ) + { + this->checkExpirationCapacityFC(timeout); + } + + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_LOSS_IC) + { + uint64 now = getCurrentTime(); + + if (now - ic_control_info.lastExpirationCheckTime > uint64(TIMER_CHECKING_PERIOD)) + { + UDPConn::checkExpiration(this->entry_->state, now); + ic_control_info.lastExpirationCheckTime = now; + } + } + + if ((retry & 0x3) == 2) + { + this->checkDeadlock(); + + checkRxThreadError(); + CHECK_INTERRUPTS(this->entry_->state); + } + + /* + * 1. NIC on master (and thus the QD connection) may become bad, check it. + * 2. Postmaster may become invalid, check it + * + * We check modulo 2 to correlate with the deadlock check above at the + * initial iteration. + */ + if ((retry & 0x3f) == 2) + { + checkQDConnectionAlive(); + CHECK_POSTMASTER_ALIVE(); + } +} + +/* + * computeTimeout + * Compute timeout value in ms. + */ +int +UDPConn::computeTimeout(int retry) +{ + if (this->unackQueue.length() == 0) + return TIMER_CHECKING_PERIOD; + + ICBufferLink *bufLink = this->unackQueue.first(); + ICBuffer *buf = GET_ICBUFFER_FROM_PRIMARY(bufLink); + + if (buf->nRetry == 0 && retry == 0) + return 0; + + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_LOSS_IC) + return TIMER_CHECKING_PERIOD; + + /* for capacity based flow control */ + return TIMEOUT(buf->nRetry); +} + +/* + * UDPConn::Send + * is used to send a tcItem to a single destination. Tuples often are + * *very small* we aggregate in our local buffer before sending into the kernel. + * + * PARAMETERS + * conn - UDPConn that the tcItem is to be sent to. + * tcItem - message to be sent. + * motionId - Node Motion Id. + */ +void +UDPConn::Send(DataBlock *data) +{ + int length = data->len; + int retry = 0; + bool doCheckExpiration = false; + bool gotStops = false; + + Assert(this->msgSize > 0); + +#ifdef AMS_VERBOSE_LOGGING + LOG(DEBUG3, "UDPConn::Send(): msgSize %d this chunk length %d this seq %d", + this->msgSize, data->len, this->conn_info.seq); +#endif + + if (this->msgSize + length <= global_param.Gp_max_packet_size) + { + memcpy(this->pBuff + this->msgSize, data->pos, data->len); + this->msgSize += length; + + this->tupleCount++; + return; + } + + /* prepare this for transmit */ + ic_statistics.totalCapacity += this->capacity; + ic_statistics.capacityCountingTime++; + + /* try to send it */ + this->prepareXmit(); + this->sndQueue.append(this->curBuff); + this->sendBuffers(); + + /* get a new buffer */ + this->curBuff = NULL; + this->pBuff = NULL; + + uint64 now = getCurrentTime(); + + if (session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_CAPACITY_IC) + doCheckExpiration = false; + else + doCheckExpiration = (now - ic_control_info.lastExpirationCheckTime) > MAX_TIME_NO_TIMER_CHECKING ? true : false; + + ic_control_info.lastPacketSendTime = 0; + this->deadlockCheckBeginTime = now; + + while (doCheckExpiration || (this->curBuff = snd_buffer_pool.get(this)) == NULL) + { + int timeout = (doCheckExpiration ? 0 : this->computeTimeout(retry)); + + if (this->entry_->pollAcks(timeout)) + { + bool rs = this->entry_->handleAcks(); + if (rs) + { + /* + * We make sure that we deal with the stop messages only after + * we get a buffer. Otherwise, if the stop message is not for + * this connection, this will lead to an error for the + * following data sending of this connection. + */ + gotStops = true; + } + } + this->checkExceptions(retry++, timeout); + doCheckExpiration = false; + } + + this->pBuff = (uint8 *) this->curBuff->pkt; + + if (gotStops) + { + /* handling stop message will make some connection not active anymore */ + this->entry_->handleStopMsgs(); + + if (!this->stillActive) + return; + } + + /* reinitialize connection */ + this->tupleCount = 0; + this->msgSize = sizeof(this->conn_info); + + /* now we can copy the input to the new buffer */ + memcpy(this->pBuff + this->msgSize, data->pos, data->len); + this->msgSize += length; + + this->tupleCount++; +} + +/* + * C++ implement for udp protocol. + */ +UDPConn::UDPConn(TransportEntry *entry) +{ + /* the field of MotionConn */ + this->sockfd = -1; + this->pBuff = nullptr; + this->msgSize = 0; + this->msgPos = nullptr; + this->recvBytes = 0; + this->tupleCount = 0; + this->stillActive = false; + this->stopRequested = false; + this->cdbProc = nullptr; + this->remoteContentId = -1; + this->remoteHostAndPort[0] = '\0'; + this->opaque_data = nullptr; + this->sent_record_typmod = 0; + + /* the field of UDPConn */ + this->capacity = -1; + this->sentSeq = 0; + this->receivedAckSeq = 0; + this->consumedSeq = 0; + this->rtt = 0; + this->dev = 0; + this->deadlockCheckBeginTime = -1; + this->curBuff = nullptr; + this->route = 0; + this->peer_len = 0; + this->pkt_q_capacity = 0; + this->pkt_q_size = 0; + this->pkt_q_head = -1; + this->pkt_q_tail = -1; + this->pkt_q = nullptr; + this->stat_total_ack_time = 0; + this->stat_count_acks = 0; + this->stat_max_ack_time = 0; + this->stat_min_ack_time = 0; + this->stat_count_resent = 0; + this->stat_max_resent = 0; + this->stat_count_dropped = 0; + + this->state = mcsNull; + this->sockfd = -1; + this->msgSize = 0; + this->tupleCount = 0; + this->stillActive = false; + this->stopRequested = false; + this->cdbProc = NULL; + this->opaque_data = NULL; + this->sent_record_typmod = 0; + + /* + * "UDPConn dummyconn(NULL)" will be called by handleMismatch() in rx thread, + * it will lead to the error: "palloc called from thread". So code below should + * be called in MakeSendEntry() and MakeRecvEntry(); + * if (global_param.createOpaqueDataCallback) + * this->opaque_data = global_param.createOpaqueDataCallback(); + */ + + this->entry_ = entry; +} + +UDPConn* +TransportEntry::GetConn(int index) +{ + Assert(index >= 0); + + if (index >= 0 && static_cast(index) < this->conns_.size()) + return this->conns_[index].get(); + + std::stringstream ss; + ss << "invalid index for conn, index: " << index << ", conn size: " << conns_.size(); + throw ICInvalidIndex(ss.str(), __FILE__, __LINE__); +} + +/* + * aggregateStatistics + * aggregate statistics. + */ +void +TransportEntry::aggregateStatistics() +{ + /* + * We first clear the stats, and then compute new stats by aggregating the + * stats from each connection. + */ + this->stat_total_ack_time = 0; + this->stat_count_acks = 0; + this->stat_max_ack_time = 0; + this->stat_min_ack_time = ~((uint64) 0); + this->stat_count_resent = 0; + this->stat_max_resent = 0; + this->stat_count_dropped = 0; + + Assert(this->numConns == static_cast(this->conns_.size())); + for (int connNo = 0; connNo < this->numConns; connNo++) + { + UDPConn *conn = this->GetConn(connNo); + + this->stat_total_ack_time += conn->stat_total_ack_time; + this->stat_count_acks += conn->stat_count_acks; + this->stat_max_ack_time = Max(this->stat_max_ack_time, conn->stat_max_ack_time); + this->stat_min_ack_time = Min(this->stat_min_ack_time, conn->stat_min_ack_time); + this->stat_count_resent += conn->stat_count_resent; + this->stat_max_resent = Max(this->stat_max_resent, conn->stat_max_resent); + this->stat_count_dropped += conn->stat_count_dropped; + } +} + +/* + * handleAck + * handle acks incoming from our upstream peers. + * + * if we receive a stop message, return true (caller will clean up). + */ +bool +TransportEntry::handleAcks() +{ + bool ret = false; + UDPConn *ackConn = NULL; + int n; + + struct sockaddr_storage peer; + socklen_t peerlen; + + struct icpkthdr *pkt = snd_control_info.ackBuffer; + bool shouldSendBuffers = false; + + for (;;) + { + + /* ready to read on our socket ? */ + peerlen = sizeof(peer); + n = recvfrom(this->txfd, (char *) pkt, MIN_PACKET_SIZE, 0, + (struct sockaddr *) &peer, &peerlen); + + if (n < 0) + { + if (errno == EWOULDBLOCK) /* had nothing to read. */ + { + this->aggregateStatistics(); + return ret; + } + + CHECK_INTERRUPTS(this->state); + + if (errno == EINTR) + continue; + + throw ICNetworkException("ERROR, interconnect error waiting for peer ack, During recvfrom() call.", __FILE__, __LINE__); + } + else if (n < int(sizeof(struct icpkthdr))) + { + continue; + } + else if (n != int(pkt->len)) + { + continue; + } + + /* + * check the CRC of the payload. + */ + if (session_param.gp_interconnect_full_crc) + { + if (!checkCRC(pkt)) + { + ic_atomic_add_fetch_u32((ic_atomic_uint32 *) &ic_statistics.crcErrors, 1); + if (IC_DEBUG2 >= session_param.log_min_messages) + LOG(DEBUG2, "received network data error, dropping bad packet, user data unaffected."); + continue; + } + } + + /* + * read packet, is this the ack we want ? + */ + if (pkt->srcContentId == global_param.segindex && + pkt->srcPid == global_param.MyProcPid && + pkt->srcListenerPort == (UDP2_GetListenPortUDP()) && + pkt->sessionId == session_param.gp_session_id && + pkt->icId == this->state->icInstanceId) + { + Assert(pkt->motNodeId == motNodeId); + LOG(DEBUG3, "TransportEntry::handleAcks(): icid: %d, motNodeId: %d, srcSeg: %d, dstSeg: %d, srcPid: %d, dstPid: %d, seq: %d, extraSeq: %d, len: %d, flags: %s", + pkt->icId, pkt->motNodeId, pkt->srcContentId, pkt->dstContentId, pkt->srcPid, pkt->dstPid, pkt->seq, pkt->extraSeq, pkt->len, flags2txt(pkt->flags)); + + /* + * packet is for me. Note here we do not need to get a connection + * lock here, since background rx thread only read the hash table. + */ + ackConn = ic_control_info.connHtab.find(pkt); + if (ackConn == NULL) + { + LOG(INFO, "Received ack for unknown connection (flags 0x%x)", pkt->flags); + continue; + } + + ackConn->stat_count_acks++; + ic_statistics.recvAckNum++; + + uint64 now = getCurrentTime(); + + ackConn->deadlockCheckBeginTime = now; + + /* + * We simply disregard pkt losses (NAK) due to process start race + * (that is, sender is started earlier than receiver. rx + * background thread may receive packets when connections are not + * created yet). + * + * Another option is to resend the packet immediately, but + * experiments do not show any benefits. + */ + if (pkt->flags & UDPIC_FLAGS_NAK) + continue; + + while (true) + { + if (pkt->flags & UDPIC_FLAGS_CAPACITY) + { + if (pkt->extraSeq > ackConn->consumedSeq) + { + ackConn->capacity += pkt->extraSeq - ackConn->consumedSeq; + ackConn->consumedSeq = pkt->extraSeq; + shouldSendBuffers = true; + } + } + else if (pkt->flags & UDPIC_FLAGS_DUPLICATE) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "GOTDUPACK [seq %d] from route %d; srcpid %d dstpid %d cmd %d flags 0x%x connseq %d", + pkt->seq, ackConn->route, pkt->srcPid, pkt->dstPid, pkt->icId, pkt->flags, ackConn->conn_info.seq); + + shouldSendBuffers |= (ackConn->handleAckForDuplicatePkt(pkt)); + break; + } + else if (pkt->flags & UDPIC_FLAGS_DISORDER) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "GOTDISORDER [seq %d] from route %d; srcpid %d dstpid %d cmd %d flags 0x%x connseq %d", + pkt->seq, ackConn->route, pkt->srcPid, pkt->dstPid, pkt->icId, pkt->flags, ackConn->conn_info.seq); + + shouldSendBuffers |= (ackConn->handleAckForDisorderPkt(pkt)); + break; + } + + /* + * don't get out of the loop if pkt->seq equals to + * ackConn->receivedAckSeq, need to check UDPIC_FLAGS_STOP + * flag + */ + if (pkt->seq < ackConn->receivedAckSeq) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "ack with bad seq?! expected (%d, %d] got %d flags 0x%x, capacity %d consumedSeq %d", + ackConn->receivedAckSeq, ackConn->sentSeq, pkt->seq, pkt->flags, ackConn->capacity, ackConn->consumedSeq); + break; + } + + /* haven't gotten a stop request, maybe this is one ? */ + if ((pkt->flags & UDPIC_FLAGS_STOP) && !ackConn->stopRequested && ackConn->stillActive) + { +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "got ack with stop; srcpid %d dstpid %d cmd %d flags 0x%x pktseq %d connseq %d", + pkt->srcPid, pkt->dstPid, pkt->icId, pkt->flags, pkt->seq, ackConn->conn_info.seq); +#endif + ackConn->stopRequested = true; + ackConn->conn_info.flags |= UDPIC_FLAGS_STOP; + ret = true; + /* continue to deal with acks */ + } + + if (pkt->seq == ackConn->receivedAckSeq) + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "ack with bad seq?! expected (%d, %d] got %d flags 0x%x, capacity %d consumedSeq %d", + ackConn->receivedAckSeq, ackConn->sentSeq, pkt->seq, pkt->flags, ackConn->capacity, ackConn->consumedSeq); + break; + } + + /* deal with a regular ack. */ + if (pkt->flags & UDPIC_FLAGS_ACK) + { + ICBufferLink *link = NULL; + ICBufferLink *next = NULL; + ICBuffer *buf = NULL; + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "GOTACK [seq %d] from route %d; srcpid %d dstpid %d cmd %d flags 0x%x connseq %d", + pkt->seq, ackConn->route, pkt->srcPid, pkt->dstPid, pkt->icId, pkt->flags, ackConn->conn_info.seq); +#endif + + link = ackConn->unackQueue.first(); + buf = GET_ICBUFFER_FROM_PRIMARY(link); + + while (!ackConn->unackQueue.is_head(link) && buf->pkt->seq <= pkt->seq) + { + next = link->next; + ackConn->handleAckedPacket(buf, now); + shouldSendBuffers = true; + link = next; + buf = GET_ICBUFFER_FROM_PRIMARY(link); + } + } + break; + } + + /* + * When there is a capacity increase or some outstanding buffers + * removed from the unack queue ring, we should try to send + * buffers for the connection. Even when stop is received, we + * still send here, since in STOP/EOS race case, we may have been + * in EOS sending logic and will not check stop message. + */ + if (shouldSendBuffers) + ackConn->sendBuffers(); + } + else + { + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "handleAck: not the ack we're looking for (flags 0x%x)...mot(%d) content(%d:%d) srcpid(%d:%d) " + "dstpid(%d) srcport(%d:%d) dstport(%d) sess(%d:%d) cmd(%d:%d)", + pkt->flags, pkt->motNodeId, pkt->srcContentId, global_param.segindex, + pkt->srcPid, global_param.MyProcPid, pkt->dstPid, pkt->srcListenerPort, + (UDP2_GetListenPortUDP()), pkt->dstListenerPort, pkt->sessionId, session_param.gp_session_id, + pkt->icId, this->state->icInstanceId); + } + } + + return ret; +} + +/* + * handleStopMsgs + * handle stop messages. + * + */ +void +TransportEntry::handleStopMsgs() +{ + int i = 0; + +#ifdef AMS_VERBOSE_LOGGING + LOG(DEBUG3, "handleStopMsgs: node %d", this->motNodeId); +#endif + + while (i < this->numConns) + { + UDPConn *conn = this->GetConn(i); + +#ifdef AMS_VERBOSE_LOGGING + LOG(DEBUG3, "handleStopMsgs: node %d route %d %s %s", this->motNodeId, conn->route, + (conn->stillActive ? "active" : "NOT active"), (conn->stopRequested ? "stop requested" : "")); + LOG(DEBUG3, "handleStopMsgs: node %d route %d msgSize %d", + this->motNodeId, conn->route, conn->msgSize); +#endif + + /* + * MPP-2427: we're guaranteed to have recently flushed, but this might + * not be empty (if we got a stop on a buffer that wasn't the one we + * were sending) ... empty it first so the outbound buffer is empty + * when we get here. + */ + conn->handleStop(); + + i++; + + if (i == this->numConns) + { + if (this->pollAcks(0)) + { + bool rs = this->handleAcks(); + if (rs) + { + /* more stops found, loop again. */ + i = 0; + continue; + } + } + } + } +} + +/* + * pollAcks + * Timeout polling of acks + */ +bool +TransportEntry::pollAcks(int timeout) +{ + struct pollfd nfd; + int n; + + nfd.fd = this->txfd; + nfd.events = POLLIN; + + n = poll(&nfd, 1, timeout); + if (n < 0) + { + CHECK_INTERRUPTS(this->state); + + if (errno == EINTR) + return false; + + throw ICNetworkException("ERROR, interconnect error waiting for peer ack During poll() call.", __FILE__, __LINE__); + /* not reached */ + } + + if (n == 0) /* timeout */ + { + return false; + } + + /* got an ack to handle (possibly a stop message) */ + if (n == 1 && (nfd.events & POLLIN)) + { + return true; + } + + return false; +} + +std::unique_ptr +TransportEntry::MakeRecvEntry(CChunkTransportStateImpl *state, + int icid, + ICExecSlice *sendSlice, + ICExecSlice *recvSlice) +{ + int incoming_count = 0; + int expectedTotalIncoming = 0; + + Assert(sendSlice->sliceIndex > 0); + Assert(recvSlice->sliceIndex >= 0); + + int motNodeID = sendSlice->sliceIndex; + int numConns = sendSlice->numPrimaryProcesses; + + std::unique_ptr pEntry = + std::make_unique(state, motNodeID, numConns, sendSlice, recvSlice); + + pEntry->conns_.resize(numConns); + for (int i = 0; i < numConns; ++i) + { + pEntry->conns_[i] = std::make_unique(pEntry.get()); + UDPConn *conn = pEntry->conns_[i].get(); + + Assert(i < sendSlice->numPrimaryProcesses); + ICCdbProcess *cdbProc = sendSlice->primaryProcesses + i; + if (cdbProc->valid) + { + conn->cdbProc = cdbProc; + + expectedTotalIncoming++; + + /* rx_buffer_queue */ + conn->pkt_q_capacity = session_param.Gp_interconnect_queue_depth; + conn->pkt_q_size = 0; + conn->pkt_q_head = 0; + conn->pkt_q_tail = 0; + + if (global_param.simpleFaultInjectorCallback) + global_param.simpleFaultInjectorCallback("interconnect_setup_palloc"); + + conn->pkt_q = (uint8 **) ic_malloc0(conn->pkt_q_capacity * sizeof(uint8 *)); + + /* update the max buffer count of our rx buffer pool. */ + rx_buffer_pool.maxCount += conn->pkt_q_capacity; + + /* + * connection header info (defining characteristics of this + * connection) + */ + memset(&conn->conn_info, 0, sizeof(conn->conn_info)); + conn->route = i; + + conn->conn_info.seq = 1; + conn->stillActive = true; + + incoming_count++; + + conn->conn_info.motNodeId = pEntry->motNodeId; + conn->conn_info.recvSliceIndex = recvSlice->sliceIndex; + conn->conn_info.sendSliceIndex = sendSlice->sliceIndex; + + conn->conn_info.srcContentId = conn->cdbProc->contentid; + conn->conn_info.dstContentId = global_param.segindex; + + conn->conn_info.srcListenerPort = conn->cdbProc->listenerPort; + conn->conn_info.dstListenerPort = UDP2_GetListenPortUDP(); + conn->conn_info.srcPid = conn->cdbProc->pid; + conn->conn_info.dstPid = global_param.MyProcPid; + conn->conn_info.sessionId = session_param.gp_session_id; + conn->conn_info.icId = icid; + conn->conn_info.flags = UDPIC_FLAGS_RECEIVER_TO_SENDER; + + ic_control_info.connHtab.add(conn); + + if (global_param.createOpaqueDataCallback) + conn->opaque_data = global_param.createOpaqueDataCallback(); + } + } + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + { + LOG(DEBUG1, "SetupUDPInterconnect will activate " + "%d incoming, %d expect incoming for ic_instancce_id %d.", + incoming_count, expectedTotalIncoming, icid); + } + + return pEntry; +} + +std::unique_ptr +TransportEntry::MakeSendEntry(CChunkTransportStateImpl *state, + int icid, + ICExecSlice *sendSlice, + ICExecSlice *recvSlice) +{ + int outgoing_count = 0; + int expectedTotalOutgoing = 0; + + Assert(sendSlice->sliceIndex > 0); + Assert(recvSlice->sliceIndex >= 0); + + int motNodeID = sendSlice->sliceIndex; + int numConns = recvSlice->numPrimaryProcesses; + + std::unique_ptr pEntry = + std::make_unique(state, motNodeID, numConns, sendSlice, recvSlice); + + pEntry->txfd = ICSenderSocket; + pEntry->txport = ICSenderPort; + pEntry->txfd_family = ICSenderFamily; + + int route = 0; + pEntry->conns_.resize(numConns); + + for (int i = 0; i < numConns; ++i) + { + pEntry->conns_[i] = std::make_unique(pEntry.get()); + UDPConn *conn = pEntry->conns_[i].get(); + + /* + * Setup a MotionConn entry for each of our outbound connections. Request + * a connection to each receiving backend's listening port. NB: Some + * mirrors could be down & have no CdbProcess entry. + */ + ICCdbProcess *cdbProc = recvSlice->primaryProcesses + i; + if (cdbProc->valid) + { + conn->cdbProc = cdbProc; + conn->sndQueue.init(ICBufferListType_Primary); + conn->unackQueue.init(ICBufferListType_Primary); + conn->capacity = session_param.Gp_interconnect_queue_depth; + + /* send buffer pool must be initialized before this. */ + snd_buffer_pool.maxCount += session_param.Gp_interconnect_snd_queue_depth; + snd_control_info.cwnd += 1; + conn->curBuff = snd_buffer_pool.get(conn); + + /* should have at least one buffer for each connection */ + Assert(conn->curBuff != NULL); + + conn->rtt = DEFAULT_RTT; + conn->dev = DEFAULT_DEV; + conn->deadlockCheckBeginTime = 0; + conn->tupleCount = 0; + conn->msgSize = sizeof(conn->conn_info); + conn->sentSeq = 0; + conn->receivedAckSeq = 0; + conn->consumedSeq = 0; + conn->pBuff = (uint8 *) conn->curBuff->pkt; + conn->state = mcsSetupOutgoingConnection; + conn->route = route++; + expectedTotalOutgoing++; + + setupOutgoingUDPConnection(icid, pEntry.get(), conn); + outgoing_count++; + + if (global_param.createOpaqueDataCallback) + conn->opaque_data = global_param.createOpaqueDataCallback(); + } + } + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + { + LOG(DEBUG1, "SetupUDPInterconnect will activate " + "%d outgoing, %d expect outgoing routes for ic_instancce_id %d.", + outgoing_count, expectedTotalOutgoing, icid); + } + + return pEntry; +} + +TransportEntry::TransportEntry(CChunkTransportStateImpl *state, + int motNodeID, + int numConns, + ICExecSlice *sendSlice, + ICExecSlice *recvSlice) +{ + /* the field of CChunkTransportStateEntry */ + this->valid = false; + this->conns = nullptr; + + /* the field of TransportEntry */ + this->txfd = -1; + this->txfd_family = -1; + this->txport = 0; + this->sendingEos = false; + this->stat_total_ack_time = 0; + this->stat_count_acks = 0; + this->stat_max_ack_time = 0; + this->stat_min_ack_time = 0; + this->stat_count_resent = 0; + this->stat_max_resent = 0; + this->stat_count_dropped = 0; + + this->motNodeId = motNodeID; + this->numConns = numConns; + this->scanStart = 0; + this->sendSlice = sendSlice; + this->recvSlice = recvSlice; + this->state = state; + this->valid = true; +} + +/* + * receiveChunksUDPIFC + * Receive chunks from the senders + * + * MUST BE CALLED WITH ic_control_info.lock LOCKED. + */ +void +TransportEntry::receiveChunksUDPIFC(int16 *srcRoute, + UDPConn *conn, + GetDataLenInPacket getLen, + DataBlock *data) +{ + int retries = 0; + bool directed = false; + +#ifdef AMS_VERBOSE_LOGGING + LOG(DEBUG5, "receivechunksUDP: motnodeid %d", this->motNodeId); +#endif + + if (conn != nullptr) + { + directed = true; + *srcRoute = conn->route; + rx_control_info.mainWaitingState.set(this->motNodeId, conn->route, this->state->icInstanceId); + } + else + { + /* non-directed receive */ + rx_control_info.mainWaitingState.set(this->motNodeId, ANY_ROUTE, this->state->icInstanceId); + } + + std::unique_lock lock(mtx); + auto timeout = std::chrono::milliseconds(MAIN_THREAD_COND_TIMEOUT_MS); + + /* we didn't have any data, so we've got to read it from the network. */ + for (;;) + { + UDPConn *rxconn = nullptr; + + /* 1. Do we have data ready */ + if (rx_control_info.mainWaitingState.reachRoute != ANY_ROUTE) + { + rxconn = this->GetConn(rx_control_info.mainWaitingState.reachRoute); + rxconn->prepareRxConnForRead(); + + LOG(DEBUG2, "receiveChunksUDPIFC: non-directed rx woke on route %d", rx_control_info.mainWaitingState.reachRoute); + rx_control_info.mainWaitingState.reset(); + } + + this->aggregateStatistics(); + if (rxconn) + { + Assert(rxconn->pBuff); + + pthread_mutex_unlock(&ic_control_info.lock); + + LOG(DEBUG2, "got data with length %d", rxconn->recvBytes); + /* successfully read into this connection's buffer. */ + rxconn->GetDataInBuf(getLen, data); + + if (!directed) + *srcRoute = rxconn->route; + + return; + } + + retries++; + + /* + * Ok, we've processed all the items currently in the queue. Arm the + * latch (before releasing the mutex), and wait for more messages to + * arrive. The RX thread will wake us up using the latch. + */ + pthread_mutex_unlock(&ic_control_info.lock); + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + { + LOG(DEBUG5, "waiting (timed) on route %d %s", rx_control_info.mainWaitingState.waitingRoute, + (rx_control_info.mainWaitingState.waitingRoute == ANY_ROUTE ? "(any route)" : "")); + } + + /* + * Wait for data to become ready. + * + * In the QD, also wake up immediately if any QE reports an + * error through the main QD-QE libpq connection. For that, ask + * the dispatcher for a file descriptor to wait on for that. + */ + cv.wait_for(lock, timeout, []{return rx_control_info.mainWaitingState.reachRoute != ANY_ROUTE;}); + + /* check the potential errors in rx thread. */ + checkRxThreadError(); + + /* do not check interrupts when holding the lock */ + CHECK_INTERRUPTS(this->state); + + /* check to see if the task coordinator should cancel */ + CHECK_CANCEL(this->state); + + /* + * 1. NIC on master (and thus the QD connection) may become bad, check + * it. 2. Postmaster may become invalid, check it + */ + if ((retries & 0x3f) == 0) + { + checkQDConnectionAlive(); + CHECK_POSTMASTER_ALIVE(); + } + + pthread_mutex_lock(&ic_control_info.lock); + + } /* for (;;) */ + + /* We either got data, or get cancelled. We never make it out to here. */ + return; /* make GCC behave */ +} + +void +UDPConn::GetDataInBuf(GetDataLenInPacket getLen, DataBlock *data) +{ + int bytesProcessed = 0; + + Assert(data); + +#ifdef AMS_VERBOSE_LOGGING + LOG(DEBUG5, "recvtuple chunk recv bytes %d msgsize %d conn->pBuff %p conn->msgPos: %p", + this->recvBytes, this->msgSize, this->pBuff, this->msgPos); +#endif + + int ic_hdr_size = sizeof(struct icpkthdr); + + data->pos = this->msgPos + ic_hdr_size; + int rc = getLen ? getLen(data->pos, this->msgSize - ic_hdr_size) : (this->msgSize - ic_hdr_size); + if (rc < 0) + { + std::stringstream ss; + ss << "Failed to call getLen in GetDataInBuf, result: " << rc; + throw ICException(ss.str(), __FILE__, __LINE__); + } + data->len = rc; + + bytesProcessed += ic_hdr_size; + bytesProcessed += data->len; + + Assert(bytesProcessed == this->msgSize); + + this->recvBytes -= this->msgSize; + if (this->recvBytes != 0) + { +#ifdef AMS_VERBOSE_LOGGING + LOG(DEBUG5, "residual message %d bytes", this->recvBytes); +#endif + this->msgPos += this->msgSize; + } + + this->msgSize = 0; +} + +/* + * RecvTupleChunkFromAnyUDPIFC_Internal + * Receive tuple chunks from any route (connections) + */ +void +TransportEntry::RecvAny(int16 *srcRoute, + GetDataLenInPacket getLen, + DataBlock *data) +{ + int index, + activeCount = 0; + bool found = false; + + UDPConn *conn; + + index = this->scanStart; + + pthread_mutex_lock(&ic_control_info.lock); + + for (int i = 0; i < this->numConns; i++, index++) + { + if (index >= this->numConns) + index = 0; + + conn = this->GetConn(index); + if (conn->stillActive) + activeCount++; + + ic_statistics.totalRecvQueueSize += conn->pkt_q_size; + ic_statistics.recvQueueSizeCountingTime++; + + if (conn->pkt_q_size > 0) + { + found = true; + conn->prepareRxConnForRead(); + break; + } + } + + if (found) + { + pthread_mutex_unlock(&ic_control_info.lock); + + conn->GetDataInBuf(getLen, data); + + *srcRoute = conn->route; + this->scanStart = index + 1; + return; + } + + /* no data pending in our queue */ + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "RecvTupleChunkFromAnyUDPIFC(): activeCount is %d", activeCount); +#endif + if (activeCount == 0) + { + pthread_mutex_unlock(&ic_control_info.lock); + return; + } + + /* receiveChunksUDPIFC() releases ic_control_info.lock as a side-effect */ + this->receiveChunksUDPIFC(srcRoute, nullptr, getLen, data); + + this->scanStart = *srcRoute + 1; +} + +/* + * RecvTupleChunkFromUDPIFC_Internal + * Receive tuple chunks from a specific route (connection) + */ +void +TransportEntry::RecvRoute(int16 srcRoute, GetDataLenInPacket getLen, DataBlock *data) +{ + UDPConn *conn = this->GetConn(srcRoute); + +#ifdef AMS_VERBOSE_LOGGING + if (!conn->stillActive) + { + LOG(INFO, "RecvTupleChunkFromUDPIFC(): connection inactive ?!"); + } +#endif + + pthread_mutex_lock(&ic_control_info.lock); + + if (!conn->stillActive) + { + pthread_mutex_unlock(&ic_control_info.lock); + return; + } + + ic_statistics.totalRecvQueueSize += conn->pkt_q_size; + ic_statistics.recvQueueSizeCountingTime++; + + if (conn->pkt_q[conn->pkt_q_head] != NULL) + { + conn->prepareRxConnForRead(); + + pthread_mutex_unlock(&ic_control_info.lock); + + conn->GetDataInBuf(getLen, data); + + return; + } + + /* no existing data, we've got to read a packet */ + /* receiveChunksUDPIFC() releases ic_control_info.lock as a side-effect */ + int16 route; + this->receiveChunksUDPIFC(&route, conn, getLen, data); +} + +/* + * TeardownUDPIFCInterconnect_Internal + * Helper function for TeardownUDPIFCInterconnect. + * + * Developers should pay attention to: + * + * 1) Do not handle interrupts/throw errors in Teardown, otherwise, Teardown may be called twice. + * It will introduce an undefined behavior. And memory leaks will be introduced. + * + * 2) Be careful about adding elog/ereport/write_log in Teardown function, + * esp, out of HOLD_INTERRUPTS/RESUME_INTERRUPTS pair, since elog/ereport/write_log may + * handle interrupts. + * + */ +void +CChunkTransportStateImpl::teardown(bool hasErrors) +{ + bool isReceiver = false; + + /* Log the start of TeardownInterconnect. */ + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_TERSE_IC) + { + LogSeverity elevel = INFO; + + if (hasErrors || !this->activated) + { + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + elevel = INFO; + else + elevel = DEBUG1; + } + else if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + elevel = DEBUG4; + + if (elevel) + { + ICExecSlice *mySlice = &this->sliceTable->slices[this->sliceId]; + LOG(elevel, "Interconnect seg%d slice%d cleanup state: %s; setup was %s", + global_param.segindex, mySlice->sliceIndex, + hasErrors ? "hasErrors" : "normal", + this->activated ? "completed" : "exited"); + } + + /* if setup did not complete, log the slicetable */ + if (!this->activated && session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + { + //elog_node_display(DEBUG3, "local slice table", this->sliceTable, true); + //TODO: print real slicetable; + LOG(DEBUG3, "local slice table: ####"); + } + } + + /* + * The long jump with CheckXXX() in receiveChunksUDPIFC() introduces the + * locked mtx, so unlock it here. + */ + mtx.unlock(); + + /* + * add lock to protect the hash table, since background thread is still + * working. + */ + pthread_mutex_lock(&ic_control_info.lock); + + if (session_param.gp_interconnect_cache_future_packets) + cleanupStartupCache(); + + /* + * Now "normal" connections which made it through our peer-registration + * step. With these we have to worry about "in-flight" data. + */ + this->DestroySendEntries(); + + /* + * Previously, there is a piece of code that deals with pending stops. Now + * it is delegated to background rx thread which will deal with any + * mismatched packets. + */ + + /* + * cleanup all of our Receiving Motion nodes, these get closed immediately + * (the receiver know for real if they want to shut down -- they aren't + * going to be processing any more data). + */ + this->DestroyRecvEntries(&isReceiver); + + /* + * now that we've moved active rx-buffers to the freelist, we can prune + * the freelist itself + */ + while (rx_buffer_pool.count > rx_buffer_pool.maxCount) + { + icpkthdr *buf = NULL; + + /* If this happened, there are some memory leaks.. */ + if (rx_buffer_pool.freeList == NULL) + { + pthread_mutex_unlock(&ic_control_info.lock); + + std::stringstream ss; + ss << "FATAL: freelist NULL: count " << rx_buffer_pool.count + << " max " << rx_buffer_pool.maxCount << " buf " << rx_buffer_pool.freeList; + throw ICFatalException(ss.str(), __FILE__, __LINE__); + } + + buf = rx_buffer_pool.get_free(); + rx_buffer_pool.release(buf); + } + + /* + * Update the history of interconnect instance id. + */ + if (global_param.Gp_role == GP_ROLE_DISPATCH_IC) + { + rx_control_info.cursorHistoryTable.update(this->icInstanceId, 0); + } + else if (global_param.Gp_role == GP_ROLE_EXECUTE_IC) + { + rx_control_info.lastTornIcId = this->icInstanceId; + } + + if (IC_DEBUG1 >= session_param.log_min_messages) + { + LOG(DEBUG1, "Interconnect State: " + "isSender %d isReceiver %d " + "snd_queue_depth %d recv_queue_depth %d Gp_max_packet_size %d " + "UNACK_QUEUE_RING_SLOTS_NUM %d TIMER_SPAN %lld DEFAULT_RTT %d " + "hasErrors %d, ic_instance_id %d ic_id_last_teardown %d " + "snd_buffer_pool.count %d snd_buffer_pool.maxCount %d snd_sock_bufsize %d recv_sock_bufsize %d " + "snd_pkt_count %d retransmits %d crc_errors %d" + " recv_pkt_count %d recv_ack_num %d" + " recv_queue_size_avg %f" + " capacity_avg %f" + " freebuf_avg %f " + "mismatch_pkt_num %d disordered_pkt_num %d duplicated_pkt_num %d" + " cwnd %f status_query_msg_num %d", + ic_control_info.isSender, isReceiver, + session_param.Gp_interconnect_snd_queue_depth, session_param.Gp_interconnect_queue_depth, global_param.Gp_max_packet_size, + UNACK_QUEUE_RING_SLOTS_NUM, TIMER_SPAN, DEFAULT_RTT, + hasErrors, this->icInstanceId, rx_control_info.lastTornIcId, + snd_buffer_pool.count, snd_buffer_pool.maxCount, ic_control_info.socketSendBufferSize, ic_control_info.socketRecvBufferSize, + ic_statistics.sndPktNum, ic_statistics.retransmits, ic_statistics.crcErrors, + ic_statistics.recvPktNum, ic_statistics.recvAckNum, + (double) ((double) ic_statistics.totalRecvQueueSize) / ((double) ic_statistics.recvQueueSizeCountingTime), + (double) ((double) ic_statistics.totalCapacity) / ((double) ic_statistics.capacityCountingTime), + (double) ((double) ic_statistics.totalBuffers) / ((double) ic_statistics.bufferCountingTime), + ic_statistics.mismatchNum, ic_statistics.disorderedPktNum, ic_statistics.duplicatedPktNum, + snd_control_info.cwnd, ic_statistics.statusQueryMsgNum); + } + + ic_control_info.isSender = false; + memset(&ic_statistics, 0, sizeof(ICStatistics)); + + pthread_mutex_unlock(&ic_control_info.lock); + + /* reset the rx thread network error flag */ + resetRxThreadError(); + + /* free sliceTable */ + if (this->sliceTable) + { + ICSliceTable *ic_tbl = this->sliceTable; + for (int i = 0; i < ic_tbl->numSlices; ++i) + { + ICExecSlice *ic_slice = ic_tbl->slices + i; + ic_free(ic_slice->children); + ic_free(ic_slice->primaryProcesses); + } + ic_free(ic_tbl->slices); + ic_free(ic_tbl); + } + + this->activated = false; + this->sliceTable = NULL; + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_TERSE_IC) + LOG(DEBUG1, "TeardownUDPIFCInterconnect_Internal successful"); +} + +void +TransportEntry::Broadcast(DataBlock *data, int *inactiveCountPtr) +{ + int *p_inactive = inactiveCountPtr; + int index, inactive = 0; + + /* add our tcItem to each of the outgoing buffers. */ + index = Max(0, global_param.segindex); /* entry-db has -1 */ + for (int i = 0; i < this->numConns; i++, index++) + { + if (index >= this->numConns) + index = 0; + + UDPConn *conn = this->GetConn(index); + + /* only send to still interested receivers. */ + if (conn->stillActive) + { + conn->Send(data); + if (!conn->stillActive) + inactive++; + } + } + + if (p_inactive != NULL) + *p_inactive = (inactive ? 1 : 0); +} + +TransportEntry* +CChunkTransportStateImpl::GetEntry(int motNodeID, bool checkValid) +{ + if (motNodeID > 0 && motNodeID <= static_cast(this->entries_.size())) + { + TransportEntry *pEntry = this->entries_[motNodeID - 1].get(); + if (pEntry != nullptr) + { + if (!checkValid) + return pEntry; + if (pEntry->motNodeId == motNodeID && pEntry->valid) + return pEntry; + } + } + + std::stringstream ss; + ss << "ERROR, Interconnect Error: Unexpected Motion Node Id: " << motNodeID + << ". This means a motion node that wasn't setup is requesting interconnect resources."; + throw ICInvalidIndex(ss.str(), __FILE__, __LINE__); +} + +/* + * The number of the Receiving Motion may be > 1, such as + * Hashjoin + * -> Redis Motion + * ... + * -> Hash + * -> Redis Motion + * ... + */ +void +CChunkTransportStateImpl::CreateRecvEntries(ICSliceTable *sliceTable) +{ + ICExecSlice *mySlice = &sliceTable->slices[sliceTable->localSlice]; + + /* now we'll do some setup for each of our Receiving Motion Nodes. */ + for (int child_index = 0; child_index < mySlice->numChildren; ++child_index) + { + int childId = mySlice->children[child_index]; + ICExecSlice *sendSlice = &sliceTable->slices[childId]; + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "Setup recving connections: my slice %d, childId %d", mySlice->sliceIndex, childId); + + if (sendSlice->sliceIndex > static_cast(this->entries_.size())) + this->entries_.resize(sendSlice->sliceIndex); + this->checkMotNodeID(sendSlice->sliceIndex); + + std::unique_ptr pEntry = + TransportEntry::MakeRecvEntry(this, sliceTable->ic_instance_id, sendSlice, mySlice); + this->entries_[sendSlice->sliceIndex - 1] = std::move(pEntry); + } +} + +void +CChunkTransportStateImpl::CreateSendEntries(ICSliceTable *sliceTable) +{ + ICExecSlice *sendSlice = &sliceTable->slices[sliceTable->localSlice]; + + if (sendSlice->parentIndex == -1) { + ic_control_info.isSender = false; + ic_control_info.lastExpirationCheckTime = 0; + return; + } + + snd_control_info.cwnd = 0; + snd_control_info.minCwnd = 0; + snd_control_info.ssthresh = 0; + + snd_buffer_pool.init(); + initUnackQueueRing(&unack_queue_ring); + ic_control_info.isSender = true; + ic_control_info.lastExpirationCheckTime = getCurrentTime(); + ic_control_info.lastPacketSendTime = ic_control_info.lastExpirationCheckTime; + ic_control_info.lastDeadlockCheckTime = ic_control_info.lastExpirationCheckTime; + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "Interconnect seg%d slice%d setting up sending motion node", + global_param.segindex, sendSlice->sliceIndex); + + if (sendSlice->sliceIndex > static_cast(this->entries_.size())) + this->entries_.resize(sendSlice->sliceIndex); + this->checkMotNodeID(sendSlice->sliceIndex); + + ICExecSlice *recvSlice = &sliceTable->slices[sendSlice->parentIndex]; + std::unique_ptr pEntry = + TransportEntry::MakeSendEntry(this, sliceTable->ic_instance_id, sendSlice, recvSlice); + + pEntry->txfd = ICSenderSocket; + pEntry->txport = ICSenderPort; + pEntry->txfd_family = ICSenderFamily; + + snd_control_info.minCwnd = snd_control_info.cwnd; + snd_control_info.ssthresh = snd_buffer_pool.maxCount; + +#ifdef TRANSFER_PROTOCOL_STATS + trans_proto_stats.init(); +#endif + + this->entries_[sendSlice->sliceIndex - 1] = std::move(pEntry); +} + +void +CChunkTransportStateImpl::DestroyRecvEntries(bool *isReceiver) +{ + ICExecSlice *mySlice = &this->sliceTable->slices[this->sliceId]; + for (int child_index = 0; child_index < mySlice->numChildren; ++child_index) + { + int childId = mySlice->children[child_index]; + ICExecSlice *aSlice = &this->sliceTable->slices[childId]; + + /* + * First check whether the entry is initialized to avoid the potential + * errors thrown out from the removeChunkTransportState, which may + * introduce some memory leaks. + */ + int motNodeID = aSlice->sliceIndex; + if (this->entries_[motNodeID - 1] == nullptr) + continue; + + TransportEntry *pEntry = this->entries_[motNodeID - 1].get(); + Assert(motNodeID == pEntry->motNodeId); + + /* now it is safe to remove. */ + if (!pEntry->valid) + continue; + +#ifdef USE_ASSERT_CHECKING + pEntry->dumpConnections("/tmp/receiving_entries"); +#endif + /* remove it */ + pEntry->valid = false; + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "Interconnect closing connections from slice%d", aSlice->sliceIndex); + + *isReceiver = true; + + /* + * receivers know that they no longer care about data from + * below ... so we can safely discard data queued in both + * directions + */ + for (size_t i = 0; i < pEntry->conns_.size(); ++i) + { + UDPConn *conn = pEntry->conns_[i].get(); + + Assert(conn); + if (!conn) + continue; + + if (conn->cdbProc == NULL) + continue; + + /* out of memory has occurred, break out */ + if (!conn->pkt_q) + break; + + rx_buffer_pool.maxCount -= conn->pkt_q_capacity; + + ic_control_info.connHtab.remove(conn); + + /* + * ReleaseBuffer() dequeues messages and moves + * them to pBuff + */ + while (conn->pkt_q_size > 0) + conn->ReleaseBuffer(NULL); + + /* we also need to clear all the out-of-order packets */ + conn->freeDisorderedPackets(); + + /* free up the packet queue */ + ic_free(conn->pkt_q); + conn->pkt_q = NULL; + + if (global_param.destroyOpaqueDataCallback) + global_param.destroyOpaqueDataCallback(&conn->opaque_data); + + if (conn->curBuff) + { + ic_free(conn->curBuff); + conn->curBuff = NULL; + } + } // for conn + + Assert(!pEntry->conns); + } // for entry +} + +/* + * computeNetworkStatistics + * Compute the max/min/avg network statistics. + */ +static inline void +computeNetworkStatistics(uint64 value, uint64 *min, uint64 *max, double *sum) +{ + if (value >= *max) + *max = value; + if (value <= *min) + *min = value; + *sum += value; +} + +void +CChunkTransportStateImpl::DestroySendEntries() +{ + ICExecSlice *mySlice = &this->sliceTable->slices[this->sliceId]; + if (mySlice->parentIndex == -1) + return; + + /* cleanup a Sending motion node. */ + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + { + ICExecSlice *parentSlice = &this->sliceTable->slices[mySlice->parentIndex]; + LOG(DEBUG1, "Interconnect seg%d slice%d closing connections to slice%d (%d peers)", + global_param.segindex, mySlice->sliceIndex, mySlice->parentIndex, + parentSlice->numPrimaryProcesses); + } + + /* + * In the olden days, we required that the error case successfully + * transmit and end-of-stream message here. But the introduction of + * cdbdisp_check_estate_for_cancel() alleviates for the QD case, and + * the cross-connection of writer gangs in the dispatcher (propagation + * of cancel between them) fixes the I-S case. + * + * So the call to forceEosToPeers() is no longer required. + */ + int motNodeID = mySlice->sliceIndex; + if (this->entries_[motNodeID - 1] == nullptr) + return; + + TransportEntry *pEntry = this->entries_[motNodeID - 1].get(); + Assert(motNodeID == pEntry->motNodeId); + + /* now it is safe to remove. */ + if (!pEntry->valid) + return; + +#ifdef USE_ASSERT_CHECKING + pEntry->dumpConnections("/tmp/sending_entries"); +#endif + + /* remove it */ + pEntry->valid = false; + + uint64 maxRtt = 0; + double avgRtt = 0; + uint64 minRtt = ~((uint64) 0); + + uint64 maxDev = 0; + double avgDev = 0; + uint64 minDev = ~((uint64) 0); + + /* connection array allocation may fail in interconnect setup. */ + for (size_t i = 0; i < pEntry->conns_.size(); ++i) + { + UDPConn *conn = pEntry->conns_[i].get(); + + Assert(conn); + if (!conn) + continue; + + if (!conn->cdbProc) + continue; + + /* compute some statistics */ + computeNetworkStatistics(conn->rtt, &minRtt, &maxRtt, &avgRtt); + computeNetworkStatistics(conn->dev, &minDev, &maxDev, &avgDev); + + conn->sndQueue.release(false); + conn->unackQueue.release(session_param.Gp_interconnect_fc_method == INTERCONNECT_FC_METHOD_CAPACITY_IC ? false : true); + + ic_control_info.connHtab.remove(conn); + + if (conn->curBuff) + { + ic_free(conn->curBuff); + conn->curBuff = NULL; + } + } + avgRtt = avgRtt / pEntry->numConns; + avgDev = avgDev / pEntry->numConns; + + /* free all send side buffers */ + snd_buffer_pool.clean(); + + Assert(!pEntry->conns); + +#ifdef TRANSFER_PROTOCOL_STATS + trans_proto_stats.dump(); +#endif + + if (IC_DEBUG1 >= session_param.log_min_messages) + { + LOG(DEBUG1, "Interconnect State: isSender %d DEFAULT_RTT %d rtt/dev [%lu/%lu, %f/%f, %lu/%lu] ", + ic_control_info.isSender, DEFAULT_RTT, (minRtt == ~((uint64) 0) ? 0 : minRtt), + (minDev == ~((uint64) 0) ? 0 : minDev), avgRtt, avgDev, maxRtt, maxDev); + } +} + +void +CChunkTransportStateImpl::checkMotNodeID(int sendMotNodeID) +{ + Assert(sendMotNodeID > 0); + + if ((sendMotNodeID) <= 0) { + std::stringstream ss; + ss <<"ERROR, Interconnect Error: Unexpected Motion Node Id: "<entries_[sendMotNodeID - 1].get(); + if (pEntry != nullptr) { + UDPConn *conn = pEntry->conns_.size() ? pEntry->conns_[0].get() : nullptr; + + Assert(static_cast(pEntry->numConns) == pEntry->conns_.size()); + std::stringstream ss; + ss <<"ERROR, interconnect error: A HTAB entry for motion node "<conns_.size()<< + " first sock " << (conn != NULL ? conn->sockfd : -2); + throw ICInvalidIndex(ss.str(), __FILE__, __LINE__); + } +} + +CChunkTransportStateImpl::CChunkTransportStateImpl(ICSliceTable *_sliceTable) +{ + activated = false; + teardownActive = false; + + sliceTable = _sliceTable; + sliceId = sliceTable->localSlice; + icInstanceId = sliceTable->ic_instance_id; + + networkTimeoutIsLogged = false; + + clientState = NULL; +} + +ICChunkTransportState* +CChunkTransportStateImpl::SetupUDP(ICSliceTable *sliceTable, SessionMotionLayerIPCParam *param) +{ + if (param) + memcpy(&session_param, param, sizeof(*param)); + + /* + * The rx-thread might have set an error since last teardown, + * technically it is not part of current query, discard it directly. + */ + resetRxThreadError(); + + try { + ICChunkTransportState *state = CChunkTransportStateImpl::setup(sliceTable); + + /* Internal error if we locked the mutex but forgot to unlock it. */ + Assert(pthread_mutex_unlock(&ic_control_info.lock) != 0); + + return state; + + } catch (...) { + /* + * Remove connections from hash table to avoid packet handling in the + * rx pthread, else the packet handling code could use memory whose + * context (InterconnectContext) would be soon reset - that could + * panic the process. + */ + ConnHashTable *ht = &ic_control_info.connHtab; + + for (int i = 0; i < ht->size; i++) + { + struct ConnHtabBin *trash; + UDPConn *conn = NULL; + + trash = ht->table[i]; + while (trash != NULL) + { + conn = trash->conn; + /* Get trash at first as trash will be pfree-ed in remove. */ + trash = trash->next; + ht->remove(conn); + } + } + pthread_mutex_unlock(&ic_control_info.lock); + + throw; + } +} + +void +CChunkTransportStateImpl::TeardownUDP(bool hasErrors) +{ + try { + CChunkTransportStateImpl::state_ = nullptr; + this->teardown(hasErrors); + delete this; + Assert(pthread_mutex_unlock(&ic_control_info.lock) != 0); + } catch (...) { + pthread_mutex_unlock(&ic_control_info.lock); + throw; + } +} + +CChunkTransportState ** +CChunkTransportStateImpl::GetTransportState() +{ + return &CChunkTransportStateImpl::state_; +} + +void +CChunkTransportStateImpl::RecvRoute(int16 motNodeID, + int16 srcRoute, + GetDataLenInPacket getLen, + DataBlock *data) +{ + try { + TransportEntry *pEntry = GetEntry(motNodeID, true); + pEntry->RecvRoute(srcRoute, getLen, data); + + /* error if mutex still held (debug build only) */ + Assert(pthread_mutex_unlock(&ic_control_info.lock) != 0); + } catch (...) { + pthread_mutex_unlock(&ic_control_info.lock); + throw; + } +} + +void +CChunkTransportStateImpl::RecvAny(int16 motNodeID, + int16 *srcRoute, + GetDataLenInPacket getLen, + DataBlock *data) +{ + try { + TransportEntry *pEntry = GetEntry(motNodeID, true); + pEntry->RecvAny(srcRoute, getLen, data); + + /* error if mutex still held (debug build only) */ + Assert(pthread_mutex_unlock(&ic_control_info.lock) != 0); + } catch (...) { + pthread_mutex_unlock(&ic_control_info.lock); + throw; + } +} + +void +CChunkTransportStateImpl::SendEOS(int motNodeID, DataBlock *data) +{ + int retry = 0; + int activeCount = 0; + int timeout = 0; + + TransportEntry *pEntry = GetEntry(motNodeID, true); + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "Interconnect seg%d slice%d sending end-of-stream to slice%d", + global_param.segindex, motNodeID, pEntry->recvSlice->sliceIndex); + + /* + * we want to add our tcItem onto each of the outgoing buffers -- this is + * guaranteed to leave things in a state where a flush is *required*. + */ + pEntry->Broadcast(data, NULL); + + pEntry->sendingEos = true; + + uint64 now = getCurrentTime(); + + /* now flush all of the buffers. */ + for (int i = 0; i < pEntry->numConns; i++) + { + UDPConn *conn = pEntry->GetConn(i); + if (conn->stillActive) + { + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "sent eos to route %d tuplecount %d seq %d flags 0x%x stillActive %s icId %d %d", + conn->route, conn->tupleCount, conn->conn_info.seq, + conn->conn_info.flags, (conn->stillActive ? "true" : "false"), + conn->conn_info.icId, conn->msgSize); + + /* prepare this for transmit */ + conn->conn_info.flags |= UDPIC_FLAGS_EOS; + + /* place it into the send queue */ + conn->prepareXmit(); + conn->sndQueue.append(conn->curBuff); + conn->sendBuffers(); + conn->curBuff = NULL; + conn->pBuff = NULL; + + conn->tupleCount = 0; + conn->msgSize = sizeof(conn->conn_info); + conn->deadlockCheckBeginTime = now; + + activeCount++; + } + } + + /* + * Now waiting for acks from receivers. + * + * Note here waiting is done in a separate phase from the EOS sending + * phase to make the processing faster when a lot of connections are slow + * and have frequent packet losses. In fault injection tests, we found + * this. + * + */ + while (activeCount > 0) + { + activeCount = 0; + + for (int i = 0; i < pEntry->numConns; i++) + { + UDPConn *conn = pEntry->GetConn(i); + if (conn->stillActive) + { + retry = 0; + ic_control_info.lastPacketSendTime = 0; + + /* wait until this queue is emptied */ + while (conn->unackQueue.length() > 0 || + conn->sndQueue.length() > 0) + { + timeout = conn->computeTimeout(retry); + + if (pEntry->pollAcks(timeout)) + pEntry->handleAcks(); + + conn->checkExceptions(retry++, timeout); + + if (retry >= MAX_TRY) + break; + } + + if ((!conn->cdbProc) || (conn->unackQueue.length() == 0 && + conn->sndQueue.length() == 0)) + { + conn->state = mcsEosSent; + conn->stillActive = false; + } + else + activeCount++; + } + } + } + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "SendEOSleaving, activeCount %d", activeCount); +} + +void +CChunkTransportStateImpl::SendStop(int16 motNodeID) +{ + Assert(this->activated); + + TransportEntry *pEntry = GetEntry(motNodeID, true); + + /* + * Note: we're only concerned with receivers here. + */ + pthread_mutex_lock(&ic_control_info.lock); + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "Interconnect needs no more input from slice%d; notifying senders to stop.", motNodeID); + + for (int i = 0; i < pEntry->numConns; i++) + { + UDPConn *conn = pEntry->GetConn(i); + + /* + * Note here, the stillActive flag of a connection may have been set + * to false by DeactiveConn. + */ + if (conn->stillActive) + { + if (conn->conn_info.flags & UDPIC_FLAGS_EOS) + { + /* + * we have a queued packet that has EOS in it. We've acked it, + * so we're done + */ + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "do sendstop: already have queued EOS packet, we're done. node %d route %d", motNodeID, i); + + conn->stillActive = false; + + /* need to drop the queues in the teardown function. */ + while (conn->pkt_q_size > 0) + conn->ReleaseBuffer(NULL); + } + else + { + conn->stopRequested = true; + conn->conn_info.flags |= UDPIC_FLAGS_STOP; + + /* + * The peer addresses for incoming connections will not be set + * until the first packet has arrived. However, when the lower + * slice does not have data to send, the corresponding peer + * address for the incoming connection will never be set. We + * will skip sending ACKs to those connections. + */ + +#ifdef FAULT_INJECTOR + if (FaultInjector_InjectFaultIfSet( + "interconnect_stop_ack_is_lost", + DDLNotSpecified, + "" /* databaseName */ , + "" /* tableName */ ) == FaultInjectorTypeSkip) + { + continue; + } +#endif + + if (conn->peer.ss_family == AF_INET || conn->peer.ss_family == AF_INET6) + { + uint32 seq = conn->conn_info.seq > 0 ? conn->conn_info.seq - 1 : 0; + + conn->sendAck(UDPIC_FLAGS_STOP | UDPIC_FLAGS_ACK | UDPIC_FLAGS_CAPACITY | conn->conn_info.flags, seq, seq); + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "sent stop message. node %d route %d seq %d", motNodeID, i, seq); + } + else + { + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG1, "first packet did not arrive yet. don't sent stop message. node %d route %d", motNodeID, i); + } + } + } + } + pthread_mutex_unlock(&ic_control_info.lock); +} + +void +TransportEntry::Send(int16 targetRoute, DataBlock *db, bool broadcast, int *recount) +{ +#ifdef AMS_VERBOSE_LOGGING + LOG(DEBUG5, "SendTupleChunkToAMS: chunk length %d", db->len); +#endif + + if (broadcast) + { + this->Broadcast(db, recount); + } + else + { + if (targetRoute < 0 || targetRoute >= this->numConns) + { + std::stringstream ss; + ss << "FATAL: SendTupleChunkToAMS: targetRoute is " << targetRoute + << ", must be between 0 and " << this->numConns << " ."; + throw ICFatalException(ss.str(), __FILE__, __LINE__); + } + + /* handle pt-to-pt message. Primary */ + UDPConn *conn = this->GetConn(targetRoute); + + /* only send to interested connections */ + if (conn->stillActive) + { + conn->Send(db); + if (!conn->stillActive) + *recount = 1; + } + /* in 4.0 logical mirror xmit eliminated. */ + } +} + +bool +TransportEntry::SendData(int16 targetRoute, DataBlock *pblocks, int num, bool broadcast) +{ + int recount = 0; + + int payload = global_param.Gp_max_packet_size - sizeof(icpkthdr); + + /* + * tcItem can actually be a chain of tcItems. we need to send out all of + * them. + */ + for (int i = 0; i < num; ++i) + { + DataBlock db = *(pblocks + i); + while (db.len > 0) + { + DataBlock toSend; + toSend.len = db.len > payload ? payload : db.len; + toSend.pos = db.pos; + this->Send(targetRoute, &toSend, broadcast, &recount); + db.len -= toSend.len; + db.pos += toSend.len; + } + } + + if (recount == 0) + return true; + + /* if we don't have any connections active, return false */ + int i = 0; + for (i = 0; i < this->numConns; i++) + { + UDPConn *conn = this->GetConn(i); + if (conn->stillActive) + break; + } + + /* if we found an active connection we're not done */ + return (i < this->numConns); + +} + +bool +CChunkTransportStateImpl::SendData(int16 motNodeID, + int16 targetRoute, + DataBlock *pblocks, + int num, + bool broadcast) +{ + TransportEntry *pEntry = GetEntry(motNodeID, true); + return pEntry->SendData(targetRoute, pblocks, num, broadcast); +} + +void +CChunkTransportStateImpl::GetFreeSpace(int16 motNodeID, int16 targetRoute, BufferBlock *b) +{ + Assert(b != NULL); + + TransportEntry *pEntry = GetEntry(motNodeID, true); + + /* handle pt-to-pt message. Primary */ + UDPConn *conn = pEntry->GetConn(targetRoute); + + b->pos = NULL; + b->len = 0; + if (conn->stillActive) + { + b->pos = conn->pBuff + conn->msgSize; + Assert(global_param.Gp_max_packet_size >= conn->msgSize); + b->len = global_param.Gp_max_packet_size - conn->msgSize; + } +} + +void +CChunkTransportStateImpl::ReduceFreeSpace(int16 motNodeID, int16 targetRoute, int length) +{ + TransportEntry *pEntry = GetEntry(motNodeID, true); + + /* handle pt-to-pt message. Primary */ + UDPConn *conn = pEntry->GetConn(targetRoute); + + /* only send to interested connections */ + if (conn->stillActive) + { + Assert(conn->msgSize + length <= global_param.Gp_max_packet_size); + conn->msgSize += length; + conn->tupleCount++; + } +} + +void +CChunkTransportStateImpl::ReleaseAndAck(int motNodeID, int route) +{ + AckSendParam param; + + TransportEntry *pEntry = GetEntry(motNodeID, true); + UDPConn *conn = pEntry->GetConn(route); + + memset(¶m, 0, sizeof(AckSendParam)); + + pthread_mutex_lock(&ic_control_info.lock); + + if (conn->pBuff != NULL) + { + conn->ReleaseBuffer(¶m); + } + else + { + pthread_mutex_unlock(&ic_control_info.lock); + throw ICFatalException("FATAL: Interconnect error: tried to release a NULL buffer", __FILE__, __LINE__); + } + + pthread_mutex_unlock(&ic_control_info.lock); + + /* + * real ack sending is after lock release to decrease the lock holding + * time. + */ + if (param.msg.len != 0) + UDPConn::sendAckWithParam(¶m); +} + +void +CChunkTransportStateImpl::DeactiveRoute(int motNodeID, int srcRoute, const char *reason) +{ + TransportEntry *pEntry = GetEntry(motNodeID, true); + UDPConn *conn = pEntry->GetConn(srcRoute); + + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + { + LOG(DEBUG3, "Interconnect finished receiving from seg%d slice%d %s pid=%d sockfd=%d; %s", + conn->remoteContentId, pEntry->sendSlice->sliceIndex, conn->remoteHostAndPort, + conn->cdbProc->pid, conn->sockfd, reason); + } + +#ifdef AMS_VERBOSE_LOGGING + LOG(INFO, "deregisterReadInterest set stillactive = false for node %d route %d (%s)", motNodeID, srcRoute, reason); +#endif + + conn->DeactiveConn(); +} + +void* +CChunkTransportStateImpl::GetOpaqueDataInConn(int16 motNodeID, int16 targetRoute) +{ + TransportEntry *pEntry = this->GetEntry(motNodeID, true); + UDPConn *conn = pEntry->GetConn(targetRoute); + return conn->opaque_data; +} + +int32* +CChunkTransportStateImpl::GetSentRecordTypmodInConn(int16 motNodeID, int16 targetRoute) +{ + TransportEntry *pEntry = this->GetEntry(motNodeID, true); + UDPConn *conn = pEntry->GetConn(targetRoute); + return &conn->sent_record_typmod; +} + +int +CChunkTransportStateImpl::GetConnNum(int motNodeID) { + TransportEntry *pEntry = this->GetEntry(motNodeID, true); + return pEntry->conns_.size(); +} + +void +CChunkTransportStateImpl::NotifyQuit() { + thread_quit = true; +} + +void +CChunkTransportStateImpl::SetVectorEngineAsUser() { + vector_engine_is_user = true; +} + +CChunkTransportStateImpl* ToDerived(CChunkTransportState *ptr) { + return static_cast(ptr); +} + +/* + * C++ interface wrapper class based on class CChunkTransportStateImpl; + */ +ICChunkTransportState* +CChunkTransportState::SetupUDP(ICSliceTable *sliceTable, + SessionMotionLayerIPCParam *param) { + return CChunkTransportStateImpl::SetupUDP(sliceTable, param); +} + +void +CChunkTransportState::TeardownUDP(bool hasErrors) { + ToDerived(this)->TeardownUDP(hasErrors); +} + +void +CChunkTransportState::RecvRoute(int16 motNodeID, int16 srcRoute, + GetDataLenInPacket getLen, DataBlock *data) { + ToDerived(this)->RecvRoute(motNodeID, srcRoute, getLen, data); +} + +void +CChunkTransportState::RecvAny(int16 motNodeID, int16 *srcRoute, + GetDataLenInPacket getLen, DataBlock *data) { + ToDerived(this)->RecvAny(motNodeID, srcRoute, getLen, data); +} + +void +CChunkTransportState::SendEOS(int motNodeID, DataBlock *data) { + ToDerived(this)->SendEOS(motNodeID, data); +} + +void +CChunkTransportState::SendStop(int16 motNodeID) { + return ToDerived(this)->SendStop(motNodeID); +} + +bool +CChunkTransportState::SendData(int16 motNodeID, int16 targetRoute, DataBlock *pblocks, + int num, bool broadcast) { + return ToDerived(this)->SendData(motNodeID, targetRoute, pblocks, num, broadcast); +} + +void +CChunkTransportState::GetFreeSpace(int16 motNodeID, int16 targetRoute, BufferBlock *b) { + return ToDerived(this)->GetFreeSpace(motNodeID, targetRoute, b); +} + +void +CChunkTransportState::ReduceFreeSpace(int16 motNodeID, int16 targetRoute, int length) { + return ToDerived(this)->ReduceFreeSpace(motNodeID, targetRoute, length); +} + +void +CChunkTransportState::ReleaseAndAck(int motNodeID, int route) { + return ToDerived(this)->ReleaseAndAck(motNodeID, route); +} + +void +CChunkTransportState::DeactiveRoute(int motNodeID, int srcRoute, const char *reason) { + return ToDerived(this)->DeactiveRoute(motNodeID, srcRoute, reason); +} + +void* +CChunkTransportState::GetOpaqueDataInConn(int16 motNodeID, int16 targetRoute) { + return ToDerived(this)->GetOpaqueDataInConn(motNodeID, targetRoute); +} + +int32* +CChunkTransportState::GetSentRecordTypmodInConn(int16 motNodeID, int16 targetRoute) { + return ToDerived(this)->GetSentRecordTypmodInConn(motNodeID, targetRoute); +} + +int +CChunkTransportState::GetConnNum(int motNodeID) { + return ToDerived(this)->GetConnNum(motNodeID); +} + +void +CChunkTransportState::NotifyQuit() { + return ToDerived(this)->NotifyQuit(); +} + +void +CChunkTransportState::SetVectorEngineAsUser() { + return ToDerived(this)->SetVectorEngineAsUser(); +} + +CChunkTransportState** +CChunkTransportState::GetTransportState() { + return CChunkTransportStateImpl::GetTransportState(); +} + +/* + * C interface wrapper of UDP implement based C++ interface calss CChunkTransportState. + */ +#ifdef __cplusplus +extern "C" { +#endif + +static void handleException() +{ + try{ + throw; + } catch (const std::bad_alloc &e) { + SetLastError(LEVEL_ERROR, "out of memory"); + } catch (const ICFatalException &e) { + SetLastError(LEVEL_FATAL, e.msg()); + } catch (const ICException &e) { + SetLastError(LEVEL_ERROR, e.msg()); + } catch (const std::exception &e) { + SetLastError(LEVEL_ERROR, e.what()); + } catch (...) { + SetLastError(LEVEL_ERROR, "something unknown wrong happened!"); + } +} + +ICChunkTransportState* +UDP2_SetupUDP(ICSliceTable *sliceTable, SessionMotionLayerIPCParam *param) +{ + try { + return CChunkTransportState::SetupUDP(sliceTable, param); + } catch (...) { + handleException(); + } + + return NULL; +} + +/* + * TeardownUDPIFCInterconnect + * Tear down UDP interconnect. + * + * This function is called to release the resources used by interconnect. + */ +void +UDP2_TeardownUDP(ICChunkTransportState *transportStates, + bool hasErrors) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + cstate->TeardownUDP(hasErrors); + } catch (...) { + handleException(); + } +} + +/* + * RecvTupleChunkFromUDPIFC + * Receive tuple chunks from a specific route (connection) + */ +void +UDP2_RecvRoute(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 srcRoute, + GetDataLenInPacket getLen, + DataBlock *data) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + cstate->RecvRoute(motNodeID, srcRoute, getLen, data); + } catch (...) { + handleException(); + } +} + +/* + * RecvTupleChunkFromAnyUDPIFC + * Receive tuple chunks from any route (connections) + */ +void +UDP2_RecvAny(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 *srcRoute, + GetDataLenInPacket getLen, + DataBlock *data) + +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + cstate->RecvAny(motNodeID, srcRoute, getLen, data); + } catch (...) { + handleException(); + } +} + +/* + * SendEOS + * broadcast eos messages to receivers. + */ +void +UDP2_SendEOS(ICChunkTransportState *transportStates, + int motNodeID, + DataBlock *data) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + cstate->SendEOS(motNodeID, data); + } catch (...) { + handleException(); + } +} + +void +UDP2_SendStop(ICChunkTransportState *transportStates, + int16 motNodeID) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + cstate->SendStop(motNodeID); + } catch (...) { + handleException(); + } +} + +bool +UDP2_SendData(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + DataBlock *pblocks, + int num, + bool broadcast) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + return cstate->SendData(motNodeID, targetRoute, pblocks, num, broadcast); + } catch (...) { + handleException(); + } + + return false; +} + +/* + * The fetches a direct pointer into our transmit buffers, along with + * an indication as to how much data can be safely shoved into the + * buffer (started at the pointed location). + * + * This works a lot like SendTupleChunkToAMS(). + */ +void +UDP2_GetFreeSpace(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + BufferBlock *b) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + cstate->GetFreeSpace(motNodeID, targetRoute, b); + } catch (...) { + handleException(); + } +} + +void +UDP2_ReduceFreeSpace(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + int length) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + cstate->ReduceFreeSpace(motNodeID, targetRoute, length); + } catch (...) { + handleException(); + } +} + +/* + * SendAck + * + * The cdbmotion code has discarded our pointer to the motion-conn + * structure, but has enough info to fully specify it. + */ +void +UDP2_ReleaseAndAck(ICChunkTransportState *transportStates, + int motNodeID, + int route) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + cstate->ReleaseAndAck(motNodeID, route); + } catch (...) { + handleException(); + } +} + +void +UDP2_DeactiveRoute(ICChunkTransportState *transportStates, int motNodeID, int + srcRoute, const char *reason) { Assert(transportStates); + + try { CChunkTransportState *cstate = + static_cast(transportStates); + cstate->DeactiveRoute(motNodeID, srcRoute, reason); + } catch (...) { + handleException(); + } +} + +void* +UDP2_GetOpaqueDataInConn(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + return cstate->GetOpaqueDataInConn(motNodeID, targetRoute); + } catch (...) { + handleException(); + } + + return NULL; +} + +int32* +UDP2_GetSentRecordTypmodInConn(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute) +{ + Assert(transportStates); + + try { + CChunkTransportState *cstate = static_cast(transportStates); + return cstate->GetSentRecordTypmodInConn(motNodeID, targetRoute); + } catch (...) { + handleException(); + } + + return NULL; +} + +void +UDP2_InitUDPIFC(GlobalMotionLayerIPCParam *param) +{ + if (param) + { + if (global_param.interconnect_address) + free(global_param.interconnect_address); + + memcpy(&global_param, param, sizeof(*param)); + global_param.interconnect_address = strdup(param->interconnect_address); + } + + try { + InitMotionUDPIFC(&UDP_listenerFd, &udp_listener_port); + + if (IC_DEBUG1 >= session_param.log_min_messages) + LOG(DEBUG1, "Interconnect listening on udp port %d ", udp_listener_port); + + } catch (...) { + handleException(); + } +} + +void +UDP2_CleanUpUDPIFC(void) +{ + if (session_param.gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG_IC) + LOG(DEBUG3, "Cleaning Up Motion Layer IPC..."); + + try { + CleanupMotionUDPIFC(); + } catch (...) { + handleException(); + } + + if (UDP_listenerFd >= 0) + closesocket(UDP_listenerFd); + + /* be safe and reset global state variables. */ + udp_listener_port = 0; + UDP_listenerFd = -1; +} + +void +UDP2_WaitQuitUDPIFC(void) +{ + /* + * Just in case ic thread is waiting on the locks. + */ + pthread_mutex_unlock(&ic_control_info.lock); + + ic_atomic_write_u32(&ic_control_info.shutdown, 1); + + if (ic_control_info.threadCreated) + { + SendDummyPacket(); + pthread_join(ic_control_info.threadHandle, NULL); + } + ic_control_info.threadCreated = false; +} + +uint32 +UDP2_GetActiveConns(void) +{ + return ic_statistics.activeConnectionsNum; +} + +int +UDP2_GetICHeaderSizeUDP(void) +{ + return sizeof(struct icpkthdr); +} + +int32 +UDP2_GetListenPortUDP(void) +{ + return udp_listener_port; +} + +#ifdef __cplusplus +} // extern "C" +#endif \ No newline at end of file diff --git a/contrib/udp2/ic_common/udp2/ic_udp2.h b/contrib/udp2/ic_common/udp2/ic_udp2.h new file mode 100644 index 00000000000..e49da698b79 --- /dev/null +++ b/contrib/udp2/ic_common/udp2/ic_udp2.h @@ -0,0 +1,102 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_udp2.h + * + * IDENTIFICATION + * contrib/udp2/ic_common/udp2/ic_udp2.h + * + *------------------------------------------------------------------------- + */ + +#ifndef IC_UDP2_H +#define IC_UDP2_H + +#include "ic_types.h" + +#ifdef __cplusplus +extern "C" { +#endif + +extern void UDP2_InitUDPIFC(struct GlobalMotionLayerIPCParam *param); +extern void UDP2_CleanUpUDPIFC(void); +extern void UDP2_WaitQuitUDPIFC(void); + +extern ICChunkTransportState* UDP2_SetupUDP(ICSliceTable *sliceTable, + SessionMotionLayerIPCParam *param); +extern void UDP2_TeardownUDP(ICChunkTransportState *transportStates, + bool hasErrors); + +// recv +extern void UDP2_RecvRoute(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 srcRoute, + GetDataLenInPacket getLen, + DataBlock *data); +extern void UDP2_RecvAny(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 *srcRoute, + GetDataLenInPacket getLen, + DataBlock *data); +extern void UDP2_SendStop(ICChunkTransportState *transportStates, int16 motNodeID); +extern void UDP2_ReleaseAndAck(ICChunkTransportState *transportStates, + int motNodeID, + int route); +extern void UDP2_DeactiveRoute(ICChunkTransportState *transportStates, + int motNodeID, + int srcRoute, + const char *reason); + +// send +extern void UDP2_SendEOS(ICChunkTransportState *transportStates, + int motNodeID, + DataBlock *data); +extern bool UDP2_SendData(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + DataBlock *pblocks, + int num, + bool broadcast); +extern void UDP2_GetFreeSpace(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + BufferBlock *b); +extern void UDP2_ReduceFreeSpace(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + int length); + + +// utility func +extern void* UDP2_GetOpaqueDataInConn(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute); +extern int32* UDP2_GetSentRecordTypmodInConn(ICChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute); + +extern uint32 UDP2_GetActiveConns(void); +extern int UDP2_GetICHeaderSizeUDP(void); +extern int32 UDP2_GetListenPortUDP(void); + +#ifdef __cplusplus +} +#endif + +#endif // IC_UDP2_H \ No newline at end of file diff --git a/contrib/udp2/ic_common/udp2/ic_udp2.hpp b/contrib/udp2/ic_common/udp2/ic_udp2.hpp new file mode 100644 index 00000000000..a7e6356fb9c --- /dev/null +++ b/contrib/udp2/ic_common/udp2/ic_udp2.hpp @@ -0,0 +1,68 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_udp2.hpp + * + * IDENTIFICATION + * contrib/udp2/ic_common/udp2/ic_udp2.hpp + * + *------------------------------------------------------------------------- + */ +#ifndef IC_UDP2_HPP +#define IC_UDP2_HPP + +#include +#include +#include +#include +#include + +#include "ic_types.h" +#include "ic_except.hpp" + +struct CChunkTransportState : public ICChunkTransportState +{ +public: + static ICChunkTransportState* SetupUDP(ICSliceTable *sliceTable, SessionMotionLayerIPCParam *param); + void TeardownUDP(bool hasErrors); + + void RecvRoute(int16 motNodeID, int16 srcRoute, GetDataLenInPacket getLen, DataBlock *data); + void RecvAny(int16 motNodeID, int16 *srcRoute, GetDataLenInPacket getLen, DataBlock *data); + void ReleaseAndAck(int motNodeID, int route); + void SendStop(int16 motNodeID); + void DeactiveRoute(int motNodeID, int srcRoute, const char *reason); + + void SendEOS(int motNodeID, DataBlock *data); + bool SendData(int16 motNodeID, int16 targetRoute, DataBlock *pblocks, int num, bool broadcast); + void GetFreeSpace(int16 motNodeID, int16 targetRoute, BufferBlock *b); + void ReduceFreeSpace(int16 motNodeID, int16 targetRoute, int length); + + void* GetOpaqueDataInConn(int16 motNodeID, int16 targetRoute); + int32* GetSentRecordTypmodInConn(int16 motNodeID, int16 targetRoute); + + int GetConnNum(int motNodeID); + + static CChunkTransportState** GetTransportState(); + + /* APIs for vector engine */ + void NotifyQuit(); + void SetVectorEngineAsUser(); +}; + +#endif // IC_UDP2_HPP \ No newline at end of file diff --git a/contrib/udp2/ic_common/udp2/ic_udp2_internal.hpp b/contrib/udp2/ic_common/udp2/ic_udp2_internal.hpp new file mode 100644 index 00000000000..baeb6a0841c --- /dev/null +++ b/contrib/udp2/ic_common/udp2/ic_udp2_internal.hpp @@ -0,0 +1,1213 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_udp2_internal.hpp + * + * IDENTIFICATION + * contrib/udp2/ic_common/udp2/ic_udp2_internal.hpp + * + *------------------------------------------------------------------------- + */ +#ifndef IC_UDP2_INTERNAL_HPP +#define IC_UDP2_INTERNAL_HPP + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "ic_udp2.hpp" +#include "ic_utility.hpp" + +namespace { + +typedef enum MotionConnState +{ + mcsNull, + mcsAccepted, + mcsSetupOutgoingConnection, + mcsConnecting, + mcsRecvRegMsg, + mcsSendRegMsg, + mcsStarted, + mcsEosSent +} MotionConnState; + +/* + * Structure used for keeping track of a pt-to-pt connection between two + * Cdb Entities (either QE or QD). + */ +typedef struct MotionConn +{ + /* socket file descriptor. */ + int sockfd; + + /* pointer to the data buffer. */ + uint8 *pBuff; + + /* size of the message in the buffer, if any. */ + int32 msgSize; + + /* position of message inside of buffer, "cursor" pointer */ + uint8 *msgPos; + + /* + * recv bytes: we can have more than one message/message fragment in recv + * queue at once + */ + int32 recvBytes; + + int tupleCount; + + /* + * false means 1) received a stop message and has handled it. 2) received + * EOS message or sent out EOS message 3) received a QueryFinishPending + * notify and has handled it. + */ + bool stillActive; + + /* + * used both by motion sender and motion receiver + * + * sender: true means receiver don't need to consume tuples any more, + * sender is also responsible to send stop message to its senders. + * + * receiver: true means have sent out a stop message to its senders. The + * stop message might be lost, stopRequested can also tell sender that no + * more data needed in the ack message. + */ + bool stopRequested; + + MotionConnState state; + + ICCdbProcess *cdbProc; + int remoteContentId; + char remoteHostAndPort[128]; /* Numeric IP addresses should never + * be longer than about 50 chars, but + * play it safe */ + + void *opaque_data; + + /* + * used by the sender. + * + * the typmod of last sent record type in current connection, + * if the connection is for broadcasting then we only check + * and update this attribute on connection 0. + */ + int32 sent_record_typmod; + +} MotionConn; + +/* + * Used to organize all of the information for a given motion node. + */ +typedef struct ChunkTransportStateEntry +{ + int motNodeId; + bool valid; + + /* Connection array + * + * MUST pay attention: use getMotionConn to get MotionConn. + * must not use `->conns[index]` to get MotionConn. Because the struct + * MotionConn is a base structure for MotionConnTCP and + * MotionConnUDP. After connection setup, the `conns` will be fill + * with MotionConnUDP/MotionConnTCP, but the pointer still is + * MotionConn which should use `CONTAINER_OF` to get the real object. + */ + MotionConn *conns; + int numConns; + + int scanStart; + + /* slice table entries */ + struct ICExecSlice *sendSlice; + struct ICExecSlice *recvSlice; + +} ChunkTransportStateEntry; + +typedef struct icpkthdr +{ + int32 motNodeId; + + /* + * three pairs which seem useful for identifying packets. + * + * MPP-4194: It turns out that these can cause collisions; but the high + * bit (1<<31) of the dstListener port is now used for disambiguation with + * mirrors. + */ + int32 srcPid; + int32 srcListenerPort; + + int32 dstPid; + int32 dstListenerPort; + + int32 sessionId; + int32 icId; + + int32 recvSliceIndex; + int32 sendSliceIndex; + int32 srcContentId; + int32 dstContentId; + + /* MPP-6042: add CRC field */ + uint32 crc; + + /* packet specific info */ + int32 flags; + uint32 len; + + /* + * The usage of seq and extraSeq field + * a) In a normal DATA packet + * seq -> the data packet sequence number + * extraSeq -> not used + * b) In a normal ACK message (UDPIC_FLAGS_ACK | UDPIC_FLAGS_CAPACITY) + * seq -> the largest seq of the continuously cached packets + * sometimes, it is special, for exampke, conn req ack, mismatch ack. + * extraSeq -> the largest seq of the consumed packets + * c) In a start race NAK message (UPDIC_FLAGS_NAK) + * seq -> the seq from the pkt + * extraSeq -> the extraSeq from the pkt + * d) In a DISORDER message (UDPIC_FLAGS_DISORDER) + * seq -> packet sequence number that triggers the disorder message + * extraSeq -> the largest seq of the received packets + * e) In a DUPLICATE message (UDPIC_FLAGS_DUPLICATE) + * seq -> packet sequence number that triggers the duplicate message + * extraSeq -> the largest seq of the continuously cached packets + * f) In a stop messege (UDPIC_FLAGS_STOP | UDPIC_FLAGS_ACK | UDPIC_FLAGS_CAPACITY) + * seq -> the largest seq of the continuously cached packets + * extraSeq -> the largest seq of the continuously cached packets + * + * + * NOTE that: EOS/STOP flags are often saved in conn_info structure of a connection. + * It is possible for them to be sent together with other flags. + * + */ + uint32 seq; + uint32 extraSeq; +} icpkthdr; + +typedef struct ICBuffer ICBuffer; +typedef struct ICBufferLink ICBufferLink; + +typedef enum ICBufferListType +{ + ICBufferListType_Primary, + ICBufferListType_Secondary, + ICBufferListType_UNDEFINED +} ICBufferListType; + +struct ICBufferLink +{ + ICBufferLink *next; + ICBufferLink *prev; +}; + +/* + * ICBufferList + * ic buffer list data structure. + * + * There are two kinds of lists. The first kind of list uses the primary next/prev pointers. + * And the second kind uses the secondary next/prev pointers. + */ +struct ICBufferList +{ + int len; + ICBufferListType type; /* primary or secondary */ + + ICBufferLink head; + +#if defined(USE_ASSERT_CHECKING) || defined(AMS_VERBOSE_LOGGING) + void icBufferListLog(); +#endif + +#ifdef USE_ASSERT_CHECKING + void icBufferListCheck(const char *prefix); +#endif + + void init(ICBufferListType type); + void destroy(); + + bool is_head(ICBufferLink *link); + int length(); + ICBufferLink* first(); + + ICBuffer* append(ICBuffer *buf); + ICBuffer* remove(ICBuffer *buf); + ICBuffer* pop(); + + void release(bool inExpirationQueue); + + void dump_to_file(FILE *ofile); +}; + +#define GET_ICBUFFER_FROM_PRIMARY(ptr) CONTAINER_OF(ptr, ICBuffer, primary) +#define GET_ICBUFFER_FROM_SECONDARY(ptr) CONTAINER_OF(ptr, ICBuffer, secondary) + +/* + * ICBuffer + * interconnect buffer data structure. + * + * In some cases, an ICBuffer may exists in two lists/queues, + * thus it has two sets of pointers. For example, an ICBuffer + * can exist in an unack queue and an expiration queue at the same time. + * + * It is important to get the ICBuffer address when we iterate a list of + * ICBuffers through primary/secondary links. The Macro GET_ICBUFFER_FROM_PRIMARY + * and GET_ICBUFFER_FROM_SECONDARY are for this purpose. + * + */ +struct ICBuffer +{ + /* primary next and prev pointers */ + ICBufferLink primary; + + /* secondary next and prev pointers */ + ICBufferLink secondary; + + /* connection that this buffer belongs to */ + MotionConn *conn; + + /* + * Three fields for expiration processing + * + * sentTime - the time this buffer was sent nRetry - the number of send + * retries unackQueueRingSlot - unack queue ring slot index + */ + uint64 sentTime; + int32 nRetry; + int32 unackQueueRingSlot; + + /* real data */ + icpkthdr pkt[0]; +}; + +static inline void* +ic_malloc(size_t size) +{ + return malloc(size); +} + +static inline void* +ic_malloc0(size_t size) +{ + void *rs = ic_malloc(size); + if (rs) + memset(rs, 0, size); + return rs; +} + +static inline void +ic_free(void *p) +{ + free(p); +} + +static inline void +ic_free_clean(void **p) +{ + ic_free(*p); + *p = NULL; +} + +static inline void +ic_usleep(long microsec) +{ + if (microsec > 0) + { + struct timeval delay; + + delay.tv_sec = microsec / 1000000L; + delay.tv_usec = microsec % 1000000L; + (void) select(0, NULL, NULL, NULL, &delay); + } +} + +/* + * Put socket into nonblock mode. + * Returns true on success, false on failure. + */ +static inline bool +ic_set_noblock(int sock) +{ + int flags; + + flags = fcntl(sock, F_GETFL); + if (flags < 0) + return false; + if (fcntl(sock, F_SETFL, (flags | O_NONBLOCK)) == -1) + return false; + return true; +} + +/* ic_atomic_xxx */ +typedef struct ic_atomic_uint32 +{ + volatile uint32 value; +} ic_atomic_uint32; + +static inline void +ic_atomic_init_u32(volatile ic_atomic_uint32 *ptr, uint32 val) +{ + ptr->value = val; +} + +static inline uint32 +ic_atomic_read_u32(volatile ic_atomic_uint32 *ptr) +{ + return ptr->value; +} + +static inline void +ic_atomic_write_u32(volatile ic_atomic_uint32 *ptr, uint32 val) +{ + ptr->value = val; +} + +static inline bool +ic_atomic_compare_exchange_u32(volatile ic_atomic_uint32 *ptr, + uint32 *expected, uint32 newval) +{ + /* FIXME: we can probably use a lower consistency model */ + return __atomic_compare_exchange_n(&ptr->value, expected, newval, false, + __ATOMIC_SEQ_CST, __ATOMIC_SEQ_CST); +} + +static inline uint32 +ic_atomic_add_fetch_u32(volatile ic_atomic_uint32 *ptr, int32 add_) +{ + return __sync_fetch_and_add(&ptr->value, add_) + add_; +} + +static inline uint32 +ic_bswap32(uint32 x) +{ + return + ((x << 24) & 0xff000000) | + ((x << 8) & 0x00ff0000) | + ((x >> 8) & 0x0000ff00) | + ((x >> 24) & 0x000000ff); +} + +#define MAX_TRY (11) +#define TIMEOUT(try) ((try) < MAX_TRY ? (timeoutArray[(try)]) : (timeoutArray[MAX_TRY])) + +#define USECS_PER_SECOND 1000000 +#define MSECS_PER_SECOND 1000 + +/* 1/4 sec in msec */ +#define RX_THREAD_POLL_TIMEOUT (250) + +/* + * Flags definitions for flag-field of UDP-messages + * + * We use bit operations to test these, flags are powers of two only + */ +#define UDPIC_FLAGS_RECEIVER_TO_SENDER (1) +#define UDPIC_FLAGS_ACK (2) +#define UDPIC_FLAGS_STOP (4) +#define UDPIC_FLAGS_EOS (8) +#define UDPIC_FLAGS_NAK (16) +#define UDPIC_FLAGS_DISORDER (32) +#define UDPIC_FLAGS_DUPLICATE (64) +#define UDPIC_FLAGS_CAPACITY (128) + +#define UDPIC_MIN_BUF_SIZE (128 * 1024) + +/* + * ConnHtabBin + * + * A connection hash table bin. + * + */ +struct UDPConn; +typedef struct ConnHtabBin ConnHtabBin; +struct ConnHtabBin +{ + UDPConn *conn; + struct ConnHtabBin *next; +}; + +/* + * ConnHashTable + * + * Connection hash table definition. + * + */ +typedef struct ConnHashTable ConnHashTable; +struct ConnHashTable +{ + ConnHtabBin **table; + int size; + + bool init(); + bool add(UDPConn *conn); + UDPConn *find(icpkthdr *hdr); + void destroy(); + void remove(UDPConn *conn); +}; + +#define CONN_HASH_VALUE(icpkt) ((uint32)((((icpkt)->srcPid ^ (icpkt)->dstPid)) + (icpkt)->dstContentId)) +#define CONN_HASH_MATCH(a, b) (((a)->motNodeId == (b)->motNodeId && \ + (a)->dstContentId == (b)->dstContentId && \ + (a)->srcContentId == (b)->srcContentId && \ + (a)->recvSliceIndex == (b)->recvSliceIndex && \ + (a)->sendSliceIndex == (b)->sendSliceIndex && \ + (a)->srcPid == (b)->srcPid && \ + (a)->dstPid == (b)->dstPid && (a)->icId == (b)->icId)) + +/* + * CursorICHistoryEntry + * + * The definition of cursor IC history entry. + */ +typedef struct CursorICHistoryEntry CursorICHistoryEntry; +struct CursorICHistoryEntry +{ + /* Interconnect instance id. */ + uint32 icId; + + /* Command id. */ + uint32 cid; + + /* + * Interconnect instance status. state 1 (value 1): interconnect is setup + * state 0 (value 0): interconnect was torn down. + */ + uint8 status; + + /* Next entry. */ + CursorICHistoryEntry *next; + + CursorICHistoryEntry(uint32 aicId, uint32 acid): + icId(aicId), cid(acid),status(1){} +}; + +/* + * CursorICHistoryTable + * + * Cursor IC history table. It is a small hash table. + */ +typedef struct CursorICHistoryTable CursorICHistoryTable; +struct CursorICHistoryTable +{ + uint32 size; + uint32 count; + CursorICHistoryEntry **table; + + void init() { + count = 0; + size = session_param.Gp_interconnect_cursor_ic_table_size; + table = (CursorICHistoryEntry **)ic_malloc0(sizeof(CursorICHistoryEntry *) * size); + } + + void add(uint32 icId, uint32 cid) { + uint32 index = icId % size; + CursorICHistoryEntry *p = new CursorICHistoryEntry(icId, cid); + + p->next = this->table[index]; + this->table[index] = p; + this->count++; + + LOG(DEBUG2, "add icid %d cid %d status %d", p->icId, p->cid, p->status); + + return; + } + + /* + * state 1 (value 1): interconnect is setup + * state 0 (value 0): interconnect was torn down. + */ + void update(uint32 icId, uint8 status) { + for (CursorICHistoryEntry *p = table[icId % size]; p; p = p->next) { + if (p->icId == icId) { + p->status = status; + return; + } + } + } + + CursorICHistoryEntry* get(uint32 icId) { + for (CursorICHistoryEntry *p = table[icId % size]; p; p = p->next) { + if (p->icId == icId) + return p; + } + return NULL; + } + + void purge() { + for (uint8 index = 0; index < size; index++) { + while (table[index]) { + CursorICHistoryEntry *trash = table[index]; + table[index] = trash->next; + delete trash; + } + } + } + + void prune(uint32 icId); +}; + +/* + * Synchronization timeout values + * + * MAIN_THREAD_COND_TIMEOUT - 1/4 second + */ +#define MAIN_THREAD_COND_TIMEOUT_MS (250) + +/* + * Used for synchronization between main thread (receiver) and background thread. + * + */ +typedef struct ThreadWaitingState ThreadWaitingState; +struct ThreadWaitingState +{ + bool waiting; + int waitingNode; + int waitingRoute; + int reachRoute; + + /* main_thread_waiting_query is needed to disambiguate for cursors */ + int waitingQuery; + + void reset() { + waiting = false; + waitingNode = -1; + waitingRoute = ANY_ROUTE; + reachRoute = ANY_ROUTE; + waitingQuery = -1; + } + + void set(int motNodeId, int route, int icId) { + waiting = true; + waitingNode = motNodeId; + waitingRoute = route; + reachRoute = ANY_ROUTE; + waitingQuery = icId; + } +}; + +/* + * ReceiveControlInfo + * + * The related control information for receiving data packets. + * Main thread (Receiver) and background thread use the information in + * this data structure to handle data packets. + * + */ +typedef struct ReceiveControlInfo ReceiveControlInfo; +struct ReceiveControlInfo +{ + /* Main thread waiting state. */ + ThreadWaitingState mainWaitingState; + + /* + * Buffers used to assemble disorder messages at receiver side. + */ + icpkthdr *disorderBuffer; + + /* The last interconnect instance id which is torn down. */ + int32 lastTornIcId; + + /* Cursor history table. */ + CursorICHistoryTable cursorHistoryTable; + + /* + * Last distributed transaction id when SetupUDPInterconnect is called. + * Coupled with cursorHistoryTable, it is used to handle multiple + * concurrent cursor cases. + */ + DistributedTransactionId lastDXatId; +}; + +/* + * RxBufferPool + * + * Receive thread buffer pool definition. The implementation of + * receive side buffer pool is different from send side buffer pool. + * It is because receive side buffer pool needs a ring buffer to + * easily implement disorder message handling logic. + */ + +typedef struct RxBufferPool RxBufferPool; +struct RxBufferPool +{ + /* The max number of buffers we can get from this pool. */ + int maxCount; + + /* The number of allocated buffers */ + int count; + + /* The list of free buffers. */ + char *freeList; + + void put(icpkthdr *buf); + void release(icpkthdr *buf); + icpkthdr* get(); + icpkthdr* get_free(); +}; + +/* + * SendBufferPool + * + * The send side buffer pool definition. + * + */ +typedef struct SendBufferPool SendBufferPool; +struct SendBufferPool +{ + /* The maximal number of buffers sender can use. */ + int maxCount; + + /* The number of buffers sender already used. */ + int count; + + /* The free buffer list at the sender side. */ + ICBufferList freeList; + + void init(); + void clean(); + ICBuffer* get(UDPConn *conn); +}; + +/* + * SendControlInfo + * + * The related control information for sending data packets and handling acks. + * Main thread use the information in this data structure to do ack handling + * and congestion control. + * + */ +typedef struct SendControlInfo SendControlInfo; +struct SendControlInfo +{ + /* The buffer used for accepting acks */ + icpkthdr *ackBuffer; + + /* congestion window */ + float cwnd; + + /* minimal congestion control window */ + float minCwnd; + + /* slow start threshold */ + float ssthresh; + +}; + +/* + * ICGlobalControlInfo + * + * Some shared control information that is used by main thread (senders, receivers, or both) + * and the background thread. + * + */ +typedef struct ICGlobalControlInfo ICGlobalControlInfo; +struct ICGlobalControlInfo +{ + /* The background thread handle. */ + pthread_t threadHandle; + + /* Keep the udp socket buffer size used. */ + uint32 socketSendBufferSize; + uint32 socketRecvBufferSize; + + uint64 lastExpirationCheckTime; + uint64 lastDeadlockCheckTime; + + /* Used to decide whether to retransmit for capacity based FC. */ + uint64 lastPacketSendTime; + + /* + * Lock and latch for coordination between main thread and + * background thread. It protects the shared data between the two threads + * (the connHtab, rx buffer pool and the mainWaitingState etc.). + */ + pthread_mutex_t lock; + + /* Am I a sender? */ + bool isSender; + + /* Flag showing whether the thread is created. */ + bool threadCreated; + + /* Error number. Actually int but we do not have ic_atomic_int32. */ + ic_atomic_uint32 eno; + + /* + * Global connection htab for both sending connections and receiving + * connections. Protected by the lock in this data structure. + */ + ConnHashTable connHtab; + + /* The connection htab used to cache future packets. */ + ConnHashTable startupCacheHtab; + + /* Used by main thread to ask the background thread to exit. */ + ic_atomic_uint32 shutdown; + + /*Serialization + * Used by ic thread in the QE to identify the current serving ic instance + * and handle the mismatch packets. It is not used by QD because QD may have + * cursors, QD may receive packets for open the cursors with lower instance + * id, QD use cursorHistoryTable to handle packets mismatch. + */ + int32 ic_instance_id; +}; + +/* + * Macro for unack queue ring, round trip time (RTT) and expiration period (RTO) + * + * UNACK_QUEUE_RING_SLOTS_NUM - the number of slots in the unack queue ring. + * this value should be greater than or equal to 2. + * TIMER_SPAN - timer period in us + * TIMER_CHECKING_PERIOD - timer checking period in us + * UNACK_QUEUE_RING_LENGTH - the whole time span of the unack queue ring + * DEFAULT_RTT - default rtt in us. + * MIN_RTT - min rtt in us + * MAX_RTT - max rtt in us + * RTT_SHIFT_COEFFICIENT - coefficient for RTT computation + * + * DEFAULT_DEV - default round trip standard deviation + * MAX_DEV - max dev + * DEV_SHIFT_COEFFICIENT - coefficient for DEV computation + * + * MAX_EXPIRATION_PERIOD - max expiration period in us + * MIN_EXPIRATION_PERIOD - min expiration period in us + * MAX_TIME_NO_TIMER_CHECKING - max time without checking timer + * DEADLOCK_CHECKING_TIME - deadlock checking time + * + * MAX_SEQS_IN_DISORDER_ACK - max number of sequences that can be transmitted in a + * disordered packet ack. + * + * + * Considerations on the settings of the values: + * + * TIMER_SPAN and UNACK_QUEUE_RING_SLOTS_NUM define the ring period. + * Currently, it is UNACK_QUEUE_RING_LENGTH (default 10 seconds). + * + * The definition of UNACK_QUEUE_RING_LENGTH is quite related to the size of + * sender side buffer and the size we may resend in a burst for an expiration event + * (which may overwhelm switch or OS if it is too large). + * Thus, we do not want to send too much data in a single expiration event. Here, a + * relatively large UNACK_QUEUE_RING_SLOTS_NUM value is used to avoid that. + * + * If the sender side buffer is X (MB), then on each slot, + * there are about X/UNACK_QUEUE_RING_SLOTS_NUM. Even we have a very large sender buffer, + * for example, 100MB, there is about 96M/2000 = 50K per slot. + * This is fine for the OS (with buffer 2M for each socket generally) and switch. + * + * Note that even when the buffers are not evenly distributed in the ring and there are some packet + * losses, the congestion control mechanism, the disorder and duplicate packet handling logic will + * assure the number of outstanding buffers (in unack queues) to be not very large. + * + * MIN_RTT/MAX_RTT/DEFAULT_RTT/MIN_EXPIRATION_PERIOD/MAX_EXPIRATION_PERIOD gives some heuristic values about + * the computation of RTT and expiration period. RTT and expiration period (RTO) are not + * constant for various kinds of hardware and workloads. Thus, they are computed dynamically. + * But we also want to bound the values of RTT and MAX_EXPIRATION_PERIOD. It is + * because there are some faults that may make RTT a very abnormal value. Thus, RTT and + * expiration period are upper and lower bounded. + * + * MAX_SEQS_IN_DISORDER_ACK should be smaller than (MIN_PACKET_SIZE - sizeof(icpkthdr))/sizeof(uint32). + * It is due to the limitation of the ack receive buffer size. + * + */ +#define UNACK_QUEUE_RING_SLOTS_NUM (2000) +#define TIMER_SPAN (session_param.Gp_interconnect_timer_period * 1000ULL) /* default: 5ms */ +#define TIMER_CHECKING_PERIOD (session_param.Gp_interconnect_timer_checking_period) /* default: 20ms */ +#define UNACK_QUEUE_RING_LENGTH (UNACK_QUEUE_RING_SLOTS_NUM * TIMER_SPAN) + +#define DEFAULT_RTT (session_param.Gp_interconnect_default_rtt * 1000) /* default: 20ms */ +#define MIN_RTT (100) /* 0.1ms */ +#define MAX_RTT (200 * 1000) /* 200ms */ +#define RTT_SHIFT_COEFFICIENT (3) /* RTT_COEFFICIENT 1/8 (0.125) */ + +#define DEFAULT_DEV (0) +#define MIN_DEV MIN_RTT +#define MAX_DEV MAX_RTT +#define DEV_SHIFT_COEFFICIENT (2) /* DEV_COEFFICIENT 1/4 (0.25) */ + +#define MAX_EXPIRATION_PERIOD (1000 * 1000) /* 1s */ +#define MIN_EXPIRATION_PERIOD (session_param.Gp_interconnect_min_rto * 1000) /* default: 20ms */ + +#define MAX_TIME_NO_TIMER_CHECKING (50 * 1000) /* 50ms */ +#define DEADLOCK_CHECKING_TIME (512 * 1000) /* 512ms */ + +#define MAX_SEQS_IN_DISORDER_ACK (4) + +/* + * UnackQueueRing + * + * An unacked queue ring is used to decide which packet is expired in constant time. + * + * Each slot of the ring represents a fixed time span, for example 1ms, and + * each slot has a associated buffer list/queue which contains the packets + * which will expire in the time span. + * + * If the current time pointer (time t) points to slot 1, + * then slot 2 represents the time span from t + 1ms to t + 2ms. + * When we check whether there are some packets expired, we start from the last + * current time recorded, and resend all the packets in the queue + * until we reach the slot that the updated current time points to. + * + */ +typedef struct UnackQueueRing UnackQueueRing; +struct UnackQueueRing +{ + /* save the current time when we check the time wheel for expiration */ + uint64 currentTime; + + /* the slot index corresponding to current time */ + int idx; + + /* the number of outstanding packets in unack queue ring */ + int numOutStanding; + + /* + * the number of outstanding packets that use the shared bandwidth in the + * congestion window. + */ + int numSharedOutStanding; + + /* time slots */ + ICBufferList slots[UNACK_QUEUE_RING_SLOTS_NUM]; +}; + +/* + * AckSendParam + * + * The parameters for ack sending. + */ +typedef struct AckSendParam +{ + /* header for the ack */ + icpkthdr msg; + + /* peer address for the ack */ + struct sockaddr_storage peer; + socklen_t peer_len; +} AckSendParam; + +/* + * ICStatistics + * + * A structure keeping various statistics about interconnect internal. + * + * Note that the statistics for ic are not accurate for multiple cursor case on QD. + * + * totalRecvQueueSize - receive queue size sum when main thread is trying to get a packet. + * recvQueueSizeCountingTime - counting times when computing totalRecvQueueSize. + * totalCapacity - the capacity sum when packets are tried to be sent. + * capacityCountingTime - counting times used to compute totalCapacity. + * totalBuffers - total buffers available when sending packets. + * bufferCountingTime - counting times when compute totalBuffers. + * activeConnectionsNum - the number of active connections. + * retransmits - the number of packet retransmits. + * mismatchNum - the number of mismatched packets received. + * crcErrors - the number of crc errors. + * sndPktNum - the number of packets sent by sender. + * recvPktNum - the number of packets received by receiver. + * disorderedPktNum - disordered packet number. + * duplicatedPktNum - duplicate packet number. + * recvAckNum - the number of Acks received. + * statusQueryMsgNum - the number of status query messages sent. + * + */ +typedef struct ICStatistics +{ + uint64 totalRecvQueueSize; + uint64 recvQueueSizeCountingTime; + uint64 totalCapacity; + uint64 capacityCountingTime; + uint64 totalBuffers; + uint64 bufferCountingTime; + uint32 activeConnectionsNum; + int32 retransmits; + int32 startupCachedPktNum; + int32 mismatchNum; + int32 crcErrors; + int32 sndPktNum; + int32 recvPktNum; + int32 disorderedPktNum; + int32 duplicatedPktNum; + int32 recvAckNum; + int32 statusQueryMsgNum; +} ICStatistics; + +struct TransportEntry; + +/* + * Structure used for keeping track of a pt-to-pt connection between two + * Cdb Entities (either QE or QD). + */ +struct UDPConn : public MotionConn +{ +public: + /* send side queue for packets to be sent */ + ICBufferList sndQueue; + int capacity; + + /* seq already sent */ + uint32 sentSeq; + + /* ack of this seq and packets with smaller seqs have been received */ + uint32 receivedAckSeq; + + /* packets with this seq or smaller seqs have been consumed */ + uint32 consumedSeq; + + uint64 rtt; + uint64 dev; + uint64 deadlockCheckBeginTime; + + ICBuffer *curBuff; + + /* + * send side unacked packet queue. Since it is often accessed at the same + * time with unack queue ring, it is protected with unqck queue ring lock. + */ + ICBufferList unackQueue; + + uint16 route; + + struct icpkthdr conn_info; + + struct sockaddr_storage peer; /* Allow for IPv4 or IPv6 */ + socklen_t peer_len; /* And remember the actual length */ + + /* a queue of maximum length Gp_interconnect_queue_depth */ + uint32 pkt_q_capacity; /* max capacity of the queue */ + uint32 pkt_q_size; /* number of packets in the queue */ + int pkt_q_head; + int pkt_q_tail; + uint8 **pkt_q; + + uint64 stat_total_ack_time; + uint64 stat_count_acks; + uint64 stat_max_ack_time; + uint64 stat_min_ack_time; + uint64 stat_count_resent; + uint64 stat_max_resent; + uint64 stat_count_dropped; + + TransportEntry *entry_; + +public: + UDPConn(TransportEntry *entry); + + void GetDataInBuf(GetDataLenInPacket getLen, DataBlock *data); + void ReleaseBuffer(AckSendParam *param); + + void setAckParam(AckSendParam *param, int32 flags, uint32 seq, uint32 extraSeq); + void sendAck(int32 flags, uint32 seq, uint32 extraSeq); + void sendDisorderAck(uint32 seq, uint32 extraSeq, uint32 lostPktCnt); + void sendStatusQueryMessage(uint32 seq); + + uint64 computeExpirationPeriod(uint32 retry); + + void freeDisorderedPackets(); + void prepareRxConnForRead(); + void DeactiveConn(); + + void handleAckedPacket(ICBuffer *buf, uint64 now); + void prepareXmit(); + void sendOnce(icpkthdr *pkt); + void handleStop(); + void sendBuffers(); + + void handleDisorderPacket(int pos, uint32 tailSeq, icpkthdr *pkt); + bool handleAckForDisorderPkt(icpkthdr *pkt); + bool handleAckForDuplicatePkt(icpkthdr *pkt); + int computeTimeout(int retry); + + void Send(DataBlock *data); + + void checkDeadlock(); + void checkExceptions(int retry, int timeout); + + void updateRetransmitStatistics(); + void checkExpirationCapacityFC(int timeout); + + static void checkNetworkTimeout(ICBuffer *buf, uint64 now, bool *networkTimeoutIsLogged); + static void checkExpiration(ICChunkTransportState *transportStates, uint64 now); + + static void sendAckWithParam(AckSendParam *param); + static void sendControlMessage(icpkthdr *pkt, int fd, struct sockaddr *addr, socklen_t peerLen); +}; + + +/* + * Used to organize all of the information for a given motion node. + */ +struct CChunkTransportStateEntry +{ + int motNodeId; + bool valid; + + /* Connection array + * + * MUST pay attention: use getMotionConn to get MotionConn. + * must not use `->conns[index]` to get MotionConn. Because the struct + * MotionConn is a base structure for MotionConnTCP and + * MotionConnUDP. After connection setup, the `conns` will be fill + * with MotionConnUDP/MotionConnTCP, but the pointer still is + * MotionConn which should use `CONTAINER_OF` to get the real object. + */ + MotionConn *conns; + int numConns; + + int scanStart; + + /* slice table entries */ + struct ICExecSlice *sendSlice; + struct ICExecSlice *recvSlice; +}; + +class CChunkTransportStateImpl; + +class TransportEntry : public CChunkTransportStateEntry +{ +public: + static std::unique_ptr + MakeRecvEntry(CChunkTransportStateImpl *state, int icid, ICExecSlice *sendSlice, ICExecSlice *recvSlice); + + static std::unique_ptr + MakeSendEntry(CChunkTransportStateImpl *state, int icid, ICExecSlice *sendSlice, ICExecSlice *recvSlice); + + TransportEntry(CChunkTransportStateImpl *state, int motNodeID, int numConns, ICExecSlice *sendSlice, ICExecSlice *recvSlice); + + UDPConn* GetConn(int index); + + void aggregateStatistics(); + + bool handleAcks(); + void handleStopMsgs(); + + bool pollAcks(int timeout); + + void dumpConnections(const char *fname); + + bool SendData(int16 targetRoute, DataBlock *pblocks, int num, bool broadcast); + void Broadcast(DataBlock *data, int *inactiveCountPtr); + void Send(int16 targetRoute, DataBlock *db, bool broadcast, int *recount); + + void RecvAny(int16 *srcRoute, GetDataLenInPacket getLen, DataBlock *data); + void RecvRoute(int16 srcRoute, GetDataLenInPacket getLen, DataBlock *data); + void receiveChunksUDPIFC(int16 *srcRoute, UDPConn *conn, GetDataLenInPacket getLen, DataBlock *data); + +public: + /* setup info */ + int txfd; + int txfd_family; + unsigned short txport; + + bool sendingEos; + + /* Statistics info for this motion on the interconnect level */ + uint64 stat_total_ack_time; + uint64 stat_count_acks; + uint64 stat_max_ack_time; + uint64 stat_min_ack_time; + uint64 stat_count_resent; + uint64 stat_max_resent; + uint64 stat_count_dropped; + + std::vector> conns_; + CChunkTransportStateImpl *state; +}; + + +class CChunkTransportStateImpl : public CChunkTransportState +{ +public: + CChunkTransportStateImpl(ICSliceTable *sliceTable); + + static ICChunkTransportState* SetupUDP(ICSliceTable *sliceTable, SessionMotionLayerIPCParam *param); + void TeardownUDP(bool hasErrors); + + void RecvRoute(int16 motNodeID, int16 srcRoute, GetDataLenInPacket getLen, DataBlock *data); + void RecvAny(int16 motNodeID, int16 *srcRoute, GetDataLenInPacket getLen, DataBlock *data); + void SendStop(int16 motNodeID); + void ReleaseAndAck(int motNodeID, int route); + void DeactiveRoute(int motNodeID, int srcRoute, const char *reason); + + void SendEOS(int motNodeID, DataBlock *data); + bool SendData(int16 motNodeID, int16 targetRoute, DataBlock *pblocks, int num, bool broadcast); + void GetFreeSpace(int16 motNodeID, int16 targetRoute, BufferBlock *b); + void ReduceFreeSpace(int16 motNodeID, int16 targetRoute, int length); + + void* GetOpaqueDataInConn(int16 motNodeID, int16 targetRoute); + int32* GetSentRecordTypmodInConn(int16 motNodeID, int16 targetRoute); + + int GetConnNum(int motNodeID); + + TransportEntry* GetEntry(int motNodeID, bool checkValid); + + static CChunkTransportState **GetTransportState(); + + /* APIs for vector engine */ + void NotifyQuit(); + void SetVectorEngineAsUser(); + +private: + void checkMotNodeID(int sendMotNodeID); + void CreateRecvEntries(ICSliceTable *sliceTable); + void CreateSendEntries(ICSliceTable *sliceTable); + void DestroyRecvEntries(bool *isReceiver); + void DestroySendEntries(); + static ICChunkTransportState* setup(ICSliceTable *sliceTable); + void teardown(bool hasErrors); + + std::vector> entries_; + + static CChunkTransportState *state_; +}; + +} // namespace + +#endif /* IC_UDP2_INTERNAL_HPP */ \ No newline at end of file diff --git a/contrib/udp2/ic_modules.c b/contrib/udp2/ic_modules.c new file mode 100644 index 00000000000..47843f08c99 --- /dev/null +++ b/contrib/udp2/ic_modules.c @@ -0,0 +1,88 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_modules.c + * + * IDENTIFICATION + * contrib/udp2/ic_modules.c + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "cdb/ml_ipc.h" +#include "ic_modules.h" +#include "ic_udp2.h" + +PG_MODULE_MAGIC; + +static +MotionIPCLayer udp2_ipc_layer = { + .ic_type = INTERCONNECT_TYPE_UDP2, + .type_name = "udp2", + + .GetMaxTupleChunkSize = GetMaxTupleChunkSizeUDP2, + .GetListenPort = GetListenPortUDP2, + + .InitMotionLayerIPC = InitMotionIPCLayerUDP2, + .CleanUpMotionLayerIPC = CleanUpMotionLayerIPCUDP2, + .WaitInterconnectQuit = WaitInterconnectQuitUDPIFC2, + .SetupInterconnect = SetupInterconnectUDP2, + .TeardownInterconnect = TeardownInterconnectUDP2, + + .SendTupleChunkToAMS = SendTupleChunkToAMSUDP2, + .SendChunk = NULL, + .SendEOS = SendEOSUDPIFC2, + .SendStopMessage = SendStopMessageUDPIFC2, + + .RecvTupleChunkFromAny = RecvTupleChunkFromAnyUDPIFC2, + .RecvTupleChunkFrom = RecvTupleChunkFromUDPIFC2, + .RecvTupleChunk = NULL, + + .DirectPutRxBuffer = MlPutRxBufferUDPIFC2, + + .DeregisterReadInterest = DeregisterReadInterestUDP2, + .GetActiveMotionConns = GetActiveMotionConnsUDPIFC2, + + .GetTransportDirectBuffer = GetTransportDirectBufferUDPIFC2, + .PutTransportDirectBuffer = PutTransportDirectBufferUDPIFC2, + +#ifdef ENABLE_IC_PROXY + .IcProxyServiceMain = ic_proxy_server_main, +#else + .IcProxyServiceMain = NULL, +#endif + + .GetMotionConnTupleRemapper = GetMotionConnTupleRemapperUDPIFC2, + .GetMotionSentRecordTypmod = GetMotionSentRecordTypmodUDPIFC2, +}; + +void +_PG_init(void) +{ + if (!process_shared_preload_libraries_in_progress) + { + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not load udp2 outside process shared preload"))); + } + + RegisterIPCLayerImpl(&udp2_ipc_layer); +} \ No newline at end of file diff --git a/contrib/udp2/ic_modules.h b/contrib/udp2/ic_modules.h new file mode 100644 index 00000000000..a05b65894f3 --- /dev/null +++ b/contrib/udp2/ic_modules.h @@ -0,0 +1,32 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_modules.h + * + * IDENTIFICATION + * contrib/udp2/ic_modules.h + * + *------------------------------------------------------------------------- + */ +#ifndef INTER_CONNECT_H +#define INTER_CONNECT_H + +extern void _PG_init(void); + +#endif // INTER_CONNECT_H \ No newline at end of file diff --git a/contrib/udp2/ic_udp2.c b/contrib/udp2/ic_udp2.c new file mode 100644 index 00000000000..31c0d5c2eeb --- /dev/null +++ b/contrib/udp2/ic_udp2.c @@ -0,0 +1,989 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_udp2.c + * + * IDENTIFICATION + * contrib/udp2/ic_udp2.c + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "cdb/cdbdisp.h" +#include "cdb/cdbgang.h" +#include "cdb/cdbmotion.h" +#include "cdb/cdbvars.h" +#include "cdb/tupchunklist.h" +#include "postmaster/bgworker.h" +#include "postmaster/postmaster.h" +#include "storage/latch.h" +#include "storage/pmsignal.h" +#include "tcop/tcopprot.h" +#include "utils/wait_event.h" +#include "utils/memutils.h" + +/* local interconnect */ +#include "ic_udp2.h" + +/* from ic_common packeage */ +#include "ic_types.h" +#include "udp2/ic_udp2.h" + + +#define HandleLastError() \ +do { \ + ICError *error = GetLastError(); \ + Assert(error); \ + if (error->level == LEVEL_ERROR) \ + { \ + Assert(error->msg); \ + elog(ERROR, "%s", error->msg); \ + } \ + if (error->level == LEVEL_FATAL) \ + { \ + Assert(error->msg); \ + elog(FATAL, "%s", error->msg); \ + } \ +} while (0) + +#define ML_CHECK_FOR_INTERRUPTS(teardownActive) \ + do {if (!teardownActive && InterruptPending) CHECK_FOR_INTERRUPTS(); } while (0) + +/* + * Resource manager + */ +typedef void (*TeardownInterconnectCallBack)(ChunkTransportState *transportStates, bool hasErrors); +typedef struct interconnect_handle_t +{ + ChunkTransportState *interconnect_context; /* Interconnect state */ + + // callback for interconnect been abort + TeardownInterconnectCallBack teardown_cb; + + ResourceOwner owner; /* owner of this handle */ + struct interconnect_handle_t *next; + struct interconnect_handle_t *prev; +} interconnect_handle_t; + +static interconnect_handle_t * open_interconnect_handles; +static bool interconnect_resowner_callback_registered; + +static void destroy_interconnect_handle(interconnect_handle_t *h); +static interconnect_handle_t * allocate_interconnect_handle(TeardownInterconnectCallBack callback); +static interconnect_handle_t * find_interconnect_handle(ChunkTransportState *icContext); + + +static void SetupGlobalMotionLayerIPCParam(GlobalMotionLayerIPCParam *param); +static void SetupSessionMotionLayerIPCParam(SessionMotionLayerIPCParam *param); +static bool CheckPostmasterIsAlive(void); +static void CheckCancelOnQD(ICChunkTransportState *pTransportStates); +static void CheckInterrupts(int teardownActive); +static void SimpleFaultInjector(const char *faultname); +static void *CreateOpaqueData(void); +static void DestroyOpaqueData(void **opaque); +static ICSliceTable* ConvertToICSliceTable(SliceTable *tbl); +static TupleChunkListItem ConvertToTupleChunk(ChunkTransportState *transportStates, DataBlock *data); +static ChunkTransportState *CreateChunkTransportState(EState *estate, ICChunkTransportState *udp2_state); + + +int +GetMaxTupleChunkSizeUDP2(void) +{ + int header_size = UDP2_GetICHeaderSizeUDP(); + return Gp_max_packet_size - header_size - TUPLE_CHUNK_HEADER_SIZE; +} + +int32 +GetListenPortUDP2(void) +{ + return UDP2_GetListenPortUDP(); +} + +void +InitMotionIPCLayerUDP2(void) +{ + GlobalMotionLayerIPCParam param; + SetupGlobalMotionLayerIPCParam(¶m); + + param.checkPostmasterIsAliveCallback = CheckPostmasterIsAlive; + param.checkInterruptsCallback = CheckInterrupts; + param.simpleFaultInjectorCallback = SimpleFaultInjector; + + param.createOpaqueDataCallback = CreateOpaqueData; + param.destroyOpaqueDataCallback = DestroyOpaqueData; + + param.checkCancelOnQDCallback = CheckCancelOnQD; + + ResetLastError(); + UDP2_InitUDPIFC(¶m); + HandleLastError(); +} + +void +CleanUpMotionLayerIPCUDP2(void) +{ + if (gp_log_interconnect >= GPVARS_VERBOSITY_DEBUG) + elog(DEBUG3, "Cleaning Up Motion Layer IPC..."); + + ResetLastError(); + UDP2_CleanUpUDPIFC(); + HandleLastError(); +} + +void +WaitInterconnectQuitUDPIFC2(void) +{ + ResetLastError(); + UDP2_WaitQuitUDPIFC(); + HandleLastError(); +} + +void +SetupInterconnectUDP2(EState *estate) +{ + if (estate->interconnect_context) + elog(ERROR, "SetupInterconnectUDP: already initialized."); + + if (!estate->es_sliceTable) + elog(ERROR, "SetupInterconnectUDP: no slice table ?"); + + SessionMotionLayerIPCParam param; + SetupSessionMotionLayerIPCParam(¶m); + + interconnect_handle_t *h; + h = allocate_interconnect_handle(TeardownInterconnectUDP2); + + ICSliceTable *tbl = ConvertToICSliceTable(estate->es_sliceTable); + + ResetLastError(); + ICChunkTransportState *udp2_state = UDP2_SetupUDP(tbl, ¶m); + HandleLastError(); + + Assert(udp2_state); + ChunkTransportState *state = CreateChunkTransportState(estate, udp2_state); + h->interconnect_context = state; + + h->interconnect_context->estate = estate; + estate->interconnect_context = h->interconnect_context; + estate->es_interconnect_is_setup = true; + + /* Check if any of the QEs has already finished with error */ + if (Gp_role == GP_ROLE_DISPATCH) + { + ChunkTransportState *pTransportStates = h->interconnect_context; + + Assert(pTransportStates); + Assert(pTransportStates->estate); + + if (cdbdisp_checkForCancel(pTransportStates->estate->dispatcherState)) + { + ereport(ERROR, (errcode(ERRCODE_GP_INTERCONNECTION_ERROR), + errmsg(CDB_MOTION_LOST_CONTACT_STRING))); + /* not reached */ + } + } +} + +void +TeardownInterconnectUDP2(ChunkTransportState *transportStates, bool hasErrors) +{ + if (transportStates == NULL || transportStates->sliceTable == NULL) + { + elog(LOG, "TeardownUDPIFCInterconnect: missing slice table."); + return; + } + + /* TODO: should pass interconnect_handle_t as arg? */ + interconnect_handle_t *h = find_interconnect_handle(transportStates); + + ResetLastError(); + HOLD_INTERRUPTS(); + + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + UDP2_TeardownUDP(udp2_state, hasErrors); + + transportStates->activated = false; + transportStates->sliceTable = NULL; + + RESUME_INTERRUPTS(); + HandleLastError(); + + if (h != NULL) + destroy_interconnect_handle(h); +} + +bool +SendTupleChunkToAMSUDP2(ChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + TupleChunkListItem tcItem) +{ + if (!transportStates) + { + elog(FATAL, "SendTupleChunkToAMS: no transport-states."); + } + + if (!transportStates->activated) + { + elog(FATAL, "SendTupleChunkToAMS: transport states inactive"); + } + + /* check em' */ + ML_CHECK_FOR_INTERRUPTS(transportStates->teardownActive); + +#ifdef AMS_VERBOSE_LOGGING + elog(DEBUG3, "sendtuplechunktoams: calling get_transport_state" + "w/transportStates %p transportState->size %d motnodeid %d route %d", + transportStates, transportStates->size, motNodeID, targetRoute); +#endif + + /* get the number of TupleChunkListItem */ + int num = 0; + TupleChunkListItem item = tcItem; + while (item) + { + num++; + item = item->p_next; + } + + /* convert to DataBlock */ + DataBlock *pblocks = (DataBlock *)palloc0(sizeof(DataBlock) * num); + item = tcItem; + for (int i = 0; i < num; ++i) + { + pblocks[i].pos = item->chunk_data; + pblocks[i].len = item->chunk_length; + + item = item->p_next; + } + + bool broadcast = (targetRoute == BROADCAST_SEGIDX); + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + bool rs = UDP2_SendData(udp2_state, + motNodeID, + targetRoute, + pblocks, + num, + broadcast); + HandleLastError(); + + return rs; +} + +void +SendEOSUDPIFC2(ChunkTransportState *transportStates, + int motNodeID, + TupleChunkListItem tcItem) +{ + if (!transportStates) + { + elog(FATAL, "SendEOSUDPIFC: missing interconnect context."); + } + else if (!transportStates->activated && !transportStates->teardownActive) + { + elog(FATAL, "SendEOSUDPIFC: context and teardown inactive."); + } + + /* check em' */ + ML_CHECK_FOR_INTERRUPTS(transportStates->teardownActive); + + DataBlock db; + db.pos = tcItem->chunk_data; + db.len = tcItem->chunk_length; + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + UDP2_SendEOS(udp2_state, motNodeID, &db); + HandleLastError(); +} + +void +SendStopMessageUDPIFC2(ChunkTransportState *transportStates, int16 motNodeID) +{ + if (!transportStates->activated) + return; + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + UDP2_SendStop(udp2_state, motNodeID); + HandleLastError(); +} + +TupleChunkListItem +RecvTupleChunkFromAnyUDPIFC2(ChunkTransportState *transportStates, + int16 motNodeID, + int16 *srcRoute) +{ + if (!transportStates) + { + elog(FATAL, "RecvTupleChunkFromAnyUDPIFC: missing context"); + } + else if (!transportStates->activated) + { + elog(FATAL, "RecvTupleChunkFromAnyUDPIFC: interconnect context not active!"); + } + + DataBlock db = {NULL, 0}; + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + UDP2_RecvAny(udp2_state, motNodeID, srcRoute, NULL, &db); + HandleLastError(); + + if (db.pos == NULL) + return NULL; + + return ConvertToTupleChunk(transportStates, &db); +} + +TupleChunkListItem +RecvTupleChunkFromUDPIFC2(ChunkTransportState *transportStates, + int16 motNodeID, + int16 srcRoute) +{ + if (!transportStates) + { + elog(FATAL, "RecvTupleChunkFromUDPIFC: missing context"); + } + else if (!transportStates->activated) + { + elog(FATAL, "RecvTupleChunkFromUDPIFC: interconnect context not active!"); + } + +#ifdef AMS_VERBOSE_LOGGING + elog(LOG, "RecvTupleChunkFromUDPIFC()."); +#endif + + /* check em' */ + ML_CHECK_FOR_INTERRUPTS(transportStates->teardownActive); + +#ifdef AMS_VERBOSE_LOGGING + elog(DEBUG5, "RecvTupleChunkFromUDPIFC(motNodID=%d, srcRoute=%d)", motNodeID, srcRoute); +#endif + + DataBlock db = {NULL, 0}; + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + UDP2_RecvRoute(udp2_state, motNodeID, srcRoute, NULL, &db); + HandleLastError(); + + if (db.pos == NULL) + return NULL; + + return ConvertToTupleChunk(transportStates, &db); +} + +void +MlPutRxBufferUDPIFC2(ChunkTransportState *transportStates, int motNodeID, int route) +{ + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + UDP2_ReleaseAndAck(udp2_state, motNodeID, route); + HandleLastError(); +} + +void +DeregisterReadInterestUDP2(ChunkTransportState *transportStates, + int motNodeID, + int srcRoute, + const char *reason) +{ + if (!transportStates) + { + elog(FATAL, "DeregisterReadInterestUDP: no transport states"); + } + + if (!transportStates->activated) + return; + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + UDP2_DeactiveRoute(udp2_state, motNodeID, srcRoute, reason); + HandleLastError(); +} + +uint32 +GetActiveMotionConnsUDPIFC2(void) +{ + return UDP2_GetActiveConns(); +} + +void +GetTransportDirectBufferUDPIFC2(ChunkTransportState * transportStates, + int16 motNodeID, + int16 targetRoute, + struct directTransportBuffer *b) +{ + if (!transportStates) + { + elog(FATAL, "GetTransportDirectBuffer: no transport states"); + } + else if (!transportStates->activated) + { + elog(FATAL, "GetTransportDirectBuffer: inactive transport states"); + } + else if (targetRoute == BROADCAST_SEGIDX) + { + elog(FATAL, "GetTransportDirectBuffer: can't direct-transport to broadcast"); + } + + BufferBlock buf = {NULL, 0}; + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + UDP2_GetFreeSpace(udp2_state, motNodeID, targetRoute, &buf); + HandleLastError(); + + b->pri = buf.pos; + b->prilen = buf.len; +} + +void +PutTransportDirectBufferUDPIFC2(ChunkTransportState * transportStates, + int16 motNodeID, + int16 targetRoute, + int length) +{ + if (!transportStates) + { + elog(FATAL, "PutTransportDirectBuffer: no transport states"); + } + else if (!transportStates->activated) + { + elog(FATAL, "PutTransportDirectBuffer: inactive transport states"); + } + else if (targetRoute == BROADCAST_SEGIDX) + { + elog(FATAL, "PutTransportDirectBuffer: can't direct-transport to broadcast"); + } + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + UDP2_ReduceFreeSpace(udp2_state, motNodeID, targetRoute, length); + HandleLastError(); +} + +TupleRemapper* +GetMotionConnTupleRemapperUDPIFC2(ChunkTransportState * transportStates, + int16 motNodeID, + int16 targetRoute) +{ + TupleRemapper *remapper = NULL; + + if (!transportStates) + { + elog(FATAL, "GetMotionConnTupleRemapper: no transport states"); + } + + if (!transportStates->activated) + { + elog(FATAL, "GetMotionConnTupleRemapper: inactive transport states"); + } + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + remapper = (TupleRemapper *)UDP2_GetOpaqueDataInConn(udp2_state, motNodeID, targetRoute); + HandleLastError(); + + return remapper; +} + +int32* +GetMotionSentRecordTypmodUDPIFC2(ChunkTransportState * transportStates, + int16 motNodeID, + int16 targetRoute) +{ + int32 *rs = NULL; + + if (!transportStates) + { + elog(FATAL, "GetMotionConnTupleRemapper: no transport states"); + } + + if (!transportStates->activated) + { + elog(FATAL, "GetMotionConnTupleRemapper: inactive transport states"); + } + + targetRoute = targetRoute == BROADCAST_SEGIDX ? 0 : targetRoute; + + ResetLastError(); + ICChunkTransportState *udp2_state = + (ICChunkTransportState *)transportStates->implement_state; + rs = UDP2_GetSentRecordTypmodInConn(udp2_state, motNodeID, targetRoute); + HandleLastError(); + + return rs; +} + +static void +SetupGlobalMotionLayerIPCParam(GlobalMotionLayerIPCParam *param) +{ + if (param == NULL) + return; + + param->interconnect_address = interconnect_address; + param->Gp_role = Gp_role; + param->ic_htab_size = ic_htab_size; + param->segment_number = getgpsegmentCount(); + param->MyProcPid = MyProcPid; + param->dbid = GpIdentity.dbid; + param->segindex = GpIdentity.segindex; + param->MyProcPort = MyProcPort != NULL; + param->myprocport_sock = param->MyProcPort ? MyProcPort->sock : -1; + param->Gp_max_packet_size = Gp_max_packet_size; + param->Gp_udp_bufsize_k = Gp_udp_bufsize_k; + param->Gp_interconnect_address_type = Gp_interconnect_address_type; +} + +static bool +CheckPostmasterIsAlive(void) +{ + return PostmasterIsAlive(); +} + +static void +CheckCancelOnQD(ICChunkTransportState *state) +{ + int nevent = 0, nrevent = 0; + int *waitFds = NULL; + WaitEvent *rEvents = NULL; + WaitEventSet *waitset = NULL; + ChunkTransportState *pTransportStates = NULL; + + if (Gp_role != GP_ROLE_DISPATCH) + return; + + pTransportStates = (ChunkTransportState *)state->clientState; + + /* get all wait sock fds */ + waitFds = cdbdisp_getWaitSocketFds(pTransportStates->estate->dispatcherState, &nevent); + if (waitFds == NULL) + return; + + /* init WaitEventSet */ + waitset = CreateWaitEventSet(CurrentMemoryContext, nevent); + rEvents = palloc(nevent * sizeof(WaitEvent)); /* returned events */ + for (int i = 0; i < nevent; ++i) + AddWaitEventToSet(waitset, WL_SOCKET_READABLE, waitFds[i], NULL, NULL); + + /* wait for event from QE */ + nrevent = WaitEventSetWait(waitset, 0, rEvents, nevent, WAIT_EVENT_INTERCONNECT); + + /* check to see if the dispatcher should cancel */ + for (int i = 0; i < nrevent; i++) + { + if (rEvents[i].events & WL_SOCKET_READABLE) + { + /* event happened on wait fds, need to check cancel */ + Assert(pTransportStates); + Assert(pTransportStates->estate); + + if (cdbdisp_checkForCancel(pTransportStates->estate->dispatcherState)) + { + ereport(ERROR, (errcode(ERRCODE_GP_INTERCONNECTION_ERROR), + errmsg(CDB_MOTION_LOST_CONTACT_STRING))); + /* not reached */ + } + break; + } + } + + if (waitset) + FreeWaitEventSet((WaitEventSet *)waitset); + if (rEvents) + pfree(rEvents); +} + +static void +CheckInterrupts(int teardownActive) +{ + ML_CHECK_FOR_INTERRUPTS(teardownActive); +} + +static void +SimpleFaultInjector(const char *faultname) +{ + SIMPLE_FAULT_INJECTOR((faultname)); +} + +static void * +CreateOpaqueData(void) +{ + return CreateTupleRemapper(); +} + +static void +DestroyOpaqueData(void **opaque) +{ + if (*opaque == NULL) + return; + *opaque = NULL; +} + +static void +SetupSessionMotionLayerIPCParam(SessionMotionLayerIPCParam *param) +{ + if (param == NULL) + return; + + TransactionId localTransId = 0; + TransactionId subtransId = 0; + + param->Gp_interconnect_queue_depth = Gp_interconnect_queue_depth; + param->Gp_interconnect_snd_queue_depth = Gp_interconnect_snd_queue_depth; + param->Gp_interconnect_timer_period = Gp_interconnect_timer_period; + param->Gp_interconnect_timer_checking_period = Gp_interconnect_timer_checking_period; + param->Gp_interconnect_default_rtt = Gp_interconnect_default_rtt; + param->Gp_interconnect_min_rto = Gp_interconnect_min_rto; + param->Gp_interconnect_transmit_timeout = Gp_interconnect_transmit_timeout; + param->Gp_interconnect_min_retries_before_timeout = Gp_interconnect_min_retries_before_timeout; + param->Gp_interconnect_debug_retry_interval = Gp_interconnect_debug_retry_interval; + param->gp_interconnect_full_crc = gp_interconnect_full_crc; + param->gp_interconnect_aggressive_retry = gp_interconnect_aggressive_retry; + param->gp_interconnect_cache_future_packets = gp_interconnect_cache_future_packets; + param->gp_interconnect_log_stats = gp_interconnect_log_stats; + param->interconnect_setup_timeout = interconnect_setup_timeout; + param->gp_log_interconnect = gp_log_interconnect; + param->gp_session_id = gp_session_id; + param->Gp_interconnect_fc_method = Gp_interconnect_fc_method; + param->gp_command_count = gp_command_count; + param->gp_interconnect_id = gp_interconnect_id; + param->log_min_messages = log_min_messages; + GetAllTransactionXids(¶m->distTransId, &localTransId, &subtransId); + +#ifdef USE_ASSERT_CHECKING + param->gp_udpic_dropseg = gp_udpic_dropseg; + param->gp_udpic_dropacks_percent = gp_udpic_dropacks_percent; + param->gp_udpic_dropxmit_percent = gp_udpic_dropxmit_percent; + param->gp_udpic_fault_inject_percent = gp_udpic_fault_inject_percent; + param->gp_udpic_fault_inject_bitmap = gp_udpic_fault_inject_bitmap; + param->gp_udpic_network_disable_ipv6 = gp_udpic_network_disable_ipv6; +#endif +} + +ICSliceTable* +ConvertToICSliceTable(SliceTable *tbl) +{ + ICSliceTable *ic_tbl = (ICSliceTable *)malloc(sizeof(ICSliceTable)); + memset(ic_tbl, 0, sizeof(ICSliceTable)); + + ic_tbl->localSlice = tbl->localSlice; + ic_tbl->ic_instance_id = tbl->ic_instance_id; + + ic_tbl->numSlices = tbl->numSlices; + ic_tbl->slices = (ICExecSlice *)malloc(sizeof(ICExecSlice) * ic_tbl->numSlices); + memset(ic_tbl->slices, 0, sizeof(ICExecSlice) * ic_tbl->numSlices); + + for (int i = 0; i < ic_tbl->numSlices; ++i) + { + ExecSlice *slice = tbl->slices + i; + ICExecSlice *ic_slice = ic_tbl->slices + i; + + ic_slice->sliceIndex = slice->sliceIndex; + ic_slice->parentIndex= slice->parentIndex; + ic_slice->numSegments = list_length(slice->segments); + + ic_slice->numChildren = list_length(slice->children); + ic_slice->children = malloc(sizeof(int) * ic_slice->numChildren); + memset(ic_slice->children, 0, sizeof(int) * ic_slice->numChildren); + + for (int i = 0; i < ic_slice->numChildren; ++i) + ic_slice->children[i] = list_nth_int(slice->children, i); + + ic_slice->numPrimaryProcesses = list_length(slice->primaryProcesses); + ic_slice->primaryProcesses = malloc(sizeof(ICCdbProcess) * ic_slice->numPrimaryProcesses); + memset(ic_slice->primaryProcesses, 0, sizeof(ICCdbProcess) * ic_slice->numPrimaryProcesses); + + for (int i = 0; i < ic_slice->numPrimaryProcesses; ++i) + { + CdbProcess *process = (CdbProcess *)list_nth(slice->primaryProcesses, i); + if (!process) + continue; + + ICCdbProcess *ic_process = ic_slice->primaryProcesses + i; + + ic_process->valid = true; + ic_process->listenerAddr = process->listenerAddr; + ic_process->listenerPort = process->listenerPort; + ic_process->pid = process->pid; + ic_process->contentid = process->contentid; + ic_process->dbid = process->dbid; + } + } + + return ic_tbl; +} + +/* + * msg MUST BE conn->msgPos, msg_size should be conn->msgSize - sizeof(icpkthdr) + * +----------------+-----------+--------------+------------+---+--------------+------------+ + * | tcp/udp header | ic header | chunk header | chunk data |...| chunk header | chunk data | + * +----------------+-----------+--------------+------------+---+--------------+------------+ + * |<-----#1 tuple chunk ----->|...|<-----#n tuple chunk ----->| + * |<------------------------- Gp_max_packet_size ------------------------>| + * + * |<------------------------ msg_size ----------------------->| + * msg + */ +TupleChunkListItem +ConvertToTupleChunk(ChunkTransportState *transportStates, DataBlock *data) +{ + TupleChunkListItem tcItem; + TupleChunkListItem firstTcItem = NULL; + TupleChunkListItem lastTcItem = NULL; + + uint32 tcSize; + int bytesProcessed = 0; + + while (bytesProcessed != data->len) + { + if (data->len - bytesProcessed < TUPLE_CHUNK_HEADER_SIZE) + { + ereport(ERROR, + (errcode(ERRCODE_GP_INTERCONNECTION_ERROR), + errmsg("interconnect error parsing message: insufficient data received"), + errdetail("conn->msgSize %d bytesProcessed %d < chunk-header %d", + data->len, bytesProcessed, TUPLE_CHUNK_HEADER_SIZE))); + } + tcSize = TUPLE_CHUNK_HEADER_SIZE + (*(uint16 *) (data->pos + bytesProcessed)); + + /* sanity check */ + if (tcSize > Gp_max_packet_size) + { + /* + * see MPP-720: it is possible that our message got messed up by a + * cancellation ? + */ + ML_CHECK_FOR_INTERRUPTS(transportStates->teardownActive); + /* + * MPP-4010: add some extra debugging. + */ + if (lastTcItem != NULL) + elog(LOG, "Interconnect error parsing message: last item length %d inplace %p", lastTcItem->chunk_length, lastTcItem->inplace); + else + elog(LOG, "Interconnect error parsing message: no last item"); + + ereport(ERROR, + (errcode(ERRCODE_GP_INTERCONNECTION_ERROR), + errmsg("interconnect error parsing message"), + errdetail("tcSize %d > max %d header %d processed %d/%d from %p", + tcSize, Gp_max_packet_size, + TUPLE_CHUNK_HEADER_SIZE, bytesProcessed, + data->len, data->pos))); + } + + Assert(tcSize <= data->len); + + /* + * We store the data inplace, and handle any necessary copying later + * on + */ + tcItem = (TupleChunkListItem) palloc(sizeof(TupleChunkListItemData)); + tcItem->p_next = NULL; + tcItem->chunk_length = tcSize; + tcItem->inplace = (char *) (data->pos + bytesProcessed); + + bytesProcessed += tcSize; + if (firstTcItem == NULL) + { + firstTcItem = tcItem; + lastTcItem = tcItem; + } + else + { + lastTcItem->p_next = tcItem; + lastTcItem = tcItem; + } + } + + return firstTcItem; +} + +static ChunkTransportState * +CreateChunkTransportState(EState *estate, ICChunkTransportState *udp2_state) +{ + MemoryContext oldContext; + ChunkTransportState *state; + + /* init ChunkTransportState */ + Assert(InterconnectContext != NULL); + oldContext = MemoryContextSwitchTo(InterconnectContext); + state = (ChunkTransportState *)palloc0(sizeof(ChunkTransportState)); + MemoryContextSwitchTo(oldContext); + + state->size = 0; + state->states = NULL; + state->activated = udp2_state->activated; + state->teardownActive = udp2_state->teardownActive; + state->aggressiveRetry = false; + state->incompleteConns = NIL; + state->sliceTable = estate->es_sliceTable; + state->sliceId = estate->es_sliceTable->localSlice; + state->estate = estate; + state->proxyContext = NULL; + + state->networkTimeoutIsLogged = false; + + /* save the reference each other */ + state->implement_state = udp2_state; + udp2_state->clientState = state; + + return state; +} + +/* + * must offer an empty proxy fucntion if ic-proxy is enabled(--enable-ic-proxy). + */ +int +ic_proxy_server_main(void) +{ + /* Establish signal handlers. */ + pqsignal(SIGTERM, die); + BackgroundWorkerUnblockSignals(); + + /* dry run */ + while (true) + { + pg_usleep(1000000); + CHECK_FOR_INTERRUPTS(); + } + + return 0; +} + +/* + * Fucntions for Resource manager + */ +static void +destroy_interconnect_handle(interconnect_handle_t * h) +{ + h->interconnect_context = NULL; + /* unlink from linked list first */ + if (h->prev) + h->prev->next = h->next; + else + open_interconnect_handles = h->next; + if (h->next) + h->next->prev = h->prev; + + pfree(h); + + if (open_interconnect_handles == NULL) + MemoryContextReset(InterconnectContext); +} + +static void +cleanup_interconnect_handle(interconnect_handle_t * h) +{ + if (h->interconnect_context == NULL) + { + destroy_interconnect_handle(h); + return; + } + h->teardown_cb(h->interconnect_context, true); +} + +static void +interconnect_abort_callback(ResourceReleasePhase phase, + bool isCommit, + bool isTopLevel, + void *arg) +{ + interconnect_handle_t *curr; + interconnect_handle_t *next; + + if (phase != RESOURCE_RELEASE_AFTER_LOCKS) + return; + + next = open_interconnect_handles; + while (next) + { + curr = next; + next = curr->next; + + if (curr->owner == CurrentResourceOwner) + { + if (isCommit) + elog(WARNING, "interconnect reference leak: %p still referenced", curr); + + cleanup_interconnect_handle(curr); + } + } +} + +static interconnect_handle_t * +allocate_interconnect_handle(TeardownInterconnectCallBack callback) +{ + interconnect_handle_t *h; + + if (InterconnectContext == NULL) + InterconnectContext = AllocSetContextCreate(TopMemoryContext, + "Interconnect Context", + ALLOCSET_DEFAULT_MINSIZE, + ALLOCSET_DEFAULT_INITSIZE, + ALLOCSET_DEFAULT_MAXSIZE); + + h = MemoryContextAllocZero(InterconnectContext, sizeof(interconnect_handle_t)); + + h->teardown_cb = callback; + h->owner = CurrentResourceOwner; + h->next = open_interconnect_handles; + h->prev = NULL; + if (open_interconnect_handles) + open_interconnect_handles->prev = h; + open_interconnect_handles = h; + + if (!interconnect_resowner_callback_registered) + { + RegisterResourceReleaseCallback(interconnect_abort_callback, NULL); + interconnect_resowner_callback_registered = true; + } + return h; +} + +static interconnect_handle_t * +find_interconnect_handle(ChunkTransportState * icContext) +{ + interconnect_handle_t *head = open_interconnect_handles; + + while (head != NULL) + { + if (head->interconnect_context == icContext) + return head; + head = head->next; + } + return NULL; +} \ No newline at end of file diff --git a/contrib/udp2/ic_udp2.h b/contrib/udp2/ic_udp2.h new file mode 100644 index 00000000000..c351d2d0224 --- /dev/null +++ b/contrib/udp2/ic_udp2.h @@ -0,0 +1,87 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * ic_udp2.h + * + * IDENTIFICATION + * contrib/udp2/ic_udp2.h + * + *------------------------------------------------------------------------- + */ +#ifndef IC_UDP_H +#define IC_UDP_H + +#include "cdb/cdbinterconnect.h" +#include "nodes/execnodes.h" /* EState, ExecSlice, SliceTable */ + +extern int GetMaxTupleChunkSizeUDP2(void); +extern int32 GetListenPortUDP2(void); + +extern void InitMotionIPCLayerUDP2(void); +extern void CleanUpMotionLayerIPCUDP2(void); + +extern void WaitInterconnectQuitUDPIFC2(void); + +extern void SetupInterconnectUDP2(EState *estate); +extern void TeardownInterconnectUDP2(ChunkTransportState * transportStates, bool hasErrors); + +extern bool SendTupleChunkToAMSUDP2(ChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + TupleChunkListItem tcItem); +extern void SendEOSUDPIFC2(ChunkTransportState * transportStates, + int motNodeID, TupleChunkListItem tcItem); +extern void SendStopMessageUDPIFC2(ChunkTransportState * transportStates, int16 motNodeID); + +extern TupleChunkListItem RecvTupleChunkFromAnyUDPIFC2(ChunkTransportState * transportStates, + int16 motNodeID, + int16 *srcRoute); +extern TupleChunkListItem RecvTupleChunkFromUDPIFC2(ChunkTransportState * transportStates, + int16 motNodeID, + int16 srcRoute); + +void MlPutRxBufferUDPIFC2(ChunkTransportState * transportStates, int motNodeID, int route); + +extern void DeregisterReadInterestUDP2(ChunkTransportState * transportStates, + int motNodeID, + int srcRoute, + const char *reason); + +extern uint32 GetActiveMotionConnsUDPIFC2(void); + +extern void GetTransportDirectBufferUDPIFC2(ChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + struct directTransportBuffer *b); +extern void PutTransportDirectBufferUDPIFC2(ChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute, + int length); + +extern TupleRemapper* GetMotionConnTupleRemapperUDPIFC2(ChunkTransportState *transportStates, + int16 motNodeID, + int16 targetRoute); + +extern int32* GetMotionSentRecordTypmodUDPIFC2(ChunkTransportState * transportStates, + int16 motNodeID, + int16 targetRoute); + +extern int ic_proxy_server_main(void); + +#endif // IC_UDP_H \ No newline at end of file diff --git a/src/Makefile.global.in b/src/Makefile.global.in index 86e86b467fe..062ec75b039 100644 --- a/src/Makefile.global.in +++ b/src/Makefile.global.in @@ -233,6 +233,7 @@ enable_shared_postgres_backend = @enable_shared_postgres_backend@ enable_link_postgres_with_shared = @enable_link_postgres_with_shared@ enable_gpcloud = @enable_gpcloud@ enable_ic_proxy = @enable_ic_proxy@ +enable_ic_udp2 = @enable_ic_udp2@ enable_pax = @enable_pax@ enable_tap_tests = @enable_tap_tests@ diff --git a/src/backend/cdb/cdbvars.c b/src/backend/cdb/cdbvars.c index 7984a9d61a4..c6540a8a8d5 100644 --- a/src/backend/cdb/cdbvars.c +++ b/src/backend/cdb/cdbvars.c @@ -201,6 +201,7 @@ int Gp_interconnect_queue_depth = 4; /* max number of messages * we drop. */ int Gp_interconnect_snd_queue_depth = 2; int Gp_interconnect_mem_size = 10; +int Gp_interconnect_cursor_ic_table_size = 128; int Gp_interconnect_timer_period = 5; int Gp_interconnect_timer_checking_period = 20; int Gp_interconnect_default_rtt = 20; diff --git a/src/backend/utils/misc/guc_gp.c b/src/backend/utils/misc/guc_gp.c index 4eb88be4220..42b61dfbbbb 100644 --- a/src/backend/utils/misc/guc_gp.c +++ b/src/backend/utils/misc/guc_gp.c @@ -3792,6 +3792,17 @@ struct config_int ConfigureNamesInt_gp[] = NULL, NULL, NULL }, + { + {"gp_interconnect_cursor_ic_table_size", PGC_USERSET, GP_ARRAY_TUNING, + gettext_noop("Sets the size of Cursor History Table in the UDP interconnect"), + gettext_noop("You can try to increase it when a UDF which contains many concurrent " + "cursor queries hangs. The default value is 128.") + }, + &Gp_interconnect_cursor_ic_table_size, + 128, 128, 102400, + NULL, NULL, NULL + }, + { {"gp_interconnect_timer_period", PGC_USERSET, GP_ARRAY_TUNING, gettext_noop("Sets the timer period (in ms) for UDP interconnect"), @@ -5102,6 +5113,9 @@ struct config_string ConfigureNamesString_gp[] = {"gp_interconnect_type", PGC_BACKEND, GP_ARRAY_TUNING, gettext_noop("Sets the protocol used for inter-node communication."), gettext_noop("Valid values are \"tcp\", \"udpifc\"" +#ifdef ENABLE_IC_UDP2 + ", \"udp2(experimental feature)\"" +#endif /* ENABLE_IC_UDP2 */ #ifdef ENABLE_IC_PROXY " and \"proxy\"" #endif /* ENABLE_IC_PROXY */ diff --git a/src/include/cdb/cdbinterconnect.h b/src/include/cdb/cdbinterconnect.h index 9f5ce7753d8..5204d4c1b94 100644 --- a/src/include/cdb/cdbinterconnect.h +++ b/src/include/cdb/cdbinterconnect.h @@ -213,6 +213,10 @@ typedef struct ChunkTransportState /* ic_proxy backend context */ struct ICProxyBackendContext *proxyContext; + + /* used by IPC layer implement */ + void *implement_state; + } ChunkTransportState; #endif /* CDBINTERCONNECT_H */ diff --git a/src/include/cdb/cdbvars.h b/src/include/cdb/cdbvars.h index fb94a3960b5..abbddb3cd8b 100644 --- a/src/include/cdb/cdbvars.h +++ b/src/include/cdb/cdbvars.h @@ -363,6 +363,16 @@ extern int Gp_interconnect_queue_depth; * */ extern int Gp_interconnect_snd_queue_depth; + +/* + * Cursor IC table size. + * + * For cursor case, there may be several concurrent interconnect + * instances on QD. The table is used to track the status of the + * instances, which is quite useful for "ACK the past and NAK the future" paradigm. + * + */ +extern int Gp_interconnect_cursor_ic_table_size; extern int Gp_interconnect_timer_period; extern int Gp_interconnect_timer_checking_period; extern int Gp_interconnect_default_rtt; diff --git a/src/include/pg_config.h.in b/src/include/pg_config.h.in index a5393e85076..aaa3ea32e8a 100644 --- a/src/include/pg_config.h.in +++ b/src/include/pg_config.h.in @@ -60,6 +60,9 @@ /* Define to 1 to build with ic-proxy support (--enable-ic-proxy) */ #undef ENABLE_IC_PROXY +/* Define to 1 to build with interconnect udp2 support (--enable-ic-udp2) */ +#undef ENABLE_IC_UDP2 + /* Define to 1 to preload with ic module */ #undef ENABLE_PRELOAD_IC_MODULE diff --git a/src/include/utils/process_shared_preload_libraries.h b/src/include/utils/process_shared_preload_libraries.h index fe098ccf401..87e4dc40a6d 100644 --- a/src/include/utils/process_shared_preload_libraries.h +++ b/src/include/utils/process_shared_preload_libraries.h @@ -1,6 +1,9 @@ #ifdef ENABLE_PRELOAD_IC_MODULE "interconnect", #endif +#ifdef ENABLE_IC_UDP2 + "udp2", +#endif #ifdef USE_PAX_STORAGE "pax", #endif diff --git a/src/include/utils/sync_guc_name.h b/src/include/utils/sync_guc_name.h index ca9f7ef45f8..6d09f49155f 100644 --- a/src/include/utils/sync_guc_name.h +++ b/src/include/utils/sync_guc_name.h @@ -74,6 +74,7 @@ "gp_initial_bad_row_limit", "gp_interconnect_address_type", "gp_interconnect_cache_future_packets", + "gp_interconnect_cursor_ic_table_size", "gp_interconnect_debug_retry_interval", "gp_interconnect_default_rtt", "gp_interconnect_fc_method",