From 413b074e78c7521c4aab6c1ce37a0f22ce6f3ca3 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 20 Apr 2026 14:07:03 +0800 Subject: [PATCH 001/102] Pipe: Downgraded and reduced the log of PipePushMetaRPCHandler (#17492) (#17519) --- .../client/async/handlers/rpc/PipePushMetaRPCHandler.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipePushMetaRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipePushMetaRPCHandler.java index ffe15dd6466c5..0928c76cdc908 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipePushMetaRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipePushMetaRPCHandler.java @@ -67,15 +67,15 @@ public void onComplete(TPushPipeMetaResp response) { } @Override - public void onError(Exception e) { - String errorMsg = + public void onError(final Exception e) { + final String errorMsg = "Failed to " + requestType + " on DataNode: " + formattedTargetLocation + ", exception: " + e.getMessage(); - LOGGER.error(errorMsg, e); + LOGGER.warn(errorMsg); responseMap.put( requestId, From c1556669dc35089945ec320e5669a419aad08db7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 21 Apr 2026 11:02:23 +0800 Subject: [PATCH 002/102] Pipe: Fixed the OPC UA bug that drop pipe may not erase the client (#17510) (#17522) --- .../pipe/sink/protocol/opcua/OpcUaSink.java | 56 +++++++++---------- 1 file changed, 26 insertions(+), 30 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/OpcUaSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/OpcUaSink.java index e198506122474..03a8e6fba83d6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/OpcUaSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/OpcUaSink.java @@ -518,42 +518,38 @@ public interface ThrowingConsumer { @Override public void close() throws Exception { - if (serverKey == null) { - return; - } - - synchronized (SERVER_KEY_TO_REFERENCE_COUNT_AND_NAME_SPACE_MAP) { - final Pair pair = - SERVER_KEY_TO_REFERENCE_COUNT_AND_NAME_SPACE_MAP.get(serverKey); - if (pair == null) { - return; - } + if (serverKey != null) { + synchronized (SERVER_KEY_TO_REFERENCE_COUNT_AND_NAME_SPACE_MAP) { + final Pair pair = + SERVER_KEY_TO_REFERENCE_COUNT_AND_NAME_SPACE_MAP.get(serverKey); + if (pair == null) { + return; + } - if (pair.getLeft().decrementAndGet() <= 0) { - try { - pair.getRight().shutdown(); - } finally { - SERVER_KEY_TO_REFERENCE_COUNT_AND_NAME_SPACE_MAP.remove(serverKey); + if (pair.getLeft().decrementAndGet() <= 0) { + try { + pair.getRight().shutdown(); + } finally { + SERVER_KEY_TO_REFERENCE_COUNT_AND_NAME_SPACE_MAP.remove(serverKey); + } } } } - if (nodeUrl == null) { - return; - } - - synchronized (CLIENT_KEY_TO_REFERENCE_COUNT_AND_CLIENT_MAP) { - final Pair pair = - CLIENT_KEY_TO_REFERENCE_COUNT_AND_CLIENT_MAP.get(nodeUrl); - if (pair == null) { - return; - } + if (nodeUrl != null) { + synchronized (CLIENT_KEY_TO_REFERENCE_COUNT_AND_CLIENT_MAP) { + final Pair pair = + CLIENT_KEY_TO_REFERENCE_COUNT_AND_CLIENT_MAP.get(nodeUrl); + if (pair == null) { + return; + } - if (pair.getLeft().decrementAndGet() <= 0) { - try { - pair.getRight().disconnect(); - } finally { - CLIENT_KEY_TO_REFERENCE_COUNT_AND_CLIENT_MAP.remove(nodeUrl); + if (pair.getLeft().decrementAndGet() <= 0) { + try { + pair.getRight().disconnect(); + } finally { + CLIENT_KEY_TO_REFERENCE_COUNT_AND_CLIENT_MAP.remove(nodeUrl); + } } } } From 8d1ad63e955a324be605baf6411544e563da1b6d Mon Sep 17 00:00:00 2001 From: Hongzhi Gao <761417898@qq.com> Date: Tue, 21 Apr 2026 11:38:04 +0800 Subject: [PATCH 003/102] [To dev/1.3] enhance cppclient tsblock deserialize validation (#17464) (#17518) * fix tsblock deserialize * fix ut error on win * Revert "fix ut error on win" This reverts commit 34b8de482c864abd64d721bdd51fe08e18b742b8. --- .../client-cpp/src/main/ColumnDecoder.cpp | 3 + iotdb-client/client-cpp/src/main/Common.cpp | 46 ++++++++++++- iotdb-client/client-cpp/src/main/Common.h | 8 +++ iotdb-client/client-cpp/src/main/TsBlock.cpp | 12 ++++ .../client-cpp/src/test/cpp/sessionIT.cpp | 67 +++++++++++++++++++ 5 files changed, 134 insertions(+), 2 deletions(-) diff --git a/iotdb-client/client-cpp/src/main/ColumnDecoder.cpp b/iotdb-client/client-cpp/src/main/ColumnDecoder.cpp index e45cb49409a0d..32f29d876f368 100644 --- a/iotdb-client/client-cpp/src/main/ColumnDecoder.cpp +++ b/iotdb-client/client-cpp/src/main/ColumnDecoder.cpp @@ -151,6 +151,9 @@ std::unique_ptr BinaryArrayColumnDecoder::readColumn( if (!nullIndicators.empty() && nullIndicators[i]) continue; int32_t length = buffer.getInt(); + if (length < 0) { + throw IoTDBException("BinaryArrayColumnDecoder: negative TEXT length"); + } std::vector value(length); for (int32_t j = 0; j < length; j++) { diff --git a/iotdb-client/client-cpp/src/main/Common.cpp b/iotdb-client/client-cpp/src/main/Common.cpp index 38e8a31d2ff37..f58b6cc21ddc1 100644 --- a/iotdb-client/client-cpp/src/main/Common.cpp +++ b/iotdb-client/client-cpp/src/main/Common.cpp @@ -19,6 +19,7 @@ #include "Common.h" #include +#include int32_t parseDateExpressionToInt(const boost::gregorian::date& date) { if (date.is_not_a_date()) { @@ -292,6 +293,10 @@ double MyStringBuffer::getDouble() { } char MyStringBuffer::getChar() { + if (pos >= str.size()) { + throw IoTDBException("MyStringBuffer::getChar: read past end (pos=" + std::to_string(pos) + + ", size=" + std::to_string(str.size()) + ")"); + } return str[pos++]; } @@ -300,8 +305,16 @@ bool MyStringBuffer::getBool() { } std::string MyStringBuffer::getString() { - size_t len = getInt(); - size_t tmpPos = pos; + const int lenInt = getInt(); + if (lenInt < 0) { + throw IoTDBException("MyStringBuffer::getString: negative length"); + } + const size_t len = static_cast(lenInt); + if (pos > str.size() || len > str.size() - pos) { + throw IoTDBException("MyStringBuffer::getString: length exceeds buffer (pos=" + std::to_string(pos) + + ", len=" + std::to_string(len) + ", size=" + std::to_string(str.size()) + ")"); + } + const size_t tmpPos = pos; pos += len; return str.substr(tmpPos, len); } @@ -350,6 +363,10 @@ void MyStringBuffer::checkBigEndian() { } const char* MyStringBuffer::getOrderedByte(size_t len) { + if (pos > str.size() || len > str.size() - pos) { + throw IoTDBException("MyStringBuffer::getOrderedByte: read past end (pos=" + std::to_string(pos) + + ", len=" + std::to_string(len) + ", size=" + std::to_string(str.size()) + ")"); + } const char* p = nullptr; if (isBigEndian) { p = str.c_str() + pos; @@ -454,3 +471,28 @@ const std::vector& BitMap::getByteArray() const { size_t BitMap::getSize() const { return this->size; } + +TEndPoint UrlUtils::parseTEndPointIpv4AndIpv6Url(const std::string& endPointUrl) { + TEndPoint endPoint; + const size_t colonPos = endPointUrl.find_last_of(':'); + if (colonPos == std::string::npos) { + endPoint.__set_ip(endPointUrl); + endPoint.__set_port(0); + return endPoint; + } + std::string ip = endPointUrl.substr(0, colonPos); + const std::string portStr = endPointUrl.substr(colonPos + 1); + try { + const int port = std::stoi(portStr); + endPoint.__set_port(port); + } catch (const std::logic_error&) { + endPoint.__set_ip(endPointUrl); + endPoint.__set_port(0); + return endPoint; + } + if (ip.size() >= 2 && ip.front() == '[' && ip.back() == ']') { + ip = ip.substr(1, ip.size() - 2); + } + endPoint.__set_ip(ip); + return endPoint; +} diff --git a/iotdb-client/client-cpp/src/main/Common.h b/iotdb-client/client-cpp/src/main/Common.h index a9f4552ecc5fd..af9cf46e62e54 100644 --- a/iotdb-client/client-cpp/src/main/Common.h +++ b/iotdb-client/client-cpp/src/main/Common.h @@ -480,5 +480,13 @@ class RpcUtils { static std::shared_ptr getTSFetchResultsResp(const TSStatus& status); }; +class UrlUtils { +public: + UrlUtils() = delete; + + /** Parse host:port; aligns with Java UrlUtils.parseTEndPointIpv4AndIpv6Url plus test edge cases. */ + static TEndPoint parseTEndPointIpv4AndIpv6Url(const std::string& endPointUrl); +}; + #endif diff --git a/iotdb-client/client-cpp/src/main/TsBlock.cpp b/iotdb-client/client-cpp/src/main/TsBlock.cpp index 7c2bac272a601..92afbef3f270f 100644 --- a/iotdb-client/client-cpp/src/main/TsBlock.cpp +++ b/iotdb-client/client-cpp/src/main/TsBlock.cpp @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ +#include #include #include #include "TsBlock.h" @@ -34,6 +35,14 @@ std::shared_ptr TsBlock::deserialize(const std::string& data) { // Read value column count int32_t valueColumnCount = buffer.getInt(); + if (valueColumnCount < 0) { + throw IoTDBException("TsBlock::deserialize: negative valueColumnCount"); + } + const int64_t minHeaderBytes = + 9LL + 2LL * static_cast(valueColumnCount); + if (minHeaderBytes > static_cast(data.size())) { + throw IoTDBException("TsBlock::deserialize: truncated header"); + } // Read value column data types std::vector valueColumnDataTypes(valueColumnCount); @@ -43,6 +52,9 @@ std::shared_ptr TsBlock::deserialize(const std::string& data) { // Read position count int32_t positionCount = buffer.getInt(); + if (positionCount < 0) { + throw IoTDBException("TsBlock::deserialize: negative positionCount"); + } // Read column encodings std::vector columnEncodings(valueColumnCount + 1); diff --git a/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp b/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp index 09154b868e47c..b2db95cc4eca0 100644 --- a/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp +++ b/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp @@ -19,6 +19,8 @@ #include "catch.hpp" #include "Session.h" +#include "TsBlock.h" +#include using namespace std; @@ -728,3 +730,68 @@ TEST_CASE("Test executeLastDataQuery ", "[testExecuteLastDataQuery]") { sessionDataSet->setFetchSize(1024); REQUIRE(sessionDataSet->hasNext() == false); } + +// Helper function for comparing TEndPoint with detailed error message +void assertTEndPointEqual(const TEndPoint& actual, + const std::string& expectedIp, + int expectedPort, + const char* file, + int line) { + if (actual.ip != expectedIp || actual.port != expectedPort) { + std::stringstream ss; + ss << "\nTEndPoint mismatch:\nExpected: " << expectedIp << ":" << expectedPort + << "\nActual: " << actual.ip << ":" << actual.port; + Catch::SourceLineInfo location(file, line); + Catch::AssertionHandler handler("TEndPoint comparison", location, ss.str(), Catch::ResultDisposition::Normal); + handler.handleMessage(Catch::ResultWas::ExplicitFailure, ss.str()); + handler.complete(); + } +} + +// Macro to simplify test assertions +#define REQUIRE_TENDPOINT(actual, expectedIp, expectedPort) \ + assertTEndPointEqual(actual, expectedIp, expectedPort, __FILE__, __LINE__) + +TEST_CASE("UrlUtils - parseTEndPointIpv4AndIpv6Url", "[UrlUtils]") { + // Test valid IPv4 addresses + SECTION("Valid IPv4") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("192.168.1.1:8080"), "192.168.1.1", 8080); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("10.0.0.1:80"), "10.0.0.1", 80); + } + + // Test valid IPv6 addresses + SECTION("Valid IPv6") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("[2001:db8::1]:8080"), "2001:db8::1", 8080); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("[::1]:80"), "::1", 80); + } + + // Test hostnames + SECTION("Hostnames") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("localhost:8080"), "localhost", 8080); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("example.com:443"), "example.com", 443); + } + + // Test edge cases + SECTION("Edge cases") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url(""), "", 0); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("127.0.0.1"), "127.0.0.1", 0); + } + + // Test invalid inputs + SECTION("Invalid inputs") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("192.168.1.1:abc"), "192.168.1.1:abc", 0); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("]invalid[:80"), "]invalid[", 80); + } + + // Test port ranges + SECTION("Port ranges") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("localhost:0"), "localhost", 0); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("127.0.0.1:65535"), "127.0.0.1", 65535); + } +} + +TEST_CASE("TsBlock deserialize rejects truncated malicious payload", "[TsBlockDeserialize]") { + std::string data(18, '\0'); + data[3] = '\x10'; + REQUIRE_THROWS_AS(TsBlock::deserialize(data), IoTDBException); +} From 9cadc71a8190bd9b114ef660edc7423df9d97ad3 Mon Sep 17 00:00:00 2001 From: Yongzao Date: Tue, 21 Apr 2026 14:21:09 +0800 Subject: [PATCH 004/102] [To dev/1.3] Load balance patch (#17463) (#17493) (#17527) --- .../manager/load/service/EventService.java | 5 ++++ .../plan/analyze/ClusterPartitionFetcher.java | 30 ++++++++++++++----- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/service/EventService.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/service/EventService.java index 5f6035f0bcab9..8267227c7ce41 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/service/EventService.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/service/EventService.java @@ -114,6 +114,11 @@ public void stopEventService() { currentEventServiceFuture = null; LOGGER.info("Event service is stopped successfully."); } + synchronized (this) { + previousNodeStatisticsMap.clear(); + previousRegionGroupStatisticsMap.clear(); + previousConsensusGroupStatisticsMap.clear(); + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java index ff19dd79cf58e..eb65666035a99 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java @@ -59,7 +59,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -421,16 +420,31 @@ private SchemaPartition parseSchemaPartitionTableResp( String database = entry1.getKey(); final Map result1 = regionReplicaMap.computeIfAbsent(database, k -> new HashMap<>()); + Map seriesPartitionTable = entry1.getValue(); + + if (seriesPartitionTable.size() == 1) { + // Fast collection in case of query for single device + Map.Entry seriesPartitionEntry = + seriesPartitionTable.entrySet().iterator().next(); + List regionReplicaSets = + partitionCache.getRegionReplicaSet( + Collections.singletonList(seriesPartitionEntry.getValue())); + result1.put(seriesPartitionEntry.getKey(), regionReplicaSets.get(0)); + continue; + } - Map orderedMap = - new LinkedHashMap<>(entry1.getValue()); - List orderedGroupIds = new ArrayList<>(orderedMap.values()); + List distinctRegionGroupIds = + new ArrayList<>(new HashSet<>(seriesPartitionTable.values())); List regionReplicaSets = - partitionCache.getRegionReplicaSet(orderedGroupIds); + partitionCache.getRegionReplicaSet(distinctRegionGroupIds); + Map groupIdToReplicaSet = new HashMap<>(); + for (int index = 0; index < distinctRegionGroupIds.size(); index++) { + groupIdToReplicaSet.put(distinctRegionGroupIds.get(index), regionReplicaSets.get(index)); + } - int index = 0; - for (Map.Entry entry2 : orderedMap.entrySet()) { - result1.put(entry2.getKey(), regionReplicaSets.get(index++)); + for (Map.Entry entry2 : + seriesPartitionTable.entrySet()) { + result1.put(entry2.getKey(), groupIdToReplicaSet.get(entry2.getValue())); } } From ecef8f7cf83483b56415ee63663fbbd0044a1af6 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 21 Apr 2026 16:42:59 +0800 Subject: [PATCH 005/102] [To dev/1.3] Pipe: Rewrote the OPC UA subscription logic to avoid the bug of third-party subscription model (#17524) * complete * Debounce --- .../pipe/sink/protocol/opcua/OpcUaSink.java | 13 +- .../protocol/opcua/server/OpcUaNameSpace.java | 158 ++++++++++++++++-- .../opcua/server/OpcUaServerBuilder.java | 14 +- .../config/constant/PipeSinkConstant.java | 5 + 4 files changed, 173 insertions(+), 17 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/OpcUaSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/OpcUaSink.java index 03a8e6fba83d6..654395efe8d7d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/OpcUaSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/OpcUaSink.java @@ -63,6 +63,8 @@ import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_IOTDB_USERNAME_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_IOTDB_USER_DEFAULT_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_IOTDB_USER_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_OPC_UA_DEBOUNCE_TIME_MS_DEFAULT_VALUE; +import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_OPC_UA_DEBOUNCE_TIME_MS_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_OPC_UA_DEFAULT_QUALITY_BAD_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_OPC_UA_DEFAULT_QUALITY_GOOD_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_OPC_UA_DEFAULT_QUALITY_KEY; @@ -101,6 +103,7 @@ import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.SINK_IOTDB_PASSWORD_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.SINK_IOTDB_USERNAME_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.SINK_IOTDB_USER_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.SINK_OPC_UA_DEBOUNCE_TIME_MS_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.SINK_OPC_UA_DEFAULT_QUALITY_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.SINK_OPC_UA_ENABLE_ANONYMOUS_ACCESS_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.SINK_OPC_UA_HISTORIZING_KEY; @@ -278,6 +281,10 @@ private void customizeServer(final PipeParameters parameters) { if (securityPolicies.isEmpty()) { throw new PipeException("The security policy cannot be empty."); } + final long debounceTimeMs = + parameters.getLongOrDefault( + Arrays.asList(CONNECTOR_OPC_UA_DEBOUNCE_TIME_MS_KEY, SINK_OPC_UA_DEBOUNCE_TIME_MS_KEY), + CONNECTOR_OPC_UA_DEBOUNCE_TIME_MS_DEFAULT_VALUE); synchronized (SERVER_KEY_TO_REFERENCE_COUNT_AND_NAME_SPACE_MAP) { serverKey = httpsBindPort + ":" + tcpBindPort; @@ -297,7 +304,8 @@ private void customizeServer(final PipeParameters parameters) { .setPassword(password) .setSecurityDir(securityDir) .setEnableAnonymousAccess(enableAnonymousAccess) - .setSecurityPolicies(securityPolicies); + .setSecurityPolicies(securityPolicies) + .setDebounceTimeMs(debounceTimeMs); final OpcUaServer newServer = builder.build(); nameSpace = new OpcUaNameSpace(newServer, builder); nameSpace.startup(); @@ -311,7 +319,8 @@ private void customizeServer(final PipeParameters parameters) { password, securityDir, enableAnonymousAccess, - securityPolicies); + securityPolicies, + debounceTimeMs); return oldValue; } } catch (final PipeException e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaNameSpace.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaNameSpace.java index 43144cbdd15b5..3a42ec8796916 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaNameSpace.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaNameSpace.java @@ -45,7 +45,7 @@ import org.eclipse.milo.opcua.sdk.server.nodes.UaFolderNode; import org.eclipse.milo.opcua.sdk.server.nodes.UaNode; import org.eclipse.milo.opcua.sdk.server.nodes.UaVariableNode; -import org.eclipse.milo.opcua.sdk.server.util.SubscriptionModel; +import org.eclipse.milo.opcua.stack.core.AttributeId; import org.eclipse.milo.opcua.stack.core.Identifiers; import org.eclipse.milo.opcua.stack.core.UaException; import org.eclipse.milo.opcua.stack.core.security.SecurityPolicy; @@ -55,6 +55,7 @@ import org.eclipse.milo.opcua.stack.core.types.builtin.NodeId; import org.eclipse.milo.opcua.stack.core.types.builtin.StatusCode; import org.eclipse.milo.opcua.stack.core.types.builtin.Variant; +import org.eclipse.milo.opcua.stack.core.types.structured.ReadValueId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,19 +69,33 @@ import java.util.Objects; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; public class OpcUaNameSpace extends ManagedNamespaceWithLifecycle { private static final Logger LOGGER = LoggerFactory.getLogger(OpcUaNameSpace.class); public static final String NAMESPACE_URI = "urn:apache:iotdb:opc-server"; - private final SubscriptionModel subscriptionModel; private final OpcUaServerBuilder builder; + // Do not use subscription model because the original subscription model has some bugs + private final ConcurrentMap> nodeSubscriptions = new ConcurrentHashMap<>(); + + // Debounce task cache: used to merge updates within a short period of time, avoiding unnecessary + // duplicate pushes + private final ConcurrentMap> debounceTasks = new ConcurrentHashMap<>(); + // Debounce interval: within 10ms, the same node is updated multiple times, and only the last one + // will be pushed (can be adjusted according to your site delay requirements, the minimum can be + // set to 1ms) + private final long debounceIntervalMs; + public OpcUaNameSpace(final OpcUaServer server, final OpcUaServerBuilder builder) { super(server, NAMESPACE_URI); this.builder = builder; + debounceIntervalMs = builder.getDebounceTimeMs(); - subscriptionModel = new SubscriptionModel(server, this); - getLifecycleManager().addLifecycle(subscriptionModel); getLifecycleManager() .addLifecycle( new Lifecycle() { @@ -245,7 +260,7 @@ private void transferTabletRowForClientServerModel( measurementNode = new UaVariableNode.UaVariableNodeBuilder(getNodeContext()) .setNodeId(nodeId) - .setAccessLevel(AccessLevel.READ_WRITE) + .setAccessLevel(AccessLevel.READ_ONLY) .setUserAccessLevel(AccessLevel.READ_ONLY) .setBrowseName(newQualifiedName(nodeName)) .setDisplayName(LocalizedText.english(nodeName)) @@ -279,7 +294,7 @@ private void transferTabletRowForClientServerModel( if (Objects.isNull(measurementNode.getValue()) || Objects.isNull(measurementNode.getValue().getSourceTime()) || measurementNode.getValue().getSourceTime().getUtcTime() < utcTimestamp) { - measurementNode.setValue(dataValue); + notifyNodeValueChange(nodeId, dataValue, measurementNode); } } else { valueNode = measurementNode; @@ -291,9 +306,11 @@ private void transferTabletRowForClientServerModel( if (Objects.isNull(valueNode.getValue()) || Objects.isNull(valueNode.getValue().getSourceTime()) || valueNode.getValue().getSourceTime().getUtcTime() < timestamp) { - valueNode.setValue( + notifyNodeValueChange( + valueNode.getNodeId(), new DataValue( - new Variant(value), currentQuality, new DateTime(timestamp), new DateTime())); + new Variant(value), currentQuality, new DateTime(timestamp), new DateTime()), + valueNode); } } } @@ -451,24 +468,131 @@ public static NodeId convertToOpcDataType(final TSDataType type) { } } + /** + * On point value changing, notify all subscribed clients proactively + * + * @param nodeId NodeId of the changing node + * @param newValue New value of the node (DataValue object containing value, status code, and + * timestamp) + * @param variableNode Corresponding UaVariableNode instance, used to update the local cached + * value of the node + */ + public void notifyNodeValueChange( + NodeId nodeId, DataValue newValue, UaVariableNode variableNode) { + // 1. Update the local cached value of the node + variableNode.setValue(newValue); + + // 2. If there are no subscribers, return directly without doing any extra operations + List subscribedItems = nodeSubscriptions.get(nodeId); + if (subscribedItems == null || subscribedItems.isEmpty()) { + return; + } + + // 2. Debounce+Async Push: Asynchronously push the expensive push operation, while merging + // high-frequency repeated updates + debounceTasks.compute( + nodeId, + (k, oldTask) -> { + // If there is already a pending push task, cancel it, we only need the latest value + if (oldTask != null && !oldTask.isDone()) { + oldTask.cancel(false); + } + + // Submit the push task to the Milo's scheduled thread pool, delay DEBOUNCE_INTERVAL_MS + // execution + return getServer() + .getScheduledExecutorService() + .schedule( + () -> { + try { + // Batch push changes to all subscribers, this time-consuming operation is put + // into the thread pool, not blocking your data update thread + for (DataItem item : subscribedItems) { + try { + item.setValue(newValue); + } catch (Exception e) { + // Single client push failure does not affect other clients + LOGGER.warn( + "Failed to push value change to client, nodeId={}", nodeId, e); + } + } + } finally { + // Task execution completed, clean up the debounce cache + debounceTasks.remove(nodeId); + } + }, + debounceIntervalMs, + TimeUnit.MILLISECONDS); + }); + } + @Override public void onDataItemsCreated(final List dataItems) { - subscriptionModel.onDataItemsCreated(dataItems); + for (DataItem item : dataItems) { + final ReadValueId readValueId = item.getReadValueId(); + // Only handle Value attribute subscription (align with the original SubscriptionModel logic, + // ignore other attribute subscriptions) + if (!AttributeId.Value.isEqual(readValueId.getAttributeId())) { + continue; + } + final NodeId nodeId = readValueId.getNodeId(); + + // 1. Add the new subscription item to the subscription mapping + nodeSubscriptions.compute( + nodeId, + (k, existingList) -> { + List list = + existingList != null ? existingList : new CopyOnWriteArrayList<>(); + list.add(item); + return list; + }); + + // 2. 【Key Optimization】Proactively push the current node's initial value when the new + // subscription item is created + // Eliminate Bad_WaitingForInitialData, no need to wait for any polling + try { + UaVariableNode node = (UaVariableNode) getNodeManager().getNode(nodeId).orElse(null); + if (node != null && node.getValue() != null) { + // Immediately push the current value to the new subscriber, the client will instantly be + // able to get the initial data + item.setValue(node.getValue()); + } + } catch (Exception e) { + LOGGER.warn("Failed to send initial value to new subscription, nodeId={}", nodeId, e); + } + } } @Override public void onDataItemsModified(final List dataItems) { - subscriptionModel.onDataItemsModified(dataItems); + // Push mode, client modifies subscription parameters (e.g. sampling interval) has no effect on + // our active push, no additional processing is needed } @Override public void onDataItemsDeleted(final List dataItems) { - subscriptionModel.onDataItemsDeleted(dataItems); + for (DataItem item : dataItems) { + final ReadValueId readValueId = item.getReadValueId(); + if (!AttributeId.Value.isEqual(readValueId.getAttributeId())) { + continue; + } + final NodeId nodeId = readValueId.getNodeId(); + + // When the client cancels the subscription, remove this subscription item from the mapping + nodeSubscriptions.computeIfPresent( + nodeId, + (k, existingList) -> { + existingList.remove(item); + // Automatically clean up the key when there are no subscribers, save memory + return existingList.isEmpty() ? null : existingList; + }); + } } @Override public void onMonitoringModeChanged(final List monitoredItems) { - subscriptionModel.onMonitoringModeChanged(monitoredItems); + // Push mode, monitoring mode change has no effect on active push, no additional processing is + // needed } /////////////////////////////// Conflict detection /////////////////////////////// @@ -478,8 +602,14 @@ public void checkEquals( final String password, final String securityDir, final boolean enableAnonymousAccess, - final Set securityPolicies) { + final Set securityPolicies, + final long debounceTimeMs) { builder.checkEquals( - user, password, Paths.get(securityDir), enableAnonymousAccess, securityPolicies); + user, + password, + Paths.get(securityDir), + enableAnonymousAccess, + securityPolicies, + debounceTimeMs); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaServerBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaServerBuilder.java index f029031b6175b..281d6eae77edd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaServerBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaServerBuilder.java @@ -86,6 +86,7 @@ public class OpcUaServerBuilder implements Closeable { private boolean enableAnonymousAccess; private Set securityPolicies; private DefaultTrustListManager trustListManager; + private long debounceTimeMs; public OpcUaServerBuilder setTcpBindPort(final int tcpBindPort) { this.tcpBindPort = tcpBindPort; @@ -123,6 +124,15 @@ public OpcUaServerBuilder setSecurityPolicies(final Set security return this; } + public OpcUaServerBuilder setDebounceTimeMs(long debounceTimeMs) { + this.debounceTimeMs = debounceTimeMs; + return this; + } + + public long getDebounceTimeMs() { + return debounceTimeMs; + } + public OpcUaServer build() throws Exception { Files.createDirectories(securityDir); if (!Files.exists(securityDir)) { @@ -314,7 +324,8 @@ void checkEquals( final String password, final Path securityDir, final boolean enableAnonymousAccess, - final Set securityPolicies) { + final Set securityPolicies, + final long debounceTimeMs) { checkEquals("user", this.user, user); checkEquals("password", this.password, password); checkEquals( @@ -323,6 +334,7 @@ void checkEquals( FileSystems.getDefault().getPath(securityDir.toAbsolutePath().toString())); checkEquals("enableAnonymousAccess option", this.enableAnonymousAccess, enableAnonymousAccess); checkEquals("securityPolicies", this.securityPolicies, securityPolicies); + checkEquals("debounceTimeMs", this.debounceTimeMs, debounceTimeMs); } private void checkEquals(final String attrName, Object thisAttr, Object thatAttr) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSinkConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSinkConstant.java index 2eaf6f903de48..e04f2578daac9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSinkConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSinkConstant.java @@ -231,6 +231,11 @@ public class PipeSinkConstant { "connector.opcua.timeout-seconds"; public static final long CONNECTOR_OPC_UA_TIMEOUT_SECONDS_DEFAULT_VALUE = 10L; + public static final String CONNECTOR_OPC_UA_DEBOUNCE_TIME_MS_KEY = + "connector.opcua.debounce-time-ms"; + public static final String SINK_OPC_UA_DEBOUNCE_TIME_MS_KEY = "sink.opcua.debounce-time-ms"; + public static final long CONNECTOR_OPC_UA_DEBOUNCE_TIME_MS_DEFAULT_VALUE = 50L; + public static final String CONNECTOR_LEADER_CACHE_ENABLE_KEY = "connector.leader-cache.enable"; public static final String SINK_LEADER_CACHE_ENABLE_KEY = "sink.leader-cache.enable"; public static final boolean CONNECTOR_LEADER_CACHE_ENABLE_DEFAULT_VALUE = true; From 4588b8d27a414fde21f6c58d3e4534866c06bb96 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 21 Apr 2026 16:46:56 +0800 Subject: [PATCH 006/102] [To dev/1.3] Pipe: Upgraded bouncy castle && Added logs for pipe tsFile parser (#17504) (#17523) * [To dev/1.3] Pipe: Upgraded bouncy castle && Added logs for pipe tsFile parser (#17504) * fixed --- .../container/TsFileInsertionDataContainer.java | 16 +++++++++++++++- .../query/TsFileInsertionQueryDataContainer.java | 11 ++++++++++- .../scan/TsFileInsertionScanDataContainer.java | 11 ++++++++++- pom.xml | 2 +- 4 files changed, 36 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java index 279996690ff92..f335cff10354a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java @@ -39,6 +39,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.io.IOException; public abstract class TsFileInsertionDataContainer implements AutoCloseable { @@ -69,13 +70,15 @@ public abstract class TsFileInsertionDataContainer implements AutoCloseable { protected Iterable tabletInsertionIterable; protected TsFileInsertionDataContainer( + final File tsFile, final String pipeName, final long creationTime, final PipePattern pattern, final long startTime, final long endTime, final PipeTaskMeta pipeTaskMeta, - final EnrichedEvent sourceEvent) { + final EnrichedEvent sourceEvent, + final boolean isWithMod) { this.pipeName = pipeName; this.creationTime = creationTime; @@ -93,6 +96,17 @@ protected TsFileInsertionDataContainer( PipeDataNodeResourceManager.memory() .forceAllocateForTabletWithRetry( IoTDBDescriptor.getInstance().getConfig().getPipeDataStructureTabletSizeInBytes()); + + LOGGER.info( + "TsFile {} has initialized {}, pipeName: {}, creation time: {}, pattern: {}, startTime: {}, endTime: {}, withMod: {}", + tsFile, + getClass().getSimpleName(), + pipeName, + creationTime, + pattern, + startTime, + endTime, + isWithMod); } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java index 15732fd0e8db2..4353e4984a259 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java @@ -114,7 +114,16 @@ public TsFileInsertionQueryDataContainer( final Map deviceIsAlignedMap, final boolean isWithMod) throws IOException { - super(pipeName, creationTime, pattern, startTime, endTime, pipeTaskMeta, sourceEvent); + super( + tsFile, + pipeName, + creationTime, + pattern, + startTime, + endTime, + pipeTaskMeta, + sourceEvent, + isWithMod); try { currentModifications = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java index 271bc317d0f96..8ed9bdcd6625a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java @@ -116,7 +116,16 @@ public TsFileInsertionScanDataContainer( final EnrichedEvent sourceEvent, final boolean isWithMod) throws IOException { - super(pipeName, creationTime, pattern, startTime, endTime, pipeTaskMeta, sourceEvent); + super( + tsFile, + pipeName, + creationTime, + pattern, + startTime, + endTime, + pipeTaskMeta, + sourceEvent, + isWithMod); this.startTime = startTime; this.endTime = endTime; diff --git a/pom.xml b/pom.xml index 35c7eeb481e15..294a1231e355f 100644 --- a/pom.xml +++ b/pom.xml @@ -60,7 +60,7 @@ 4.2.0 - 1.81 + 1.84 2.9.3 3.3.0 From 63ea91a83f870d8fc9fae4386273096c1f1b98eb Mon Sep 17 00:00:00 2001 From: Weihao Li <60659567+Wei-hao-Li@users.noreply.github.com> Date: Wed, 22 Apr 2026 09:25:36 +0800 Subject: [PATCH 007/102] [To dev/1.3] Enable show queries to be executed immediately when the available memory in the memoryPool is insufficient (#17529) --- .../queryengine/common/MPPQueryContext.java | 14 +++ .../exchange/MPPDataExchangeManager.java | 56 ++++++++- .../exchange/SharedTsBlockQueue.java | 57 ++++++--- .../execution/exchange/sink/SinkChannel.java | 62 ++++++++-- .../exchange/source/SourceHandle.java | 90 ++++++++++---- .../fragment/FragmentInstanceContext.java | 13 ++ .../fragment/FragmentInstanceManager.java | 2 + .../execution/memory/MemoryPool.java | 56 +++++++-- .../db/queryengine/plan/analyze/Analysis.java | 2 +- .../plan/execution/QueryExecution.java | 7 +- .../plan/planner/OperatorTreeGenerator.java | 6 +- .../plan/planner/plan/FragmentInstance.java | 2 + .../exchange/LocalSinkChannelTest.java | 10 +- .../exchange/SharedTsBlockQueueTest.java | 11 +- .../exchange/ShuffleSinkHandleTest.java | 5 +- .../execution/exchange/SourceHandleTest.java | 5 +- .../queryengine/execution/exchange/Utils.java | 26 ++-- .../execution/memory/MemoryPoolTest.java | 116 +++++++++++++++--- 18 files changed, 427 insertions(+), 113 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index 7479e832a9016..13773c150906c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -100,6 +100,12 @@ public class MPPQueryContext { private boolean userQuery = false; + /** + * When true (e.g. SHOW QUERIES), operator and exchange memory may use fallback when pool is + * insufficient. Set from analysis via {@link #setNeedSetHighestPriority(boolean)}. + */ + private boolean needSetHighestPriority = false; + @TestOnly public MPPQueryContext(QueryId queryId) { this.queryId = queryId; @@ -406,6 +412,14 @@ public void setUserQuery(boolean userQuery) { this.userQuery = userQuery; } + public boolean needSetHighestPriority() { + return needSetHighestPriority; + } + + public void setNeedSetHighestPriority(boolean needSetHighestPriority) { + this.needSetHighestPriority = needSetHighestPriority; + } + public String getClientHostName() { if (session == null || session.getCliHostname() == null) { return "UNKNOWN"; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/MPPDataExchangeManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/MPPDataExchangeManager.java index ac0b411109089..61dc05d4bfd34 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/MPPDataExchangeManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/MPPDataExchangeManager.java @@ -23,6 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.sync.SyncDataNodeMPPDataExchangeServiceClient; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.queryengine.exception.exchange.GetTsBlockFromClosedOrAbortedChannelException; import org.apache.iotdb.db.queryengine.execution.driver.DriverContext; import org.apache.iotdb.db.queryengine.execution.exchange.sink.DownStreamChannelIndex; @@ -656,7 +657,11 @@ private synchronized ISinkChannel createLocalSinkChannel( } queue = new SharedTsBlockQueue( - localFragmentInstanceId, localPlanNodeId, localMemoryManager, executorService); + localFragmentInstanceId, + localPlanNodeId, + localMemoryManager, + executorService, + instanceContext.isHighestPriority()); } return new LocalSinkChannel( @@ -680,7 +685,8 @@ public ISinkChannel createLocalSinkChannelForPipeline( driverContext.getDriverTaskID().getFragmentInstanceId().toThrift(), planNodeId, localMemoryManager, - executorService); + executorService, + driverContext.getFragmentInstanceContext().isHighestPriority()); queue.allowAddingTsBlock(); return new LocalSinkChannel( queue, @@ -718,6 +724,7 @@ private ISinkChannel createSinkChannel( tsBlockSerdeFactory.get(), new ISinkChannelListenerImpl( localFragmentInstanceId, instanceContext, instanceContext::failed, cnt), + instanceContext.isHighestPriority(), mppDataExchangeServiceClientManager); } @@ -802,6 +809,7 @@ public ISourceHandle createLocalSourceHandleForPipeline( context.getDriverTaskID().toString()); } + @TestOnly public synchronized ISourceHandle createLocalSourceHandleForFragment( TFragmentInstanceId localFragmentInstanceId, String localPlanNodeId, @@ -809,6 +817,24 @@ public synchronized ISourceHandle createLocalSourceHandleForFragment( TFragmentInstanceId remoteFragmentInstanceId, int index, IMPPDataExchangeManagerCallback onFailureCallback) { + return createLocalSourceHandleForFragment( + localFragmentInstanceId, + localPlanNodeId, + remotePlanNodeId, + remoteFragmentInstanceId, + index, + onFailureCallback, + false); + } + + public synchronized ISourceHandle createLocalSourceHandleForFragment( + TFragmentInstanceId localFragmentInstanceId, + String localPlanNodeId, + String remotePlanNodeId, + TFragmentInstanceId remoteFragmentInstanceId, + int index, + IMPPDataExchangeManagerCallback onFailureCallback, + boolean isHighestPriority) { if (sourceHandles.containsKey(localFragmentInstanceId) && sourceHandles.get(localFragmentInstanceId).containsKey(localPlanNodeId)) { throw new IllegalStateException( @@ -840,7 +866,11 @@ public synchronized ISourceHandle createLocalSourceHandleForFragment( } queue = new SharedTsBlockQueue( - remoteFragmentInstanceId, remotePlanNodeId, localMemoryManager, executorService); + remoteFragmentInstanceId, + remotePlanNodeId, + localMemoryManager, + executorService, + isHighestPriority); } LocalSourceHandle localSourceHandle = new LocalSourceHandle( @@ -854,6 +884,7 @@ public synchronized ISourceHandle createLocalSourceHandleForFragment( return localSourceHandle; } + @TestOnly @Override public ISourceHandle createSourceHandle( TFragmentInstanceId localFragmentInstanceId, @@ -862,6 +893,24 @@ public ISourceHandle createSourceHandle( TEndPoint remoteEndpoint, TFragmentInstanceId remoteFragmentInstanceId, IMPPDataExchangeManagerCallback onFailureCallback) { + return createSourceHandle( + localFragmentInstanceId, + localPlanNodeId, + indexOfUpstreamSinkHandle, + remoteEndpoint, + remoteFragmentInstanceId, + onFailureCallback, + false); + } + + public ISourceHandle createSourceHandle( + TFragmentInstanceId localFragmentInstanceId, + String localPlanNodeId, + int indexOfUpstreamSinkHandle, + TEndPoint remoteEndpoint, + TFragmentInstanceId remoteFragmentInstanceId, + IMPPDataExchangeManagerCallback onFailureCallback, + boolean isHighestPriority) { Map sourceHandleMap = sourceHandles.get(localFragmentInstanceId); if (sourceHandleMap != null && sourceHandleMap.containsKey(localPlanNodeId)) { throw new IllegalStateException( @@ -891,6 +940,7 @@ public ISourceHandle createSourceHandle( executorService, tsBlockSerdeFactory.get(), new SourceHandleListenerImpl(onFailureCallback), + isHighestPriority, mppDataExchangeServiceClientManager); sourceHandles .computeIfAbsent(localFragmentInstanceId, key -> new ConcurrentHashMap<>()) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/SharedTsBlockQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/SharedTsBlockQueue.java index 555cf9efe5ad9..f4c21e2fdfa91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/SharedTsBlockQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/SharedTsBlockQueue.java @@ -19,11 +19,13 @@ package org.apache.iotdb.db.queryengine.execution.exchange; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.common.FragmentInstanceId; import org.apache.iotdb.db.queryengine.execution.exchange.sink.LocalSinkChannel; import org.apache.iotdb.db.queryengine.execution.exchange.source.LocalSourceHandle; import org.apache.iotdb.db.queryengine.execution.memory.LocalMemoryManager; +import org.apache.iotdb.db.queryengine.execution.memory.MemoryPool.MemoryReservationResult; import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId; import com.google.common.util.concurrent.ListenableFuture; @@ -62,7 +64,7 @@ public class SharedTsBlockQueue { private long bufferRetainedSizeInBytes = 0L; - private final Queue queue = new LinkedList<>(); + private final Queue> queue = new LinkedList<>(); private SettableFuture blocked = SettableFuture.create(); @@ -82,17 +84,28 @@ public class SharedTsBlockQueue { private long maxBytesCanReserve = IoTDBDescriptor.getInstance().getConfig().getMaxBytesPerFragmentInstance(); + private final boolean isHighestPriority; private volatile Throwable abortedCause = null; // used for SharedTsBlockQueue listener private final ExecutorService executorService; + @TestOnly public SharedTsBlockQueue( TFragmentInstanceId fragmentInstanceId, String planNodeId, LocalMemoryManager localMemoryManager, ExecutorService executorService) { + this(fragmentInstanceId, planNodeId, localMemoryManager, executorService, false); + } + + public SharedTsBlockQueue( + TFragmentInstanceId fragmentInstanceId, + String planNodeId, + LocalMemoryManager localMemoryManager, + ExecutorService executorService, + boolean isHighestPriority) { this.localFragmentInstanceId = Validate.notNull(fragmentInstanceId, "fragment instance ID cannot be null"); this.fullFragmentInstanceId = @@ -101,6 +114,7 @@ public SharedTsBlockQueue( this.localMemoryManager = Validate.notNull(localMemoryManager, "local memory manager cannot be null"); this.executorService = Validate.notNull(executorService, "ExecutorService can not be null."); + this.isHighestPriority = isHighestPriority; } public boolean hasNoMoreTsBlocks() { @@ -195,15 +209,18 @@ public TsBlock remove() { } throw new IllegalStateException("queue has been destroyed"); } - TsBlock tsBlock = queue.remove(); - localMemoryManager - .getQueryPool() - .free( - localFragmentInstanceId.getQueryId(), - fullFragmentInstanceId, - localPlanNodeId, - tsBlock.getSizeInBytes()); - bufferRetainedSizeInBytes -= tsBlock.getSizeInBytes(); + Pair tsBlockWithReservedBytes = queue.remove(); + long reservedBytes = tsBlockWithReservedBytes.right; + if (reservedBytes > 0) { + localMemoryManager + .getQueryPool() + .free( + localFragmentInstanceId.getQueryId(), + fullFragmentInstanceId, + localPlanNodeId, + reservedBytes); + bufferRetainedSizeInBytes -= reservedBytes; + } // Every time LocalSourceHandle consumes a TsBlock, it needs to send the event // to // corresponding LocalSinkChannel. @@ -213,7 +230,7 @@ public TsBlock remove() { if (blocked.isDone() && queue.isEmpty() && !noMoreTsBlocks) { blocked = SettableFuture.create(); } - return tsBlock; + return tsBlockWithReservedBytes.left; } /** @@ -236,20 +253,22 @@ public ListenableFuture add(TsBlock tsBlock) { localFragmentInstanceId.queryId, fullFragmentInstanceId, localPlanNodeId); alreadyRegistered = true; } - Pair, Boolean> pair = + MemoryReservationResult reserveResult = localMemoryManager .getQueryPool() - .reserve( + .reserveWithPriority( localFragmentInstanceId.getQueryId(), fullFragmentInstanceId, localPlanNodeId, tsBlock.getSizeInBytes(), - maxBytesCanReserve); - blockedOnMemory = pair.left; - bufferRetainedSizeInBytes += tsBlock.getSizeInBytes(); + maxBytesCanReserve, + isHighestPriority); + blockedOnMemory = reserveResult.getFuture(); + long reservedBytes = reserveResult.getReservedBytes(); + bufferRetainedSizeInBytes += reservedBytes; // reserve memory failed, we should wait until there is enough memory - if (!Boolean.TRUE.equals(pair.right)) { + if (!reserveResult.isReserveSuccess()) { SettableFuture channelBlocked = SettableFuture.create(); blockedOnMemory.addListener( () -> { @@ -264,7 +283,7 @@ public ListenableFuture add(TsBlock tsBlock) { channelBlocked.set(null); return; } - queue.add(tsBlock); + queue.add(new Pair<>(tsBlock, reservedBytes)); if (!blocked.isDone()) { blocked.set(null); } @@ -281,7 +300,7 @@ public ListenableFuture add(TsBlock tsBlock) { executorService); return channelBlocked; } else { // reserve memory succeeded, add the TsBlock directly - queue.add(tsBlock); + queue.add(new Pair<>(tsBlock, reservedBytes)); if (!blocked.isDone()) { blocked.set(null); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/sink/SinkChannel.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/sink/SinkChannel.java index ca6fdadc993e6..47ce6128fb76a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/sink/SinkChannel.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/sink/SinkChannel.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.queryengine.exception.exchange.GetTsBlockFromClosedOrAbortedChannelException; import org.apache.iotdb.db.queryengine.execution.exchange.MPPDataExchangeManager.SinkListener; import org.apache.iotdb.db.queryengine.execution.memory.LocalMemoryManager; +import org.apache.iotdb.db.queryengine.execution.memory.MemoryPool.MemoryReservationResult; import org.apache.iotdb.db.queryengine.metric.DataExchangeCostMetricSet; import org.apache.iotdb.db.queryengine.metric.DataExchangeCountMetricSet; import org.apache.iotdb.db.utils.SetThreadName; @@ -119,6 +120,8 @@ public class SinkChannel implements ISinkChannel { private long maxBytesCanReserve = IoTDBDescriptor.getInstance().getConfig().getMaxBytesPerFragmentInstance(); + private final boolean isHighestPriority; + private static final DataExchangeCostMetricSet DATA_EXCHANGE_COST_METRIC_SET = DataExchangeCostMetricSet.getInstance(); private static final DataExchangeCountMetricSet DATA_EXCHANGE_COUNT_METRIC_SET = @@ -128,6 +131,34 @@ public class SinkChannel implements ISinkChannel { RamUsageEstimator.shallowSizeOfInstance(SinkChannel.class) + RamUsageEstimator.shallowSizeOfInstance(TFragmentInstanceId.class) * 2; + @TestOnly + @SuppressWarnings("squid:S107") + public SinkChannel( + TEndPoint remoteEndpoint, + TFragmentInstanceId remoteFragmentInstanceId, + String remotePlanNodeId, + String localPlanNodeId, + TFragmentInstanceId localFragmentInstanceId, + LocalMemoryManager localMemoryManager, + ExecutorService executorService, + TsBlockSerde serde, + SinkListener sinkListener, + IClientManager + mppDataExchangeServiceClientManager) { + this( + remoteEndpoint, + remoteFragmentInstanceId, + remotePlanNodeId, + localPlanNodeId, + localFragmentInstanceId, + localMemoryManager, + executorService, + serde, + sinkListener, + false, + mppDataExchangeServiceClientManager); + } + @SuppressWarnings("squid:S107") public SinkChannel( TEndPoint remoteEndpoint, @@ -139,6 +170,7 @@ public SinkChannel( ExecutorService executorService, TsBlockSerde serde, SinkListener sinkListener, + boolean isHighestPriority, IClientManager mppDataExchangeServiceClientManager) { this.remoteEndpoint = Validate.notNull(remoteEndpoint, "remoteEndPoint can not be null."); @@ -155,6 +187,7 @@ public SinkChannel( this.executorService = Validate.notNull(executorService, "executorService can not be null."); this.serde = Validate.notNull(serde, "serde can not be null."); this.sinkListener = Validate.notNull(sinkListener, "sinkListener can not be null."); + this.isHighestPriority = isHighestPriority; this.mppDataExchangeServiceClientManager = mppDataExchangeServiceClientManager; this.retryIntervalInMs = DEFAULT_RETRY_INTERVAL_IN_MS; this.threadName = @@ -204,21 +237,22 @@ public synchronized void send(TsBlock tsBlock) { long sizeInBytes = tsBlock.getSizeInBytes(); int startSequenceId; startSequenceId = nextSequenceId; - blocked = + MemoryReservationResult reserveResult = localMemoryManager .getQueryPool() - .reserve( + .reserveWithPriority( localFragmentInstanceId.getQueryId(), fullFragmentInstanceId, localPlanNodeId, sizeInBytes, - maxBytesCanReserve) - .left; - bufferRetainedSizeInBytes += sizeInBytes; + maxBytesCanReserve, + isHighestPriority); + blocked = reserveResult.getFuture(); + bufferRetainedSizeInBytes += reserveResult.getReservedBytes(); sequenceIdToTsBlock.put(nextSequenceId, new Pair<>(tsBlock, currentTsBlockSize)); nextSequenceId += 1; - currentTsBlockSize = sizeInBytes; + currentTsBlockSize = reserveResult.getReservedBytes(); submitSendNewDataBlockEventTask(startSequenceId, ImmutableList.of(sizeInBytes)); } finally { @@ -433,19 +467,21 @@ public synchronized void open() { return; } // SinkChannel is opened when ShuffleSinkHandle choose it as the next channel - this.blocked = + MemoryReservationResult reserveResult = localMemoryManager .getQueryPool() - .reserve( + .reserveWithPriority( localFragmentInstanceId.getQueryId(), fullFragmentInstanceId, localPlanNodeId, DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES, - maxBytesCanReserve) // actually we only know maxBytesCanReserve after - // the handle is created, so we use DEFAULT here. It is ok to use DEFAULT here because - // at first this SinkChannel has not reserved memory. - .left; - this.bufferRetainedSizeInBytes = DEFAULT_MAX_TSBLOCK_SIZE_IN_BYTES; + maxBytesCanReserve, + isHighestPriority); // actually we only know maxBytesCanReserve after + // the handle is created, so we use DEFAULT here. It is ok to use DEFAULT here because + // at first this SinkChannel has not reserved memory. + this.blocked = reserveResult.getFuture(); + this.bufferRetainedSizeInBytes = reserveResult.getReservedBytes(); + this.currentTsBlockSize = reserveResult.getReservedBytes(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/source/SourceHandle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/source/SourceHandle.java index 2d1a06fcd3217..0e0b94094b566 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/source/SourceHandle.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/exchange/source/SourceHandle.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.queryengine.common.FragmentInstanceId; import org.apache.iotdb.db.queryengine.execution.exchange.MPPDataExchangeManager.SourceHandleListener; import org.apache.iotdb.db.queryengine.execution.memory.LocalMemoryManager; +import org.apache.iotdb.db.queryengine.execution.memory.MemoryPool.MemoryReservationResult; import org.apache.iotdb.db.queryengine.metric.DataExchangeCostMetricSet; import org.apache.iotdb.db.queryengine.metric.DataExchangeCountMetricSet; import org.apache.iotdb.db.utils.SetThreadName; @@ -115,6 +116,8 @@ public class SourceHandle implements ISourceHandle { */ private boolean canGetTsBlockFromRemote = false; + private final boolean isHighestPriority; + private static final DataExchangeCostMetricSet DATA_EXCHANGE_COST_METRIC_SET = DataExchangeCostMetricSet.getInstance(); private static final DataExchangeCountMetricSet DATA_EXCHANGE_COUNT_METRIC_SET = @@ -124,6 +127,7 @@ public class SourceHandle implements ISourceHandle { RamUsageEstimator.shallowSizeOfInstance(SourceHandle.class) + RamUsageEstimator.shallowSizeOfInstance(TFragmentInstanceId.class) * 2; + @TestOnly @SuppressWarnings("squid:S107") public SourceHandle( TEndPoint remoteEndpoint, @@ -137,6 +141,34 @@ public SourceHandle( SourceHandleListener sourceHandleListener, IClientManager mppDataExchangeServiceClientManager) { + this( + remoteEndpoint, + remoteFragmentInstanceId, + localFragmentInstanceId, + localPlanNodeId, + indexOfUpstreamSinkHandle, + localMemoryManager, + executorService, + serde, + sourceHandleListener, + false, + mppDataExchangeServiceClientManager); + } + + @SuppressWarnings("squid:S107") + public SourceHandle( + TEndPoint remoteEndpoint, + TFragmentInstanceId remoteFragmentInstanceId, + TFragmentInstanceId localFragmentInstanceId, + String localPlanNodeId, + int indexOfUpstreamSinkHandle, + LocalMemoryManager localMemoryManager, + ExecutorService executorService, + TsBlockSerde serde, + SourceHandleListener sourceHandleListener, + boolean isHighestPriority, + IClientManager + mppDataExchangeServiceClientManager) { this.remoteEndpoint = Validate.notNull(remoteEndpoint, "remoteEndpoint can not be null."); this.remoteFragmentInstanceId = Validate.notNull(remoteFragmentInstanceId, "remoteFragmentInstanceId can not be null."); @@ -152,6 +184,7 @@ public SourceHandle( this.serde = Validate.notNull(serde, "serde can not be null."); this.sourceHandleListener = Validate.notNull(sourceHandleListener, "sourceHandleListener can not be null."); + this.isHighestPriority = isHighestPriority; this.bufferRetainedSizeInBytes = 0L; this.mppDataExchangeServiceClientManager = mppDataExchangeServiceClientManager; this.retryIntervalInMs = DEFAULT_RETRY_INTERVAL_IN_MS; @@ -192,19 +225,24 @@ public synchronized ByteBuffer getSerializedTsBlock() { if (tsBlock == null) { return null; } - long retainedSize = sequenceIdToDataBlockSize.remove(currSequenceId); + Long retainedSize = sequenceIdToDataBlockSize.remove(currSequenceId); + if (retainedSize == null) { + throw new IllegalStateException("Reserved data block size is null."); + } if (LOGGER.isDebugEnabled()) { LOGGER.debug("[GetTsBlockFromBuffer] sequenceId:{}, size:{}", currSequenceId, retainedSize); } currSequenceId += 1; - bufferRetainedSizeInBytes -= retainedSize; - localMemoryManager - .getQueryPool() - .free( - localFragmentInstanceId.getQueryId(), - fullFragmentInstanceId, - localPlanNodeId, - retainedSize); + if (retainedSize > 0) { + bufferRetainedSizeInBytes -= retainedSize; + localMemoryManager + .getQueryPool() + .free( + localFragmentInstanceId.getQueryId(), + fullFragmentInstanceId, + localPlanNodeId, + retainedSize); + } if (sequenceIdToTsBlock.isEmpty() && !isFinished()) { if (LOGGER.isDebugEnabled()) { @@ -241,18 +279,24 @@ private synchronized void trySubmitGetDataBlocksTask() { if (bytesToReserve == null) { throw new IllegalStateException("Data block size is null."); } - pair = + MemoryReservationResult reserveResult = localMemoryManager .getQueryPool() - .reserve( + .reserveWithPriority( localFragmentInstanceId.getQueryId(), fullFragmentInstanceId, localPlanNodeId, bytesToReserve, - maxBytesCanReserve); - bufferRetainedSizeInBytes += bytesToReserve; + maxBytesCanReserve, + isHighestPriority); + pair = new Pair<>(reserveResult.getFuture(), reserveResult.isReserveSuccess()); + // actually reserve size is not equals raw size, update the actually reserve size to the map + if (reserveResult.getReservedBytes() != bytesToReserve) { + sequenceIdToDataBlockSize.put(endSequenceId, reserveResult.getReservedBytes()); + } + bufferRetainedSizeInBytes += reserveResult.getReservedBytes(); endSequenceId += 1; - reservedBytes += bytesToReserve; + reservedBytes += reserveResult.getReservedBytes(); if (!Boolean.TRUE.equals(pair.right)) { blockedSize = bytesToReserve; break; @@ -619,14 +663,16 @@ private void fail(Throwable t) { if (aborted || closed) { return; } - bufferRetainedSizeInBytes -= reservedBytes; - localMemoryManager - .getQueryPool() - .free( - localFragmentInstanceId.getQueryId(), - fullFragmentInstanceId, - localPlanNodeId, - reservedBytes); + if (reservedBytes > 0) { + bufferRetainedSizeInBytes -= reservedBytes; + localMemoryManager + .getQueryPool() + .free( + localFragmentInstanceId.getQueryId(), + fullFragmentInstanceId, + localPlanNodeId, + reservedBytes); + } sourceHandleListener.onFailure(SourceHandle.this, t); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index f655df432bff7..f02b86579b496 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -152,6 +152,7 @@ public class FragmentInstanceContext extends QueryContext { private long unclosedUnseqFileNum = 0; private long closedSeqFileNum = 0; private long closedUnseqFileNum = 0; + private boolean highestPriority = false; public static FragmentInstanceContext createFragmentInstanceContext( FragmentInstanceId id, FragmentInstanceStateMachine stateMachine, SessionInfo sessionInfo) { @@ -1128,6 +1129,18 @@ public boolean ignoreNotExistsDevice() { return ignoreNotExistsDevice; } + /** + * Same flag as {@link + * org.apache.iotdb.db.queryengine.plan.analyze.IAnalysis#needSetHighestPriority()}. + */ + public boolean isHighestPriority() { + return highestPriority; + } + + public void setHighestPriority(boolean highestPriority) { + this.highestPriority = highestPriority; + } + public boolean isSingleSourcePath() { return singleSourcePath; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceManager.java index e8d0fd8243201..9dacdf44d887d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceManager.java @@ -158,6 +158,7 @@ public FragmentInstanceInfo execDataQueryFragmentInstance( dataRegion, instance.getGlobalTimePredicate(), dataNodeQueryContextMap)); + context.setHighestPriority(instance.isHighestPriority()); try { List driverFactories = @@ -259,6 +260,7 @@ public FragmentInstanceInfo execSchemaQueryFragmentInstance( fragmentInstanceId -> createFragmentInstanceContext( fragmentInstanceId, stateMachine, instance.getSessionInfo())); + context.setHighestPriority(instance.isHighestPriority()); try { List driverFactories = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/memory/MemoryPool.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/memory/MemoryPool.java index a9ab6ed5d8120..9a00d5a87de0d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/memory/MemoryPool.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/memory/MemoryPool.java @@ -23,10 +23,8 @@ import org.apache.iotdb.db.exception.runtime.MemoryLeakException; import com.google.common.util.concurrent.AbstractFuture; -import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.commons.lang3.Validate; -import org.apache.tsfile.utils.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,6 +39,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import static com.google.common.util.concurrent.Futures.immediateVoidFuture; + /** A thread-safe memory pool. */ public class MemoryPool { @@ -111,6 +111,31 @@ public boolean set(@Nullable V value) { } } + public static class MemoryReservationResult { + private final ListenableFuture future; + private final boolean reserveSuccess; + private final long reservedBytes; + + public MemoryReservationResult( + ListenableFuture future, boolean reserveSuccess, long reservedBytes) { + this.future = future; + this.reserveSuccess = reserveSuccess; + this.reservedBytes = reservedBytes; + } + + public ListenableFuture getFuture() { + return future; + } + + public boolean isReserveSuccess() { + return reserveSuccess; + } + + public long getReservedBytes() { + return reservedBytes; + } + } + private final String id; private final long maxBytes; private final long maxBytesPerFragmentInstance; @@ -220,18 +245,20 @@ public void deRegisterFragmentInstanceFromQueryMemoryMap( } /** - * Reserve memory with bytesToReserve. + * Reserve memory with bytesToReserve respect priority. * - * @return if reserve succeed, pair.right will be true, otherwise false + * @return if reserve succeed, reservedBytes may be zero or equals with bytesToReserve; if reserve + * failed, reservedBytes must be equals with bytesToReserve * @throws IllegalArgumentException throw exception if current query requests more memory than can * be allocated. */ - public Pair, Boolean> reserve( + public MemoryReservationResult reserveWithPriority( String queryId, String fragmentInstanceId, String planNodeId, long bytesToReserve, - long maxBytesCanReserve) { + long maxBytesCanReserve, + boolean isHighestPriority) { Validate.notNull(queryId, "queryId can not be null."); Validate.notNull(fragmentInstanceId, "fragmentInstanceId can not be null."); Validate.notNull(planNodeId, "planNodeId can not be null."); @@ -250,19 +277,21 @@ public Pair, Boolean> reserve( "Query is aborted since it requests more memory than can be allocated."); } - ListenableFuture result; if (tryReserve(queryId, fragmentInstanceId, planNodeId, bytesToReserve, maxBytesCanReserve)) { - result = Futures.immediateFuture(null); - return new Pair<>(result, Boolean.TRUE); + return new MemoryReservationResult(immediateVoidFuture(), true, bytesToReserve); } else { + rollbackReserve(queryId, fragmentInstanceId, planNodeId, bytesToReserve); + if (isHighestPriority) { + // SHOW QUERIES: treat as success with zero bytes reserved from pool when insufficient. + return new MemoryReservationResult(immediateVoidFuture(), true, 0L); + } LOGGER.debug( "Blocked reserve request: {} bytes memory for planNodeId{}", bytesToReserve, planNodeId); - rollbackReserve(queryId, fragmentInstanceId, planNodeId, bytesToReserve); - result = + ListenableFuture result = MemoryReservationFuture.create( queryId, fragmentInstanceId, planNodeId, bytesToReserve, maxBytesCanReserve); memoryReservationFutures.add((MemoryReservationFuture) result); - return new Pair<>(result, Boolean.FALSE); + return new MemoryReservationResult(result, false, bytesToReserve); } } @@ -293,7 +322,8 @@ public boolean tryReserveForTest( /** * Cancel the specified memory reservation. If the reservation has finished, do nothing. * - * @param future The future returned from {@link #reserve(String, String, String, long, long)} + * @param future The future returned from {@link #reserveWithPriority(String, String, String, + * long, long, boolean)} * @return If the future has not complete, return the number of bytes being reserved. Otherwise, * return 0. */ diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analysis.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analysis.java index 01d1020934dda..63b2d379ddd93 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analysis.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/Analysis.java @@ -503,7 +503,7 @@ public boolean isQuery() { public boolean needSetHighestPriority() { // if is this Statement is ShowQueryStatement, set its instances to the highest priority, so // that the sub-tasks of the ShowQueries instances could be executed first. - return StatementType.SHOW_QUERIES.equals(statement.getType()); + return statement != null && StatementType.SHOW_QUERIES.equals(statement.getType()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java index 8de6165525e29..5af143bfca83f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java @@ -125,6 +125,7 @@ public QueryExecution(IPlanner planner, MPPQueryContext context, ExecutorService this.context = context; this.planner = planner; this.analysis = analyze(context); + context.setNeedSetHighestPriority(analysis.needSetHighestPriority()); this.stateMachine = new QueryStateMachine(context.getQueryId(), executor); // We add the abort logic inside the QueryExecution. @@ -587,7 +588,8 @@ private void initResultHandle() { context.getResultNodeContext().getUpStreamPlanNodeId().getId(), context.getResultNodeContext().getUpStreamFragmentInstanceId().toThrift(), 0, // Upstream of result ExchangeNode will only have one child. - stateMachine::transitionToFailed) + stateMachine::transitionToFailed, + context.needSetHighestPriority()) : MPPDataExchangeService.getInstance() .getMPPDataExchangeManager() .createSourceHandle( @@ -596,7 +598,8 @@ private void initResultHandle() { 0, upstreamEndPoint, context.getResultNodeContext().getUpStreamFragmentInstanceId().toThrift(), - stateMachine::transitionToFailed); + stateMachine::transitionToFailed, + context.needSetHighestPriority()); } @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java index e0aeacb7f56b0..0649f808c4ffd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java @@ -2624,14 +2624,16 @@ public Operator visitExchange(ExchangeNode node, LocalExecutionPlanContext conte node.getUpstreamPlanNodeId().getId(), remoteInstanceId.toThrift(), node.getIndexOfUpstreamSinkHandle(), - context.getInstanceContext()::failed) + context.getInstanceContext()::failed, + context.getInstanceContext().isHighestPriority()) : MPP_DATA_EXCHANGE_MANAGER.createSourceHandle( localInstanceId.toThrift(), node.getPlanNodeId().getId(), node.getIndexOfUpstreamSinkHandle(), upstreamEndPoint, remoteInstanceId.toThrift(), - context.getInstanceContext()::failed); + context.getInstanceContext()::failed, + context.getInstanceContext().isHighestPriority()); if (!isSameNode) { context.addExchangeSumNum(1); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/FragmentInstance.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/FragmentInstance.java index d212f4ca72526..b96485199ad4e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/FragmentInstance.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/FragmentInstance.java @@ -223,6 +223,7 @@ public static FragmentInstance deserializeFrom(ByteBuffer buffer) { fragmentInstance.hostDataNode = hasHostDataNode ? ThriftCommonsSerDeUtils.deserializeTDataNodeLocation(buffer) : null; fragmentInstance.isExplainAnalyze = ReadWriteIOUtils.readBool(buffer); + fragmentInstance.setHighestPriority(ReadWriteIOUtils.readBool(buffer)); return fragmentInstance; } @@ -247,6 +248,7 @@ public ByteBuffer serializeToByteBuffer() { ThriftCommonsSerDeUtils.serializeTDataNodeLocation(hostDataNode, outputStream); } ReadWriteIOUtils.write(isExplainAnalyze, outputStream); + ReadWriteIOUtils.write(isHighestPriority, outputStream); return ByteBuffer.wrap(publicBAOS.getBuf(), 0, publicBAOS.size()); } catch (IOException e) { LOGGER.error("Unexpected error occurs when serializing this FragmentInstance.", e); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/LocalSinkChannelTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/LocalSinkChannelTest.java index 8d9c4b9cd6569..b3c50aaa517c5 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/LocalSinkChannelTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/LocalSinkChannelTest.java @@ -92,13 +92,14 @@ public void testSend() { Assert.assertFalse(localSinkChannel.isFinished()); Assert.assertEquals(11 * mockTsBlockSize, localSinkChannel.getBufferRetainedSizeInBytes()); Mockito.verify(spyMemoryPool, Mockito.times(11)) - .reserve( + .reserveWithPriority( queryId, FragmentInstanceId.createFragmentInstanceIdFromTFragmentInstanceId( remoteFragmentInstanceId), remotePlanNodeId, mockTsBlockSize, - Long.MAX_VALUE); + Long.MAX_VALUE, + false); // Receive TsBlocks. int numOfReceivedTsblocks = 0; @@ -184,13 +185,14 @@ public void testAbort() { Assert.assertFalse(localSinkChannel.isFinished()); Assert.assertEquals(11 * mockTsBlockSize, localSinkChannel.getBufferRetainedSizeInBytes()); Mockito.verify(spyMemoryPool, Mockito.times(11)) - .reserve( + .reserveWithPriority( queryId, FragmentInstanceId.createFragmentInstanceIdFromTFragmentInstanceId( remoteFragmentInstanceId), remotePlanNodeId, mockTsBlockSize, - Long.MAX_VALUE); + Long.MAX_VALUE, + false); // Abort. localSinkChannel.abort(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/SharedTsBlockQueueTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/SharedTsBlockQueueTest.java index 46196d1c990d1..a95fd35ba76c9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/SharedTsBlockQueueTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/SharedTsBlockQueueTest.java @@ -21,13 +21,13 @@ import org.apache.iotdb.db.queryengine.execution.memory.LocalMemoryManager; import org.apache.iotdb.db.queryengine.execution.memory.MemoryPool; +import org.apache.iotdb.db.queryengine.execution.memory.MemoryPool.MemoryReservationResult; import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import org.apache.commons.lang3.Validate; import org.apache.tsfile.read.common.block.TsBlock; -import org.apache.tsfile.utils.Pair; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -62,15 +62,16 @@ public void testAsyncListenerAfterAbortDoesNotAddTsBlock() { MemoryPool mockMemoryPool = Mockito.mock(MemoryPool.class); Mockito.when(mockLocalMemoryManager.getQueryPool()).thenReturn(mockMemoryPool); - // reserve() returns (manualFuture, false) — simulating memory blocked + // reserveWithPriority() returns blocked future and reserve failure. Mockito.when( - mockMemoryPool.reserve( + mockMemoryPool.reserveWithPriority( Mockito.anyString(), Mockito.anyString(), Mockito.anyString(), Mockito.anyLong(), - Mockito.anyLong())) - .thenReturn(new Pair<>(manualFuture, Boolean.FALSE)); + Mockito.anyLong(), + Mockito.anyBoolean())) + .thenReturn(new MemoryReservationResult(manualFuture, false, 1024L)); // tryCancel returns 0 — simulating future already completed (can't cancel) Mockito.when(mockMemoryPool.tryCancel(Mockito.any())).thenReturn(0L); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/ShuffleSinkHandleTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/ShuffleSinkHandleTest.java index 489ec5c9ed607..4190c2fa61abe 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/ShuffleSinkHandleTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/ShuffleSinkHandleTest.java @@ -104,13 +104,14 @@ public void testAbort() { Assert.assertFalse(localSinkChannel.isFinished()); Assert.assertEquals(11 * mockTsBlockSize, localSinkChannel.getBufferRetainedSizeInBytes()); Mockito.verify(spyMemoryPool, Mockito.times(11)) - .reserve( + .reserveWithPriority( queryId, FragmentInstanceId.createFragmentInstanceIdFromTFragmentInstanceId( remoteFragmentInstanceId), remotePlanNodeId, mockTsBlockSize, - Long.MAX_VALUE); + Long.MAX_VALUE, + false); // Abort. shuffleSinkHandle.abort(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/SourceHandleTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/SourceHandleTest.java index ee8493372065a..40721c93db785 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/SourceHandleTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/SourceHandleTest.java @@ -254,13 +254,14 @@ public void testBlockedOneTimeReceive() { .collect(Collectors.toList())); try { Mockito.verify(spyMemoryPool, Mockito.timeout(10_000).times(6)) - .reserve( + .reserveWithPriority( queryId, FragmentInstanceId.createFragmentInstanceIdFromTFragmentInstanceId( localFragmentInstanceId), localPlanNodeId, MOCK_TSBLOCK_SIZE, - maxBytesCanReserve); + maxBytesCanReserve, + false); Mockito.verify(mockClient, Mockito.timeout(10_0000).times(1)) .getDataBlock( Mockito.argThat( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/Utils.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/Utils.java index 327d4a34c39e7..b09498ad949dd 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/Utils.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/exchange/Utils.java @@ -20,11 +20,11 @@ package org.apache.iotdb.db.queryengine.execution.exchange; import org.apache.iotdb.db.queryengine.execution.memory.MemoryPool; +import org.apache.iotdb.db.queryengine.execution.memory.MemoryPool.MemoryReservationResult; import com.google.common.util.concurrent.SettableFuture; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.read.common.block.column.TsBlockSerde; -import org.apache.tsfile.utils.Pair; import org.mockito.Mockito; import org.mockito.stubbing.Answer; @@ -68,21 +68,25 @@ public static MemoryPool createMockBlockedMemoryPool( settableFuture.get().set(null); AtomicReference reservedBytes = new AtomicReference<>(0L); Mockito.when( - mockMemoryPool.reserve( + mockMemoryPool.reserveWithPriority( Mockito.eq(queryId), Mockito.eq(fragmentInstanceId), Mockito.eq(planNodeId), Mockito.anyLong(), - Mockito.anyLong())) + Mockito.anyLong(), + Mockito.anyBoolean())) .thenAnswer( invocation -> { long bytesToReserve = invocation.getArgument(3); if (reservedBytes.get() + bytesToReserve <= capacityInBytes) { - reservedBytes.updateAndGet(v -> v + (long) invocation.getArgument(3)); - return new Pair<>(settableFuture.get(), true); + reservedBytes.updateAndGet(v -> v + bytesToReserve); + return new MemoryReservationResult(settableFuture.get(), true, bytesToReserve); } else { + if (invocation.getArgument(5)) { + return new MemoryReservationResult(settableFuture.get(), true, 0L); + } settableFuture.set(SettableFuture.create()); - return new Pair<>(settableFuture.get(), false); + return new MemoryReservationResult(settableFuture.get(), false, bytesToReserve); } }); Mockito.doAnswer( @@ -124,13 +128,17 @@ public static MemoryPool createMockBlockedMemoryPool( public static MemoryPool createMockNonBlockedMemoryPool() { MemoryPool mockMemoryPool = Mockito.mock(MemoryPool.class); Mockito.when( - mockMemoryPool.reserve( + mockMemoryPool.reserveWithPriority( Mockito.anyString(), Mockito.anyString(), Mockito.anyString(), Mockito.anyLong(), - Mockito.anyLong())) - .thenReturn(new Pair<>(immediateFuture(null), true)); + Mockito.anyLong(), + Mockito.anyBoolean())) + .thenAnswer( + invocation -> + new MemoryReservationResult( + immediateFuture(null), true, invocation.getArgument(3))); Mockito.when( mockMemoryPool.tryReserve( Mockito.anyString(), diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/memory/MemoryPoolTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/memory/MemoryPoolTest.java index bae3c0643f45c..cf7e15dc71f46 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/memory/MemoryPoolTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/memory/MemoryPoolTest.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.queryengine.execution.memory; +import org.apache.iotdb.db.queryengine.execution.memory.MemoryPool.MemoryReservationResult; + import com.google.common.util.concurrent.ListenableFuture; import org.junit.Assert; import org.junit.Before; @@ -94,7 +96,9 @@ public void testOverTryReserve() { public void testReserve() { ListenableFuture future = - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE).left; + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE, false) + .getFuture(); Assert.assertTrue(future.isDone()); Assert.assertEquals(256L, pool.getQueryMemoryReservedBytes(QUERY_ID)); Assert.assertEquals(256L, pool.getReservedBytes()); @@ -104,7 +108,8 @@ public void testReserve() { public void tesReserveZero() { try { - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 0L, Long.MAX_VALUE); + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 0L, Long.MAX_VALUE, false); Assert.fail("Expect IllegalArgumentException"); } catch (IllegalArgumentException ignore) { } @@ -114,7 +119,8 @@ public void tesReserveZero() { public void testReserveNegative() { try { - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, -1L, Long.MAX_VALUE); + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, -1L, Long.MAX_VALUE, false); Assert.fail("Expect IllegalArgumentException"); } catch (IllegalArgumentException ignore) { } @@ -124,7 +130,9 @@ public void testReserveNegative() { public void testReserveAll() { ListenableFuture future = - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, Long.MAX_VALUE).left; + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, Long.MAX_VALUE, false) + .getFuture(); Assert.assertTrue(future.isDone()); Assert.assertEquals(512L, pool.getQueryMemoryReservedBytes(QUERY_ID)); Assert.assertEquals(512L, pool.getReservedBytes()); @@ -134,11 +142,15 @@ public void testReserveAll() { public void testOverReserve() { ListenableFuture future = - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE).left; + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE, false) + .getFuture(); Assert.assertTrue(future.isDone()); Assert.assertEquals(256L, pool.getQueryMemoryReservedBytes(QUERY_ID)); Assert.assertEquals(256L, pool.getReservedBytes()); - future = pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L).left; + future = + pool.reserveWithPriority(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L, false) + .getFuture(); Assert.assertFalse(future.isDone()); Assert.assertEquals(256L, pool.getQueryMemoryReservedBytes(QUERY_ID)); Assert.assertEquals(256L, pool.getReservedBytes()); @@ -148,11 +160,13 @@ public void testOverReserve() { public void testReserveAndFree() { Assert.assertTrue( - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, Long.MAX_VALUE) - .left + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, Long.MAX_VALUE, false) + .getFuture() .isDone()); ListenableFuture future = - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L).left; + pool.reserveWithPriority(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L, false) + .getFuture(); Assert.assertFalse(future.isDone()); Assert.assertEquals(512L, pool.getQueryMemoryReservedBytes(QUERY_ID)); Assert.assertEquals(512L, pool.getReservedBytes()); @@ -166,18 +180,22 @@ public void testReserveAndFree() { public void testMultiReserveAndFree() { Assert.assertTrue( - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE) - .left + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE, false) + .getFuture() .isDone()); Assert.assertEquals(256L, pool.getQueryMemoryReservedBytes(QUERY_ID)); Assert.assertEquals(256L, pool.getReservedBytes()); ListenableFuture future1 = - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L).left; + pool.reserveWithPriority(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L, false) + .getFuture(); ListenableFuture future2 = - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L).left; + pool.reserveWithPriority(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L, false) + .getFuture(); ListenableFuture future3 = - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L).left; + pool.reserveWithPriority(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, 513L, false) + .getFuture(); Assert.assertFalse(future1.isDone()); Assert.assertFalse(future2.isDone()); Assert.assertFalse(future3.isDone()); @@ -284,7 +302,8 @@ public void testTryCancelBlockedReservation() { pool.tryReserveForTest(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, Long.MAX_VALUE)); ListenableFuture f = - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, 512L).left; + pool.reserveWithPriority(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, 512L, false) + .getFuture(); Assert.assertFalse(f.isDone()); // Cancel the reservation. Assert.assertEquals(256L, pool.tryCancel(f)); @@ -296,11 +315,76 @@ public void testTryCancelBlockedReservation() { public void testTryCancelCompletedReservation() { ListenableFuture f = - pool.reserve(QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE).left; + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE, false) + .getFuture(); Assert.assertTrue(f.isDone()); // Cancel the reservation. Assert.assertEquals(0L, pool.tryCancel(f)); Assert.assertTrue(f.isDone()); Assert.assertFalse(f.isCancelled()); } + + /** + * Normal query: requested bytes exceed what the pool can still provide — reserve fails (blocked + * future, not immediate success). + */ + @Test + public void testReserveWithPriorityNormalQueryExceedsAvailable() { + MemoryReservationResult r1 = + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, Long.MAX_VALUE, false); + Assert.assertTrue(r1.isReserveSuccess()); + Assert.assertEquals(512L, r1.getReservedBytes()); + Assert.assertTrue(r1.getFuture().isDone()); + + MemoryReservationResult r2 = + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, Long.MAX_VALUE, false); + Assert.assertTrue(r2.isReserveSuccess()); + Assert.assertEquals(512L, r2.getReservedBytes()); + Assert.assertEquals(1024L, pool.getReservedBytes()); + + MemoryReservationResult r3 = + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE, false); + Assert.assertFalse(r3.isReserveSuccess()); + Assert.assertEquals(256L, r3.getReservedBytes()); + Assert.assertFalse(r3.getFuture().isDone()); + Assert.assertEquals(1024L, pool.getReservedBytes()); + } + + /** SHOW QUERIES path: exceeds pool capacity — treated as success with zero bytes from pool. */ + @Test + public void testReserveWithPriorityShowQueriesExceedsAvailable() { + Assert.assertTrue( + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, Long.MAX_VALUE, false) + .isReserveSuccess()); + Assert.assertTrue( + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 512L, Long.MAX_VALUE, false) + .isReserveSuccess()); + Assert.assertEquals(1024L, pool.getReservedBytes()); + + MemoryReservationResult r = + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE, true); + Assert.assertTrue(r.isReserveSuccess()); + Assert.assertEquals(0L, r.getReservedBytes()); + Assert.assertTrue(r.getFuture().isDone()); + Assert.assertEquals(1024L, pool.getReservedBytes()); + } + + /** SHOW QUERIES path: pool has room — same as normal successful reserve. */ + @Test + public void testReserveWithPriorityShowQueriesWithinAvailable() { + MemoryReservationResult r = + pool.reserveWithPriority( + QUERY_ID, FRAGMENT_INSTANCE_ID, PLAN_NODE_ID, 256L, Long.MAX_VALUE, true); + Assert.assertTrue(r.isReserveSuccess()); + Assert.assertEquals(256L, r.getReservedBytes()); + Assert.assertTrue(r.getFuture().isDone()); + Assert.assertEquals(256L, pool.getReservedBytes()); + } } From 57c012620c969b36231a95ee1fe6d090f81cf31d Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Wed, 22 Apr 2026 17:17:08 +0800 Subject: [PATCH 008/102] [to dev/1.3] fix AbstractEnv.ensureNodeStatus & Fix: send set configuration only to target nodes and harden compaction schedule interruption handling (#17447) --- .../iotdb/it/env/cluster/env/AbstractEnv.java | 28 ++++++++++++++++--- .../confignode/manager/node/NodeManager.java | 2 +- .../CompactionScheduleTaskManager.java | 9 ++++++ .../CompactionScheduleTaskWorker.java | 21 ++++++++++++-- .../compaction/schedule/TTLScheduleTask.java | 17 +++++++++-- 5 files changed, 68 insertions(+), 9 deletions(-) diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/env/AbstractEnv.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/env/AbstractEnv.java index 60ff77aa03c52..07b7364d0042d 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/env/AbstractEnv.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/env/AbstractEnv.java @@ -56,7 +56,10 @@ import org.apache.iotdb.session.Session; import org.apache.iotdb.session.pool.SessionPool; +import org.apache.thrift.TConfiguration; import org.apache.thrift.TException; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransportException; import org.slf4j.Logger; import java.io.File; @@ -1119,7 +1122,7 @@ public void shutdownForciblyAllDataNodes() { @Override public void ensureNodeStatus( - final List nodes, final List targetStatus) + final List nodes, final List targetStatusList) throws IllegalStateException { Throwable lastException = null; for (int i = 0; i < retryCount; i++) { @@ -1147,7 +1150,9 @@ public void ensureNodeStatus( + node.getClientRpcEndPoint().getPort(), node.getDataNodeId())); for (int j = 0; j < nodes.size(); j++) { - final String endpoint = nodes.get(j).getIpAndPortString(); + BaseNodeWrapper nodeWrapper = nodes.get(j); + String ipAndPortString = nodeWrapper.getIpAndPortString(); + final String endpoint = ipAndPortString; if (!nodeIds.containsKey(endpoint)) { // Node not exist // Notice: Never modify this line, since the NodeLocation might be modified in IT @@ -1155,12 +1160,27 @@ public void ensureNodeStatus( continue; } final String status = showClusterResp.getNodeStatus().get(nodeIds.get(endpoint)); - if (!targetStatus.get(j).getStatus().equals(status)) { + final NodeStatus targetStatus = targetStatusList.get(j); + if (!targetStatus.getStatus().equals(status)) { // Error status errorMessages.add( String.format( "Node %s is in status %s, but expected %s", - endpoint, status, targetStatus.get(j))); + endpoint, status, targetStatusList.get(j))); + continue; + } + if (nodeWrapper instanceof DataNodeWrapper && targetStatus.equals(NodeStatus.Running)) { + final String[] ipPort = nodeWrapper.getIpAndPortString().split(":"); + final String ip = ipPort[0]; + final int port = Integer.parseInt(ipPort[1]); + try (TSocket socket = new TSocket(new TConfiguration(), ip, port, 1000)) { + socket.open(); + } catch (final TTransportException e) { + errorMessages.add( + String.format( + "DataNode %s is not reachable: %s", + nodeWrapper.getIpAndPortString(), e.getMessage())); + } } } if (errorMessages.isEmpty()) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java index 2e0ac35d5df98..81cad2f3befdf 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java @@ -930,7 +930,7 @@ public List setConfiguration(TSetConfigurationReq req) { if (!targetDataNodes.isEmpty()) { DataNodeAsyncRequestContext clientHandler = new DataNodeAsyncRequestContext<>( - CnToDnAsyncRequestType.SET_CONFIGURATION, req, dataNodeLocationMap); + CnToDnAsyncRequestType.SET_CONFIGURATION, req, targetDataNodes); CnToDnInternalServiceAsyncRequestManager.getInstance() .sendAsyncRequestWithRetry(clientHandler); responseList.addAll(clientHandler.getResponseList()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleTaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleTaskManager.java index 8348b87813768..516b1489a206d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleTaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleTaskManager.java @@ -65,6 +65,7 @@ public class CompactionScheduleTaskManager implements IService { ConcurrentHashMap.newKeySet(); private ReentrantLock lock = new ReentrantLock(); private volatile boolean init = false; + private volatile boolean isStoppingAllScheduleTask = false; @Override public void start() throws StartupException { @@ -76,8 +77,13 @@ public void start() throws StartupException { logger.info("Compaction schedule task manager started."); } + public boolean isStoppingAllScheduleTask() { + return isStoppingAllScheduleTask; + } + public void stopCompactionScheduleTasks() throws InterruptedException { lock.lock(); + isStoppingAllScheduleTask = true; try { for (Future task : submitCompactionScheduleTaskFutures) { task.cancel(true); @@ -121,6 +127,7 @@ public void checkAndMayApplyConfigurationChange() throws InterruptedException { public void startScheduleTasks() { lock.lock(); + isStoppingAllScheduleTask = false; try { // compaction selector for (int workerId = 0; workerId < compactionSelectorNum; workerId++) { @@ -144,6 +151,7 @@ public void startScheduleTasks() { @Override public void stop() { lock.lock(); + isStoppingAllScheduleTask = true; try { if (!init) { return; @@ -160,6 +168,7 @@ public void stop() { @Override public void waitAndStop(long milliseconds) { lock.lock(); + isStoppingAllScheduleTask = true; try { if (!init) { return; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleTaskWorker.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleTaskWorker.java index 17ad0dd4334e6..714d232d62538 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleTaskWorker.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/CompactionScheduleTaskWorker.java @@ -72,9 +72,26 @@ public Void call() { dataRegion.executeCompaction(); } } catch (InterruptedException ignored) { + boolean isStoppedByUser = + CompactionScheduleTaskManager.getInstance().isStoppingAllScheduleTask(); logger.info( - "[CompactionScheduleTaskWorker-{}] compaction schedule is interrupted", workerId); - return null; + "[CompactionScheduleTaskWorker-{}] compaction schedule is interrupted, isStopByUser: {}", + workerId, + isStoppedByUser); + if (isStoppedByUser) { + return null; + } + } catch (Exception e) { + logger.error( + "[CompactionScheduleTaskWorker-{}] Failed to execute compaction schedule task", + workerId, + e); + } catch (Throwable t) { + logger.error( + "[CompactionScheduleTaskWorker-{}] Failed to execute compaction schedule task and cannot recover", + workerId, + t); + throw t; } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/TTLScheduleTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/TTLScheduleTask.java index 393a9f6d2dc68..c8cba2b52927b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/TTLScheduleTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/schedule/TTLScheduleTask.java @@ -66,8 +66,21 @@ public Void call() throws Exception { } } } catch (InterruptedException ignored) { - logger.info("[TTLCheckTask-{}] TTL checker is interrupted", workerId); - return null; + boolean isStoppedByUser = + CompactionScheduleTaskManager.getInstance().isStoppingAllScheduleTask(); + logger.info( + "[TTLCheckTask-{}] TTL checker is interrupted, isStoppedByUser: {}", + workerId, + isStoppedByUser); + if (isStoppedByUser) { + return null; + } + } catch (Exception e) { + logger.error("[TTLCheckTask-{}] Failed to execute ttl check", workerId, e); + } catch (Throwable t) { + logger.error( + "[TTLCheckTask-{}] Failed to execute ttl check and cannot recover", workerId, t); + throw t; } } } From 5c4853bdae838fc8fc697ee802975b5610307dc1 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 22 Apr 2026 17:23:44 +0800 Subject: [PATCH 009/102] fix (#17533) --- .../opcua/client/IoTDBOpcUaClient.java | 65 ++++++++++++++----- 1 file changed, 50 insertions(+), 15 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/client/IoTDBOpcUaClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/client/IoTDBOpcUaClient.java index 2019c0fe83336..7595b75747e1e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/client/IoTDBOpcUaClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/client/IoTDBOpcUaClient.java @@ -25,6 +25,7 @@ import org.apache.iotdb.db.pipe.sink.protocol.opcua.server.OpcUaNameSpace; import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.MeasurementSchema; @@ -56,9 +57,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.nio.file.Paths; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Objects; import java.util.concurrent.ExecutionException; @@ -130,7 +132,6 @@ private void transferTabletRowForClientServerModel( StatusCode currentQuality = sink.getDefaultQuality(); Object value = null; long timestamp = 0; - NodeId nodeId = null; NodeId opcDataType = null; for (int i = 0; i < measurementSchemas.size(); ++i) { @@ -153,17 +154,43 @@ private void transferTabletRowForClientServerModel( "When the 'with-quality' mode is enabled, the measurement must be either \"value-name\" or \"quality-name\""); continue; } - nodeId = new NodeId(NAME_SPACE_INDEX, String.join("/", segments)); final long utcTimestamp = timestampToUtc(timestamps.get(timestamps.size() > 1 ? i : 0)); - value = values.get(i); - timestamp = utcTimestamp; - opcDataType = convertToOpcDataType(type); + if (Objects.isNull(sink.getValueName())) { + writeValue( + values.get(i), + utcTimestamp, + convertToOpcDataType(type), + currentQuality, + segments, + name); + } else { + value = values.get(i); + timestamp = utcTimestamp; + opcDataType = convertToOpcDataType(type); + } } if (Objects.isNull(value)) { return; } + writeValue(value, timestamp, opcDataType, currentQuality, segments, null); + } + + private void writeValue( + final Object value, + final long timestamp, + final NodeId opcDataType, + final StatusCode currentQuality, + final String[] segments, + final @Nullable String name) + throws Exception { + final NodeId nodeId = + new NodeId( + NAME_SPACE_INDEX, + Objects.nonNull(name) + ? String.join("/", segments) + "/" + name + : String.join("/", segments)); final Variant variant = new Variant(value); final DataValue dataValue = new DataValue(variant, currentQuality, new DateTime(timestamp), new DateTime()); @@ -171,36 +198,41 @@ private void transferTabletRowForClientServerModel( if (writeStatus.getValue() == StatusCodes.Bad_NodeIdUnknown) { final AddNodesResponse addStatus = - client.addNodes(getNodesToAdd(segments, opcDataType, variant)).get(); + client.addNodes(getNodesToAdd(segments, name, opcDataType, variant)).get(); for (final AddNodesResult result : addStatus.getResults()) { if (!result.getStatusCode().equals(StatusCode.GOOD) && !(result.getStatusCode().getValue() == StatusCodes.Bad_NodeIdExists)) { throw new PipeException( "Failed to create nodes after transfer data value, creation status: " + addStatus - + getErrorString(segments, opcDataType, value, writeStatus)); + + getErrorString(segments, name, opcDataType, value, writeStatus)); } } writeStatus = client.writeValue(nodeId, dataValue).get(); if (writeStatus.getValue() != StatusCode.GOOD.getValue()) { throw new PipeException( "Failed to transfer dataValue after successfully created nodes" - + getErrorString(segments, opcDataType, value, writeStatus)); + + getErrorString(segments, name, opcDataType, value, writeStatus)); } } else if (writeStatus.getValue() != StatusCode.GOOD.getValue()) { throw new PipeException( "Failed to transfer dataValue" - + getErrorString(segments, opcDataType, value, writeStatus)); + + getErrorString(segments, name, opcDataType, value, writeStatus)); } } private static String getErrorString( final String[] segments, + final @Nullable String name, final NodeId dataType, final Object value, final StatusCode writeStatus) { - return ", segments: " - + Arrays.toString(segments) + return ", measurement: " + + (Objects.nonNull(name) + ? String.join(TsFileConstant.PATH_SEPARATOR, segments) + + TsFileConstant.PATH_SEPARATOR + + name + : String.join(TsFileConstant.PATH_SEPARATOR, segments)) + ", dataType: " + dataType + ", value: " @@ -210,7 +242,10 @@ private static String getErrorString( } public List getNodesToAdd( - final String[] segments, final NodeId opcDataType, final Variant initialValue) { + final String[] segments, + final @Nullable String name, + final NodeId opcDataType, + final Variant initialValue) { final List addNodesItems = new ArrayList<>(); final StringBuilder sb = new StringBuilder(segments[0]); ExpandedNodeId curNodeId = new NodeId(NAME_SPACE_INDEX, segments[0]).expanded(); @@ -226,7 +261,7 @@ public List getNodesToAdd( Identifiers.FolderType.expanded())); // segments.length >= 3 - for (int i = 1; i < segments.length - 1; ++i) { + for (int i = 1; i < (Objects.nonNull(name) ? segments.length : segments.length - 1); ++i) { sb.append("/").append(segments[i]); final ExpandedNodeId nextId = new NodeId(NAME_SPACE_INDEX, sb.toString()).expanded(); addNodesItems.add( @@ -242,7 +277,7 @@ public List getNodesToAdd( curNodeId = nextId; } - final String measurementName = segments[segments.length - 1]; + final String measurementName = Objects.nonNull(name) ? name : segments[segments.length - 1]; sb.append("/").append(measurementName); addNodesItems.add( new AddNodesItem( From 9fff324b02325b44bb88da9d027b364bef236ff8 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 23 Apr 2026 14:30:11 +0800 Subject: [PATCH 010/102] fix: catch exception during TVList ownership transfer to avoid memory leak --- .../fragment/FragmentInstanceContext.java | 20 ++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index f02b86579b496..08e23eea69c74 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -32,6 +32,7 @@ import org.apache.iotdb.db.queryengine.common.FragmentInstanceId; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.common.SessionInfo; +import org.apache.iotdb.db.queryengine.exception.MemoryNotEnoughException; import org.apache.iotdb.db.queryengine.metric.DriverSchedulerMetricSet; import org.apache.iotdb.db.queryengine.metric.QueryRelatedResourceMetricSet; import org.apache.iotdb.db.queryengine.metric.QueryResourceMetricSet; @@ -899,9 +900,22 @@ private void releaseTVListOwnedByQuery() { memoryReservationManager.releaseMemoryVirtually(tvList.getReservedMemoryBytes()); FragmentInstanceContext queryContext = (FragmentInstanceContext) queryContextSet.iterator().next(); - queryContext - .getMemoryReservationContext() - .reserveMemoryVirtually(releasedBytes.left, releasedBytes.right); + try { + queryContext + .getMemoryReservationContext() + .reserveMemoryVirtually(releasedBytes.left, releasedBytes.right); + } catch (MemoryNotEnoughException ex) { + LOGGER.warn( + "MemoryNotEnoughException when transferring TVList ownership from query {} to another query {}.", + this.getId(), + queryContext.getId()); + } catch (RuntimeException ex) { + LOGGER.warn( + "Unexpected Exception when transferring TVList ownership from query {} to another query {}.", + this.getId(), + queryContext.getId(), + ex); + } if (LOGGER.isDebugEnabled()) { LOGGER.debug( From e96ce958bac13ffeb51057e84751394b70f371ea Mon Sep 17 00:00:00 2001 From: Weihao Li <60659567+Wei-hao-Li@users.noreply.github.com> Date: Mon, 27 Apr 2026 14:25:05 +0800 Subject: [PATCH 011/102] [To dev/1.3] Fix appearance of dispatching FI failed because of thrift frame is oversize (#17550) --- .../TCompressedElasticFramedTransport.java | 1 + .../iotdb/rpc/TElasticFramedTransport.java | 11 ++++++++++ .../org/apache/iotdb/rpc/TSStatusCode.java | 1 + .../db/protocol/thrift/OperationType.java | 3 ++- .../FragmentInstanceDispatcherImpl.java | 11 ++++++++++ .../iotdb/db/utils/ErrorHandlingUtils.java | 21 ++++++++++++++++++- 6 files changed, 46 insertions(+), 2 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java index a3b4f38064a88..8579f46a3cbd1 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java @@ -75,6 +75,7 @@ public void flush() throws TTransportException { writeCompressBuffer.resizeIfNecessary(maxCompressedLength); int compressedLength = compress(writeBuffer.getBuffer(), 0, length, writeCompressBuffer.getBuffer(), 0); + checkWriteFrameSize(compressedLength); RpcStat.writeCompressedBytes.addAndGet(compressedLength); TFramedTransport.encodeFrameSize(compressedLength, i32buf); underlying.write(i32buf, 0, 4); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java index 67948a19f049d..d98914ce934a3 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java @@ -199,9 +199,20 @@ protected void readFrame() throws TTransportException { readBuffer.fill(underlying, size); } + protected void checkWriteFrameSize(int size) throws TTransportException { + if (size <= thriftMaxFrameSize) { + return; + } + close(); + throw new TTransportException( + TTransportException.CORRUPTED_DATA, + "Frame size (" + size + ") larger than protect max size (" + thriftMaxFrameSize + ")!"); + } + @Override public void flush() throws TTransportException { int length = writeBuffer.getPos(); + checkWriteFrameSize(length); TFramedTransport.encodeFrameSize(length, i32buf); underlying.write(i32buf, 0, 4); underlying.write(writeBuffer.getBuffer(), 0, length); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index c2f76d294847c..e2855ed0a5032 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -47,6 +47,7 @@ public enum TSStatusCode { INTERNAL_SERVER_ERROR(305), DISPATCH_ERROR(306), LICENSE_ERROR(307), + THRIFT_FRAME_OVERSIZE(308), // Client, REDIRECTION_RECOMMEND(400), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/OperationType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/OperationType.java index 065a568353f7a..5838dac25bfba 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/OperationType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/OperationType.java @@ -50,7 +50,8 @@ public enum OperationType { CREATE_SCHEMA_TEMPLATE("createSchemaTemplate"), CHECK_AUTHORITY("checkAuthority"), EXECUTE_NON_QUERY_PLAN("executeNonQueryPlan"), - QUERY_LATENCY("queryLatency"); + QUERY_LATENCY("queryLatency"), + DISPATCH_FRAGMENT_INSTANCE("dispatchFragmentInstance"); private final String name; OperationType(String name) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java index 7bcdd1fb6947f..41d325a5761a5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java @@ -57,6 +57,7 @@ import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransportException; import org.apache.tsfile.utils.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,6 +72,7 @@ import static com.google.common.util.concurrent.Futures.immediateFuture; import static org.apache.iotdb.db.queryengine.metric.QueryExecutionMetricSet.DISPATCH_READ; +import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onThriftFrameOversizeException; public class FragmentInstanceDispatcherImpl implements IFragInstanceDispatcher { @@ -518,6 +520,15 @@ private void dispatchRemoteHelper(final FragmentInstance instance, final TEndPoi TSStatusCode.EXECUTE_STATEMENT_ERROR, String.format("unknown read type [%s]", instance.getType()))); } + } catch (TException e) { + Throwable rootCause = ExceptionUtils.getRootCause(e); + if (rootCause instanceof TTransportException + && ((TTransportException) rootCause).getType() == TTransportException.CORRUPTED_DATA) { + // Don't set DISPATCH_ERROR status to avoid retry if dispatch failed because of thrift frame + // is oversize + throw new FragmentInstanceDispatchException(onThriftFrameOversizeException(rootCause)); + } + throw e; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ErrorHandlingUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ErrorHandlingUtils.java index 963cecceb104c..765f19c31bf0c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ErrorHandlingUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ErrorHandlingUtils.java @@ -46,6 +46,7 @@ import java.util.concurrent.ExecutionException; import static org.apache.iotdb.commons.utils.StatusUtils.needRetry; +import static org.apache.iotdb.db.protocol.thrift.OperationType.DISPATCH_FRAGMENT_INSTANCE; public class ErrorHandlingUtils { @@ -60,10 +61,11 @@ private ErrorHandlingUtils() {} "The read statement is not allowed in batch: "; private static final String ERROR_OPERATION_LOG = "Status code: {}, operation: {} failed"; + private static final String EXCEPTION_PATTERN = "[%s] Exception occurred: %s failed. "; public static TSStatus onNpeOrUnexpectedException( Exception e, String operation, TSStatusCode statusCode) { - String message = String.format("[%s] Exception occurred: %s failed. ", statusCode, operation); + String message = String.format(EXCEPTION_PATTERN, statusCode, operation); if (e instanceof IOException || e instanceof NullPointerException) { LOGGER.error(ERROR_OPERATION_LOG, statusCode, operation, e); } else { @@ -87,6 +89,23 @@ public static TSStatus onNpeOrUnexpectedException( return onNpeOrUnexpectedException(e, operation.getName(), statusCode); } + public static Throwable getRootCause(Throwable e) { + while (e.getCause() != null) { + e = e.getCause(); + } + return e; + } + + public static TSStatus onThriftFrameOversizeException(Throwable t) { + TSStatus status = + new TSStatus(TSStatusCode.THRIFT_FRAME_OVERSIZE.getStatusCode()).setNeedRetry(false); + String message = + String.format(EXCEPTION_PATTERN, status, DISPATCH_FRAGMENT_INSTANCE) + + getRootCause(t).getMessage(); + LOGGER.warn(message); + return status.setMessage(message); + } + public static TSStatus onQueryException(Exception e, String operation, TSStatusCode statusCode) { TSStatus status = tryCatchQueryException(e); if (status != null) { From d5770c523e8f4ac2e71ba1ae29becfbc426a3dc0 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 28 Apr 2026 17:42:42 +0800 Subject: [PATCH 012/102] [To dev/1.3] Fixed the sequence of limit & offset in show-devices --- .../plan/planner/LogicalPlanVisitor.java | 4 ++-- .../logical/RegionScanLogicalPlannerTest.java | 8 ++++---- .../logical/SchemaQueryLogicalPlannerTest.java | 17 +++++++++++++++++ 3 files changed, 23 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java index be0b28a0282ac..69fd1208535d3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java @@ -585,8 +585,8 @@ public PlanNode visitShowDevices( planBuilder = planBuilder .planDeviceRegionScan(analysis.getDevicePathToContextMap(), false) - .planLimit(showDevicesStatement.getLimit()) - .planOffset(showDevicesStatement.getOffset()); + .planOffset(showDevicesStatement.getOffset()) + .planLimit(showDevicesStatement.getLimit()); return planBuilder.getRoot(); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/RegionScanLogicalPlannerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/RegionScanLogicalPlannerTest.java index 437cad010e9c0..2b2d169cafa51 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/RegionScanLogicalPlannerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/RegionScanLogicalPlannerTest.java @@ -224,13 +224,13 @@ public void testShowDevicesWithTimeConditionWithLimitOffset() throws IllegalPath DeviceRegionScanNode regionScanNode = new DeviceRegionScanNode(queryId.genPlanNodeId(), deviceContextMap, false, null); - LimitNode limitNode = new LimitNode(queryId.genPlanNodeId(), 20); - limitNode.addChild(regionScanNode); OffsetNode offsetNode = new OffsetNode(queryId.genPlanNodeId(), 10); - offsetNode.addChild(limitNode); + offsetNode.addChild(regionScanNode); + LimitNode limitNode = new LimitNode(queryId.genPlanNodeId(), 20); + limitNode.addChild(offsetNode); PlanNode actualPlan = parseSQLToPlanNode(sql); - Assert.assertEquals(actualPlan, offsetNode); + Assert.assertEquals(actualPlan, limitNode); } @Test diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/SchemaQueryLogicalPlannerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/SchemaQueryLogicalPlannerTest.java index 60a0d4835624e..9274e4d04404b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/SchemaQueryLogicalPlannerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/SchemaQueryLogicalPlannerTest.java @@ -47,6 +47,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.LimitNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.OffsetNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.process.RawDataAggregationNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.DeviceRegionScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.AggregationStep; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.CrossSeriesAggregationDescriptor; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; @@ -648,6 +649,22 @@ public void testShowDevicesWherePathContains() { } } + @Test + public void testShowDevicesWithTimeConditionPaginationOrder() { + String sql = "SHOW DEVICES root.ln.wf01.wt01 WHERE time >= 10 limit 20 offset 10"; + try { + LimitNode limitNode = (LimitNode) parseSQLToPlanNode(sql); + Assert.assertEquals(20, limitNode.getLimit()); + + OffsetNode offsetNode = (OffsetNode) limitNode.getChild(); + Assert.assertEquals(10, offsetNode.getOffset()); + Assert.assertTrue(offsetNode.getChild() instanceof DeviceRegionScanNode); + } catch (Exception e) { + e.printStackTrace(); + fail(); + } + } + @Test public void testCountNodes() { String sql = "COUNT NODES root.ln LEVEL=1"; From aa1e322fa8184441eb821e7a18ac60db05c83912 Mon Sep 17 00:00:00 2001 From: Zhenyu Luo Date: Tue, 28 Apr 2026 17:47:41 +0800 Subject: [PATCH 013/102] Fix duplicate deletion emission in TsFileSplitter (#17538) --- .../load/splitter/DeletionData.java | 4 + .../load/splitter/TsFileSplitter.java | 1 - ...tchedCompactionWithTsFileSplitterTest.java | 86 +++++++++++++++++++ 3 files changed, 90 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/DeletionData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/DeletionData.java index 186426650fe50..dc4908ef2c38f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/DeletionData.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/DeletionData.java @@ -59,6 +59,10 @@ public void serialize(DataOutputStream stream) throws IOException { deletion.serializeWithoutFileOffset(stream); } + public Deletion getModEntry() { + return this.deletion; + } + public static DeletionData deserialize(InputStream stream) throws IllegalPathException, IOException { return new DeletionData(Deletion.deserializeWithoutFileOffset(new DataInputStream(stream))); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/TsFileSplitter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/TsFileSplitter.java index af401f84aa777..4571bdb1531d2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/TsFileSplitter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/TsFileSplitter.java @@ -156,7 +156,6 @@ private void processTimeChunkOrNonAlignedChunk(TsFileSequenceReader reader, byte long chunkOffset = reader.position(); timeChunkIndexOfCurrentValueColumn = pageIndex2TimesList.size(); consumeAllAlignedChunkData(chunkOffset, pageIndex2ChunkData); - handleModification(offset2Deletions, chunkOffset); ChunkHeader header = reader.readChunkHeader(marker); String measurementId = header.getMeasurementID(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/BatchedCompactionWithTsFileSplitterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/BatchedCompactionWithTsFileSplitterTest.java index 37d38411ff671..1a407648a51f2 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/BatchedCompactionWithTsFileSplitterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/BatchedCompactionWithTsFileSplitterTest.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.load.LoadFileException; @@ -30,10 +31,15 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.task.CompactionTaskSummary; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.CompactionUtils; import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionCheckerUtils; +import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionTestFileWriter; +import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; +import org.apache.iotdb.db.storageengine.dataregion.modification.Modification; +import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.generator.TsFileNameGenerator; import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileResourceUtils; import org.apache.iotdb.db.storageengine.load.splitter.AlignedChunkData; +import org.apache.iotdb.db.storageengine.load.splitter.DeletionData; import org.apache.iotdb.db.storageengine.load.splitter.TsFileData; import org.apache.iotdb.db.storageengine.load.splitter.TsFileSplitter; @@ -54,7 +60,9 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; +import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -233,6 +241,84 @@ public void testCompactionFlushPageAndSplitByTimePartition() consumeChunkDataAndValidate(targetResource); } + @Test + public void testDeletionDataShouldOnlyBeGeneratedOnceAtEnd() + throws IOException, MetadataException, LoadFileException, IllegalPathException { + TsFileResource resource = createAlignedMultiDeviceFile(); + try (ModificationFile modificationFile = ModificationFile.getNormalMods(resource)) { + modificationFile.write( + new Deletion(new MeasurementPath("root.testsg.d0.s0"), Long.MAX_VALUE, 100)); + modificationFile.write( + new Deletion(new MeasurementPath("root.testsg.d0.s1"), Long.MAX_VALUE, 200)); + modificationFile.write( + new Deletion(new MeasurementPath("root.testsg.d1.s0"), Long.MAX_VALUE, 300)); + modificationFile.write( + new Deletion(new MeasurementPath("root.testsg.d1.s1"), Long.MAX_VALUE, 400)); + } + + List expectedMods = + new ArrayList<>(ModificationFile.getNormalMods(resource).getModifications()); + List deletionMods = new ArrayList<>(); + File actualModsFile = new File(resource.getTsFilePath() + ".mods"); + try (ModificationFile actualModificationFile = + new ModificationFile(actualModsFile.getAbsolutePath())) { + TsFileSplitter splitter = + new TsFileSplitter( + resource.getTsFile(), + tsFileData -> { + if (tsFileData instanceof DeletionData) { + deletionMods.add(((DeletionData) tsFileData).getModEntry()); + } + return true; + }); + splitter.splitTsFileByDataPartition(); + } + + List actualMods; + try (ModificationFile actualModificationFile = + new ModificationFile(actualModsFile.getAbsolutePath())) { + actualMods = new ArrayList<>(actualModificationFile.getModifications()); + } + Assert.assertEquals(expectedMods, actualMods); + Files.deleteIfExists(actualModsFile.toPath()); + } + + private TsFileResource createAlignedMultiDeviceFile() throws IOException { + TsFileResource resource = createEmptyFileAndResource(true); + try (CompactionTestFileWriter writer = new CompactionTestFileWriter(resource)) { + writer.startChunkGroup("d0"); + writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue( + Arrays.asList("s0", "s1"), + new TimeRange[][] { + new TimeRange[] {new TimeRange(1, 100), new TimeRange(200, 300)}, + new TimeRange[] { + new TimeRange(604799900, 604800020), new TimeRange(604810020, 604820020) + } + }, + TSEncoding.PLAIN, + CompressionType.LZ4, + Arrays.asList(false, false)); + writer.endChunkGroup(); + + writer.startChunkGroup("d1"); + writer.generateSimpleAlignedSeriesToCurrentDeviceWithNullValue( + Arrays.asList("s0", "s1"), + new TimeRange[][] { + new TimeRange[] {new TimeRange(1, 100), new TimeRange(200, 300)}, + new TimeRange[] { + new TimeRange(604799900, 604800020), new TimeRange(604810020, 604820020) + } + }, + TSEncoding.PLAIN, + CompressionType.LZ4, + Arrays.asList(false, false)); + writer.endChunkGroup(); + + writer.endFile(); + } + return resource; + } + private TsFileResource performCompaction() throws StorageEngineException, IOException, From edfd97d9c6a3514a711d1227fb6be3b5f23387e5 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 29 Apr 2026 10:10:46 +0800 Subject: [PATCH 014/102] Pipe: Optimized the shrink/expand method of blocks (#17555) --- .../pipe/resource/memory/PipeMemoryBlock.java | 9 ++ .../resource/memory/PipeMemoryManager.java | 91 ++++++++++++------- 2 files changed, 65 insertions(+), 35 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryBlock.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryBlock.java index 07f5b904523c8..62069a2fb2504 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryBlock.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryBlock.java @@ -24,6 +24,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Objects; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -62,6 +63,7 @@ public void setMemoryUsageInBytes(final long memoryUsageInBytes) { public PipeMemoryBlock setShrinkMethod(final LongUnaryOperator shrinkMethod) { this.shrinkMethod.set(shrinkMethod); + pipeMemoryManager.addShrinkableBlock(this); return this; } @@ -72,6 +74,7 @@ public PipeMemoryBlock setShrinkCallback(final BiConsumer shrinkCall public PipeMemoryBlock setExpandMethod(final LongUnaryOperator extendMethod) { this.expandMethod.set(extendMethod); + pipeMemoryManager.addExpandableBlock(this); return this; } @@ -177,6 +180,12 @@ public void close() { if (lock.tryLock(50, TimeUnit.MICROSECONDS)) { try { pipeMemoryManager.release(this); + if (Objects.nonNull(shrinkMethod.get())) { + pipeMemoryManager.removeShrinkableBlock(this); + } + if (Objects.nonNull(expandMethod.get())) { + pipeMemoryManager.removeExpandableBlock(this); + } if (isInterrupted) { LOGGER.warn("{} is released after thread interruption.", this); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java index 87be4d5fb62f4..0d49487f7504f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java @@ -56,6 +56,8 @@ public class PipeMemoryManager { // Only non-zero memory blocks will be added to this set. private final Set allocatedBlocks = new HashSet<>(); + private final Set shrinkableBlocks = new HashSet<>(); + private final Set expandableBlocks = new HashSet<>(); public PipeMemoryManager() { PipeDataNodeAgent.runtime() @@ -531,8 +533,9 @@ private PipeMemoryBlock registerMemoryBlock(long sizeInBytes, PipeMemoryBlockTyp return returnedMemoryBlock; } + // Single-threaded logic private boolean tryShrinkUntilFreeMemorySatisfy(long sizeInBytes) { - final List shuffledBlocks = new ArrayList<>(allocatedBlocks); + final List shuffledBlocks = new ArrayList<>(shrinkableBlocks); Collections.shuffle(shuffledBlocks); while (true) { @@ -551,46 +554,64 @@ private boolean tryShrinkUntilFreeMemorySatisfy(long sizeInBytes) { } } + void addShrinkableBlock(final PipeMemoryBlock block) { + shrinkableBlocks.add(block); + } + + void removeShrinkableBlock(final PipeMemoryBlock block) { + shrinkableBlocks.remove(block); + } + public synchronized void tryExpandAllAndCheckConsistency() { - allocatedBlocks.forEach(PipeMemoryBlock::expand); + expandableBlocks.forEach(PipeMemoryBlock::expand); + + if (LOGGER.isDebugEnabled()) { + final long blockSum = + allocatedBlocks.stream().mapToLong(PipeMemoryBlock::getMemoryUsageInBytes).sum(); + if (blockSum != usedMemorySizeInBytes) { + LOGGER.debug( + "tryExpandAllAndCheckConsistency: memory usage is not consistent with allocated blocks," + + " usedMemorySizeInBytes is {} but sum of all blocks is {}", + usedMemorySizeInBytes, + blockSum); + } - long blockSum = - allocatedBlocks.stream().mapToLong(PipeMemoryBlock::getMemoryUsageInBytes).sum(); - if (blockSum != usedMemorySizeInBytes) { - LOGGER.warn( - "tryExpandAllAndCheckConsistency: memory usage is not consistent with allocated blocks," - + " usedMemorySizeInBytes is {} but sum of all blocks is {}", - usedMemorySizeInBytes, - blockSum); - } + final long tabletBlockSum = + allocatedBlocks.stream() + .filter(PipeTabletMemoryBlock.class::isInstance) + .mapToLong(PipeMemoryBlock::getMemoryUsageInBytes) + .sum(); + if (tabletBlockSum != usedMemorySizeInBytesOfTablets) { + LOGGER.debug( + "tryExpandAllAndCheckConsistency: memory usage of tablets is not consistent with allocated blocks," + + " usedMemorySizeInBytesOfTablets is {} but sum of all tablet blocks is {}", + usedMemorySizeInBytesOfTablets, + tabletBlockSum); + } - long tabletBlockSum = - allocatedBlocks.stream() - .filter(PipeTabletMemoryBlock.class::isInstance) - .mapToLong(PipeMemoryBlock::getMemoryUsageInBytes) - .sum(); - if (tabletBlockSum != usedMemorySizeInBytesOfTablets) { - LOGGER.warn( - "tryExpandAllAndCheckConsistency: memory usage of tablets is not consistent with allocated blocks," - + " usedMemorySizeInBytesOfTablets is {} but sum of all tablet blocks is {}", - usedMemorySizeInBytesOfTablets, - tabletBlockSum); - } - - long tsFileBlockSum = - allocatedBlocks.stream() - .filter(PipeTsFileMemoryBlock.class::isInstance) - .mapToLong(PipeMemoryBlock::getMemoryUsageInBytes) - .sum(); - if (tsFileBlockSum != usedMemorySizeInBytesOfTsFiles) { - LOGGER.warn( - "tryExpandAllAndCheckConsistency: memory usage of tsfiles is not consistent with allocated blocks," - + " usedMemorySizeInBytesOfTsFiles is {} but sum of all tsfile blocks is {}", - usedMemorySizeInBytesOfTsFiles, - tsFileBlockSum); + final long tsFileBlockSum = + allocatedBlocks.stream() + .filter(PipeTsFileMemoryBlock.class::isInstance) + .mapToLong(PipeMemoryBlock::getMemoryUsageInBytes) + .sum(); + if (tsFileBlockSum != usedMemorySizeInBytesOfTsFiles) { + LOGGER.debug( + "tryExpandAllAndCheckConsistency: memory usage of tsfiles is not consistent with allocated blocks," + + " usedMemorySizeInBytesOfTsFiles is {} but sum of all tsfile blocks is {}", + usedMemorySizeInBytesOfTsFiles, + tsFileBlockSum); + } } } + void addExpandableBlock(final PipeMemoryBlock block) { + expandableBlocks.add(block); + } + + void removeExpandableBlock(final PipeMemoryBlock block) { + expandableBlocks.remove(block); + } + public synchronized void release(PipeMemoryBlock block) { if (!PIPE_MEMORY_MANAGEMENT_ENABLED || block == null || block.isReleased()) { return; From 754a1502bacd2178088c9fca2fc191215cf953ba Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Wed, 29 Apr 2026 10:49:40 +0800 Subject: [PATCH 015/102] [To dev/1.3] Subscription: support encrypted password auth in consumer builder (#17552) --- .../IoTDBEncryptedPasswordPullConsumerIT.java | 168 ++++++++++++++++++ .../subscription/config/ConsumerConfig.java | 12 ++ .../subscription/config/ConsumerConstant.java | 1 + .../consumer/SubscriptionConsumer.java | 22 +++ .../consumer/SubscriptionProvider.java | 12 ++ .../consumer/SubscriptionPullConsumer.java | 6 + .../consumer/SubscriptionPushConsumer.java | 6 + .../confignode/manager/PermissionManager.java | 9 + .../confignode/persistence/AuthorInfo.java | 52 ++++++ .../impl/pipe/task/CreatePipeProcedureV2.java | 67 +++++++ .../CreateSubscriptionProcedure.java | 9 +- .../pipe/task/CreatePipeProcedureV2Test.java | 61 +++++++ .../config/constant/PipeSourceConstant.java | 7 + .../meta/consumer/ConsumerGroupMeta.java | 4 + .../meta/consumer/ConsumerMeta.java | 20 +++ .../subscription/meta/topic/TopicMeta.java | 11 ++ .../subscription/topic/TopicDeSerTest.java | 11 ++ 17 files changed, 477 insertions(+), 1 deletion(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/regression/param/IoTDBEncryptedPasswordPullConsumerIT.java diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/regression/param/IoTDBEncryptedPasswordPullConsumerIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/regression/param/IoTDBEncryptedPasswordPullConsumerIT.java new file mode 100644 index 0000000000000..9987ebe97322d --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/regression/param/IoTDBEncryptedPasswordPullConsumerIT.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.subscription.it.triple.regression.param; + +import org.apache.iotdb.commons.utils.AuthUtils; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.MultiClusterIT2SubscriptionRegressionMisc; +import org.apache.iotdb.rpc.IoTDBConnectionException; +import org.apache.iotdb.rpc.StatementExecutionException; +import org.apache.iotdb.session.subscription.consumer.SubscriptionPullConsumer; +import org.apache.iotdb.subscription.it.triple.regression.AbstractSubscriptionRegressionIT; + +import org.apache.thrift.TException; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +@RunWith(IoTDBTestRunner.class) +@Category({MultiClusterIT2SubscriptionRegressionMisc.class}) +public class IoTDBEncryptedPasswordPullConsumerIT extends AbstractSubscriptionRegressionIT { + + private static final String DATABASE = "root.TestEncryptedPasswordPullConsumer"; + private static final String DEVICE = DATABASE + ".d_0"; + private static final String TOPIC_NAME = "TestEncryptedPasswordPullConsumerTopic"; + private static final String USERNAME = "encrypted_user"; + private static final String PASSWORD = "EncryptedUser@123"; + private static final String ENCRYPTED_PASSWORD = AuthUtils.encryptPassword(PASSWORD); + private static final String WRONG_ENCRYPTED_PASSWORD = + AuthUtils.encryptPassword("WrongEncryptedUser@123"); + + private static final List SCHEMA_LIST = new ArrayList<>(); + + static { + SCHEMA_LIST.add(new MeasurementSchema("s_0", TSDataType.INT64)); + SCHEMA_LIST.add(new MeasurementSchema("s_1", TSDataType.DOUBLE)); + } + + private SubscriptionPullConsumer consumer; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + createDB(DATABASE); + createTopic_s(TOPIC_NAME, "root.**", null, null, false); + session_src.createTimeseries( + DEVICE + ".s_0", TSDataType.INT64, TSEncoding.GORILLA, CompressionType.LZ4); + session_src.createTimeseries( + DEVICE + ".s_1", TSDataType.DOUBLE, TSEncoding.TS_2DIFF, CompressionType.LZ4); + session_dest.createTimeseries( + DEVICE + ".s_0", TSDataType.INT64, TSEncoding.GORILLA, CompressionType.LZ4); + session_dest.createTimeseries( + DEVICE + ".s_1", TSDataType.DOUBLE, TSEncoding.TS_2DIFF, CompressionType.LZ4); + session_src.executeNonQueryStatement("create user " + USERNAME + " '" + PASSWORD + "'"); + session_src.executeNonQueryStatement("grant read,write on root.** to user " + USERNAME); + Assert.assertTrue(subs.getTopic(TOPIC_NAME).isPresent()); + } + + @Override + @After + public void tearDown() throws Exception { + try { + if (consumer != null) { + consumer.close(); + } + } catch (final Exception ignored) { + } + try { + subs.dropTopic(TOPIC_NAME); + } catch (final Exception ignored) { + } + try { + session_src.executeNonQueryStatement("drop user " + USERNAME); + } catch (final Exception ignored) { + } + dropDB(DATABASE); + super.tearDown(); + } + + @Test + public void testSubscribeWithEncryptedPassword() + throws TException, + IoTDBConnectionException, + IOException, + StatementExecutionException, + InterruptedException { + consumer = createConsumer("encrypted-password-group", ENCRYPTED_PASSWORD); + + consumer.open(); + consumer.subscribe(TOPIC_NAME); + Assert.assertEquals(1, subs.getSubscriptions().size()); + + insertData(1706659200000L); + consume_data(consumer, session_dest); + check_count( + 4, + "select count(s_0) from " + DEVICE + " where time >= 1706659200000", + "encrypted password consumption"); + } + + @Test + public void testSubscribeFailsWithWrongEncryptedPassword() + throws IoTDBConnectionException, StatementExecutionException { + consumer = createConsumer("wrong-encrypted-password-group", WRONG_ENCRYPTED_PASSWORD); + + try { + consumer.open(); + consumer.subscribe(TOPIC_NAME); + Assert.fail("subscribe should fail when encrypted password mismatches"); + } catch (final Exception ignored) { + Assert.assertTrue(subs.getSubscriptions().isEmpty()); + } + } + + private SubscriptionPullConsumer createConsumer( + final String consumerGroupId, final String encryptedPassword) { + return new SubscriptionPullConsumer.Builder() + .host(SRC_HOST) + .port(SRC_PORT) + .username(USERNAME) + .encryptedPassword(encryptedPassword) + .consumerId("consumer_" + consumerGroupId) + .consumerGroupId(consumerGroupId) + .buildPullConsumer(); + } + + private void insertData(long timestamp) + throws IoTDBConnectionException, StatementExecutionException { + final Tablet tablet = new Tablet(DEVICE, SCHEMA_LIST, 10); + for (int row = 0; row < 5; row++) { + final int rowIndex = tablet.rowSize++; + tablet.addTimestamp(rowIndex, timestamp); + tablet.addValue("s_0", rowIndex, row * 20L + row); + tablet.addValue("s_1", rowIndex, row + 2.45); + timestamp += row * 2000; + } + session_src.insertTablet(tablet); + } +} diff --git a/iotdb-client/subscription/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConfig.java b/iotdb-client/subscription/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConfig.java index 5094ae7eea42e..97c35baf8f4a3 100644 --- a/iotdb-client/subscription/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConfig.java +++ b/iotdb-client/subscription/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConfig.java @@ -68,6 +68,18 @@ public String getConsumerGroupId() { return getString(ConsumerConstant.CONSUMER_GROUP_ID_KEY); } + public String getUsername() { + return getString(ConsumerConstant.USERNAME_KEY); + } + + public String getPassword() { + return getString(ConsumerConstant.PASSWORD_KEY); + } + + public String getEncryptedPassword() { + return getString(ConsumerConstant.ENCRYPTED_PASSWORD_KEY); + } + public long getHeartbeatIntervalMs() { return getLongOrDefault( ConsumerConstant.HEARTBEAT_INTERVAL_MS_KEY, diff --git a/iotdb-client/subscription/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java b/iotdb-client/subscription/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java index 504893b80ed55..21b13e5d3f128 100644 --- a/iotdb-client/subscription/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java +++ b/iotdb-client/subscription/src/main/java/org/apache/iotdb/rpc/subscription/config/ConsumerConstant.java @@ -31,6 +31,7 @@ public class ConsumerConstant { public static final String USERNAME_KEY = "username"; public static final String PASSWORD_KEY = "password"; + public static final String ENCRYPTED_PASSWORD_KEY = "encrypted-password"; public static final String CONSUMER_ID_KEY = "consumer-id"; public static final String CONSUMER_GROUP_ID_KEY = "group-id"; diff --git a/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionConsumer.java b/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionConsumer.java index 92c6b2c9a3e29..56f1c2e9b9b02 100644 --- a/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionConsumer.java +++ b/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionConsumer.java @@ -100,6 +100,7 @@ abstract class SubscriptionConsumer implements AutoCloseable { private final String username; private final String password; + private final String encryptedPassword; protected String consumerId; protected String consumerGroupId; @@ -177,6 +178,7 @@ protected SubscriptionConsumer(final Builder builder) { this.username = builder.username; this.password = builder.password; + this.encryptedPassword = builder.encryptedPassword; this.consumerId = builder.consumerId; this.consumerGroupId = builder.consumerGroupId; @@ -206,6 +208,7 @@ protected SubscriptionConsumer(final Builder builder, final Properties propertie (String) properties.getOrDefault( ConsumerConstant.PASSWORD_KEY, SessionConfig.DEFAULT_PASSWORD)) + .encryptedPassword((String) properties.get(ConsumerConstant.ENCRYPTED_PASSWORD_KEY)) .consumerId((String) properties.get(ConsumerConstant.CONSUMER_ID_KEY)) .consumerGroupId((String) properties.get(ConsumerConstant.CONSUMER_GROUP_ID_KEY)) .heartbeatIntervalMs( @@ -386,6 +389,7 @@ SubscriptionProvider constructProviderAndHandshake(final TEndPoint endPoint) endPoint, this.username, this.password, + this.encryptedPassword, this.consumerId, this.consumerGroupId, this.thriftMaxFrameSize, @@ -1401,6 +1405,7 @@ public abstract static class Builder { protected String username = SessionConfig.DEFAULT_USER; protected String password = SessionConfig.DEFAULT_PASSWORD; + protected String encryptedPassword; protected String consumerId; protected String consumerGroupId; @@ -1437,10 +1442,27 @@ public Builder username(final String username) { } public Builder password(final String password) { + if (!Objects.equals(password, SessionConfig.DEFAULT_PASSWORD) + && Objects.nonNull(this.encryptedPassword)) { + throw new IllegalStateException( + "password and encryptedPassword are mutually exclusive; encryptedPassword is already set"); + } this.password = password; return this; } + public Builder encryptedPassword(final String encryptedPassword) { + if (Objects.isNull(encryptedPassword)) { + return this; + } + if (!Objects.equals(this.password, SessionConfig.DEFAULT_PASSWORD)) { + throw new IllegalStateException( + "password and encryptedPassword are mutually exclusive; password is already set"); + } + this.encryptedPassword = encryptedPassword; + return this; + } + public Builder consumerId(@Nullable final String consumerId) { if (Objects.isNull(consumerId)) { return this; diff --git a/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionProvider.java b/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionProvider.java index 9f153efeb106c..25d9c5da09e69 100644 --- a/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionProvider.java +++ b/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionProvider.java @@ -86,11 +86,15 @@ final class SubscriptionProvider extends SubscriptionSession { private final long heartbeatIntervalMs; private final int connectionTimeoutInMs; private int dataNodeId; + private final String username; + private final String password; + private final String encryptedPassword; SubscriptionProvider( final TEndPoint endPoint, final String username, final String password, + final String encryptedPassword, final String consumerId, final String consumerGroupId, final int thriftMaxFrameSize, @@ -101,6 +105,9 @@ final class SubscriptionProvider extends SubscriptionSession { this.endPoint = endPoint; this.consumerId = consumerId; this.consumerGroupId = consumerGroupId; + this.username = username; + this.password = password; + this.encryptedPassword = encryptedPassword; this.heartbeatIntervalMs = heartbeatIntervalMs; this.connectionTimeoutInMs = connectionTimeoutInMs; } @@ -149,6 +156,11 @@ synchronized void handshake() throws SubscriptionException, IoTDBConnectionExcep final Map consumerAttributes = new HashMap<>(); consumerAttributes.put(ConsumerConstant.CONSUMER_GROUP_ID_KEY, consumerGroupId); consumerAttributes.put(ConsumerConstant.CONSUMER_ID_KEY, consumerId); + consumerAttributes.put(ConsumerConstant.USERNAME_KEY, username); + consumerAttributes.put(ConsumerConstant.PASSWORD_KEY, password); + if (encryptedPassword != null) { + consumerAttributes.put(ConsumerConstant.ENCRYPTED_PASSWORD_KEY, encryptedPassword); + } consumerAttributes.put( ConsumerConstant.HEARTBEAT_INTERVAL_MS_KEY, String.valueOf(heartbeatIntervalMs)); consumerAttributes.put( diff --git a/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionPullConsumer.java b/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionPullConsumer.java index a77716fe02cdf..227843811b15b 100644 --- a/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionPullConsumer.java +++ b/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionPullConsumer.java @@ -329,6 +329,12 @@ public Builder password(final String password) { return this; } + @Override + public Builder encryptedPassword(final String encryptedPassword) { + super.encryptedPassword(encryptedPassword); + return this; + } + @Override public Builder consumerId(final String consumerId) { super.consumerId(consumerId); diff --git a/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionPushConsumer.java b/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionPushConsumer.java index 2a41327809025..b70633bef35a8 100644 --- a/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionPushConsumer.java +++ b/iotdb-client/subscription/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionPushConsumer.java @@ -257,6 +257,12 @@ public Builder password(final String password) { return this; } + @Override + public Builder encryptedPassword(final String encryptedPassword) { + super.encryptedPassword(encryptedPassword); + return this; + } + @Override public Builder consumerId(final String consumerId) { super.consumerId(consumerId); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java index 9592692d1bc45..b4f31d84023fa 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java @@ -110,6 +110,15 @@ public TPermissionInfoResp login(String username, String password) { return authorInfo.login(username, password); } + public TPermissionInfoResp login( + final String username, final String password, final boolean useEncryptedPassword) { + return authorInfo.login(username, password, useEncryptedPassword); + } + + public String login4Pipe(final String userName, final String password) { + return authorInfo.login4Pipe(userName, password); + } + public TPermissionInfoResp checkUserPrivileges( String username, List paths, int permission) { return authorInfo.checkUserPrivileges(username, paths, permission); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java index e9e1058d79ba6..3bd17b5b5b76f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/AuthorInfo.java @@ -113,6 +113,58 @@ public TPermissionInfoResp login(String username, String password) { return result; } + public TPermissionInfoResp login( + final String username, final String password, final boolean useEncryptedPassword) { + if (!useEncryptedPassword) { + return login(username, password); + } + + boolean status = false; + String loginMessage = null; + TSStatus tsStatus = new TSStatus(); + TPermissionInfoResp result = new TPermissionInfoResp(); + try { + final User user = authorizer.getUser(username); + status = user != null && password != null && password.equals(user.getPassword()); + if (status) { + result = getUserPermissionInfo(username); + result.setStatus(RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS, "Login successfully")); + } else { + result = AuthUtils.generateEmptyPermissionInfoResp(); + } + } catch (AuthException e) { + LOGGER.error("meet error while logging in.", e); + loginMessage = e.getMessage(); + } + if (!status) { + tsStatus.setMessage(loginMessage != null ? loginMessage : "Authentication failed."); + tsStatus.setCode(TSStatusCode.WRONG_LOGIN_PASSWORD.getStatusCode()); + result.setStatus(tsStatus); + } + return result; + } + + public String login4Pipe(final String username, final String password) { + try { + final User user = authorizer.getUser(username); + if (user == null) { + return null; + } + if (password == null) { + return user.getPassword(); + } + final TPermissionInfoResp loginResp = login(username, password); + if (loginResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode() + && loginResp.isSetUserInfo()) { + return loginResp.getUserInfo().getPassword(); + } + return null; + } catch (final AuthException e) { + LOGGER.error("meet error while logging in for pipe.", e); + return null; + } + } + // if All paths fail, return No permission // if some paths fail, return SUCCESS and failed index list // if all path success, return success and empty index list diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java index 26e9eb9bba145..ad18cc95bb050 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java @@ -25,11 +25,13 @@ import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; +import org.apache.iotdb.commons.pipe.agent.plugin.builtin.BuiltinPipePlugin; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeRuntimeMeta; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStaticMeta; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStatus; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeType; +import org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant; import org.apache.iotdb.commons.schema.SchemaConstant; import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.CreatePipePlanV2; @@ -41,8 +43,10 @@ import org.apache.iotdb.confignode.procedure.impl.pipe.PipeTaskOperation; import org.apache.iotdb.confignode.procedure.store.ProcedureType; import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq; +import org.apache.iotdb.confignode.rpc.thrift.TPermissionInfoResp; import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.pipe.api.PipePlugin; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.iotdb.rpc.TSStatusCode; @@ -53,6 +57,8 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -136,9 +142,70 @@ public boolean executeFromValidateTask(ConfigNodeProcedureEnv env) throws PipeEx createPipeRequest.getProcessorAttributes(), createPipeRequest.getConnectorAttributes()); + checkAndEnrichSourceAuthentication(env, createPipeRequest.getExtractorAttributes()); + return pipeTaskInfo.get().checkBeforeCreatePipe(createPipeRequest); } + public static void checkAndEnrichSourceAuthentication( + final ConfigNodeProcedureEnv env, final Map sourceAttributes) { + if (Objects.isNull(sourceAttributes)) { + return; + } + final PipeParameters sourceParameters = new PipeParameters(sourceAttributes); + + final String pluginName = + sourceParameters + .getStringOrDefault( + Arrays.asList(PipeSourceConstant.EXTRACTOR_KEY, PipeSourceConstant.SOURCE_KEY), + BuiltinPipePlugin.IOTDB_EXTRACTOR.getPipePluginName()) + .toLowerCase(); + + if (!pluginName.equals(BuiltinPipePlugin.IOTDB_EXTRACTOR.getPipePluginName()) + && !pluginName.equals(BuiltinPipePlugin.IOTDB_SOURCE.getPipePluginName())) { + return; + } + + if (sourceParameters.hasAttribute(PipeSourceConstant.EXTRACTOR_IOTDB_USER_KEY) + || sourceParameters.hasAttribute(PipeSourceConstant.SOURCE_IOTDB_USER_KEY) + || sourceParameters.hasAttribute(PipeSourceConstant.EXTRACTOR_IOTDB_USERNAME_KEY) + || sourceParameters.hasAttribute(PipeSourceConstant.SOURCE_IOTDB_USERNAME_KEY) + || sourceParameters.hasAttribute(PipeSourceConstant.EXTRACTOR_IOTDB_PASSWORD_KEY) + || sourceParameters.hasAttribute(PipeSourceConstant.SOURCE_IOTDB_PASSWORD_KEY)) { + final String username = + sourceParameters.getStringByKeys( + PipeSourceConstant.EXTRACTOR_IOTDB_USER_KEY, + PipeSourceConstant.SOURCE_IOTDB_USER_KEY, + PipeSourceConstant.EXTRACTOR_IOTDB_USERNAME_KEY, + PipeSourceConstant.SOURCE_IOTDB_USERNAME_KEY); + final String password = + sourceParameters.getStringByKeys( + PipeSourceConstant.EXTRACTOR_IOTDB_PASSWORD_KEY, + PipeSourceConstant.SOURCE_IOTDB_PASSWORD_KEY); + String hashedPassword = null; + if (Objects.nonNull(password)) { + final TPermissionInfoResp loginResp = + env.getConfigManager().getPermissionManager().login(username, password, true); + if (Objects.nonNull(loginResp) + && Objects.nonNull(loginResp.getStatus()) + && loginResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + hashedPassword = password; + } + } + if (Objects.isNull(hashedPassword)) { + hashedPassword = + env.getConfigManager().getPermissionManager().login4Pipe(username, password); + } + if (Objects.isNull(hashedPassword)) { + throw new PipeException("Authentication failed."); + } + sourceParameters.addOrReplaceEquivalentAttributes( + new PipeParameters( + Collections.singletonMap( + PipeSourceConstant.SOURCE_IOTDB_PASSWORD_KEY, hashedPassword))); + } + } + @Override public void executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) { LOGGER.info( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedure.java index 4a48ebdd35d6a..80e4e5115541a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedure.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStaticMeta; import org.apache.iotdb.commons.subscription.meta.consumer.ConsumerGroupMeta; +import org.apache.iotdb.commons.subscription.meta.consumer.ConsumerMeta; import org.apache.iotdb.commons.subscription.meta.topic.TopicMeta; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; @@ -89,7 +90,11 @@ protected boolean executeFromValidate(final ConfigNodeProcedureEnv env) subscriptionInfo.get().validateBeforeSubscribe(subscribeReq); // Construct AlterConsumerGroupProcedure + final String consumerId = subscribeReq.getConsumerId(); final String consumerGroupId = subscribeReq.getConsumerGroupId(); + final ConsumerGroupMeta consumerGroupMeta = + subscriptionInfo.get().getConsumerGroupMeta(consumerGroupId); + final ConsumerMeta consumerMeta = consumerGroupMeta.getConsumerMeta(consumerId); final ConsumerGroupMeta updatedConsumerGroupMeta = subscriptionInfo.get().deepCopyConsumerGroupMeta(consumerGroupId); updatedConsumerGroupMeta.addSubscription( @@ -110,7 +115,9 @@ protected boolean executeFromValidate(final ConfigNodeProcedureEnv env) new CreatePipeProcedureV2( new TCreatePipeReq() .setPipeName(pipeName) - .setExtractorAttributes(topicMeta.generateExtractorAttributes()) + .setExtractorAttributes( + topicMeta.generateExtractorAttributes( + consumerMeta.getUsername(), consumerMeta.getSubscriptionAuthPassword())) .setProcessorAttributes(topicMeta.generateProcessorAttributes()) .setConnectorAttributes(topicMeta.generateConnectorAttributes(consumerGroupId)), pipeTaskInfo)); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2Test.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2Test.java index 6c095de98529a..ac6ca8c740c36 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2Test.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2Test.java @@ -19,11 +19,19 @@ package org.apache.iotdb.confignode.procedure.impl.pipe.task; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.pipe.config.constant.PipeSourceConstant; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.PermissionManager; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.store.ProcedureFactory; import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq; +import org.apache.iotdb.confignode.rpc.thrift.TPermissionInfoResp; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.utils.PublicBAOS; import org.junit.Test; +import org.mockito.Mockito; import java.io.DataOutputStream; import java.nio.ByteBuffer; @@ -68,4 +76,57 @@ public void serializeDeserializeTest() { fail(); } } + + @Test + public void testCheckAndEnrichSourceAuthenticationWithEncryptedPassword() { + final ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + final ConfigManager configManager = Mockito.mock(ConfigManager.class); + final PermissionManager permissionManager = Mockito.mock(PermissionManager.class); + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getPermissionManager()).thenReturn(permissionManager); + + final TPermissionInfoResp loginResp = new TPermissionInfoResp(); + loginResp.setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + Mockito.when(permissionManager.login("user", "encrypted-password", true)).thenReturn(loginResp); + + final Map sourceAttributes = new HashMap<>(); + sourceAttributes.put("extractor", "iotdb-source"); + sourceAttributes.put("username", "user"); + sourceAttributes.put("password", "encrypted-password"); + + CreatePipeProcedureV2.checkAndEnrichSourceAuthentication(env, sourceAttributes); + + assertEquals( + "encrypted-password", sourceAttributes.get(PipeSourceConstant.SOURCE_IOTDB_PASSWORD_KEY)); + Mockito.verify(permissionManager).login("user", "encrypted-password", true); + Mockito.verify(permissionManager, Mockito.never()) + .login4Pipe(Mockito.anyString(), Mockito.any()); + } + + @Test + public void testCheckAndEnrichSourceAuthenticationFallsBackToRawPassword() { + final ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + final ConfigManager configManager = Mockito.mock(ConfigManager.class); + final PermissionManager permissionManager = Mockito.mock(PermissionManager.class); + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getPermissionManager()).thenReturn(permissionManager); + + final TPermissionInfoResp loginResp = new TPermissionInfoResp(); + loginResp.setStatus(new TSStatus(TSStatusCode.WRONG_LOGIN_PASSWORD.getStatusCode())); + Mockito.when(permissionManager.login("user", "raw-password", true)).thenReturn(loginResp); + Mockito.when(permissionManager.login4Pipe("user", "raw-password")) + .thenReturn("hashed-password"); + + final Map sourceAttributes = new HashMap<>(); + sourceAttributes.put(PipeSourceConstant.SOURCE_KEY, "iotdb-source"); + sourceAttributes.put(PipeSourceConstant.SOURCE_IOTDB_USERNAME_KEY, "user"); + sourceAttributes.put(PipeSourceConstant.SOURCE_IOTDB_PASSWORD_KEY, "raw-password"); + + CreatePipeProcedureV2.checkAndEnrichSourceAuthentication(env, sourceAttributes); + + assertEquals( + "hashed-password", sourceAttributes.get(PipeSourceConstant.SOURCE_IOTDB_PASSWORD_KEY)); + Mockito.verify(permissionManager).login("user", "raw-password", true); + Mockito.verify(permissionManager).login4Pipe("user", "raw-password"); + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSourceConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSourceConstant.java index e9adf64bc03d1..aba780fb0d1af 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSourceConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeSourceConstant.java @@ -24,6 +24,13 @@ public class PipeSourceConstant { public static final String EXTRACTOR_KEY = "extractor"; public static final String SOURCE_KEY = "source"; + public static final String EXTRACTOR_IOTDB_USER_KEY = "extractor.user"; + public static final String SOURCE_IOTDB_USER_KEY = "source.user"; + public static final String EXTRACTOR_IOTDB_USERNAME_KEY = "extractor.username"; + public static final String SOURCE_IOTDB_USERNAME_KEY = "source.username"; + public static final String EXTRACTOR_IOTDB_PASSWORD_KEY = "extractor.password"; + public static final String SOURCE_IOTDB_PASSWORD_KEY = "source.password"; + public static final String EXTRACTOR_INCLUSION_KEY = "extractor.inclusion"; public static final String SOURCE_INCLUSION_KEY = "source.inclusion"; public static final String EXTRACTOR_INCLUSION_DEFAULT_VALUE = "data.insert"; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerGroupMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerGroupMeta.java index b316c5f155de6..498f3427690f5 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerGroupMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerGroupMeta.java @@ -136,6 +136,10 @@ public boolean containsConsumer(final String consumerId) { return consumerIdToConsumerMeta.containsKey(consumerId); } + public ConsumerMeta getConsumerMeta(final String consumerId) { + return consumerIdToConsumerMeta.get(consumerId); + } + public boolean isEmpty() { // When there are no consumers in a consumer group, it means that the ConsumerGroupMeta is // empty, and at this time, the topicNameToSubscribedConsumerIdSet is also empty. diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerMeta.java index f1bb9b4608531..75cca9ccbcdd9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerMeta.java @@ -54,6 +54,26 @@ public String getConsumerId() { return consumerId; } + public String getConsumerGroupId() { + return config.getConsumerGroupId(); + } + + public String getUsername() { + return config.getUsername(); + } + + public String getPassword() { + return config.getPassword(); + } + + public String getEncryptedPassword() { + return config.getEncryptedPassword(); + } + + public String getSubscriptionAuthPassword() { + return Objects.nonNull(getEncryptedPassword()) ? getEncryptedPassword() : getPassword(); + } + public ByteBuffer serialize() throws IOException { PublicBAOS byteArrayOutputStream = new PublicBAOS(); DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java index 93cd87e4ae386..ee9177112e710 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/topic/TopicMeta.java @@ -181,11 +181,22 @@ public static TopicMeta deserialize(final ByteBuffer byteBuffer) { /////////////////////////////// utilities /////////////////////////////// public Map generateExtractorAttributes() { + return generateExtractorAttributes(null, null); + } + + public Map generateExtractorAttributes( + final String username, final String password) { final Map extractorAttributes = new HashMap<>(); // disable meta sync extractorAttributes.put("source", "iotdb-source"); extractorAttributes.put("inclusion", "data.insert"); extractorAttributes.put("inclusion.exclusion", "data.delete"); + if (Objects.nonNull(username)) { + extractorAttributes.put("username", username); + } + if (Objects.nonNull(password)) { + extractorAttributes.put("password", password); + } // path extractorAttributes.putAll(config.getAttributesWithPathOrPattern()); // time diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/subscription/topic/TopicDeSerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/subscription/topic/TopicDeSerTest.java index 4973f06d27cf2..d9c280e14938c 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/subscription/topic/TopicDeSerTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/subscription/topic/TopicDeSerTest.java @@ -54,4 +54,15 @@ public void test() throws IOException { Assert.assertEquals( topicMeta.getSubscribedConsumerGroupIds(), topicMeta2.getSubscribedConsumerGroupIds()); } + + @Test + public void testGenerateExtractorAttributesWithEncryptedPassword() { + final TopicMeta topicMeta = new TopicMeta("test_topic", 1, new HashMap<>()); + + final Map extractorAttributes = + topicMeta.generateExtractorAttributes("test_user", "encrypted-password"); + + Assert.assertEquals("test_user", extractorAttributes.get("username")); + Assert.assertEquals("encrypted-password", extractorAttributes.get("password")); + } } From 7363d35e09759c63d06fbeb2286804ddfd6de118 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 29 Apr 2026 10:53:20 +0800 Subject: [PATCH 016/102] [To dev/1.3] Pipe: Fixed the failed logic for compressing progressReportEvent (#17457) (#17557) --- .../dataregion/realtime/PipeRealtimeDataRegionSource.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/PipeRealtimeDataRegionSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/PipeRealtimeDataRegionSource.java index 3c03ee732d05c..367c83274215f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/PipeRealtimeDataRegionSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/PipeRealtimeDataRegionSource.java @@ -382,8 +382,11 @@ protected void extractProgressReportEvent(final PipeRealtimeEvent event) { } pendingQueue.pollLast(); } - if (pendingQueue.peekLast() instanceof ProgressReportEvent) { - final ProgressReportEvent oldEvent = (ProgressReportEvent) pendingQueue.peekLast(); + final Event last = pendingQueue.peekLast(); + if (last instanceof PipeRealtimeEvent + && ((PipeRealtimeEvent) last).getEvent() instanceof ProgressReportEvent) { + final ProgressReportEvent oldEvent = + (ProgressReportEvent) ((PipeRealtimeEvent) last).getEvent(); oldEvent.bindProgressIndex( oldEvent .getProgressIndex() From 7717d5190e338d47db0120d0e21ef22f2bbb7d28 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 29 Apr 2026 14:48:45 +0800 Subject: [PATCH 017/102] [To dev/1.3] Fixed the bug that session cannot query minus value when only pass paths (#17390) --- .../it/aligned/IoTDBAlignedLastQueryIT.java | 3 +- .../it/aligned/IoTDBAlignedSeriesQueryIT.java | 320 +++++++++--------- .../aligned/IoTDBLastQueryWithDeletionIT.java | 3 +- .../IoTDBLastQueryWithoutLastCacheIT.java | 3 +- ...QueryWithoutValueFilterWithDeletionIT.java | 84 ++--- .../iotdb/db/it/utils/AlignedWriteUtil.java | 2 + .../iotdb/session/it/IoTDBSessionQueryIT.java | 7 +- .../org/apache/iotdb/session/Session.java | 3 +- 8 files changed, 221 insertions(+), 204 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBAlignedLastQueryIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBAlignedLastQueryIT.java index 214a2c31806f4..c66de9e73d523 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBAlignedLastQueryIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBAlignedLastQueryIT.java @@ -124,7 +124,8 @@ public void selectAllAlignedAndNonAlignedLastTest() { "40,root.sg1.d2.s2,40,INT32", "30,root.sg1.d2.s3,30,INT64", "30,root.sg1.d2.s4,false,BOOLEAN", - "40,root.sg1.d2.s5,non_aligned_test40,TEXT")); + "40,root.sg1.d2.s5,non_aligned_test40,TEXT", + "-40,root.sg1.d2.s6,40.0,DOUBLE")); try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBAlignedSeriesQueryIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBAlignedSeriesQueryIT.java index 016e2d59daab7..a361083106896 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBAlignedSeriesQueryIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBAlignedSeriesQueryIT.java @@ -165,46 +165,47 @@ public void selectAllAlignedAndNonAlignedTest() { String[] retArray = new String[] { - "1,1.0,1,1,true,aligned_test1,1.0,1,1,true,non_aligned_test1", - "2,2.0,2,2,null,aligned_test2,2.0,2,2,null,non_aligned_test2", - "3,30000.0,null,30000,true,aligned_unseq_test3,3.0,null,3,false,non_aligned_test3", - "4,4.0,4,null,true,aligned_test4,4.0,4,null,true,non_aligned_test4", - "5,5.0,5,null,true,aligned_test5,5.0,5,null,true,non_aligned_test5", - "6,6.0,6,6,true,null,6.0,6,6,true,null", - "7,7.0,7,7,false,aligned_test7,7.0,7,7,false,non_aligned_test7", - "8,8.0,8,8,null,aligned_test8,8.0,8,8,null,non_aligned_test8", - "9,9.0,9,9,false,aligned_test9,9.0,9,9,false,non_aligned_test9", - "10,null,10,10,true,aligned_test10,null,10,10,true,non_aligned_test10", - "11,11.0,11,11,null,null,11.0,11,11,null,null", - "12,12.0,12,12,null,null,12.0,12,12,null,null", - "13,130000.0,130000,130000,true,aligned_unseq_test13,13.0,13,13,null,null", - "14,14.0,14,14,null,null,14.0,14,14,null,null", - "15,15.0,15,15,null,null,15.0,15,15,null,null", - "16,16.0,16,16,null,null,16.0,16,16,null,null", - "17,17.0,17,17,null,null,17.0,17,17,null,null", - "18,18.0,18,18,null,null,18.0,18,18,null,null", - "19,19.0,19,19,null,null,19.0,19,19,null,null", - "20,20.0,20,20,null,null,20.0,20,20,null,null", - "21,null,null,21,true,null,null,null,21,true,null", - "22,null,null,22,true,null,null,null,22,true,null", - "23,230000.0,null,230000,false,null,null,null,23,true,null", - "24,null,null,24,true,null,null,null,24,true,null", - "25,null,null,25,true,null,null,null,25,true,null", - "26,null,null,26,false,null,null,null,26,false,null", - "27,null,null,27,false,null,null,null,27,false,null", - "28,null,null,28,false,null,null,null,28,false,null", - "29,null,null,29,false,null,null,null,29,false,null", - "30,null,null,30,false,null,null,null,30,false,null", - "31,null,31,null,null,aligned_test31,null,31,null,null,non_aligned_test31", - "32,null,32,null,null,aligned_test32,null,32,null,null,non_aligned_test32", - "33,null,33,null,null,aligned_test33,null,33,null,null,non_aligned_test33", - "34,null,34,null,null,aligned_test34,null,34,null,null,non_aligned_test34", - "35,null,35,null,null,aligned_test35,null,35,null,null,non_aligned_test35", - "36,null,36,null,null,aligned_test36,null,36,null,null,non_aligned_test36", - "37,null,37,null,null,aligned_test37,null,37,null,null,non_aligned_test37", - "38,null,38,null,null,aligned_test38,null,38,null,null,non_aligned_test38", - "39,null,39,null,null,aligned_test39,null,39,null,null,non_aligned_test39", - "40,null,40,null,null,aligned_test40,null,40,null,null,non_aligned_test40", + "-40,null,null,null,null,null,null,null,null,null,null,40.0", + "1,1.0,1,1,true,aligned_test1,1.0,1,1,true,non_aligned_test1,null", + "2,2.0,2,2,null,aligned_test2,2.0,2,2,null,non_aligned_test2,null", + "3,30000.0,null,30000,true,aligned_unseq_test3,3.0,null,3,false,non_aligned_test3,null", + "4,4.0,4,null,true,aligned_test4,4.0,4,null,true,non_aligned_test4,null", + "5,5.0,5,null,true,aligned_test5,5.0,5,null,true,non_aligned_test5,null", + "6,6.0,6,6,true,null,6.0,6,6,true,null,null", + "7,7.0,7,7,false,aligned_test7,7.0,7,7,false,non_aligned_test7,null", + "8,8.0,8,8,null,aligned_test8,8.0,8,8,null,non_aligned_test8,null", + "9,9.0,9,9,false,aligned_test9,9.0,9,9,false,non_aligned_test9,null", + "10,null,10,10,true,aligned_test10,null,10,10,true,non_aligned_test10,null", + "11,11.0,11,11,null,null,11.0,11,11,null,null,null", + "12,12.0,12,12,null,null,12.0,12,12,null,null,null", + "13,130000.0,130000,130000,true,aligned_unseq_test13,13.0,13,13,null,null,null", + "14,14.0,14,14,null,null,14.0,14,14,null,null,null", + "15,15.0,15,15,null,null,15.0,15,15,null,null,null", + "16,16.0,16,16,null,null,16.0,16,16,null,null,null", + "17,17.0,17,17,null,null,17.0,17,17,null,null,null", + "18,18.0,18,18,null,null,18.0,18,18,null,null,null", + "19,19.0,19,19,null,null,19.0,19,19,null,null,null", + "20,20.0,20,20,null,null,20.0,20,20,null,null,null", + "21,null,null,21,true,null,null,null,21,true,null,null", + "22,null,null,22,true,null,null,null,22,true,null,null", + "23,230000.0,null,230000,false,null,null,null,23,true,null,null", + "24,null,null,24,true,null,null,null,24,true,null,null", + "25,null,null,25,true,null,null,null,25,true,null,null", + "26,null,null,26,false,null,null,null,26,false,null,null", + "27,null,null,27,false,null,null,null,27,false,null,null", + "28,null,null,28,false,null,null,null,28,false,null,null", + "29,null,null,29,false,null,null,null,29,false,null,null", + "30,null,null,30,false,null,null,null,30,false,null,null", + "31,null,31,null,null,aligned_test31,null,31,null,null,non_aligned_test31,null", + "32,null,32,null,null,aligned_test32,null,32,null,null,non_aligned_test32,null", + "33,null,33,null,null,aligned_test33,null,33,null,null,non_aligned_test33,null", + "34,null,34,null,null,aligned_test34,null,34,null,null,non_aligned_test34,null", + "35,null,35,null,null,aligned_test35,null,35,null,null,non_aligned_test35,null", + "36,null,36,null,null,aligned_test36,null,36,null,null,non_aligned_test36,null", + "37,null,37,null,null,aligned_test37,null,37,null,null,non_aligned_test37,null", + "38,null,38,null,null,aligned_test38,null,38,null,null,non_aligned_test38,null", + "39,null,39,null,null,aligned_test39,null,39,null,null,non_aligned_test39,null", + "40,null,40,null,null,aligned_test40,null,40,null,null,non_aligned_test40,null", }; String[] columnNames = { @@ -217,7 +218,8 @@ public void selectAllAlignedAndNonAlignedTest() { "root.sg1.d2.s2", "root.sg1.d2.s3", "root.sg1.d2.s4", - "root.sg1.d2.s5" + "root.sg1.d2.s5", + "root.sg1.d2.s6", }; try (Connection connection = EnvFactory.getEnv().getConnection(); @@ -766,11 +768,11 @@ public void selectAllAlignedAndNonAlignedTest1() { String[] retArray = new String[] { - "13,130000.0,130000,130000,true,aligned_unseq_test13,13.0,13,13,null,null", - "17,17.0,17,17,null,null,17.0,17,17,null,null", - "18,18.0,18,18,null,null,18.0,18,18,null,null", - "19,19.0,19,19,null,null,19.0,19,19,null,null", - "20,20.0,20,20,null,null,20.0,20,20,null,null", + "13,130000.0,130000,130000,true,aligned_unseq_test13,13.0,13,13,null,null,null", + "17,17.0,17,17,null,null,17.0,17,17,null,null,null", + "18,18.0,18,18,null,null,18.0,18,18,null,null,null", + "19,19.0,19,19,null,null,19.0,19,19,null,null,null", + "20,20.0,20,20,null,null,20.0,20,20,null,null,null", }; String[] columnNames = { @@ -783,7 +785,8 @@ public void selectAllAlignedAndNonAlignedTest1() { "root.sg1.d2.s2", "root.sg1.d2.s3", "root.sg1.d2.s4", - "root.sg1.d2.s5" + "root.sg1.d2.s5", + "root.sg1.d2.s6" }; try (Connection connection = EnvFactory.getEnv().getConnection(); @@ -823,29 +826,29 @@ public void selectAllAlignedAndNonAlignedTest2() { String[] retArray = new String[] { - "3,30000.0,null,30000,true,aligned_unseq_test3,3.0,null,3,false,non_aligned_test3", - "7,7.0,7,7,false,aligned_test7,7.0,7,7,false,non_aligned_test7", - "9,9.0,9,9,false,aligned_test9,9.0,9,9,false,non_aligned_test9", - "13,130000.0,130000,130000,true,aligned_unseq_test13,13.0,13,13,null,null", - "17,17.0,17,17,null,null,17.0,17,17,null,null", - "18,18.0,18,18,null,null,18.0,18,18,null,null", - "19,19.0,19,19,null,null,19.0,19,19,null,null", - "20,20.0,20,20,null,null,20.0,20,20,null,null", - "26,null,null,26,false,null,null,null,26,false,null", - "27,null,null,27,false,null,null,null,27,false,null", - "28,null,null,28,false,null,null,null,28,false,null", - "29,null,null,29,false,null,null,null,29,false,null", - "30,null,null,30,false,null,null,null,30,false,null", - "31,null,31,null,null,aligned_test31,null,31,null,null,non_aligned_test31", - "32,null,32,null,null,aligned_test32,null,32,null,null,non_aligned_test32", - "33,null,33,null,null,aligned_test33,null,33,null,null,non_aligned_test33", - "34,null,34,null,null,aligned_test34,null,34,null,null,non_aligned_test34", - "35,null,35,null,null,aligned_test35,null,35,null,null,non_aligned_test35", - "36,null,36,null,null,aligned_test36,null,36,null,null,non_aligned_test36", - "37,null,37,null,null,aligned_test37,null,37,null,null,non_aligned_test37", - "38,null,38,null,null,aligned_test38,null,38,null,null,non_aligned_test38", - "39,null,39,null,null,aligned_test39,null,39,null,null,non_aligned_test39", - "40,null,40,null,null,aligned_test40,null,40,null,null,non_aligned_test40", + "3,30000.0,null,30000,true,aligned_unseq_test3,3.0,null,3,false,non_aligned_test3,null", + "7,7.0,7,7,false,aligned_test7,7.0,7,7,false,non_aligned_test7,null", + "9,9.0,9,9,false,aligned_test9,9.0,9,9,false,non_aligned_test9,null", + "13,130000.0,130000,130000,true,aligned_unseq_test13,13.0,13,13,null,null,null", + "17,17.0,17,17,null,null,17.0,17,17,null,null,null", + "18,18.0,18,18,null,null,18.0,18,18,null,null,null", + "19,19.0,19,19,null,null,19.0,19,19,null,null,null", + "20,20.0,20,20,null,null,20.0,20,20,null,null,null", + "26,null,null,26,false,null,null,null,26,false,null,null", + "27,null,null,27,false,null,null,null,27,false,null,null", + "28,null,null,28,false,null,null,null,28,false,null,null", + "29,null,null,29,false,null,null,null,29,false,null,null", + "30,null,null,30,false,null,null,null,30,false,null,null", + "31,null,31,null,null,aligned_test31,null,31,null,null,non_aligned_test31,null", + "32,null,32,null,null,aligned_test32,null,32,null,null,non_aligned_test32,null", + "33,null,33,null,null,aligned_test33,null,33,null,null,non_aligned_test33,null", + "34,null,34,null,null,aligned_test34,null,34,null,null,non_aligned_test34,null", + "35,null,35,null,null,aligned_test35,null,35,null,null,non_aligned_test35,null", + "36,null,36,null,null,aligned_test36,null,36,null,null,non_aligned_test36,null", + "37,null,37,null,null,aligned_test37,null,37,null,null,non_aligned_test37,null", + "38,null,38,null,null,aligned_test38,null,38,null,null,non_aligned_test38,null", + "39,null,39,null,null,aligned_test39,null,39,null,null,non_aligned_test39,null", + "40,null,40,null,null,aligned_test40,null,40,null,null,non_aligned_test40,null", }; String[] columnNames = { @@ -858,7 +861,8 @@ public void selectAllAlignedAndNonAlignedTest2() { "root.sg1.d2.s2", "root.sg1.d2.s3", "root.sg1.d2.s4", - "root.sg1.d2.s5" + "root.sg1.d2.s5", + "root.sg1.d2.s6" }; try (Connection connection = EnvFactory.getEnv().getConnection(); @@ -1304,7 +1308,8 @@ public void countAllAlignedWithoutTimeFilterTest() { @Test public void countAllAlignedAndNonAlignedWithoutTimeFilterTest() { - String[] retArray = new String[] {"20", "29", "28", "19", "20", "19", "29", "28", "18", "19"}; + String[] retArray = + new String[] {"20", "29", "28", "19", "20", "19", "29", "28", "18", "19", "1"}; String[] columnNames = { "count(root.sg1.d1.s1)", "count(root.sg1.d1.s2)", @@ -1315,7 +1320,8 @@ public void countAllAlignedAndNonAlignedWithoutTimeFilterTest() { "count(root.sg1.d2.s2)", "count(root.sg1.d2.s3)", "count(root.sg1.d2.s4)", - "count(root.sg1.d2.s5)" + "count(root.sg1.d2.s5)", + "count(root.sg1.d2.s6)", }; try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { @@ -1850,89 +1856,90 @@ public void selectAllAlignedWithoutValueFilterAlignByDeviceTest() { public void selectAllAlignedAndNonAlignedAlignByDeviceTest() { String[] retArray = new String[] { - "1,root.sg1.d1,1.0,1,1,true,aligned_test1", - "2,root.sg1.d1,2.0,2,2,null,aligned_test2", - "3,root.sg1.d1,30000.0,null,30000,true,aligned_unseq_test3", - "4,root.sg1.d1,4.0,4,null,true,aligned_test4", - "5,root.sg1.d1,5.0,5,null,true,aligned_test5", - "6,root.sg1.d1,6.0,6,6,true,null", - "7,root.sg1.d1,7.0,7,7,false,aligned_test7", - "8,root.sg1.d1,8.0,8,8,null,aligned_test8", - "9,root.sg1.d1,9.0,9,9,false,aligned_test9", - "10,root.sg1.d1,null,10,10,true,aligned_test10", - "11,root.sg1.d1,11.0,11,11,null,null", - "12,root.sg1.d1,12.0,12,12,null,null", - "13,root.sg1.d1,130000.0,130000,130000,true,aligned_unseq_test13", - "14,root.sg1.d1,14.0,14,14,null,null", - "15,root.sg1.d1,15.0,15,15,null,null", - "16,root.sg1.d1,16.0,16,16,null,null", - "17,root.sg1.d1,17.0,17,17,null,null", - "18,root.sg1.d1,18.0,18,18,null,null", - "19,root.sg1.d1,19.0,19,19,null,null", - "20,root.sg1.d1,20.0,20,20,null,null", - "21,root.sg1.d1,null,null,21,true,null", - "22,root.sg1.d1,null,null,22,true,null", - "23,root.sg1.d1,230000.0,null,230000,false,null", - "24,root.sg1.d1,null,null,24,true,null", - "25,root.sg1.d1,null,null,25,true,null", - "26,root.sg1.d1,null,null,26,false,null", - "27,root.sg1.d1,null,null,27,false,null", - "28,root.sg1.d1,null,null,28,false,null", - "29,root.sg1.d1,null,null,29,false,null", - "30,root.sg1.d1,null,null,30,false,null", - "31,root.sg1.d1,null,31,null,null,aligned_test31", - "32,root.sg1.d1,null,32,null,null,aligned_test32", - "33,root.sg1.d1,null,33,null,null,aligned_test33", - "34,root.sg1.d1,null,34,null,null,aligned_test34", - "35,root.sg1.d1,null,35,null,null,aligned_test35", - "36,root.sg1.d1,null,36,null,null,aligned_test36", - "37,root.sg1.d1,null,37,null,null,aligned_test37", - "38,root.sg1.d1,null,38,null,null,aligned_test38", - "39,root.sg1.d1,null,39,null,null,aligned_test39", - "40,root.sg1.d1,null,40,null,null,aligned_test40", - "1,root.sg1.d2,1.0,1,1,true,non_aligned_test1", - "2,root.sg1.d2,2.0,2,2,null,non_aligned_test2", - "3,root.sg1.d2,3.0,null,3,false,non_aligned_test3", - "4,root.sg1.d2,4.0,4,null,true,non_aligned_test4", - "5,root.sg1.d2,5.0,5,null,true,non_aligned_test5", - "6,root.sg1.d2,6.0,6,6,true,null", - "7,root.sg1.d2,7.0,7,7,false,non_aligned_test7", - "8,root.sg1.d2,8.0,8,8,null,non_aligned_test8", - "9,root.sg1.d2,9.0,9,9,false,non_aligned_test9", - "10,root.sg1.d2,null,10,10,true,non_aligned_test10", - "11,root.sg1.d2,11.0,11,11,null,null", - "12,root.sg1.d2,12.0,12,12,null,null", - "13,root.sg1.d2,13.0,13,13,null,null", - "14,root.sg1.d2,14.0,14,14,null,null", - "15,root.sg1.d2,15.0,15,15,null,null", - "16,root.sg1.d2,16.0,16,16,null,null", - "17,root.sg1.d2,17.0,17,17,null,null", - "18,root.sg1.d2,18.0,18,18,null,null", - "19,root.sg1.d2,19.0,19,19,null,null", - "20,root.sg1.d2,20.0,20,20,null,null", - "21,root.sg1.d2,null,null,21,true,null", - "22,root.sg1.d2,null,null,22,true,null", - "23,root.sg1.d2,null,null,23,true,null", - "24,root.sg1.d2,null,null,24,true,null", - "25,root.sg1.d2,null,null,25,true,null", - "26,root.sg1.d2,null,null,26,false,null", - "27,root.sg1.d2,null,null,27,false,null", - "28,root.sg1.d2,null,null,28,false,null", - "29,root.sg1.d2,null,null,29,false,null", - "30,root.sg1.d2,null,null,30,false,null", - "31,root.sg1.d2,null,31,null,null,non_aligned_test31", - "32,root.sg1.d2,null,32,null,null,non_aligned_test32", - "33,root.sg1.d2,null,33,null,null,non_aligned_test33", - "34,root.sg1.d2,null,34,null,null,non_aligned_test34", - "35,root.sg1.d2,null,35,null,null,non_aligned_test35", - "36,root.sg1.d2,null,36,null,null,non_aligned_test36", - "37,root.sg1.d2,null,37,null,null,non_aligned_test37", - "38,root.sg1.d2,null,38,null,null,non_aligned_test38", - "39,root.sg1.d2,null,39,null,null,non_aligned_test39", - "40,root.sg1.d2,null,40,null,null,non_aligned_test40", + "1,root.sg1.d1,1.0,1,1,true,aligned_test1,null", + "2,root.sg1.d1,2.0,2,2,null,aligned_test2,null", + "3,root.sg1.d1,30000.0,null,30000,true,aligned_unseq_test3,null", + "4,root.sg1.d1,4.0,4,null,true,aligned_test4,null", + "5,root.sg1.d1,5.0,5,null,true,aligned_test5,null", + "6,root.sg1.d1,6.0,6,6,true,null,null", + "7,root.sg1.d1,7.0,7,7,false,aligned_test7,null", + "8,root.sg1.d1,8.0,8,8,null,aligned_test8,null", + "9,root.sg1.d1,9.0,9,9,false,aligned_test9,null", + "10,root.sg1.d1,null,10,10,true,aligned_test10,null", + "11,root.sg1.d1,11.0,11,11,null,null,null", + "12,root.sg1.d1,12.0,12,12,null,null,null", + "13,root.sg1.d1,130000.0,130000,130000,true,aligned_unseq_test13,null", + "14,root.sg1.d1,14.0,14,14,null,null,null", + "15,root.sg1.d1,15.0,15,15,null,null,null", + "16,root.sg1.d1,16.0,16,16,null,null,null", + "17,root.sg1.d1,17.0,17,17,null,null,null", + "18,root.sg1.d1,18.0,18,18,null,null,null", + "19,root.sg1.d1,19.0,19,19,null,null,null", + "20,root.sg1.d1,20.0,20,20,null,null,null", + "21,root.sg1.d1,null,null,21,true,null,null", + "22,root.sg1.d1,null,null,22,true,null,null", + "23,root.sg1.d1,230000.0,null,230000,false,null,null", + "24,root.sg1.d1,null,null,24,true,null,null", + "25,root.sg1.d1,null,null,25,true,null,null", + "26,root.sg1.d1,null,null,26,false,null,null", + "27,root.sg1.d1,null,null,27,false,null,null", + "28,root.sg1.d1,null,null,28,false,null,null", + "29,root.sg1.d1,null,null,29,false,null,null", + "30,root.sg1.d1,null,null,30,false,null,null", + "31,root.sg1.d1,null,31,null,null,aligned_test31,null", + "32,root.sg1.d1,null,32,null,null,aligned_test32,null", + "33,root.sg1.d1,null,33,null,null,aligned_test33,null", + "34,root.sg1.d1,null,34,null,null,aligned_test34,null", + "35,root.sg1.d1,null,35,null,null,aligned_test35,null", + "36,root.sg1.d1,null,36,null,null,aligned_test36,null", + "37,root.sg1.d1,null,37,null,null,aligned_test37,null", + "38,root.sg1.d1,null,38,null,null,aligned_test38,null", + "39,root.sg1.d1,null,39,null,null,aligned_test39,null", + "40,root.sg1.d1,null,40,null,null,aligned_test40,null", + "-40,root.sg1.d2,null,null,null,null,null,40.0", + "1,root.sg1.d2,1.0,1,1,true,non_aligned_test1,null", + "2,root.sg1.d2,2.0,2,2,null,non_aligned_test2,null", + "3,root.sg1.d2,3.0,null,3,false,non_aligned_test3,null", + "4,root.sg1.d2,4.0,4,null,true,non_aligned_test4,null", + "5,root.sg1.d2,5.0,5,null,true,non_aligned_test5,null", + "6,root.sg1.d2,6.0,6,6,true,null,null", + "7,root.sg1.d2,7.0,7,7,false,non_aligned_test7,null", + "8,root.sg1.d2,8.0,8,8,null,non_aligned_test8,null", + "9,root.sg1.d2,9.0,9,9,false,non_aligned_test9,null", + "10,root.sg1.d2,null,10,10,true,non_aligned_test10,null", + "11,root.sg1.d2,11.0,11,11,null,null,null", + "12,root.sg1.d2,12.0,12,12,null,null,null", + "13,root.sg1.d2,13.0,13,13,null,null,null", + "14,root.sg1.d2,14.0,14,14,null,null,null", + "15,root.sg1.d2,15.0,15,15,null,null,null", + "16,root.sg1.d2,16.0,16,16,null,null,null", + "17,root.sg1.d2,17.0,17,17,null,null,null", + "18,root.sg1.d2,18.0,18,18,null,null,null", + "19,root.sg1.d2,19.0,19,19,null,null,null", + "20,root.sg1.d2,20.0,20,20,null,null,null", + "21,root.sg1.d2,null,null,21,true,null,null", + "22,root.sg1.d2,null,null,22,true,null,null", + "23,root.sg1.d2,null,null,23,true,null,null", + "24,root.sg1.d2,null,null,24,true,null,null", + "25,root.sg1.d2,null,null,25,true,null,null", + "26,root.sg1.d2,null,null,26,false,null,null", + "27,root.sg1.d2,null,null,27,false,null,null", + "28,root.sg1.d2,null,null,28,false,null,null", + "29,root.sg1.d2,null,null,29,false,null,null", + "30,root.sg1.d2,null,null,30,false,null,null", + "31,root.sg1.d2,null,31,null,null,non_aligned_test31,null", + "32,root.sg1.d2,null,32,null,null,non_aligned_test32,null", + "33,root.sg1.d2,null,33,null,null,non_aligned_test33,null", + "34,root.sg1.d2,null,34,null,null,non_aligned_test34,null", + "35,root.sg1.d2,null,35,null,null,non_aligned_test35,null", + "36,root.sg1.d2,null,36,null,null,non_aligned_test36,null", + "37,root.sg1.d2,null,37,null,null,non_aligned_test37,null", + "38,root.sg1.d2,null,38,null,null,non_aligned_test38,null", + "39,root.sg1.d2,null,39,null,null,non_aligned_test39,null", + "40,root.sg1.d2,null,40,null,null,non_aligned_test40,null", }; - String[] columnNames = {"Device", "s1", "s2", "s3", "s4", "s5"}; + String[] columnNames = {"Device", "s1", "s2", "s3", "s4", "s5", "s6"}; try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { @@ -2518,9 +2525,10 @@ public void countAllAlignedWithoutTimeFilterAlignByDeviceTest() { @Test public void countAllAlignedAndNonAlignedWithoutTimeFilterAlignByDeviceTest() { - String[] retArray = new String[] {"root.sg1.d1,20,29,28,19,20,", "root.sg1.d2,19,29,28,18,19,"}; + String[] retArray = + new String[] {"root.sg1.d1,20,29,28,19,20,null,", "root.sg1.d2,19,29,28,18,19,1,"}; String[] columnNames = { - "Device", "count(s1)", "count(s2)", "count(s3)", "count(s4)", "count(s5)" + "Device", "count(s1)", "count(s2)", "count(s3)", "count(s4)", "count(s5)", "count(s6)" }; try (Connection connection = EnvFactory.getEnv().getConnection(); diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBLastQueryWithDeletionIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBLastQueryWithDeletionIT.java index f02e2fd6ec8ae..cd2ce061c0031 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBLastQueryWithDeletionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBLastQueryWithDeletionIT.java @@ -125,7 +125,8 @@ public void selectAllAlignedAndNonAlignedLastTest() { "40,root.sg1.d2.s2,40,INT32", "30,root.sg1.d2.s3,30,INT64", "30,root.sg1.d2.s4,false,BOOLEAN", - "40,root.sg1.d2.s5,non_aligned_test40,TEXT")); + "40,root.sg1.d2.s5,non_aligned_test40,TEXT", + "-40,root.sg1.d2.s6,40.0,DOUBLE")); try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement(); diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBLastQueryWithoutLastCacheIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBLastQueryWithoutLastCacheIT.java index 25b6a2098c9c3..b9b8cd24bc17f 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBLastQueryWithoutLastCacheIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBLastQueryWithoutLastCacheIT.java @@ -122,7 +122,8 @@ public void selectAllAlignedAndNonAlignedLastTest() { "40,root.sg1.d2.s2,40,INT32", "30,root.sg1.d2.s3,30,INT64", "30,root.sg1.d2.s4,false,BOOLEAN", - "40,root.sg1.d2.s5,non_aligned_test40,TEXT")); + "40,root.sg1.d2.s5,non_aligned_test40,TEXT", + "-40,root.sg1.d2.s6,40.0,DOUBLE")); try (Connection connection = EnvFactory.getEnv().getConnection(); Statement statement = connection.createStatement()) { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBRawQueryWithoutValueFilterWithDeletionIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBRawQueryWithoutValueFilterWithDeletionIT.java index 5bd02dbacacc8..67787a8ddd233 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBRawQueryWithoutValueFilterWithDeletionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/aligned/IoTDBRawQueryWithoutValueFilterWithDeletionIT.java @@ -155,46 +155,47 @@ public void selectAllAlignedAndNonAlignedTest() { String[] retArray = new String[] { - "1,null,1,true,aligned_test1,1.0,1,1,true,non_aligned_test1", - "2,null,2,null,aligned_test2,2.0,2,2,null,non_aligned_test2", - "3,null,30000,true,aligned_unseq_test3,3.0,null,3,false,non_aligned_test3", - "4,null,null,true,aligned_test4,4.0,4,null,true,non_aligned_test4", - "5,null,null,true,aligned_test5,5.0,5,null,true,non_aligned_test5", - "6,null,6,true,null,6.0,6,6,true,null", - "7,null,7,false,aligned_test7,7.0,7,7,false,non_aligned_test7", - "8,null,8,null,aligned_test8,8.0,8,8,null,non_aligned_test8", - "9,null,9,false,aligned_test9,9.0,9,9,false,non_aligned_test9", - "10,null,10,true,aligned_test10,null,10,10,true,non_aligned_test10", - "11,null,11,null,null,11.0,11,11,null,null", - "12,null,12,null,null,12.0,12,12,null,null", - "13,null,130000,true,aligned_unseq_test13,13.0,13,13,null,null", - "14,null,14,null,null,14.0,14,14,null,null", - "15,null,15,null,null,15.0,15,15,null,null", - "16,null,16,null,null,16.0,16,16,null,null", - "17,null,17,null,null,17.0,17,17,null,null", - "18,null,18,null,null,18.0,18,18,null,null", - "19,null,19,null,null,19.0,19,19,null,null", - "20,null,20,null,null,20.0,20,20,null,null", - "21,null,21,true,null,null,null,21,true,null", - "22,null,22,true,null,null,null,22,true,null", - "23,230000.0,230000,false,null,null,null,23,true,null", - "24,null,24,true,null,null,null,24,true,null", - "25,null,25,true,null,null,null,25,true,null", - "26,null,26,false,null,null,null,26,false,null", - "27,null,27,false,null,null,null,27,false,null", - "28,null,28,false,null,null,null,28,false,null", - "29,null,29,false,null,null,null,29,false,null", - "30,null,30,false,null,null,null,30,false,null", - "31,null,null,null,null,null,31,null,null,non_aligned_test31", - "32,null,null,null,aligned_test32,null,32,null,null,non_aligned_test32", - "33,null,null,null,aligned_test33,null,33,null,null,non_aligned_test33", - "34,null,null,null,aligned_test34,null,34,null,null,non_aligned_test34", - "35,null,null,null,aligned_test35,null,35,null,null,non_aligned_test35", - "36,null,null,null,aligned_test36,null,36,null,null,non_aligned_test36", - "37,null,null,null,aligned_test37,null,37,null,null,non_aligned_test37", - "38,null,null,null,aligned_test38,null,38,null,null,non_aligned_test38", - "39,null,null,null,aligned_test39,null,39,null,null,non_aligned_test39", - "40,null,null,null,aligned_test40,null,40,null,null,non_aligned_test40", + "-40,null,null,null,null,null,null,null,null,null,40.0", + "1,null,1,true,aligned_test1,1.0,1,1,true,non_aligned_test1,null", + "2,null,2,null,aligned_test2,2.0,2,2,null,non_aligned_test2,null", + "3,null,30000,true,aligned_unseq_test3,3.0,null,3,false,non_aligned_test3,null", + "4,null,null,true,aligned_test4,4.0,4,null,true,non_aligned_test4,null", + "5,null,null,true,aligned_test5,5.0,5,null,true,non_aligned_test5,null", + "6,null,6,true,null,6.0,6,6,true,null,null", + "7,null,7,false,aligned_test7,7.0,7,7,false,non_aligned_test7,null", + "8,null,8,null,aligned_test8,8.0,8,8,null,non_aligned_test8,null", + "9,null,9,false,aligned_test9,9.0,9,9,false,non_aligned_test9,null", + "10,null,10,true,aligned_test10,null,10,10,true,non_aligned_test10,null", + "11,null,11,null,null,11.0,11,11,null,null,null", + "12,null,12,null,null,12.0,12,12,null,null,null", + "13,null,130000,true,aligned_unseq_test13,13.0,13,13,null,null,null", + "14,null,14,null,null,14.0,14,14,null,null,null", + "15,null,15,null,null,15.0,15,15,null,null,null", + "16,null,16,null,null,16.0,16,16,null,null,null", + "17,null,17,null,null,17.0,17,17,null,null,null", + "18,null,18,null,null,18.0,18,18,null,null,null", + "19,null,19,null,null,19.0,19,19,null,null,null", + "20,null,20,null,null,20.0,20,20,null,null,null", + "21,null,21,true,null,null,null,21,true,null,null", + "22,null,22,true,null,null,null,22,true,null,null", + "23,230000.0,230000,false,null,null,null,23,true,null,null", + "24,null,24,true,null,null,null,24,true,null,null", + "25,null,25,true,null,null,null,25,true,null,null", + "26,null,26,false,null,null,null,26,false,null,null", + "27,null,27,false,null,null,null,27,false,null,null", + "28,null,28,false,null,null,null,28,false,null,null", + "29,null,29,false,null,null,null,29,false,null,null", + "30,null,30,false,null,null,null,30,false,null,null", + "31,null,null,null,null,null,31,null,null,non_aligned_test31,null", + "32,null,null,null,aligned_test32,null,32,null,null,non_aligned_test32,null", + "33,null,null,null,aligned_test33,null,33,null,null,non_aligned_test33,null", + "34,null,null,null,aligned_test34,null,34,null,null,non_aligned_test34,null", + "35,null,null,null,aligned_test35,null,35,null,null,non_aligned_test35,null", + "36,null,null,null,aligned_test36,null,36,null,null,non_aligned_test36,null", + "37,null,null,null,aligned_test37,null,37,null,null,non_aligned_test37,null", + "38,null,null,null,aligned_test38,null,38,null,null,non_aligned_test38,null", + "39,null,null,null,aligned_test39,null,39,null,null,non_aligned_test39,null", + "40,null,null,null,aligned_test40,null,40,null,null,non_aligned_test40,null", }; String[] columnNames = { @@ -206,7 +207,8 @@ public void selectAllAlignedAndNonAlignedTest() { "root.sg1.d2.s2", "root.sg1.d2.s3", "root.sg1.d2.s4", - "root.sg1.d2.s5" + "root.sg1.d2.s5", + "root.sg1.d2.s6" }; try (Connection connection = EnvFactory.getEnv().getConnection(); diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/utils/AlignedWriteUtil.java b/integration-test/src/test/java/org/apache/iotdb/db/it/utils/AlignedWriteUtil.java index 798f1b2b889d4..b0cee529f0da0 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/utils/AlignedWriteUtil.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/utils/AlignedWriteUtil.java @@ -44,6 +44,7 @@ public class AlignedWriteUtil { "create timeseries root.sg1.d2.s3 WITH DATATYPE=INT64", "create timeseries root.sg1.d2.s4 WITH DATATYPE=BOOLEAN", "create timeseries root.sg1.d2.s5 WITH DATATYPE=TEXT", + "create timeseries root.sg1.d2.s6 WITH DATATYPE=DOUBLE", "insert into root.sg1.d1(time, s1, s2, s3, s4, s5) aligned values(1, 1.0, 1, 1, TRUE, 'aligned_test1')", "insert into root.sg1.d1(time, s1, s2, s3, s5) aligned values(2, 2.0, 2, 2, 'aligned_test2')", "insert into root.sg1.d1(time, s1, s3, s4, s5) aligned values(3, 3.0, 3, FALSE, 'aligned_test3')", @@ -130,6 +131,7 @@ public class AlignedWriteUtil { "insert into root.sg1.d2(time, s2, s5) values(38, 38, 'non_aligned_test38')", "insert into root.sg1.d2(time, s2, s5) values(39, 39, 'non_aligned_test39')", "insert into root.sg1.d2(time, s2, s5) values(40, 40, 'non_aligned_test40')", + "insert into root.sg1.d2(time, s6) values(-40, 40)", }; public static void insertData() { diff --git a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionQueryIT.java b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionQueryIT.java index 7ade93490a79d..30544f4f8dc57 100644 --- a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionQueryIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionQueryIT.java @@ -153,10 +153,13 @@ public void rawDataQueryWithTimeRangeTest2() throws IoTDBConnectionException { @Test public void lastQueryTest() throws IoTDBConnectionException { - String[] retArray = new String[] {"23,root.sg1.d1.s1,230000.0,FLOAT"}; + Set retArray = + new HashSet<>( + Arrays.asList("-40,root.sg1.d2.s6,40.0,DOUBLE", "23,root.sg1.d1.s1,230000.0,FLOAT")); - List selectedPaths = Collections.singletonList("root.sg1.d1.s1"); + List selectedPaths = Arrays.asList("root.sg1.d1.s1", "root.sg1.d2.s6"); + // Does not guarantee sequence try (ISession session = EnvFactory.getEnv().getSessionConnection()) { try (SessionDataSet resultSet = session.executeLastDataQuery(selectedPaths)) { assertResultSetEqual(resultSet, lastQueryColumnNames, retArray, true); diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java index ff7c927871d94..461aef3624511 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java @@ -1042,8 +1042,7 @@ public SessionDataSet executeLastDataQuery(List paths, long lastTime, lo @Override public SessionDataSet executeLastDataQuery(List paths) throws StatementExecutionException, IoTDBConnectionException { - long time = 0L; - return executeLastDataQuery(paths, time, queryTimeoutInMs); + return executeLastDataQuery(paths, Long.MIN_VALUE, queryTimeoutInMs); } @Override From e1a6847aef4885ab7e8aa0052104b31e0708ee4b Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 29 Apr 2026 14:55:16 +0800 Subject: [PATCH 018/102] [To dev/1.3] Fixed multiple bugs related to schema (#17405) (#17420) --- .../rescon/MemSchemaRegionStatistics.java | 2 +- .../schemaregion/SchemaRegionPlanType.java | 1 + .../schemaregion/impl/SchemaRegionMemoryImpl.java | 4 ++-- .../mnode/iterator/AbstractTraverserIterator.java | 3 +++ .../mnode/iterator/MemoryTraverserIterator.java | 14 ++++++++++---- .../mnode/iterator/CachedTraverserIterator.java | 7 +++++++ .../impl/pbtree/schemafile/WrappedSegment.java | 12 +++++------- .../schemaRegion/SchemaRegionManagementTest.java | 12 ++++++++++++ .../iotdb/commons/schema/filter/SchemaFilter.java | 7 ++++++- .../apache/iotdb/commons/schema/ttl/TTLCache.java | 3 +++ 10 files changed, 50 insertions(+), 15 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaRegionStatistics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaRegionStatistics.java index be960b3807b63..a3a833f06abf6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaRegionStatistics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/rescon/MemSchemaRegionStatistics.java @@ -168,7 +168,7 @@ public void clear() { memoryUsage.getAndSet(0); measurementNumber.getAndSet(0); devicesNumber.getAndSet(0); - viewNumber.getAndAdd(0); + viewNumber.getAndSet(0); templateUsage.forEach( (templateId, cnt) -> schemaEngineStatistics.deactivateTemplate(templateId, cnt)); templateUsage.clear(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanType.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanType.java index 09a69f2b4f699..41ec8e649c599 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanType.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/SchemaRegionPlanType.java @@ -39,6 +39,7 @@ public enum SchemaRegionPlanType { ACTIVATE_TEMPLATE_IN_CLUSTER((byte) 63), PRE_DELETE_TIMESERIES_IN_CLUSTER((byte) 64), ROLLBACK_PRE_DELETE_TIMESERIES((byte) 65), + // endregion PRE_DEACTIVATE_TEMPLATE((byte) 0), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java index 6fc556a1089da..a55de193914ef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java @@ -857,8 +857,8 @@ public void createLogicalView(ICreateLogicalViewPlan plan) throws MetadataExcept regionStatistics.getGlobalMemoryUsage(), regionStatistics.getGlobalSeriesNumber()); } + final List pathList = plan.getViewPathList(); try { - List pathList = plan.getViewPathList(); Map viewPathToSourceMap = plan.getViewPathToSourceExpressionMap(); for (PartialPath path : pathList) { @@ -873,7 +873,7 @@ public void createLogicalView(ICreateLogicalViewPlan plan) throws MetadataExcept throw new RuntimeException(e); } // update statistics - regionStatistics.addView(1L); + regionStatistics.addView(pathList.size()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/iterator/AbstractTraverserIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/iterator/AbstractTraverserIterator.java index f7f6d7d1ae65f..e9aa30333c7db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/iterator/AbstractTraverserIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/iterator/AbstractTraverserIterator.java @@ -107,12 +107,15 @@ public boolean hasNext() { if (skipPreDeletedSchema && nextMatchedNode.isMeasurement() && nextMatchedNode.getAsMeasurementMNode().isPreDeleted()) { + releaseSkippedNode(nextMatchedNode); nextMatchedNode = null; } } return true; } + protected abstract void releaseSkippedNode(final N node); + @Override public N next() { if (!hasNext()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/iterator/MemoryTraverserIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/iterator/MemoryTraverserIterator.java index 15eb95df4ab90..684b398c47249 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/iterator/MemoryTraverserIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/iterator/MemoryTraverserIterator.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.mem.mnode.iterator; import org.apache.iotdb.commons.exception.MetadataException; @@ -30,11 +31,16 @@ // only use for IConfigMNode and IMemMNode public class MemoryTraverserIterator> extends AbstractTraverserIterator { public MemoryTraverserIterator( - IMTreeStore store, - IDeviceMNode parent, - Map templateMap, - IMNodeFactory nodeFactory) + final IMTreeStore store, + final IDeviceMNode parent, + final Map templateMap, + final IMNodeFactory nodeFactory) throws MetadataException { super(store, parent, templateMap, nodeFactory); } + + @Override + protected void releaseSkippedNode(final N node) { + // Do nothing + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/iterator/CachedTraverserIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/iterator/CachedTraverserIterator.java index 0d14e2251c4d5..0654c93b834ff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/iterator/CachedTraverserIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/iterator/CachedTraverserIterator.java @@ -48,4 +48,11 @@ public void close() { } super.close(); } + + @Override + protected void releaseSkippedNode(final ICachedMNode node) { + if (usingDirectChildrenIterator) { + store.unPin(node); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/schemafile/WrappedSegment.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/schemafile/WrappedSegment.java index e46f1aea0251c..65e1edebcf999 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/schemafile/WrappedSegment.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/schemafile/WrappedSegment.java @@ -105,17 +105,16 @@ public WrappedSegment(ByteBuffer buffer, boolean override) { } } - public WrappedSegment(ByteBuffer buffer) throws RecordDuplicatedException { + public WrappedSegment(ByteBuffer buffer) { this(buffer, true); } @TestOnly - public WrappedSegment(int size) throws RecordDuplicatedException { + public WrappedSegment(int size) { this(ByteBuffer.allocate(size)); } - public static ISegment initAsSegment(ByteBuffer buffer) - throws RecordDuplicatedException { + public static ISegment initAsSegment(ByteBuffer buffer) { if (buffer == null) { return null; } @@ -559,7 +558,6 @@ public Queue getAllRecords() throws MetadataException { @Override public int updateRecord(String key, ByteBuffer uBuffer) throws MetadataException { - int idx = binarySearchOnKeys(key); if (idx < 0) { throw new MetadataException(String.format("Record[key:%s] Not Existed.", key)); @@ -614,8 +612,8 @@ public int removeRecord(String key) { } // shift offsets forward - if (idx != recordNum) { - int shift = recordNum - idx; + if (idx != recordNum - 1) { + int shift = recordNum - idx - 1; this.buffer.position(SchemaFileConfig.SEG_HEADER_SIZE + idx * 2); ShortBuffer lb = this.buffer.asReadOnlyBuffer().asShortBuffer(); lb.get(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionManagementTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionManagementTest.java index d027871121910..2d79a6a4766c8 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionManagementTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionManagementTest.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion; +import org.apache.iotdb.db.schemaengine.schemaregion.SchemaRegionPlanType; import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ISchemaInfo; import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ITimeSeriesSchemaInfo; import org.apache.iotdb.db.schemaengine.schemaregion.write.req.SchemaRegionWritePlanFactory; @@ -40,6 +41,7 @@ import org.junit.Test; import java.io.File; +import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -229,4 +231,14 @@ public void testSnapshotPerformance() throws Exception { config.setSchemaRegionConsensusProtocolClass(schemaRegionConsensusProtocolClass); } } + + @Test + public void testSchemaRegionPlanType() throws Exception { + Assert.assertEquals( + SchemaRegionPlanType.values().length, + Arrays.stream(SchemaRegionPlanType.values()) + .map(SchemaRegionPlanType::getPlanType) + .distinct() + .count()); + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/filter/SchemaFilter.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/filter/SchemaFilter.java index a9e5fea0b73ca..d59e59ba2bc88 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/filter/SchemaFilter.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/filter/SchemaFilter.java @@ -91,7 +91,12 @@ public static List extract(SchemaFilter schemaFilter, SchemaFilter } private static void internalExtract( - List result, SchemaFilter schemaFilter, SchemaFilterType filterType) { + final List result, + final SchemaFilter schemaFilter, + final SchemaFilterType filterType) { + if (schemaFilter == null) { + return; + } if (schemaFilter.getSchemaFilterType().equals(filterType)) { result.add(schemaFilter); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/ttl/TTLCache.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/ttl/TTLCache.java index b3a3e3a891cd7..59f4653f0ae1f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/ttl/TTLCache.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/ttl/TTLCache.java @@ -335,6 +335,9 @@ public void deserialize(InputStream bufferedInputStream) public void clear() { ttlCacheTree.removeAllChildren(); ttlCacheTree.addChild(IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD, Long.MAX_VALUE); + + // root.** + ttlCount = 1; } static class CacheNode { From 7da19b9e75675b0ab22e7167117ade51f7570ca7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 29 Apr 2026 15:26:52 +0800 Subject: [PATCH 019/102] [To dev/1.3] Pipe: Fixed the bug that a realtime only pipe will not transfer historical data when it is altered to a historical pipe (#17223) (#17226) --- .../pipe/it/autocreate/IoTDBPipeAlterIT.java | 45 +++++++++++++++++++ .../impl/pipe/task/AlterPipeProcedureV2.java | 8 +++- 2 files changed, 52 insertions(+), 1 deletion(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAlterIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAlterIT.java index 3e4200daf323a..b964415938650 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAlterIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAlterIT.java @@ -534,4 +534,49 @@ public void testAlterPipeSourceAndProcessor() { TestUtils.assertDataEventuallyOnEnv( receiverEnv, "count timeSeries", "count(timeseries),", Collections.singleton("3,")); } + + @Test + public void testAlterPipeRealtime() { + final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); + + // Insert data on sender + TestUtils.executeNonQueries( + senderEnv, + Arrays.asList( + "insert into root.db.d1 (time, at1) values (1000, 1), (1500, 2), (2000, 3), (2500, 4), (3000, 5)", + "flush"), + null); + + // Create pipe + final String sql = + String.format( + "create pipe a2b with source ('history.enable'='false') with sink ('node-urls'='%s')", + receiverDataNode.getIpAndPortString()); + + try (final Connection connection = senderEnv.getConnection(); + final Statement statement = connection.createStatement()) { + statement.execute(sql); + } catch (final SQLException e) { + fail(e.getMessage()); + } + + TestUtils.assertDataAlwaysOnEnv( + receiverEnv, + "count timeSeries root.db.**", + "count(timeseries),", + Collections.singleton("0,")); + + try (final Connection connection = senderEnv.getConnection(); + final Statement statement = connection.createStatement()) { + statement.execute("alter pipe a2b modify source ('history.enable'='true')"); + } catch (final SQLException e) { + fail(e.getMessage()); + } + + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "count timeSeries root.db.**", + "count(timeseries),", + Collections.singleton("1,")); + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/AlterPipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/AlterPipeProcedureV2.java index 5a7b9b8a5b262..dcebb0b3d33e9 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/AlterPipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/AlterPipeProcedureV2.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.agent.task.PipeTaskAgent; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeMeta; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeRuntimeMeta; @@ -169,7 +170,12 @@ public void executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) { updatedConsensusGroupIdToTaskMetaMap.put( regionGroupId.getId(), new PipeTaskMeta( - currentPipeTaskMeta.getProgressIndex(), + PipeTaskAgent.isRealtimeOnlyPipe( + currentPipeStaticMeta.getExtractorParameters()) + && !PipeTaskAgent.isRealtimeOnlyPipe( + updatedPipeStaticMeta.getExtractorParameters()) + ? MinimumProgressIndex.INSTANCE + : currentPipeTaskMeta.getProgressIndex(), PipeTaskMeta.isNewlyAdded(currentPipeTaskMeta.getLeaderNodeId()) && !(!PipeTaskAgent.isHistoryOnlyPipe( currentPipeStaticMeta.getExtractorParameters()) From c40b8780df70354162702e05f029a13b1d0db4f0 Mon Sep 17 00:00:00 2001 From: Yunxiang Su Date: Thu, 30 Apr 2026 09:09:00 +0800 Subject: [PATCH 020/102] [To del/1.3] Fix Percentile, Quantile UDFs and update Cluster UDF (#17435) --- .../iotdb/libudf/it/dlearn/DLearnIT.java | 82 +++++++ .../src/assembly/tools/register-UDF.bat | 1 + .../src/assembly/tools/register-UDF.sh | 1 + .../iotdb/library/dlearn/UDTFCluster.java | 216 ++++++++++++++++++ .../dlearn/util/cluster/ClusterUtils.java | 156 +++++++++++++ .../library/dlearn/util/cluster/KMeans.java | 130 +++++++++++ .../library/dlearn/util/cluster/KShape.java | 190 +++++++++++++++ .../dlearn/util/cluster/MedoidShape.java | 213 +++++++++++++++++ .../iotdb/library/dprofile/UDAFQuantile.java | 78 +++---- .../dprofile/util/ExactOrderStatistics.java | 53 ++--- .../iotdb/library/dprofile/util/GKArray.java | 15 +- .../iotdb/library/match/PatternExecutor.java | 9 +- 12 files changed, 1066 insertions(+), 78 deletions(-) create mode 100644 library-udf/src/main/java/org/apache/iotdb/library/dlearn/UDTFCluster.java create mode 100644 library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/ClusterUtils.java create mode 100644 library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/KMeans.java create mode 100644 library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/KShape.java create mode 100644 library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/MedoidShape.java diff --git a/integration-test/src/test/java/org/apache/iotdb/libudf/it/dlearn/DLearnIT.java b/integration-test/src/test/java/org/apache/iotdb/libudf/it/dlearn/DLearnIT.java index 480dff4e46dca..b4a1c8daa92ab 100644 --- a/integration-test/src/test/java/org/apache/iotdb/libudf/it/dlearn/DLearnIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/libudf/it/dlearn/DLearnIT.java @@ -94,6 +94,11 @@ private static void createTimeSeries() { + "datatype=double, " + "encoding=plain, " + "compression=uncompressed"); + statement.addBatch( + "create timeseries root.vehicle.d3.s1 with " + + "datatype=int32, " + + "encoding=plain, " + + "compression=uncompressed"); statement.executeBatch(); } catch (SQLException throwable) { fail(throwable.getMessage()); @@ -168,6 +173,16 @@ private static void generateData() { String.format( "insert into root.vehicle.d2(timestamp,s1,s2,s3,s4) values(%d,%d,%d,%d,%d)", 900, 4, 4, 4, 4)); + // Toy series for cluster UDF (l=3, k=2): windows [1,2,3], [10,20,30], [1,5,1]. With + // norm=false, + // k-means groups the first two windows; k-shape / medoidshape group windows 0 and 2 + // (shape-related). + int[] toy = {1, 2, 3, 10, 20, 30, 1, 5, 1}; + for (int i = 0; i < toy.length; i++) { + statement.addBatch( + String.format( + "insert into root.vehicle.d3(timestamp,s1) values(%d,%d)", (i + 1) * 100, toy[i])); + } statement.executeBatch(); } catch (SQLException throwable) { fail(throwable.getMessage()); @@ -179,6 +194,7 @@ private static void registerUDF() { Statement statement = connection.createStatement()) { statement.execute("create function iqr as 'org.apache.iotdb.library.anomaly.UDTFIQR'"); statement.execute("create function ar as 'org.apache.iotdb.library.dlearn.UDTFAR'"); + statement.execute("create function cluster as 'org.apache.iotdb.library.dlearn.UDTFCluster'"); } catch (SQLException throwable) { fail(throwable.getMessage()); } @@ -308,4 +324,70 @@ public void testAR4() { fail(throwable.getMessage()); } } + + @Test + public void testCluster1() { + String sqlStr = + "select cluster(d3.s1, 'l'='3', 'k'='2', 'method'='kmeans', 'norm'='false', " + + "'maxiter'='50', 'output'='label') from root.vehicle"; + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery(sqlStr)) { + resultSet.next(); + int l0 = resultSet.getInt(2); + resultSet.next(); + int l1 = resultSet.getInt(2); + resultSet.next(); + int l2 = resultSet.getInt(2); + Assert.assertFalse(resultSet.next()); + Assert.assertEquals(l0, l2); + Assert.assertNotEquals(l0, l1); + } catch (SQLException throwable) { + fail(throwable.getMessage()); + } + } + + @Test + public void testCluster2() { + String sqlStr = + "select cluster(d3.s1, 'l'='3', 'k'='2', 'method'='kshape', 'norm'='true', " + + "'maxiter'='50', 'output'='label') from root.vehicle"; + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery(sqlStr)) { + resultSet.next(); + int l0 = resultSet.getInt(2); + resultSet.next(); + int l1 = resultSet.getInt(2); + resultSet.next(); + int l2 = resultSet.getInt(2); + Assert.assertFalse(resultSet.next()); + Assert.assertEquals(l0, l1); + Assert.assertNotEquals(l0, l2); + } catch (SQLException throwable) { + fail(throwable.getMessage()); + } + } + + @Test + public void testCluster3() { + String sqlStr = + "select cluster(d3.s1, 'l'='3', 'k'='2', 'method'='medoidshape', 'norm'='true', " + + "'sample_rate'='1', 'maxiter'='50', 'output'='label') from root.vehicle"; + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery(sqlStr)) { + resultSet.next(); + int l0 = resultSet.getInt(2); + resultSet.next(); + int l1 = resultSet.getInt(2); + resultSet.next(); + int l2 = resultSet.getInt(2); + Assert.assertFalse(resultSet.next()); + Assert.assertEquals(l0, l1); + Assert.assertNotEquals(l0, l2); + } catch (SQLException throwable) { + fail(throwable.getMessage()); + } + } } diff --git a/library-udf/src/assembly/tools/register-UDF.bat b/library-udf/src/assembly/tools/register-UDF.bat index c6683bc96ac5e..f30db5d2a3a7d 100644 --- a/library-udf/src/assembly/tools/register-UDF.bat +++ b/library-udf/src/assembly/tools/register-UDF.bat @@ -102,6 +102,7 @@ call ../sbin/start-cli.bat -h %host% -p %rpcPort% -u %user% -pw %pass% -e "creat @REM Machine Learning call ../sbin/start-cli.bat -h %host% -p %rpcPort% -u %user% -pw %pass% -e "create function ar as 'org.apache.iotdb.library.dlearn.UDTFAR'" +call ../sbin/start-cli.bat -h %host% -p %rpcPort% -u %user% -pw %pass% -e "create function cluster as 'org.apache.iotdb.library.dlearn.UDTFCluster'" @REM Match call ../sbin/start-cli.bat -h %host% -p %rpcPort% -u %user% -pw %pass% -e "create function pattern_match as 'org.apache.iotdb.library.match.UDAFPatternMatch'" diff --git a/library-udf/src/assembly/tools/register-UDF.sh b/library-udf/src/assembly/tools/register-UDF.sh index 16ab59f143baa..faaa4df68b5d1 100755 --- a/library-udf/src/assembly/tools/register-UDF.sh +++ b/library-udf/src/assembly/tools/register-UDF.sh @@ -102,6 +102,7 @@ pass=root # Machine Learning ../sbin/start-cli.sh -h $host -p $rpcPort -u $user -pw $pass -e "create function ar as 'org.apache.iotdb.library.dlearn.UDTFAR'" +../sbin/start-cli.sh -h $host -p $rpcPort -u $user -pw $pass -e "create function cluster as 'org.apache.iotdb.library.dlearn.UDTFCluster'" # Match ../sbin/start-cli.sh -h $host -p $rpcPort -u $user -pw $pass -e "create function pattern_match as 'org.apache.iotdb.library.match.UDAFPatternMatch'" diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dlearn/UDTFCluster.java b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/UDTFCluster.java new file mode 100644 index 0000000000000..f64f9747992bc --- /dev/null +++ b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/UDTFCluster.java @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.library.dlearn; + +import org.apache.iotdb.library.dlearn.util.cluster.KMeans; +import org.apache.iotdb.library.dlearn.util.cluster.KShape; +import org.apache.iotdb.library.dlearn.util.cluster.MedoidShape; +import org.apache.iotdb.library.util.Util; +import org.apache.iotdb.udf.api.UDTF; +import org.apache.iotdb.udf.api.access.Row; +import org.apache.iotdb.udf.api.collector.PointCollector; +import org.apache.iotdb.udf.api.customizer.config.UDTFConfigurations; +import org.apache.iotdb.udf.api.customizer.parameter.UDFParameterValidator; +import org.apache.iotdb.udf.api.customizer.parameter.UDFParameters; +import org.apache.iotdb.udf.api.customizer.strategy.RowByRowAccessStrategy; +import org.apache.iotdb.udf.api.exception.UDFException; +import org.apache.iotdb.udf.api.type.Type; + +import java.util.ArrayList; +import java.util.List; + +/** + * Clusters a time series by partitioning it into non-overlapping subsequences of length l. + * Parameters: l, k, method (default kmeans), norm, maxiter, output; medoidshape also uses + * sample_rate (greedy sampling ratio; use 1 when the window count is small). Requires at least k + * windows. + */ +public class UDTFCluster implements UDTF { + + private static final String METHOD_KMEANS = "kmeans"; + private static final String METHOD_KSHAPE = "kshape"; + private static final String METHOD_MEDOIDSHAPE = "medoidshape"; + + private static final String OUTPUT_LABEL = "label"; + private static final String OUTPUT_CENTROID = "centroid"; + + private static final int DEFAULT_MAX_ITER = 200; + private static final double DEFAULT_SAMPLE_RATE = 0.3; + private static final String DEFAULT_METHOD = METHOD_KMEANS; + + private int l; + private int k; + private String method; + private boolean norm; + private int maxIter; + private String output; + private double sampleRate; + + private final List timestamps = new ArrayList<>(); + private final List values = new ArrayList<>(); + + @Override + public void validate(UDFParameterValidator validator) throws Exception { + validator + .validateInputSeriesNumber(1) + .validateInputSeriesDataType(0, Type.INT32, Type.INT64, Type.FLOAT, Type.DOUBLE) + .validate( + x -> (int) x > 0, + "Parameter l must be a positive integer.", + validator.getParameters().getInt("l")) + .validate( + x -> (int) x >= 2, + "Parameter k must be at least 2.", + validator.getParameters().getInt("k")) + .validate( + x -> { + String m = ((String) x).toLowerCase(); + return METHOD_KMEANS.equals(m) + || METHOD_KSHAPE.equals(m) + || METHOD_MEDOIDSHAPE.equals(m); + }, + "Parameter method must be one of: kmeans, kshape, medoidshape.", + validator.getParameters().getStringOrDefault("method", DEFAULT_METHOD)) + .validate( + x -> (int) x >= 1, + "Parameter maxiter must be a positive integer.", + validator.getParameters().getIntOrDefault("maxiter", DEFAULT_MAX_ITER)) + .validate( + x -> { + String o = ((String) x).toLowerCase(); + return OUTPUT_LABEL.equals(o) || OUTPUT_CENTROID.equals(o); + }, + "Parameter output must be label or centroid.", + validator.getParameters().getStringOrDefault("output", OUTPUT_LABEL)) + .validate( + x -> { + double d = ((Number) x).doubleValue(); + return d > 0 && d <= 1.0; + }, + "Parameter sample_rate must be in (0, 1].", + validator.getParameters().getDoubleOrDefault("sample_rate", DEFAULT_SAMPLE_RATE)); + } + + @Override + public void beforeStart(UDFParameters parameters, UDTFConfigurations configurations) + throws Exception { + this.output = parameters.getStringOrDefault("output", OUTPUT_LABEL).toLowerCase(); + if (OUTPUT_CENTROID.equals(output)) { + configurations.setAccessStrategy(new RowByRowAccessStrategy()).setOutputDataType(Type.DOUBLE); + } else { + configurations.setAccessStrategy(new RowByRowAccessStrategy()).setOutputDataType(Type.INT32); + } + this.l = parameters.getInt("l"); + this.k = parameters.getInt("k"); + this.method = parameters.getStringOrDefault("method", DEFAULT_METHOD).toLowerCase(); + this.norm = parameters.getBooleanOrDefault("norm", true); + this.maxIter = parameters.getIntOrDefault("maxiter", DEFAULT_MAX_ITER); + this.sampleRate = parameters.getDoubleOrDefault("sample_rate", DEFAULT_SAMPLE_RATE); + timestamps.clear(); + values.clear(); + } + + @Override + public void transform(Row row, PointCollector collector) throws Exception { + if (!row.isNull(0)) { + timestamps.add(row.getTime()); + values.add(Util.getValueAsDouble(row)); + } + } + + @Override + public void terminate(PointCollector collector) throws Exception { + int n = values.size(); + if (n < l) { + throw new UDFException( + "Time series length must be at least l; got " + n + " points, l=" + l + "."); + } + int numWindows = n / l; + if (numWindows < k) { + throw new UDFException( + "Not enough non-overlapping windows: got " + + numWindows + + " windows, need at least k=" + + k + + "."); + } + + double[][] windows = new double[numWindows][l]; + long[] windowStartTime = new long[numWindows]; + for (int w = 0; w < numWindows; w++) { + windowStartTime[w] = timestamps.get(w * l); + for (int j = 0; j < l; j++) { + windows[w][j] = values.get(w * l + j); + } + } + + if (OUTPUT_LABEL.equals(output)) { + int[] labels; + if (METHOD_KMEANS.equals(method)) { + KMeans km = new KMeans(); + km.fit(windows, k, norm, maxIter); + labels = km.getLabels(); + } else if (METHOD_KSHAPE.equals(method)) { + KShape ks = new KShape(); + ks.fit(windows, k, norm, maxIter); + labels = ks.getLabels(); + } else if (METHOD_MEDOIDSHAPE.equals(method)) { + MedoidShape ms = new MedoidShape(); + ms.setSampleRate(sampleRate); + ms.fit(windows, k, norm, maxIter); + labels = ms.getLabels(); + } else { + throw new UDFException("Unsupported method: " + method); + } + for (int w = 0; w < numWindows; w++) { + collector.putInt(windowStartTime[w], labels[w]); + } + } else { + double[][] centroids; + if (METHOD_KMEANS.equals(method)) { + KMeans km = new KMeans(); + km.fit(windows, k, norm, maxIter); + centroids = km.getCentroids(); + } else if (METHOD_KSHAPE.equals(method)) { + KShape ks = new KShape(); + ks.fit(windows, k, norm, maxIter); + centroids = ks.getCentroids(); + } else if (METHOD_MEDOIDSHAPE.equals(method)) { + MedoidShape ms = new MedoidShape(); + ms.setSampleRate(sampleRate); + ms.fit(windows, k, norm, maxIter); + centroids = ms.getCentroids(); + } else { + throw new UDFException("Unsupported method: " + method); + } + emitConcatenatedCentroids(collector, centroids); + } + } + + private static void emitConcatenatedCentroids(PointCollector collector, double[][] centroids) + throws Exception { + long t = 0L; + for (double[] row : centroids) { + for (double v : row) { + collector.putDouble(t++, v); + } + } + } +} diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/ClusterUtils.java b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/ClusterUtils.java new file mode 100644 index 0000000000000..a646adb2bed30 --- /dev/null +++ b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/ClusterUtils.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.library.dlearn.util.cluster; + +import org.apache.commons.math3.complex.Complex; +import org.apache.commons.math3.transform.DftNormalization; +import org.apache.commons.math3.transform.FastFourierTransformer; +import org.apache.commons.math3.transform.TransformType; + +import java.util.Arrays; + +/** + * Subsequence z-normalize, Euclidean distance, and FFT-based NCC / SBD (shared by KShape and + * MedoidShape). + */ +public final class ClusterUtils { + + public static final double EPS = 1e-9; + + private static final FastFourierTransformer FFT = + new FastFourierTransformer(DftNormalization.STANDARD); + + private ClusterUtils() {} + + public static double[] maybeZNormalize(double[] a, boolean normalize) { + if (normalize) { + return zNormalize(a); + } + return Arrays.copyOf(a, a.length); + } + + public static double[] zNormalize(double[] a) { + int n = a.length; + double sum = 0.0; + for (double v : a) { + sum += v; + } + double mean = sum / n; + double var = 0.0; + for (double v : a) { + double d = v - mean; + var += d * d; + } + var /= n; + double std = Math.sqrt(Math.max(var, 0.0)); + double[] z = new double[n]; + if (std < EPS) { + return z; + } + for (int i = 0; i < n; i++) { + z[i] = (a[i] - mean) / std; + } + return z; + } + + public static double squaredEuclidean(double[] a, double[] b) { + double s = 0.0; + for (int i = 0; i < a.length; i++) { + double d = a[i] - b[i]; + s += d * d; + } + return s; + } + + public static int findLargestCluster(int[] counts) { + int best = 0; + for (int i = 1; i < counts.length; i++) { + if (counts[i] > counts[best]) { + best = i; + } + } + return best; + } + + /** + * Maximum over the normalized cross-correlation sequence (FFT); used for SBD and MedoidShape + * objective. + */ + public static double maxNcc(double[] x, double[] y) { + double[] cc = nccFft(x, y); + double max = Double.NEGATIVE_INFINITY; + for (double v : cc) { + if (v > max) { + max = v; + } + } + return max; + } + + /** SBD: 1 − max NCC (consistent with the NCC-based definition in k-Shape / FastKShape). */ + public static double shapeDistance(double[] x, double[] y) { + return 1.0 - maxNcc(x, y); + } + + public static double symmetricSbd(double[] a, double[] b) { + return 0.5 * (shapeDistance(a, b) + shapeDistance(b, a)); + } + + private static double[] nccFft(double[] x, double[] y) { + int xLen = x.length; + double den = l2Norm(x) * l2Norm(y); + if (den < 1e-9) { + den = Double.POSITIVE_INFINITY; + } + int fftSize = 1 << (32 - Integer.numberOfLeadingZeros(2 * xLen - 1)); + + Complex[] cx = new Complex[fftSize]; + Complex[] cy = new Complex[fftSize]; + for (int i = 0; i < fftSize; i++) { + cx[i] = new Complex(i < xLen ? x[i] : 0.0, 0.0); + cy[i] = new Complex(i < xLen ? y[i] : 0.0, 0.0); + } + Complex[] fx = FFT.transform(cx, TransformType.FORWARD); + Complex[] fy = FFT.transform(cy, TransformType.FORWARD); + Complex[] prod = new Complex[fftSize]; + for (int i = 0; i < fftSize; i++) { + prod[i] = fx[i].multiply(fy[i].conjugate()); + } + Complex[] ccFull = FFT.transform(prod, TransformType.INVERSE); + + double[] ccPacked = new double[2 * xLen - 1]; + int p = 0; + for (int i = fftSize - (xLen - 1); i < fftSize; i++) { + ccPacked[p++] = ccFull[i].getReal() / den; + } + for (int i = 0; i < xLen; i++) { + ccPacked[p++] = ccFull[i].getReal() / den; + } + return ccPacked; + } + + private static double l2Norm(double[] v) { + double s = 0.0; + for (double x : v) { + s += x * x; + } + return Math.sqrt(s); + } +} diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/KMeans.java b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/KMeans.java new file mode 100644 index 0000000000000..37c44eade96bf --- /dev/null +++ b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/KMeans.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.library.dlearn.util.cluster; + +import java.util.Arrays; + +/** + * Univariate subsequence k-means (Lloyd); optionally z-normalize, then cluster in Euclidean space. + */ +public class KMeans { + + private double[][] centroids; + private int[] labels; + + public void fit(double[][] samples, int k, boolean normalize, int maxIterations) { + validate(samples, k, maxIterations); + int n = samples.length; + int dim = samples[0].length; + + double[][] z = new double[n][dim]; + for (int i = 0; i < n; i++) { + z[i] = ClusterUtils.maybeZNormalize(samples[i], normalize); + } + + centroids = new double[k][dim]; + for (int c = 0; c < k; c++) { + System.arraycopy(z[c], 0, centroids[c], 0, dim); + } + + labels = new int[n]; + Arrays.fill(labels, -1); + + for (int iter = 0; iter < maxIterations; iter++) { + double[][] prevCentroids = new double[k][dim]; + for (int c = 0; c < k; c++) { + System.arraycopy(centroids[c], 0, prevCentroids[c], 0, dim); + } + + boolean changed = false; + for (int i = 0; i < n; i++) { + int best = 0; + double bestDist = Double.POSITIVE_INFINITY; + for (int c = 0; c < k; c++) { + double d = ClusterUtils.squaredEuclidean(z[i], centroids[c]); + if (d < bestDist) { + bestDist = d; + best = c; + } + } + if (labels[i] != best) { + labels[i] = best; + changed = true; + } + } + + double[][] newCentroids = new double[k][dim]; + int[] counts = new int[k]; + for (int i = 0; i < n; i++) { + int c = labels[i]; + counts[c]++; + for (int d = 0; d < dim; d++) { + newCentroids[c][d] += z[i][d]; + } + } + for (int c = 0; c < k; c++) { + if (counts[c] == 0) { + int donor = ClusterUtils.findLargestCluster(counts); + System.arraycopy(prevCentroids[donor], 0, centroids[c], 0, dim); + for (int d = 0; d < dim; d++) { + centroids[c][d] += (d == 0 ? 1e-4 : -1e-4); + } + } else { + for (int d = 0; d < dim; d++) { + centroids[c][d] = newCentroids[c][d] / counts[c]; + } + } + } + + if (!changed) { + break; + } + } + } + + public double[][] getCentroids() { + return centroids; + } + + public int[] getLabels() { + return labels; + } + + private static void validate(double[][] samples, int k, int maxIterations) { + if (samples == null || samples.length == 0) { + throw new IllegalArgumentException("samples must be non-empty."); + } + if (k < 2 || k > samples.length) { + throw new IllegalArgumentException("k must satisfy 2 <= k <= samples.length."); + } + if (maxIterations < 1) { + throw new IllegalArgumentException("maxIterations must be at least 1."); + } + int dim = samples[0].length; + if (dim == 0) { + throw new IllegalArgumentException("sample dimension must be positive."); + } + for (double[] row : samples) { + if (row == null || row.length != dim) { + throw new IllegalArgumentException("All samples must have the same length."); + } + } + } +} diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/KShape.java b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/KShape.java new file mode 100644 index 0000000000000..315e1c51d2f06 --- /dev/null +++ b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/KShape.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.library.dlearn.util.cluster; + +import org.apache.commons.math3.linear.MatrixUtils; +import org.apache.commons.math3.linear.RealMatrix; +import org.apache.commons.math3.linear.RealVector; +import org.apache.commons.math3.linear.SingularValueDecomposition; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * k-Shape: assignment uses {@link ClusterUtils#shapeDistance} (SBD = 1 − max NCC); centroids are + * the first right singular vector of the cluster matrix from SVD, sign correction, then z-normalize + * or L2 normalization. + */ +public class KShape { + + private double[][] centroids; + private int[] labels; + + public void fit(double[][] samples, int k, boolean normalize, int maxIterations) { + validate(samples, k, maxIterations); + int n = samples.length; + int dim = samples[0].length; + + double[][] z = new double[n][dim]; + for (int i = 0; i < n; i++) { + z[i] = ClusterUtils.maybeZNormalize(samples[i], normalize); + } + + centroids = new double[k][dim]; + for (int c = 0; c < k; c++) { + System.arraycopy(z[c], 0, centroids[c], 0, dim); + } + + labels = new int[n]; + Arrays.fill(labels, -1); + + for (int iter = 0; iter < maxIterations; iter++) { + double[][] prevCentroids = new double[k][dim]; + for (int c = 0; c < k; c++) { + System.arraycopy(centroids[c], 0, prevCentroids[c], 0, dim); + } + + boolean changed = false; + for (int i = 0; i < n; i++) { + int best = 0; + double bestDist = Double.POSITIVE_INFINITY; + for (int c = 0; c < k; c++) { + double d = ClusterUtils.shapeDistance(z[i], centroids[c]); + if (d < bestDist) { + bestDist = d; + best = c; + } + } + if (labels[i] != best) { + labels[i] = best; + changed = true; + } + } + + int[] counts = new int[k]; + @SuppressWarnings("unchecked") + List[] byCluster = new List[k]; + for (int c = 0; c < k; c++) { + byCluster[c] = new ArrayList<>(); + } + for (int i = 0; i < n; i++) { + int c = labels[i]; + counts[c]++; + byCluster[c].add(z[i]); + } + + for (int c = 0; c < k; c++) { + if (counts[c] == 0) { + int donor = ClusterUtils.findLargestCluster(counts); + System.arraycopy(prevCentroids[donor], 0, centroids[c], 0, dim); + } else { + List members = byCluster[c]; + double[][] mat = new double[members.size()][dim]; + for (int i = 0; i < members.size(); i++) { + mat[i] = members.get(i); + } + centroids[c] = centroidFromSvd(mat, normalize); + } + } + + if (!changed) { + break; + } + } + } + + public double[][] getCentroids() { + return centroids; + } + + public int[] getLabels() { + return labels; + } + + private static double[] centroidFromSvd(double[][] members, boolean zNormalizeCentroid) { + int m = members.length; + int dim = members[0].length; + if (m == 1) { + double[] u = Arrays.copyOf(members[0], dim); + return zNormalizeCentroid ? ClusterUtils.zNormalize(u) : l2Unit(u); + } + RealMatrix y = MatrixUtils.createRealMatrix(members); + SingularValueDecomposition svd = new SingularValueDecomposition(y); + RealMatrix v = svd.getV(); + RealVector col0 = v.getColumnVector(0); + double[] r = col0.toArray(); + double sumDot = 0.0; + for (double[] row : members) { + sumDot += dot(row, r); + } + if (sumDot < 0) { + for (int i = 0; i < r.length; i++) { + r[i] = -r[i]; + } + } + return zNormalizeCentroid ? ClusterUtils.zNormalize(r) : l2Unit(r); + } + + private static double dot(double[] a, double[] b) { + double s = 0.0; + for (int i = 0; i < a.length; i++) { + s += a[i] * b[i]; + } + return s; + } + + private static double[] l2Unit(double[] v) { + double s = 0.0; + for (double x : v) { + s += x * x; + } + s = Math.sqrt(s); + if (s < ClusterUtils.EPS) { + return new double[v.length]; + } + double[] o = new double[v.length]; + for (int i = 0; i < v.length; i++) { + o[i] = v[i] / s; + } + return o; + } + + private static void validate(double[][] samples, int k, int maxIterations) { + if (samples == null || samples.length == 0) { + throw new IllegalArgumentException("samples must be non-empty."); + } + if (k < 2 || k > samples.length) { + throw new IllegalArgumentException("k must satisfy 2 <= k <= samples.length."); + } + if (maxIterations < 1) { + throw new IllegalArgumentException("maxIterations must be at least 1."); + } + int dim = samples[0].length; + if (dim == 0) { + throw new IllegalArgumentException("sample dimension must be positive."); + } + for (double[] row : samples) { + if (row == null || row.length != dim) { + throw new IllegalArgumentException("All samples must have the same length."); + } + } + } +} diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/MedoidShape.java b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/MedoidShape.java new file mode 100644 index 0000000000000..31f9d8c8df1a0 --- /dev/null +++ b/library-udf/src/main/java/org/apache/iotdb/library/dlearn/util/cluster/MedoidShape.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.library.dlearn.util.cluster; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Random; +import java.util.Set; + +/** + * Coarse clustering: {@link KMeans} uses {@code min(2k, n)} clusters (n = number of windows); + * greedy fastKShape picks k representatives; both labels and the objective use {@link + * ClusterUtils#maxNcc}. + */ +public class MedoidShape { + + private double sampleRate = 0.3; + private Random random = new Random(); + + /** Overrides the RNG used for greedy sampling (default is {@link Random#Random()}). */ + public void setRandom(Random random) { + this.random = Objects.requireNonNull(random); + } + + private double[][] centroids; + private int[] labels; + + public void setSampleRate(double sampleRate) { + if (sampleRate <= 0 || sampleRate > 1.0) { + throw new IllegalArgumentException("sampleRate must be in (0, 1]."); + } + this.sampleRate = sampleRate; + } + + public double getSampleRate() { + return sampleRate; + } + + public void fit(double[][] samples, int k, boolean normalize, int maxIterations) { + validate(samples, k, maxIterations); + int n = samples.length; + int dim = samples[0].length; + + int coarseK = Math.min(2 * k, n); + + double[][] x = new double[n][dim]; + for (int i = 0; i < n; i++) { + x[i] = ClusterUtils.maybeZNormalize(samples[i], normalize); + } + + KMeans coarse = new KMeans(); + coarse.fit(x, coarseK, false, maxIterations); + double[][] euclideanCentroids = coarse.getCentroids(); + int[] kmLabels = coarse.getLabels(); + long[] clusterSize = new long[coarseK]; + for (int lb : kmLabels) { + clusterSize[lb]++; + } + + centroids = fastKShape(x, k, sampleRate, dim, euclideanCentroids, clusterSize, random); + + labels = new int[n]; + for (int i = 0; i < n; i++) { + double maxNcc = Double.NEGATIVE_INFINITY; + int label = -1; + for (int j = 0; j < k; j++) { + double cur = ClusterUtils.maxNcc(x[i], centroids[j]); + if (cur > maxNcc) { + maxNcc = cur; + label = j; + } + } + labels[i] = label; + } + } + + public double[][] getCentroids() { + return centroids; + } + + public int[] getLabels() { + return labels; + } + + private static double[][] fastKShape( + double[][] x, + int k, + double r, + int dim, + double[][] euclideanCentroids, + long[] clusterSize, + Random rnd) { + int n = x.length; + if (n <= k) { + double[][] out = new double[k][dim]; + for (int i = 0; i < n; i++) { + out[i] = Arrays.copyOf(x[i], dim); + } + for (int i = n; i < k; i++) { + out[i] = Arrays.copyOf(x[n - 1], dim); + } + return out; + } + + List picked = new ArrayList<>(); + Set coresetIdx = new HashSet<>(); + + for (int round = 0; round < k; round++) { + List pool = new ArrayList<>(); + for (int i = 0; i < n; i++) { + if (!coresetIdx.contains(i)) { + pool.add(i); + } + } + if (pool.isEmpty()) { + throw new IllegalStateException("fastKShape: empty candidate pool."); + } + int sampleCount = Math.max(1, (int) (r * n)); + sampleCount = Math.min(sampleCount, pool.size()); + Collections.shuffle(pool, rnd); + List sampleIdx = pool.subList(0, sampleCount); + + double maxDelta = Double.NEGATIVE_INFINITY; + double[] bestSeg = null; + int bestIdx = -1; + + for (int idx : sampleIdx) { + double[] seq = x[idx]; + picked.add(seq); + double delta = evaluateAim(picked, euclideanCentroids, clusterSize); + picked.remove(picked.size() - 1); + if (delta > maxDelta) { + maxDelta = delta; + bestSeg = Arrays.copyOf(seq, dim); + bestIdx = idx; + } + } + + if (bestSeg == null) { + throw new IllegalStateException("fastKShape: no candidate selected."); + } + picked.add(bestSeg); + coresetIdx.add(bestIdx); + } + + double[][] out = new double[k][dim]; + for (int i = 0; i < k; i++) { + out[i] = picked.get(i); + } + return out; + } + + private static double evaluateAim( + List curCentroids, double[][] euclideanCentroids, long[] clusterSize) { + double res = 0.0; + for (int i = 0; i < euclideanCentroids.length; i++) { + double maxNcc = Double.NEGATIVE_INFINITY; + for (double[] cur : curCentroids) { + double n = ClusterUtils.maxNcc(cur, euclideanCentroids[i]); + if (n > maxNcc) { + maxNcc = n; + } + } + res += maxNcc * clusterSize[i]; + } + return res; + } + + private static void validate(double[][] samples, int k, int maxIterations) { + if (samples == null || samples.length == 0) { + throw new IllegalArgumentException("samples must be non-empty."); + } + if (k < 2) { + throw new IllegalArgumentException("k must be at least 2."); + } + if (k > samples.length) { + throw new IllegalArgumentException("k must not exceed the number of samples."); + } + if (maxIterations < 1) { + throw new IllegalArgumentException("maxIterations must be at least 1."); + } + int dim = samples[0].length; + if (dim == 0) { + throw new IllegalArgumentException("sample dimension must be positive."); + } + for (double[] row : samples) { + if (row == null || row.length != dim) { + throw new IllegalArgumentException("All samples must have the same length."); + } + } + } +} diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFQuantile.java b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFQuantile.java index 047beafe1fc6a..0518e7a3d768a 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFQuantile.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/UDAFQuantile.java @@ -65,59 +65,65 @@ public void beforeStart(UDFParameters parameters, UDTFConfigurations configurati @Override public void transform(Row row, PointCollector collector) throws Exception { - double res = Util.getValueAsDouble(row); - sketch.update(dataToLong(res)); + final long encoded; + switch (dataType) { + case INT32: + encoded = row.getInt(0); + break; + case INT64: + encoded = row.getLong(0); + break; + default: + encoded = dataToLong(Util.getValueAsDouble(row)); + break; + } + sketch.update(encoded); } @Override public void terminate(PointCollector collector) throws Exception { - long result = sketch.findMinValueWithRank((long) (rank * sketch.getN())); - double res = longToResult(result); + long n = sketch.getN(); + // Nearest-rank: k-th smallest uses getApproxRank (strictly-less-than count) in [0, n-1]; + // rank=1 must map to k=n-1, not k=n which is unreachable and can overshoot the max sample. + long k = 0; + if (n > 0) { + k = (long) Math.ceil(rank * n) - 1; + if (k < 0) { + k = 0; + } else if (k >= n) { + k = n - 1; + } + } + long result = sketch.findMinValueWithRank(k); switch (dataType) { case INT32: - collector.putInt(0, (int) res); + collector.putInt(0, (int) result); break; case INT64: - collector.putLong(0, (long) res); + collector.putLong(0, result); break; case FLOAT: - collector.putFloat(0, (float) res); + collector.putFloat(0, (float) longToResult(result)); break; case DOUBLE: - collector.putDouble(0, res); + collector.putDouble(0, longToResult(result)); break; - case TIMESTAMP: - case DATE: - case TEXT: - case STRING: - case BLOB: - case BOOLEAN: default: break; } } - private long dataToLong(Object data) { - long result; + private long dataToLong(double res) { switch (dataType) { - case INT32: - return (int) data; case FLOAT: - result = Float.floatToIntBits((float) data); - return (float) data >= 0f ? result : result ^ Long.MAX_VALUE; - case INT64: - return (long) data; + float f = (float) res; + long flBits = Float.floatToIntBits(f); + return f >= 0f ? flBits : flBits ^ Long.MAX_VALUE; case DOUBLE: - result = Double.doubleToLongBits((double) data); - return (double) data >= 0d ? result : result ^ Long.MAX_VALUE; - case BLOB: - case BOOLEAN: - case STRING: - case TEXT: - case DATE: - case TIMESTAMP: + long d = Double.doubleToLongBits(res); + return res >= 0d ? d : d ^ Long.MAX_VALUE; default: - return (long) data; + return (long) res; } } @@ -129,16 +135,8 @@ private double longToResult(long result) { case DOUBLE: result = (result >>> 63) == 0 ? result : result ^ Long.MAX_VALUE; return Double.longBitsToDouble(result); - case INT64: - case INT32: - case DATE: - case TEXT: - case STRING: - case BOOLEAN: - case BLOB: - case TIMESTAMP: default: - return (result); + return (double) result; } } } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/ExactOrderStatistics.java b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/ExactOrderStatistics.java index e1f0baa7c0602..47ca5e2b12fd8 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/ExactOrderStatistics.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/ExactOrderStatistics.java @@ -31,7 +31,14 @@ import java.io.IOException; import java.util.NoSuchElementException; -/** Util for computing median, MAD, percentile. */ +/** + * Util for computing median, MAD, percentile. + * + *

Percentile / quantile ({@link #getPercentile}) uses discrete nearest-rank: for sorted + * size {@code n} and {@code phi} in (0, 1], take 1-based rank {@code k = ceil(n * phi)} and 0-based + * index {@code k - 1}, clamped to {@code [0, n - 1]}. No interpolation; {@code phi = 0.5} is not + * required to match {@link #getMedian}. + */ public class ExactOrderStatistics { private final Type dataType; @@ -55,12 +62,6 @@ public ExactOrderStatistics(Type type) throws UDFInputSeriesDataTypeNotValidExce case DOUBLE: doubleArrayList = new DoubleArrayList(); break; - case STRING: - case TEXT: - case BOOLEAN: - case BLOB: - case DATE: - case TIMESTAMP: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -88,12 +89,6 @@ public void insert(Row row) throws UDFInputSeriesDataTypeNotValidException, IOEx doubleArrayList.add(vd); } break; - case DATE: - case TIMESTAMP: - case BLOB: - case BOOLEAN: - case TEXT: - case STRING: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -111,12 +106,6 @@ public double getMedian() throws UDFInputSeriesDataTypeNotValidException { return getMedian(floatArrayList); case DOUBLE: return getMedian(doubleArrayList); - case TEXT: - case STRING: - case BOOLEAN: - case BLOB: - case TIMESTAMP: - case DATE: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -199,12 +188,6 @@ public double getMad() throws UDFInputSeriesDataTypeNotValidException { return getMad(floatArrayList); case DOUBLE: return getMad(doubleArrayList); - case TIMESTAMP: - case DATE: - case BLOB: - case BOOLEAN: - case STRING: - case TEXT: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -251,12 +234,18 @@ public static double getMad(LongArrayList nums) { } } + /** Discrete nearest-rank index into sorted data of length {@code n}; see class Javadoc. */ + private static int discreteNearestRankIndex(int n, double phi) { + int idx = (int) Math.ceil(n * phi) - 1; + return Math.max(0, Math.min(n - 1, idx)); + } + public static float getPercentile(FloatArrayList nums, double phi) { if (nums.isEmpty()) { throw new NoSuchElementException(); } else { nums.sortThis(); - return nums.get((int) Math.ceil(nums.size() * phi)); + return nums.get(discreteNearestRankIndex(nums.size(), phi)); } } @@ -265,7 +254,7 @@ public static double getPercentile(DoubleArrayList nums, double phi) { throw new NoSuchElementException(); } else { nums.sortThis(); - return nums.get((int) Math.ceil(nums.size() * phi)); + return nums.get(discreteNearestRankIndex(nums.size(), phi)); } } @@ -279,12 +268,6 @@ public String getPercentile(double phi) throws UDFInputSeriesDataTypeNotValidExc return Float.toString(getPercentile(floatArrayList, phi)); case DOUBLE: return Double.toString(getPercentile(doubleArrayList, phi)); - case STRING: - case TEXT: - case BOOLEAN: - case BLOB: - case DATE: - case TIMESTAMP: default: // This will not happen. throw new UDFInputSeriesDataTypeNotValidException( @@ -297,7 +280,7 @@ public static int getPercentile(IntArrayList nums, double phi) { throw new NoSuchElementException(); } else { nums.sortThis(); - return nums.get((int) Math.ceil(nums.size() * phi)); + return nums.get(discreteNearestRankIndex(nums.size(), phi)); } } @@ -306,7 +289,7 @@ public static long getPercentile(LongArrayList nums, double phi) { throw new NoSuchElementException(); } else { nums.sortThis(); - return nums.get((int) Math.ceil(nums.size() * phi)); + return nums.get(discreteNearestRankIndex(nums.size(), phi)); } } } diff --git a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/GKArray.java b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/GKArray.java index 1870bdfb7a4c2..7dbcc934e7860 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/GKArray.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/dprofile/util/GKArray.java @@ -124,6 +124,19 @@ private void compress(List additionalEntries) { i++; + } else if (i >= additionalEntries.size()) { + // Only sketch entries left (must check before comparing additionalEntries.get(i)). + if (j + 1 < entries.size() + && entries.get(j).g + entries.get(j + 1).g + entries.get(j + 1).delta + <= removalThreshold) { + // Removable from sketch. + entries.get(j + 1).g += entries.get(j).g; + } else { + mergedEntries.add(entries.get(j)); + } + + j++; + } else if (additionalEntries.get(i).v < entries.get(j).v) { if (additionalEntries.get(i).g + entries.get(j).g + entries.get(j).delta <= removalThreshold) { @@ -136,7 +149,7 @@ private void compress(List additionalEntries) { i++; - } else { // the same as i == additionalEntries.size() + } else { if (j + 1 < entries.size() && entries.get(j).g + entries.get(j + 1).g + entries.get(j + 1).delta <= removalThreshold) { diff --git a/library-udf/src/main/java/org/apache/iotdb/library/match/PatternExecutor.java b/library-udf/src/main/java/org/apache/iotdb/library/match/PatternExecutor.java index 01d1c533493b6..4e5e73206709e 100644 --- a/library-udf/src/main/java/org/apache/iotdb/library/match/PatternExecutor.java +++ b/library-udf/src/main/java/org/apache/iotdb/library/match/PatternExecutor.java @@ -154,7 +154,7 @@ public List executeQuery(PatternContext queryCtx) { /** Execute the query in a particular smooth iteration */ private void executeQueryInSI(PatternContext queryCtx) { - int dsi = 0; + int dsi = 0; // dsi: data section index if (queryCtx.getDatasetSize() == null) { queryCtx.setDatasetSize( queryCtx.getDataPoints().get(queryCtx.getDataPoints().size() - 1).getX() @@ -237,11 +237,13 @@ private double calcHeight(List points) { */ private List

findCurveSections( List tangents, List points, double minHeightPerc) { + List
sections = new ArrayList<>(); Double lastTg = null; Point lastPt = null; double totalHeight = calcHeight(points); double lastSectHeight = 0; + for (int i = 0; i < tangents.size(); i++) { Double tangent = tangents.get(i); Point pt = points.get(i); @@ -257,6 +259,7 @@ private List
findCurveSections( && (!(minHeightPerc > 0) || lastSectHeight / totalHeight > minHeightPerc)) { Section newSection = new Section(sign); sections.add(newSection); + newSection.getPoints().add(lastPt); newSection.getTangents().add(lastTg); } @@ -340,7 +343,6 @@ private boolean matchIn( || currSect.getNext().get(0).getSize() == currSect.getNext().get(0).getTimes())) { matchValue = this.calculateMatch(dataSectsForQ, newQSections, queryCtx, false); if (matchValue != null) { - // Keep only one (best) match if the same area is selected in different smooth iterations int duplicateMatchIdx = PatternMatchConfig.REMOVE_EQUAL_MATCHES @@ -488,6 +490,7 @@ private PatternCalculationResult calculatePointsMatch( querySect.setPoints(querySections.get(si).getPoints()); querySect.setWidth(calcWidth(querySect.getPoints())); querySect.setHeight(calcHeight(querySect.getPoints())); + if (querySect.getHeight() == 0) { continue; } @@ -507,6 +510,7 @@ private PatternCalculationResult calculatePointsMatch( } dataSect.setWidth(calcWidth(dataSect.getPoints())); dataSect.setHeight(calcHeight(dataSect.getPoints())); + if (dataSect.getHeight() == 0) { continue; } @@ -540,6 +544,7 @@ private PatternCalculationResult calculatePointsMatch( * scaleFactorY); } querySect.setCentroidY(querySect.getCentroidY() / querySect.getPoints().size()); + centroidsDifference = querySect.getPoints().get(0).getY() * (PatternMatchConfig.RESCALING_Y From 291f01c60410901700a96b1cd3dc110d0500b61f Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 30 Apr 2026 09:38:31 +0800 Subject: [PATCH 021/102] [To dev/1.3] Pipe: Fixed the bug that air gap receiver may not respond in temporary timeout exception & Optimized the directory check in receiver & Optimized the configNode pipe logic (#17556) & Optimized the clear logic of Schema Region (#17553) (#17576) * Optimized the clear logic of Schema Region && Pipe: Fixed the bug that the historical pipe does not work for deletion-only sync (#17553) * Pipe: Fixed the bug that air gap receiver may not respond in temporary timeout exception & Optimized the directory check in receiver & Optimized the configNode pipe logic (#17556) --- .../confignode/manager/ProcedureManager.java | 4 +- .../heartbeat/PipeHeartbeatParser.java | 38 ++-- .../PipeHandleLeaderChangeProcedure.java | 6 +- .../heartbeat/PipeHeartbeatParserTest.java | 182 ++++++++++++++++++ .../PipeHandleLeaderChangeProcedureTest.java | 46 +++++ .../protocol/airgap/IoTDBAirGapReceiver.java | 5 + .../impl/SchemaRegionMemoryImpl.java | 4 +- .../airgap/IoTDBAirGapReceiverTest.java | 103 ++++++++++ .../pipe/receiver/IoTDBFileReceiver.java | 72 +++++-- .../receiver/PipeReceiverFilePathUtils.java | 42 ++++ .../pipe/receiver/IoTDBFileReceiverTest.java | 46 +++++ 11 files changed, 509 insertions(+), 39 deletions(-) create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParserTest.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverFilePathUtils.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java index f0f1b9e6bfbff..4912f2dad5c8b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java @@ -1443,7 +1443,7 @@ public void pipeHandleLeaderChange( } } - public void pipeHandleMetaChange( + public boolean pipeHandleMetaChange( boolean needWriteConsensusOnConfigNodes, boolean needPushPipeMetaToDataNodes) { try { final long procedureId = @@ -1451,8 +1451,10 @@ public void pipeHandleMetaChange( new PipeHandleMetaChangeProcedure( needWriteConsensusOnConfigNodes, needPushPipeMetaToDataNodes)); LOGGER.info("PipeHandleMetaChangeProcedure was submitted, procedureId: {}.", procedureId); + return true; } catch (Exception e) { LOGGER.warn("PipeHandleMetaChangeProcedure was failed to submit.", e); + return false; } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParser.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParser.java index ace07f5e2d3d3..6dc11ddd3f3bd 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParser.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParser.java @@ -59,7 +59,7 @@ public class PipeHeartbeatParser { this.configManager = configManager; heartbeatCounter = 0; - registeredNodeNumber = 1; + registeredNodeNumber = getExpectedHeartbeatNodeCount(); needWriteConsensusOnConfigNodes = new AtomicBoolean(false); needPushPipeMetaToDataNodes = new AtomicBoolean(false); @@ -73,17 +73,8 @@ synchronized void parseHeartbeat(final int nodeId, final PipeHeartbeat pipeHeart if (heartbeatCount % registeredNodeNumber == 0) { canSubmitHandleMetaChangeProcedure.set(true); - // registeredNodeNumber may be changed, update it here when we can submit procedure - registeredNodeNumber = configManager.getNodeManager().getRegisteredNodeCount(); - if (registeredNodeNumber <= 0) { - LOGGER.warn( - "registeredNodeNumber is {} when parseHeartbeat from node (id={}).", - registeredNodeNumber, - nodeId); - // registeredNodeNumber can not be set to 0 in this class, otherwise may cause - // DivideByZeroException - registeredNodeNumber = 1; - } + // The expected reporter set may be changed, update it at the end of the current round. + registeredNodeNumber = getExpectedHeartbeatNodeCount(); } if (pipeHeartbeat.isEmpty() @@ -114,14 +105,14 @@ synchronized void parseHeartbeat(final int nodeId, final PipeHeartbeat pipeHeart if (canSubmitHandleMetaChangeProcedure.get() && (needWriteConsensusOnConfigNodes.get() || needPushPipeMetaToDataNodes.get())) { - configManager + if (configManager .getProcedureManager() .pipeHandleMetaChange( - needWriteConsensusOnConfigNodes.get(), needPushPipeMetaToDataNodes.get()); - - // Reset flags after procedure is submitted - needWriteConsensusOnConfigNodes.set(false); - needPushPipeMetaToDataNodes.set(false); + needWriteConsensusOnConfigNodes.get(), + needPushPipeMetaToDataNodes.get())) { + needWriteConsensusOnConfigNodes.set(false); + needPushPipeMetaToDataNodes.set(false); + } } } finally { configManager.getPipeManager().getPipeTaskCoordinator().unlock(); @@ -129,6 +120,17 @@ synchronized void parseHeartbeat(final int nodeId, final PipeHeartbeat pipeHeart }); } + private int getExpectedHeartbeatNodeCount() { + final int expectedNodeCount = + configManager.getNodeManager().getRegisteredDataNodeCount() + + (PipeConfig.getInstance().isSeperatedPipeHeartbeatEnabled() ? 1 : 0); + if (expectedNodeCount <= 0) { + LOGGER.warn("Expected pipe heartbeat node count is {}, fallback to 1.", expectedNodeCount); + return 1; + } + return expectedNodeCount; + } + private void parseHeartbeatAndSaveMetaChangeLocally( final AtomicReference pipeTaskInfo, final int nodeId, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedure.java index f18737e8b7a7e..61f6f3cae2aaf 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedure.java @@ -164,7 +164,11 @@ public void deserialize(ByteBuffer byteBuffer) { final int oldDataRegionLeaderId = ReadWriteIOUtils.readInt(byteBuffer); final int newDataRegionLeaderId = ReadWriteIOUtils.readInt(byteBuffer); regionGroupToOldAndNewLeaderPairMap.put( - new TConsensusGroupId(TConsensusGroupType.DataRegion, dataRegionGroupId), + new TConsensusGroupId( + dataRegionGroupId == Integer.MIN_VALUE + ? TConsensusGroupType.ConfigRegion + : TConsensusGroupType.DataRegion, + dataRegionGroupId), new Pair<>(oldDataRegionLeaderId, newDataRegionLeaderId)); } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParserTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParserTest.java new file mode 100644 index 0000000000000..d5a46d42c84a4 --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParserTest.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.manager.pipe.coordinator.runtime.heartbeat; + +import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.ProcedureManager; +import org.apache.iotdb.confignode.manager.node.NodeManager; +import org.apache.iotdb.confignode.manager.pipe.coordinator.PipeManager; +import org.apache.iotdb.confignode.manager.pipe.coordinator.runtime.PipeRuntimeCoordinator; +import org.apache.iotdb.confignode.manager.pipe.coordinator.task.PipeTaskCoordinator; +import org.apache.iotdb.confignode.persistence.pipe.PipeTaskInfo; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.lang.reflect.Field; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class PipeHeartbeatParserTest { + + private boolean originalSeparatedPipeHeartbeatEnabled; + + @Before + public void setUp() { + originalSeparatedPipeHeartbeatEnabled = + CommonDescriptor.getInstance().getConfig().isSeperatedPipeHeartbeatEnabled(); + } + + @After + public void tearDown() { + CommonDescriptor.getInstance() + .getConfig() + .setSeperatedPipeHeartbeatEnabled(originalSeparatedPipeHeartbeatEnabled); + } + + @Test + public void testParseHeartbeatCountsOnlyDataNodesWhenSeparatedHeartbeatDisabled() + throws Exception { + CommonDescriptor.getInstance().getConfig().setSeperatedPipeHeartbeatEnabled(false); + + final ParserTestContext context = createParserTestContext(2); + setMetaChangeFlags(context.parser, true, false); + + context.parser.parseHeartbeat(1, emptyHeartbeat()); + verify(context.procedureManager, never()).pipeHandleMetaChange(anyBoolean(), anyBoolean()); + + context.parser.parseHeartbeat(2, emptyHeartbeat()); + verify(context.procedureManager, times(1)).pipeHandleMetaChange(true, false); + } + + @Test + public void testParseHeartbeatCountsLocalConfigNodeWhenSeparatedHeartbeatEnabled() + throws Exception { + CommonDescriptor.getInstance().getConfig().setSeperatedPipeHeartbeatEnabled(true); + + final ParserTestContext context = createParserTestContext(2); + setMetaChangeFlags(context.parser, true, false); + + context.parser.parseHeartbeat(1, emptyHeartbeat()); + context.parser.parseHeartbeat(2, emptyHeartbeat()); + verify(context.procedureManager, never()).pipeHandleMetaChange(anyBoolean(), anyBoolean()); + + context.parser.parseHeartbeat(3, emptyHeartbeat()); + verify(context.procedureManager, times(1)).pipeHandleMetaChange(true, false); + } + + @Test + public void testParseHeartbeatKeepsPendingFlagsWhenProcedureSubmissionFails() throws Exception { + CommonDescriptor.getInstance().getConfig().setSeperatedPipeHeartbeatEnabled(false); + + final ParserTestContext context = createParserTestContext(2); + when(context.procedureManager.pipeHandleMetaChange(anyBoolean(), anyBoolean())) + .thenReturn(false, true); + setMetaChangeFlags(context.parser, true, false); + + context.parser.parseHeartbeat(1, emptyHeartbeat()); + verify(context.procedureManager, never()).pipeHandleMetaChange(anyBoolean(), anyBoolean()); + + context.parser.parseHeartbeat(2, emptyHeartbeat()); + verify(context.procedureManager, times(1)).pipeHandleMetaChange(true, false); + + context.parser.parseHeartbeat(3, emptyHeartbeat()); + verify(context.procedureManager, times(1)).pipeHandleMetaChange(true, false); + + context.parser.parseHeartbeat(4, emptyHeartbeat()); + verify(context.procedureManager, times(2)).pipeHandleMetaChange(true, false); + } + + private ParserTestContext createParserTestContext(final int registeredDataNodeCount) { + final ConfigManager configManager = Mockito.mock(ConfigManager.class); + final NodeManager nodeManager = Mockito.mock(NodeManager.class); + final ProcedureManager procedureManager = Mockito.mock(ProcedureManager.class); + final PipeManager pipeManager = Mockito.mock(PipeManager.class); + final PipeRuntimeCoordinator pipeRuntimeCoordinator = + Mockito.mock(PipeRuntimeCoordinator.class); + final PipeTaskCoordinator pipeTaskCoordinator = Mockito.mock(PipeTaskCoordinator.class); + final ExecutorService procedureSubmitter = Mockito.mock(ExecutorService.class); + + when(configManager.getNodeManager()).thenReturn(nodeManager); + when(configManager.getProcedureManager()).thenReturn(procedureManager); + when(configManager.getPipeManager()).thenReturn(pipeManager); + when(nodeManager.getRegisteredDataNodeCount()).thenReturn(registeredDataNodeCount); + when(pipeManager.getPipeRuntimeCoordinator()).thenReturn(pipeRuntimeCoordinator); + when(pipeManager.getPipeTaskCoordinator()).thenReturn(pipeTaskCoordinator); + when(pipeRuntimeCoordinator.getProcedureSubmitter()).thenReturn(procedureSubmitter); + when(pipeTaskCoordinator.tryLock()).thenReturn(new AtomicReference<>(new PipeTaskInfo())); + when(procedureManager.pipeHandleMetaChange(anyBoolean(), anyBoolean())).thenReturn(true); + Mockito.doAnswer( + invocation -> { + ((Runnable) invocation.getArgument(0)).run(); + return CompletableFuture.completedFuture(null); + }) + .when(procedureSubmitter) + .submit(any(Runnable.class)); + + return new ParserTestContext(new PipeHeartbeatParser(configManager), procedureManager); + } + + private void setMetaChangeFlags( + final PipeHeartbeatParser parser, + final boolean needWriteConsensusOnConfigNodes, + final boolean needPushPipeMetaToDataNodes) + throws Exception { + setAtomicBooleanField( + parser, "needWriteConsensusOnConfigNodes", needWriteConsensusOnConfigNodes); + setAtomicBooleanField(parser, "needPushPipeMetaToDataNodes", needPushPipeMetaToDataNodes); + } + + private void setAtomicBooleanField( + final PipeHeartbeatParser parser, final String fieldName, final boolean value) + throws Exception { + final Field field = PipeHeartbeatParser.class.getDeclaredField(fieldName); + field.setAccessible(true); + ((AtomicBoolean) field.get(parser)).set(value); + } + + private PipeHeartbeat emptyHeartbeat() { + return new PipeHeartbeat(Collections.emptyList(), null, null, null); + } + + private static class ParserTestContext { + private final PipeHeartbeatParser parser; + private final ProcedureManager procedureManager; + + private ParserTestContext( + final PipeHeartbeatParser parser, final ProcedureManager procedureManager) { + this.parser = parser; + this.procedureManager = procedureManager; + } + } +} diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedureTest.java index b76e291b0c3b9..75c0963a27fab 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedureTest.java @@ -21,7 +21,10 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.confignode.procedure.Procedure; +import org.apache.iotdb.confignode.procedure.state.ProcedureState; import org.apache.iotdb.confignode.procedure.store.ProcedureFactory; +import org.apache.iotdb.confignode.procedure.store.ProcedureType; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.PublicBAOS; @@ -45,6 +48,9 @@ public void serializeDeserializeTest() { leaderMap.put(new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new Pair<>(1, 2)); leaderMap.put(new TConsensusGroupId(TConsensusGroupType.DataRegion, 2), new Pair<>(2, 3)); leaderMap.put(new TConsensusGroupId(TConsensusGroupType.DataRegion, 3), new Pair<>(4, 5)); + leaderMap.put( + new TConsensusGroupId(TConsensusGroupType.ConfigRegion, Integer.MIN_VALUE), + new Pair<>(6, 7)); PipeHandleLeaderChangeProcedure proc = new PipeHandleLeaderChangeProcedure(leaderMap); @@ -60,4 +66,44 @@ public void serializeDeserializeTest() { fail(); } } + + @Test + public void deserializeOldFormatConfigRegionTest() { + PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); + + Map> leaderMap = new HashMap<>(); + leaderMap.put( + new TConsensusGroupId(TConsensusGroupType.ConfigRegion, Integer.MIN_VALUE), + new Pair<>(6, 7)); + + try { + outputStream.writeShort(ProcedureType.PIPE_HANDLE_LEADER_CHANGE_PROCEDURE.getTypeCode()); + outputStream.writeLong(Procedure.NO_PROC_ID); + outputStream.writeInt(ProcedureState.INITIALIZING.ordinal()); + outputStream.writeLong(0L); + outputStream.writeLong(0L); + outputStream.writeLong(Procedure.NO_PROC_ID); + outputStream.writeLong(Procedure.NO_TIMEOUT); + outputStream.writeInt(-1); + outputStream.write((byte) 0); + outputStream.writeInt(-1); + outputStream.write((byte) 0); + outputStream.writeInt(0); + outputStream.write((byte) 0); + outputStream.writeInt(leaderMap.size()); + outputStream.writeInt(Integer.MIN_VALUE); + outputStream.writeInt(6); + outputStream.writeInt(7); + + ByteBuffer buffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + PipeHandleLeaderChangeProcedure proc = + (PipeHandleLeaderChangeProcedure) ProcedureFactory.getInstance().create(buffer); + + assertEquals(new PipeHandleLeaderChangeProcedure(leaderMap), proc); + } catch (Exception e) { + fail(); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiver.java index 8658d12b6a89f..278c1ccaaefc5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiver.java @@ -178,6 +178,11 @@ private void handleReq(final AirGapPseudoTPipeTransferRequest req, final long st if (System.currentTimeMillis() - startTime < PipeConfig.getInstance().getPipeAirGapRetryMaxMs()) { handleReq(req, startTime); + } else { + LOGGER.warn( + "Pipe air gap receiver {}: Temporary unavailable retry timed out, returning FAIL to sender.", + receiverId); + fail(); } } else { LOGGER.warn( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java index a55de193914ef..0f6490eceb467 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/impl/SchemaRegionMemoryImpl.java @@ -393,7 +393,9 @@ public synchronized void clear() { logWriter.close(); logWriter = null; } - tagManager.clear(); + if (tagManager != null) { + tagManager.clear(); + } isRecovering = true; initialized = false; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiverTest.java index 19dea8140a190..e23db1f1ca8cd 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/airgap/IoTDBAirGapReceiverTest.java @@ -19,18 +19,32 @@ package org.apache.iotdb.db.pipe.receiver.protocol.airgap; +import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.pipe.receiver.IoTDBReceiver; import org.apache.iotdb.commons.pipe.sink.payload.airgap.AirGapELanguageConstant; +import org.apache.iotdb.commons.pipe.sink.payload.airgap.AirGapOneByteResponse; +import org.apache.iotdb.commons.pipe.sink.payload.airgap.AirGapPseudoTPipeTransferRequest; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.IoTDBSinkRequestVersion; +import org.apache.iotdb.db.pipe.receiver.protocol.thrift.IoTDBDataNodeReceiverAgent; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; import org.apache.tsfile.utils.BytesUtils; import org.junit.Assert; import org.junit.Test; import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; +import java.lang.reflect.Field; +import java.lang.reflect.Method; import java.net.Socket; +import java.nio.ByteBuffer; public class IoTDBAirGapReceiverTest { @@ -69,4 +83,93 @@ public void testRejectNestedELanguagePrefix() throws Exception { Assert.assertThrows(IOException.class, () -> receiver.readData(inputStream)); Assert.assertTrue(exception.getMessage().contains("nested E-Language prefix")); } + + @Test + public void testTemporaryUnavailableRetryTimeoutReturnsFail() throws Exception { + final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig(); + final long originalRetryLocalIntervalMs = commonConfig.getPipeAirGapRetryLocalIntervalMs(); + final long originalRetryMaxMs = commonConfig.getPipeAirGapRetryMaxMs(); + + try { + commonConfig.setPipeAirGapRetryLocalIntervalMs(0); + commonConfig.setPipeAirGapRetryMaxMs(1); + + final RecordingSocket socket = new RecordingSocket(); + final IoTDBAirGapReceiver receiver = new IoTDBAirGapReceiver(socket, 3L); + final StubIoTDBDataNodeReceiverAgent stubAgent = new StubIoTDBDataNodeReceiverAgent(); + stubAgent.setStubReceiver( + new StubReceiver( + new TSStatus( + TSStatusCode.PIPE_RECEIVER_TEMPORARY_UNAVAILABLE_EXCEPTION.getStatusCode()))); + setField(receiver, "agent", stubAgent); + + final AirGapPseudoTPipeTransferRequest req = new AirGapPseudoTPipeTransferRequest(); + req.setVersion(IoTDBSinkRequestVersion.VERSION_1.getVersion()); + req.setType((short) 0); + req.setBody(ByteBuffer.allocate(0)); + + final Method handleReq = + IoTDBAirGapReceiver.class.getDeclaredMethod( + "handleReq", AirGapPseudoTPipeTransferRequest.class, long.class); + handleReq.setAccessible(true); + handleReq.invoke(receiver, req, System.currentTimeMillis() - 10_000L); + + Assert.assertArrayEquals(AirGapOneByteResponse.FAIL, socket.getWrittenBytes()); + } finally { + commonConfig.setPipeAirGapRetryLocalIntervalMs(originalRetryLocalIntervalMs); + commonConfig.setPipeAirGapRetryMaxMs(originalRetryMaxMs); + } + } + + private static void setField(final Object target, final String fieldName, final Object value) + throws Exception { + final Field field = IoTDBAirGapReceiver.class.getDeclaredField(fieldName); + field.setAccessible(true); + field.set(target, value); + } + + private static class RecordingSocket extends Socket { + + private final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + + @Override + public OutputStream getOutputStream() { + return outputStream; + } + + byte[] getWrittenBytes() { + return outputStream.toByteArray(); + } + } + + private static class StubIoTDBDataNodeReceiverAgent extends IoTDBDataNodeReceiverAgent { + + void setStubReceiver(final IoTDBReceiver receiver) { + setReceiverWithSpecifiedClient(null, receiver); + } + } + + private static class StubReceiver implements IoTDBReceiver { + + private final TPipeTransferResp response; + + private StubReceiver(final TSStatus status) { + response = new TPipeTransferResp(status); + } + + @Override + public TPipeTransferResp receive(final TPipeTransferReq req) { + return response; + } + + @Override + public void handleExit() { + // noop for unit test + } + + @Override + public IoTDBSinkRequestVersion getVersion() { + return IoTDBSinkRequestVersion.VERSION_1; + } + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java index 9153acf9b2e64..61d9155f99681 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java @@ -461,16 +461,7 @@ protected final void updateWritingFileIfNeeded(final String fileName, final bool receiverFileDirWithIdSuffix.get().getPath()); } } - Path baseDir = receiverFileDirWithIdSuffix.get().toPath().toAbsolutePath().normalize(); - Path targetPath = baseDir.resolve(fileName).toAbsolutePath().normalize(); - - if (!targetPath.startsWith(baseDir)) { - LOGGER.error( - "Receiver id = {}: Path traversal attempt detected! Filename: {}", - receiverId.get(), - fileName); - throw new IOException("Illegal fileName: " + fileName + " (Path traversal detected)"); - } + final Path targetPath = resolveReceiverFilePath(fileName); writingFile = targetPath.toFile(); writingFileWriter = new RandomAccessFile(writingFile, "rw"); @@ -481,7 +472,37 @@ protected final void updateWritingFileIfNeeded(final String fileName, final bool } private boolean isFileExistedAndNameCorrect(final String fileName) { - return writingFile != null && writingFile.exists() && writingFile.getName().equals(fileName); + try { + return writingFile != null + && writingFile.exists() + && receiverFileDirWithIdSuffix.get() != null + && writingFile + .toPath() + .toAbsolutePath() + .normalize() + .equals(resolveReceiverFilePath(fileName)); + } catch (final IOException e) { + PipeLogger.log( + LOGGER::warn, + e, + "Receiver id = %s: Illegal file name %s when checking writing file.", + receiverId.get(), + fileName); + return false; + } + } + + private Path resolveReceiverFilePath(final String fileName) throws IOException { + try { + return PipeReceiverFilePathUtils.resolveFilePath( + receiverFileDirWithIdSuffix.get().toPath(), fileName); + } catch (final IOException e) { + LOGGER.error( + "Receiver id = {}: Path traversal attempt detected! Filename: {}", + receiverId.get(), + fileName); + throw e; + } } private void closeCurrentWritingFileWriter(final boolean fsyncAfterClose) { @@ -641,11 +662,23 @@ protected final TPipeTransferResp handleTransferFileSealV1(final PipeTransferFil } protected final TPipeTransferResp handleTransferFileSealV2(final PipeTransferFileSealReqV2 req) { - final List files = - req.getFileNames().stream() - .map(fileName -> new File(receiverFileDirWithIdSuffix.get(), fileName)) - .collect(Collectors.toList()); + final List fileNames = req.getFileNames(); try { + final List files = + fileNames.stream() + .map( + fileName -> { + if (Objects.isNull(fileName)) { + return null; + } + try { + return resolveReceiverFilePath(fileName).toFile(); + } catch (final IOException e) { + throw new IllegalArgumentException(e); + } + }) + .collect(Collectors.toList()); + if (!isWritingFileAvailable()) { final TSStatus status = RpcUtils.getStatus( @@ -707,17 +740,20 @@ protected final TPipeTransferResp handleTransferFileSealV2(final PipeTransferFil } return new TPipeTransferResp(status); } catch (final Exception e) { + final Throwable rootCause = e instanceof IllegalArgumentException ? e.getCause() : e; PipeLogger.log( LOGGER::warn, - e, + rootCause, "Receiver id = %s: Failed to seal file %s from req %s.", receiverId.get(), - files, + fileNames, req); return new TPipeTransferResp( RpcUtils.getStatus( TSStatusCode.PIPE_TRANSFER_FILE_ERROR, - String.format("Failed to seal file %s because %s", writingFile, e.getMessage()))); + String.format( + "Failed to seal file %s because %s", + fileNames, rootCause == null ? e.getMessage() : rootCause.getMessage()))); } finally { // If the writing file is not sealed successfully, the writing file will be deleted. // All pieces of the writing file and its mod(if exists) should be retransmitted by the diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverFilePathUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverFilePathUtils.java new file mode 100644 index 0000000000000..bc7275d4ebe16 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverFilePathUtils.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.receiver; + +import java.io.IOException; +import java.nio.file.Path; + +public final class PipeReceiverFilePathUtils { + + private PipeReceiverFilePathUtils() { + // Utility class + } + + public static Path resolveFilePath(final Path baseDir, final String fileName) throws IOException { + final Path normalizedBaseDir = baseDir.toAbsolutePath().normalize(); + final Path normalizedTargetPath = + normalizedBaseDir.resolve(fileName).toAbsolutePath().normalize(); + + if (!normalizedTargetPath.startsWith(normalizedBaseDir)) { + throw new IOException("Illegal fileName: " + fileName + " (Path traversal detected)"); + } + + return normalizedTargetPath; + } +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiverTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiverTest.java index 6658927b9bb4e..337cece8a4887 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiverTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiverTest.java @@ -21,8 +21,10 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeRequestType; import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferFileSealReqV1; import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferFileSealReqV2; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; @@ -33,6 +35,8 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collections; import java.util.List; public class IoTDBFileReceiverTest { @@ -63,6 +67,25 @@ public void testAllowNormalFileName() throws Exception { } } + @Test + public void testRejectPathTraversalFileNameInSealRequest() throws Exception { + final Path baseDir = Files.createTempDirectory("iotdb-file-receiver-test"); + final DummyFileReceiver receiver = new DummyFileReceiver(baseDir.toFile()); + try { + receiver.createWritingFile("normal.tsfile", false); + + final TPipeTransferResp response = + receiver.sealFiles( + Arrays.asList("../outside.mod", "normal.tsfile"), Arrays.asList(0L, 0L)); + + Assert.assertEquals( + TSStatusCode.PIPE_TRANSFER_FILE_ERROR.getStatusCode(), response.getStatus().getCode()); + Assert.assertTrue(response.getStatus().getMessage().contains("Illegal fileName")); + } finally { + receiver.handleExit(); + } + } + private static class DummyFileReceiver extends IoTDBFileReceiver { DummyFileReceiver(final File baseDir) { @@ -73,6 +96,12 @@ void createWritingFile(final String fileName, final boolean isSingleFile) throws updateWritingFileIfNeeded(fileName, isSingleFile); } + TPipeTransferResp sealFiles(final List fileNames, final List fileLengths) + throws IOException { + return handleTransferFileSealV2( + DummyFileSealReqV2.toTPipeTransferReq(fileNames, fileLengths, Collections.emptyMap())); + } + File getWritingFileInBaseDir(final String fileName) { return receiverFileDirWithIdSuffix.get().toPath().resolve(fileName).toFile(); } @@ -125,4 +154,21 @@ public TPipeTransferResp receive(TPipeTransferReq req) { return null; } } + + private static class DummyFileSealReqV2 extends PipeTransferFileSealReqV2 { + + static DummyFileSealReqV2 toTPipeTransferReq( + final List fileNames, + final List fileLengths, + final java.util.Map parameters) + throws IOException { + return (DummyFileSealReqV2) + new DummyFileSealReqV2().convertToTPipeTransferReq(fileNames, fileLengths, parameters); + } + + @Override + protected PipeRequestType getPlanType() { + return PipeRequestType.TRANSFER_SCHEMA_SNAPSHOT_SEAL; + } + } } From f2953e416fba66fe842dc7219e8eb22d8df7d0c6 Mon Sep 17 00:00:00 2001 From: shizy Date: Thu, 30 Apr 2026 10:25:51 +0800 Subject: [PATCH 022/102] [To dev/1.3] mon: add diagnose information for MemoryNotEnoughException (#17579) --- .../fragment/FragmentInstanceContext.java | 2 +- .../utils/ResourceByPathUtils.java | 15 +++++-- .../memtable/AbstractWritableMemChunk.java | 2 +- .../memtable/AlignedReadOnlyMemChunk.java | 4 +- .../dataregion/memtable/ReadOnlyMemChunk.java | 4 +- .../db/utils/datastructure/AlignedTVList.java | 5 ++- .../iotdb/db/utils/datastructure/TVList.java | 42 ++++++++++++++++++- .../FragmentInstanceExecutionTest.java | 4 +- .../memtable/PrimitiveMemTableTest.java | 2 +- 9 files changed, 64 insertions(+), 16 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java index 08e23eea69c74..a224e5b2c3586 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceContext.java @@ -871,7 +871,7 @@ public void releaseResourceWhenAllDriversAreClosed() { */ private void releaseTVListOwnedByQuery() { for (TVList tvList : tvListSet) { - long tvListRamSize = tvList.calculateRamSize(); + long tvListRamSize = tvList.calculateRamSize().getRamSize(); tvList.lockQueryList(); Set queryContextSet = tvList.getQueryContextSet(); try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index b00c8737bd9b0..f3ada63d96f53 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.query.QueryProcessException; +import org.apache.iotdb.db.queryengine.exception.MemoryNotEnoughException; import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; @@ -148,7 +149,7 @@ protected Map prepareTvListMapForQuery( // mutable tvlist TVList list = memChunk.getWorkingTVList(); TVList cloneList = null; - long tvListRamSize = list.calculateRamSize(); + TVList.RamInfo listRamInfo = list.calculateRamSize(); list.lockQueryList(); try { if (copyTimeFilter != null @@ -189,8 +190,8 @@ protected Map prepareTvListMapForQuery( if (firstQuery instanceof FragmentInstanceContext) { MemoryReservationManager memoryReservationManager = ((FragmentInstanceContext) firstQuery).getMemoryReservationContext(); - memoryReservationManager.reserveMemoryCumulatively(tvListRamSize); - list.setReservedMemoryBytes(tvListRamSize); + memoryReservationManager.reserveMemoryCumulatively(listRamInfo.getRamSize()); + list.setReservedMemoryBytes(listRamInfo.getRamSize()); } list.setOwnerQuery(firstQuery); @@ -200,6 +201,14 @@ protected Map prepareTvListMapForQuery( tvListQueryMap.put(cloneList, cloneList.rowCount()); } } + } catch (MemoryNotEnoughException ex) { + LOGGER.warn( + "Failed to reserve memory for TVList: ramSize {}, timestampsSize {}, arrayMemCost {}, rowCount {}, dataTypes {}", + listRamInfo.getRamSize(), + listRamInfo.getTimestampsSize(), + listRamInfo.getArrayMemCost(), + listRamInfo.getRowCount(), + listRamInfo.getDataTypes()); } finally { list.unlockQueryList(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java index 9db0196a3a113..1b9dbbcae60d7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java @@ -99,7 +99,7 @@ protected void maybeReleaseTvList(TVList tvList) { } private void tryReleaseTvList(TVList tvList) { - long tvListRamSize = tvList.calculateRamSize(); + long tvListRamSize = tvList.calculateRamSize().getRamSize(); tvList.lockQueryList(); try { if (tvList.getQueryContextSet().isEmpty()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index bbab08d444284..30976fb6790f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -122,7 +122,7 @@ public void sortTvLists() { // We must update queryRowCount here, otherwise, it may be used later to build // BitMaps, causing bitmap array size mismatch and possible out of bound. entry.setValue(alignedTvList.sort()); - long alignedTvListRamSize = alignedTvList.calculateRamSize(); + long alignedTvListRamSize = alignedTvList.calculateRamSize().getRamSize(); alignedTvList.lockQueryList(); try { FragmentInstanceContext ownerQuery = @@ -367,7 +367,7 @@ public IPointReader getPointReader() { int queryLength = entry.getValue(); if (!alignedTvList.isSorted() && queryLength > alignedTvList.seqRowCount()) { entry.setValue(alignedTvList.sort()); - long alignedTvListRamSize = alignedTvList.calculateRamSize(); + long alignedTvListRamSize = alignedTvList.calculateRamSize().getRamSize(); alignedTvList.lockQueryList(); try { FragmentInstanceContext ownerQuery = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index 38c8e9e2d13cc..9e46740a759c8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -136,7 +136,7 @@ public void sortTvLists() { int queryRowCount = entry.getValue(); if (!tvList.isSorted() && queryRowCount > tvList.seqRowCount()) { entry.setValue(tvList.sort()); - long tvListRamSize = tvList.calculateRamSize(); + long tvListRamSize = tvList.calculateRamSize().getRamSize(); tvList.lockQueryList(); try { FragmentInstanceContext ownerQuery = (FragmentInstanceContext) tvList.getOwnerQuery(); @@ -288,7 +288,7 @@ public IPointReader getPointReader() { int queryLength = entry.getValue(); if (!tvList.isSorted() && queryLength > tvList.seqRowCount()) { entry.setValue(tvList.sort()); - long tvListRamSize = tvList.calculateRamSize(); + long tvListRamSize = tvList.calculateRamSize().getRamSize(); tvList.lockQueryList(); try { FragmentInstanceContext ownerQuery = (FragmentInstanceContext) tvList.getOwnerQuery(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 090b057498748..615bfcaf4cab0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -836,8 +836,9 @@ public TSDataType getDataType() { } @Override - public synchronized long calculateRamSize() { - return timestamps.size() * alignedTvListArrayMemCost(); + public synchronized RamInfo calculateRamSize() { + return new RamInfo( + timestamps.size(), alignedTvListArrayMemCost(), rowCount, new ArrayList<>(dataTypes)); } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index 0f4ab88441aa6..b4fd0d52221e1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.storageengine.rescon.memory.PrimitiveArrayManager; import org.apache.iotdb.db.utils.MathUtils; +import com.google.common.collect.ImmutableList; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.read.TimeValuePair; @@ -59,6 +60,42 @@ public abstract class TVList implements WALEntryValue { protected static final String ERR_DATATYPE_NOT_CONSISTENT = "DataType not consistent"; + + public static class RamInfo { + private final int timestampsSize; + private final long arrayMemCost; + private final int rowCount; + private final List dataTypes; + + public RamInfo( + int timestampCount, long arrayMemCost, int rowCount, List dataTypes) { + this.timestampsSize = timestampCount; + this.rowCount = rowCount; + this.arrayMemCost = arrayMemCost; + this.dataTypes = dataTypes; + } + + public long getRamSize() { + return timestampsSize * arrayMemCost; + } + + public int getTimestampsSize() { + return timestampsSize; + } + + public int getRowCount() { + return rowCount; + } + + public long getArrayMemCost() { + return arrayMemCost; + } + + public List getDataTypes() { + return dataTypes; + } + } + // list of timestamp array, add 1 when expanded -> data point timestamp array // index relation: arrayIndex -> elementIndex protected List timestamps; @@ -156,8 +193,9 @@ public static long tvListArrayMemCost(TSDataType type) { return size; } - public synchronized long calculateRamSize() { - return timestamps.size() * tvListArrayMemCost(); + public synchronized RamInfo calculateRamSize() { + return new RamInfo( + timestamps.size(), tvListArrayMemCost(), rowCount, ImmutableList.of(getDataType())); } public synchronized boolean isSorted() { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecutionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecutionTest.java index 30b9df14b3acb..22e5c360ae3c5 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecutionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecutionTest.java @@ -145,7 +145,7 @@ public void testTVListOwnerTransfer() throws InterruptedException { // mock flush's behavior fragmentInstanceContext1 .getMemoryReservationContext() - .reserveMemoryCumulatively(tvList.calculateRamSize()); + .reserveMemoryCumulatively(tvList.calculateRamSize().getRamSize()); tvList.setOwnerQuery(fragmentInstanceContext1); fragmentInstanceContext1.decrementNumOfUnClosedDriver(); @@ -226,7 +226,7 @@ public void testTVListCloneForQuery() { pointReader.nextTimeValuePair(); } assertTrue(tvList.isSorted()); - assertEquals(tvList.calculateRamSize(), tvList.getReservedMemoryBytes()); + assertEquals(tvList.calculateRamSize().getRamSize(), tvList.getReservedMemoryBytes()); } catch (QueryProcessException | IOException | MetadataException diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java index f9186b774e1f3..ff476c60e4e5d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java @@ -679,7 +679,7 @@ public void testReleaseWithNotEnoughMemory() throws CpuNotEnoughException { Mockito.mock(MemoryReservationManager.class); Mockito.doThrow(new MemoryNotEnoughException("")) .when(memoryReservationManager) - .reserveMemoryCumulatively(list.calculateRamSize()); + .reserveMemoryCumulatively(list.calculateRamSize().getRamSize()); // create FragmentInstanceId QueryId queryId = new QueryId("stub_query"); From 250d97a32a16a8334dbb1012638bab4cafdf453a Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 30 Apr 2026 11:19:04 +0800 Subject: [PATCH 023/102] [To dev/1.3] Pipe: Fixed the bug that Disruptor may not clear the reference & will wait long time after pipe close (#17549) (#17569) * Pipe: Implementing DisruptorQueue (#16639) * Pipe: Fixed the bug that Disruptor may not clear the reference & will wait long time after pipe close (#17549) * fix * fix * del --------- Co-authored-by: Zhenyu Luo --- LICENSE | 12 + iotdb-core/datanode/pom.xml | 4 - .../common/heartbeat/PipeHeartbeatEvent.java | 4 +- .../realtime/assigner/DisruptorQueue.java | 15 +- .../DisruptorQueueExceptionHandler.java | 3 +- .../assigner/PipeDataRegionAssigner.java | 2 +- .../disruptor/BatchEventProcessor.java | 149 +++++++++ .../realtime/disruptor/Disruptor.java | 139 +++++++++ .../realtime/disruptor/EventFactory.java | 38 +++ .../realtime/disruptor/EventHandler.java | 41 +++ .../realtime/disruptor/ExceptionHandler.java | 45 +++ .../disruptor/MultiProducerSequencer.java | 259 +++++++++++++++ .../realtime/disruptor/RingBuffer.java | 295 ++++++++++++++++++ .../realtime/disruptor/Sequence.java | 122 ++++++++ .../realtime/disruptor/SequenceBarrier.java | 82 +++++ .../realtime/disruptor/SequenceGroups.java | 77 +++++ .../PipeInsertionDataNodeListener.java | 45 +-- .../disruptor/DisruptorShutdownTest.java | 183 +++++++++++ pom.xml | 6 - 19 files changed, 1480 insertions(+), 41 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/BatchEventProcessor.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/Disruptor.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/EventFactory.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/EventHandler.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/ExceptionHandler.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/MultiProducerSequencer.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/RingBuffer.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/Sequence.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/SequenceBarrier.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/SequenceGroups.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/DisruptorShutdownTest.java diff --git a/LICENSE b/LICENSE index 3f3de45e81a7c..0a1e5e42a9ec1 100644 --- a/LICENSE +++ b/LICENSE @@ -303,3 +303,15 @@ The following files include code modified from Dropwizard Metrics project. Copyright (c) 2010-2013 Coda Hale, Yammer.com, 2014-2021 Dropwizard Team Project page: https://github.com/dropwizard/metrics License: https://github.com/dropwizard/metrics/blob/release/4.2.x/LICENSE + +-------------------------------------------------------------------------------- + +The following files include code modified from LMax Disruptor project. + +./iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/* + +LMax Disruptor is open source software licensed under the Apache License 2.0 and supported by the Apache Software Foundation. +Project page: https://github.com/LMAX-Exchange/disruptor +License: https://github.com/LMAX-Exchange/disruptor/blob/master/LICENCE.txt + +-------------------------------------------------------------------------------- diff --git a/iotdb-core/datanode/pom.xml b/iotdb-core/datanode/pom.xml index d6f8d12434581..299e50f1be932 100644 --- a/iotdb-core/datanode/pom.xml +++ b/iotdb-core/datanode/pom.xml @@ -308,10 +308,6 @@ com.google.guava guava - - com.lmax - disruptor - org.java-websocket Java-WebSocket diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java index ad29b2854424a..567e14f4fe386 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java @@ -27,10 +27,10 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeHeartbeatEventMetrics; +import org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor.RingBuffer; import org.apache.iotdb.db.utils.DateTimeUtils; import org.apache.iotdb.pipe.api.event.Event; -import com.lmax.disruptor.RingBuffer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -183,7 +183,7 @@ public void onTransferred() { /////////////////////////////// Queue size Reporting /////////////////////////////// - public void recordDisruptorSize(final RingBuffer ringBuffer) { + public void recordDisruptorSize(final RingBuffer ringBuffer) { if (shouldPrintMessage) { disruptorSize = ringBuffer.getBufferSize() - (int) ringBuffer.remainingCapacity(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java index 2a2fa11074989..52ac137ae4e6f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java @@ -26,12 +26,10 @@ import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; +import org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor.Disruptor; +import org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor.EventHandler; +import org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor.RingBuffer; -import com.lmax.disruptor.BlockingWaitStrategy; -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.RingBuffer; -import com.lmax.disruptor.dsl.Disruptor; -import com.lmax.disruptor.dsl.ProducerType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,9 +73,8 @@ public DisruptorQueue( 32, Math.toIntExact( allocatedMemoryBlock.getMemoryUsageInBytes() / ringBufferEntrySizeInBytes)), - THREAD_FACTORY, - ProducerType.MULTI, - new BlockingWaitStrategy()); + THREAD_FACTORY); + disruptor.handleEventsWith( (container, sequence, endOfBatch) -> { final PipeRealtimeEvent realtimeEvent = container.getEvent(); @@ -127,7 +124,7 @@ private void mayPrintExceedingLog() { private static class EventContainer { - private PipeRealtimeEvent event; + private volatile PipeRealtimeEvent event; private EventContainer() {} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueueExceptionHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueueExceptionHandler.java index 91ad0224fc538..5330f3486f50e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueueExceptionHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueueExceptionHandler.java @@ -19,7 +19,8 @@ package org.apache.iotdb.db.pipe.source.dataregion.realtime.assigner; -import com.lmax.disruptor.ExceptionHandler; +import org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor.ExceptionHandler; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java index e3067a13126eb..0b4eb547144af 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -202,7 +202,7 @@ public void stopAssignTo(final PipeRealtimeDataRegionSource extractor) { matcher.deregister(extractor); } - public boolean notMoreExtractorNeededToBeAssigned() { + public boolean notMoreSourceNeededToBeAssigned() { return matcher.getRegisterCount() == 0; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/BatchEventProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/BatchEventProcessor.java new file mode 100644 index 0000000000000..d0432821cf77c --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/BatchEventProcessor.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Batch event processor for consuming events + * + *

This implementation is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) + * and simplified for IoTDB's Pipe module (removed complex lifecycle management). + * + *

Core algorithm preserved from LMAX Disruptor: + * + *

    + *
  • Batch processing loop + *
  • Sequence tracking + *
  • endOfBatch detection + *
+ * + * @param event type + */ +public final class BatchEventProcessor implements Runnable { + private static final Logger LOGGER = LoggerFactory.getLogger(BatchEventProcessor.class); + + private final RingBuffer ringBuffer; + private final SequenceBarrier sequenceBarrier; + private final EventHandler eventHandler; + private final Sequence sequence = new Sequence(); + private ExceptionHandler exceptionHandler = new DefaultExceptionHandler<>(); + private volatile boolean running = true; + + public BatchEventProcessor( + RingBuffer ringBuffer, SequenceBarrier barrier, EventHandler eventHandler) { + this.ringBuffer = ringBuffer; + this.sequenceBarrier = barrier; + this.eventHandler = eventHandler; + } + + public Sequence getSequence() { + return sequence; + } + + public void setExceptionHandler(ExceptionHandler exceptionHandler) { + this.exceptionHandler = exceptionHandler; + } + + public void halt() { + running = false; + } + + @Override + public void run() { + long nextSequence = sequence.get() + 1L; + + while (running) { + try { + // Wait for available sequence + final long availableSequence = sequenceBarrier.waitFor(nextSequence); + + // Batch process all available events + nextSequence = processAvailableEvents(nextSequence, availableSequence); + + } catch (final InterruptedException ex) { + if (running) { + Thread.currentThread().interrupt(); + LOGGER.info("Processor interrupted"); + } + break; + } catch (final Throwable ex) { + exceptionHandler.handleEventException(ex, nextSequence, ringBuffer.get(nextSequence)); + sequence.set(nextSequence); + nextSequence++; + } + } + + if (!running) { + drainRemainingPublishedEvents(nextSequence); + } + LOGGER.info("Processor stopped"); + } + + private long processAvailableEvents(long nextSequence, long availableSequence) throws Throwable { + while (nextSequence <= availableSequence) { + final T event = ringBuffer.get(nextSequence); + eventHandler.onEvent(event, nextSequence, nextSequence == availableSequence); + nextSequence++; + } + + sequence.set(availableSequence); + return nextSequence; + } + + private void drainRemainingPublishedEvents(long nextSequence) { + final long availableSequence = sequenceBarrier.getCursor(); + if (availableSequence < nextSequence) { + return; + } + + final long highestPublishedSequence = + sequenceBarrier.getHighestPublishedSequence(nextSequence, availableSequence); + while (nextSequence <= highestPublishedSequence) { + final T event = ringBuffer.get(nextSequence); + try { + eventHandler.onEvent(event, nextSequence, nextSequence == highestPublishedSequence); + } catch (final Throwable ex) { + exceptionHandler.handleEventException(ex, nextSequence, event); + } finally { + sequence.set(nextSequence); + } + nextSequence++; + } + } + + private static class DefaultExceptionHandler implements ExceptionHandler { + @Override + public void handleEventException(Throwable ex, long sequence, T event) { + LoggerFactory.getLogger(getClass()).error("Exception processing: {} {}", sequence, event, ex); + } + + @Override + public void handleOnStartException(Throwable ex) { + LoggerFactory.getLogger(getClass()).error("Exception during onStart()", ex); + } + + @Override + public void handleOnShutdownException(Throwable ex) { + LoggerFactory.getLogger(getClass()).error("Exception during onShutdown()", ex); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/Disruptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/Disruptor.java new file mode 100644 index 0000000000000..f3a647012856e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/Disruptor.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ThreadFactory; + +/** + * Simplified Disruptor implementation for IoTDB Pipe + * + *

This implementation is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) + * and simplified for IoTDB's specific use case in the Pipe module. + * + *

Key simplifications: + * + *

    + *
  • Single event handler support (no complex dependency graphs) + *
  • Simplified lifecycle management + *
  • Removed wait strategies (using simple sleep-based waiting) + *
+ * + * @param event type + */ +public class Disruptor { + private static final Logger LOGGER = LoggerFactory.getLogger(Disruptor.class); + + private final RingBuffer ringBuffer; + private final ThreadFactory threadFactory; + private BatchEventProcessor processor; + private Thread processorThread; + private ExceptionHandler exceptionHandler; + private volatile boolean started = false; + + /** + * Create a Disruptor instance + * + * @param eventFactory factory for creating pre-allocated events + * @param ringBufferSize buffer size (must be power of 2) + * @param threadFactory factory for creating consumer thread + */ + public Disruptor(EventFactory eventFactory, int ringBufferSize, ThreadFactory threadFactory) { + this.ringBuffer = RingBuffer.createMultiProducer(eventFactory, ringBufferSize); + this.threadFactory = threadFactory; + } + + /** + * Configure event handler for processing events + * + *

Creates a batch event processor that will run in its own thread + * + * @param handler event handler implementation + * @return this instance for method chaining + */ + public Disruptor handleEventsWith(final EventHandler handler) { + SequenceBarrier barrier = ringBuffer.newBarrier(); + processor = new BatchEventProcessor<>(ringBuffer, barrier, handler); + + if (exceptionHandler != null) { + processor.setExceptionHandler(exceptionHandler); + } + + ringBuffer.addGatingSequences(processor.getSequence()); + return this; + } + + /** + * Set exception handler for error handling + * + * @param exceptionHandler handler for processing exceptions + */ + public void setDefaultExceptionHandler(ExceptionHandler exceptionHandler) { + this.exceptionHandler = exceptionHandler; + if (processor != null) { + processor.setExceptionHandler(exceptionHandler); + } + } + + public RingBuffer start() { + if (started) { + throw new IllegalStateException("Disruptor already started"); + } + + if (processor == null) { + throw new IllegalStateException("No event handler configured"); + } + + processorThread = threadFactory.newThread(processor); + processorThread.start(); + started = true; + + LOGGER.info("Disruptor started with buffer size: {}", ringBuffer.getBufferSize()); + return ringBuffer; + } + + public void shutdown() { + if (!started) { + return; + } + + if (processor != null) { + processor.halt(); + } + + if (processorThread != null) { + try { + processorThread.interrupt(); + processorThread.join(5000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("Interrupted waiting for processor to stop"); + } + if (processorThread.isAlive()) { + LOGGER.warn("Timed out waiting for processor to stop"); + } + } + + started = false; + LOGGER.info("Disruptor shutdown completed"); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/EventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/EventFactory.java new file mode 100644 index 0000000000000..785033c68247b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/EventFactory.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +/** + * Event factory for pre-allocating events in RingBuffer + * + *

This interface is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) and + * adapted for IoTDB's Pipe module. + * + * @param event type + */ +@FunctionalInterface +public interface EventFactory { + /** + * Create new event instance + * + * @return new event + */ + T newInstance(); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/EventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/EventHandler.java new file mode 100644 index 0000000000000..1fc81a3762373 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/EventHandler.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +/** + * Event handler for processing events from RingBuffer + * + *

This interface is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) and + * adapted for IoTDB's Pipe module. + * + * @param event type + */ +@FunctionalInterface +public interface EventHandler { + /** + * Handle event + * + * @param event the event + * @param sequence sequence number + * @param endOfBatch whether this is the last event in current batch + * @throws Exception if processing fails + */ + void onEvent(T event, long sequence, boolean endOfBatch) throws Exception; +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/ExceptionHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/ExceptionHandler.java new file mode 100644 index 0000000000000..28396b51ffed1 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/ExceptionHandler.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +/** + * Exception handler for event processing errors + * + *

This interface is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) and + * adapted for IoTDB's Pipe module. + * + * @param event type + */ +public interface ExceptionHandler { + /** + * Handle exception during event processing + * + * @param ex exception + * @param sequence sequence number + * @param event the event + */ + void handleEventException(Throwable ex, long sequence, T event); + + /** Handle exception during processor start */ + void handleOnStartException(Throwable ex); + + /** Handle exception during processor shutdown */ + void handleOnShutdownException(Throwable ex); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/MultiProducerSequencer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/MultiProducerSequencer.java new file mode 100644 index 0000000000000..d40ed96839870 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/MultiProducerSequencer.java @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +import java.util.concurrent.atomic.AtomicIntegerArray; +import java.util.concurrent.locks.LockSupport; + +/** + * Multi-producer sequencer for coordinating concurrent publishers + * + *

This implementation is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) + * and preserves the core lock-free multi-producer algorithm for IoTDB's Pipe module. + * + *

Key features preserved from LMAX Disruptor: + * + *

    + *
  • Lock-free CAS-based sequence claiming + *
  • Availability buffer for out-of-order publishing detection + *
  • Backpressure via gating sequences + *
  • Cache line padding to prevent false sharing + *
+ */ +public final class MultiProducerSequencer { + + /** Ring buffer size (must be power of 2) - immutable after construction */ + private final int bufferSize; + + /** + * Producer cursor tracking highest claimed sequence Updated via CAS in next() method Volatile + * reads/writes handled by Sequence class + */ + private final Sequence cursor = new Sequence(); + + /** + * Array of consumer sequences for backpressure control MUST be volatile for safe publication when + * modified by SequenceGroups Array reference is replaced atomically via + * AtomicReferenceFieldUpdater + */ + volatile Sequence[] gatingSequences; + + /** + * Cached minimum gating sequence to reduce contention Updated opportunistically in next() to + * avoid expensive array scan Does not need to be perfectly accurate (conservative is safe) + */ + private final Sequence gatingSequenceCache = new Sequence(); + + /** + * CRITICAL: Availability flags for tracking published sequences + * + *

Handles out-of-order publishing in multi-producer scenario: - Thread A claims seq 10, still + * writing - Thread B claims seq 11, finishes and publishes - Consumer MUST wait for seq 10 before + * reading seq 11 + * + *

Memory visibility guarantees: - Writers use lazySet() for store-store barrier (cheaper than + * volatile write) - Readers use get() for volatile read (ensures visibility across threads) + * + *

AtomicIntegerArray provides same semantics as Unsafe without reflection + */ + private final AtomicIntegerArray availableBuffer; + + /** Mask for fast modulo: sequence & indexMask == sequence % bufferSize */ + private final int indexMask; + + /** Shift for calculating wrap count: sequence >>> indexShift */ + private final int indexShift; + + public MultiProducerSequencer(int bufferSize, Sequence[] gatingSequences) { + if (bufferSize < 1) { + throw new IllegalArgumentException("bufferSize must not be less than 1"); + } + if (Integer.bitCount(bufferSize) != 1) { + throw new IllegalArgumentException("bufferSize must be a power of 2"); + } + + this.bufferSize = bufferSize; + this.gatingSequences = gatingSequences != null ? gatingSequences : new Sequence[0]; + this.availableBuffer = new AtomicIntegerArray(bufferSize); + this.indexMask = bufferSize - 1; + this.indexShift = log2(bufferSize); + + initialiseAvailableBuffer(); + } + + /** + * Claim next n sequences for publishing + * + *

Uses CAS loop to atomically claim sequence numbers. Implements backpressure by parking when + * buffer is full. + * + * @param n number of sequences to claim + * @return highest claimed sequence number + */ + public long next(int n) { + if (n < 1) { + throw new IllegalArgumentException("n must be > 0"); + } + + long current; + long next; + + do { + current = cursor.get(); + next = current + n; + + final long wrapPoint = next - bufferSize; + final long cachedGatingSequence = gatingSequenceCache.get(); + + if (wrapPoint > cachedGatingSequence || cachedGatingSequence > current) { + long gatingSequence = Sequence.getMinimumSequence(gatingSequences, current); + + if (wrapPoint > gatingSequence) { + LockSupport.parkNanos(1); + continue; + } + + gatingSequenceCache.set(gatingSequence); + } else if (cursor.compareAndSet(current, next)) { + break; + } + } while (true); + + return next; + } + + /** Publish sequence */ + public void publish(final long sequence) { + setAvailable(sequence); + } + + /** Publish batch */ + public void publish(long lo, long hi) { + for (long l = lo; l <= hi; l++) { + setAvailable(l); + } + } + + /** + * CORE: Check if sequence is available for consumption Uses volatile read to ensure visibility of + * published sequences + */ + public boolean isAvailable(long sequence) { + int index = calculateIndex(sequence); + int flag = calculateAvailabilityFlag(sequence); + return availableBuffer.get(index) == flag; + } + + /** CORE: Get highest published - exact same algorithm */ + public long getHighestPublishedSequence(long lowerBound, long availableSequence) { + for (long sequence = lowerBound; sequence <= availableSequence; sequence++) { + if (!isAvailable(sequence)) { + return sequence - 1; + } + } + return availableSequence; + } + + public Sequence getCursor() { + return cursor; + } + + public int getBufferSize() { + return bufferSize; + } + + public long remainingCapacity() { + long consumed = Sequence.getMinimumSequence(gatingSequences, cursor.get()); + long produced = cursor.get(); + return bufferSize - (produced - consumed); + } + + /** + * Add gating sequences for consumer tracking + * + *

Atomically adds sequences to track consumer progress + * + * @param gatingSequences consumer sequences to add + */ + public void addGatingSequences(Sequence... gatingSequences) { + SequenceGroups.addSequences(this, this.cursor, gatingSequences); + } + + /** + * Create a sequence barrier for consumers + * + * @param sequencesToTrack upstream sequences to wait for + * @return new sequence barrier + */ + public SequenceBarrier newBarrier(Sequence... sequencesToTrack) { + return new SequenceBarrier(this, sequencesToTrack); + } + + /** Initialize available buffer */ + private void initialiseAvailableBuffer() { + for (int i = availableBuffer.length() - 1; i != 0; i--) { + setAvailableBufferValue(i, -1); + } + setAvailableBufferValue(0, -1); + } + + /** + * CORE: Mark sequence as available for consumption + * + *

Uses lazySet() which provides: - Store-store barrier (ensures all prior writes are visible) + * - Cheaper than full volatile write (no store-load barrier) - Sufficient for this use case + * (readers use volatile get) + */ + private void setAvailable(final long sequence) { + setAvailableBufferValue(calculateIndex(sequence), calculateAvailabilityFlag(sequence)); + } + + /** + * Set availability flag with release semantics lazySet() ensures previous event writes are + * visible before flag update + */ + private void setAvailableBufferValue(int index, int flag) { + availableBuffer.lazySet(index, flag); + } + + /** Calculate availability flag */ + private int calculateAvailabilityFlag(final long sequence) { + return (int) (sequence >>> indexShift); + } + + /** Calculate index */ + private int calculateIndex(final long sequence) { + return ((int) sequence) & indexMask; + } + + /** + * Calculate log2 for index shift calculation + * + * @param i input value (must be power of 2) + * @return log2 of input + */ + private static int log2(int i) { + int r = 0; + while ((i >>= 1) != 0) { + ++r; + } + return r; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/RingBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/RingBuffer.java new file mode 100644 index 0000000000000..2af784b603d6c --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/RingBuffer.java @@ -0,0 +1,295 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +/** + * Left-hand side padding for cache line alignment + * + *

Prevents false sharing by ensuring RingBuffer fields don't share cache lines with preceding + * objects + */ +abstract class RingBufferPad { + protected long p1, p2, p3, p4, p5, p6, p7; +} + +/** + * Core fields for RingBuffer implementation + * + *

Contains the actual event storage array and sequencing state + */ +abstract class RingBufferFields extends RingBufferPad { + /** Pre-allocated event storage with padding to prevent false sharing */ + private final Object[] entries; + + /** Total number of events in the buffer (must be power of 2) */ + protected final int bufferSize; + + /** Mask for fast modulo operation (bufferSize - 1) */ + protected final int indexMask; + + /** Sequencer for managing producer/consumer coordination */ + protected final MultiProducerSequencer sequencer; + + /** + * Initialize ring buffer fields + * + * @param eventFactory factory for pre-allocating events + * @param sequencer multi-producer sequencer + */ + RingBufferFields(EventFactory eventFactory, MultiProducerSequencer sequencer) { + this.sequencer = sequencer; + this.bufferSize = sequencer.getBufferSize(); + + if (bufferSize < 1) { + throw new IllegalArgumentException("bufferSize must not be less than 1"); + } + if (Integer.bitCount(bufferSize) != 1) { + throw new IllegalArgumentException("bufferSize must be a power of 2"); + } + + this.indexMask = bufferSize - 1; + // Allocate array with padding on both sides to prevent false sharing + this.entries = new Object[bufferSize]; + fill(eventFactory); + } + + /** + * Pre-allocate all events in the buffer + * + * @param eventFactory factory for creating event instances + */ + private void fill(EventFactory eventFactory) { + for (int i = 0; i < bufferSize; i++) { + // Store events starting after front padding + entries[i] = eventFactory.newInstance(); + } + } + + /** + * Get event at sequence using direct memory access + * + * @param sequence sequence number + * @return event at the sequence position + */ + @SuppressWarnings("unchecked") + protected final E elementAt(long sequence) { + // Use Unsafe for lock-free array access with proper memory barriers + return (E) entries[(int) (sequence & indexMask)]; + } +} + +/** + * Lock-free ring buffer for storing pre-allocated event objects + * + *

This implementation is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) + * and preserves the core ring buffer algorithm for IoTDB's Pipe module. + * + *

Supports multi-producer concurrent access with zero-garbage design. Events are pre-allocated + * and reused, avoiding GC pressure. Uses cache line padding to prevent false sharing. + * + * @param event type + */ +public final class RingBuffer extends RingBufferFields { + /** Initial cursor value for the ring buffer */ + public static final long INITIAL_CURSOR_VALUE = Sequence.INITIAL_VALUE; + + /** + * Right-hand side padding for cache line alignment + * + *

Prevents false sharing by ensuring RingBuffer fields don't share cache lines with following + * objects + */ + protected long p1, p2, p3, p4, p5, p6, p7; + + /** + * Construct a RingBuffer with given factory and sequencer + * + * @param eventFactory factory to create and pre-allocate events + * @param sequencer multi-producer sequencer for sequence management + */ + private RingBuffer(EventFactory eventFactory, MultiProducerSequencer sequencer) { + super(eventFactory, sequencer); + } + + /** + * Create a multi-producer RingBuffer + * + *

Supports concurrent publishing from multiple threads using lock-free CAS operations + * + * @param factory event factory for creating event instances + * @param bufferSize buffer size (must be power of 2) + * @param event type + * @return newly created ring buffer + */ + public static RingBuffer createMultiProducer(EventFactory factory, int bufferSize) { + MultiProducerSequencer sequencer = new MultiProducerSequencer(bufferSize, new Sequence[0]); + return new RingBuffer<>(factory, sequencer); + } + + /** + * Get the event at a specific sequence + * + * @param sequence sequence number to retrieve + * @return event at the given sequence + */ + public E get(long sequence) { + return elementAt(sequence); + } + + /** + * Claim the next sequence for publishing + * + *

Blocks if buffer is full until space becomes available + * + * @return claimed sequence number + */ + public long next() { + return sequencer.next(1); + } + + /** + * Claim next n sequences for batch publishing + * + * @param n number of sequences to claim + * @return highest claimed sequence number + */ + public long next(int n) { + return sequencer.next(n); + } + + /** + * Publish a single sequence + * + *

Makes the event at this sequence visible to consumers + * + * @param sequence sequence to publish + */ + public void publish(long sequence) { + sequencer.publish(sequence); + } + + /** + * Publish a batch of sequences + * + * @param lo lowest sequence in the batch (inclusive) + * @param hi highest sequence in the batch (inclusive) + */ + public void publish(long lo, long hi) { + sequencer.publish(lo, hi); + } + + /** + * Publish event using a translator function + * + *

Provides a higher-level API for publishing events with custom translation logic + * + * @param translator function to populate the event + * @param arg0 argument passed to translator + * @param argument type + */ + public void publishEvent(EventTranslator translator, A arg0) { + final long sequence = sequencer.next(1); + translateAndPublish(translator, sequence, arg0); + } + + /** + * Translate event and publish atomically + * + * @param translator event translator function + * @param sequence claimed sequence number + * @param arg0 argument for translation + * @param argument type + */ + private void translateAndPublish(EventTranslator translator, long sequence, A arg0) { + try { + translator.translateTo(get(sequence), sequence, arg0); + } finally { + sequencer.publish(sequence); + } + } + + /** + * Add gating sequences for consumer tracking + * + *

Gating sequences represent consumer progress and prevent overwriting unprocessed events + * + * @param gatingSequences consumer sequences to track + */ + public void addGatingSequences(Sequence... gatingSequences) { + sequencer.addGatingSequences(gatingSequences); + } + + /** + * Create a sequence barrier for consumers + * + *

Barrier coordinates when events become available for processing + * + * @param sequencesToTrack upstream sequences to wait for + * @return new sequence barrier + */ + public SequenceBarrier newBarrier(Sequence... sequencesToTrack) { + return sequencer.newBarrier(sequencesToTrack); + } + + /** + * Get current producer cursor position + * + * @return current cursor value + */ + public long getCursor() { + return sequencer.getCursor().get(); + } + + /** + * Get the buffer size + * + * @return configured buffer size + */ + public int getBufferSize() { + return bufferSize; + } + + /** + * Get remaining capacity in the buffer + * + * @return number of available slots + */ + public long remainingCapacity() { + return sequencer.remainingCapacity(); + } + + /** + * Function interface for translating data into events + * + * @param event type + * @param argument type + */ + @FunctionalInterface + public interface EventTranslator { + /** + * Translate argument into event + * + * @param event pre-allocated event to populate + * @param sequence sequence number for this event + * @param arg source data + */ + void translateTo(E event, long sequence, A arg); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/Sequence.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/Sequence.java new file mode 100644 index 0000000000000..1f1d3445969f1 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/Sequence.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +import java.util.concurrent.atomic.AtomicLong; + +/** Left-hand side padding for cache line alignment */ +class LhsPadding { + protected long p1, p2, p3, p4, p5, p6, p7; +} + +/** Value class holding the actual sequence */ +class Value extends LhsPadding { + protected AtomicLong value = new AtomicLong(); +} + +/** Right-hand side padding for cache line alignment */ +class RhsPadding extends Value { + protected long p9, p10, p11, p12, p13, p14, p15; +} + +/** + * Lock-free sequence counter with cache line padding + * + *

This implementation is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) + * and preserves the core sequence tracking mechanism for IoTDB's Pipe module. + * + *

Key design features: + * + *

+ */ +public class Sequence extends RhsPadding { + public static final long INITIAL_VALUE = -1L; + + /** Create sequence with initial value -1 */ + public Sequence() { + value.set(INITIAL_VALUE); + } + + /** Volatile read */ + public long get() { + return value.get(); + } + + /** + * Ordered write (store-store barrier only) + * + *

CRITICAL: Cheaper than volatile write, sufficient for most cases + */ + public void set(final long value) { + this.value.set(value); + } + + /** + * CAS operation - CORE for lock-free design + * + * @param expectedValue expected current value + * @param newValue new value + * @return true if successful + */ + public boolean compareAndSet(final long expectedValue, final long newValue) { + return value.compareAndSet(expectedValue, newValue); + } + + /** Atomically increment */ + public long incrementAndGet() { + return addAndGet(1L); + } + + /** Atomically add */ + public long addAndGet(final long increment) { + long currentValue; + long newValue; + + do { + currentValue = get(); + newValue = currentValue + increment; + } while (!compareAndSet(currentValue, newValue)); + + return newValue; + } + + @Override + public String toString() { + return Long.toString(get()); + } + + /** Get minimum sequence from array - CORE utility method */ + public static long getMinimumSequence(final Sequence[] sequences, long minimum) { + for (int i = 0, n = sequences.length; i < n; i++) { + long value = sequences[i].get(); + minimum = Math.min(minimum, value); + } + return minimum; + } + + public static long getMinimumSequence(final Sequence[] sequences) { + return getMinimumSequence(sequences, Long.MAX_VALUE); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/SequenceBarrier.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/SequenceBarrier.java new file mode 100644 index 0000000000000..80f41162fc73a --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/SequenceBarrier.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +/** + * Sequence barrier for consumer coordination + * + *

This implementation is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) + * and simplified for IoTDB's Pipe module (removed Alert mechanism - IoTDB doesn't need it). + * + *

Core features preserved from LMAX Disruptor: + * + *

    + *
  • waitFor() logic for waiting sequences + *
  • Scan available buffer for out-of-order publishing + *
+ */ +public class SequenceBarrier { + private final MultiProducerSequencer sequencer; + private final Sequence[] dependentSequences; + + public SequenceBarrier(MultiProducerSequencer sequencer, Sequence[] dependentSequences) { + this.sequencer = sequencer; + this.dependentSequences = dependentSequences != null ? dependentSequences : new Sequence[0]; + } + + /** + * CORE: Wait for sequence to become available (MUST keep logic) + * + * @param sequence sequence to wait for + * @return highest available sequence + * @throws InterruptedException if interrupted + */ + public long waitFor(long sequence) throws InterruptedException { + // Wait for cursor + long availableSequence; + while ((availableSequence = sequencer.getCursor().get()) < sequence) { + Thread.sleep(1); + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException(); + } + } + + // Wait for dependent sequences + if (dependentSequences.length > 0) { + while (Sequence.getMinimumSequence(dependentSequences) < sequence) { + Thread.sleep(1); + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException(); + } + } + } + + // CORE: Scan available buffer for highest continuously published sequence + return sequencer.getHighestPublishedSequence(sequence, availableSequence); + } + + public long getCursor() { + return sequencer.getCursor().get(); + } + + public long getHighestPublishedSequence(long lowerBound, long availableSequence) { + return sequencer.getHighestPublishedSequence(lowerBound, availableSequence); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/SequenceGroups.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/SequenceGroups.java new file mode 100644 index 0000000000000..af5039070f0e4 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/SequenceGroups.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; + +/** + * Utility for atomic management of sequence arrays + * + *

This implementation is based on LMAX Disruptor (https://github.com/LMAX-Exchange/disruptor) + * and adapted for IoTDB's Pipe module. + * + *

Provides thread-safe operations for adding and removing sequences from gating sequence arrays + * used to track consumer progress. + */ +final class SequenceGroups { + + /** Field updater for atomic array replacement */ + private static final AtomicReferenceFieldUpdater + SEQUENCE_UPDATER = + AtomicReferenceFieldUpdater.newUpdater( + MultiProducerSequencer.class, Sequence[].class, "gatingSequences"); + + /** + * Atomically add sequences to the gating sequence array + * + *

Uses CAS loop to ensure thread-safe addition even under concurrent modification + * + * @param sequencer the multi-producer sequencer + * @param cursor the current cursor sequence + * @param sequencesToAdd sequences to add + */ + static void addSequences( + final MultiProducerSequencer sequencer, + final Sequence cursor, + final Sequence... sequencesToAdd) { + long cursorSequence; + Sequence[] updatedSequences; + Sequence[] currentSequences; + + do { + currentSequences = sequencer.gatingSequences; + updatedSequences = new Sequence[currentSequences.length + sequencesToAdd.length]; + System.arraycopy(currentSequences, 0, updatedSequences, 0, currentSequences.length); + + cursorSequence = cursor.get(); + + int index = currentSequences.length; + for (Sequence sequence : sequencesToAdd) { + sequence.set(cursorSequence); + updatedSequences[index++] = sequence; + } + } while (!SEQUENCE_UPDATER.compareAndSet(sequencer, currentSequences, updatedSequences)); + + cursorSequence = cursor.get(); + for (Sequence sequence : sequencesToAdd) { + sequence.set(cursorSequence); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index d6cfa6f6abc6f..aaa98220178bb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -68,26 +68,35 @@ public synchronized void startListenAndAssign( } public synchronized void stopListenAndAssign( - String dataRegionId, PipeRealtimeDataRegionSource extractor) { - final PipeDataRegionAssigner assigner = dataRegionId2Assigner.get(dataRegionId); - if (assigner == null) { - return; - } - - assigner.stopAssignTo(extractor); - - if (extractor.isNeedListenToTsFile()) { - listenToTsFileExtractorCount.decrementAndGet(); - } - if (extractor.isNeedListenToInsertNode()) { - listenToInsertNodeExtractorCount.decrementAndGet(); + final String dataRegionId, final PipeRealtimeDataRegionSource extractor) { + PipeDataRegionAssigner assignerToClose = null; + + synchronized (this) { + final PipeDataRegionAssigner assigner = dataRegionId2Assigner.get(dataRegionId); + if (assigner == null) { + return; + } + + assigner.stopAssignTo(extractor); + + if (extractor.isNeedListenToTsFile()) { + listenToTsFileExtractorCount.decrementAndGet(); + } + if (extractor.isNeedListenToInsertNode()) { + listenToInsertNodeExtractorCount.decrementAndGet(); + } + + if (assigner.notMoreSourceNeededToBeAssigned()) { + // The removed assigner will is the same as the one referenced by the variable `assigner` + dataRegionId2Assigner.remove(dataRegionId); + // This will help to release the memory occupied by the assigner + assignerToClose = assigner; + } } - if (assigner.notMoreExtractorNeededToBeAssigned()) { - // The removed assigner will is the same as the one referenced by the variable `assigner` - dataRegionId2Assigner.remove(dataRegionId); - // This will help to release the memory occupied by the assigner - assigner.close(); + if (assignerToClose != null) { + // Closing the disruptor may block for a while, so keep it out of the global listener lock. + assignerToClose.close(); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/DisruptorShutdownTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/DisruptorShutdownTest.java new file mode 100644 index 0000000000000..3fd40c4d4f2b5 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/DisruptorShutdownTest.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +public class DisruptorShutdownTest { + + @Test + public void testBatchEventProcessorDrainsPublishedEventsOnShutdownInterrupt() throws Exception { + final RingBuffer ringBuffer = RingBuffer.createMultiProducer(TestEvent::new, 32); + ringBuffer.publishEvent((event, sequence, value) -> event.value = value, 1); + + final TestSequenceBarrier barrier = new TestSequenceBarrier(0L); + final AtomicInteger handledEventCount = new AtomicInteger(); + final BatchEventProcessor processor = + new BatchEventProcessor<>( + ringBuffer, + barrier, + (event, sequence, endOfBatch) -> handledEventCount.incrementAndGet()); + + final Thread processorThread = new Thread(processor, "pipe-batch-event-processor-test"); + processorThread.start(); + + Assert.assertTrue(barrier.awaitWaitForCall()); + processor.halt(); + barrier.interruptWait(); + + processorThread.join(TimeUnit.SECONDS.toMillis(5)); + + Assert.assertFalse(processorThread.isAlive()); + Assert.assertEquals(1, handledEventCount.get()); + Assert.assertEquals(0L, processor.getSequence().get()); + } + + @Test + public void testBatchEventProcessorDrainsEventsPublishedAfterCurrentBatchWhenHalting() + throws Exception { + final RingBuffer ringBuffer = RingBuffer.createMultiProducer(TestEvent::new, 32); + ringBuffer.publishEvent((event, sequence, value) -> event.value = value, 1); + ringBuffer.publishEvent((event, sequence, value) -> event.value = value, 2); + + final SnapshotSequenceBarrier barrier = new SnapshotSequenceBarrier(0L, 1L); + final AtomicInteger handledEventCount = new AtomicInteger(); + final AtomicReference> processorReference = + new AtomicReference<>(); + final BatchEventProcessor processor = + new BatchEventProcessor<>( + ringBuffer, + barrier, + (event, sequence, endOfBatch) -> { + handledEventCount.incrementAndGet(); + if (event.value == 1) { + processorReference.get().halt(); + } + }); + processorReference.set(processor); + + final Thread processorThread = + new Thread(processor, "pipe-batch-event-processor-snapshot-test"); + processorThread.start(); + processorThread.join(TimeUnit.SECONDS.toMillis(5)); + + Assert.assertFalse(processorThread.isAlive()); + Assert.assertEquals(2, handledEventCount.get()); + Assert.assertEquals(1L, processor.getSequence().get()); + } + + @Test + public void testDisruptorShutdownInterruptsWaitingProcessor() throws Exception { + final AtomicReference processorThreadReference = new AtomicReference<>(); + final ThreadFactory threadFactory = + runnable -> { + final Thread thread = new Thread(runnable, "pipe-disruptor-shutdown-test"); + processorThreadReference.set(thread); + return thread; + }; + + final Disruptor disruptor = new Disruptor<>(TestEvent::new, 32, threadFactory); + disruptor.handleEventsWith((event, sequence, endOfBatch) -> {}); + disruptor.start(); + + final Thread processorThread = processorThreadReference.get(); + Assert.assertNotNull(processorThread); + + TimeUnit.MILLISECONDS.sleep(50); + disruptor.shutdown(); + + Assert.assertFalse(processorThread.isAlive()); + } + + private static class TestEvent { + private int value; + } + + private static class TestSequenceBarrier extends SequenceBarrier { + + private final long cursor; + private final CountDownLatch waitForCalled = new CountDownLatch(1); + private final CountDownLatch interruptWait = new CountDownLatch(1); + + private TestSequenceBarrier(final long cursor) { + super(new MultiProducerSequencer(32, new Sequence[0]), new Sequence[0]); + this.cursor = cursor; + } + + @Override + public long waitFor(final long sequence) throws InterruptedException { + waitForCalled.countDown(); + interruptWait.await(); + throw new InterruptedException(); + } + + @Override + public long getCursor() { + return cursor; + } + + @Override + public long getHighestPublishedSequence(final long lowerBound, final long availableSequence) { + return availableSequence; + } + + private boolean awaitWaitForCall() throws InterruptedException { + return waitForCalled.await(5, TimeUnit.SECONDS); + } + + private void interruptWait() { + interruptWait.countDown(); + } + } + + private static class SnapshotSequenceBarrier extends SequenceBarrier { + + private final long waitForResult; + private final long cursor; + + private SnapshotSequenceBarrier(final long waitForResult, final long cursor) { + super(new MultiProducerSequencer(32, new Sequence[0]), new Sequence[0]); + this.waitForResult = waitForResult; + this.cursor = cursor; + } + + @Override + public long waitFor(final long sequence) { + return waitForResult; + } + + @Override + public long getCursor() { + return cursor; + } + + @Override + public long getHighestPublishedSequence(final long lowerBound, final long availableSequence) { + return availableSequence; + } + } +} diff --git a/pom.xml b/pom.xml index 294a1231e355f..cbe907c71b934 100644 --- a/pom.xml +++ b/pom.xml @@ -75,7 +75,6 @@ 2.11.1 4.4 false - 3.4.4 1.21.1 4.2.19 11.1.0 @@ -454,11 +453,6 @@ h2-mvstore ${h2.version} - - com.lmax - disruptor - ${disruptor.version} - org.apache.httpcomponents httpclient From 04e668426c4d33b7010397d499363035122d1f9f Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Thu, 30 Apr 2026 17:23:00 +0800 Subject: [PATCH 024/102] [to dev/1.3] Fixed swallowed exceptions in query execution --- .../iotdb/db/queryengine/execution/driver/DataDriver.java | 5 ++++- .../schemaengine/schemaregion/utils/ResourceByPathUtils.java | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriver.java index c9c41f5b993ab..6caad1a175d27 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/driver/DataDriver.java @@ -31,6 +31,7 @@ import java.util.List; +import static com.google.common.base.Throwables.throwIfUnchecked; import static org.apache.iotdb.db.queryengine.metric.QueryExecutionMetricSet.QUERY_RESOURCE_INIT; import static org.apache.iotdb.db.storageengine.dataregion.VirtualDataRegion.UNFINISHED_QUERY_DATA_SOURCE; @@ -68,7 +69,9 @@ protected boolean init(SettableFuture blockedFuture) { "Failed to do the initialization for driver {} ", driverContext.getDriverTaskID(), t); driverContext.failed(t); blockedFuture.setException(t); - return false; + throwIfUnchecked(t); + // should never happen + throw new AssertionError(t); } } return true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index f3ada63d96f53..c64d32c1e327b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -209,6 +209,7 @@ protected Map prepareTvListMapForQuery( listRamInfo.getArrayMemCost(), listRamInfo.getRowCount(), listRamInfo.getDataTypes()); + throw ex; } finally { list.unlockQueryList(); } From 81247b48e44b268b0d7091a6ff728f001b546931 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 6 May 2026 16:48:20 +0800 Subject: [PATCH 025/102] [To dev/1.3] Fixed the updateAndGet side effect in DeviceCacheEntry and PipeRemainingOperator (#17584) (#17602) --- .../PipeConfigNodeRemainingTimeOperator.java | 24 ++++------ ...DataNodeRemainingEventAndTimeOperator.java | 48 +++++++------------ .../cache/schema/DeviceCacheEntry.java | 35 +++++++------- 3 files changed, 43 insertions(+), 64 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/overview/PipeConfigNodeRemainingTimeOperator.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/overview/PipeConfigNodeRemainingTimeOperator.java index c27e4cd898b06..ff9a7ea2648d5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/overview/PipeConfigNodeRemainingTimeOperator.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/metric/overview/PipeConfigNodeRemainingTimeOperator.java @@ -66,14 +66,11 @@ class PipeConfigNodeRemainingTimeOperator extends PipeRemainingOperator { .reduce(Long::sum) .orElse(0L); - configRegionCommitMeter.updateAndGet( - meter -> { - if (Objects.nonNull(meter)) { - lastConfigRegionCommitSmoothingValue = - pipeRemainingTimeCommitRateAverageTime.getMeterRate(meter); - } - return meter; - }); + final Meter configRegionMeter = configRegionCommitMeter.get(); + if (Objects.nonNull(configRegionMeter)) { + lastConfigRegionCommitSmoothingValue = + pipeRemainingTimeCommitRateAverageTime.getMeterRate(configRegionMeter); + } final double configRegionRemainingTime; if (totalConfigRegionWriteEventCount <= 0) { @@ -101,13 +98,10 @@ void register(final IoTDBConfigRegionSource extractor) { //////////////////////////// Rate //////////////////////////// void markConfigRegionCommit() { - configRegionCommitMeter.updateAndGet( - meter -> { - if (Objects.nonNull(meter)) { - meter.mark(); - } - return meter; - }); + final Meter meter = configRegionCommitMeter.get(); + if (Objects.nonNull(meter)) { + meter.mark(); + } } //////////////////////////// Switch //////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java index 38cfb579622a5..6ff3de4d9495c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java @@ -138,14 +138,11 @@ public double getRemainingTime() { + rawTabletEventCount.get() + insertNodeEventCount.get(); - dataRegionCommitMeter.updateAndGet( - meter -> { - if (Objects.nonNull(meter)) { - lastDataRegionCommitSmoothingValue = - pipeRemainingTimeCommitRateAverageTime.getMeterRate(meter); - } - return meter; - }); + final Meter dataRegionMeter = dataRegionCommitMeter.get(); + if (Objects.nonNull(dataRegionMeter)) { + lastDataRegionCommitSmoothingValue = + pipeRemainingTimeCommitRateAverageTime.getMeterRate(dataRegionMeter); + } final double dataRegionRemainingTime; if (totalDataRegionWriteEventCount <= 0) { dataRegionRemainingTime = 0; @@ -162,14 +159,11 @@ public double getRemainingTime() { .reduce(Long::sum) .orElse(0L); - schemaRegionCommitMeter.updateAndGet( - meter -> { - if (Objects.nonNull(meter)) { - lastSchemaRegionCommitSmoothingValue = - pipeRemainingTimeCommitRateAverageTime.getMeterRate(meter); - } - return meter; - }); + final Meter schemaRegionMeter = schemaRegionCommitMeter.get(); + if (Objects.nonNull(schemaRegionMeter)) { + lastSchemaRegionCommitSmoothingValue = + pipeRemainingTimeCommitRateAverageTime.getMeterRate(schemaRegionMeter); + } final double schemaRegionRemainingTime; if (totalSchemaRegionWriteEventCount <= 0) { schemaRegionRemainingTime = 0; @@ -199,23 +193,17 @@ void register(final IoTDBSchemaRegionSource source) { //////////////////////////// Rate //////////////////////////// void markDataRegionCommit() { - dataRegionCommitMeter.updateAndGet( - meter -> { - if (Objects.nonNull(meter)) { - meter.mark(); - } - return meter; - }); + final Meter meter = dataRegionCommitMeter.get(); + if (Objects.nonNull(meter)) { + meter.mark(); + } } void markSchemaRegionCommit() { - schemaRegionCommitMeter.updateAndGet( - meter -> { - if (Objects.nonNull(meter)) { - meter.mark(); - } - return meter; - }); + final Meter meter = schemaRegionCommitMeter.get(); + if (Objects.nonNull(meter)) { + meter.mark(); + } } void markTsFileCollectInvocationCount(final long collectInvocationCount) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/schema/DeviceCacheEntry.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/schema/DeviceCacheEntry.java index 3337e2f1420e9..ecd4c297e5729 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/schema/DeviceCacheEntry.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/schema/DeviceCacheEntry.java @@ -30,7 +30,6 @@ import java.util.Map; import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import static org.apache.iotdb.commons.schema.SchemaConstant.NON_TEMPLATE; @@ -94,15 +93,14 @@ IDeviceSchema getDeviceSchema() { } int invalidateSchema() { - final AtomicInteger size = new AtomicInteger(0); - deviceSchema.updateAndGet( - schema -> { - if (Objects.nonNull(schema)) { - size.set(schema.estimateSize()); - } - return null; - }); - return size.get(); + IDeviceSchema schema; + do { + schema = deviceSchema.get(); + if (Objects.isNull(schema)) { + return 0; + } + } while (!deviceSchema.compareAndSet(schema, null)); + return schema.estimateSize(); } /////////////////////////////// Last Cache /////////////////////////////// @@ -150,15 +148,14 @@ boolean updateInputMap(final @Nonnull Map updateMap) { } int invalidateLastCache() { - final AtomicInteger size = new AtomicInteger(0); - lastCache.updateAndGet( - cacheEntry -> { - if (Objects.nonNull(cacheEntry)) { - size.set(cacheEntry.estimateSize()); - } - return null; - }); - return size.get(); + DeviceLastCache cacheEntry; + do { + cacheEntry = lastCache.get(); + if (Objects.isNull(cacheEntry)) { + return 0; + } + } while (!lastCache.compareAndSet(cacheEntry, null)); + return cacheEntry.estimateSize(); } /////////////////////////////// Management /////////////////////////////// From a550448d6136d12ac47f3d96fa8cb570687913cc Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 6 May 2026 17:34:49 +0800 Subject: [PATCH 026/102] Pipe: Fixed the bug that the parent class loader may wrongly load ext pipe plugins (#17580) (#17600) --- .../plugin/service/PipePluginClassLoader.java | 42 ++++- .../service/PipePluginClassLoaderTest.java | 175 ++++++++++++++++++ 2 files changed, 216 insertions(+), 1 deletion(-) create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/agent/plugin/service/PipePluginClassLoaderTest.java diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/service/PipePluginClassLoader.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/service/PipePluginClassLoader.java index a2bc1df1a2797..745d93566fc40 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/service/PipePluginClassLoader.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/service/PipePluginClassLoader.java @@ -35,6 +35,10 @@ @ThreadSafe public class PipePluginClassLoader extends URLClassLoader { + private static final String[] PARENT_FIRST_CLASS_PREFIXES = { + "java.", "javax.", "jdk.", "sun.", "org.slf4j.", "org.apache.iotdb.pipe.api." + }; + private final String libRoot; /** @@ -50,7 +54,11 @@ public class PipePluginClassLoader extends URLClassLoader { private volatile boolean deprecated; public PipePluginClassLoader(String libRoot) throws IOException { - super(new URL[0]); + this(libRoot, ClassLoader.getSystemClassLoader()); + } + + PipePluginClassLoader(String libRoot, ClassLoader parent) throws IOException { + super(new URL[0], parent); this.libRoot = libRoot; activeInstanceCount = new AtomicLong(0); deprecated = false; @@ -82,6 +90,38 @@ public synchronized void markAsDeprecated() throws IOException { closeIfPossible(); } + @Override + protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + synchronized (getClassLoadingLock(name)) { + Class loadedClass = findLoadedClass(name); + if (loadedClass == null) { + loadedClass = + shouldLoadFromParentFirst(name) ? super.loadClass(name, false) : loadClassLocally(name); + } + if (resolve) { + resolveClass(loadedClass); + } + return loadedClass; + } + } + + private Class loadClassLocally(String name) throws ClassNotFoundException { + try { + return findClass(name); + } catch (ClassNotFoundException e) { + return super.loadClass(name, false); + } + } + + private boolean shouldLoadFromParentFirst(String name) { + for (String prefix : PARENT_FIRST_CLASS_PREFIXES) { + if (name.startsWith(prefix)) { + return true; + } + } + return false; + } + private void closeIfPossible() throws IOException { if (deprecated && activeInstanceCount.get() == 0) { close(); diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/agent/plugin/service/PipePluginClassLoaderTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/agent/plugin/service/PipePluginClassLoaderTest.java new file mode 100644 index 0000000000000..39126656fc1ab --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/agent/plugin/service/PipePluginClassLoaderTest.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.agent.plugin.service; + +import org.junit.Assert; +import org.junit.Test; + +import javax.tools.JavaCompiler; +import javax.tools.StandardJavaFileManager; +import javax.tools.ToolProvider; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.jar.JarEntry; +import java.util.jar.JarOutputStream; +import java.util.stream.Stream; + +public class PipePluginClassLoaderTest { + + @Test + public void testPluginClassesShouldOverrideParentClasses() throws Exception { + final Path tempDir = Files.createTempDirectory("pipe-plugin-classloader-test"); + try { + final Path parentSources = Files.createDirectory(tempDir.resolve("parent-sources")); + final Path parentClasses = Files.createDirectory(tempDir.resolve("parent-classes")); + final Path childSources = Files.createDirectory(tempDir.resolve("child-sources")); + final Path childClasses = Files.createDirectory(tempDir.resolve("child-classes")); + + final String sampleSource = + "package test.plugin;" + + "public class Sample {" + + " public String ping() {" + + " return test.dep.Helper.value();" + + " }" + + "}"; + final String parentHelperSource = + "package test.dep;" + + "public class Helper {" + + " public static String value() {" + + " return \"parent\";" + + " }" + + "}"; + final String childHelperSource = + "package test.dep;" + + "public class Helper {" + + " public static String value() {" + + " return \"child\";" + + " }" + + "}"; + + compile( + parentSources, + parentClasses, + createSources(sampleSource, false), + createSources(parentHelperSource, true)); + compile( + childSources, + childClasses, + createSources(sampleSource, false), + createSources(childHelperSource, true)); + + final Path parentJar = tempDir.resolve("parent.jar"); + final Path childJar = tempDir.resolve("child.jar"); + createJar(parentJar, parentClasses, Arrays.asList("test/plugin/Sample.class")); + createJar( + childJar, + childClasses, + Arrays.asList("test/plugin/Sample.class", "test/dep/Helper.class")); + + try (final URLClassLoader parentClassLoader = + new URLClassLoader(new URL[] {parentJar.toUri().toURL()}, null); + final PipePluginClassLoader pluginClassLoader = + new PipePluginClassLoader(childJar.toString(), parentClassLoader)) { + final Class sampleClass = Class.forName("test.plugin.Sample", true, pluginClassLoader); + Assert.assertSame(pluginClassLoader, sampleClass.getClassLoader()); + final Object sample = sampleClass.getDeclaredConstructor().newInstance(); + Assert.assertEquals("child", sampleClass.getMethod("ping").invoke(sample)); + } + } finally { + deleteRecursively(tempDir); + } + } + + private static Map createSources( + final String source, final boolean helperSource) { + final Map sources = new LinkedHashMap<>(); + sources.put(helperSource ? "test.dep.Helper" : "test.plugin.Sample", source); + return sources; + } + + private static void compile( + final Path sourcesDir, final Path classesDir, final Map... sourceGroups) + throws IOException { + final JavaCompiler compiler = ToolProvider.getSystemJavaCompiler(); + Assert.assertNotNull("A JDK is required to run this test.", compiler); + + final List sourceFiles = new ArrayList<>(); + for (final Map sourceGroup : sourceGroups) { + for (final Map.Entry entry : sourceGroup.entrySet()) { + final Path sourceFile = sourcesDir.resolve(entry.getKey().replace('.', '/') + ".java"); + Files.createDirectories(sourceFile.getParent()); + Files.write(sourceFile, entry.getValue().getBytes(StandardCharsets.UTF_8)); + sourceFiles.add(sourceFile.toFile()); + } + } + + try (final StandardJavaFileManager fileManager = + compiler.getStandardFileManager(null, null, StandardCharsets.UTF_8)) { + final boolean success = + compiler + .getTask( + null, + fileManager, + null, + Arrays.asList("-d", classesDir.toString()), + null, + fileManager.getJavaFileObjectsFromFiles(sourceFiles)) + .call(); + Assert.assertTrue(success); + } + } + + private static void createJar( + final Path jarPath, final Path classesDir, final List classEntries) + throws IOException { + try (final JarOutputStream jarOutputStream = + new JarOutputStream(Files.newOutputStream(jarPath))) { + for (final String classEntry : classEntries) { + jarOutputStream.putNextEntry(new JarEntry(classEntry)); + jarOutputStream.write(Files.readAllBytes(classesDir.resolve(classEntry))); + jarOutputStream.closeEntry(); + } + } + } + + private static void deleteRecursively(final Path path) throws IOException { + if (path == null || !Files.exists(path)) { + return; + } + try (final Stream stream = Files.walk(path)) { + for (final Path subPath : + (Iterable) stream.sorted(Comparator.reverseOrder())::iterator) { + Files.deleteIfExists(subPath); + } + } + } +} From 5b569a5539a355f4dcabcbedc187ce6d330cfd3e Mon Sep 17 00:00:00 2001 From: Haonan Date: Wed, 6 May 2026 17:36:20 +0800 Subject: [PATCH 027/102] [To dev/1.3] Optimize the configuration logic of dn_thrift_max_frame_size (#17603) * Add thrift max frame size calculate logic * Add thrift max frame size calculate logic * fix review --- .../TCompressedElasticFramedTransport.java | 8 +- .../iotdb/rpc/TElasticFramedTransport.java | 80 ++++++++++++++----- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 10 ++- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 4 - .../conf/iotdb-system.properties.template | 4 +- .../iotdb/commons/conf/IoTDBConstant.java | 1 - 6 files changed, 72 insertions(+), 35 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java index 8579f46a3cbd1..07768dd98bdd8 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TCompressedElasticFramedTransport.java @@ -44,13 +44,7 @@ protected TCompressedElasticFramedTransport( protected void readFrame() throws TTransportException { underlying.readAll(i32buf, 0, 4); int size = TFramedTransport.decodeFrameSize(i32buf); - - if (size < 0) { - close(); - throw new TTransportException( - TTransportException.CORRUPTED_DATA, "Read a negative frame size (" + size + ")!"); - } - + checkFrameSize(size); readBuffer.fill(underlying, size); RpcStat.readCompressedBytes.addAndGet(size); try { diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java index d98914ce934a3..2524e8ae4cd33 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java @@ -174,29 +174,71 @@ public int read(byte[] buf, int off, int len) throws TTransportException { protected void readFrame() throws TTransportException { underlying.readAll(i32buf, 0, 4); int size = TFramedTransport.decodeFrameSize(i32buf); + checkFrameSize(size); + readBuffer.fill(underlying, size); + } - if (size < 0) { - close(); - throw new TTransportException( - TTransportException.CORRUPTED_DATA, "Read a negative frame size (" + size + ")!"); + protected void checkFrameSize(int size) throws TTransportException { + final int HTTP_GET_SIGNATURE = 0x47455420; // "GET " + final int HTTP_POST_SIGNATURE = 0x504F5354; // "POST" + final int TLS_MIN_VERSION = 0x160300; + final int TLS_MAX_VERSION = 0x160303; + final int TLS_LENGTH_HIGH_MAX = 0x02; + + FrameError error = null; + if (size == HTTP_GET_SIGNATURE || size == HTTP_POST_SIGNATURE) { + error = FrameError.HTTP_REQUEST; + } else { + int high24 = size >>> 8; + if (high24 >= TLS_MIN_VERSION + && high24 <= TLS_MAX_VERSION + && (i32buf[3] & 0xFF) <= TLS_LENGTH_HIGH_MAX) { + error = FrameError.TLS_REQUEST; + } else if (size < 0) { + error = FrameError.NEGATIVE_FRAME_SIZE; + } else if (size > thriftMaxFrameSize) { + error = FrameError.FRAME_SIZE_EXCEEDED; + } } - if (size > thriftMaxFrameSize) { - close(); - if (size == 1195725856L || size == 1347375956L) { - // if someone sends HTTP GET/POST to this port, the size will be read as the following - throw new TTransportException( - TTransportException.CORRUPTED_DATA, - "Singular frame size (" - + size - + ") detected, you may be sending HTTP GET/POST requests to the Thrift-RPC port, please confirm that you are using the right port"); - } else { - throw new TTransportException( - TTransportException.CORRUPTED_DATA, - "Frame size (" + size + ") larger than protect max size (" + thriftMaxFrameSize + ")!"); - } + if (error == null) { + return; + } + + SocketAddress remoteAddress = null; + if (underlying instanceof TSocket) { + remoteAddress = ((TSocket) underlying).getSocket().getRemoteSocketAddress(); + } + String remoteInfo = (remoteAddress == null) ? "" : " from " + remoteAddress; + close(); + + error.throwException(size, remoteInfo, thriftMaxFrameSize); + } + + private enum FrameError { + HTTP_REQUEST( + "Singular frame size (%d) detected, you may be sending HTTP GET/POST%s " + + "requests to the Thrift-RPC port, please confirm that you are using the right port"), + TLS_REQUEST( + "Singular frame size (%d) detected, you may be sending TLS ClientHello " + + "requests%s to the Non-SSL Thrift-RPC port, please confirm that you are using " + + "the right configuration"), + NEGATIVE_FRAME_SIZE("Read a negative frame size (%d)%s!"), + FRAME_SIZE_EXCEEDED("Frame size (%d) larger than protect max size (%d)%s!"); + + private final String messageFormat; + + FrameError(String messageFormat) { + this.messageFormat = messageFormat; + } + + void throwException(int size, String remoteInfo, int maxSize) throws TTransportException { + String message = + (this == FRAME_SIZE_EXCEEDED) + ? String.format(messageFormat, size, maxSize, remoteInfo) + : String.format(messageFormat, size, remoteInfo); + throw new TTransportException(TTransportException.CORRUPTED_DATA, message); } - readBuffer.fill(underlying, size); } protected void checkWriteFrameSize(int size) throws TTransportException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 989dc75746400..5e89c4ec083a8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -868,7 +868,7 @@ public class IoTDBConfig { private float udfCollectorMemoryBudgetInMB = (float) (1.0 / 3 * udfMemoryBudgetInMB); /** Unit: byte */ - private int thriftMaxFrameSize = 536870912; + private int thriftMaxFrameSize = getDefaultThriftMaxFrameSize(); private int thriftDefaultBufferSize = RpcUtils.THRIFT_DEFAULT_BUF_CAPACITY; @@ -2784,10 +2784,16 @@ public int getThriftMaxFrameSize() { } public void setThriftMaxFrameSize(int thriftMaxFrameSize) { - this.thriftMaxFrameSize = thriftMaxFrameSize; + this.thriftMaxFrameSize = + thriftMaxFrameSize <= 0 ? getDefaultThriftMaxFrameSize() : thriftMaxFrameSize; BaseRpcTransportFactory.setThriftMaxFrameSize(this.thriftMaxFrameSize); } + private static int getDefaultThriftMaxFrameSize() { + return Math.min( + 64 * 1024 * 1024, (int) Math.min(Runtime.getRuntime().maxMemory() / 64, Integer.MAX_VALUE)); + } + public int getThriftDefaultBufferSize() { return thriftDefaultBufferSize; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index c755381e88dd9..e965611016de9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -861,10 +861,6 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException properties.getProperty( "dn_thrift_max_frame_size", String.valueOf(conf.getThriftMaxFrameSize())))); - if (conf.getThriftMaxFrameSize() < IoTDBConstant.LEFT_SIZE_IN_REQUEST * 2) { - conf.setThriftMaxFrameSize(IoTDBConstant.LEFT_SIZE_IN_REQUEST * 2); - } - conf.setThriftDefaultBufferSize( Integer.parseInt( properties.getProperty( diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 018551d889a57..c7862e4886ec4 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -509,10 +509,10 @@ dn_rpc_min_concurrent_client_num=1 # Datatype: int dn_rpc_max_concurrent_client_num=1000 -# thrift max frame size, 512MB by default +# thrift max frame size in bytes. When set to 0, use min(64MB, datanode heap memory / 64) # effectiveMode: restart # Datatype: int -dn_thrift_max_frame_size=536870912 +dn_thrift_max_frame_size=0 # thrift init buffer size # effectiveMode: restart diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java index 18ac41409ba9d..f9f69f486e6ce 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java @@ -268,7 +268,6 @@ private IoTDBConstant() {} public static final String MQTT_MAX_MESSAGE_SIZE = "mqtt_max_message_size"; // thrift - public static final int LEFT_SIZE_IN_REQUEST = 4 * 1024 * 1024; public static final int DEFAULT_FETCH_SIZE = 5000; public static final int DEFAULT_CONNECTION_TIMEOUT_MS = 0; From e6ad00536026ba71d76cfbcf3f609faf811c13db Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 6 May 2026 17:36:45 +0800 Subject: [PATCH 028/102] Pipe: Fixed the bug that the pipe will restart a pipe stopped by user but encountered sync failure later (#17586) (#17605) * fix * fix --- .../persistence/pipe/PipeTaskInfo.java | 8 ++ .../pipe/PipeTaskInfoAutoRestartTest.java | 128 ++++++++++++++++++ 2 files changed, 136 insertions(+) create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfoAutoRestartTest.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java index 2bfdbf9e49a3b..199b835c839e2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java @@ -798,6 +798,14 @@ private boolean recordDataNodePushPipeMetaExceptionsInternal( final PipeRuntimeMeta runtimeMeta = pipeMetaKeeper.getPipeMeta(message.getPipeName()).getRuntimeMeta(); + // Keep user-stopped pipes out of the auto-restart flow. Otherwise, a failed + // STOPPED meta sync can turn a manually stopped pipe into a runtime-stopped one + // and the next PipeMetaSyncer round will restart it automatically. + if (PipeStatus.STOPPED.equals(runtimeMeta.getStatus().get()) + && !runtimeMeta.getIsStoppedByRuntimeException()) { + return; + } + // Mark the status of the pipe with exception as stopped runtimeMeta.getStatus().set(PipeStatus.STOPPED); runtimeMeta.setIsStoppedByRuntimeException(true); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfoAutoRestartTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfoAutoRestartTest.java new file mode 100644 index 0000000000000..762c3bf045c5b --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfoAutoRestartTest.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.persistence.pipe; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeRuntimeMeta; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStaticMeta; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStatus; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.CreatePipePlanV2; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusPlanV2; +import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaResp; +import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaRespExceptionMessage; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class PipeTaskInfoAutoRestartTest { + + private static final int DATA_NODE_ID = 1; + + private PipeTaskInfo pipeTaskInfo; + + @Before + public void setUp() { + pipeTaskInfo = new PipeTaskInfo(); + } + + @Test + public void testRecordDataNodePushPipeMetaExceptionsMarksRunningPipeForAutoRestart() { + final String pipeName = "runningPipe"; + createPipe(pipeName, PipeStatus.RUNNING); + + Assert.assertTrue( + pipeTaskInfo.recordDataNodePushPipeMetaExceptions(createErrorRespMap(pipeName))); + + final PipeRuntimeMeta runtimeMeta = + pipeTaskInfo.getPipeMetaByPipeName(pipeName).getRuntimeMeta(); + Assert.assertEquals(PipeStatus.STOPPED, runtimeMeta.getStatus().get()); + Assert.assertTrue(runtimeMeta.getIsStoppedByRuntimeException()); + + Assert.assertTrue(pipeTaskInfo.autoRestart()); + Assert.assertEquals(PipeStatus.RUNNING, runtimeMeta.getStatus().get()); + } + + @Test + public void testRecordDataNodePushPipeMetaExceptionsKeepsUserStoppedPipeOutOfAutoRestart() { + final String pipeName = "stoppedPipe"; + createPipe(pipeName, PipeStatus.STOPPED); + + Assert.assertTrue(pipeTaskInfo.isPipeStoppedByUser(pipeName)); + Assert.assertTrue( + pipeTaskInfo.recordDataNodePushPipeMetaExceptions(createErrorRespMap(pipeName))); + + final PipeRuntimeMeta runtimeMeta = + pipeTaskInfo.getPipeMetaByPipeName(pipeName).getRuntimeMeta(); + Assert.assertEquals(PipeStatus.STOPPED, runtimeMeta.getStatus().get()); + Assert.assertFalse(runtimeMeta.getIsStoppedByRuntimeException()); + Assert.assertTrue(pipeTaskInfo.isPipeStoppedByUser(pipeName)); + + Assert.assertFalse(pipeTaskInfo.autoRestart()); + Assert.assertEquals(PipeStatus.STOPPED, runtimeMeta.getStatus().get()); + } + + private Map createErrorRespMap(final String pipeName) { + final TPushPipeMetaRespExceptionMessage exceptionMessage = + new TPushPipeMetaRespExceptionMessage( + pipeName, "failed to push pipe meta", System.currentTimeMillis()); + final TPushPipeMetaResp resp = + new TPushPipeMetaResp() + .setStatus(new TSStatus(TSStatusCode.PIPE_PUSH_META_ERROR.getStatusCode())) + .setExceptionMessages(Collections.singletonList(exceptionMessage)); + return Collections.singletonMap(DATA_NODE_ID, resp); + } + + private void createPipe(final String pipeName, final PipeStatus initialStatus) { + final Map extractorAttributes = new HashMap<>(); + final Map processorAttributes = new HashMap<>(); + final Map connectorAttributes = new HashMap<>(); + extractorAttributes.put("extractor", "iotdb-source"); + processorAttributes.put("processor", "do-nothing-processor"); + connectorAttributes.put("connector", "iotdb-thrift-sink"); + + final PipeTaskMeta pipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, DATA_NODE_ID); + final ConcurrentMap pipeTasks = new ConcurrentHashMap<>(); + pipeTasks.put(DATA_NODE_ID, pipeTaskMeta); + + final PipeStaticMeta pipeStaticMeta = + new PipeStaticMeta( + pipeName, + System.currentTimeMillis(), + extractorAttributes, + processorAttributes, + connectorAttributes); + final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(pipeTasks); + pipeTaskInfo.createPipe(new CreatePipePlanV2(pipeStaticMeta, pipeRuntimeMeta)); + + if (PipeStatus.RUNNING.equals(initialStatus)) { + pipeTaskInfo.setPipeStatus(new SetPipeStatusPlanV2(pipeName, PipeStatus.RUNNING)); + } + } +} From aa48de75feacb17347169780bffe2f6a6992a8e3 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 7 May 2026 09:32:37 +0800 Subject: [PATCH 029/102] Pipe: Fixed the bug that the timeout ms is regarded as s (#17590) (#17607) --- .../pipe/agent/task/PipeDataNodeTaskAgent.java | 2 +- .../commons/pipe/agent/task/PipeTaskAgent.java | 16 ++++++++++++++-- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index c437507f4b085..9038e7c3a7191 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -374,7 +374,7 @@ public void stopAllPipesWithCriticalExceptionAndTrackException( ///////////////////////// Heartbeat ///////////////////////// public void collectPipeMetaList(final TDataNodeHeartbeatResp resp) throws TException { - if (!tryReadLockWithTimeOut( + if (!tryReadLockWithTimeOutInMs( CommonDescriptor.getInstance().getConfig().getDnConnectionTimeoutInMS() * 2L / 3)) { return; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java index 726bc1b6a1f93..fcff317b3f337 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java @@ -125,6 +125,10 @@ protected boolean tryReadLockWithTimeOut(final long timeOutInSeconds) { } } + protected boolean tryReadLockWithTimeOutInMs(final long timeOutInMs) { + return tryReadLockWithTimeOut(convertMsToCeilSeconds(timeOutInMs)); + } + protected void releaseReadLock() { pipeMetaKeeper.releaseReadLock(); } @@ -143,10 +147,18 @@ protected boolean tryWriteLockWithTimeOut(final long timeOutInSeconds) { } } + protected boolean tryWriteLockWithTimeOutInMs(final long timeOutInMs) { + return tryWriteLockWithTimeOut(convertMsToCeilSeconds(timeOutInMs)); + } + protected void releaseWriteLock() { pipeMetaKeeper.releaseWriteLock(); } + private long convertMsToCeilSeconds(final long timeOutInMs) { + return Math.max(1L, (Math.max(0L, timeOutInMs) + 999L) / 1000L); + } + ////////////////////////// Pipe Task Management Entry ////////////////////////// public TPushPipeMetaRespExceptionMessage handleSinglePipeMetaChanges( @@ -363,7 +375,7 @@ protected TPushPipeMetaRespExceptionMessage handleDropPipeInternal(final String public List handlePipeMetaChanges( final List pipeMetaListFromCoordinator) { - if (!tryWriteLockWithTimeOut( + if (!tryWriteLockWithTimeOutInMs( CommonDescriptor.getInstance().getConfig().getDnConnectionTimeoutInMS() * 2L / 3)) { return null; } @@ -1091,7 +1103,7 @@ private void stopAllPipesWithCriticalExceptionInternal(final int currentNodeId) public void collectPipeMetaList(final TPipeHeartbeatReq req, final TPipeHeartbeatResp resp) throws TException { - if (!tryReadLockWithTimeOut( + if (!tryReadLockWithTimeOutInMs( CommonDescriptor.getInstance().getConfig().getDnConnectionTimeoutInMS() * 2L / 3)) { return; } From 494c5915cf1cac476e1adb4c9e1887fd8562de4d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 7 May 2026 09:34:19 +0800 Subject: [PATCH 030/102] Pipe: Fixed the potential leak file handle at the receiver (#17589) (#17606) --- .../pipe/receiver/IoTDBFileReceiver.java | 26 +++- .../pipe/receiver/IoTDBFileReceiverTest.java | 124 +++++++++++++++++- 2 files changed, 145 insertions(+), 5 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java index 61d9155f99681..7346ae2bccd48 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java @@ -126,6 +126,10 @@ protected TPipeTransferResp handleTransferHandshakeV1(final PipeTransferHandshak String.format( "Pipe-Receiver-%s-%s:%s", receiverId.get(), getSenderHost(), getSenderPort())); + // Handshake restarts the transfer session. Reset the current writing state before recycling the + // old receiver dir, otherwise the old file handle can survive across handshakes. + resetCurrentWritingFileState(); + // Clear the original receiver file dir if exists if (receiverFileDirWithIdSuffix.get() != null) { if (receiverFileDirWithIdSuffix.get().exists()) { @@ -547,6 +551,11 @@ private void deleteCurrentWritingFile() { } } + private void resetCurrentWritingFileState() { + closeCurrentWritingFileWriter(false); + writingFile = null; + } + private void deleteFile(final File file) { if (file.exists()) { try { @@ -589,6 +598,8 @@ private boolean isWritingFileOffsetCorrect(final long offset) throws IOException } protected final TPipeTransferResp handleTransferFileSealV1(final PipeTransferFileSealReqV1 req) { + File sealedWritingFile = null; + boolean shouldDeleteSealedFile = true; try { if (!isWritingFileAvailable()) { final TSStatus status = @@ -605,7 +616,8 @@ protected final TPipeTransferResp handleTransferFileSealV1(final PipeTransferFil return resp; } - final String fileAbsolutePath = writingFile.getAbsolutePath(); + sealedWritingFile = writingFile; + final String fileAbsolutePath = sealedWritingFile.getAbsolutePath(); // Sync here is necessary to ensure that the data is written to the disk. Or data region may // load the file before the data is written to the disk and cause unexpected behavior after @@ -624,11 +636,13 @@ protected final TPipeTransferResp handleTransferFileSealV1(final PipeTransferFil writingFileWriter.close(); writingFileWriter = null; - // writingFile will be deleted after load if no exception occurs + // Clear the reference before loading so the next file transfer can not reuse the same path. + // The loader owns cleanup after a successful load. writingFile = null; final TSStatus status = loadFileV1(req, fileAbsolutePath); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + shouldDeleteSealedFile = false; LOGGER.info( "Receiver id = {}: Seal file {} successfully.", receiverId.get(), fileAbsolutePath); } else { @@ -657,7 +671,13 @@ protected final TPipeTransferResp handleTransferFileSealV1(final PipeTransferFil // All pieces of the writing file and its mod (if exists) should be retransmitted by the // sender. closeCurrentWritingFileWriter(false); - deleteCurrentWritingFile(); + if (shouldDeleteSealedFile) { + if (writingFile != null) { + deleteCurrentWritingFile(); + } else if (sealedWritingFile != null) { + deleteFile(sealedWritingFile); + } + } } } diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiverTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiverTest.java index 337cece8a4887..c692cb1513a4f 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiverTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiverTest.java @@ -20,10 +20,12 @@ package org.apache.iotdb.commons.pipe.receiver; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeRequestType; import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferFileSealReqV1; import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferFileSealReqV2; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferHandshakeV1Req; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; @@ -33,6 +35,8 @@ import java.io.File; import java.io.IOException; +import java.io.RandomAccessFile; +import java.lang.reflect.Field; import java.nio.file.Files; import java.nio.file.Path; import java.util.Arrays; @@ -86,9 +90,61 @@ public void testRejectPathTraversalFileNameInSealRequest() throws Exception { } } + @Test + public void testHandshakeResetsWritingFileState() throws Exception { + final Path baseDir = Files.createTempDirectory("iotdb-file-receiver-test"); + final DummyFileReceiver receiver = new DummyFileReceiver(baseDir.toFile()); + try { + receiver.handshake(); + receiver.createWritingFile("normal.tsfile", true); + receiver.writeToCurrentWritingFile(new byte[] {1, 2, 3}); + + final File oldReceiverDir = receiver.getCurrentReceiverDir(); + Assert.assertNotNull(receiver.getCurrentWritingFile()); + Assert.assertNotNull(receiver.getCurrentWritingFileWriter()); + + receiver.handshake(); + + Assert.assertFalse(oldReceiverDir.exists()); + Assert.assertNull(receiver.getCurrentWritingFile()); + Assert.assertNull(receiver.getCurrentWritingFileWriter()); + Assert.assertNotEquals( + oldReceiverDir.getAbsolutePath(), receiver.getCurrentReceiverDir().getAbsolutePath()); + } finally { + receiver.handleExit(); + } + } + + @Test + public void testSealFileV1FailureDeletesTransferredFile() throws Exception { + final Path baseDir = Files.createTempDirectory("iotdb-file-receiver-test"); + final DummyFileReceiver receiver = new DummyFileReceiver(baseDir.toFile()); + try { + receiver.createWritingFile("normal.tsfile", true); + receiver.writeToCurrentWritingFile(new byte[] {1, 2, 3}); + receiver.setLoadFileV1Status( + new TSStatus(TSStatusCode.PIPE_TRANSFER_FILE_ERROR.getStatusCode())); + + final File transferredFile = receiver.getWritingFileInBaseDir("normal.tsfile"); + final TPipeTransferResp response = receiver.sealFileV1("normal.tsfile", 3L); + + Assert.assertEquals( + TSStatusCode.PIPE_TRANSFER_FILE_ERROR.getStatusCode(), response.getStatus().getCode()); + Assert.assertFalse(transferredFile.exists()); + Assert.assertNull(receiver.getCurrentWritingFile()); + Assert.assertNull(receiver.getCurrentWritingFileWriter()); + } finally { + receiver.handleExit(); + } + } + private static class DummyFileReceiver extends IoTDBFileReceiver { + private final File receiverFileBaseDir; + private TSStatus loadFileV1Status = new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + DummyFileReceiver(final File baseDir) { + receiverFileBaseDir = baseDir; receiverFileDirWithIdSuffix.set(baseDir); } @@ -96,6 +152,24 @@ void createWritingFile(final String fileName, final boolean isSingleFile) throws updateWritingFileIfNeeded(fileName, isSingleFile); } + void handshake() throws IOException { + handleTransferHandshakeV1( + DummyHandshakeReq.toTPipeTransferReq( + CommonDescriptor.getInstance().getConfig().getTimestampPrecision())); + } + + void writeToCurrentWritingFile(final byte[] bytes) throws Exception { + getCurrentWritingFileWriter().write(bytes); + } + + void setLoadFileV1Status(final TSStatus status) { + loadFileV1Status = status; + } + + TPipeTransferResp sealFileV1(final String fileName, final long fileLength) throws IOException { + return handleTransferFileSealV1(DummyFileSealReqV1.toTPipeTransferReq(fileName, fileLength)); + } + TPipeTransferResp sealFiles(final List fileNames, final List fileLengths) throws IOException { return handleTransferFileSealV2( @@ -106,9 +180,27 @@ File getWritingFileInBaseDir(final String fileName) { return receiverFileDirWithIdSuffix.get().toPath().resolve(fileName).toFile(); } + File getCurrentReceiverDir() { + return receiverFileDirWithIdSuffix.get(); + } + + File getCurrentWritingFile() throws Exception { + return (File) getField("writingFile").get(this); + } + + RandomAccessFile getCurrentWritingFileWriter() throws Exception { + return (RandomAccessFile) getField("writingFileWriter").get(this); + } + + private Field getField(final String fieldName) throws NoSuchFieldException { + final Field field = IoTDBFileReceiver.class.getDeclaredField(fieldName); + field.setAccessible(true); + return field; + } + @Override protected String getReceiverFileBaseDir() { - return receiverFileDirWithIdSuffix.get().getAbsolutePath(); + return receiverFileBaseDir.getAbsolutePath(); } @Override @@ -134,7 +226,7 @@ protected TSStatus login() { @Override protected TSStatus loadFileV1( final PipeTransferFileSealReqV1 req, final String fileAbsolutePath) { - return new TSStatus(200); + return loadFileV1Status; } @Override @@ -155,6 +247,34 @@ public TPipeTransferResp receive(TPipeTransferReq req) { } } + private static class DummyHandshakeReq extends PipeTransferHandshakeV1Req { + + static DummyHandshakeReq toTPipeTransferReq(final String timestampPrecision) + throws IOException { + return (DummyHandshakeReq) + new DummyHandshakeReq().convertToTPipeTransferReq(timestampPrecision); + } + + @Override + protected PipeRequestType getPlanType() { + return PipeRequestType.HANDSHAKE_DATANODE_V1; + } + } + + private static class DummyFileSealReqV1 extends PipeTransferFileSealReqV1 { + + static DummyFileSealReqV1 toTPipeTransferReq(final String fileName, final long fileLength) + throws IOException { + return (DummyFileSealReqV1) + new DummyFileSealReqV1().convertToTPipeTransferReq(fileName, fileLength); + } + + @Override + protected PipeRequestType getPlanType() { + return PipeRequestType.TRANSFER_TS_FILE_SEAL; + } + } + private static class DummyFileSealReqV2 extends PipeTransferFileSealReqV2 { static DummyFileSealReqV2 toTPipeTransferReq( From 3b96b715bb6ac81474e203e5c1737c08029213b7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 7 May 2026 09:34:34 +0800 Subject: [PATCH 031/102] [To dev/1.3] Pipe: Reset tablet pipeDataStructureTabletSizeInBytes to 16MB & Enable stopping exception restart by manual stop pipe (#17588) (#17604) * Fixed the bug that create attribute does not support attribute.None & Pipe: Reset tablet pipeDataStructureTabletSizeInBytes to 16MB & Enable stopping exception restart by manual stop pipe (#17588) * stop pipe * 16 * fix * legacy * Update PipeTaskCoordinator.java * fix-access --- .../consensus/request/ConfigPhysicalPlan.java | 4 + .../request/ConfigPhysicalPlanType.java | 1 + ...usWithStoppedByRuntimeExceptionPlanV2.java | 109 ++++++++++++++++++ .../coordinator/task/PipeTaskCoordinator.java | 13 +-- .../executor/ConfigPlanExecutor.java | 4 + .../confignode/persistence/pipe/PipeInfo.java | 20 ++++ .../persistence/pipe/PipeTaskInfo.java | 20 ++++ .../impl/pipe/task/StopPipeProcedureV2.java | 26 ++++- .../request/ConfigPhysicalPlanSerDeTest.java | 25 +++- .../pipe/task/StopPipeProcedureV2Test.java | 105 +++++++++++++++++ .../iotdb/commons/conf/CommonConfig.java | 4 +- 11 files changed, 311 insertions(+), 20 deletions(-) create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/SetPipeStatusWithStoppedByRuntimeExceptionPlanV2.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java index b7a0936ca450f..06c10c3972395 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java @@ -73,6 +73,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.task.DropPipePlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.OperateMultiplePipesPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusPlanV2; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusWithStoppedByRuntimeExceptionPlanV2; import org.apache.iotdb.confignode.consensus.request.write.procedure.DeleteProcedurePlan; import org.apache.iotdb.confignode.consensus.request.write.procedure.UpdateProcedurePlan; import org.apache.iotdb.confignode.consensus.request.write.quota.SetSpaceQuotaPlan; @@ -354,6 +355,9 @@ public static ConfigPhysicalPlan create(final ByteBuffer buffer) throws IOExcept case SetPipeStatusV2: plan = new SetPipeStatusPlanV2(); break; + case SetPipeStatusWithStoppedByRuntimeExceptionV2: + plan = new SetPipeStatusWithStoppedByRuntimeExceptionPlanV2(); + break; case DropPipeV2: plan = new DropPipePlanV2(); break; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java index 24e5b9ba9c631..b986299596675 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java @@ -217,6 +217,7 @@ public enum ConfigPhysicalPlanType { ShowPipeV2((short) 1503), AlterPipeV2((short) 1504), OperateMultiplePipesV2((short) 1505), + SetPipeStatusWithStoppedByRuntimeExceptionV2((short) 1506), /** Pipe Runtime. */ PipeHandleLeaderChange((short) 1600), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/SetPipeStatusWithStoppedByRuntimeExceptionPlanV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/SetPipeStatusWithStoppedByRuntimeExceptionPlanV2.java new file mode 100644 index 0000000000000..35ee503d5c5fc --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/SetPipeStatusWithStoppedByRuntimeExceptionPlanV2.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.consensus.request.write.pipe.task; + +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStatus; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class SetPipeStatusWithStoppedByRuntimeExceptionPlanV2 extends ConfigPhysicalPlan { + + private String pipeName; + private PipeStatus status; + private boolean stoppedByRuntimeException; + + public SetPipeStatusWithStoppedByRuntimeExceptionPlanV2() { + super(ConfigPhysicalPlanType.SetPipeStatusWithStoppedByRuntimeExceptionV2); + } + + public SetPipeStatusWithStoppedByRuntimeExceptionPlanV2( + final String pipeName, final PipeStatus status, final boolean stoppedByRuntimeException) { + super(ConfigPhysicalPlanType.SetPipeStatusWithStoppedByRuntimeExceptionV2); + this.pipeName = pipeName; + this.status = status; + this.stoppedByRuntimeException = stoppedByRuntimeException; + } + + public String getPipeName() { + return pipeName; + } + + public PipeStatus getPipeStatus() { + return status; + } + + public boolean isStoppedByRuntimeException() { + return stoppedByRuntimeException; + } + + @Override + protected void serializeImpl(final DataOutputStream stream) throws IOException { + stream.writeShort(getType().getPlanType()); + ReadWriteIOUtils.write(pipeName, stream); + ReadWriteIOUtils.write(status.getType(), stream); + ReadWriteIOUtils.write(stoppedByRuntimeException, stream); + } + + @Override + protected void deserializeImpl(final ByteBuffer buffer) throws IOException { + pipeName = ReadWriteIOUtils.readString(buffer); + status = PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(buffer)); + stoppedByRuntimeException = ReadWriteIOUtils.readBool(buffer); + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final SetPipeStatusWithStoppedByRuntimeExceptionPlanV2 that = + (SetPipeStatusWithStoppedByRuntimeExceptionPlanV2) obj; + return stoppedByRuntimeException == that.stoppedByRuntimeException + && pipeName.equals(that.pipeName) + && status.equals(that.status); + } + + @Override + public int hashCode() { + return Objects.hash(pipeName, status, stoppedByRuntimeException); + } + + @Override + public String toString() { + return "SetPipeStatusWithStoppedByRuntimeExceptionPlanV2{" + + "pipeName='" + + pipeName + + "', status='" + + status + + "', stoppedByRuntimeException='" + + stoppedByRuntimeException + + "'}"; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinator.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinator.java index ee1ccd2a8fba4..f3430db3932fe 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinator.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinator.java @@ -120,19 +120,8 @@ public TSStatus startPipe(String pipeName) { /** Caller should ensure that the method is called in the lock {@link #lock()}. */ public TSStatus stopPipe(String pipeName) { - final boolean isStoppedByRuntimeException = pipeTaskInfo.isStoppedByRuntimeException(pipeName); final TSStatus status = configManager.getProcedureManager().stopPipe(pipeName); - if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - if (isStoppedByRuntimeException) { - // Even if the return status is success, it doesn't imply the success of the - // `executeFromOperateOnDataNodes` phase of stopping pipe. However, we still need to set - // `isStoppedByRuntimeException` to false to avoid auto-restart. Meanwhile, - // `isStoppedByRuntimeException` does not need to be synchronized with DNs. - LOGGER.info("Pipe {} has stopped manually, stop its auto restart process.", pipeName); - pipeTaskInfo.setIsStoppedByRuntimeExceptionToFalse(pipeName); - configManager.getProcedureManager().pipeHandleMetaChange(true, false); - } - } else { + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { LOGGER.warn("Failed to stop pipe {}. Result status: {}.", pipeName, status); } return status; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index 4facb308d8a2c..09017a841c73c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -98,6 +98,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.task.DropPipePlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.OperateMultiplePipesPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusPlanV2; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusWithStoppedByRuntimeExceptionPlanV2; import org.apache.iotdb.confignode.consensus.request.write.procedure.DeleteProcedurePlan; import org.apache.iotdb.confignode.consensus.request.write.procedure.UpdateProcedurePlan; import org.apache.iotdb.confignode.consensus.request.write.quota.SetSpaceQuotaPlan; @@ -497,6 +498,9 @@ public TSStatus executeNonQueryPlan(ConfigPhysicalPlan physicalPlan) return pipeInfo.createPipe((CreatePipePlanV2) physicalPlan); case SetPipeStatusV2: return pipeInfo.setPipeStatus((SetPipeStatusPlanV2) physicalPlan); + case SetPipeStatusWithStoppedByRuntimeExceptionV2: + return pipeInfo.setPipeStatusWithStoppedByRuntimeException( + (SetPipeStatusWithStoppedByRuntimeExceptionPlanV2) physicalPlan); case DropPipeV2: return pipeInfo.dropPipe((DropPipePlanV2) physicalPlan); case AlterPipeV2: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeInfo.java index d09e5c82845a0..032534f9161e6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeInfo.java @@ -29,6 +29,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.task.DropPipePlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.OperateMultiplePipesPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusPlanV2; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusWithStoppedByRuntimeExceptionPlanV2; import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; import org.apache.iotdb.confignode.manager.pipe.agent.runtime.PipeConfigRegionListener; import org.apache.iotdb.confignode.manager.pipe.agent.task.PipeConfigNodeSubtask; @@ -124,6 +125,25 @@ public TSStatus setPipeStatus(final SetPipeStatusPlanV2 plan) { } } + public TSStatus setPipeStatusWithStoppedByRuntimeException( + final SetPipeStatusWithStoppedByRuntimeExceptionPlanV2 plan) { + try { + pipeTaskInfo.setPipeStatusWithStoppedByRuntimeException(plan); + + PipeConfigNodeAgent.task() + .handleSinglePipeMetaChanges(pipeTaskInfo.getPipeMetaByPipeName(plan.getPipeName())); + PipeTemporaryMetaInCoordinatorMetrics.getInstance() + .handleTemporaryMetaChanges(pipeTaskInfo.getPipeMetaList()); + return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } catch (final Exception e) { + LOGGER.error("Failed to set pipe status with stopped-by-runtime-exception flag", e); + return new TSStatus(TSStatusCode.PIPE_ERROR.getStatusCode()) + .setMessage( + "Failed to set pipe status with stopped-by-runtime-exception flag, because " + + e.getMessage()); + } + } + public TSStatus dropPipe(final DropPipePlanV2 plan) { try { final Optional pipeMetaBeforeDrop = diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java index 199b835c839e2..022144f1a9159 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java @@ -48,6 +48,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.task.DropPipePlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.OperateMultiplePipesPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusPlanV2; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusWithStoppedByRuntimeExceptionPlanV2; import org.apache.iotdb.confignode.consensus.response.pipe.task.PipeTableResp; import org.apache.iotdb.confignode.manager.pipe.resource.PipeConfigNodeResourceManager; import org.apache.iotdb.confignode.procedure.impl.pipe.runtime.PipeHandleMetaChangeProcedure; @@ -523,6 +524,25 @@ public TSStatus setPipeStatus(final SetPipeStatusPlanV2 plan) { } } + public TSStatus setPipeStatusWithStoppedByRuntimeException( + final SetPipeStatusWithStoppedByRuntimeExceptionPlanV2 plan) { + acquireWriteLock(); + try { + pipeMetaKeeper + .getPipeMeta(plan.getPipeName()) + .getRuntimeMeta() + .getStatus() + .set(plan.getPipeStatus()); + pipeMetaKeeper + .getPipeMeta(plan.getPipeName()) + .getRuntimeMeta() + .setIsStoppedByRuntimeException(plan.isStoppedByRuntimeException()); + return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } finally { + releaseWriteLock(); + } + } + public TSStatus dropPipe(final DropPipePlanV2 plan) { acquireWriteLock(); try { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java index 5349cc65640d0..b2e1a584ec54a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java @@ -21,7 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStatus; -import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusPlanV2; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusWithStoppedByRuntimeExceptionPlanV2; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.impl.pipe.AbstractOperatePipeProcedureV2; import org.apache.iotdb.confignode.procedure.impl.pipe.PipeTaskOperation; @@ -44,6 +44,7 @@ public class StopPipeProcedureV2 extends AbstractOperatePipeProcedureV2 { private static final Logger LOGGER = LoggerFactory.getLogger(StopPipeProcedureV2.class); private String pipeName; + private boolean isStoppedByRuntimeExceptionBeforeStop; public StopPipeProcedureV2() { super(); @@ -71,7 +72,8 @@ public boolean executeFromValidateTask(ConfigNodeProcedureEnv env) throws PipeEx @Override public void executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) throws PipeException { LOGGER.info("StopPipeProcedureV2: executeFromCalculateInfoForTask({})", pipeName); - // Do nothing + isStoppedByRuntimeExceptionBeforeStop = + pipeTaskInfo.get().isStoppedByRuntimeException(pipeName); } @Override @@ -83,7 +85,9 @@ public void executeFromWriteConfigNodeConsensus(ConfigNodeProcedureEnv env) thro response = env.getConfigManager() .getConsensusManager() - .write(new SetPipeStatusPlanV2(pipeName, PipeStatus.STOPPED)); + .write( + new SetPipeStatusWithStoppedByRuntimeExceptionPlanV2( + pipeName, PipeStatus.STOPPED, false)); } catch (ConsensusException e) { LOGGER.warn("Failed in the write API executing the consensus layer due to: ", e); response = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -128,7 +132,9 @@ public void rollbackFromWriteConfigNodeConsensus(ConfigNodeProcedureEnv env) { response = env.getConfigManager() .getConsensusManager() - .write(new SetPipeStatusPlanV2(pipeName, PipeStatus.RUNNING)); + .write( + new SetPipeStatusWithStoppedByRuntimeExceptionPlanV2( + pipeName, PipeStatus.RUNNING, isStoppedByRuntimeExceptionBeforeStop)); } catch (ConsensusException e) { LOGGER.warn("Failed in the write API executing the consensus layer due to: ", e); response = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -159,12 +165,16 @@ public void serialize(DataOutputStream stream) throws IOException { stream.writeShort(ProcedureType.STOP_PIPE_PROCEDURE_V2.getTypeCode()); super.serialize(stream); ReadWriteIOUtils.write(pipeName, stream); + ReadWriteIOUtils.write(isStoppedByRuntimeExceptionBeforeStop, stream); } @Override public void deserialize(ByteBuffer byteBuffer) { super.deserialize(byteBuffer); pipeName = ReadWriteIOUtils.readString(byteBuffer); + // Legacy persisted procedures do not carry this field. + isStoppedByRuntimeExceptionBeforeStop = + byteBuffer.hasRemaining() && ReadWriteIOUtils.readBool(byteBuffer); } @Override @@ -179,11 +189,17 @@ public boolean equals(Object o) { return getProcId() == that.getProcId() && getCurrentState().equals(that.getCurrentState()) && getCycles() == that.getCycles() + && isStoppedByRuntimeExceptionBeforeStop == that.isStoppedByRuntimeExceptionBeforeStop && pipeName.equals(that.pipeName); } @Override public int hashCode() { - return Objects.hash(getProcId(), getCurrentState(), getCycles(), pipeName); + return Objects.hash( + getProcId(), + getCurrentState(), + getCycles(), + pipeName, + isStoppedByRuntimeExceptionBeforeStop); } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java index be2ba4b696017..3b0554baa11fa 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java @@ -95,6 +95,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.task.DropPipePlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.OperateMultiplePipesPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusPlanV2; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusWithStoppedByRuntimeExceptionPlanV2; import org.apache.iotdb.confignode.consensus.request.write.procedure.DeleteProcedurePlan; import org.apache.iotdb.confignode.consensus.request.write.procedure.UpdateProcedurePlan; import org.apache.iotdb.confignode.consensus.request.write.quota.SetSpaceQuotaPlan; @@ -902,6 +903,28 @@ public void SetPipeStatusPlanV2Test() throws IOException { Assert.assertEquals(setPipeStatusPlanV2.getPipeStatus(), setPipeStatusPlanV21.getPipeStatus()); } + @Test + public void SetPipeStatusWithStoppedByRuntimeExceptionPlanV2Test() throws IOException { + final SetPipeStatusWithStoppedByRuntimeExceptionPlanV2 + setPipeStatusWithStoppedByRuntimeExceptionPlanV2 = + new SetPipeStatusWithStoppedByRuntimeExceptionPlanV2( + "pipe", org.apache.iotdb.commons.pipe.agent.task.meta.PipeStatus.STOPPED, true); + final SetPipeStatusWithStoppedByRuntimeExceptionPlanV2 + setPipeStatusWithStoppedByRuntimeExceptionPlanV21 = + (SetPipeStatusWithStoppedByRuntimeExceptionPlanV2) + ConfigPhysicalPlan.Factory.create( + setPipeStatusWithStoppedByRuntimeExceptionPlanV2.serializeToByteBuffer()); + Assert.assertEquals( + setPipeStatusWithStoppedByRuntimeExceptionPlanV2.getPipeName(), + setPipeStatusWithStoppedByRuntimeExceptionPlanV21.getPipeName()); + Assert.assertEquals( + setPipeStatusWithStoppedByRuntimeExceptionPlanV2.getPipeStatus(), + setPipeStatusWithStoppedByRuntimeExceptionPlanV21.getPipeStatus()); + Assert.assertEquals( + setPipeStatusWithStoppedByRuntimeExceptionPlanV2.isStoppedByRuntimeException(), + setPipeStatusWithStoppedByRuntimeExceptionPlanV21.isStoppedByRuntimeException()); + } + @Test public void DropPipePlanV2Test() throws IOException { DropPipePlanV2 dropPipePlanV2 = new DropPipePlanV2("demo"); @@ -944,7 +967,7 @@ public void OperateMultiplePipesPlanV2Test() throws IOException { new SetPipeStatusPlanV2( "testSet", org.apache.iotdb.commons.pipe.agent.task.meta.PipeStatus.RUNNING); - List subPlans = new ArrayList<>(); + final List subPlans = new ArrayList<>(); subPlans.add(createPipePlanV2); subPlans.add(alterPipePlanV2); subPlans.add(dropPipePlanV2); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2Test.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2Test.java index e6d676ea27b1c..e3da356059b5e 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2Test.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2Test.java @@ -19,18 +19,73 @@ package org.apache.iotdb.confignode.procedure.impl.pipe.task; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeRuntimeMeta; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStaticMeta; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStatus; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.CreatePipePlanV2; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.SetPipeStatusWithStoppedByRuntimeExceptionPlanV2; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; +import org.apache.iotdb.confignode.persistence.pipe.PipeTaskInfo; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.store.ProcedureFactory; +import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.utils.PublicBAOS; import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; import java.io.DataOutputStream; import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicReference; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; public class StopPipeProcedureV2Test { + + private static class TestStopPipeProcedureV2 extends StopPipeProcedureV2 { + + private TestStopPipeProcedureV2(final String pipeName) throws PipeException { + super(pipeName); + } + + private void setPipeTaskInfo(final PipeTaskInfo pipeTaskInfo) { + this.pipeTaskInfo = new AtomicReference<>(pipeTaskInfo); + } + } + + private static PipeTaskInfo createExceptionStoppedPipeTaskInfo(final String pipeName) { + final PipeTaskInfo pipeTaskInfo = new PipeTaskInfo(); + + final PipeTaskMeta pipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); + final ConcurrentMap pipeTasks = new ConcurrentHashMap<>(); + pipeTasks.put(1, pipeTaskMeta); + + final PipeStaticMeta pipeStaticMeta = + new PipeStaticMeta( + pipeName, + System.currentTimeMillis(), + Collections.singletonMap("extractor", "iotdb-source"), + Collections.singletonMap("processor", "do-nothing-processor"), + Collections.singletonMap("connector", "iotdb-thrift-connector")); + final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(pipeTasks); + pipeRuntimeMeta.getStatus().set(PipeStatus.STOPPED); + pipeRuntimeMeta.setIsStoppedByRuntimeException(true); + + pipeTaskInfo.createPipe(new CreatePipePlanV2(pipeStaticMeta, pipeRuntimeMeta)); + return pipeTaskInfo; + } + @Test public void serializeDeserializeTest() { PublicBAOS byteArrayOutputStream = new PublicBAOS(); @@ -50,4 +105,54 @@ public void serializeDeserializeTest() { fail(); } } + + @Test + public void serializeDeserializeLegacyFormatTest() { + final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); + + final StopPipeProcedureV2 proc = new StopPipeProcedureV2("testPipe"); + + try { + proc.serialize(outputStream); + final ByteBuffer buffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size() - 1); + final StopPipeProcedureV2 proc2 = + (StopPipeProcedureV2) ProcedureFactory.getInstance().create(buffer); + + assertEquals(proc, proc2); + } catch (Exception e) { + fail(); + } + } + + @Test + public void testStopPipeWritesStatusAndRuntimeExceptionFlagAtomically() throws Exception { + final String pipeName = "testPipe"; + final TestStopPipeProcedureV2 proc = new TestStopPipeProcedureV2(pipeName); + proc.setPipeTaskInfo(createExceptionStoppedPipeTaskInfo(pipeName)); + proc.executeFromCalculateInfoForTask(Mockito.mock(ConfigNodeProcedureEnv.class)); + + final ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + final ConfigManager configManager = Mockito.mock(ConfigManager.class); + final ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); + Mockito.when(consensusManager.write(Mockito.any(ConfigPhysicalPlan.class))) + .thenReturn(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + + proc.executeFromWriteConfigNodeConsensus(env); + proc.rollbackFromWriteConfigNodeConsensus(env); + + final ArgumentCaptor planCaptor = + ArgumentCaptor.forClass(ConfigPhysicalPlan.class); + Mockito.verify(consensusManager, Mockito.times(2)).write(planCaptor.capture()); + + assertEquals( + new SetPipeStatusWithStoppedByRuntimeExceptionPlanV2(pipeName, PipeStatus.STOPPED, false), + planCaptor.getAllValues().get(0)); + assertEquals( + new SetPipeStatusWithStoppedByRuntimeExceptionPlanV2(pipeName, PipeStatus.RUNNING, true), + planCaptor.getAllValues().get(1)); + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index ec9ce06fd1d41..22e282d85a83f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -215,8 +215,8 @@ public class CommonConfig { private int pipeDataStructureTabletRowSize = 2048; - // 128MB - private int pipeDataStructureTabletSizeInBytes = 60 * 1024 * 1024; + // 16MB + private int pipeDataStructureTabletSizeInBytes = 16 * 1024 * 1024; private double pipeDataStructureTabletMemoryBlockAllocationRejectThreshold = 0.3; private double pipeDataStructureTsFileMemoryBlockAllocationRejectThreshold = 0.3; private volatile double pipeTotalFloatingMemoryProportion = 0.5; From 0e9ee216bd5c5b8c3c74e6ec3b9b36de6e907e08 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 7 May 2026 09:34:46 +0800 Subject: [PATCH 032/102] Subscription: Fixed multiple bugs (#17563) (#17599) --- .../CreateSubscriptionProcedure.java | 6 +- .../DropSubscriptionProcedure.java | 6 +- .../CreateSubscriptionProcedureTest.java | 134 ++++++++++++++++++ .../DropSubscriptionProcedureTest.java | 99 +++++++++++++ .../meta/consumer/ConsumerGroupMeta.java | 7 +- .../consumer/ConsumerGroupDeSerTest.java | 20 +++ 6 files changed, 266 insertions(+), 6 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedure.java index 80e4e5115541a..6d468908ed71a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedure.java @@ -87,6 +87,9 @@ protected boolean executeFromValidate(final ConfigNodeProcedureEnv env) throws SubscriptionException { LOGGER.info("CreateSubscriptionProcedure: executeFromValidate"); + alterConsumerGroupProcedure = null; + createPipeProcedures = new ArrayList<>(); + subscriptionInfo.get().validateBeforeSubscribe(subscribeReq); // Construct AlterConsumerGroupProcedure @@ -160,8 +163,7 @@ protected void executeFromOperateOnConfigNodes(final ConfigNodeProcedureEnv env) response = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); response.setMessage(e.getMessage()); } - if (response.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() - && response.getSubStatusSize() > 0) { + if (response.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { throw new SubscriptionException( String.format( "Failed to create subscription with request %s on config nodes, because %s", diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/DropSubscriptionProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/DropSubscriptionProcedure.java index 6741a6c1e2a84..6f668f29c5dac 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/DropSubscriptionProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/DropSubscriptionProcedure.java @@ -85,6 +85,9 @@ protected boolean executeFromValidate(final ConfigNodeProcedureEnv env) throws SubscriptionException { LOGGER.info("DropSubscriptionProcedure: executeFromValidate"); + alterConsumerGroupProcedure = null; + dropPipeProcedures = new ArrayList<>(); + subscriptionInfo.get().validateBeforeUnsubscribe(unsubscribeReq); // Construct AlterConsumerGroupProcedure @@ -141,8 +144,7 @@ protected void executeFromOperateOnConfigNodes(final ConfigNodeProcedureEnv env) response = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); response.setMessage(e.getMessage()); } - if (response.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() - && response.getSubStatusSize() > 0) { + if (response.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { throw new SubscriptionException( String.format( "Failed to drop subscription with request %s on config nodes, because %s", diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedureTest.java index 93d9941fbf333..e2a4d0615d886 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/CreateSubscriptionProcedureTest.java @@ -19,18 +19,36 @@ package org.apache.iotdb.confignode.procedure.impl.subscription.subscription; +import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.subscription.meta.consumer.ConsumerGroupMeta; import org.apache.iotdb.commons.subscription.meta.consumer.ConsumerMeta; +import org.apache.iotdb.commons.subscription.meta.topic.TopicMeta; +import org.apache.iotdb.confignode.consensus.request.write.pipe.task.CreatePipePlanV2; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.PermissionManager; +import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; +import org.apache.iotdb.confignode.manager.load.LoadManager; +import org.apache.iotdb.confignode.manager.pipe.coordinator.PipeManager; +import org.apache.iotdb.confignode.manager.pipe.coordinator.plugin.PipePluginCoordinator; +import org.apache.iotdb.confignode.persistence.pipe.PipePluginInfo; +import org.apache.iotdb.confignode.persistence.pipe.PipeTaskInfo; +import org.apache.iotdb.confignode.persistence.subscription.SubscriptionInfo; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.impl.pipe.task.CreatePipeProcedureV2; import org.apache.iotdb.confignode.procedure.impl.subscription.consumer.AlterConsumerGroupProcedure; import org.apache.iotdb.confignode.procedure.store.ProcedureFactory; import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq; import org.apache.iotdb.confignode.rpc.thrift.TSubscribeReq; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.tsfile.utils.PublicBAOS; +import org.junit.Assert; import org.junit.Test; +import org.mockito.Mockito; import java.io.DataOutputStream; +import java.lang.reflect.Field; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; @@ -39,6 +57,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -102,4 +121,119 @@ public void serializeDeserializeTest() { fail(); } } + + @Test + public void executeFromOperateOnConfigNodesShouldFailOnTopLevelConsensusError() throws Exception { + final CreateSubscriptionProcedure proc = + new CreateSubscriptionProcedure( + new TSubscribeReq( + "old_consumer", "test_consumer_group", Collections.singleton("test_topic"))); + proc.setAlterConsumerGroupProcedure(Mockito.mock(AlterConsumerGroupProcedure.class)); + + final CreatePipeProcedureV2 createPipeProcedure = Mockito.mock(CreatePipeProcedureV2.class); + Mockito.when(createPipeProcedure.constructPlan()) + .thenReturn(Mockito.mock(CreatePipePlanV2.class)); + proc.setCreatePipeProcedures(Collections.singletonList(createPipeProcedure)); + + try { + proc.executeFromOperateOnConfigNodes( + mockConsensusFailureEnv( + new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()) + .setMessage("consensus write failed"))); + fail(); + } catch (SubscriptionException e) { + Assert.assertTrue(e.getMessage().contains("Failed to create subscription")); + } + } + + @Test + public void executeFromValidateShouldResetCreatePipeProceduresOnRetry() throws Exception { + final Map consumerAttributes = new HashMap<>(); + consumerAttributes.put("username", "user"); + consumerAttributes.put("password", "password"); + + final ConsumerGroupMeta consumerGroupMeta = + new ConsumerGroupMeta( + "test_consumer_group", 1, new ConsumerMeta("old_consumer", 1, consumerAttributes)); + final TopicMeta topicMeta = new TopicMeta("test_topic", 1, Collections.emptyMap()); + + final SubscriptionInfo subscriptionInfo = Mockito.mock(SubscriptionInfo.class); + Mockito.when(subscriptionInfo.getConsumerGroupMeta("test_consumer_group")) + .thenReturn(consumerGroupMeta); + Mockito.when(subscriptionInfo.deepCopyConsumerGroupMeta("test_consumer_group")) + .thenAnswer(invocation -> consumerGroupMeta.deepCopy()); + Mockito.when( + subscriptionInfo.isTopicSubscribedByConsumerGroup("test_topic", "test_consumer_group")) + .thenReturn(false); + Mockito.when(subscriptionInfo.deepCopyTopicMeta("test_topic")).thenReturn(topicMeta); + + final PipeTaskInfo pipeTaskInfo = Mockito.mock(PipeTaskInfo.class); + Mockito.when(pipeTaskInfo.checkBeforeCreatePipe(Mockito.any(TCreatePipeReq.class))) + .thenReturn(true); + + final CreateSubscriptionProcedure proc = + new CreateSubscriptionProcedure( + new TSubscribeReq( + "old_consumer", "test_consumer_group", Collections.singleton("test_topic"))); + setField(proc, "subscriptionInfo", new AtomicReference<>(subscriptionInfo)); + setField(proc, "pipeTaskInfo", new AtomicReference<>(pipeTaskInfo)); + + final ConfigNodeProcedureEnv env = mockCreateSubscriptionValidationEnv(); + proc.executeFromValidate(env); + Assert.assertEquals(1, proc.getCreatePipeProcedures().size()); + + proc.executeFromValidate(env); + Assert.assertEquals(1, proc.getCreatePipeProcedures().size()); + } + + private static ConfigNodeProcedureEnv mockConsensusFailureEnv(final TSStatus response) + throws Exception { + final ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + final ConfigManager configManager = Mockito.mock(ConfigManager.class); + final ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); + + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); + Mockito.when(consensusManager.write(Mockito.any())).thenReturn(response); + + return env; + } + + private static ConfigNodeProcedureEnv mockCreateSubscriptionValidationEnv() { + final ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + final ConfigManager configManager = Mockito.mock(ConfigManager.class); + final PermissionManager permissionManager = Mockito.mock(PermissionManager.class); + final PipeManager pipeManager = Mockito.mock(PipeManager.class); + final PipePluginCoordinator pipePluginCoordinator = Mockito.mock(PipePluginCoordinator.class); + final PipePluginInfo pipePluginInfo = Mockito.mock(PipePluginInfo.class); + final LoadManager loadManager = Mockito.mock(LoadManager.class); + + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getPermissionManager()).thenReturn(permissionManager); + Mockito.when(configManager.getPipeManager()).thenReturn(pipeManager); + Mockito.when(pipeManager.getPipePluginCoordinator()).thenReturn(pipePluginCoordinator); + Mockito.when(pipePluginCoordinator.getPipePluginInfo()).thenReturn(pipePluginInfo); + Mockito.when(configManager.getLoadManager()).thenReturn(loadManager); + Mockito.when(loadManager.getRegionLeaderMap()).thenReturn(Collections.emptyMap()); + Mockito.when(permissionManager.login4Pipe(Mockito.anyString(), Mockito.any())) + .thenReturn("hashedPassword"); + + return env; + } + + private static void setField(final Object target, final String fieldName, final Object value) + throws Exception { + Class clazz = target.getClass(); + while (clazz != null) { + try { + final Field field = clazz.getDeclaredField(fieldName); + field.setAccessible(true); + field.set(target, value); + return; + } catch (NoSuchFieldException e) { + clazz = clazz.getSuperclass(); + } + } + throw new NoSuchFieldException(fieldName); + } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/DropSubscriptionProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/DropSubscriptionProcedureTest.java index 9ecce2a522ca4..910648bbe5132 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/DropSubscriptionProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/subscription/subscription/DropSubscriptionProcedureTest.java @@ -19,24 +19,37 @@ package org.apache.iotdb.confignode.procedure.impl.subscription.subscription; +import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.subscription.meta.consumer.ConsumerGroupMeta; import org.apache.iotdb.commons.subscription.meta.consumer.ConsumerMeta; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; +import org.apache.iotdb.confignode.persistence.pipe.PipeTaskInfo; +import org.apache.iotdb.confignode.persistence.subscription.SubscriptionInfo; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.impl.pipe.task.DropPipeProcedureV2; import org.apache.iotdb.confignode.procedure.impl.subscription.consumer.AlterConsumerGroupProcedure; import org.apache.iotdb.confignode.procedure.store.ProcedureFactory; import org.apache.iotdb.confignode.rpc.thrift.TUnsubscribeReq; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.rpc.subscription.exception.SubscriptionException; import org.apache.tsfile.utils.PublicBAOS; +import org.junit.Assert; import org.junit.Test; +import org.mockito.Mockito; import java.io.DataOutputStream; +import java.lang.reflect.Field; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -91,4 +104,90 @@ public void serializeDeserializeTest() { fail(); } } + + @Test + public void executeFromOperateOnConfigNodesShouldFailOnTopLevelConsensusError() throws Exception { + final DropSubscriptionProcedure proc = + new DropSubscriptionProcedure( + new TUnsubscribeReq( + "old_consumer", "test_consumer_group", Collections.singleton("test_topic"))); + proc.setAlterConsumerGroupProcedure(Mockito.mock(AlterConsumerGroupProcedure.class)); + + final DropPipeProcedureV2 dropPipeProcedure = Mockito.mock(DropPipeProcedureV2.class); + Mockito.when(dropPipeProcedure.getPipeName()).thenReturn("pipe_topic"); + proc.setDropPipeProcedures(Collections.singletonList(dropPipeProcedure)); + + try { + proc.executeFromOperateOnConfigNodes( + mockConsensusFailureEnv( + new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()) + .setMessage("consensus write failed"))); + fail(); + } catch (SubscriptionException e) { + Assert.assertTrue(e.getMessage().contains("Failed to drop subscription")); + } + } + + @Test + public void executeFromValidateShouldResetDropPipeProceduresOnRetry() throws Exception { + final Map consumerAttributes = new HashMap<>(); + consumerAttributes.put("username", "user"); + consumerAttributes.put("password", "password"); + + final ConsumerGroupMeta consumerGroupMeta = + new ConsumerGroupMeta( + "test_consumer_group", 1, new ConsumerMeta("old_consumer", 1, consumerAttributes)); + consumerGroupMeta.addSubscription("old_consumer", Collections.singleton("test_topic")); + + final SubscriptionInfo subscriptionInfo = Mockito.mock(SubscriptionInfo.class); + Mockito.when(subscriptionInfo.getConsumerGroupMeta("test_consumer_group")) + .thenReturn(consumerGroupMeta); + Mockito.when(subscriptionInfo.deepCopyConsumerGroupMeta("test_consumer_group")) + .thenAnswer(invocation -> consumerGroupMeta.deepCopy()); + + final PipeTaskInfo pipeTaskInfo = Mockito.mock(PipeTaskInfo.class); + + final DropSubscriptionProcedure proc = + new DropSubscriptionProcedure( + new TUnsubscribeReq( + "old_consumer", "test_consumer_group", Collections.singleton("test_topic"))); + setField(proc, "subscriptionInfo", new AtomicReference<>(subscriptionInfo)); + setField(proc, "pipeTaskInfo", new AtomicReference<>(pipeTaskInfo)); + + final ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + proc.executeFromValidate(env); + Assert.assertEquals(1, proc.getDropPipeProcedures().size()); + + proc.executeFromValidate(env); + Assert.assertEquals(1, proc.getDropPipeProcedures().size()); + } + + private static ConfigNodeProcedureEnv mockConsensusFailureEnv(final TSStatus response) + throws Exception { + final ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + final ConfigManager configManager = Mockito.mock(ConfigManager.class); + final ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); + + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); + Mockito.when(consensusManager.write(Mockito.any())).thenReturn(response); + + return env; + } + + private static void setField(final Object target, final String fieldName, final Object value) + throws Exception { + Class clazz = target.getClass(); + while (clazz != null) { + try { + final Field field = clazz.getDeclaredField(fieldName); + field.setAccessible(true); + field.set(target, value); + return; + } catch (NoSuchFieldException e) { + clazz = clazz.getSuperclass(); + } + } + throw new NoSuchFieldException(fieldName); + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerGroupMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerGroupMeta.java index 498f3427690f5..640d154937b4b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerGroupMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/meta/consumer/ConsumerGroupMeta.java @@ -69,8 +69,11 @@ public ConsumerGroupMeta deepCopy() { final ConsumerGroupMeta copied = new ConsumerGroupMeta(); copied.consumerGroupId = consumerGroupId; copied.creationTime = creationTime; - copied.topicNameToSubscribedConsumerIdSet = - new ConcurrentHashMap<>(topicNameToSubscribedConsumerIdSet); + copied.topicNameToSubscribedConsumerIdSet = new ConcurrentHashMap<>(); + topicNameToSubscribedConsumerIdSet.forEach( + (topicName, subscribedConsumerIds) -> + copied.topicNameToSubscribedConsumerIdSet.put( + topicName, new HashSet<>(subscribedConsumerIds))); copied.consumerIdToConsumerMeta = new ConcurrentHashMap<>(consumerIdToConsumerMeta); copied.topicNameToSubscriptionCreationTime = new ConcurrentHashMap<>(topicNameToSubscriptionCreationTime); diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/subscription/consumer/ConsumerGroupDeSerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/subscription/consumer/ConsumerGroupDeSerTest.java index d41b3706cd1f2..9ef7191ba6683 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/subscription/consumer/ConsumerGroupDeSerTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/subscription/consumer/ConsumerGroupDeSerTest.java @@ -69,4 +69,24 @@ public void test() throws IOException { consumerGroupMeta.getConsumerGroupId(), consumerGroupMeta2.getConsumerGroupId()); Assert.assertEquals(consumerGroupMeta.getCreationTime(), consumerGroupMeta2.getCreationTime()); } + + @Test + public void testDeepCopyShouldNotShareSubscribedConsumerSets() { + Map consumerAttributes = new HashMap<>(); + consumerAttributes.put("username", "user"); + consumerAttributes.put("password", "password"); + + ConsumerGroupMeta consumerGroupMeta = + new ConsumerGroupMeta( + "test_consumer_group", 1, new ConsumerMeta("test_consumer1", 1, consumerAttributes)); + consumerGroupMeta.addSubscription("test_consumer1", Collections.singleton("test_topic")); + + ConsumerGroupMeta copiedConsumerGroupMeta = consumerGroupMeta.deepCopy(); + copiedConsumerGroupMeta.removeSubscription( + "test_consumer1", Collections.singleton("test_topic")); + + Assert.assertTrue( + consumerGroupMeta.getConsumersSubscribingTopic("test_topic").contains("test_consumer1")); + Assert.assertTrue(copiedConsumerGroupMeta.getConsumersSubscribingTopic("test_topic").isEmpty()); + } } From 8dc2e37708fc5d3803533d6d8967b11b6977f4ac Mon Sep 17 00:00:00 2001 From: Jackie Tien Date: Thu, 7 May 2026 15:52:40 +0800 Subject: [PATCH 033/102] [To dev/1.3] Refuse unreasonable string length in thrift frame (cherry picked from commit f5fbaa2a3c9da6743a06e07fa680fe3714a89b14) --- .../iotdb/rpc/TElasticFramedTransport.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java index 2524e8ae4cd33..40811a00f41df 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TElasticFramedTransport.java @@ -224,7 +224,8 @@ private enum FrameError { + "requests%s to the Non-SSL Thrift-RPC port, please confirm that you are using " + "the right configuration"), NEGATIVE_FRAME_SIZE("Read a negative frame size (%d)%s!"), - FRAME_SIZE_EXCEEDED("Frame size (%d) larger than protect max size (%d)%s!"); + FRAME_SIZE_EXCEEDED("Frame size (%d) larger than protect max size (%d)%s!"), + STRING_LENGTH_EXCEEDED("String length (%d) larger than protect max size (%d)%s!"); private final String messageFormat; @@ -232,9 +233,9 @@ private enum FrameError { this.messageFormat = messageFormat; } - void throwException(int size, String remoteInfo, int maxSize) throws TTransportException { + void throwException(long size, String remoteInfo, int maxSize) throws TTransportException { String message = - (this == FRAME_SIZE_EXCEEDED) + (this == FRAME_SIZE_EXCEEDED || this == STRING_LENGTH_EXCEEDED) ? String.format(messageFormat, size, maxSize, remoteInfo) : String.format(messageFormat, size, remoteInfo); throw new TTransportException(TTransportException.CORRUPTED_DATA, message); @@ -277,8 +278,15 @@ public void updateKnownMessageSize(long size) throws TTransportException { @Override public void checkReadBytesAvailable(long numBytes) throws TTransportException { - // do nothing now. - // here we can do some checkm, e.g., see whether the memory is enough. + if (numBytes >= thriftMaxFrameSize) { + SocketAddress remoteAddress = null; + if (underlying instanceof TSocket) { + remoteAddress = ((TSocket) underlying).getSocket().getRemoteSocketAddress(); + } + String remoteInfo = (remoteAddress == null) ? "" : " from " + remoteAddress; + close(); + FrameError.STRING_LENGTH_EXCEEDED.throwException(numBytes, remoteInfo, thriftMaxFrameSize); + } } @Override From 310dfd1c282d3d6e222ef880e0c886ea059a2741 Mon Sep 17 00:00:00 2001 From: Weihao Li <60659567+Wei-hao-Li@users.noreply.github.com> Date: Fri, 8 May 2026 10:33:54 +0800 Subject: [PATCH 034/102] [To dev/1.3] Fix NPE because of thread of DriverScheduler was not closed immediately when stop DN Signed-off-by: Weihao Li <18110526956@163.com> --- .../iotdb/db/queryengine/execution/schedule/DriverScheduler.java | 1 + 1 file changed, 1 insertion(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/DriverScheduler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/DriverScheduler.java index a629c151fa901..183e51b7e1ef1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/DriverScheduler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/DriverScheduler.java @@ -162,6 +162,7 @@ public void stop() { t -> { try { t.close(); + t.interrupt(); } catch (IOException e) { // Only a field is set, there's no chance to throw an IOException } From 77e9403058bf7e2197c26cd40ca1469310020e83 Mon Sep 17 00:00:00 2001 From: Jackie Tien Date: Fri, 8 May 2026 14:43:28 +0800 Subject: [PATCH 035/102] [To dev/1.3] Clean up dead RPC thread config and use node-specific selectorNum (#17618) --- .../apache/iotdb/it/env/cluster/EnvUtils.java | 2 +- .../it/env/cluster/config/MppBaseConfig.java | 2 + .../async/AsyncAINodeHeartbeatClientPool.java | 4 +- .../AsyncConfigNodeHeartbeatClientPool.java | 4 +- .../AsyncDataNodeHeartbeatClientPool.java | 4 +- ...oCnInternalServiceAsyncRequestManager.java | 8 ++- ...oDnInternalServiceAsyncRequestManager.java | 5 ++ .../confignode/conf/ConfigNodeConfig.java | 28 +++++++++ .../confignode/conf/ConfigNodeDescriptor.java | 18 ++++++ .../consensus/config/IoTConsensusConfig.java | 22 ------- .../consensus/config/PipeConsensusConfig.java | 28 --------- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 43 ++++--------- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 49 +++++---------- .../db/consensus/DataRegionConsensusImpl.java | 4 -- .../client/DataNodeClientPoolFactory.java | 28 --------- ...oCnInternalServiceAsyncRequestManager.java | 5 ++ ...odeExternalServiceAsyncRequestManager.java | 10 +++- ...DataNodeMPPServiceAsyncRequestManager.java | 10 +++- ...oDnInternalServiceAsyncRequestManager.java | 5 ++ .../db/queryengine/plan/Coordinator.java | 13 +--- .../executor/ClusterConfigTaskExecutor.java | 24 +++----- .../plan/planner/TreeModelPlanner.java | 9 --- .../plan/scheduler/ClusterScheduler.java | 5 -- .../FragmentInstanceDispatcherImpl.java | 7 --- .../conf/iotdb-system.properties.template | 32 ++++++---- .../commons/client/ClientPoolFactory.java | 60 ++++++++++++++++--- .../client/property/ClientPoolProperty.java | 16 ++++- .../client/property/ThriftClientProperty.java | 5 +- .../client/request/AsyncRequestManager.java | 6 +- ...odeInternalServiceAsyncRequestManager.java | 10 +++- ...DataNodeInternalServiceRequestManager.java | 10 +++- .../iotdb/commons/concurrent/ThreadName.java | 2 - .../iotdb/commons/conf/CommonConfig.java | 15 ++++- .../iotdb/commons/conf/CommonDescriptor.java | 36 +++++++++-- .../commons/client/ClientManagerTest.java | 19 +++++- 35 files changed, 304 insertions(+), 244 deletions(-) diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/EnvUtils.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/EnvUtils.java index 9663fa371e95a..136399a3778b0 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/EnvUtils.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/EnvUtils.java @@ -117,7 +117,7 @@ private static String getWindowsSearchPortCmd(final List ports) { } private static String getUnixSearchPortCmd(final List ports) { - return "lsof -iTCP -sTCP:LISTEN -P -n | awk '{print $9}' | grep -E " + return "lsof -iTCP -sTCP:LISTEN,TIME_WAIT -P -n | awk '{print $9}' | grep -E " + ports.stream().map(String::valueOf).collect(Collectors.joining("|")) + "\""; } diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppBaseConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppBaseConfig.java index 473d2f1b64c7f..975ae1a5682df 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppBaseConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppBaseConfig.java @@ -40,6 +40,8 @@ public abstract class MppBaseConfig { /** Create an empty MppPersistentConfig. */ protected MppBaseConfig() { this.properties = new Properties(); + this.properties.setProperty("cn_selector_thread_nums_of_client_manager", "1"); + this.properties.setProperty("dn_selector_thread_nums_of_client_manager", "1"); } /** diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncAINodeHeartbeatClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncAINodeHeartbeatClientPool.java index e09ccc79becbf..8ec455142b8ba 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncAINodeHeartbeatClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncAINodeHeartbeatClientPool.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.ainode.AsyncAINodeServiceClient; import org.apache.iotdb.confignode.client.async.handlers.heartbeat.AINodeHeartbeatHandler; +import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; public class AsyncAINodeHeartbeatClientPool { @@ -34,7 +35,8 @@ private AsyncAINodeHeartbeatClientPool() { clientManager = new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncAINodeHeartbeatServiceClientPoolFactory()); + new ClientPoolFactory.AsyncAINodeHeartbeatServiceClientPoolFactory( + ConfigNodeDescriptor.getInstance().getConf().getSelectorNumOfClientManager())); } public void getAINodeHeartBeat( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncConfigNodeHeartbeatClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncConfigNodeHeartbeatClientPool.java index 7b6bca5d0d9e7..a6dffbe0eef63 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncConfigNodeHeartbeatClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncConfigNodeHeartbeatClientPool.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.async.AsyncConfigNodeInternalServiceClient; import org.apache.iotdb.confignode.client.async.handlers.heartbeat.ConfigNodeHeartbeatHandler; +import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.confignode.rpc.thrift.TConfigNodeHeartbeatReq; public class AsyncConfigNodeHeartbeatClientPool { @@ -34,7 +35,8 @@ private AsyncConfigNodeHeartbeatClientPool() { clientManager = new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncConfigNodeHeartbeatServiceClientPoolFactory()); + new ClientPoolFactory.AsyncConfigNodeHeartbeatServiceClientPoolFactory( + ConfigNodeDescriptor.getInstance().getConf().getSelectorNumOfClientManager())); } /** diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeHeartbeatClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeHeartbeatClientPool.java index 8d67d150efbbb..18a8120a9e459 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeHeartbeatClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/AsyncDataNodeHeartbeatClientPool.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.async.AsyncDataNodeInternalServiceClient; import org.apache.iotdb.confignode.client.async.handlers.heartbeat.DataNodeHeartbeatHandler; +import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatReq; /** Asynchronously send RPC requests to DataNodes. See queryengine.thrift for more details. */ @@ -35,7 +36,8 @@ private AsyncDataNodeHeartbeatClientPool() { clientManager = new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncDataNodeHeartbeatServiceClientPoolFactory()); + new ClientPoolFactory.AsyncDataNodeHeartbeatServiceClientPoolFactory( + ConfigNodeDescriptor.getInstance().getConf().getSelectorNumOfClientManager())); } /** diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToCnInternalServiceAsyncRequestManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToCnInternalServiceAsyncRequestManager.java index 19eaf9d9a40d2..00267e5a8ccef 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToCnInternalServiceAsyncRequestManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToCnInternalServiceAsyncRequestManager.java @@ -30,6 +30,7 @@ import org.apache.iotdb.confignode.client.async.handlers.rpc.ConfigNodeAsyncRequestRPCHandler; import org.apache.iotdb.confignode.client.async.handlers.rpc.ConfigNodeTSStatusRPCHandler; import org.apache.iotdb.confignode.client.async.handlers.rpc.SubmitTestConnectionTaskToConfigNodeRPCHandler; +import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +41,10 @@ public class CnToCnInternalServiceAsyncRequestManager private static final Logger LOGGER = LoggerFactory.getLogger(CnToCnInternalServiceAsyncRequestManager.class); + public CnToCnInternalServiceAsyncRequestManager(int selectorNumOfAsyncClientManager) { + super(selectorNumOfAsyncClientManager); + } + @Override protected void initActionMapBuilder() { actionMapBuilder.put( @@ -71,7 +76,8 @@ protected void adjustClientTimeoutIfNecessary( private static class ClientPoolHolder { private static final CnToCnInternalServiceAsyncRequestManager INSTANCE = - new CnToCnInternalServiceAsyncRequestManager(); + new CnToCnInternalServiceAsyncRequestManager( + ConfigNodeDescriptor.getInstance().getConf().getSelectorNumOfClientManager()); private ClientPoolHolder() { // Empty constructor diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java index b9aac92a16504..e4dcaad2bfb96 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/CnToDnInternalServiceAsyncRequestManager.java @@ -46,6 +46,7 @@ import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.CheckSchemaRegionUsingTemplateRPCHandler; import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.ConsumerGroupPushMetaRPCHandler; import org.apache.iotdb.confignode.client.async.handlers.rpc.subscription.TopicPushMetaRPCHandler; +import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; import org.apache.iotdb.mpp.rpc.thrift.TActiveTriggerInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TAlterViewReq; import org.apache.iotdb.mpp.rpc.thrift.TCheckSchemaRegionUsingTemplateReq; @@ -105,6 +106,10 @@ public class CnToDnInternalServiceAsyncRequestManager private static final Logger LOGGER = LoggerFactory.getLogger(CnToDnInternalServiceAsyncRequestManager.class); + private CnToDnInternalServiceAsyncRequestManager() { + super(ConfigNodeDescriptor.getInstance().getConf().getSelectorNumOfClientManager()); + } + @SuppressWarnings("unchecked") @Override protected void initActionMapBuilder() { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java index 03d12ee46a696..77b353ea59d4d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java @@ -136,6 +136,17 @@ public class ConfigNodeConfig { */ private int maxClientNumForEachNode = DefaultProperty.MAX_CLIENT_NUM_FOR_EACH_NODE; + private int maxIdleClientNumForEachNode = DefaultProperty.MAX_IDLE_CLIENT_NUM_FOR_EACH_NODE; + + /** + * ClientManager will have so many selector threads (TAsyncClientManager) to distribute to its + * clients. + */ + private int selectorNumOfClientManager = + Runtime.getRuntime().availableProcessors() / 4 > 0 + ? Runtime.getRuntime().availableProcessors() / 4 + : 1; + /** System directory, including version file for each database and metadata. */ private String systemDir = IoTDBConstant.CN_DEFAULT_DATA_DIR + File.separator + IoTDBConstant.SYSTEM_FOLDER_NAME; @@ -448,6 +459,23 @@ public ConfigNodeConfig setMaxClientNumForEachNode(int maxClientNumForEachNode) return this; } + public int getMaxIdleClientNumForEachNode() { + return maxIdleClientNumForEachNode; + } + + public ConfigNodeConfig setMaxIdleClientNumForEachNode(int maxIdleClientNumForEachNode) { + this.maxIdleClientNumForEachNode = maxIdleClientNumForEachNode; + return this; + } + + public int getSelectorNumOfClientManager() { + return selectorNumOfClientManager; + } + + public void setSelectorNumOfClientManager(int selectorNumOfClientManager) { + this.selectorNumOfClientManager = selectorNumOfClientManager; + } + public String getConsensusDir() { return consensusDir; } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java index 614bae210dfa1..83cf1b612d01d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java @@ -299,6 +299,24 @@ private void loadProperties(TrimProperties properties) throws BadNodeUrlExceptio String.valueOf(conf.getMaxClientNumForEachNode())) .trim())); + int cnMaxIdleClientNumForEachNode = + Integer.parseInt( + properties.getProperty( + "cn_max_idle_client_count_for_each_node_in_client_manager", + String.valueOf(conf.getMaxIdleClientNumForEachNode()))); + if (cnMaxIdleClientNumForEachNode >= 0) { + conf.setMaxIdleClientNumForEachNode(cnMaxIdleClientNumForEachNode); + } + + int cnSelectorNumOfClientManager = + Integer.parseInt( + properties.getProperty( + "cn_selector_thread_nums_of_client_manager", + String.valueOf(conf.getSelectorNumOfClientManager()))); + if (cnSelectorNumOfClientManager > 0) { + conf.setSelectorNumOfClientManager(cnSelectorNumOfClientManager); + } + conf.setSystemDir(properties.getProperty("cn_system_dir", conf.getSystemDir()).trim()); conf.setConsensusDir(properties.getProperty("cn_consensus_dir", conf.getConsensusDir()).trim()); diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/IoTConsensusConfig.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/IoTConsensusConfig.java index 6c89e19291b35..25b084c5329d5 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/IoTConsensusConfig.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/IoTConsensusConfig.java @@ -70,8 +70,6 @@ public Builder setReplication(Replication replication) { public static class RPC { - private final int rpcSelectorThreadNum; - private final int rpcMinConcurrentClientNum; private final int rpcMaxConcurrentClientNum; private final int thriftServerAwaitTimeForStopService; private final boolean isRpcThriftCompressionEnabled; @@ -83,8 +81,6 @@ public static class RPC { private final int maxClientNumForEachNode; private RPC( - int rpcSelectorThreadNum, - int rpcMinConcurrentClientNum, int rpcMaxConcurrentClientNum, int thriftServerAwaitTimeForStopService, boolean isRpcThriftCompressionEnabled, @@ -93,8 +89,6 @@ private RPC( boolean printLogWhenThriftClientEncounterException, int thriftMaxFrameSize, int maxClientNumForEachNode) { - this.rpcSelectorThreadNum = rpcSelectorThreadNum; - this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum; this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum; this.thriftServerAwaitTimeForStopService = thriftServerAwaitTimeForStopService; this.isRpcThriftCompressionEnabled = isRpcThriftCompressionEnabled; @@ -105,14 +99,6 @@ private RPC( this.maxClientNumForEachNode = maxClientNumForEachNode; } - public int getRpcSelectorThreadNum() { - return rpcSelectorThreadNum; - } - - public int getRpcMinConcurrentClientNum() { - return rpcMinConcurrentClientNum; - } - public int getRpcMaxConcurrentClientNum() { return rpcMaxConcurrentClientNum; } @@ -151,7 +137,6 @@ public static RPC.Builder newBuilder() { public static class Builder { - private int rpcSelectorThreadNum = 1; private int rpcMinConcurrentClientNum = Runtime.getRuntime().availableProcessors(); private int rpcMaxConcurrentClientNum = 65535; private int thriftServerAwaitTimeForStopService = 60; @@ -163,11 +148,6 @@ public static class Builder { private int thriftMaxFrameSize = 536870912; private int maxClientNumForEachNode = DefaultProperty.MAX_CLIENT_NUM_FOR_EACH_NODE; - public RPC.Builder setRpcSelectorThreadNum(int rpcSelectorThreadNum) { - this.rpcSelectorThreadNum = rpcSelectorThreadNum; - return this; - } - public RPC.Builder setRpcMinConcurrentClientNum(int rpcMinConcurrentClientNum) { this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum; return this; @@ -218,8 +198,6 @@ public Builder setMaxClientNumForEachNode(int maxClientNumForEachNode) { public RPC build() { return new RPC( - rpcSelectorThreadNum, - rpcMinConcurrentClientNum, rpcMaxConcurrentClientNum, thriftServerAwaitTimeForStopService, isRpcThriftCompressionEnabled, diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java index 2cb149b601b01..81c6e53e73a12 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/config/PipeConsensusConfig.java @@ -79,8 +79,6 @@ public PipeConsensusConfig build() { } public static class RPC { - private final int rpcSelectorThreadNum; - private final int rpcMinConcurrentClientNum; private final int rpcMaxConcurrentClientNum; private final int thriftServerAwaitTimeForStopService; private final boolean isRpcThriftCompressionEnabled; @@ -88,15 +86,11 @@ public static class RPC { private final int thriftMaxFrameSize; public RPC( - int rpcSelectorThreadNum, - int rpcMinConcurrentClientNum, int rpcMaxConcurrentClientNum, int thriftServerAwaitTimeForStopService, boolean isRpcThriftCompressionEnabled, int connectionTimeoutInMs, int thriftMaxFrameSize) { - this.rpcSelectorThreadNum = rpcSelectorThreadNum; - this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum; this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum; this.thriftServerAwaitTimeForStopService = thriftServerAwaitTimeForStopService; this.isRpcThriftCompressionEnabled = isRpcThriftCompressionEnabled; @@ -104,14 +98,6 @@ public RPC( this.thriftMaxFrameSize = thriftMaxFrameSize; } - public int getRpcSelectorThreadNum() { - return rpcSelectorThreadNum; - } - - public int getRpcMinConcurrentClientNum() { - return rpcMinConcurrentClientNum; - } - public int getRpcMaxConcurrentClientNum() { return rpcMaxConcurrentClientNum; } @@ -137,24 +123,12 @@ public static RPC.Builder newBuilder() { } public static class Builder { - private int rpcSelectorThreadNum = 1; - private int rpcMinConcurrentClientNum = Runtime.getRuntime().availableProcessors(); private int rpcMaxConcurrentClientNum = 65535; private int thriftServerAwaitTimeForStopService = 60; private boolean isRpcThriftCompressionEnabled = false; private int connectionTimeoutInMs = (int) TimeUnit.SECONDS.toMillis(60); private int thriftMaxFrameSize = 536870912; - public RPC.Builder setRpcSelectorThreadNum(int rpcSelectorThreadNum) { - this.rpcSelectorThreadNum = rpcSelectorThreadNum; - return this; - } - - public RPC.Builder setRpcMinConcurrentClientNum(int rpcMinConcurrentClientNum) { - this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum; - return this; - } - public RPC.Builder setRpcMaxConcurrentClientNum(int rpcMaxConcurrentClientNum) { this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum; return this; @@ -183,8 +157,6 @@ public RPC.Builder setThriftMaxFrameSize(int thriftMaxFrameSize) { public RPC build() { return new RPC( - rpcSelectorThreadNum, - rpcMinConcurrentClientNum, rpcMaxConcurrentClientNum, thriftServerAwaitTimeForStopService, isRpcThriftCompressionEnabled, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 5e89c4ec083a8..d3f13d1439232 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -139,12 +139,6 @@ public class IoTDBConfig { /** ssl key Store password. */ private String keyStorePwd = ""; - /** Rpc Selector thread num */ - private int rpcSelectorThreadCount = 1; - - /** Min concurrent client number */ - private int rpcMinConcurrentClientNum = Runtime.getRuntime().availableProcessors(); - /** Max concurrent client number */ private int rpcMaxConcurrentClientNum = 1000; @@ -999,6 +993,8 @@ public class IoTDBConfig { */ private int maxClientNumForEachNode = DefaultProperty.MAX_CLIENT_NUM_FOR_EACH_NODE; + private int maxIdleClientNumForEachNode = DefaultProperty.MAX_IDLE_CLIENT_NUM_FOR_EACH_NODE; + /** * Cache size of partition cache in {@link * org.apache.iotdb.db.queryengine.plan.analyze.ClusterPartitionFetcher} @@ -1035,9 +1031,6 @@ public class IoTDBConfig { /** ThreadPool size for read operation in coordinator */ private int coordinatorReadExecutorSize = 20; - /** ThreadPool size for write operation in coordinator */ - private int coordinatorWriteExecutorSize = 50; - private int[] schemaMemoryProportion = new int[] {5, 4, 1}; /** Memory allocated for schemaRegion */ @@ -1872,22 +1865,6 @@ public void setUnSeqTsFileSize(long unSeqTsFileSize) { this.unSeqTsFileSize = unSeqTsFileSize; } - public int getRpcSelectorThreadCount() { - return rpcSelectorThreadCount; - } - - public void setRpcSelectorThreadCount(int rpcSelectorThreadCount) { - this.rpcSelectorThreadCount = rpcSelectorThreadCount; - } - - public int getRpcMinConcurrentClientNum() { - return rpcMinConcurrentClientNum; - } - - public void setRpcMinConcurrentClientNum(int rpcMinConcurrentClientNum) { - this.rpcMinConcurrentClientNum = rpcMinConcurrentClientNum; - } - public int getRpcMaxConcurrentClientNum() { return rpcMaxConcurrentClientNum; } @@ -3338,6 +3315,14 @@ public void setMaxClientNumForEachNode(int maxClientNumForEachNode) { this.maxClientNumForEachNode = maxClientNumForEachNode; } + public int getMaxIdleClientNumForEachNode() { + return maxIdleClientNumForEachNode; + } + + public void setMaxIdleClientNumForEachNode(int maxIdleClientNumForEachNode) { + this.maxIdleClientNumForEachNode = maxIdleClientNumForEachNode; + } + public int getSelectorNumOfClientManager() { return selectorNumOfClientManager; } @@ -3484,14 +3469,6 @@ public void setCoordinatorReadExecutorSize(int coordinatorReadExecutorSize) { this.coordinatorReadExecutorSize = coordinatorReadExecutorSize; } - public int getCoordinatorWriteExecutorSize() { - return coordinatorWriteExecutorSize; - } - - public void setCoordinatorWriteExecutorSize(int coordinatorWriteExecutorSize) { - this.coordinatorWriteExecutorSize = coordinatorWriteExecutorSize; - } - public TEndPoint getAddressAndPort() { return new TEndPoint(rpcAddress, rpcPort); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index e965611016de9..b01fca1a88f82 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -287,13 +287,23 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException String.valueOf(conf.getMaxClientNumForEachNode())) .trim())); - conf.setSelectorNumOfClientManager( + int dnMaxIdleClientNumForEachNode = Integer.parseInt( - properties - .getProperty( - "dn_selector_thread_count_of_client_manager", - String.valueOf(conf.getSelectorNumOfClientManager())) - .trim())); + properties.getProperty( + "dn_max_idle_client_count_for_each_node_in_client_manager", + String.valueOf(conf.getMaxIdleClientNumForEachNode()))); + if (dnMaxIdleClientNumForEachNode >= 0) { + conf.setMaxIdleClientNumForEachNode(dnMaxIdleClientNumForEachNode); + } + + int dnSelectorNumOfClientManager = + Integer.parseInt( + properties.getProperty( + "dn_selector_thread_nums_of_client_manager", + String.valueOf(conf.getSelectorNumOfClientManager()))); + if (dnSelectorNumOfClientManager > 0) { + conf.setSelectorNumOfClientManager(dnSelectorNumOfClientManager); + } conf.setRpcPort( Integer.parseInt( @@ -775,28 +785,6 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException properties.getProperty( "0.13_data_insert_adapt", String.valueOf(conf.isEnable13DataInsertAdapt())))); - int rpcSelectorThreadNum = - Integer.parseInt( - properties.getProperty( - "dn_rpc_selector_thread_count", - Integer.toString(conf.getRpcSelectorThreadCount()).trim())); - if (rpcSelectorThreadNum <= 0) { - rpcSelectorThreadNum = 1; - } - - conf.setRpcSelectorThreadCount(rpcSelectorThreadNum); - - int minConcurrentClientNum = - Integer.parseInt( - properties.getProperty( - "dn_rpc_min_concurrent_client_num", - Integer.toString(conf.getRpcMinConcurrentClientNum()).trim())); - if (minConcurrentClientNum <= 0) { - minConcurrentClientNum = Runtime.getRuntime().availableProcessors(); - } - - conf.setRpcMinConcurrentClientNum(minConcurrentClientNum); - int maxConcurrentClientNum = Integer.parseInt( properties.getProperty( @@ -1024,11 +1012,6 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException properties.getProperty( "coordinator_read_executor_size", Integer.toString(conf.getCoordinatorReadExecutorSize())))); - conf.setCoordinatorWriteExecutorSize( - Integer.parseInt( - properties.getProperty( - "coordinator_write_executor_size", - Integer.toString(conf.getCoordinatorWriteExecutorSize())))); // Commons commonDescriptor.loadCommonProps(properties); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java index 7dbc720a4a852..fa8e1efc6d8ed 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java @@ -123,8 +123,6 @@ private static ConsensusConfig buildConsensusConfig() { .setRpc( RPC.newBuilder() .setConnectionTimeoutInMs(CONF.getConnectionTimeoutInMS()) - .setRpcSelectorThreadNum(CONF.getRpcSelectorThreadCount()) - .setRpcMinConcurrentClientNum(CONF.getRpcMinConcurrentClientNum()) .setRpcMaxConcurrentClientNum(CONF.getRpcMaxConcurrentClientNum()) .setRpcThriftCompressionEnabled(CONF.isRpcThriftCompressionEnable()) .setSelectorNumOfClientManager(CONF.getSelectorNumOfClientManager()) @@ -151,8 +149,6 @@ private static ConsensusConfig buildConsensusConfig() { PipeConsensusConfig.RPC .newBuilder() .setConnectionTimeoutInMs(CONF.getConnectionTimeoutInMS()) - .setRpcSelectorThreadNum(CONF.getRpcSelectorThreadCount()) - .setRpcMinConcurrentClientNum(CONF.getRpcMinConcurrentClientNum()) .setRpcMaxConcurrentClientNum(CONF.getRpcMaxConcurrentClientNum()) .setIsRpcThriftCompressionEnabled(CONF.isRpcThriftCompressionEnable()) .setThriftServerAwaitTimeForStopService( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/DataNodeClientPoolFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/DataNodeClientPoolFactory.java index b5f5df430129f..102b1d2cbad94 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/DataNodeClientPoolFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/DataNodeClientPoolFactory.java @@ -61,32 +61,4 @@ public GenericKeyedObjectPool createClientPool return clientPool; } } - - public static class ClusterDeletionConfigNodeClientPoolFactory - implements IClientPoolFactory { - - @Override - public GenericKeyedObjectPool createClientPool( - ClientManager manager) { - GenericKeyedObjectPool clientPool = - new GenericKeyedObjectPool<>( - new ConfigNodeClient.Factory( - manager, - new ThriftClientProperty.Builder() - .setConnectionTimeoutMs(conf.getConnectionTimeoutInMS() * 10) - .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnable()) - .setSelectorNumOfAsyncClientManager( - conf.getSelectorNumOfClientManager() / 10 > 0 - ? conf.getSelectorNumOfClientManager() / 10 - : 1) - .build()), - new ClientPoolProperty.Builder() - .setMaxClientNumForEachNode(conf.getMaxClientNumForEachNode()) - .build() - .getConfig()); - ClientManagerMetrics.getInstance() - .registerClientManager(this.getClass().getSimpleName(), clientPool); - return clientPool; - } - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnInternalServiceAsyncRequestManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnInternalServiceAsyncRequestManager.java index b6a218a732e2b..9ac6b70841136 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnInternalServiceAsyncRequestManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/cn/DnToCnInternalServiceAsyncRequestManager.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; import org.apache.iotdb.commons.client.request.ConfigNodeInternalServiceAsyncRequestManager; import org.apache.iotdb.commons.client.request.TestConnectionUtils; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,6 +36,10 @@ public class DnToCnInternalServiceAsyncRequestManager private static final Logger LOGGER = LoggerFactory.getLogger(DnToCnInternalServiceAsyncRequestManager.class); + public DnToCnInternalServiceAsyncRequestManager() { + super(IoTDBDescriptor.getInstance().getConfig().getSelectorNumOfClientManager()); + } + @Override protected void initActionMapBuilder() { actionMapBuilder.put( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeExternalServiceAsyncRequestManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeExternalServiceAsyncRequestManager.java index dd56e1366c03b..29c3f76790716 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeExternalServiceAsyncRequestManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeExternalServiceAsyncRequestManager.java @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.client.request.AsyncRequestContext; import org.apache.iotdb.commons.client.request.AsyncRequestManager; import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,12 +39,17 @@ public class DataNodeExternalServiceAsyncRequestManager private static final Logger LOGGER = LoggerFactory.getLogger(DataNodeExternalServiceAsyncRequestManager.class); + private DataNodeExternalServiceAsyncRequestManager() { + super(IoTDBDescriptor.getInstance().getConfig().getSelectorNumOfClientManager()); + } + @Override - protected void initClientManager() { + protected void initClientManager(int selectorNumOfAsyncClientManager) { clientManager = new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncDataNodeExternalServiceClientPoolFactory()); + new ClientPoolFactory.AsyncDataNodeExternalServiceClientPoolFactory( + selectorNumOfAsyncClientManager)); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeMPPServiceAsyncRequestManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeMPPServiceAsyncRequestManager.java index ab08d83f2645b..e4d190e1571a4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeMPPServiceAsyncRequestManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DataNodeMPPServiceAsyncRequestManager.java @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.client.request.AsyncRequestContext; import org.apache.iotdb.commons.client.request.AsyncRequestManager; import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,14 +38,17 @@ public class DataNodeMPPServiceAsyncRequestManager private static final Logger LOGGER = LoggerFactory.getLogger(DataNodeMPPServiceAsyncRequestManager.class); - public DataNodeMPPServiceAsyncRequestManager() {} + public DataNodeMPPServiceAsyncRequestManager() { + super(IoTDBDescriptor.getInstance().getConfig().getSelectorNumOfClientManager()); + } @Override - protected void initClientManager() { + protected void initClientManager(int selectorNumOfAsyncClientManager) { clientManager = new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncDataNodeMPPDataExchangeServiceClientPoolFactory()); + new ClientPoolFactory.AsyncDataNodeMPPDataExchangeServiceClientPoolFactory( + selectorNumOfAsyncClientManager)); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnInternalServiceAsyncRequestManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnInternalServiceAsyncRequestManager.java index 88766458b649d..c2ca62cf15d48 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnInternalServiceAsyncRequestManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/dn/DnToDnInternalServiceAsyncRequestManager.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.client.request.AsyncRequestContext; import org.apache.iotdb.commons.client.request.AsyncRequestRPCHandler; import org.apache.iotdb.commons.client.request.DataNodeInternalServiceRequestManager; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +33,10 @@ public class DnToDnInternalServiceAsyncRequestManager private static final Logger LOGGER = LoggerFactory.getLogger(DnToDnInternalServiceAsyncRequestManager.class); + private DnToDnInternalServiceAsyncRequestManager() { + super(IoTDBDescriptor.getInstance().getConfig().getSelectorNumOfClientManager()); + } + @Override protected void initActionMapBuilder() { actionMapBuilder.put( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java index 8afdfce7d5c65..2f173861f22e4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/Coordinator.java @@ -92,10 +92,10 @@ public class Coordinator { ASYNC_INTERNAL_SERVICE_CLIENT_MANAGER = new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncDataNodeInternalServiceClientPoolFactory()); + new ClientPoolFactory.AsyncDataNodeInternalServiceClientPoolFactory( + CONFIG.getSelectorNumOfClientManager())); private final ExecutorService executor; - private final ExecutorService writeOperationExecutor; private final ScheduledExecutorService scheduledExecutor; private final ExecutorService dispatchExecutor; @@ -109,7 +109,6 @@ public class Coordinator { private Coordinator() { this.queryExecutionMap = new ConcurrentHashMap<>(); this.executor = getQueryExecutor(); - this.writeOperationExecutor = getWriteExecutor(); this.scheduledExecutor = getScheduledExecutor(); int dispatchThreadNum = Math.max(20, Runtime.getRuntime().availableProcessors() * 2); this.dispatchExecutor = @@ -220,8 +219,6 @@ private IQueryExecution createQueryExecutionForTreeModel( TreeModelPlanner treeModelPlanner = new TreeModelPlanner( statement, - executor, - writeOperationExecutor, scheduledExecutor, partitionFetcher, schemaFetcher, @@ -248,12 +245,6 @@ private ExecutorService getQueryExecutor() { coordinatorReadExecutorSize, ThreadName.MPP_COORDINATOR_EXECUTOR_POOL.getName()); } - private ExecutorService getWriteExecutor() { - int coordinatorWriteExecutorSize = CONFIG.getCoordinatorWriteExecutorSize(); - return IoTDBThreadPoolFactory.newFixedThreadPool( - coordinatorWriteExecutorSize, ThreadName.MPP_COORDINATOR_WRITE_EXECUTOR.getName()); - } - private ScheduledExecutorService getScheduledExecutor() { return IoTDBThreadPoolFactory.newScheduledThreadPool( COORDINATOR_SCHEDULED_EXECUTOR_SIZE, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index 023f7e3379368..d582832864a8f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -143,7 +143,6 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; -import org.apache.iotdb.db.protocol.client.DataNodeClientPoolFactory; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.schematree.ISchemaTree; import org.apache.iotdb.db.queryengine.plan.Coordinator; @@ -306,13 +305,6 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor { private static final IClientManager CONFIG_NODE_CLIENT_MANAGER = ConfigNodeClientManager.getInstance(); - /** FIXME Consolidate this clientManager with the upper one. */ - private static final IClientManager - CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER = - new IClientManager.Factory() - .createClientManager( - new DataNodeClientPoolFactory.ClusterDeletionConfigNodeClientPoolFactory()); - private static final class ClusterConfigTaskExecutorHolder { private static final ClusterConfigTaskExecutor INSTANCE = new ClusterConfigTaskExecutor(); @@ -1598,7 +1590,7 @@ public SettableFuture deactivateSchemaTemplate( req.setPathPatternTree( serializePatternListToByteBuffer(deactivateTemplateStatement.getPathPatternList())); try (final ConfigNodeClient client = - CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { TSStatus tsStatus; do { try { @@ -1699,7 +1691,7 @@ public SettableFuture alterSchemaTemplate( alterSchemaTemplateStatement.getOperationType(), alterSchemaTemplateStatement.getTemplateAlterInfo())); try (final ConfigNodeClient client = - CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { TSStatus tsStatus; do { try { @@ -1754,7 +1746,7 @@ public SettableFuture unsetSchemaTemplate( req.setTemplateName(unsetSchemaTemplateStatement.getTemplateName()); req.setPath(unsetSchemaTemplateStatement.getPath().getFullPath()); try (final ConfigNodeClient client = - CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { TSStatus tsStatus; do { try { @@ -2406,7 +2398,7 @@ public SettableFuture deleteTimeSeries( queryId, serializePatternListToByteBuffer(deleteTimeSeriesStatement.getPathPatternList())); try (ConfigNodeClient client = - CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { TSStatus tsStatus; do { try { @@ -2452,7 +2444,7 @@ public SettableFuture deleteLogicalView( queryId, serializePatternListToByteBuffer(deleteLogicalViewStatement.getPathPatternList())); try (ConfigNodeClient client = - CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { TSStatus tsStatus; do { try { @@ -2541,7 +2533,7 @@ public SettableFuture renameLogicalView( new TDeleteLogicalViewReq( queryId, serializePatternListToByteBuffer(Collections.singletonList(oldName))); try (ConfigNodeClient client = - CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { TSStatus tsStatus; do { try { @@ -2613,7 +2605,7 @@ public SettableFuture alterLogicalView( new TAlterLogicalViewReq( context.getQueryId().getId(), ByteBuffer.wrap(stream.toByteArray())); try (final ConfigNodeClient client = - CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { TSStatus tsStatus; do { try { @@ -2675,7 +2667,7 @@ public TSStatus alterLogicalViewByPipe( .setIsGeneratedByPipe(shouldMarkAsPipeRequest); TSStatus tsStatus; try (ConfigNodeClient client = - CLUSTER_DELETION_CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { do { try { tsStatus = client.alterLogicalView(req); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java index 1da701a8e3529..2812d81cf27fc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java @@ -48,15 +48,12 @@ import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; public class TreeModelPlanner implements IPlanner { private final Statement statement; - private final ExecutorService executor; - private final ExecutorService writeOperationExecutor; private final ScheduledExecutorService scheduledExecutor; private final IPartitionFetcher partitionFetcher; @@ -71,8 +68,6 @@ public class TreeModelPlanner implements IPlanner { public TreeModelPlanner( Statement statement, - ExecutorService executor, - ExecutorService writeOperationExecutor, ScheduledExecutorService scheduledExecutor, IPartitionFetcher partitionFetcher, ISchemaFetcher schemaFetcher, @@ -80,8 +75,6 @@ public TreeModelPlanner( IClientManager asyncInternalServiceClientManager) { this.statement = statement; - this.executor = executor; - this.writeOperationExecutor = writeOperationExecutor; this.scheduledExecutor = scheduledExecutor; this.partitionFetcher = partitionFetcher; this.schemaFetcher = schemaFetcher; @@ -134,8 +127,6 @@ public IScheduler doSchedule( stateMachine, distributedPlan, context.getQueryType(), - executor, - writeOperationExecutor, scheduledExecutor, syncInternalServiceClientManager, asyncInternalServiceClientManager); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/ClusterScheduler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/ClusterScheduler.java index 33f1bd3a19455..283e480ae38f9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/ClusterScheduler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/ClusterScheduler.java @@ -39,7 +39,6 @@ import java.util.List; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; @@ -74,8 +73,6 @@ public ClusterScheduler( QueryStateMachine stateMachine, DistributedQueryPlan distributedQueryPlan, QueryType queryType, - ExecutorService executor, - ExecutorService writeOperationExecutor, ScheduledExecutorService scheduledExecutor, IClientManager syncInternalServiceClientManager, IClientManager @@ -88,8 +85,6 @@ public ClusterScheduler( new FragmentInstanceDispatcherImpl( queryType, queryContext, - executor, - writeOperationExecutor, syncInternalServiceClientManager, asyncInternalServiceClientManager); if (queryType == QueryType.READ) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java index 41d325a5761a5..3312444f8822e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/FragmentInstanceDispatcherImpl.java @@ -65,7 +65,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -81,8 +80,6 @@ public class FragmentInstanceDispatcherImpl implements IFragInstanceDispatcher { private static final CommonConfig COMMON_CONFIG = CommonDescriptor.getInstance().getConfig(); - private final ExecutorService executor; - private final ExecutorService writeOperationExecutor; private final QueryType type; private final MPPQueryContext queryContext; private final String localhostIpAddr; @@ -104,15 +101,11 @@ public class FragmentInstanceDispatcherImpl implements IFragInstanceDispatcher { public FragmentInstanceDispatcherImpl( QueryType type, MPPQueryContext queryContext, - ExecutorService executor, - ExecutorService writeOperationExecutor, IClientManager syncInternalServiceClientManager, IClientManager asyncInternalServiceClientManager) { this.type = type; this.queryContext = queryContext; - this.executor = executor; - this.writeOperationExecutor = writeOperationExecutor; this.syncInternalServiceClientManager = syncInternalServiceClientManager; this.asyncInternalServiceClientManager = asyncInternalServiceClientManager; this.localhostIpAddr = IoTDBDescriptor.getInstance().getConfig().getInternalAddress(); diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index c7862e4886ec4..5447bcf5fd4ad 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -465,9 +465,10 @@ cn_rpc_max_concurrent_client_num=3000 cn_connection_timeout_ms=60000 # selector thread (TAsyncClientManager) nums for async thread in a clientManager +# When <= 0, use max(1, CPU core number / 4). # effectiveMode: restart # Datatype: int -cn_selector_thread_nums_of_client_manager=1 +cn_selector_thread_nums_of_client_manager=0 # The maximum number of clients that can be allocated for a node in a clientManager. # when the number of the client to a single node exceeds this number, the thread for applying for a client will be blocked @@ -476,6 +477,14 @@ cn_selector_thread_nums_of_client_manager=1 # Datatype: int cn_max_client_count_for_each_node_in_client_manager=1000 +# The maximum number of idle clients that can be retained for a node in a clientManager. +# When the number of idle clients to a single node exceeds this number, excess idle clients will be evicted. +# Idle clients are determined by a time threshold (default 1 minute of inactivity). +# 0 means no idle clients will be retained, connections are destroyed immediately upon return. +# effectiveMode: restart +# Datatype: int +# cn_max_idle_client_count_for_each_node_in_client_manager=1000 + # The maximum session idle time. unit: ms # Idle sessions are the ones that performs neither query or non-query operations for a period of time # Set to 0 to disable session timeout @@ -494,16 +503,6 @@ dn_rpc_thrift_compression_enable=false # this feature is under development, set this as false before it is done. dn_rpc_advanced_compression_enable=false -# the number of rpc selector -# effectiveMode: restart -# Datatype: int -dn_rpc_selector_thread_count=1 - -# The min number of concurrent clients that can be connected to the dataNode. -# effectiveMode: restart -# Datatype: int -dn_rpc_min_concurrent_client_num=1 - # The maximum number of concurrent clients that can be connected to the dataNode. # effectiveMode: restart # Datatype: int @@ -525,9 +524,10 @@ dn_thrift_init_buffer_size=1024 dn_connection_timeout_ms=60000 # selector thread (TAsyncClientManager) nums for async thread in a clientManager +# When <= 0, use max(1, CPU core number / 4). # effectiveMode: restart # Datatype: int -dn_selector_thread_count_of_client_manager=1 +dn_selector_thread_nums_of_client_manager=0 # The maximum number of clients that can be allocated for a node in a clientManager. # When the number of the client to a single node exceeds this number, the thread for applying for a client will be blocked @@ -536,6 +536,14 @@ dn_selector_thread_count_of_client_manager=1 # Datatype: int dn_max_client_count_for_each_node_in_client_manager=1000 +# The maximum number of idle clients that can be retained for a node in a clientManager. +# When the number of idle clients to a single node exceeds this number, excess idle clients will be evicted. +# Idle clients are determined by a time threshold (default 1 minute of inactivity). +# 0 means no idle clients will be retained, connections are destroyed immediately upon return. +# effectiveMode: restart +# Datatype: int +# dn_max_idle_client_count_for_each_node_in_client_manager=1000 + #################### ### REST Service Configuration #################### diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java index 3ff47a2c5e1d6..a70a6aa911d93 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java @@ -72,6 +72,12 @@ public GenericKeyedObjectPool createCli public static class AsyncConfigNodeInternalServiceClientPoolFactory implements IClientPoolFactory { + private final int selectorNumOfAsyncClientManager; + + public AsyncConfigNodeInternalServiceClientPoolFactory(int selectorNumOfAsyncClientManager) { + this.selectorNumOfAsyncClientManager = selectorNumOfAsyncClientManager; + } + @Override public GenericKeyedObjectPool createClientPool( ClientManager manager) { @@ -82,7 +88,7 @@ public GenericKeyedObjectPool c new ThriftClientProperty.Builder() .setConnectionTimeoutMs(conf.getCnConnectionTimeoutInMS()) .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) - .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) + .setSelectorNumOfAsyncClientManager(selectorNumOfAsyncClientManager) .build(), ThreadName.ASYNC_CONFIGNODE_CLIENT_POOL.getName()), new ClientPoolProperty.Builder() @@ -120,6 +126,12 @@ public GenericKeyedObjectPool crea public static class AsyncDataNodeInternalServiceClientPoolFactory implements IClientPoolFactory { + private final int selectorNumOfAsyncClientManager; + + public AsyncDataNodeInternalServiceClientPoolFactory(int selectorNumOfAsyncClientManager) { + this.selectorNumOfAsyncClientManager = selectorNumOfAsyncClientManager; + } + @Override public GenericKeyedObjectPool createClientPool( ClientManager manager) { @@ -130,7 +142,8 @@ public GenericKeyedObjectPool cre new ThriftClientProperty.Builder() .setConnectionTimeoutMs(conf.getDnConnectionTimeoutInMS()) .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) - .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) + .setSelectorNumOfAsyncClientManager(selectorNumOfAsyncClientManager) + .setPrintLogWhenEncounterException(false) .build(), ThreadName.ASYNC_DATANODE_CLIENT_POOL.getName()), new ClientPoolProperty.Builder() @@ -145,6 +158,12 @@ public GenericKeyedObjectPool cre public static class AsyncDataNodeExternalServiceClientPoolFactory implements IClientPoolFactory { + private final int selectorNumOfAsyncClientManager; + + public AsyncDataNodeExternalServiceClientPoolFactory(int selectorNumOfAsyncClientManager) { + this.selectorNumOfAsyncClientManager = selectorNumOfAsyncClientManager; + } + @Override public GenericKeyedObjectPool createClientPool( ClientManager manager) { @@ -155,7 +174,7 @@ public GenericKeyedObjectPool cre new ThriftClientProperty.Builder() .setConnectionTimeoutMs(conf.getDnConnectionTimeoutInMS()) .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) - .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) + .setSelectorNumOfAsyncClientManager(selectorNumOfAsyncClientManager) .build(), ThreadName.ASYNC_DATANODE_CLIENT_POOL.getName()), new ClientPoolProperty.Builder() @@ -170,6 +189,12 @@ public GenericKeyedObjectPool cre public static class AsyncConfigNodeHeartbeatServiceClientPoolFactory implements IClientPoolFactory { + private final int selectorNumOfAsyncClientManager; + + public AsyncConfigNodeHeartbeatServiceClientPoolFactory(int selectorNumOfAsyncClientManager) { + this.selectorNumOfAsyncClientManager = selectorNumOfAsyncClientManager; + } + @Override public GenericKeyedObjectPool createClientPool( ClientManager manager) { @@ -181,7 +206,7 @@ public GenericKeyedObjectPool c new ThriftClientProperty.Builder() .setConnectionTimeoutMs(conf.getCnConnectionTimeoutInMS()) .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) - .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) + .setSelectorNumOfAsyncClientManager(selectorNumOfAsyncClientManager) .setPrintLogWhenEncounterException(false) .build(), ThreadName.ASYNC_CONFIGNODE_HEARTBEAT_CLIENT_POOL.getName()), @@ -196,6 +221,13 @@ public GenericKeyedObjectPool c public static class AsyncDataNodeHeartbeatServiceClientPoolFactory implements IClientPoolFactory { + + private final int selectorNumOfAsyncClientManager; + + public AsyncDataNodeHeartbeatServiceClientPoolFactory(int selectorNumOfAsyncClientManager) { + this.selectorNumOfAsyncClientManager = selectorNumOfAsyncClientManager; + } + @Override public GenericKeyedObjectPool createClientPool( ClientManager manager) { @@ -206,7 +238,7 @@ public GenericKeyedObjectPool cre new ThriftClientProperty.Builder() .setConnectionTimeoutMs(conf.getCnConnectionTimeoutInMS()) .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) - .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) + .setSelectorNumOfAsyncClientManager(selectorNumOfAsyncClientManager) .setPrintLogWhenEncounterException(false) .build(), ThreadName.ASYNC_DATANODE_HEARTBEAT_CLIENT_POOL.getName()), @@ -246,6 +278,13 @@ public static class SyncDataNodeMPPDataExchangeServiceClientPoolFactory public static class AsyncDataNodeMPPDataExchangeServiceClientPoolFactory implements IClientPoolFactory { + private final int selectorNumOfAsyncClientManager; + + public AsyncDataNodeMPPDataExchangeServiceClientPoolFactory( + int selectorNumOfAsyncClientManager) { + this.selectorNumOfAsyncClientManager = selectorNumOfAsyncClientManager; + } + @Override public GenericKeyedObjectPool createClientPool( @@ -257,7 +296,7 @@ public static class AsyncDataNodeMPPDataExchangeServiceClientPoolFactory new ThriftClientProperty.Builder() .setConnectionTimeoutMs(conf.getDnConnectionTimeoutInMS()) .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) - .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) + .setSelectorNumOfAsyncClientManager(selectorNumOfAsyncClientManager) .build(), ThreadName.ASYNC_DATANODE_MPP_DATA_EXCHANGE_CLIENT_POOL.getName()), new ClientPoolProperty.Builder() @@ -323,6 +362,13 @@ public GenericKeyedObjectPool cre public static class AsyncAINodeHeartbeatServiceClientPoolFactory implements IClientPoolFactory { + + private final int selectorNumOfAsyncClientManager; + + public AsyncAINodeHeartbeatServiceClientPoolFactory(int selectorNumOfAsyncClientManager) { + this.selectorNumOfAsyncClientManager = selectorNumOfAsyncClientManager; + } + @Override public GenericKeyedObjectPool createClientPool( ClientManager manager) { @@ -333,7 +379,7 @@ public GenericKeyedObjectPool createClientP new ThriftClientProperty.Builder() .setConnectionTimeoutMs(conf.getCnConnectionTimeoutInMS()) .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled()) - .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager()) + .setSelectorNumOfAsyncClientManager(selectorNumOfAsyncClientManager) .setPrintLogWhenEncounterException(false) .build(), ThreadName.ASYNC_DATANODE_HEARTBEAT_CLIENT_POOL.getName()), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/ClientPoolProperty.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/ClientPoolProperty.java index 5bb7c22ee4994..1f818afe543da 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/ClientPoolProperty.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/ClientPoolProperty.java @@ -19,6 +19,8 @@ package org.apache.iotdb.commons.client.property; +import org.apache.iotdb.commons.conf.CommonDescriptor; + import org.apache.commons.pool2.impl.GenericKeyedObjectPoolConfig; import java.time.Duration; @@ -49,7 +51,11 @@ public static class Builder { * the maximum number of clients that can be allocated for a node. When some clients are idle * for more than {@code maxIdleTimeForClient}, they will be cleaned up. */ - private int maxClientNumForEachNode = DefaultProperty.MAX_CLIENT_NUM_FOR_EACH_NODE; + private int maxClientNumForEachNode = + CommonDescriptor.getInstance().getConfig().getMaxClientNumForEachNode(); + + private int maxIdleClientNumForEachNode = + CommonDescriptor.getInstance().getConfig().getMaxIdleClientNumForEachNode(); /** * the minimum amount of time a client may sit idle in the pool before it is eligible for @@ -74,6 +80,11 @@ public Builder setMaxClientNumForEachNode(int maxClientNumForEachNode) { return this; } + public Builder setMaxIdleClientNumForEachNode(int maxIdleClientNumForEachNode) { + this.maxIdleClientNumForEachNode = maxIdleClientNumForEachNode; + return this; + } + public Builder setMinIdleTimeForClient(long minIdleTimeForClient) { this.minIdleTimeForClient = minIdleTimeForClient; return this; @@ -87,7 +98,7 @@ public Builder setTimeBetweenEvictionRuns(long timeBetweenEvictionRuns) { public ClientPoolProperty build() { GenericKeyedObjectPoolConfig poolConfig = new GenericKeyedObjectPoolConfig<>(); poolConfig.setMaxTotalPerKey(maxClientNumForEachNode); - poolConfig.setMaxIdlePerKey(maxClientNumForEachNode); + poolConfig.setMaxIdlePerKey(maxIdleClientNumForEachNode); poolConfig.setTimeBetweenEvictionRuns(Duration.ofMillis(timeBetweenEvictionRuns)); poolConfig.setMinEvictableIdleTime(Duration.ofMillis(minIdleTimeForClient)); poolConfig.setMaxWait(Duration.ofMillis(waitClientTimeoutMs)); @@ -105,5 +116,6 @@ private DefaultProperty() {} public static final long MIN_IDLE_TIME_FOR_CLIENT_MS = TimeUnit.MINUTES.toMillis(1); public static final long TIME_BETWEEN_EVICTION_RUNS_MS = TimeUnit.MINUTES.toMillis(1); public static final int MAX_CLIENT_NUM_FOR_EACH_NODE = 1000; + public static final int MAX_IDLE_CLIENT_NUM_FOR_EACH_NODE = 1000; } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/ThriftClientProperty.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/ThriftClientProperty.java index f8fe16166a8c8..f157ee7df936f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/ThriftClientProperty.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/property/ThriftClientProperty.java @@ -117,7 +117,10 @@ private DefaultProperty() {} public static final boolean RPC_THRIFT_COMPRESSED_ENABLED = false; public static final int CONNECTION_TIMEOUT_MS = (int) TimeUnit.SECONDS.toMillis(20); public static final int CONNECTION_NEVER_TIMEOUT_MS = 0; - public static final int SELECTOR_NUM_OF_ASYNC_CLIENT_MANAGER = 1; + public static final int SELECTOR_NUM_OF_ASYNC_CLIENT_MANAGER = + Runtime.getRuntime().availableProcessors() / 4 > 0 + ? Runtime.getRuntime().availableProcessors() / 4 + : 1; public static final boolean PRINT_LOG_WHEN_ENCOUNTER_EXCEPTION = true; } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestManager.java index 3cca39bb635da..0290d33d3a46b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/AsyncRequestManager.java @@ -53,15 +53,15 @@ public abstract class AsyncRequestManager { private static final int MAX_RETRY_NUM = 6; - protected AsyncRequestManager() { - initClientManager(); + protected AsyncRequestManager(int selectorNumOfAsyncClientManager) { + initClientManager(selectorNumOfAsyncClientManager); actionMapBuilder = ImmutableMap.builder(); initActionMapBuilder(); this.actionMap = this.actionMapBuilder.build(); checkActionMapCompleteness(); } - protected abstract void initClientManager(); + protected abstract void initClientManager(int selectorNumOfAsyncClientManager); protected abstract void initActionMapBuilder(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/ConfigNodeInternalServiceAsyncRequestManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/ConfigNodeInternalServiceAsyncRequestManager.java index 791a1e5df0e96..3b50c29fba6bf 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/ConfigNodeInternalServiceAsyncRequestManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/ConfigNodeInternalServiceAsyncRequestManager.java @@ -28,12 +28,18 @@ public abstract class ConfigNodeInternalServiceAsyncRequestManager extends AsyncRequestManager< RequestType, TConfigNodeLocation, AsyncConfigNodeInternalServiceClient> { + + protected ConfigNodeInternalServiceAsyncRequestManager(int selectorNumOfAsyncClientManager) { + super(selectorNumOfAsyncClientManager); + } + @Override - protected void initClientManager() { + protected void initClientManager(int selectorNumOfAsyncClientManager) { clientManager = new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncConfigNodeInternalServiceClientPoolFactory()); + new ClientPoolFactory.AsyncConfigNodeInternalServiceClientPoolFactory( + selectorNumOfAsyncClientManager)); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/DataNodeInternalServiceRequestManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/DataNodeInternalServiceRequestManager.java index fcb1b01857df8..722d4f241ebe1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/DataNodeInternalServiceRequestManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/request/DataNodeInternalServiceRequestManager.java @@ -28,12 +28,18 @@ public abstract class DataNodeInternalServiceRequestManager extends AsyncRequestManager< RequestType, TDataNodeLocation, AsyncDataNodeInternalServiceClient> { + + protected DataNodeInternalServiceRequestManager(int selectorNumOfAsyncClientManager) { + super(selectorNumOfAsyncClientManager); + } + @Override - protected void initClientManager() { + protected void initClientManager(int selectorNumOfAsyncClientManager) { clientManager = new IClientManager.Factory() .createClientManager( - new ClientPoolFactory.AsyncDataNodeInternalServiceClientPoolFactory()); + new ClientPoolFactory.AsyncDataNodeInternalServiceClientPoolFactory( + selectorNumOfAsyncClientManager)); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index 696b6b8ce07fb..4f5ad140e9919 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -44,7 +44,6 @@ public enum ThreadName { MPP_COORDINATOR_EXECUTOR_POOL("MPP-Coordinator-Executor"), DATANODE_INTERNAL_RPC_SERVICE("DataNodeInternalRPC-Service"), DATANODE_INTERNAL_RPC_PROCESSOR("DataNodeInternalRPC-Processor"), - MPP_COORDINATOR_WRITE_EXECUTOR("MPP-Coordinator-Write-Executor"), ASYNC_DATANODE_MPP_DATA_EXCHANGE_CLIENT_POOL("AsyncDataNodeMPPDataExchangeServiceClientPool"), // -------------------------- Compaction -------------------------- COMPACTION_WORKER("Compaction-Worker"), @@ -221,7 +220,6 @@ public enum ThreadName { MPP_COORDINATOR_EXECUTOR_POOL, DATANODE_INTERNAL_RPC_SERVICE, DATANODE_INTERNAL_RPC_PROCESSOR, - MPP_COORDINATOR_WRITE_EXECUTOR, ASYNC_DATANODE_MPP_DATA_EXCHANGE_CLIENT_POOL)); private static final Set compactionThreadNames = new HashSet<>(Arrays.asList(COMPACTION_WORKER, COMPACTION_SUB_TASK, COMPACTION_SCHEDULE)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 22e282d85a83f..e1f52ba65ff6d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -146,13 +146,18 @@ public class CommonConfig { * ClientManager will have so many selector threads (TAsyncClientManager) to distribute to its * clients. */ - private int selectorNumOfClientManager = 1; + private int selectorNumOfClientManager = + Runtime.getRuntime().availableProcessors() / 4 > 0 + ? Runtime.getRuntime().availableProcessors() / 4 + : 1; /** Whether to use thrift compression. */ private boolean isRpcThriftCompressionEnabled = false; private int maxClientNumForEachNode = DefaultProperty.MAX_CLIENT_NUM_FOR_EACH_NODE; + private int maxIdleClientNumForEachNode = DefaultProperty.MAX_IDLE_CLIENT_NUM_FOR_EACH_NODE; + /** What will the system do when unrecoverable error occurs. */ private HandleSystemErrorStrategy handleSystemErrorStrategy = HandleSystemErrorStrategy.CHANGE_TO_READ_ONLY; @@ -630,6 +635,14 @@ public void setMaxClientNumForEachNode(int maxClientNumForEachNode) { this.maxClientNumForEachNode = maxClientNumForEachNode; } + public int getMaxIdleClientNumForEachNode() { + return maxIdleClientNumForEachNode; + } + + public void setMaxIdleClientNumForEachNode(int maxIdleClientNumForEachNode) { + this.maxIdleClientNumForEachNode = maxIdleClientNumForEachNode; + } + HandleSystemErrorStrategy getHandleSystemErrorStrategy() { return handleSystemErrorStrategy; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java index 961d4fbb8f248..ab7d949044d0b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java @@ -128,13 +128,16 @@ public void loadCommonProps(TrimProperties properties) throws IOException { "cn_connection_timeout_ms", String.valueOf(config.getCnConnectionTimeoutInMS())) .trim())); - config.setSelectorNumOfClientManager( + int cnSelectorNumOfClientManager = Integer.parseInt( properties .getProperty( "cn_selector_thread_nums_of_client_manager", String.valueOf(config.getSelectorNumOfClientManager())) - .trim())); + .trim()); + if (cnSelectorNumOfClientManager > 0) { + config.setSelectorNumOfClientManager(cnSelectorNumOfClientManager); + } config.setMaxClientNumForEachNode( Integer.parseInt( @@ -144,6 +147,17 @@ public void loadCommonProps(TrimProperties properties) throws IOException { String.valueOf(config.getMaxClientNumForEachNode())) .trim())); + int cnMaxIdleClientNumForEachNode = + Integer.parseInt( + properties + .getProperty( + "cn_max_idle_client_count_for_each_node_in_client_manager", + String.valueOf(config.getMaxIdleClientNumForEachNode())) + .trim()); + if (cnMaxIdleClientNumForEachNode >= 0) { + config.setMaxIdleClientNumForEachNode(cnMaxIdleClientNumForEachNode); + } + config.setDnConnectionTimeoutInMS( Integer.parseInt( properties @@ -159,13 +173,16 @@ public void loadCommonProps(TrimProperties properties) throws IOException { String.valueOf(config.isRpcThriftCompressionEnabled())) .trim())); - config.setSelectorNumOfClientManager( + int dnSelectorNumOfClientManager = Integer.parseInt( properties .getProperty( "dn_selector_thread_nums_of_client_manager", String.valueOf(config.getSelectorNumOfClientManager())) - .trim())); + .trim()); + if (dnSelectorNumOfClientManager > 0) { + config.setSelectorNumOfClientManager(dnSelectorNumOfClientManager); + } config.setMaxClientNumForEachNode( Integer.parseInt( @@ -175,6 +192,17 @@ public void loadCommonProps(TrimProperties properties) throws IOException { String.valueOf(config.getMaxClientNumForEachNode())) .trim())); + int dnMaxIdleClientNumForEachNode = + Integer.parseInt( + properties + .getProperty( + "dn_max_idle_client_count_for_each_node_in_client_manager", + String.valueOf(config.getMaxIdleClientNumForEachNode())) + .trim()); + if (dnMaxIdleClientNumForEachNode >= 0) { + config.setMaxIdleClientNumForEachNode(dnMaxIdleClientNumForEachNode); + } + config.setHandleSystemErrorStrategy( HandleSystemErrorStrategy.valueOf( properties diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/ClientManagerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/ClientManagerTest.java index 70c32ca4801d8..7296a0b3f78ca 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/ClientManagerTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/client/ClientManagerTest.java @@ -214,6 +214,7 @@ public void evictionTest() throws Exception { manager, new ThriftClientProperty.Builder().build()), new ClientPoolProperty.Builder() .setMaxClientNumForEachNode(maxClientForEachNode) + .setMaxIdleClientNumForEachNode(maxClientForEachNode) .setMinIdleTimeForClient(minIdleDuration) .setTimeBetweenEvictionRuns(evictionRunsDuration) .build() @@ -294,6 +295,7 @@ public void maxTotalTest() throws Exception { manager, new ThriftClientProperty.Builder().build()), new ClientPoolProperty.Builder() .setMaxClientNumForEachNode(maxTotalClientForEachNode) + .setMaxIdleClientNumForEachNode(maxTotalClientForEachNode) .setWaitClientTimeoutMs(waitClientTimeoutMs) .build() .getConfig()); @@ -369,6 +371,7 @@ public void maxWaitClientTimeoutTest() throws Exception { new ClientPoolProperty.Builder() .setWaitClientTimeoutMs(waitClientTimeoutMS) .setMaxClientNumForEachNode(maxTotalClientForEachNode) + .setMaxIdleClientNumForEachNode(maxTotalClientForEachNode) .build() .getConfig()); } @@ -617,7 +620,13 @@ public GenericKeyedObjectPool crea new ThriftClientProperty.Builder() .setConnectionTimeoutMs(CONNECTION_TIMEOUT) .build()), - new ClientPoolProperty.Builder().build().getConfig()); + new ClientPoolProperty.Builder() + .setMaxClientNumForEachNode( + ClientPoolProperty.DefaultProperty.MAX_CLIENT_NUM_FOR_EACH_NODE) + .setMaxIdleClientNumForEachNode( + ClientPoolProperty.DefaultProperty.MAX_IDLE_CLIENT_NUM_FOR_EACH_NODE) + .build() + .getConfig()); } } @@ -632,7 +641,13 @@ public GenericKeyedObjectPool cre manager, new ThriftClientProperty.Builder().setConnectionTimeoutMs(CONNECTION_TIMEOUT).build(), ThreadName.ASYNC_DATANODE_CLIENT_POOL.getName()), - new ClientPoolProperty.Builder().build().getConfig()); + new ClientPoolProperty.Builder() + .setMaxClientNumForEachNode( + ClientPoolProperty.DefaultProperty.MAX_CLIENT_NUM_FOR_EACH_NODE) + .setMaxIdleClientNumForEachNode( + ClientPoolProperty.DefaultProperty.MAX_IDLE_CLIENT_NUM_FOR_EACH_NODE) + .build() + .getConfig()); } } } From c7ede27b8da16f86b5f10b96b39527fab6865e95 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 9 May 2026 10:36:55 +0800 Subject: [PATCH 036/102] [To dev/1.3] Pipe: Implemented skipReportOnCommit for PipeRealtimeEvent to avoid premature report for unordered flush (#17561) (#17620) * Pipe: Removed the failed partial insert from auth check & Implemented skipReportOnCommit for PipeRealtimeEvent to avoid premature report for unordered flush (#17561) * schema * sptls * fix * Update PipeRealtimeEventTest.java --- .../event/realtime/PipeRealtimeEvent.java | 10 ++ .../db/pipe/event/PipeRealtimeEventTest.java | 95 +++++++++++++++++++ 2 files changed, 105 insertions(+) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeRealtimeEventTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java index c8474e360da78..48ab8259fe4b0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java @@ -140,6 +140,16 @@ public ProgressIndex getProgressIndex() { return event.getProgressIndex(); } + @Override + public void skipReportOnCommit() { + event.skipReportOnCommit(); + } + + @Override + public boolean isShouldReportOnCommit() { + return event.isShouldReportOnCommit(); + } + @Override public void skipParsingPattern() { event.skipParsingPattern(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeRealtimeEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeRealtimeEventTest.java new file mode 100644 index 0000000000000..525951fa4e85e --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeRealtimeEventTest.java @@ -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. + */ + +package org.apache.iotdb.db.pipe.event; + +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; + +import org.junit.Assert; +import org.junit.Test; + +public class PipeRealtimeEventTest { + + @Test + public void testSkipReportOnCommitIsDelegatedToInnerEvent() { + final TestEnrichedEvent innerEvent = new TestEnrichedEvent(); + final PipeRealtimeEvent realtimeEvent = new PipeRealtimeEvent(innerEvent, null, null, null); + + Assert.assertTrue(innerEvent.isShouldReportOnCommit()); + Assert.assertTrue(realtimeEvent.isShouldReportOnCommit()); + + realtimeEvent.skipReportOnCommit(); + + Assert.assertFalse(innerEvent.isShouldReportOnCommit()); + Assert.assertFalse(realtimeEvent.isShouldReportOnCommit()); + } + + private static class TestEnrichedEvent extends EnrichedEvent { + + private TestEnrichedEvent() { + super(null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); + } + + @Override + public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { + return true; + } + + @Override + public boolean internallyDecreaseResourceReferenceCount(final String holderMessage) { + return true; + } + + @Override + public ProgressIndex getProgressIndex() { + return MinimumProgressIndex.INSTANCE; + } + + @Override + public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( + final String pipeName, + final long creationTime, + final PipeTaskMeta pipeTaskMeta, + final PipePattern pattern, + final long startTime, + final long endTime) { + return this; + } + + @Override + public boolean isGeneratedByPipe() { + return false; + } + + @Override + public boolean mayEventTimeOverlappedWithTimeRange() { + return true; + } + + @Override + public boolean mayEventPathsOverlappedWithPattern() { + return true; + } + } +} From 35482e4b58ce6f2734e1f26859542144e3c2a845 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 9 May 2026 10:37:11 +0800 Subject: [PATCH 037/102] Fixed the print-tsfile.sh to avoid wrong IOTDB_HOME (#17596) (#17621) --- .../src/assembly/resources/tools/tsfile/print-tsfile.sh | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/assembly/resources/tools/tsfile/print-tsfile.sh b/iotdb-core/datanode/src/assembly/resources/tools/tsfile/print-tsfile.sh index 8864ca877e744..8e27793a7ff0d 100644 --- a/iotdb-core/datanode/src/assembly/resources/tools/tsfile/print-tsfile.sh +++ b/iotdb-core/datanode/src/assembly/resources/tools/tsfile/print-tsfile.sh @@ -24,9 +24,11 @@ echo --------------------- source "$(dirname "$0")/../../sbin/iotdb-common.sh" #get_iotdb_include and checkAllVariables is in iotdb-common.sh +if [ -z "${IOTDB_HOME}" ]; then + export IOTDB_HOME="$(cd "`dirname "$0"`"/../..; pwd)" +fi VARS=$(get_iotdb_include "$*") checkAllVariables -export IOTDB_HOME="${IOTDB_HOME}/.." eval set -- "$VARS" @@ -48,5 +50,5 @@ done MAIN_CLASS=org.apache.iotdb.db.tools.TsFileSketchTool -"$JAVA" -cp "$CLASSPATH" "$MAIN_CLASS" "$@" -exit $? \ No newline at end of file +"$JAVA" $JVM_OPTS -cp "$CLASSPATH" "$MAIN_CLASS" "$@" +exit $? From 2ea083dcf6add81e9bb43ef536728c7d53e47f96 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 9 May 2026 14:48:31 +0800 Subject: [PATCH 038/102] [To dev/1.3] Pipe: Fixed the event clear logic of drop pipe (#17560) (#17619) * Pipe: Fixed the event clear logic of drop pipe (#17560) * drop-1 * wd * drop * fix * local * triple * by * spt * bug-fix * no-pipe-task-key * Update IoTDBDataRegionAsyncSink.java * triple * Fix * comp * comp-fix02 * drop-n --- .../task/connection/PipeEventCollector.java | 20 ++- .../PipeRealtimePriorityBlockingQueue.java | 13 +- .../task/subtask/sink/PipeSinkSubtask.java | 13 +- .../sink/PipeSinkSubtaskLifeCycle.java | 5 +- .../subtask/sink/PipeSinkSubtaskManager.java | 2 +- .../evolvable/batch/PipeTabletEventBatch.java | 7 +- .../batch/PipeTransferBatchReqBuilder.java | 9 +- .../async/IoTDBDataRegionAsyncSink.java | 50 +++++-- .../thrift/sync/IoTDBDataRegionSyncSink.java | 5 +- .../websocket/WebSocketConnectorServer.java | 141 ++++++++++++++---- .../protocol/websocket/WebSocketSink.java | 11 +- .../SubscriptionSinkSubtaskLifeCycle.java | 3 +- .../SubscriptionSinkSubtaskManager.java | 2 +- .../connection/PipeEventCollectorTest.java | 86 +++++++++++ .../subtask/sink/PipeSinkSubtaskTest.java | 61 ++++++++ .../iotdb/db/pipe/sink/PipeSinkTest.java | 95 ++++++++++++ .../task/connection/BlockingPendingQueue.java | 55 ++++++- .../commons/pipe/datastructure/Triple.java | 63 ++++++++ .../commons/pipe/sink/protocol/IoTDBSink.java | 5 +- .../PipeConnectorWithEventDiscard.java | 25 ++++ 20 files changed, 599 insertions(+), 72 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollectorTest.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskTest.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/Triple.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/PipeConnectorWithEventDiscard.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java index 4728f62c94fd5..387d4ff7ec621 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java @@ -178,7 +178,8 @@ private void parseAndCollectEvent(final PipeSchemaRegionWritePlanEvent deleteDat private void collectEvent(final Event event) { if (event instanceof EnrichedEvent) { - if (!((EnrichedEvent) event).increaseReferenceCount(PipeEventCollector.class.getName())) { + final EnrichedEvent enrichedEvent = (EnrichedEvent) event; + if (!enrichedEvent.increaseReferenceCount(PipeEventCollector.class.getName())) { LOGGER.warn("PipeEventCollector: The event {} is already released, skipping it.", event); isFailedToIncreaseReferenceCount = true; return; @@ -186,18 +187,25 @@ private void collectEvent(final Event event) { // Assign a commit id for this event in order to report progress in order. PipeEventCommitManager.getInstance() - .enrichWithCommitterKeyAndCommitId((EnrichedEvent) event, creationTime, regionId); + .enrichWithCommitterKeyAndCommitId(enrichedEvent, creationTime, regionId); - // Assign a rebootTime for pipeConsensus - ((EnrichedEvent) event).setRebootTimes(PipeDataNodeAgent.runtime().getRebootTimes()); + // Assign a rebootTime for iotConsensusV2 + enrichedEvent.setRebootTimes(PipeDataNodeAgent.runtime().getRebootTimes()); + + if (enrichedEvent.getPipeName() != null + && pendingQueue.isPipeDropped(enrichedEvent.getPipeName(), creationTime, regionId)) { + enrichedEvent.clearReferenceCount(PipeEventCollector.class.getName()); + return; + } } if (event instanceof PipeHeartbeatEvent) { ((PipeHeartbeatEvent) event).recordConnectorQueueSize(pendingQueue); } - pendingQueue.offer(event); - collectInvocationCount.incrementAndGet(); + if (pendingQueue.offer(event)) { + collectInvocationCount.incrementAndGet(); + } } public void resetFlags() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeRealtimePriorityBlockingQueue.java index 6d227ac31fdce..f972bba0e6ede 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeRealtimePriorityBlockingQueue.java @@ -73,7 +73,9 @@ public PipeRealtimePriorityBlockingQueue() { @Override public boolean offer(final Event event) { - checkBeforeOffer(event); + if (!checkBeforeOffer(event)) { + return false; + } if (event instanceof TsFileInsertionEvent) { tsfileInsertEventDeque.add((TsFileInsertionEvent) event); @@ -356,13 +358,14 @@ public void discardAllEvents() { } @Override - public void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { - super.discardEventsOfPipe(pipeNameToDrop, regionId); + public void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + super.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); tsfileInsertEventDeque.removeIf( event -> { if (event instanceof EnrichedEvent - && pipeNameToDrop.equals(((EnrichedEvent) event).getPipeName()) - && regionId == ((EnrichedEvent) event).getRegionId()) { + && isEventFromPipe( + ((EnrichedEvent) event), pipeNameToDrop, creationTimeToDrop, regionId)) { if (((EnrichedEvent) event) .clearReferenceCount(PipeRealtimePriorityBlockingQueue.class.getName())) { eventCounter.decreaseEventCount(event); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java index 4b4794891f6cc..ae48fcb277861 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.sink.protocol.IoTDBSink; +import org.apache.iotdb.commons.pipe.sink.protocol.PipeConnectorWithEventDiscard; import org.apache.iotdb.commons.utils.ErrorHandlingCommonUtils; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; @@ -199,9 +200,10 @@ public void close() { * When a pipe is dropped, the connector maybe reused and will not be closed. So we just discard * its queued events in the output pipe connector. */ - public void discardEventsOfPipe(final String pipeNameToDrop, int regionId) { + public void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { // Try to remove the events as much as possible - inputPendingQueue.discardEventsOfPipe(pipeNameToDrop, regionId); + inputPendingQueue.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); try { increaseHighPriorityTaskCount(); @@ -215,6 +217,7 @@ public void discardEventsOfPipe(final String pipeNameToDrop, int regionId) { // will. if (lastEvent instanceof EnrichedEvent && pipeNameToDrop.equals(((EnrichedEvent) lastEvent).getPipeName()) + && creationTimeToDrop == ((EnrichedEvent) lastEvent).getCreationTime() && regionId == ((EnrichedEvent) lastEvent).getRegionId()) { // Do not clear the last event's reference counts because it may be on transferring lastEvent = null; @@ -238,6 +241,7 @@ public void discardEventsOfPipe(final String pipeNameToDrop, int regionId) { // "nonnull" detection. if (lastExceptionEvent instanceof EnrichedEvent && pipeNameToDrop.equals(((EnrichedEvent) lastExceptionEvent).getPipeName()) + && creationTimeToDrop == ((EnrichedEvent) lastExceptionEvent).getCreationTime() && regionId == ((EnrichedEvent) lastExceptionEvent).getRegionId()) { clearReferenceCountAndReleaseLastExceptionEvent(); } @@ -246,8 +250,9 @@ public void discardEventsOfPipe(final String pipeNameToDrop, int regionId) { decreaseHighPriorityTaskCount(); } - if (outputPipeConnector instanceof IoTDBSink) { - ((IoTDBSink) outputPipeConnector).discardEventsOfPipe(pipeNameToDrop, regionId); + if (outputPipeConnector instanceof PipeConnectorWithEventDiscard) { + ((PipeConnectorWithEventDiscard) outputPipeConnector) + .discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskLifeCycle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskLifeCycle.java index 35f7983075dfe..1780f5a87efa8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskLifeCycle.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskLifeCycle.java @@ -92,12 +92,13 @@ public synchronized void register() { * {@link PipeSinkSubtask} should never be used again * @throws IllegalStateException if {@link PipeSinkSubtaskLifeCycle#registeredTaskCount} <= 0 */ - public synchronized boolean deregister(final String pipeNameToDeregister, int regionId) { + public synchronized boolean deregister( + final String pipeNameToDeregister, final long creationTimeToDeregister, final int regionId) { if (registeredTaskCount <= 0) { throw new IllegalStateException("registeredTaskCount <= 0"); } - subtask.discardEventsOfPipe(pipeNameToDeregister, regionId); + subtask.discardEventsOfPipe(pipeNameToDeregister, creationTimeToDeregister, regionId); try { if (registeredTaskCount > 1) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskManager.java index caff425f790b3..d7f81c12dbc98 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskManager.java @@ -209,7 +209,7 @@ public synchronized void deregister( // Shall not be empty final PipeSinkSubtaskExecutor executor = lifeCycles.get(0).executor; - lifeCycles.removeIf(o -> o.deregister(pipeName, regionId)); + lifeCycles.removeIf(o -> o.deregister(pipeName, creationTime, regionId)); if (lifeCycles.isEmpty()) { attributeSortedString2SubtaskLifeCycleMap.remove(attributeSortedString); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventBatch.java index 96bddd0d672f0..c44e12a4bbf20 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventBatch.java @@ -154,10 +154,13 @@ public synchronized void close() { * Discard all events of the given pipe. This method only clears the reference count of the events * and discard them, but do not modify other objects (such as buffers) for simplicity. */ - public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { + public synchronized void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { events.removeIf( event -> { - if (pipeNameToDrop.equals(event.getPipeName()) && regionId == event.getRegionId()) { + if (pipeNameToDrop.equals(event.getPipeName()) + && creationTimeToDrop == event.getCreationTime() + && regionId == event.getRegionId()) { event.clearReferenceCount(IoTDBDataRegionAsyncSink.class.getName()); return true; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java index 9fa706e985f1e..ac5f568f1c6af 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java @@ -195,9 +195,12 @@ public boolean isEmpty() { && endPointToBatch.values().stream().allMatch(PipeTabletEventPlainBatch::isEmpty); } - public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { - defaultBatch.discardEventsOfPipe(pipeNameToDrop, regionId); - endPointToBatch.values().forEach(batch -> batch.discardEventsOfPipe(pipeNameToDrop, regionId)); + public synchronized void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + defaultBatch.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); + endPointToBatch + .values() + .forEach(batch -> batch.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId)); } public int size() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java index 07381d15e28b6..fe3d44bedb488 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.datastructure.Triple; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.resource.log.PipeLogger; import org.apache.iotdb.commons.pipe.sink.protocol.IoTDBSink; @@ -74,6 +75,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -121,6 +123,10 @@ public class IoTDBDataRegionAsyncSink extends IoTDBSink { private final Map pendingHandlers = new ConcurrentHashMap<>(); + // Pipe name, creation time, region id + private final Set> droppedPipeTaskKeys = + ConcurrentHashMap.newKeySet(); + private boolean enableSendTsFileLimit; private volatile boolean isConnectionException; @@ -660,8 +666,15 @@ private void retryTransfer(final PipeTsFileInsertionEvent tsFileInsertionEvent) public void addFailureEventToRetryQueue(final Event event, final Exception e) { isConnectionException = e instanceof PipeConnectionException || ThriftClient.isConnectionBroken(e); - if (event instanceof EnrichedEvent && ((EnrichedEvent) event).isReleased()) { - return; + if (event instanceof EnrichedEvent) { + final EnrichedEvent enrichedEvent = (EnrichedEvent) event; + if (enrichedEvent.isReleased()) { + return; + } + if (isDroppedPipe(enrichedEvent)) { + enrichedEvent.clearReferenceCount(IoTDBDataRegionAsyncSink.class.getName()); + return; + } } if (isClosed.get()) { @@ -707,15 +720,18 @@ public boolean isEnableSendTsFileLimit() { //////////////////////////// Operations for close //////////////////////////// @Override - public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { - if (isTabletBatchModeEnabled) { - tabletBatchBuilder.discardEventsOfPipe(pipeNameToDrop, regionId); + public synchronized void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + droppedPipeTaskKeys.add(new Triple<>(pipeNameToDrop, creationTimeToDrop, regionId)); + + if (isTabletBatchModeEnabled && Objects.nonNull(tabletBatchBuilder)) { + tabletBatchBuilder.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); } retryEventQueue.removeIf( event -> { if (event instanceof EnrichedEvent - && pipeNameToDrop.equals(((EnrichedEvent) event).getPipeName()) - && regionId == ((EnrichedEvent) event).getRegionId()) { + && isDroppedPipe( + (EnrichedEvent) event, pipeNameToDrop, creationTimeToDrop, regionId)) { ((EnrichedEvent) event).clearReferenceCount(IoTDBDataRegionAsyncSink.class.getName()); retryEventQueueEventCounter.decreaseEventCount(event); return true; @@ -726,8 +742,8 @@ public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final retryTsFileQueue.removeIf( event -> { if (event instanceof EnrichedEvent - && pipeNameToDrop.equals(((EnrichedEvent) event).getPipeName()) - && regionId == ((EnrichedEvent) event).getRegionId()) { + && isDroppedPipe( + (EnrichedEvent) event, pipeNameToDrop, creationTimeToDrop, regionId)) { ((EnrichedEvent) event).clearReferenceCount(IoTDBDataRegionAsyncSink.class.getName()); retryEventQueueEventCounter.decreaseEventCount(event); return true; @@ -771,6 +787,7 @@ public synchronized void close() { // clear reference count of events in retry queue after closing async client clearRetryEventsReferenceCount(); + droppedPipeTaskKeys.clear(); super.close(); } @@ -827,6 +844,21 @@ public void setTransferTsFileCounter(AtomicInteger transferTsFileCounter) { this.transferTsFileCounter = transferTsFileCounter; } + private boolean isDroppedPipe(final EnrichedEvent event) { + return droppedPipeTaskKeys.contains( + new Triple<>(event.getPipeName(), event.getCreationTime(), event.getRegionId())); + } + + private static boolean isDroppedPipe( + final EnrichedEvent event, + final String pipeNameToDrop, + final long creationTimeToDrop, + final int regionId) { + return pipeNameToDrop.equals(event.getPipeName()) + && creationTimeToDrop == event.getCreationTime() + && regionId == event.getRegionId(); + } + @Override public void setTabletBatchSizeHistogram(Histogram tabletBatchSizeHistogram) { if (tabletBatchBuilder != null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java index a13f40b1b8308..552b8cf1cae1d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java @@ -521,9 +521,10 @@ public TPipeTransferReq compressIfNeeded(final TPipeTransferReq req) throws IOEx } @Override - public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { + public synchronized void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { if (Objects.nonNull(tabletBatchBuilder)) { - tabletBatchBuilder.discardEventsOfPipe(pipeNameToDrop, regionId); + tabletBatchBuilder.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketConnectorServer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketConnectorServer.java index 6da644604955e..2a8b5c8c3c07d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketConnectorServer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketConnectorServer.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.sink.protocol.websocket; +import org.apache.iotdb.commons.pipe.datastructure.Triple; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.pipe.api.event.Event; @@ -39,6 +40,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; @@ -57,6 +59,10 @@ public class WebSocketConnectorServer extends WebSocketServer { private final ConcurrentHashMap> eventsWaitingForAck = new ConcurrentHashMap<>(); + // Pipe name, creation time, region id + private final Set> droppedPipeTaskKeys = + ConcurrentHashMap.newKeySet(); + private final BidiMap router = new DualTreeBidiMap(null, Comparator.comparing(Object::hashCode)) {}; @@ -97,13 +103,8 @@ public synchronized void unregister(WebSocketSink connector) { eventWrappers = new ArrayList<>(eventTransferQueue); eventTransferQueue.clear(); } - eventWrappers.forEach( - (eventWrapper) -> { - if (eventWrapper.event instanceof EnrichedEvent) { - ((EnrichedEvent) eventWrapper.event) - .decreaseReferenceCount(WebSocketConnectorServer.class.getName(), false); - } - }); + eventWrappers.forEach(eventWrapper -> discardEvent(eventWrapper.event)); + eventWrappers.clear(); synchronized (eventTransferQueue) { eventTransferQueue.notifyAll(); } @@ -113,13 +114,36 @@ public synchronized void unregister(WebSocketSink connector) { if (eventsWaitingForAck.containsKey(pipeName)) { eventsWaitingForAck .remove(pipeName) - .forEach( - (eventId, eventWrapper) -> { - if (eventWrapper.event instanceof EnrichedEvent) { - ((EnrichedEvent) eventWrapper.event) - .decreaseReferenceCount(WebSocketConnectorServer.class.getName(), false); - } - }); + .forEach((eventId, eventWrapper) -> discardEvent(eventWrapper.event)); + } + + droppedPipeTaskKeys.removeIf(key -> key.getFirst().equals(pipeName)); + } + + public synchronized void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + droppedPipeTaskKeys.add(new Triple<>(pipeNameToDrop, creationTimeToDrop, regionId)); + + final PriorityBlockingQueue eventTransferQueue = + eventsWaitingForTransfer.get(pipeNameToDrop); + if (eventTransferQueue != null) { + eventTransferQueue.removeIf( + eventWrapper -> + discardIfMatches(eventWrapper.event, pipeNameToDrop, creationTimeToDrop, regionId)); + synchronized (eventTransferQueue) { + eventTransferQueue.notifyAll(); + } + } + + final ConcurrentHashMap eventId2EventMap = + eventsWaitingForAck.get(pipeNameToDrop); + if (eventId2EventMap != null) { + eventId2EventMap + .entrySet() + .removeIf( + entry -> + discardIfMatches( + entry.getValue().event, pipeNameToDrop, creationTimeToDrop, regionId)); } } @@ -300,21 +324,24 @@ public void onError(WebSocket webSocket, Exception e) { } public void addEvent(Event event, WebSocketSink connector) { + if (isDroppedPipe(event)) { + discardEvent(event); + return; + } + + final String pipeName = connector.getPipeName(); final PriorityBlockingQueue queue = - eventsWaitingForTransfer.get(connector.getPipeName()); + eventsWaitingForTransfer.get(pipeName); if (queue == null) { LOGGER.warn("The pipe {} was dropped so the event {} will be dropped.", connector, event); - if (event instanceof EnrichedEvent) { - ((EnrichedEvent) event) - .decreaseReferenceCount(WebSocketConnectorServer.class.getName(), false); - } + discardEvent(event); return; } if (queue.size() >= 5) { synchronized (queue) { - while (queue.size() >= 5) { + while (queue.size() >= 5 && isQueueAvailable(pipeName, queue) && !isDroppedPipe(event)) { try { queue.wait(); } catch (InterruptedException e) { @@ -323,12 +350,22 @@ public void addEvent(Event event, WebSocketSink connector) { } } + if (!isQueueAvailable(pipeName, queue) || isDroppedPipe(event)) { + discardEvent(event); + return; + } + queue.put( new EventWaitingForTransfer(eventIdGenerator.incrementAndGet(), connector, event)); return; } } + if (!isQueueAvailable(pipeName, queue) || isDroppedPipe(event)) { + discardEvent(event); + return; + } + synchronized (queue) { queue.put(new EventWaitingForTransfer(eventIdGenerator.incrementAndGet(), connector, event)); } @@ -377,6 +414,11 @@ private void transfer(String pipeName, EventWaitingForTransfer element) { final WebSocketSink connector = element.connector; try { + if (isDroppedPipe(event)) { + discardEvent(event); + return; + } + ByteBuffer tabletBuffer; if (event instanceof PipeRawTabletInsertionEvent) { tabletBuffer = ((PipeRawTabletInsertionEvent) event).convertToTablet().serialize(); @@ -387,7 +429,11 @@ private void transfer(String pipeName, EventWaitingForTransfer element) { } if (tabletBuffer == null) { - connector.commit((EnrichedEvent) event); + if (isDroppedPipe(event)) { + discardEvent(event); + } else { + connector.commit((EnrichedEvent) event); + } return; } @@ -398,11 +444,17 @@ private void transfer(String pipeName, EventWaitingForTransfer element) { server.broadcast(payload, Collections.singletonList(router.get(pipeName))); + if (isDroppedPipe(event)) { + discardEvent(event); + return; + } + final ConcurrentHashMap eventId2EventMap = eventsWaitingForAck.get(pipeName); if (eventId2EventMap == null) { LOGGER.warn( "The pipe {} was dropped so the event ack {} will be ignored.", pipeName, eventId); + discardEvent(event); return; } eventId2EventMap.put(eventId, new EventWaitingForAck(connector, event)); @@ -410,13 +462,10 @@ private void transfer(String pipeName, EventWaitingForTransfer element) { synchronized (server) { final PriorityBlockingQueue queue = eventsWaitingForTransfer.get(pipeName); - if (queue == null) { + if (queue == null || isDroppedPipe(event)) { LOGGER.warn( "The pipe {} was dropped so the event {} will be dropped.", pipeName, eventId); - if (event instanceof EnrichedEvent) { - ((EnrichedEvent) event) - .decreaseReferenceCount(WebSocketConnectorServer.class.getName(), false); - } + discardEvent(event); return; } @@ -465,4 +514,44 @@ public EventWaitingForAck(WebSocketSink connector, Event event) { this.event = event; } } + + private boolean discardIfMatches( + final Event event, + final String pipeNameToDrop, + final long creationTimeToDrop, + final int regionId) { + if (!(event instanceof EnrichedEvent)) { + return false; + } + + final EnrichedEvent enrichedEvent = (EnrichedEvent) event; + if (!pipeNameToDrop.equals(enrichedEvent.getPipeName()) + || creationTimeToDrop != enrichedEvent.getCreationTime() + || regionId != enrichedEvent.getRegionId()) { + return false; + } + + discardEvent(enrichedEvent); + return true; + } + + private boolean isDroppedPipe(final Event event) { + return event instanceof EnrichedEvent + && droppedPipeTaskKeys.contains( + new Triple<>( + ((EnrichedEvent) event).getPipeName(), + ((EnrichedEvent) event).getCreationTime(), + ((EnrichedEvent) event).getRegionId())); + } + + private boolean isQueueAvailable( + final String pipeName, final PriorityBlockingQueue queue) { + return eventsWaitingForTransfer.get(pipeName) == queue; + } + + private void discardEvent(final Event event) { + if (event instanceof EnrichedEvent) { + ((EnrichedEvent) event).clearReferenceCount(WebSocketSink.class.getName()); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketSink.java index c89486bc2c81d..40fccc12c999f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketSink.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.commons.pipe.sink.protocol.PipeConnectorWithEventDiscard; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; @@ -39,7 +40,7 @@ import java.util.Arrays; import java.util.Optional; -public class WebSocketSink implements PipeConnector { +public class WebSocketSink implements PipeConnector, PipeConnectorWithEventDiscard { private static final Logger LOGGER = LoggerFactory.getLogger(WebSocketSink.class); @@ -164,6 +165,14 @@ public void close() throws Exception { } } + @Override + public void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + if (server != null) { + server.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); + } + } + public void commit(EnrichedEvent enrichedEvent) { Optional.ofNullable(enrichedEvent) .ifPresent(event -> event.decreaseReferenceCount(WebSocketSink.class.getName(), true)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskLifeCycle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskLifeCycle.java index 98163697374da..af871feaa7e0d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskLifeCycle.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskLifeCycle.java @@ -63,7 +63,8 @@ public synchronized void register() { } @Override - public synchronized boolean deregister(final String pipeNameToDeregister, int regionId) { + public synchronized boolean deregister( + final String pipeNameToDeregister, final long creationTimeToDeregister, final int regionId) { if (registeredTaskCount <= 0) { throw new IllegalStateException("registeredTaskCount <= 0"); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskManager.java index 07def3ff4d39a..f4547673eaa42 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskManager.java @@ -167,7 +167,7 @@ public synchronized void deregister( final PipeSinkSubtaskLifeCycle lifeCycle = attributeSortedString2SubtaskLifeCycleMap.get(attributeSortedString); - if (lifeCycle.deregister(pipeName, regionId)) { + if (lifeCycle.deregister(pipeName, creationTime, regionId)) { attributeSortedString2SubtaskLifeCycleMap.remove(attributeSortedString); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollectorTest.java new file mode 100644 index 0000000000000..d54db821007f8 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollectorTest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.agent.task.connection; + +import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.db.pipe.agent.task.subtask.sink.PipeRealtimePriorityBlockingQueue; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; +import org.apache.iotdb.pipe.api.event.Event; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; + +public class PipeEventCollectorTest { + + @Test + public void testCollectorDoesNotOfferEventsOfDroppedPipeToUnboundedPendingQueue() { + verifyCollectorDoesNotOfferEventsOfDroppedPipe( + new UnboundedBlockingPendingQueue<>(new PipeDataRegionEventCounter())); + } + + @Test + public void testCollectorDoesNotOfferEventsOfDroppedPipeToRealtimePendingQueue() { + verifyCollectorDoesNotOfferEventsOfDroppedPipe(new PipeRealtimePriorityBlockingQueue()); + } + + private void verifyCollectorDoesNotOfferEventsOfDroppedPipe( + final UnboundedBlockingPendingQueue pendingQueue) { + pendingQueue.discardEventsOfPipe("pipe", 1L, 1); + + final PipeEventCollector droppedPipeCollector = + new PipeEventCollector(pendingQueue, 1L, 1, false, false); + final PipeRawTabletInsertionEvent droppedPipeEvent = + createPipeRawTabletInsertionEvent("pipe", 1L); + droppedPipeCollector.collect(droppedPipeEvent); + + Assert.assertTrue(droppedPipeEvent.isReleased()); + Assert.assertEquals(0, pendingQueue.size()); + + final PipeEventCollector recreatedPipeCollector = + new PipeEventCollector(pendingQueue, 2L, 1, false, false); + final PipeRawTabletInsertionEvent recreatedPipeEvent = + createPipeRawTabletInsertionEvent("pipe", 2L); + recreatedPipeCollector.collect(recreatedPipeEvent); + + Assert.assertFalse(recreatedPipeEvent.isReleased()); + Assert.assertEquals(1, pendingQueue.size()); + + pendingQueue.discardAllEvents(); + Assert.assertTrue(recreatedPipeEvent.isReleased()); + } + + private PipeRawTabletInsertionEvent createPipeRawTabletInsertionEvent( + final String pipeName, final long creationTime) { + final List schemaList = + Arrays.asList(new MeasurementSchema("s1", TSDataType.INT64)); + final Tablet tablet = new Tablet("root.db.d1", schemaList, 1); + tablet.addTimestamp(0, 1L); + tablet.addValue("s1", 0, 1L); + return new PipeRawTabletInsertionEvent( + tablet, false, pipeName, creationTime, null, null, false); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskTest.java new file mode 100644 index 0000000000000..ddfc699721b92 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.agent.task.subtask.sink; + +import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.sink.protocol.PipeConnectorWithEventDiscard; +import org.apache.iotdb.pipe.api.PipeConnector; + +import org.junit.Test; +import org.mockito.Mockito; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.withSettings; + +public class PipeSinkSubtaskTest { + + @Test + public void testDiscardEventsOfPipeDelegatesToConnector() { + final PipeConnector connector = + mock( + PipeConnector.class, + withSettings().extraInterfaces(PipeConnectorWithEventDiscard.class)); + final UnboundedBlockingPendingQueue pendingQueue = mock(UnboundedBlockingPendingQueue.class); + + final PipeSinkSubtask subtask = + Mockito.spy( + new PipeSinkSubtask( + "PipeSinkSubtaskTest", + System.currentTimeMillis(), + "data_test", + 0, + (UnboundedBlockingPendingQueue) pendingQueue, + connector)); + + try { + subtask.discardEventsOfPipe("pipe", 1L, 1); + + verify((PipeConnectorWithEventDiscard) connector).discardEventsOfPipe("pipe", 1L, 1); + } finally { + subtask.close(); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeSinkTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeSinkTest.java index fae6430876273..537ae6648b2b1 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeSinkTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeSinkTest.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.sink; import org.apache.iotdb.commons.pipe.agent.plugin.builtin.BuiltinPipePlugin; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant; import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskSinkRuntimeEnvironment; @@ -28,14 +29,18 @@ import org.apache.iotdb.db.pipe.sink.protocol.opcua.OpcUaSink; import org.apache.iotdb.db.pipe.sink.protocol.thrift.async.IoTDBDataRegionAsyncSink; import org.apache.iotdb.db.pipe.sink.protocol.thrift.sync.IoTDBDataRegionSyncSink; +import org.apache.iotdb.db.pipe.sink.protocol.websocket.WebSocketConnectorServer; +import org.apache.iotdb.db.pipe.sink.protocol.websocket.WebSocketSink; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.MeasurementSchema; import org.junit.Assert; import org.junit.Test; +import org.mockito.Mockito; import java.security.SecureRandom; import java.util.Arrays; @@ -103,6 +108,85 @@ public void testIoTDBThriftAsyncConnectorToOthers() { } } + @Test + public void testAsyncSinkDropDoesNotRequeueDroppedPipeEvents() throws Exception { + try (final IoTDBDataRegionAsyncSink connector = new IoTDBDataRegionAsyncSink()) { + final PipeParameters parameters = + new PipeParameters( + new HashMap() { + { + put( + PipeSinkConstant.CONNECTOR_KEY, + BuiltinPipePlugin.IOTDB_THRIFT_ASYNC_CONNECTOR.getPipePluginName()); + put(PipeSinkConstant.CONNECTOR_IOTDB_NODE_URLS_KEY, "127.0.0.1:6668"); + } + }); + connector.validate(new PipeParameterValidator(parameters)); + connector.customize( + parameters, + new PipeTaskRuntimeConfiguration(new PipeTaskSinkRuntimeEnvironment("pipe", 1L, 1))); + + final PipeRawTabletInsertionEvent droppedEvent = + createPipeRawTabletInsertionEvent("pipe", 1L, 1); + droppedEvent.increaseReferenceCount("test"); + droppedEvent.setCommitterKeyAndCommitId(new CommitterKey("pipe", 1L, 1, -1), 1L); + + connector.discardEventsOfPipe("pipe", 1L, 1); + connector.addFailureEventToRetryQueue(droppedEvent, new PipeException("test")); + + Assert.assertEquals(0, connector.getRetryEventQueueSize()); + Assert.assertTrue(droppedEvent.isReleased()); + + final PipeRawTabletInsertionEvent recreatedPipeEvent = + createPipeRawTabletInsertionEvent("pipe", 2L, 1); + recreatedPipeEvent.increaseReferenceCount("test"); + recreatedPipeEvent.setCommitterKeyAndCommitId(new CommitterKey("pipe", 2L, 1, -1), 1L); + + connector.addFailureEventToRetryQueue(recreatedPipeEvent, new PipeException("test")); + + Assert.assertEquals(1, connector.getRetryEventQueueSize()); + } + } + + @Test + public void testWebSocketSinkDropDoesNotRequeueDroppedPipeEvents() { + final String pipeName = "pipe_" + System.nanoTime(); + final WebSocketConnectorServer server = WebSocketConnectorServer.getOrCreateInstance(0); + final WebSocketSink connector = Mockito.mock(WebSocketSink.class); + Mockito.when(connector.getPipeName()).thenReturn(pipeName); + + server.register(connector); + try { + final PipeRawTabletInsertionEvent droppedEvent = + createPipeRawTabletInsertionEvent(pipeName, 1L, 1); + droppedEvent.increaseReferenceCount(WebSocketSink.class.getName()); + droppedEvent.setCommitterKeyAndCommitId(new CommitterKey(pipeName, 1L, 1, -1), 1L); + server.addEvent(droppedEvent, connector); + + server.discardEventsOfPipe(pipeName, 1L, 1); + Assert.assertTrue(droppedEvent.isReleased()); + + final PipeRawTabletInsertionEvent recreatedDroppedPipeEvent = + createPipeRawTabletInsertionEvent(pipeName, 1L, 1); + recreatedDroppedPipeEvent.increaseReferenceCount(WebSocketSink.class.getName()); + recreatedDroppedPipeEvent.setCommitterKeyAndCommitId( + new CommitterKey(pipeName, 1L, 1, -1), 2L); + server.addEvent(recreatedDroppedPipeEvent, connector); + + Assert.assertTrue(recreatedDroppedPipeEvent.isReleased()); + + final PipeRawTabletInsertionEvent recreatedPipeEvent = + createPipeRawTabletInsertionEvent(pipeName, 2L, 1); + recreatedPipeEvent.increaseReferenceCount(WebSocketSink.class.getName()); + recreatedPipeEvent.setCommitterKeyAndCommitId(new CommitterKey(pipeName, 2L, 1, -1), 3L); + server.addEvent(recreatedPipeEvent, connector); + + Assert.assertFalse(recreatedPipeEvent.isReleased()); + } finally { + server.unregister(connector); + } + } + @Test public void testOpcUaSink() { final List schemaList = @@ -181,4 +265,15 @@ public void testOpcUaSink() { Assert.fail(); } } + + private PipeRawTabletInsertionEvent createPipeRawTabletInsertionEvent( + final String pipeName, final long creationTime, final int regionId) { + final List schemaList = + Arrays.asList(new MeasurementSchema("s1", TSDataType.INT64)); + final Tablet tablet = new Tablet("root.db.d" + regionId, schemaList, 1); + tablet.addTimestamp(0, 1L); + tablet.addValue("s1", 0, 1L); + return new PipeRawTabletInsertionEvent( + tablet, false, pipeName, creationTime, null, null, false); + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java index 8773b03f9f308..8d920121363a3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java @@ -20,6 +20,7 @@ package org.apache.iotdb.commons.pipe.agent.task.connection; import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.datastructure.Triple; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.metric.PipeEventCounter; import org.apache.iotdb.pipe.api.event.Event; @@ -27,7 +28,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Set; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; @@ -44,6 +47,10 @@ public abstract class BlockingPendingQueue { protected final AtomicBoolean isClosed = new AtomicBoolean(false); + // Pipe name, creation time, region id + protected final Set> droppedPipeTaskKeys = + ConcurrentHashMap.newKeySet(); + protected BlockingPendingQueue( final BlockingQueue pendingQueue, final PipeEventCounter eventCounter) { this.pendingQueue = pendingQueue; @@ -51,7 +58,10 @@ protected BlockingPendingQueue( } public boolean offer(final E event) { - checkBeforeOffer(event); + if (!checkBeforeOffer(event)) { + return false; + } + final boolean offered = pendingQueue.offer(event); if (offered) { eventCounter.increaseEventCount(event); @@ -60,7 +70,9 @@ public boolean offer(final E event) { } public boolean put(final E event) { - checkBeforeOffer(event); + if (!checkBeforeOffer(event)) { + return false; + } try { pendingQueue.put(event); eventCounter.increaseEventCount(event); @@ -101,6 +113,7 @@ public void clear() { isClosed.set(true); pendingQueue.clear(); eventCounter.reset(); + droppedPipeTaskKeys.clear(); } /** DO NOT FORGET to set eventCounter to new value after invoking this method. */ @@ -120,14 +133,17 @@ public void discardAllEvents() { return true; }); eventCounter.reset(); + droppedPipeTaskKeys.clear(); } - public void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { + public void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + droppedPipeTaskKeys.add(new Triple<>(pipeNameToDrop, creationTimeToDrop, regionId)); pendingQueue.removeIf( event -> { if (event instanceof EnrichedEvent - && pipeNameToDrop.equals(((EnrichedEvent) event).getPipeName()) - && regionId == ((EnrichedEvent) event).getRegionId()) { + && isEventFromPipe( + ((EnrichedEvent) event), pipeNameToDrop, creationTimeToDrop, regionId)) { if (((EnrichedEvent) event).clearReferenceCount(BlockingPendingQueue.class.getName())) { eventCounter.decreaseEventCount(event); } @@ -157,9 +173,34 @@ public int getPipeHeartbeatEventCount() { return eventCounter.getPipeHeartbeatEventCount(); } - protected void checkBeforeOffer(final E event) { - if (isClosed.get() && event instanceof EnrichedEvent) { + protected boolean checkBeforeOffer(final E event) { + final boolean shouldReject = isClosed.get() || isEventFromDroppedPipe(event); + if (shouldReject && event instanceof EnrichedEvent) { ((EnrichedEvent) event).clearReferenceCount(BlockingPendingQueue.class.getName()); } + return !shouldReject; + } + + protected static boolean isEventFromPipe( + final EnrichedEvent event, + final String pipeNameToDrop, + final long creationTimeToDrop, + final int regionId) { + return pipeNameToDrop.equals(event.getPipeName()) + && creationTimeToDrop == event.getCreationTime() + && regionId == event.getRegionId(); + } + + protected boolean isEventFromDroppedPipe(final E event) { + return event instanceof EnrichedEvent + && ((EnrichedEvent) event).getPipeName() != null + && isPipeDropped( + ((EnrichedEvent) event).getPipeName(), + ((EnrichedEvent) event).getCreationTime(), + ((EnrichedEvent) event).getRegionId()); + } + + public boolean isPipeDropped(final String pipeName, final long creationTime, final int regionId) { + return droppedPipeTaskKeys.contains(new Triple<>(pipeName, creationTime, regionId)); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/Triple.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/Triple.java new file mode 100644 index 0000000000000..275ccb20ea3ac --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/Triple.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.datastructure; + +import java.util.Objects; + +public class Triple { + public final L first; + public final M second; + public final R third; + + public Triple(final L first, final M second, final R third) { + this.first = first; + this.second = second; + this.third = third; + } + + public L getFirst() { + return first; + } + + public M getSecond() { + return second; + } + + public R getThird() { + return third; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Triple triple = (Triple) o; + return first.equals(triple.first) && second.equals(triple.second) && third.equals(triple.third); + } + + @Override + public int hashCode() { + return Objects.hash(first, second, third); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSink.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSink.java index 66f3eda6a1a48..2f2d54e7b4fe0 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSink.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBSink.java @@ -139,7 +139,7 @@ import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.SINK_RATE_LIMIT_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.SINK_SKIP_IF_KEY; -public abstract class IoTDBSink implements PipeConnector { +public abstract class IoTDBSink implements PipeConnector, PipeConnectorWithEventDiscard { private static final String PARSE_URL_ERROR_FORMATTER = "Exception occurred while parsing node urls from target servers: {}"; @@ -621,7 +621,8 @@ public void rateLimitIfNeeded( * When a pipe is dropped, the connector maybe reused and will not be closed. We need to discard * its batched or queued events in the output pipe connector. */ - public synchronized void discardEventsOfPipe(final String pipeName, final int regionId) { + public synchronized void discardEventsOfPipe( + final String pipeName, final long creationTime, final int regionId) { // Do nothing by default } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/PipeConnectorWithEventDiscard.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/PipeConnectorWithEventDiscard.java new file mode 100644 index 0000000000000..ab4dbcf90750f --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/PipeConnectorWithEventDiscard.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.sink.protocol; + +public interface PipeConnectorWithEventDiscard { + + void discardEventsOfPipe(String pipeName, long creationTime, int regionId); +} From f6995bbb80d4b8529476420eb660fd32c82988fe Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 9 May 2026 15:12:44 +0800 Subject: [PATCH 039/102] [To dev/1.3] Load: Fixed multiple bugs (#17565) (#17623) * Load: Fixed multiple bugs (#17565) * fix * test-del --- .../storageengine/load/LoadTsFileManager.java | 54 ++++++--- .../load/memory/LoadTsFileMemoryManager.java | 13 ++- .../memory/LoadTsFileMemoryManagerTest.java | 106 ++++++++++++++++++ 3 files changed, 154 insertions(+), 19 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/load/memory/LoadTsFileMemoryManagerTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java index ce1bb680f21e2..0b85e3f0e644a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java @@ -128,6 +128,34 @@ public LoadTsFileManager() { activeLoadAgent.start(); } + private long getCleanupTaskDelayInMs() { + return CONFIG.getLoadCleanupTaskExecutionDelayTimeSeconds() * 1000L; + } + + private void createCleanupTaskIfAbsent(final String uuid) { + synchronized (uuid2CleanupTask) { + if (uuid2CleanupTask.containsKey(uuid)) { + return; + } + + final CleanupTask cleanupTask = new CleanupTask(uuid, getCleanupTaskDelayInMs()); + uuid2CleanupTask.put(uuid, cleanupTask); + cleanupTaskQueue.add(cleanupTask); + } + } + + private void rescheduleCleanupTask(final CleanupTask cleanupTask) { + synchronized (uuid2CleanupTask) { + if (uuid2CleanupTask.get(cleanupTask.uuid) != cleanupTask) { + return; + } + + cleanupTaskQueue.remove(cleanupTask); + cleanupTask.resetScheduledTime(); + cleanupTaskQueue.add(cleanupTask); + } + } + private void registerCleanupTaskExecutor() { PipeDataNodeAgent.runtime() .registerPeriodicalJob( @@ -199,26 +227,13 @@ private void recover() { uuid2WriterManager.put(uuid, writerManager); writerManager.close(); - synchronized (uuid2CleanupTask) { - final CleanupTask cleanupTask = - new CleanupTask( - uuid, CONFIG.getLoadCleanupTaskExecutionDelayTimeSeconds() * 1000); - uuid2CleanupTask.put(uuid, cleanupTask); - cleanupTaskQueue.add(cleanupTask); - } + createCleanupTaskIfAbsent(uuid); }); } public void writeToDataRegion(DataRegion dataRegion, LoadTsFilePieceNode pieceNode, String uuid) throws IOException, PageException { - if (!uuid2WriterManager.containsKey(uuid)) { - synchronized (uuid2CleanupTask) { - final CleanupTask cleanupTask = - new CleanupTask(uuid, CONFIG.getLoadCleanupTaskExecutionDelayTimeSeconds() * 1000); - uuid2CleanupTask.put(uuid, cleanupTask); - cleanupTaskQueue.add(cleanupTask); - } - } + createCleanupTaskIfAbsent(uuid); final Optional cleanupTask = Optional.ofNullable(uuid2CleanupTask.get(uuid)); cleanupTask.ifPresent(CleanupTask::markLoadTaskRunning); @@ -293,6 +308,8 @@ public boolean loadAll( return false; } + createCleanupTaskIfAbsent(uuid); + final Optional cleanupTask = Optional.ofNullable(uuid2CleanupTask.get(uuid)); cleanupTask.ifPresent(CleanupTask::markLoadTaskRunning); try { @@ -315,9 +332,10 @@ public boolean deleteAll(String uuid) { private void clean(String uuid) { synchronized (uuid2CleanupTask) { - final CleanupTask cleanupTask = uuid2CleanupTask.get(uuid); + final CleanupTask cleanupTask = uuid2CleanupTask.remove(uuid); if (cleanupTask != null) { cleanupTask.cancel(); + cleanupTaskQueue.remove(cleanupTask); } } @@ -744,12 +762,12 @@ private CleanupTask(String uuid, long delayInMs) { public void markLoadTaskRunning() { isLoadTaskRunning = true; - resetScheduledTime(); + rescheduleCleanupTask(this); } public void markLoadTaskNotRunning() { isLoadTaskRunning = false; - resetScheduledTime(); + rescheduleCleanupTask(this); } public void resetScheduledTime() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/memory/LoadTsFileMemoryManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/memory/LoadTsFileMemoryManager.java index 6c85d2280fd0b..c3f8881530050 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/memory/LoadTsFileMemoryManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/memory/LoadTsFileMemoryManager.java @@ -126,7 +126,18 @@ public synchronized LoadTsFileDataCacheMemoryBlock allocateDataCacheMemoryBlock( if (dataCacheMemoryBlock == null) { final long actuallyAllocateMemoryInBytes = tryAllocateFromQuery(MEMORY_TOTAL_SIZE_FROM_QUERY_IN_BYTES >> 2); - dataCacheMemoryBlock = new LoadTsFileDataCacheMemoryBlock(actuallyAllocateMemoryInBytes); + try { + dataCacheMemoryBlock = new LoadTsFileDataCacheMemoryBlock(actuallyAllocateMemoryInBytes); + } catch (RuntimeException e) { + if (actuallyAllocateMemoryInBytes > 0) { + try { + releaseToQuery(actuallyAllocateMemoryInBytes); + } catch (RuntimeException releaseException) { + e.addSuppressed(releaseException); + } + } + throw e; + } LOGGER.info( "Create Data Cache Memory Block {}, allocate memory {}", dataCacheMemoryBlock, diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/load/memory/LoadTsFileMemoryManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/load/memory/LoadTsFileMemoryManagerTest.java new file mode 100644 index 0000000000000..f3a1bf7e4116f --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/load/memory/LoadTsFileMemoryManagerTest.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.load.memory; + +import org.apache.iotdb.db.exception.load.LoadRuntimeOutOfMemoryException; + +import org.junit.Assert; +import org.junit.Test; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.util.concurrent.atomic.AtomicLong; + +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +public class LoadTsFileMemoryManagerTest { + + @Test + public void testAllocateDataCacheMemoryBlockDoesNotDoubleCountMemory() throws Exception { + final long allocatedMemoryInBytes = 2L * 1024 * 1024; + final LoadTsFileMemoryManager manager = spy(newMemoryManager()); + + doAnswer( + invocation -> { + setUsedMemorySize(manager, allocatedMemoryInBytes); + return allocatedMemoryInBytes; + }) + .when(manager) + .tryAllocateFromQuery(anyLong()); + + manager.allocateDataCacheMemoryBlock(); + + Assert.assertEquals(allocatedMemoryInBytes, manager.getUsedMemorySizeInBytes()); + Assert.assertNotNull(getDataCacheMemoryBlock(manager)); + } + + @Test + public void testAllocateDataCacheMemoryBlockRollsBackPartialAllocationOnFailure() + throws Exception { + final long allocatedMemoryInBytes = 512L; + final LoadTsFileMemoryManager manager = spy(newMemoryManager()); + + doAnswer( + invocation -> { + setUsedMemorySize(manager, allocatedMemoryInBytes); + return allocatedMemoryInBytes; + }) + .when(manager) + .tryAllocateFromQuery(anyLong()); + doAnswer( + invocation -> { + setUsedMemorySize(manager, 0L); + return null; + }) + .when(manager) + .releaseToQuery(anyLong()); + + try { + manager.allocateDataCacheMemoryBlock(); + Assert.fail("Expected LoadRuntimeOutOfMemoryException"); + } catch (LoadRuntimeOutOfMemoryException e) { + Assert.assertEquals(0L, manager.getUsedMemorySizeInBytes()); + Assert.assertNull(getDataCacheMemoryBlock(manager)); + } + } + + private static LoadTsFileMemoryManager newMemoryManager() throws Exception { + final Constructor constructor = + LoadTsFileMemoryManager.class.getDeclaredConstructor(); + constructor.setAccessible(true); + return constructor.newInstance(); + } + + private static void setUsedMemorySize( + final LoadTsFileMemoryManager manager, final long usedMemorySizeInBytes) throws Exception { + final Field field = LoadTsFileMemoryManager.class.getDeclaredField("usedMemorySizeInBytes"); + field.setAccessible(true); + ((AtomicLong) field.get(manager)).set(usedMemorySizeInBytes); + } + + private static LoadTsFileDataCacheMemoryBlock getDataCacheMemoryBlock( + final LoadTsFileMemoryManager manager) throws Exception { + final Field field = LoadTsFileMemoryManager.class.getDeclaredField("dataCacheMemoryBlock"); + field.setAccessible(true); + return (LoadTsFileDataCacheMemoryBlock) field.get(manager); + } +} From ea45c4f9415fa2f6c90ef55463ed6675e01e5adb Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 9 May 2026 19:21:24 +0800 Subject: [PATCH 040/102] [To dev/1.3] Pipe: Fixed the first-chunk calculation bug for scan parser (#17597) (#17622) * Pipe: Fixed the first-chunk calculation bug for scan parser (#17597) * Update TsFileInsertionDataContainerTest.java * Update TsFileInsertionDataContainerTest.java --- .../TsFileInsertionScanDataContainer.java | 21 ++++++-- .../TsFileInsertionDataContainerTest.java | 51 +++++++++++++++++++ 2 files changed, 67 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java index 8ed9bdcd6625a..07b91cd68af7c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java @@ -605,11 +605,6 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { final long chunkSize = timeChunkSize + valueChunkSize; if (chunkSize + chunkHeader.getDataSize() > allocatedMemoryBlockForChunk.getMemoryUsageInBytes()) { - if (valueChunkList.size() == 1 - && chunkSize > allocatedMemoryBlockForChunk.getMemoryUsageInBytes()) { - PipeDataNodeResourceManager.memory() - .forceResize(allocatedMemoryBlockForChunk, chunkSize); - } needReturn = recordAlignedChunk(valueChunkList, marker); } } @@ -619,9 +614,11 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { firstChunkHeader4NextSequentialValueChunks = chunkHeader; return; } + resizeChunkMemoryBlockIfFirstValueChunkExceedsLimit(valueChunkList, chunkHeader); } else { chunkHeader = firstChunkHeader4NextSequentialValueChunks; firstChunkHeader4NextSequentialValueChunks = null; + resizeChunkMemoryBlockIfFirstValueChunkExceedsLimit(valueChunkList, chunkHeader); } Chunk chunk = @@ -690,6 +687,20 @@ private boolean recordAlignedChunk(final List valueChunkList, final byte return false; } + private void resizeChunkMemoryBlockIfFirstValueChunkExceedsLimit( + final List valueChunkList, final ChunkHeader valueChunkHeader) { + if (!valueChunkList.isEmpty() || lastIndex < 0) { + return; + } + + final long chunkSize = + PipeMemoryWeightUtil.calculateChunkRamBytesUsed(timeChunkList.get(lastIndex)) + + valueChunkHeader.getDataSize(); + if (chunkSize > allocatedMemoryBlockForChunk.getMemoryUsageInBytes()) { + PipeDataNodeResourceManager.memory().forceResize(allocatedMemoryBlockForChunk, chunkSize); + } + } + @Override public void close() { super.close(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java index bd4e3923815f4..51fc1f3a55617 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.pipe.event.common.tsfile.container.TsFileInsertionDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.container.query.TsFileInsertionQueryDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.container.scan.TsFileInsertionScanDataContainer; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionTestFileWriter; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; @@ -56,6 +57,7 @@ import java.io.File; import java.io.IOException; +import java.lang.reflect.Field; import java.time.LocalDate; import java.util.ArrayList; import java.util.Arrays; @@ -119,6 +121,47 @@ public void testScanContainer() throws Exception { System.out.println(System.currentTimeMillis() - startTime); } + @Test + public void testScanParserResizesChunkMemoryForFirstAlignedValueChunk() throws Exception { + final long originalPipeMaxReaderChunkSize = + PipeConfig.getInstance().getPipeMaxReaderChunkSize(); + CommonDescriptor.getInstance().getConfig().setPipeMaxReaderChunkSize(0); + + alignedTsFile = new File("single-aligned-value-chunk.tsfile"); + final List schemaList = new ArrayList<>(); + schemaList.add(new MeasurementSchema("s1", TSDataType.INT64)); + + final Tablet tablet = new Tablet("root.sg.d", schemaList, 2); + tablet.rowSize = 2; + tablet.addTimestamp(0, 1); + tablet.addValue("s1", 0, 1L); + tablet.addTimestamp(1, 2); + tablet.addValue("s1", 1, 2L); + + try { + try (final TsFileWriter writer = new TsFileWriter(alignedTsFile)) { + writer.registerAlignedTimeseries(new Path("root.sg.d"), schemaList); + writer.writeAligned(tablet); + } + + try (final TsFileInsertionScanDataContainer parser = + new TsFileInsertionScanDataContainer( + alignedTsFile, + new PrefixPipePattern("root"), + Long.MIN_VALUE, + Long.MAX_VALUE, + null, + null, + false)) { + Assert.assertTrue(getAllocatedChunkMemory(parser).getMemoryUsageInBytes() > 0); + } + } finally { + CommonDescriptor.getInstance() + .getConfig() + .setPipeMaxReaderChunkSize(originalPipeMaxReaderChunkSize); + } + } + public void testToTabletInsertionEvents(final boolean isQuery) throws Exception { // Test empty chunk testMixedTsFileWithEmptyChunk(isQuery); @@ -645,4 +688,12 @@ private int getNonNullSize(final Tablet tablet) { } return count; } + + private PipeMemoryBlock getAllocatedChunkMemory(final TsFileInsertionScanDataContainer parser) + throws NoSuchFieldException, IllegalAccessException { + final Field field = + TsFileInsertionScanDataContainer.class.getDeclaredField("allocatedMemoryBlockForChunk"); + field.setAccessible(true); + return (PipeMemoryBlock) field.get(parser); + } } From 9b5973e4691b2c9c9691ade145613bf817e98cf5 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 11 May 2026 09:52:59 +0800 Subject: [PATCH 041/102] [To dev/1.3] Fixed multiple bugs of insertion (#17570) (#17608) * Fixed multiple bugs of insertion (#17570) * insert-fix * source/sink * source/sink-2 * sptls * fix * sink * compile * fix-bug * no-compose * fix-13 --- .../sink/protocol/IoTDBConfigRegionSink.java | 9 +- .../db/pipe/agent/task/PipeDataNodeTask.java | 28 +- .../execution/PipeSinkSubtaskExecutor.java | 2 +- .../task/stage/PipeTaskProcessorStage.java | 20 +- .../agent/task/stage/PipeTaskSinkStage.java | 13 +- .../agent/task/stage/PipeTaskSourceStage.java | 19 +- .../airgap/IoTDBSchemaRegionAirGapSink.java | 7 +- .../PipeTransferTabletBatchEventHandler.java | 10 +- ...eTransferTabletInsertNodeEventHandler.java | 2 +- ...peTransferTabletInsertionEventHandler.java | 7 +- .../PipeTransferTabletRawEventHandler.java | 6 +- .../handler/PipeTransferTrackableHandler.java | 24 +- .../handler/PipeTransferTsFileHandler.java | 27 +- .../realtime/assigner/DisruptorQueue.java | 4 +- .../PipeInsertionDataNodeListener.java | 38 +-- .../plan/node/write/InsertTabletNode.java | 63 ++++- .../storageengine/dataregion/DataRegion.java | 243 ++++++++++++------ .../task/stage/SubscriptionTaskSinkStage.java | 11 +- .../dataregion/DataRegionTest.java | 220 ++++++++++++++++ .../commons/client/ClientPoolFactory.java | 4 +- .../iotdb/commons/concurrent/ThreadName.java | 12 +- .../iotdb/commons/conf/CommonConfig.java | 93 ++++--- .../commons/pipe/config/PipeDescriptor.java | 8 +- 23 files changed, 604 insertions(+), 266 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionSink.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionSink.java index 5ee983d7945bd..b5f62ef4ccf48 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionSink.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/sink/protocol/IoTDBConfigRegionSink.java @@ -88,7 +88,7 @@ protected IoTDBSyncClientManager constructClient( protected PipeTransferFilePieceReq getTransferSingleFilePieceReq( final String fileName, final long position, final byte[] payLoad) { throw new UnsupportedOperationException( - "The config region connector does not support transferring single file piece req."); + "The config region sink does not support transferring single file piece req."); } @Override @@ -105,13 +105,13 @@ protected void mayLimitRateAndRecordIO(final long requiredBytes) { @Override public void transfer(final TabletInsertionEvent tabletInsertionEvent) throws Exception { throw new UnsupportedOperationException( - "IoTDBConfigRegionConnector can't transfer TabletInsertionEvent."); + "IoTDBConfigRegionSink can't transfer TabletInsertionEvent."); } @Override public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exception { throw new UnsupportedOperationException( - "IoTDBConfigRegionConnector can't transfer TsFileInsertionEvent."); + "IoTDBConfigRegionSink can't transfer TsFileInsertionEvent."); } @Override @@ -121,8 +121,7 @@ public void transfer(final Event event) throws Exception { } else if (event instanceof PipeConfigRegionSnapshotEvent) { doTransferWrapper((PipeConfigRegionSnapshotEvent) event); } else if (!(event instanceof PipeHeartbeatEvent)) { - LOGGER.warn( - "IoTDBConfigRegionConnector does not support transferring generic event: {}.", event); + LOGGER.warn("IoTDBConfigRegionSink does not support transferring generic event: {}.", event); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTask.java index d33ec44a86e53..0d0b955c2109d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTask.java @@ -32,32 +32,32 @@ public class PipeDataNodeTask implements PipeTask { private final String pipeName; private final int regionId; - private final PipeTaskStage extractorStage; + private final PipeTaskStage sourceStage; private final PipeTaskStage processorStage; - private final PipeTaskStage connectorStage; + private final PipeTaskStage sinkStage; private volatile boolean isCompleted = false; public PipeDataNodeTask( final String pipeName, final int regionId, - final PipeTaskStage extractorStage, + final PipeTaskStage sourceStage, final PipeTaskStage processorStage, - final PipeTaskStage connectorStage) { + final PipeTaskStage sinkStage) { this.pipeName = pipeName; this.regionId = regionId; - this.extractorStage = extractorStage; + this.sourceStage = sourceStage; this.processorStage = processorStage; - this.connectorStage = connectorStage; + this.sinkStage = sinkStage; } @Override public void create() { final long startTime = System.currentTimeMillis(); - extractorStage.create(); + sourceStage.create(); processorStage.create(); - connectorStage.create(); + sinkStage.create(); LOGGER.info( "Create pipe DN task {} successfully within {} ms", this, @@ -67,9 +67,9 @@ public void create() { @Override public void drop() { final long startTime = System.currentTimeMillis(); - extractorStage.drop(); + sourceStage.drop(); processorStage.drop(); - connectorStage.drop(); + sinkStage.drop(); LOGGER.info( "Drop pipe DN task {} successfully within {} ms", this, @@ -79,9 +79,9 @@ public void drop() { @Override public void start() { final long startTime = System.currentTimeMillis(); - extractorStage.start(); + sourceStage.start(); processorStage.start(); - connectorStage.start(); + sinkStage.start(); LOGGER.info( "Start pipe DN task {} successfully within {} ms", this, @@ -91,9 +91,9 @@ public void start() { @Override public void stop() { final long startTime = System.currentTimeMillis(); - extractorStage.stop(); + sourceStage.stop(); processorStage.stop(); - connectorStage.stop(); + sinkStage.stop(); LOGGER.info( "Stop pipe DN task {} successfully within {} ms", this, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSinkSubtaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSinkSubtaskExecutor.java index 9a88ad74d7ccb..4abc8daed3002 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSinkSubtaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSinkSubtaskExecutor.java @@ -31,7 +31,7 @@ public class PipeSinkSubtaskExecutor extends PipeSubtaskExecutor { public PipeSinkSubtaskExecutor() { super( PipeConfig.getInstance().getPipeSubtaskExecutorMaxThreadNum(), - ThreadName.PIPE_CONNECTOR_EXECUTOR_POOL.getName() + "-" + id.get(), + ThreadName.PIPE_SINK_EXECUTOR_POOL.getName() + "-" + id.get(), ThreadName.PIPE_SUBTASK_CALLBACK_EXECUTOR_POOL.getName() + "-" + id.getAndIncrement(), true); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskProcessorStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskProcessorStage.java index ddc194716d2d1..c5f58a248e265 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskProcessorStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskProcessorStage.java @@ -54,8 +54,8 @@ public class PipeTaskProcessorStage extends PipeTaskStage { * @param creationTime pipe creation time * @param pipeProcessorParameters used to create {@link PipeProcessor} * @param regionId {@link DataRegion} id - * @param pipeExtractorInputEventSupplier used to input {@link Event}s from {@link PipeExtractor} - * @param pipeConnectorOutputPendingQueue used to output {@link Event}s to {@link PipeConnector} + * @param pipeSourceInputEventSupplier used to input {@link Event}s from {@link PipeExtractor} + * @param pipeSinkOutputPendingQueue used to output {@link Event}s to {@link PipeConnector} * @throws PipeException if failed to {@link PipeProcessor#validate(PipeParameterValidator)} or * {@link PipeProcessor#customize(PipeParameters, PipeProcessorRuntimeConfiguration)}} */ @@ -64,8 +64,8 @@ public PipeTaskProcessorStage( final long creationTime, final PipeParameters pipeProcessorParameters, final int regionId, - final EventSupplier pipeExtractorInputEventSupplier, - final UnboundedBlockingPendingQueue pipeConnectorOutputPendingQueue, + final EventSupplier pipeSourceInputEventSupplier, + final UnboundedBlockingPendingQueue pipeSinkOutputPendingQueue, final PipeProcessorSubtaskExecutor executor, final PipeTaskMeta pipeTaskMeta, final boolean forceTabletFormat, @@ -99,22 +99,18 @@ public PipeTaskProcessorStage( // removed, the new subtask will have the same pipeName and regionId as the // old one, so we need creationTime to make their hash code different in the map. final String taskId = pipeName + "_" + regionId + "_" + creationTime; - final PipeEventCollector pipeConnectorOutputEventCollector = + final PipeEventCollector pipeSinkOutputEventCollector = new PipeEventCollector( - pipeConnectorOutputPendingQueue, - creationTime, - regionId, - forceTabletFormat, - skipParsing); + pipeSinkOutputPendingQueue, creationTime, regionId, forceTabletFormat, skipParsing); this.pipeProcessorSubtask = new PipeProcessorSubtask( taskId, pipeName, creationTime, regionId, - pipeExtractorInputEventSupplier, + pipeSourceInputEventSupplier, pipeProcessor, - pipeConnectorOutputEventCollector); + pipeSinkOutputEventCollector); this.executor = executor; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskSinkStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskSinkStage.java index a22fbb536d704..88eac560cde2b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskSinkStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskSinkStage.java @@ -38,7 +38,7 @@ public class PipeTaskSinkStage extends PipeTaskStage { protected final int regionId; protected final Supplier executor; - protected String connectorSubtaskId; + protected String sinkSubtaskId; public PipeTaskSinkStage( String pipeName, @@ -56,7 +56,7 @@ public PipeTaskSinkStage( } protected void registerSubtask() { - this.connectorSubtaskId = + this.sinkSubtaskId = PipeSinkSubtaskManager.instance() .register( executor, @@ -71,21 +71,20 @@ public void createSubtask() throws PipeException { @Override public void startSubtask() throws PipeException { - PipeSinkSubtaskManager.instance().start(connectorSubtaskId); + PipeSinkSubtaskManager.instance().start(sinkSubtaskId); } @Override public void stopSubtask() throws PipeException { - PipeSinkSubtaskManager.instance().stop(connectorSubtaskId); + PipeSinkSubtaskManager.instance().stop(sinkSubtaskId); } @Override public void dropSubtask() throws PipeException { - PipeSinkSubtaskManager.instance() - .deregister(pipeName, creationTime, regionId, connectorSubtaskId); + PipeSinkSubtaskManager.instance().deregister(pipeName, creationTime, regionId, sinkSubtaskId); } public UnboundedBlockingPendingQueue getPipeSinkPendingQueue() { - return PipeSinkSubtaskManager.instance().getPipeSinkPendingQueue(connectorSubtaskId); + return PipeSinkSubtaskManager.instance().getPipeSinkPendingQueue(sinkSubtaskId); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskSourceStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskSourceStage.java index 6acc0fc3d4af2..240b5499e92b2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskSourceStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskSourceStage.java @@ -44,32 +44,31 @@ public class PipeTaskSourceStage extends PipeTaskStage { public PipeTaskSourceStage( String pipeName, long creationTime, - PipeParameters extractorParameters, + PipeParameters sourceParameters, int regionId, PipeTaskMeta pipeTaskMeta) { pipeExtractor = StorageEngine.getInstance().getAllDataRegionIds().contains(new DataRegionId(regionId)) - ? PipeDataNodeAgent.plugin().dataRegion().reflectSource(extractorParameters) - : PipeDataNodeAgent.plugin().schemaRegion().reflectSource(extractorParameters); + ? PipeDataNodeAgent.plugin().dataRegion().reflectSource(sourceParameters) + : PipeDataNodeAgent.plugin().schemaRegion().reflectSource(sourceParameters); - // Validate and customize should be called before createSubtask. this allows extractor exposing + // Validate and customize should be called before createSubtask. this allows source exposing // exceptions in advance. try { - // 1. Validate extractor parameters - pipeExtractor.validate(new PipeParameterValidator(extractorParameters)); + // 1. Validate source parameters + pipeExtractor.validate(new PipeParameterValidator(sourceParameters)); - // 2. Customize extractor + // 2. Customize source final PipeTaskRuntimeConfiguration runtimeConfiguration = new PipeTaskRuntimeConfiguration( new PipeTaskSourceRuntimeEnvironment(pipeName, creationTime, regionId, pipeTaskMeta)); - pipeExtractor.customize(extractorParameters, runtimeConfiguration); + pipeExtractor.customize(sourceParameters, runtimeConfiguration); } catch (Exception e) { try { pipeExtractor.close(); } catch (Exception closeException) { LOGGER.warn( - "Failed to close extractor after failed to initialize extractor. " - + "Ignore this exception.", + "Failed to close source after failed to initialize source. " + "Ignore this exception.", closeException); } throw new PipeException(e.getMessage(), e); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBSchemaRegionAirGapSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBSchemaRegionAirGapSink.java index 8280709446a3b..6b7787ba16282 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBSchemaRegionAirGapSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBSchemaRegionAirGapSink.java @@ -47,13 +47,13 @@ public class IoTDBSchemaRegionAirGapSink extends IoTDBDataNodeAirGapSink { @Override public void transfer(final TabletInsertionEvent tabletInsertionEvent) throws Exception { throw new UnsupportedOperationException( - "IoTDBSchemaRegionAirGapConnector can't transfer TabletInsertionEvent."); + "IoTDBSchemaRegionAirGapSink can't transfer TabletInsertionEvent."); } @Override public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exception { throw new UnsupportedOperationException( - "IoTDBSchemaRegionAirGapConnector can't transfer TsFileInsertionEvent."); + "IoTDBSchemaRegionAirGapSink can't transfer TsFileInsertionEvent."); } @Override @@ -68,8 +68,7 @@ public void transfer(final Event event) throws Exception { doTransferWrapper(socket, (PipeSchemaRegionSnapshotEvent) event); } else if (!(event instanceof PipeHeartbeatEvent)) { LOGGER.warn( - "IoTDBSchemaRegionAirGapConnector does not support transferring generic event: {}.", - event); + "IoTDBSchemaRegionAirGapSink does not support transferring generic event: {}.", event); } } catch (final IOException e) { isSocketAlive.set(socketIndex, false); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java index 110d3cb645027..8bcb9d47009c1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java @@ -69,7 +69,7 @@ public PipeTransferTabletBatchEventHandler( public void transfer(final AsyncPipeDataTransferServiceClient client) throws TException { for (final Map.Entry, Long> entry : pipeName2BytesAccumulated.entrySet()) { - connector.rateLimitIfNeeded( + sink.rateLimitIfNeeded( entry.getKey().getLeft(), entry.getKey().getRight(), client.getEndPoint(), @@ -92,13 +92,11 @@ protected boolean onCompleteInternal(final TPipeTransferResp response) { // Only handle the failed statuses to avoid string format performance overhead if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { - connector - .statusHandler() - .handle(status, response.getStatus().getMessage(), events.toString()); + sink.statusHandler().handle(status, response.getStatus().getMessage(), events.toString()); } for (final Pair redirectPair : LeaderCacheUtils.parseRecommendedRedirections(status)) { - connector.updateLeaderCache(redirectPair.getLeft(), redirectPair.getRight()); + sink.updateLeaderCache(redirectPair.getLeft(), redirectPair.getRight()); } events.forEach( @@ -123,7 +121,7 @@ protected void onErrorInternal(final Exception exception) { events.size(), events.stream().map(EnrichedEvent::getPipeName).collect(Collectors.toSet())); } finally { - connector.addFailureEventsToRetryQueue(events, exception); + sink.addFailureEventsToRetryQueue(events, exception); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java index 70ba7f4cfc5b7..912a1e724f748 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java @@ -46,7 +46,7 @@ protected void doTransfer( @Override protected void updateLeaderCache(final TSStatus status) { - connector.updateLeaderCache( + sink.updateLeaderCache( ((PipeInsertNodeTabletInsertionEvent) event).getDeviceId(), status.getRedirectNode()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertionEventHandler.java index 66a1f4a013b22..a8f1136b897d3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertionEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertionEventHandler.java @@ -54,7 +54,7 @@ protected PipeTransferTabletInsertionEventHandler( public void transfer(final AsyncPipeDataTransferServiceClient client) throws TException { if (event instanceof EnrichedEvent) { - connector.rateLimitIfNeeded( + sink.rateLimitIfNeeded( ((EnrichedEvent) event).getPipeName(), ((EnrichedEvent) event).getCreationTime(), client.getEndPoint(), @@ -77,8 +77,7 @@ protected boolean onCompleteInternal(final TPipeTransferResp response) { // Only handle the failed statuses to avoid string format performance overhead if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { - connector - .statusHandler() + sink.statusHandler() .handle(response.getStatus(), response.getStatus().getMessage(), event.toString()); } if (event instanceof EnrichedEvent) { @@ -109,7 +108,7 @@ protected void onErrorInternal(final Exception exception) { event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitterKey() : null, event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitIds() : null); } finally { - connector.addFailureEventToRetryQueue(event, exception); + sink.addFailureEventToRetryQueue(event, exception); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletRawEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletRawEventHandler.java index ff1daa05c2859..b64e446827aff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletRawEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletRawEventHandler.java @@ -32,8 +32,8 @@ public class PipeTransferTabletRawEventHandler extends PipeTransferTabletInserti public PipeTransferTabletRawEventHandler( final PipeRawTabletInsertionEvent event, final TPipeTransferReq req, - final IoTDBDataRegionAsyncSink connector) { - super(event, req, connector); + final IoTDBDataRegionAsyncSink sink) { + super(event, req, sink); } @Override @@ -45,7 +45,7 @@ protected void doTransfer( @Override protected void updateLeaderCache(final TSStatus status) { - connector.updateLeaderCache( + sink.updateLeaderCache( ((PipeRawTabletInsertionEvent) event).getDeviceId(), status.getRedirectNode()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java index 21f7c144bed22..a8b4a3b7a79a7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java @@ -36,18 +36,18 @@ public abstract class PipeTransferTrackableHandler implements AsyncMethodCallback, AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTransferTsFileHandler.class); - protected final IoTDBDataRegionAsyncSink connector; + protected final IoTDBDataRegionAsyncSink sink; protected volatile AsyncPipeDataTransferServiceClient client; - public PipeTransferTrackableHandler(final IoTDBDataRegionAsyncSink connector) { - this.connector = connector; + public PipeTransferTrackableHandler(final IoTDBDataRegionAsyncSink sink) { + this.sink = sink; } @Override public void onComplete(final TPipeTransferResp response) { - if (connector.isClosed()) { + if (sink.isClosed()) { clearEventsReferenceCount(); - connector.eliminateHandler(this, true); + sink.eliminateHandler(this, true); return; } @@ -56,7 +56,7 @@ public void onComplete(final TPipeTransferResp response) { // completed // NOTE: We should not clear the reference count of events, as this would cause the // `org.apache.iotdb.pipe.it.dual.tablemodel.manual.basic.IoTDBPipeDataSinkIT#testSinkTsFileFormat3` test to fail. - connector.eliminateHandler(this, false); + sink.eliminateHandler(this, false); } } @@ -67,14 +67,14 @@ public void onError(final Exception exception) { client.setPrintLogWhenEncounterException(false); } - if (connector.isClosed()) { + if (sink.isClosed()) { clearEventsReferenceCount(); - connector.eliminateHandler(this, true); + sink.eliminateHandler(this, true); return; } onErrorInternal(exception); - connector.eliminateHandler(this, false); + sink.eliminateHandler(this, false); } /** @@ -93,10 +93,10 @@ protected boolean tryTransfer( this.client = client; } // track handler before checking if connector is closed - connector.trackHandler(this); - if (connector.isClosed()) { + sink.trackHandler(this); + if (sink.isClosed()) { clearEventsReferenceCount(); - connector.eliminateHandler(this, true); + sink.eliminateHandler(this, true); client.setShouldReturnSelf(true); client.returnSelf( (e) -> { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java index a9426ed7b8b3b..6742199ac1043 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java @@ -159,7 +159,7 @@ public void transfer( if (client == null) { LOGGER.warn( "Client has been returned to the pool. Current handler status is {}. Will not transfer {}.", - connector.isClosed() ? "CLOSED" : "NOT CLOSED", + sink.isClosed() ? "CLOSED" : "NOT CLOSED", tsFile); return; } @@ -168,7 +168,7 @@ public void transfer( client.setTimeoutDynamically(clientManager.getConnectionTimeout()); PipeResourceMetrics.getInstance().recordDiskIO(readFileBufferSize); - if (connector.isEnableSendTsFileLimit()) { + if (sink.isEnableSendTsFileLimit()) { TsFileSendRateLimiter.getInstance().acquire(readFileBufferSize); } final int readLength = reader.read(readBuffer); @@ -192,11 +192,11 @@ public void transfer( ? PipeTransferTsFileSealWithModReq.toTPipeTransferReq( modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length()) : PipeTransferTsFileSealReq.toTPipeTransferReq(tsFile.getName(), tsFile.length()); - final TPipeTransferReq req = connector.compressIfNeeded(uncompressedReq); + final TPipeTransferReq req = sink.compressIfNeeded(uncompressedReq); pipeName2WeightMap.forEach( (pipePair, weight) -> - connector.rateLimitIfNeeded( + sink.rateLimitIfNeeded( pipePair.getLeft(), pipePair.getRight(), client.getEndPoint(), @@ -219,11 +219,11 @@ public void transfer( currentFile.getName(), position, payload) : PipeTransferTsFilePieceReq.toTPipeTransferReq( currentFile.getName(), position, payload); - final TPipeTransferReq req = connector.compressIfNeeded(uncompressedReq); + final TPipeTransferReq req = sink.compressIfNeeded(uncompressedReq); pipeName2WeightMap.forEach( (pipePair, weight) -> - connector.rateLimitIfNeeded( + sink.rateLimitIfNeeded( pipePair.getLeft(), pipePair.getRight(), client.getEndPoint(), @@ -241,7 +241,7 @@ public void onComplete(final TPipeTransferResp response) { try { super.onComplete(response); } finally { - if (connector.isClosed()) { + if (sink.isClosed()) { returnClientIfNecessary(); } } @@ -255,8 +255,7 @@ protected boolean onCompleteInternal(final TPipeTransferResp response) { // Only handle the failed statuses to avoid string format performance overhead if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { - connector - .statusHandler() + sink.statusHandler() .handle( status, String.format( @@ -330,9 +329,7 @@ protected boolean onCompleteInternal(final TPipeTransferResp response) { // Only handle the failed statuses to avoid string format performance overhead if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() && status.getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { - connector - .statusHandler() - .handle(status, response.getStatus().getMessage(), tsFile.getName()); + sink.statusHandler().handle(status, response.getStatus().getMessage(), tsFile.getName()); } } @@ -404,7 +401,7 @@ protected void onErrorInternal(final Exception exception) { returnClientIfNecessary(); } finally { if (eventsHadBeenAddedToRetryQueue.compareAndSet(false, true)) { - connector.addFailureEventsToRetryQueue(events, exception); + sink.addFailureEventsToRetryQueue(events, exception); } } } @@ -415,7 +412,7 @@ private void returnClientIfNecessary() { return; } - if (connector.isClosed()) { + if (sink.isClosed()) { closeClient(); } @@ -439,7 +436,7 @@ protected void doTransfer( if (client == null) { LOGGER.warn( "Client has been returned to the pool. Current handler status is {}. Will not transfer {}.", - connector.isClosed() ? "CLOSED" : "NOT CLOSED", + sink.isClosed() ? "CLOSED" : "NOT CLOSED", tsFile); return; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java index 52ac137ae4e6f..2019eba85603b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java @@ -35,13 +35,13 @@ import java.util.function.Consumer; -import static org.apache.iotdb.commons.concurrent.ThreadName.PIPE_EXTRACTOR_DISRUPTOR; +import static org.apache.iotdb.commons.concurrent.ThreadName.PIPE_SOURCE_DISRUPTOR; public class DisruptorQueue { private static final Logger LOGGER = LoggerFactory.getLogger(DisruptorQueue.class); private static final IoTDBDaemonThreadFactory THREAD_FACTORY = - new IoTDBDaemonThreadFactory(PIPE_EXTRACTOR_DISRUPTOR.getName()); + new IoTDBDaemonThreadFactory(PIPE_SOURCE_DISRUPTOR.getName()); private final PipeMemoryBlock allocatedMemoryBlock; private final Disruptor disruptor; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index aaa98220178bb..882d4aff0d8eb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -41,34 +41,34 @@ * *

All events extracted by this listener will be first published to different * PipeEventDataRegionAssigners (identified by data region id), and then PipeEventDataRegionAssigner - * will filter events and assign them to different PipeRealtimeEventDataRegionExtractors. + * will filter events and assign them to different PipeRealtimeEventDataRegionSources. */ public class PipeInsertionDataNodeListener { private final ConcurrentMap dataRegionId2Assigner = new ConcurrentHashMap<>(); - private final AtomicInteger listenToTsFileExtractorCount = new AtomicInteger(0); - private final AtomicInteger listenToInsertNodeExtractorCount = new AtomicInteger(0); + private final AtomicInteger listenToTsFileSourceCount = new AtomicInteger(0); + private final AtomicInteger listenToInsertNodeSourceCount = new AtomicInteger(0); //////////////////////////// start & stop //////////////////////////// public synchronized void startListenAndAssign( - String dataRegionId, PipeRealtimeDataRegionSource extractor) { + final String dataRegionId, final PipeRealtimeDataRegionSource source) { dataRegionId2Assigner .computeIfAbsent(dataRegionId, o -> new PipeDataRegionAssigner(dataRegionId)) - .startAssignTo(extractor); + .startAssignTo(source); - if (extractor.isNeedListenToTsFile()) { - listenToTsFileExtractorCount.incrementAndGet(); + if (source.isNeedListenToTsFile()) { + listenToTsFileSourceCount.incrementAndGet(); } - if (extractor.isNeedListenToInsertNode()) { - listenToInsertNodeExtractorCount.incrementAndGet(); + if (source.isNeedListenToInsertNode()) { + listenToInsertNodeSourceCount.incrementAndGet(); } } public synchronized void stopListenAndAssign( - final String dataRegionId, final PipeRealtimeDataRegionSource extractor) { + final String dataRegionId, final PipeRealtimeDataRegionSource source) { PipeDataRegionAssigner assignerToClose = null; synchronized (this) { @@ -77,13 +77,13 @@ public synchronized void stopListenAndAssign( return; } - assigner.stopAssignTo(extractor); + assigner.stopAssignTo(source); - if (extractor.isNeedListenToTsFile()) { - listenToTsFileExtractorCount.decrementAndGet(); + if (source.isNeedListenToTsFile()) { + listenToTsFileSourceCount.decrementAndGet(); } - if (extractor.isNeedListenToInsertNode()) { - listenToInsertNodeExtractorCount.decrementAndGet(); + if (source.isNeedListenToInsertNode()) { + listenToInsertNodeSourceCount.decrementAndGet(); } if (assigner.notMoreSourceNeededToBeAssigned()) { @@ -104,8 +104,8 @@ public synchronized void stopListenAndAssign( public void listenToTsFile( final String dataRegionId, final TsFileResource tsFileResource, final boolean isLoaded) { - // We don't judge whether listenToTsFileExtractorCount.get() == 0 here on purpose - // because extractors may use tsfile events when some exceptions occur in the + // We don't judge whether listenToTsFileSourceCount.get() == 0 here on purpose + // because sources may use tsfile events when some exceptions occur in the // insert nodes listening process. final PipeDataRegionAssigner assigner = dataRegionId2Assigner.get(dataRegionId); @@ -120,8 +120,8 @@ public void listenToTsFile( } public void listenToInsertNode( - String dataRegionId, InsertNode insertNode, TsFileResource tsFileResource) { - if (listenToInsertNodeExtractorCount.get() == 0) { + final String dataRegionId, final InsertNode insertNode, final TsFileResource tsFileResource) { + if (listenToInsertNodeSourceCount.get() == 0) { return; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java index a366aed71b3e1..faacc10eccded 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.write; import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.PartialPath; @@ -37,6 +38,7 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.iotdb.db.utils.QueryDataSetUtils; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.exception.NotImplementedException; @@ -1076,68 +1078,105 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitInsertTablet(this, context); } - public TimeValuePair composeLastTimeValuePair(int measurementIndex) { + public TimeValuePair composeLastTimeValuePair( + int measurementIndex, int startOffset, int endOffset) { if (measurementIndex >= columns.length || Objects.isNull(dataTypes[measurementIndex])) { return null; } // get non-null value - int lastIdx = rowCount - 1; + int lastIdx = Math.min(endOffset - 1, rowCount - 1); if (bitMaps != null && bitMaps[measurementIndex] != null) { BitMap bitMap = bitMaps[measurementIndex]; - while (lastIdx >= 0) { + while (lastIdx >= startOffset) { if (!bitMap.isMarked(lastIdx)) { break; } lastIdx--; } } - if (lastIdx < 0) { + if (lastIdx < startOffset) { return null; } + return composeTimeValuePair(measurementIndex, lastIdx); + } + protected TimeValuePair composeLastTimeValuePair( + final int measurementIndex, + final TSStatus[] results, + final int startOffset, + final int endOffset) { + if (results == null) { + return composeLastTimeValuePair(measurementIndex, startOffset, endOffset); + } + if (measurementIndex >= columns.length || Objects.isNull(dataTypes[measurementIndex])) { + return null; + } + + final BitMap bitMap = bitMaps == null ? null : bitMaps[measurementIndex]; + int lastIdx = Math.min(endOffset - 1, rowCount - 1); + while (lastIdx >= startOffset) { + if (results[lastIdx] != null + && results[lastIdx].getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + lastIdx--; + continue; + } + if (bitMap != null && bitMap.isMarked(lastIdx)) { + lastIdx--; + continue; + } + break; + } + return lastIdx < startOffset ? null : composeTimeValuePair(measurementIndex, lastIdx); + } + + private TimeValuePair composeTimeValuePair(final int measurementIndex, final int rowIndex) { TsPrimitiveType value; switch (dataTypes[measurementIndex]) { case INT32: case DATE: int[] intValues = (int[]) columns[measurementIndex]; - value = new TsPrimitiveType.TsInt(intValues[lastIdx]); + value = new TsPrimitiveType.TsInt(intValues[rowIndex]); break; case INT64: case TIMESTAMP: long[] longValues = (long[]) columns[measurementIndex]; - value = new TsPrimitiveType.TsLong(longValues[lastIdx]); + value = new TsPrimitiveType.TsLong(longValues[rowIndex]); break; case FLOAT: float[] floatValues = (float[]) columns[measurementIndex]; - value = new TsPrimitiveType.TsFloat(floatValues[lastIdx]); + value = new TsPrimitiveType.TsFloat(floatValues[rowIndex]); break; case DOUBLE: double[] doubleValues = (double[]) columns[measurementIndex]; - value = new TsPrimitiveType.TsDouble(doubleValues[lastIdx]); + value = new TsPrimitiveType.TsDouble(doubleValues[rowIndex]); break; case BOOLEAN: boolean[] boolValues = (boolean[]) columns[measurementIndex]; - value = new TsPrimitiveType.TsBoolean(boolValues[lastIdx]); + value = new TsPrimitiveType.TsBoolean(boolValues[rowIndex]); break; case TEXT: case BLOB: case STRING: Binary[] binaryValues = (Binary[]) columns[measurementIndex]; - value = new TsPrimitiveType.TsBinary(binaryValues[lastIdx]); + value = new TsPrimitiveType.TsBinary(binaryValues[rowIndex]); break; default: throw new UnSupportedDataTypeException( String.format(DATATYPE_UNSUPPORTED, dataTypes[measurementIndex])); } - return new TimeValuePair(times[lastIdx], value); + return new TimeValuePair(times[rowIndex], value); } public void updateLastCache(final String databaseName) { + updateLastCache(databaseName, null); + } + + public void updateLastCache(final String databaseName, final TSStatus[] results) { final String[] rawMeasurements = getRawMeasurements(); final TimeValuePair[] timeValuePairs = new TimeValuePair[rawMeasurements.length]; for (int i = 0; i < rawMeasurements.length; i++) { - timeValuePairs[i] = composeLastTimeValuePair(i); + timeValuePairs[i] = composeLastTimeValuePair(i, results, 0, rowCount); } DataNodeSchemaCache.getInstance() .updateLastCacheIfExists( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 3cad57962e51e..d06362ffd3794 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -1193,7 +1193,7 @@ public void insertTablet(InsertTabletNode insertTabletNode) if (!insertTabletNode.isGeneratedByRemoteConsensusLeader()) { // disable updating last cache on follower startTime = System.nanoTime(); - tryToUpdateInsertTabletLastCache(insertTabletNode); + tryToUpdateInsertTabletLastCache(insertTabletNode, results); PERFORMANCE_OVERVIEW_METRICS.recordScheduleUpdateLastCacheCost( System.nanoTime() - startTime); } @@ -1261,13 +1261,13 @@ private boolean insertTabletToTsFileProcessor( return true; } - TsFileProcessor tsFileProcessor = getOrCreateTsFileProcessor(timePartitionId, sequence); - if (tsFileProcessor == null) { + final TsFileProcessor tsFileProcessor; + try { + tsFileProcessor = getOrCreateTsFileProcessor(timePartitionId, sequence); + } catch (WriteProcessException e) { + final TSStatus failureStatus = RpcUtils.getStatus(e.getErrorCode(), e.getMessage()); for (int i = start; i < end; i++) { - results[i] = - RpcUtils.getStatus( - TSStatusCode.INTERNAL_SERVER_ERROR, - "can not create TsFileProcessor, timePartitionId: " + timePartitionId); + results[i] = failureStatus; } return false; } @@ -1289,8 +1289,9 @@ private boolean insertTabletToTsFileProcessor( return true; } - private void tryToUpdateInsertTabletLastCache(InsertTabletNode node) { - node.updateLastCache(getDatabaseName()); + private void tryToUpdateInsertTabletLastCache( + final InsertTabletNode node, final TSStatus[] results) { + node.updateLastCache(getDatabaseName(), results); } private TsFileProcessor insertToTsFileProcessor( @@ -1300,9 +1301,6 @@ private TsFileProcessor insertToTsFileProcessor( return null; } TsFileProcessor tsFileProcessor = getOrCreateTsFileProcessor(timePartitionId, sequence); - if (tsFileProcessor == null) { - return null; - } long[] costsForMetrics = new long[4]; tsFileProcessor.insert(insertRowNode, costsForMetrics); PERFORMANCE_OVERVIEW_METRICS.recordCreateMemtableBlockCost(costsForMetrics[0]); @@ -1325,9 +1323,11 @@ private List insertToTsFileProcessors( if (insertRowNode.allMeasurementFailed()) { continue; } - TsFileProcessor tsFileProcessor = - getOrCreateTsFileProcessor(timePartitionIds[i], areSequence[i]); - if (tsFileProcessor == null) { + final TsFileProcessor tsFileProcessor; + try { + tsFileProcessor = getOrCreateTsFileProcessor(timePartitionIds[i], areSequence[i]); + } catch (WriteProcessException e) { + insertRowsNode.getResults().put(i, RpcUtils.getStatus(e.getErrorCode(), e.getMessage())); continue; } int finalI = i; @@ -1335,43 +1335,28 @@ private List insertToTsFileProcessors( tsFileProcessor, (k, v) -> { if (v == null) { - v = new InsertRowsNode(insertRowsNode.getPlanNodeId()); - v.setSearchIndex(insertRowNode.getSearchIndex()); - v.setAligned(insertRowNode.isAligned()); - if (insertRowNode.isGeneratedByPipe()) { - v.markAsGeneratedByPipe(); - } - if (insertRowNode.isGeneratedByRemoteConsensusLeader()) { - v.markAsGeneratedByRemoteConsensusLeader(); - } + v = createGroupedInsertRowsNode(insertRowsNode, insertRowNode); } - if (v.isAligned() != insertRowNode.isAligned()) { - v.setMixingAlignment(true); - } - v.addOneInsertRowNode(insertRowNode, finalI); - v.updateProgressIndex(insertRowNode.getProgressIndex()); + appendInsertRowNode(v, insertRowNode, finalI); return v; }); } List executedInsertRowNodeList = new ArrayList<>(); for (Map.Entry entry : tsFileProcessorMap.entrySet()) { - TsFileProcessor tsFileProcessor = entry.getKey(); InsertRowsNode subInsertRowsNode = entry.getValue(); try { - tsFileProcessor.insert(subInsertRowsNode, costsForMetrics); + List insertedProcessors = + insertRowsWithTypeConsistencyCheck(entry.getKey(), subInsertRowsNode, costsForMetrics); + executedInsertRowNodeList.addAll(subInsertRowsNode.getInsertRowNodeList()); + for (TsFileProcessor tsFileProcessor : insertedProcessors) { + // check memtable size and may asyncTryToFlush the work memtable + if (tsFileProcessor.shouldFlush()) { + fileFlushPolicy.apply(this, tsFileProcessor, tsFileProcessor.isSequence()); + } + } } catch (WriteProcessException e) { - insertRowsNode - .getResults() - .put( - subInsertRowsNode.getInsertRowNodeIndexList().get(0), - RpcUtils.getStatus(e.getErrorCode(), e.getMessage())); - } - executedInsertRowNodeList.addAll(subInsertRowsNode.getInsertRowNodeList()); - - // check memtable size and may asyncTryToFlush the work memtable - if (entry.getKey().shouldFlush()) { - fileFlushPolicy.apply(this, tsFileProcessor, tsFileProcessor.isSequence()); + recordInsertRowsFailure(insertRowsNode, subInsertRowsNode, e); } } @@ -1382,6 +1367,121 @@ private List insertToTsFileProcessors( return executedInsertRowNodeList; } + private List insertRowsWithTypeConsistencyCheck( + TsFileProcessor tsFileProcessor, InsertRowsNode subInsertRowsNode, long[] costsForMetrics) + throws WriteProcessException { + tsFileProcessor.insert(subInsertRowsNode, costsForMetrics); + return Collections.singletonList(tsFileProcessor); + } + + private InsertRowsNode createGroupedInsertRowsNode( + final InsertRowsNode sourceInsertRowsNode, final InsertRowNode firstInsertRowNode) { + final InsertRowsNode groupedInsertRowsNode = + new InsertRowsNode(sourceInsertRowsNode.getPlanNodeId()); + initializeGroupedInsertRowsNode(groupedInsertRowsNode, firstInsertRowNode); + return groupedInsertRowsNode; + } + + private InsertRowsNode createGroupedInsertRowsNode( + final InsertRowsOfOneDeviceNode sourceInsertRowsNode, + final InsertRowNode firstInsertRowNode) { + final InsertRowsNode groupedInsertRowsNode = + new InsertRowsNode(sourceInsertRowsNode.getPlanNodeId()); + initializeGroupedInsertRowsNode(groupedInsertRowsNode, firstInsertRowNode); + return groupedInsertRowsNode; + } + + private void initializeGroupedInsertRowsNode( + final InsertRowsNode groupedInsertRowsNode, final InsertRowNode firstInsertRowNode) { + groupedInsertRowsNode.setSearchIndex(firstInsertRowNode.getSearchIndex()); + groupedInsertRowsNode.setAligned(firstInsertRowNode.isAligned()); + if (firstInsertRowNode.isGeneratedByPipe()) { + groupedInsertRowsNode.markAsGeneratedByPipe(); + } + if (firstInsertRowNode.isGeneratedByRemoteConsensusLeader()) { + groupedInsertRowsNode.markAsGeneratedByRemoteConsensusLeader(); + } + } + + private void appendInsertRowNode( + final InsertRowsNode groupedInsertRowsNode, + final InsertRowNode insertRowNode, + final int insertRowNodeIndex) { + if (groupedInsertRowsNode.isAligned() != insertRowNode.isAligned()) { + groupedInsertRowsNode.setMixingAlignment(true); + } + groupedInsertRowsNode.addOneInsertRowNode(insertRowNode, insertRowNodeIndex); + groupedInsertRowsNode.updateProgressIndex(insertRowNode.getProgressIndex()); + } + + private void flushWorkingProcessorsForTimePartition(final long timePartitionId) { + TsFileProcessor workSequenceProcessor = workSequenceTsFileProcessors.get(timePartitionId); + if (workSequenceProcessor != null) { + fileFlushPolicy.apply(this, workSequenceProcessor, workSequenceProcessor.isSequence()); + } + TsFileProcessor workUnsequenceProcessor = workUnsequenceTsFileProcessors.get(timePartitionId); + if (workUnsequenceProcessor != null) { + fileFlushPolicy.apply(this, workUnsequenceProcessor, workUnsequenceProcessor.isSequence()); + } + } + + private List retryInsertRowsAfterFlush( + final InsertRowsNode subInsertRowsNode, + final long timePartitionId, + final long[] costsForMetrics) + throws WriteProcessException { + final Map retriedProcessorMap = new HashMap<>(); + for (int i = 0; i < subInsertRowsNode.getInsertRowNodeList().size(); i++) { + final InsertRowNode insertRowNode = subInsertRowsNode.getInsertRowNodeList().get(i); + final boolean isSequence = + config.isEnableSeparateData() + && insertRowNode.getTime() + > lastFlushTimeMap.getFlushedTime(timePartitionId, insertRowNode.getDeviceID()); + final TsFileProcessor retriedProcessor = + getOrCreateTsFileProcessor(timePartitionId, isSequence); + final int insertRowNodeIndex = subInsertRowsNode.getInsertRowNodeIndexList().get(i); + retriedProcessorMap.compute( + retriedProcessor, + (k, v) -> { + if (v == null) { + v = createGroupedInsertRowsNode(subInsertRowsNode, insertRowNode); + } + appendInsertRowNode(v, insertRowNode, insertRowNodeIndex); + return v; + }); + } + + final List insertedProcessors = new ArrayList<>(retriedProcessorMap.size()); + for (Entry retriedEntry : retriedProcessorMap.entrySet()) { + final TsFileProcessor retriedProcessor = retriedEntry.getKey(); + retriedProcessor.insert(retriedEntry.getValue(), costsForMetrics); + insertedProcessors.add(retriedProcessor); + } + return insertedProcessors; + } + + private void recordInsertRowsFailure( + final InsertRowsNode sourceInsertRowsNode, + final InsertRowsNode failedInsertRowsNode, + final WriteProcessException exception) { + final TSStatus failureStatus = + RpcUtils.getStatus(exception.getErrorCode(), exception.getMessage()); + for (Integer failedIndex : failedInsertRowsNode.getInsertRowNodeIndexList()) { + sourceInsertRowsNode.getResults().put(failedIndex, failureStatus); + } + } + + private void recordInsertRowsFailure( + final InsertRowsOfOneDeviceNode sourceInsertRowsNode, + final InsertRowsNode failedInsertRowsNode, + final WriteProcessException exception) { + final TSStatus failureStatus = + RpcUtils.getStatus(exception.getErrorCode(), exception.getMessage()); + for (Integer failedIndex : failedInsertRowsNode.getInsertRowNodeIndexList()) { + sourceInsertRowsNode.getResults().put(failedIndex, failureStatus); + } + } + private void tryToUpdateInsertRowsLastCache(List nodeList) { for (InsertRowNode node : nodeList) { node.updateLastCache(databaseName); @@ -1440,7 +1540,8 @@ public void submitAFlushTaskWhenShouldFlush(TsFileProcessor tsFileProcessor) { } } - private TsFileProcessor getOrCreateTsFileProcessor(long timeRangeId, boolean sequence) { + protected TsFileProcessor getOrCreateTsFileProcessor(long timeRangeId, boolean sequence) + throws WriteProcessException { TsFileProcessor tsFileProcessor = null; int retryCnt = 0; do { @@ -1466,7 +1567,7 @@ private TsFileProcessor getOrCreateTsFileProcessor(long timeRangeId, boolean seq "disk space is insufficient when creating TsFile processor, change system mode to read-only", e); CommonDescriptor.getInstance().getConfig().setNodeStatus(NodeStatus.ReadOnly); - break; + throw new WriteProcessException(e.getMessage(), e.getErrorCode(), true); } catch (IOException e) { if (retryCnt < 3) { logger.warn("meet IOException when creating TsFileProcessor, retry it again", e); @@ -1475,11 +1576,15 @@ private TsFileProcessor getOrCreateTsFileProcessor(long timeRangeId, boolean seq logger.error( "meet IOException when creating TsFileProcessor, change system mode to error", e); CommonDescriptor.getInstance().getConfig().handleUnrecoverableError(); - break; + throw new WriteProcessException( + String.format( + "Failed to create TsFileProcessor for database %s, timePartitionId %s", + databaseName, timeRangeId), + e); } } catch (ExceedQuotaException e) { logger.error(e.getMessage()); - break; + throw new WriteProcessException(e.getMessage(), e.getErrorCode(), true); } } while (tsFileProcessor == null); return tsFileProcessor; @@ -3663,8 +3768,13 @@ public void insert(InsertRowsOfOneDeviceNode insertRowsOfOneDeviceNode) config.isEnableSeparateData() && insertRowNode.getTime() > lastFlushTimeMap.getFlushedTime(timePartitionId, insertRowNode.getDeviceID()); - TsFileProcessor tsFileProcessor = getOrCreateTsFileProcessor(timePartitionId, isSequence); - if (tsFileProcessor == null) { + final TsFileProcessor tsFileProcessor; + try { + tsFileProcessor = getOrCreateTsFileProcessor(timePartitionId, isSequence); + } catch (WriteProcessException e) { + insertRowsOfOneDeviceNode + .getResults() + .put(i, RpcUtils.getStatus(e.getErrorCode(), e.getMessage())); continue; } int finalI = i; @@ -3672,39 +3782,28 @@ public void insert(InsertRowsOfOneDeviceNode insertRowsOfOneDeviceNode) tsFileProcessor, (k, v) -> { if (v == null) { - v = new InsertRowsNode(insertRowsOfOneDeviceNode.getPlanNodeId()); - v.setSearchIndex(insertRowNode.getSearchIndex()); - v.setAligned(insertRowNode.isAligned()); - if (insertRowNode.isGeneratedByPipe()) { - v.markAsGeneratedByPipe(); - } - if (insertRowNode.isGeneratedByRemoteConsensusLeader()) { - v.markAsGeneratedByRemoteConsensusLeader(); - } + v = createGroupedInsertRowsNode(insertRowsOfOneDeviceNode, insertRowNode); } - v.addOneInsertRowNode(insertRowNode, finalI); - v.updateProgressIndex(insertRowNode.getProgressIndex()); + appendInsertRowNode(v, insertRowNode, finalI); return v; }); } List executedInsertRowNodeList = new ArrayList<>(); for (Map.Entry entry : tsFileProcessorMap.entrySet()) { - TsFileProcessor tsFileProcessor = entry.getKey(); InsertRowsNode subInsertRowsNode = entry.getValue(); try { - tsFileProcessor.insert(subInsertRowsNode, costsForMetrics); + List insertedProcessors = + insertRowsWithTypeConsistencyCheck( + entry.getKey(), subInsertRowsNode, costsForMetrics); + executedInsertRowNodeList.addAll(subInsertRowsNode.getInsertRowNodeList()); + for (TsFileProcessor tsFileProcessor : insertedProcessors) { + // check memtable size and may asyncTryToFlush the work memtable + if (tsFileProcessor.shouldFlush()) { + fileFlushPolicy.apply(this, tsFileProcessor, tsFileProcessor.isSequence()); + } + } } catch (WriteProcessException e) { - insertRowsOfOneDeviceNode - .getResults() - .put( - subInsertRowsNode.getInsertRowNodeIndexList().get(0), - RpcUtils.getStatus(e.getErrorCode(), e.getMessage())); - } - executedInsertRowNodeList.addAll(subInsertRowsNode.getInsertRowNodeList()); - - // check memtable size and may asyncTryToFlush the work memtable - if (tsFileProcessor.shouldFlush()) { - fileFlushPolicy.apply(this, tsFileProcessor, tsFileProcessor.isSequence()); + recordInsertRowsFailure(insertRowsOfOneDeviceNode, subInsertRowsNode, e); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskSinkStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskSinkStage.java index 73fca57a1fd69..45c6d6f86cfc8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskSinkStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskSinkStage.java @@ -41,7 +41,7 @@ public SubscriptionTaskSinkStage( @Override protected void registerSubtask() { - this.connectorSubtaskId = + this.sinkSubtaskId = SubscriptionSinkSubtaskManager.instance() .register( executor.get(), @@ -56,22 +56,21 @@ public void createSubtask() throws PipeException { @Override public void startSubtask() throws PipeException { - SubscriptionSinkSubtaskManager.instance().start(connectorSubtaskId); + SubscriptionSinkSubtaskManager.instance().start(sinkSubtaskId); } @Override public void stopSubtask() throws PipeException { - SubscriptionSinkSubtaskManager.instance().stop(connectorSubtaskId); + SubscriptionSinkSubtaskManager.instance().stop(sinkSubtaskId); } @Override public void dropSubtask() throws PipeException { SubscriptionSinkSubtaskManager.instance() - .deregister(pipeName, creationTime, regionId, connectorSubtaskId); + .deregister(pipeName, creationTime, regionId, sinkSubtaskId); } public UnboundedBlockingPendingQueue getPipeSinkPendingQueue() { - return SubscriptionSinkSubtaskManager.instance() - .getPipeConnectorPendingQueue(connectorSubtaskId); + return SubscriptionSinkSubtaskManager.instance().getPipeConnectorPendingQueue(sinkSubtaskId); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java index 5082b8fc4f09a..f7a79eb8bf8e5 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.storageengine.dataregion; +import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.consensus.DataRegionId; @@ -27,8 +28,10 @@ import org.apache.iotdb.commons.exception.ShutdownException; import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.BatchProcessException; import org.apache.iotdb.db.exception.DataRegionException; import org.apache.iotdb.db.exception.TsFileProcessorException; import org.apache.iotdb.db.exception.WriteProcessException; @@ -36,6 +39,7 @@ import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; +import org.apache.iotdb.db.queryengine.plan.analyze.cache.schema.DataNodeSchemaCache; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; @@ -62,6 +66,8 @@ import org.apache.iotdb.db.storageengine.rescon.memory.SystemInfo; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.db.utils.constant.TestConstant; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.PlainDeviceID; @@ -78,6 +84,7 @@ import org.junit.Before; import org.junit.Ignore; import org.junit.Test; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,6 +95,9 @@ import java.util.Collections; import java.util.List; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; + public class DataRegionTest { private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); private static final CommonConfig COMMON_CONFIG = CommonDescriptor.getInstance().getConfig(); @@ -126,6 +136,7 @@ public void tearDown() throws Exception { dataRegion.syncDeleteDataFiles(); StorageEngine.getInstance().deleteDataRegion(new DataRegionId(0)); } + DataNodeSchemaCache.getInstance().cleanUp(); EnvironmentUtils.cleanDir(TestConstant.OUTPUT_DATA_DIR); CompactionTaskManager.getInstance().stop(); EnvironmentUtils.cleanEnv(); @@ -975,6 +986,189 @@ public void testInsertUnSequenceRows() dataRegion1.syncDeleteDataFiles(); } + @Test + public void testInsertRowPropagatesTsFileProcessorCreationFailure() + throws IllegalPathException, DataRegionException, TsFileProcessorException { + final HookedDataRegion dataRegion1 = new HookedDataRegion(systemDir, "root.fail_row"); + dataRegion1.setTsFileProcessorSupplier( + (timePartitionId, sequence) -> { + throw new WriteProcessRejectException("mock creation failure"); + }); + + final TSRecord record = new TSRecord(1, "root.fail_row"); + record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(1))); + final InsertRowNode insertRowNode = buildInsertRowNodeByTSRecord(record); + + try { + dataRegion1.insert(insertRowNode); + Assert.fail("Expected WriteProcessRejectException"); + } catch (WriteProcessRejectException e) { + Assert.assertTrue(e.getMessage().contains("mock creation failure")); + } catch (WriteProcessException e) { + Assert.fail("Expected WriteProcessRejectException but got " + e.getClass().getSimpleName()); + } finally { + dataRegion1.syncDeleteDataFiles(); + } + } + + @Test + public void testInsertRowsMarkAllFailedRowsForSameProcessor() throws Exception { + final HookedDataRegion dataRegion1 = new HookedDataRegion(systemDir, "root.fail_rows"); + final TsFileProcessor processor = Mockito.mock(TsFileProcessor.class); + Mockito.doThrow(new WriteProcessException("mock insert rows failure")) + .when(processor) + .insert(any(InsertRowsNode.class), any(long[].class)); + Mockito.when(processor.shouldFlush()).thenReturn(false); + Mockito.when(processor.isSequence()).thenReturn(true); + dataRegion1.setTsFileProcessorSupplier((timePartitionId, sequence) -> processor); + + final List indexList = Arrays.asList(0, 1); + final List nodes = new ArrayList<>(); + for (long time : new long[] {1, 2}) { + final TSRecord record = new TSRecord(time, "root.fail_rows"); + record.addTuple( + DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(time))); + nodes.add(buildInsertRowNodeByTSRecord(record)); + } + final InsertRowsNode insertRowsNode = new InsertRowsNode(new PlanNodeId(""), indexList, nodes); + + try { + dataRegion1.insert(insertRowsNode); + Assert.fail("Expected BatchProcessException"); + } catch (BatchProcessException e) { + Assert.assertEquals(2, insertRowsNode.getResults().size()); + Assert.assertEquals( + TSStatusCode.WRITE_PROCESS_ERROR.getStatusCode(), + insertRowsNode.getResults().get(0).getCode()); + Assert.assertEquals( + TSStatusCode.WRITE_PROCESS_ERROR.getStatusCode(), + insertRowsNode.getResults().get(1).getCode()); + } finally { + dataRegion1.syncDeleteDataFiles(); + } + } + + @Test + public void testInsertRowsLastCacheSkipsFailedRows() throws Exception { + final boolean originalLastCacheEnable = COMMON_CONFIG.isLastCacheEnable(); + COMMON_CONFIG.setLastCacheEnable(true); + + final HookedDataRegion dataRegion1 = new HookedDataRegion(systemDir, "root.cache_rows"); + final TsFileProcessor successProcessor = Mockito.mock(TsFileProcessor.class); + Mockito.when(successProcessor.shouldFlush()).thenReturn(false); + Mockito.when(successProcessor.isSequence()).thenReturn(true); + final long failingTime = TimePartitionUtils.getTimePartitionInterval() + 1; + final long failingPartitionId = TimePartitionUtils.getTimePartitionId(failingTime); + dataRegion1.setTsFileProcessorSupplier( + (timePartitionId, sequence) -> { + if (timePartitionId == failingPartitionId) { + throw new WriteProcessException("mock row failure"); + } + return successProcessor; + }); + + final MeasurementPath lastCachePath = + new MeasurementPath( + "root.cache_rows", + measurementId, + new MeasurementSchema( + measurementId, TSDataType.INT32, TSEncoding.PLAIN, CompressionType.UNCOMPRESSED)); + DataNodeSchemaCache.getInstance() + .declareLastCache(dataRegion1.getDatabaseName(), lastCachePath); + + final List indexList = Arrays.asList(0, 1); + final List nodes = new ArrayList<>(); + final long[] times = new long[] {1, failingTime}; + final int[] values = new int[] {10, 20}; + for (int i = 0; i < times.length; i++) { + final long time = times[i]; + final TSRecord record = new TSRecord(time, "root.cache_rows"); + record.addTuple( + DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(values[i]))); + nodes.add(buildInsertRowNodeByTSRecord(record)); + } + final InsertRowsNode insertRowsNode = new InsertRowsNode(new PlanNodeId(""), indexList, nodes); + + try { + dataRegion1.insert(insertRowsNode); + Assert.fail("Expected BatchProcessException"); + } catch (BatchProcessException e) { + final TimeValuePair lastCache = DataNodeSchemaCache.getInstance().getLastCache(lastCachePath); + Assert.assertNotNull(lastCache); + Assert.assertEquals(1, lastCache.getTimestamp()); + Assert.assertEquals(10, lastCache.getValue().getInt()); + } finally { + dataRegion1.syncDeleteDataFiles(); + COMMON_CONFIG.setLastCacheEnable(originalLastCacheEnable); + } + } + + @Test + public void testInsertTabletLastCacheSkipsFailedRows() throws Exception { + final boolean originalLastCacheEnable = COMMON_CONFIG.isLastCacheEnable(); + COMMON_CONFIG.setLastCacheEnable(true); + + final HookedDataRegion dataRegion1 = new HookedDataRegion(systemDir, "root.cache_tablet"); + final TsFileProcessor processor = Mockito.mock(TsFileProcessor.class); + Mockito.doAnswer( + invocation -> { + TSStatus[] results = invocation.getArgument(3); + results[0] = RpcUtils.SUCCESS_STATUS; + results[1] = + RpcUtils.getStatus( + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode(), "mock row failure"); + throw new WriteProcessException("mock tablet failure"); + }) + .when(processor) + .insertTablet(any(InsertTabletNode.class), anyInt(), anyInt(), any(TSStatus[].class)); + Mockito.when(processor.shouldFlush()).thenReturn(false); + Mockito.when(processor.isSequence()).thenReturn(true); + dataRegion1.setTsFileProcessorSupplier((timePartitionId, sequence) -> processor); + + final MeasurementPath lastCachePath = + new MeasurementPath( + "root.cache_tablet", + measurementId, + new MeasurementSchema( + measurementId, TSDataType.INT32, TSEncoding.PLAIN, CompressionType.UNCOMPRESSED)); + DataNodeSchemaCache.getInstance() + .declareLastCache(dataRegion1.getDatabaseName(), lastCachePath); + + final String[] measurements = new String[] {measurementId}; + final TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32}; + final MeasurementSchema[] measurementSchemas = + new MeasurementSchema[] { + new MeasurementSchema(measurementId, TSDataType.INT32, TSEncoding.PLAIN) + }; + final long[] times = new long[] {1, 2}; + final Object[] columns = new Object[] {new int[] {10, 20}}; + final InsertTabletNode insertTabletNode = + new InsertTabletNode( + new QueryId("test_write").genPlanNodeId(), + new PartialPath("root.cache_tablet"), + false, + measurements, + dataTypes, + measurementSchemas, + times, + null, + columns, + times.length); + + try { + dataRegion1.insertTablet(insertTabletNode); + Assert.fail("Expected BatchProcessException"); + } catch (BatchProcessException e) { + final TimeValuePair lastCache = DataNodeSchemaCache.getInstance().getLastCache(lastCachePath); + Assert.assertNotNull(lastCache); + Assert.assertEquals(1, lastCache.getTimestamp()); + Assert.assertEquals(10, lastCache.getValue().getInt()); + } finally { + dataRegion1.syncDeleteDataFiles(); + COMMON_CONFIG.setLastCacheEnable(originalLastCacheEnable); + } + } + @Test public void testSmallReportProportionInsertRow() throws WriteProcessException, @@ -1490,6 +1684,32 @@ public DummyDataRegion(String systemInfoDir, String storageGroupName) } } + private interface TsFileProcessorSupplier { + TsFileProcessor get(long timePartitionId, boolean sequence) throws WriteProcessException; + } + + private static class HookedDataRegion extends DummyDataRegion { + private TsFileProcessorSupplier tsFileProcessorSupplier; + + private HookedDataRegion(String systemInfoDir, String storageGroupName) + throws DataRegionException { + super(systemInfoDir, storageGroupName); + } + + private void setTsFileProcessorSupplier(TsFileProcessorSupplier tsFileProcessorSupplier) { + this.tsFileProcessorSupplier = tsFileProcessorSupplier; + } + + @Override + protected TsFileProcessor getOrCreateTsFileProcessor(long timeRangeId, boolean sequence) + throws WriteProcessException { + if (tsFileProcessorSupplier != null) { + return tsFileProcessorSupplier.get(timeRangeId, sequence); + } + return super.getOrCreateTsFileProcessor(timeRangeId, sequence); + } + } + // -- test for deleting data directly // -- delete data and file only when: // 1. tsfile is closed diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java index a70a6aa911d93..0a6b1da048ac3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java @@ -323,7 +323,7 @@ public GenericKeyedObjectPool cre .setRpcThriftCompressionEnabled(conf.isPipeSinkRPCThriftCompressionEnabled()) .setSelectorNumOfAsyncClientManager(conf.getPipeAsyncSinkSelectorNumber()) .build(), - ThreadName.PIPE_ASYNC_CONNECTOR_CLIENT_POOL.getName()), + ThreadName.PIPE_ASYNC_SINK_CLIENT_POOL.getName()), new ClientPoolProperty.Builder() .setMaxClientNumForEachNode(conf.getPipeAsyncSinkMaxClientNumber()) .build() @@ -349,7 +349,7 @@ public GenericKeyedObjectPool cre .setSelectorNumOfAsyncClientManager(conf.getPipeAsyncSinkSelectorNumber()) .setPrintLogWhenEncounterException(conf.isPrintLogWhenEncounterException()) .build(), - ThreadName.PIPE_ASYNC_CONNECTOR_CLIENT_POOL.getName()), + ThreadName.PIPE_ASYNC_SINK_CLIENT_POOL.getName()), new ClientPoolProperty.Builder() .setMaxClientNumForEachNode(conf.getPipeAsyncSinkMaxTsFileClientNumber()) .build() diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index 4f5ad140e9919..a902893094056 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -128,10 +128,10 @@ public enum ThreadName { GPRC_DEFAULT_WORKER_ELG("grpc-default-worker-ELG"), GROUP_MANAGEMENT("groupManagement"), // -------------------------- Compute -------------------------- - PIPE_EXTRACTOR_DISRUPTOR("Pipe-Extractor-Disruptor"), + PIPE_SOURCE_DISRUPTOR("Pipe-Source-Disruptor"), PIPE_PROCESSOR_EXECUTOR_POOL("Pipe-Processor-Executor-Pool"), - PIPE_CONNECTOR_EXECUTOR_POOL("Pipe-Connector-Executor-Pool"), PIPE_CONSENSUS_EXECUTOR_POOL("Pipe-Consensus-Executor-Pool"), + PIPE_SINK_EXECUTOR_POOL("Pipe-Sink-Executor-Pool"), PIPE_CONFIGNODE_EXECUTOR_POOL("Pipe-ConfigNode-Executor-Pool"), PIPE_SUBTASK_CALLBACK_EXECUTOR_POOL("Pipe-SubTask-Callback-Executor-Pool"), PIPE_TSFILE_ASYNC_SEND_POOL("Pipe-TsFile-Async-Send-Pool"), @@ -141,7 +141,7 @@ public enum ThreadName { PIPE_RUNTIME_PERIODICAL_JOB_EXECUTOR("Pipe-Runtime-Periodical-Job-Executor"), PIPE_RUNTIME_PERIODICAL_PHANTOM_REFERENCE_CLEANER( "Pipe-Runtime-Periodical-Phantom-Reference-Cleaner"), - PIPE_ASYNC_CONNECTOR_CLIENT_POOL("Pipe-Async-Connector-Client-Pool"), + PIPE_ASYNC_SINK_CLIENT_POOL("Pipe-Async-Sink-Client-Pool"), PIPE_RECEIVER_AIR_GAP_AGENT("Pipe-Receiver-Air-Gap-Agent"), PIPE_AIR_GAP_RECEIVER("Pipe-Air-Gap-Receiver"), PIPE_PARALLEL_EXECUTION_POOL("Pipe-Parallel-Execution-Pool"), @@ -287,9 +287,9 @@ public enum ThreadName { private static final Set computeThreadNames = new HashSet<>( Arrays.asList( - PIPE_EXTRACTOR_DISRUPTOR, + PIPE_SOURCE_DISRUPTOR, PIPE_PROCESSOR_EXECUTOR_POOL, - PIPE_CONNECTOR_EXECUTOR_POOL, + PIPE_SINK_EXECUTOR_POOL, PIPE_CONSENSUS_EXECUTOR_POOL, PIPE_CONFIGNODE_EXECUTOR_POOL, PIPE_SUBTASK_CALLBACK_EXECUTOR_POOL, @@ -298,7 +298,7 @@ public enum ThreadName { PIPE_RUNTIME_PROCEDURE_SUBMITTER, PIPE_RUNTIME_PERIODICAL_JOB_EXECUTOR, PIPE_RUNTIME_PERIODICAL_PHANTOM_REFERENCE_CLEANER, - PIPE_ASYNC_CONNECTOR_CLIENT_POOL, + PIPE_ASYNC_SINK_CLIENT_POOL, PIPE_RECEIVER_AIR_GAP_AGENT, PIPE_AIR_GAP_RECEIVER, PIPE_PARALLEL_EXECUTION_POOL, diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index e1f52ba65ff6d..37929ed18603f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -282,11 +282,11 @@ public class CommonConfig { private int pipeAsyncSinkForcedRetryTabletEventQueueSize = 20; private int pipeAsyncSinkForcedRetryTotalEventQueueSize = 30; private long pipeAsyncSinkMaxRetryExecutionTimeMsPerCall = 500; - private int pipeAsyncConnectorSelectorNumber = + private int pipeAsyncSinkSelectorNumber = Math.max(4, Runtime.getRuntime().availableProcessors() / 2); - private int pipeAsyncConnectorMaxClientNumber = + private int pipeAsyncSinkMaxClientNumber = Math.max(32, Runtime.getRuntime().availableProcessors() * 2); - private int pipeAsyncConnectorMaxTsFileClientNumber = + private int pipeAsyncSinkMaxTsFileClientNumber = Math.max(16, Runtime.getRuntime().availableProcessors()); private boolean printLogWhenEncounterException = false; @@ -294,8 +294,7 @@ public class CommonConfig { private double pipeAllSinksRateLimitBytesPerSecond = -1; private int rateLimiterHotReloadCheckIntervalMs = 1000; - private int pipeConnectorRequestSliceThresholdBytes = - (int) (RpcUtils.THRIFT_FRAME_MAX_SIZE * 0.8); + private int pipeSinkRequestSliceThresholdBytes = (int) (RpcUtils.THRIFT_FRAME_MAX_SIZE * 0.8); private boolean isSeperatedPipeHeartbeatEnabled = true; private int pipeHeartbeatIntervalSecondsForCollectingPipeMeta = 3; @@ -1015,7 +1014,7 @@ public int getPipeSinkHandshakeTimeoutMs() { } public void setPipeSinkHandshakeTimeoutMs(long pipeSinkHandshakeTimeoutMs) { - final int fPipeConnectorHandshakeTimeoutMs = this.pipeSinkHandshakeTimeoutMs; + final int fPipeSinkHandshakeTimeoutMs = this.pipeSinkHandshakeTimeoutMs; try { this.pipeSinkHandshakeTimeoutMs = Math.toIntExact(pipeSinkHandshakeTimeoutMs); } catch (ArithmeticException e) { @@ -1023,7 +1022,7 @@ public void setPipeSinkHandshakeTimeoutMs(long pipeSinkHandshakeTimeoutMs) { logger.warn( "Given pipe connector handshake timeout is too large, set to {} ms.", Integer.MAX_VALUE); } finally { - if (fPipeConnectorHandshakeTimeoutMs != this.pipeSinkHandshakeTimeoutMs) { + if (fPipeSinkHandshakeTimeoutMs != this.pipeSinkHandshakeTimeoutMs) { logger.info("pipeSinkHandshakeTimeoutMs is set to {}.", this.pipeSinkHandshakeTimeoutMs); } } @@ -1054,16 +1053,16 @@ public int getPipeSinkTransferTimeoutMs() { } public void setPipeSinkTransferTimeoutMs(long pipeSinkTransferTimeoutMs) { - final int fPipeConnectorTransferTimeoutMs = this.pipeSinkTransferTimeoutMs; + final int fPipeSinkTransferTimeoutMs = this.pipeSinkTransferTimeoutMs; try { this.pipeSinkTransferTimeoutMs = Math.toIntExact(pipeSinkTransferTimeoutMs); } catch (ArithmeticException e) { this.pipeSinkTransferTimeoutMs = Integer.MAX_VALUE; logger.warn( - "Given pipe connector transfer timeout is too large, set to {} ms.", Integer.MAX_VALUE); + "Given pipe sink transfer timeout is too large, set to {} ms.", Integer.MAX_VALUE); } finally { - if (fPipeConnectorTransferTimeoutMs != this.pipeSinkTransferTimeoutMs) { - logger.info("pipeConnectorTransferTimeoutMs is set to {}.", pipeSinkTransferTimeoutMs); + if (fPipeSinkTransferTimeoutMs != this.pipeSinkTransferTimeoutMs) { + logger.info("pipeSinkTransferTimeoutMs is set to {}.", pipeSinkTransferTimeoutMs); } } } @@ -1077,7 +1076,7 @@ public void setPipeSinkReadFileBufferSize(int pipeSinkReadFileBufferSize) { return; } this.pipeSinkReadFileBufferSize = pipeSinkReadFileBufferSize; - logger.info("pipeConnectorReadFileBufferSize is set to {}.", pipeSinkReadFileBufferSize); + logger.info("pipeSinkReadFileBufferSize is set to {}.", pipeSinkReadFileBufferSize); } public boolean isPipeSinkReadFileBufferMemoryControlEnabled() { @@ -1177,60 +1176,58 @@ public long getPipeAsyncSinkMaxRetryExecutionTimeMsPerCall() { } public int getPipeAsyncSinkSelectorNumber() { - return pipeAsyncConnectorSelectorNumber; + return pipeAsyncSinkSelectorNumber; } - public void setPipeAsyncConnectorSelectorNumber(int pipeAsyncConnectorSelectorNumber) { - if (pipeAsyncConnectorSelectorNumber <= 0) { + public void setPipeAsyncSinkSelectorNumber(int pipeAsyncSinkSelectorNumber) { + if (pipeAsyncSinkSelectorNumber <= 0) { logger.info( - "pipeAsyncConnectorSelectorNumber should be greater than 0, configuring it not to change."); + "pipeAsyncSinkSelectorNumber should be greater than 0, configuring it not to change."); return; } - pipeAsyncConnectorSelectorNumber = Math.max(4, pipeAsyncConnectorSelectorNumber); - if (this.pipeAsyncConnectorSelectorNumber == pipeAsyncConnectorSelectorNumber) { + pipeAsyncSinkSelectorNumber = Math.max(4, pipeAsyncSinkSelectorNumber); + if (this.pipeAsyncSinkSelectorNumber == pipeAsyncSinkSelectorNumber) { return; } - this.pipeAsyncConnectorSelectorNumber = pipeAsyncConnectorSelectorNumber; - logger.info("pipeAsyncConnectorSelectorNumber is set to {}.", pipeAsyncConnectorSelectorNumber); + this.pipeAsyncSinkSelectorNumber = pipeAsyncSinkSelectorNumber; + logger.info("pipeAsyncSinkSelectorNumber is set to {}.", pipeAsyncSinkSelectorNumber); } public int getPipeAsyncSinkMaxClientNumber() { - return pipeAsyncConnectorMaxClientNumber; + return pipeAsyncSinkMaxClientNumber; } - public void setPipeAsyncConnectorMaxClientNumber(int pipeAsyncConnectorMaxClientNumber) { - if (pipeAsyncConnectorMaxClientNumber <= 0) { + public void setPipeAsyncSinkMaxClientNumber(int pipeAsyncSinkMaxClientNumber) { + if (pipeAsyncSinkMaxClientNumber <= 0) { logger.info( - " pipeAsyncConnectorMaxClientNumber should be greater than 0, configuring it not to change."); + " pipeAsyncSinkMaxClientNumber should be greater than 0, configuring it not to change."); return; } - pipeAsyncConnectorMaxClientNumber = Math.max(32, pipeAsyncConnectorMaxClientNumber); - if (this.pipeAsyncConnectorMaxClientNumber == pipeAsyncConnectorMaxClientNumber) { + pipeAsyncSinkMaxClientNumber = Math.max(32, pipeAsyncSinkMaxClientNumber); + if (this.pipeAsyncSinkMaxClientNumber == pipeAsyncSinkMaxClientNumber) { return; } - this.pipeAsyncConnectorMaxClientNumber = pipeAsyncConnectorMaxClientNumber; - logger.info( - "pipeAsyncConnectorMaxClientNumber is set to {}.", pipeAsyncConnectorMaxClientNumber); + this.pipeAsyncSinkMaxClientNumber = pipeAsyncSinkMaxClientNumber; + logger.info("pipeAsyncSinkMaxClientNumber is set to {}.", pipeAsyncSinkMaxClientNumber); } public int getPipeAsyncSinkMaxTsFileClientNumber() { - return pipeAsyncConnectorMaxTsFileClientNumber; + return pipeAsyncSinkMaxTsFileClientNumber; } - public void setPipeAsyncConnectorMaxTsFileClientNumber( - int pipeAsyncConnectorMaxTsFileClientNumber) { - if (pipeAsyncConnectorMaxTsFileClientNumber <= 0) { + public void setPipeAsyncSinkMaxTsFileClientNumber(int pipeAsyncSinkMaxTsFileClientNumber) { + if (pipeAsyncSinkMaxTsFileClientNumber <= 0) { logger.info( - "pipeAsyncConnectorMaxTsFileClientNumber should be greater than 0, configuring it not to change."); + "pipeAsyncSinkMaxTsFileClientNumber should be greater than 0, configuring it not to change."); return; } - pipeAsyncConnectorMaxTsFileClientNumber = Math.max(16, pipeAsyncConnectorMaxTsFileClientNumber); - if (this.pipeAsyncConnectorMaxTsFileClientNumber == pipeAsyncConnectorMaxTsFileClientNumber) { + pipeAsyncSinkMaxTsFileClientNumber = Math.max(16, pipeAsyncSinkMaxTsFileClientNumber); + if (this.pipeAsyncSinkMaxTsFileClientNumber == pipeAsyncSinkMaxTsFileClientNumber) { return; } - this.pipeAsyncConnectorMaxTsFileClientNumber = pipeAsyncConnectorMaxTsFileClientNumber; + this.pipeAsyncSinkMaxTsFileClientNumber = pipeAsyncSinkMaxTsFileClientNumber; logger.info( - "pipeAsyncConnectorMaxClientNumber is set to {}.", pipeAsyncConnectorMaxTsFileClientNumber); + "pipeAsyncSinkMaxTsFileClientNumber is set to {}.", pipeAsyncSinkMaxTsFileClientNumber); } public boolean isPrintLogWhenEncounterException() { @@ -1334,12 +1331,12 @@ public long getPipeSinkRetryIntervalMs() { return pipeSinkRetryIntervalMs; } - public void setPipeSinkRetryIntervalMs(long pipeConnectorRetryIntervalMs) { - if (this.pipeSinkRetryIntervalMs == pipeConnectorRetryIntervalMs) { + public void setPipeSinkRetryIntervalMs(long pipeSinkRetryIntervalMs) { + if (this.pipeSinkRetryIntervalMs == pipeSinkRetryIntervalMs) { return; } - this.pipeSinkRetryIntervalMs = pipeConnectorRetryIntervalMs; - logger.info("pipeSinkRetryIntervalMs is set to {}", pipeConnectorRetryIntervalMs); + this.pipeSinkRetryIntervalMs = pipeSinkRetryIntervalMs; + logger.info("pipeSinkRetryIntervalMs is set to {}", pipeSinkRetryIntervalMs); } public boolean isPipeSinkRetryLocallyForConnectionError() { @@ -2140,18 +2137,16 @@ public void setRateLimiterHotReloadCheckIntervalMs(int rateLimiterHotReloadCheck } public int getPipeSinkRequestSliceThresholdBytes() { - return pipeConnectorRequestSliceThresholdBytes; + return pipeSinkRequestSliceThresholdBytes; } - public void setPipeConnectorRequestSliceThresholdBytes( - int pipeConnectorRequestSliceThresholdBytes) { - if (this.pipeConnectorRequestSliceThresholdBytes == pipeConnectorRequestSliceThresholdBytes) { + public void setPipeSinkRequestSliceThresholdBytes(int pipeSinkRequestSliceThresholdBytes) { + if (this.pipeSinkRequestSliceThresholdBytes == pipeSinkRequestSliceThresholdBytes) { return; } - this.pipeConnectorRequestSliceThresholdBytes = pipeConnectorRequestSliceThresholdBytes; + this.pipeSinkRequestSliceThresholdBytes = pipeSinkRequestSliceThresholdBytes; logger.info( - "pipeConnectorRequestSliceThresholdBytes is set to {}", - pipeConnectorRequestSliceThresholdBytes); + "pipeConnectorRequestSliceThresholdBytes is set to {}", pipeSinkRequestSliceThresholdBytes); } public long getTwoStageAggregateMaxCombinerLiveTimeInMs() { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index 02284671803d5..83db711e5ec63 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -443,7 +443,7 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr "rate_limiter_hot_reload_check_interval_ms", String.valueOf(config.getRateLimiterHotReloadCheckIntervalMs())))); - config.setPipeConnectorRequestSliceThresholdBytes( + config.setPipeSinkRequestSliceThresholdBytes( Integer.parseInt( properties.getProperty( "pipe_connector_request_slice_threshold_bytes", @@ -613,7 +613,7 @@ public static void loadPipeExternalConfig( "pipe_async_connector_selector_number", isHotModify); if (value != null) { - config.setPipeAsyncConnectorSelectorNumber(Integer.parseInt(value)); + config.setPipeAsyncSinkSelectorNumber(Integer.parseInt(value)); } value = @@ -623,7 +623,7 @@ public static void loadPipeExternalConfig( "pipe_async_connector_max_client_number", isHotModify); if (value != null) { - config.setPipeAsyncConnectorMaxClientNumber(Integer.parseInt(value)); + config.setPipeAsyncSinkMaxClientNumber(Integer.parseInt(value)); } value = @@ -633,7 +633,7 @@ public static void loadPipeExternalConfig( "pipe_async_connector_max_tsfile_client_number", isHotModify); if (value != null) { - config.setPipeAsyncConnectorMaxTsFileClientNumber(Integer.parseInt(value)); + config.setPipeAsyncSinkMaxTsFileClientNumber(Integer.parseInt(value)); } value = From 6a1961b0f3625b855231e0be18c47a32e60bcc51 Mon Sep 17 00:00:00 2001 From: Jiang Tian Date: Tue, 12 May 2026 09:38:46 +0800 Subject: [PATCH 042/102] [To dev/1.3] Fix that WALBuffer waits for flush instead of file-roll (#17628) (#17633) * Fix that WALBuffer waits for flush instead of file-roll (#17628) (cherry picked from commit cfbcc249c53cf5ec090c19ed69bb59a05d7efb07) * fix test compilation --- .../dataregion/wal/buffer/IWALBuffer.java | 6 +- .../dataregion/wal/buffer/WALBuffer.java | 18 +- .../dataregion/wal/node/WALNode.java | 10 +- .../wal/node/WALNodeWaitForRollFileTest.java | 406 ++++++++++++++++++ 4 files changed, 427 insertions(+), 13 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNodeWaitForRollFileTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/IWALBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/IWALBuffer.java index 978ea78116268..082dcca8c05bb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/IWALBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/IWALBuffer.java @@ -51,7 +51,7 @@ public interface IWALBuffer extends AutoCloseable { * * @throws InterruptedException when interrupted by the flush thread */ - void waitForFlush() throws InterruptedException; + void waitForRollFile() throws InterruptedException; /** * Wait for next flush operation done, if the predicate == true after entering a locked @@ -60,14 +60,14 @@ public interface IWALBuffer extends AutoCloseable { * @param waitPredicate the condition which should be satisfied before waiting. * @throws InterruptedException when interrupted by the flush thread */ - public void waitForFlush(Predicate waitPredicate) throws InterruptedException; + public void waitForRollFile(Predicate waitPredicate) throws InterruptedException; /** * Wait for next flush operation done. * * @throws InterruptedException when interrupted by the flush thread */ - boolean waitForFlush(long time, TimeUnit unit) throws InterruptedException; + boolean waitForRollFile(long time, TimeUnit unit) throws InterruptedException; /** Return true when all wal entries all consumed and flushed. */ boolean isAllWALEntriesConsumed(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java index 36c404faa7c2a..08d1ad384bbb7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java @@ -88,6 +88,7 @@ public class WALBuffer extends AbstractWALBuffer { private final Lock buffersLock = new ReentrantLock(); // condition to guarantee correctness of switching buffers private final Condition idleBufferReadyCondition = buffersLock.newCondition(); + private final Condition rollLogWriterCondition = buffersLock.newCondition(); // last writer position when fsync is called, help record each entry's position private long lastFsyncPosition; @@ -168,6 +169,13 @@ private int getCompressedByteBufferSize(int size) { protected File rollLogWriter(long searchIndex, WALFileStatus fileStatus) throws IOException { File file = super.rollLogWriter(searchIndex, fileStatus); currentWALFileWriter.setCompressedByteBuffer(compressedByteBuffer); + buffersLock.lock(); + try { + // notify WALReader that new file is generated, and it can read new file + rollLogWriterCondition.signalAll(); + } finally { + buffersLock.unlock(); + } return file; } @@ -640,7 +648,7 @@ private void switchSyncingBufferToIdle() { } @Override - public void waitForFlush() throws InterruptedException { + public void waitForRollFile() throws InterruptedException { buffersLock.lock(); try { idleBufferReadyCondition.await(); @@ -650,11 +658,11 @@ public void waitForFlush() throws InterruptedException { } @Override - public void waitForFlush(Predicate waitPredicate) throws InterruptedException { + public void waitForRollFile(Predicate waitPredicate) throws InterruptedException { buffersLock.lock(); try { if (waitPredicate.test(this)) { - idleBufferReadyCondition.await(); + rollLogWriterCondition.await(); } } finally { buffersLock.unlock(); @@ -662,10 +670,10 @@ public void waitForFlush(Predicate waitPredicate) throws InterruptedE } @Override - public boolean waitForFlush(long time, TimeUnit unit) throws InterruptedException { + public boolean waitForRollFile(long time, TimeUnit unit) throws InterruptedException { buffersLock.lock(); try { - return idleBufferReadyCondition.await(time, unit); + return rollLogWriterCondition.await(time, unit); } finally { buffersLock.unlock(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java index 003c74763da77..b0e19364ddb21 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java @@ -92,7 +92,7 @@ public class WALNode implements IWALNode { // no iot consensus, all insert nodes can be safely deleted public static final long DEFAULT_SAFELY_DELETED_SEARCH_INDEX = Long.MAX_VALUE; // timeout threshold when waiting for next wal entry - private static final long WAIT_FOR_NEXT_WAL_ENTRY_TIMEOUT_IN_SEC = 30; + public static final long WAIT_FOR_NEXT_WAL_ENTRY_TIMEOUT_IN_SEC = 30; private static final WritingMetrics WRITING_METRICS = WritingMetrics.getInstance(); // unique identifier of this WALNode @@ -733,7 +733,7 @@ public void waitForNextReady() throws InterruptedException { while (!hasNext()) { if (!walFileRolled) { boolean timeout = - !buffer.waitForFlush(WAIT_FOR_NEXT_WAL_ENTRY_TIMEOUT_IN_SEC, TimeUnit.SECONDS); + !buffer.waitForRollFile(WAIT_FOR_NEXT_WAL_ENTRY_TIMEOUT_IN_SEC, TimeUnit.SECONDS); if (timeout) { bufferLastSearchIndex = buffer.getCurrentSearchIndex(); logger.info( @@ -746,7 +746,7 @@ public void waitForNextReady() throws InterruptedException { } else { // only wait when the search index of the buffer remains the same as the previous check long finalBufferLastSearchIndex = bufferLastSearchIndex; - buffer.waitForFlush(buf -> buf.getCurrentSearchIndex() == finalBufferLastSearchIndex); + buffer.waitForRollFile(buf -> buf.getCurrentSearchIndex() == finalBufferLastSearchIndex); } } } @@ -755,8 +755,8 @@ public void waitForNextReady() throws InterruptedException { public void waitForNextReady(long time, TimeUnit unit) throws InterruptedException, TimeoutException { if (!hasNext()) { - boolean timeout = !buffer.waitForFlush(time, unit); - if (timeout || !hasNext()) { + boolean timeout = !buffer.waitForRollFile(time, unit); + if (timeout && !hasNext()) { throw new TimeoutException(); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNodeWaitForRollFileTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNodeWaitForRollFileTest.java new file mode 100644 index 0000000000000..b75efaefca4fa --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNodeWaitForRollFileTest.java @@ -0,0 +1,406 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iotdb.db.storageengine.dataregion.wal.node; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.consensus.ConsensusFactory; +import org.apache.iotdb.consensus.iot.log.ConsensusReqReader; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.ContinuousSameSearchIndexSeparatorNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; +import org.apache.iotdb.db.storageengine.dataregion.memtable.PrimitiveMemTable; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALMode; +import org.apache.iotdb.db.utils.EnvironmentUtils; +import org.apache.iotdb.db.utils.constant.TestConstant; + +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.awaitility.Awaitility; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class WALNodeWaitForRollFileTest { + private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + private static final String identifier = String.valueOf(Integer.MAX_VALUE); + private static final String logDirectory = + TestConstant.BASE_OUTPUT_PATH.concat("wal-roll-file-test"); + private static final String databasePath = "root.test_sg"; + private static final String devicePath = databasePath + ".test_d"; + private static final String dataRegionId = "1"; + private WALMode prevMode; + private String prevConsensus; + private WALNode walNode; + private long originWALThreshold; + + @Before + public void setUp() throws Exception { + originWALThreshold = config.getWalFileSizeThresholdInByte(); + EnvironmentUtils.cleanDir(logDirectory); + prevMode = config.getWalMode(); + prevConsensus = config.getDataRegionConsensusProtocolClass(); + config.setWalMode(WALMode.SYNC); + config.setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS); + config.setWalFileSizeThresholdInByte(2 * 1024 * 1024); + walNode = new WALNode(identifier, logDirectory); + } + + @After + public void tearDown() throws Exception { + walNode.close(); + config.setWalMode(prevMode); + config.setDataRegionConsensusProtocolClass(prevConsensus); + config.setWalFileSizeThresholdInByte(originWALThreshold); + EnvironmentUtils.cleanDir(logDirectory); + StorageEngine.getInstance().reset(); + } + + /** + * Verifies that waitForNextReady(time, unit) throws TimeoutException when no WAL data is + * available at the requested search index. This uses waitForRollFile internally. + */ + @Test + public void testWaitForNextReadyTimesOutWhenNoData() throws Exception { + ConsensusReqReader.ReqIterator iterator = walNode.getReqIterator(1); + assertFalse(iterator.hasNext()); + try { + iterator.waitForNextReady(1, TimeUnit.SECONDS); + fail("Expected TimeoutException"); + } catch (TimeoutException e) { + // expected + } + } + + /** + * Verifies that waitForNextReady(time, unit) does NOT wake up from a buffer flush alone — it + * requires a WAL file roll. This is the core behavioral change: the old waitForFlush would return + * on any buffer sync, but waitForRollFile only returns when a new WAL file is created. + */ + @Test + public void testWaitForNextReadyNotWokenByFlushWithoutRoll() throws Exception { + IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); + walNode.onMemTableCreated(memTable, logDirectory + File.separator + "test.tsfile"); + + // write a small amount of data (not enough to trigger roll) + InsertTabletNode insertTabletNode = getInsertTabletNode(devicePath, new long[] {1}); + insertTabletNode.setSearchIndex(1); + walNode.log(memTable.getMemTableId(), insertTabletNode, 0, insertTabletNode.getRowCount()); + + Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> walNode.isAllWALEntriesConsumed()); + + // data is flushed to buffer but no WAL file roll happened yet, iterator at search index 1 + // should not find data (because the current-writing WAL file is not readable by the iterator) + ConsensusReqReader.ReqIterator iterator = walNode.getReqIterator(1); + + try { + long start = System.currentTimeMillis(); + iterator.waitForNextReady(2, TimeUnit.SECONDS); + if (System.currentTimeMillis() - start + < WALNode.WAIT_FOR_NEXT_WAL_ENTRY_TIMEOUT_IN_SEC * 1000) { + fail("The data should not be found before timeout"); + } + } catch (TimeoutException e) { + // expected: flush happened but no roll, so waitForRollFile timed out + } + } + + /** + * Verifies that waitForNextReady succeeds after a WAL file roll makes data readable. The iterator + * should wake up when rollLogWriter signals the rollLogWriterCondition. + */ + @Test + public void testWaitForNextReadySucceedsAfterRollFile() throws Exception { + IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); + walNode.onMemTableCreated(memTable, logDirectory + File.separator + "test.tsfile"); + + // write data with search index + for (int i = 1; i <= 5; i++) { + InsertTabletNode insertTabletNode = getInsertTabletNode(devicePath, new long[] {i}); + insertTabletNode.setSearchIndex(i); + walNode.log(memTable.getMemTableId(), insertTabletNode, 0, insertTabletNode.getRowCount()); + } + + Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> walNode.isAllWALEntriesConsumed()); + + // roll the WAL file so the data is in a closed file readable by the iterator + walNode.rollWALFile(); + Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> walNode.isAllWALEntriesConsumed()); + + // iterator at search index 1 should find the data after roll + ConsensusReqReader.ReqIterator iterator = walNode.getReqIterator(1); + assertTrue(iterator.hasNext()); + assertNotNull(iterator.next()); + } + + /** + * Verifies that waitForNextReady wakes up when a WAL file roll is triggered concurrently. A + * background thread rolls the WAL file while the main thread waits on the iterator. + */ + @Test(timeout = 30000) + public void testWaitForNextReadyWakesUpOnConcurrentRoll() throws Exception { + IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); + walNode.onMemTableCreated(memTable, logDirectory + File.separator + "test.tsfile"); + + // write data with search index + InsertTabletNode insertTabletNode = getInsertTabletNode(devicePath, new long[] {1}); + insertTabletNode.setSearchIndex(1); + walNode.log(memTable.getMemTableId(), insertTabletNode, 0, insertTabletNode.getRowCount()); + walNode.log( + memTable.getMemTableId(), new ContinuousSameSearchIndexSeparatorNode(new PlanNodeId(""))); + + Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> walNode.isAllWALEntriesConsumed()); + + ConsensusReqReader.ReqIterator iterator = walNode.getReqIterator(1); + + AtomicBoolean found = new AtomicBoolean(false); + AtomicReference error = new AtomicReference<>(); + ExecutorService executor = Executors.newSingleThreadExecutor(); + + // background: wait for data to become available via waitForNextReady + Future waitFuture = + executor.submit( + () -> { + try { + iterator.waitForNextReady(15, TimeUnit.SECONDS); + if (iterator.hasNext()) { + found.set(true); + } + } catch (Exception e) { + error.set(e); + } + }); + + // give the waiter thread time to start blocking + Thread.sleep(500); + + // trigger WAL file roll — this should signal rollLogWriterCondition and wake up the iterator + walNode.rollWALFile(); + Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> walNode.isAllWALEntriesConsumed()); + + waitFuture.get(20, TimeUnit.SECONDS); + executor.shutdown(); + + if (error.get() != null) { + throw error.get(); + } + assertTrue("Iterator should have found data after WAL file roll", found.get()); + } + + /** + * Verifies that the no-arg waitForNextReady eventually proceeds when enough data is written to + * trigger an automatic WAL file roll (file size exceeds threshold). Uses a small WAL file size + * threshold to trigger the roll quickly. + */ + @Test(timeout = 60000) + public void testWaitForNextReadyWithAutoRollOnSizeThreshold() throws Exception { + // use small WAL file size to trigger auto-roll + config.setWalFileSizeThresholdInByte(1024); + + try { + IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); + walNode.onMemTableCreated(memTable, logDirectory + File.separator + "test.tsfile"); + + // write initial data with search index + InsertTabletNode first = getInsertTabletNode(devicePath, new long[] {1}); + first.setSearchIndex(1); + walNode.log(memTable.getMemTableId(), first, 0, first.getRowCount()); + + Awaitility.await() + .atMost(10, TimeUnit.SECONDS) + .until(() -> walNode.isAllWALEntriesConsumed()); + + ConsensusReqReader.ReqIterator iterator = walNode.getReqIterator(1); + + AtomicBoolean found = new AtomicBoolean(false); + AtomicReference error = new AtomicReference<>(); + ExecutorService executor = Executors.newSingleThreadExecutor(); + + Future waitFuture = + executor.submit( + () -> { + try { + iterator.waitForNextReady(30, TimeUnit.SECONDS); + if (iterator.hasNext()) { + found.set(true); + } + } catch (Exception e) { + error.set(e); + } + }); + + Thread.sleep(500); + + // write more data to exceed the small threshold and trigger auto-roll + for (int i = 2; i <= 50; i++) { + InsertTabletNode node = getInsertTabletNode(devicePath, new long[] {i}); + node.setSearchIndex(i); + walNode.log(memTable.getMemTableId(), node, 0, node.getRowCount()); + } + + waitFuture.get(40, TimeUnit.SECONDS); + executor.shutdown(); + + if (error.get() != null) { + fail("waitForNextReady threw unexpected exception: " + error.get().getMessage()); + } + assertTrue("Iterator should have found data after auto WAL file roll", found.get()); + } finally { + config.setWalFileSizeThresholdInByte(2 * 1024 * 1024); + } + } + + /** + * Verifies that the no-arg waitForNextReady() automatically triggers a WAL file roll after the + * timeout expires (WAIT_FOR_NEXT_WAL_ENTRY_TIMEOUT_IN_SEC = 30s). The flow is: data written to + * buffer → waitForRollFile(30s) times out → rollWALFile() called → data moves to closed file → + * hasNext() returns true → method returns. + */ + @Test(timeout = 120000) + public void testWaitForNextReadyAutoTriggersRollOnTimeout() throws Exception { + IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); + walNode.onMemTableCreated(memTable, logDirectory + File.separator + "test.tsfile"); + + // write data with search index — stays in the current (active) WAL file + InsertTabletNode insertTabletNode = getInsertTabletNode(devicePath, new long[] {1}); + insertTabletNode.setSearchIndex(1); + walNode.log(memTable.getMemTableId(), insertTabletNode, 0, insertTabletNode.getRowCount()); + walNode.log( + memTable.getMemTableId(), new ContinuousSameSearchIndexSeparatorNode(new PlanNodeId(""))); + + Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> walNode.isAllWALEntriesConsumed()); + + // iterator cannot read the active WAL file, so hasNext() should be false + ConsensusReqReader.ReqIterator iterator = walNode.getReqIterator(1); + assertFalse("Data should not be visible before WAL file roll", iterator.hasNext()); + + AtomicBoolean found = new AtomicBoolean(false); + AtomicReference error = new AtomicReference<>(); + ExecutorService executor = Executors.newSingleThreadExecutor(); + + long startTime = System.currentTimeMillis(); + + // call the no-arg waitForNextReady() — it should: + // 1) wait 30s for rollLogWriterCondition (timeout) + // 2) auto-call rollWALFile() + // 3) data becomes readable, hasNext() returns true, method returns + Future waitFuture = + executor.submit( + () -> { + try { + iterator.waitForNextReady(); + if (iterator.hasNext()) { + found.set(true); + } + } catch (Exception e) { + error.set(e); + } + }); + + waitFuture.get(90, TimeUnit.SECONDS); + executor.shutdown(); + + long elapsed = System.currentTimeMillis() - startTime; + + if (error.get() != null) { + fail("waitForNextReady() threw unexpected exception: " + error.get().getMessage()); + } + assertTrue("Iterator should have found data after auto-triggered WAL file roll", found.get()); + assertTrue( + "Should have waited at least 30s for the timeout to trigger auto-roll, but only waited " + + elapsed + + "ms", + elapsed >= TimeUnit.SECONDS.toMillis(WALNode.WAIT_FOR_NEXT_WAL_ENTRY_TIMEOUT_IN_SEC - 1)); + } + + private InsertTabletNode getInsertTabletNode(String devicePath, long[] times) + throws IllegalPathException { + String[] measurements = new String[] {"s1", "s2", "s3", "s4", "s5", "s6"}; + TSDataType[] dataTypes = new TSDataType[6]; + dataTypes[0] = TSDataType.DOUBLE; + dataTypes[1] = TSDataType.FLOAT; + dataTypes[2] = TSDataType.INT64; + dataTypes[3] = TSDataType.INT32; + dataTypes[4] = TSDataType.BOOLEAN; + dataTypes[5] = TSDataType.TEXT; + + Object[] columns = new Object[6]; + columns[0] = new double[times.length]; + columns[1] = new float[times.length]; + columns[2] = new long[times.length]; + columns[3] = new int[times.length]; + columns[4] = new boolean[times.length]; + columns[5] = new Binary[times.length]; + + for (int r = 0; r < times.length; r++) { + ((double[]) columns[0])[r] = 1.0d + r; + ((float[]) columns[1])[r] = 2.0f + r; + ((long[]) columns[2])[r] = 10000L + r; + ((int[]) columns[3])[r] = 100 + r; + ((boolean[]) columns[4])[r] = (r % 2 == 0); + ((Binary[]) columns[5])[r] = new Binary("hh" + r, TSFileConfig.STRING_CHARSET); + } + + BitMap[] bitMaps = new BitMap[dataTypes.length]; + for (int i = 0; i < dataTypes.length; i++) { + if (bitMaps[i] == null) { + bitMaps[i] = new BitMap(times.length); + } + bitMaps[i].mark(i % times.length); + } + MeasurementSchema[] schemas = new MeasurementSchema[6]; + for (int i = 0; i < 6; i++) { + schemas[i] = new MeasurementSchema(measurements[i], dataTypes[i], TSEncoding.PLAIN); + } + + return new InsertTabletNode( + new PlanNodeId(""), + new PartialPath(devicePath), + false, + measurements, + dataTypes, + schemas, + times, + bitMaps, + columns, + times.length); + } +} From f3b83aafb1fe0cabc0a9de89f23e103b3abaa3f7 Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Tue, 12 May 2026 18:20:54 +0800 Subject: [PATCH 043/102] [to dev/1.3] Fix nested ColumnTransformer close --- .../dag/column/CaseWhenThenColumnTransformer.java | 10 ++++++++++ .../dag/column/binary/BinaryColumnTransformer.java | 7 +++++++ .../dag/column/multi/MappableUDFColumnTransformer.java | 4 ++++ .../dag/column/ternary/TernaryColumnTransformer.java | 8 ++++++++ .../dag/column/unary/UnaryColumnTransformer.java | 6 ++++++ 5 files changed, 35 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/CaseWhenThenColumnTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/CaseWhenThenColumnTransformer.java index 68910f2aa4b3a..94156d1c71b21 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/CaseWhenThenColumnTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/CaseWhenThenColumnTransformer.java @@ -128,4 +128,14 @@ protected void evaluate() { protected void checkType() { // do nothing } + + @Override + public void close() { + super.close(); + for (Pair whenThenTransformer : whenThenTransformers) { + whenThenTransformer.left.close(); + whenThenTransformer.right.close(); + } + elseTransformer.close(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/binary/BinaryColumnTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/binary/BinaryColumnTransformer.java index 6e074abe08f4a..996c6763afd52 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/binary/BinaryColumnTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/binary/BinaryColumnTransformer.java @@ -63,4 +63,11 @@ public ColumnTransformer getLeftTransformer() { public ColumnTransformer getRightTransformer() { return rightTransformer; } + + @Override + public void close() { + super.close(); + this.leftTransformer.close(); + this.rightTransformer.close(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/multi/MappableUDFColumnTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/multi/MappableUDFColumnTransformer.java index b3562a2920884..74f15ab7055c4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/multi/MappableUDFColumnTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/multi/MappableUDFColumnTransformer.java @@ -70,6 +70,10 @@ public ColumnTransformer[] getInputColumnTransformers() { @Override public void close() { + super.close(); + for (ColumnTransformer inputColumnTransformer : inputColumnTransformers) { + inputColumnTransformer.close(); + } // finalize executor executor.beforeDestroy(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/ternary/TernaryColumnTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/ternary/TernaryColumnTransformer.java index 6d747d3dae835..1638d273bffb0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/ternary/TernaryColumnTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/ternary/TernaryColumnTransformer.java @@ -54,4 +54,12 @@ public ColumnTransformer getSecondColumnTransformer() { public ColumnTransformer getThirdColumnTransformer() { return thirdColumnTransformer; } + + @Override + public void close() { + super.close(); + firstColumnTransformer.close(); + secondColumnTransformer.close(); + thirdColumnTransformer.close(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/UnaryColumnTransformer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/UnaryColumnTransformer.java index 0f1f48cfee852..e29ea0bf56f73 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/UnaryColumnTransformer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/transformation/dag/column/unary/UnaryColumnTransformer.java @@ -53,4 +53,10 @@ protected void checkType() { } protected abstract void doTransform(Column column, ColumnBuilder columnBuilder); + + @Override + public void close() { + super.close(); + childColumnTransformer.close(); + } } From 96754ec2fb2dfd3b069ce2d87ef85bb1f48d011d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 13 May 2026 10:16:37 +0800 Subject: [PATCH 044/102] Pipe: Optimized the thread executor of the terminate event (#17638) (#17647) --- .../common/terminate/PipeTerminateEvent.java | 31 +++++++++++++------ 1 file changed, 21 insertions(+), 10 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java index a8bf19ebf3b17..4bf79a3df030c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java @@ -48,17 +48,28 @@ public class PipeTerminateEvent extends EnrichedEvent { private final boolean shouldMark; + private static final int TERMINATE_EXECUTOR_THREAD_COUNT = + IoTDBDescriptor.getInstance().getConfig().getPipeTaskThreadCount(); + + private static final int TERMINATE_EXECUTOR_QUEUE_SIZE = + Math.max(1024, TERMINATE_EXECUTOR_THREAD_COUNT * 64); + // Do not use call run policy to avoid deadlock - private static final ExecutorService terminateExecutor = - new WrappedThreadPoolExecutor( - 0, - IoTDBDescriptor.getInstance().getConfig().getPipeTaskThreadCount(), - 0L, - TimeUnit.SECONDS, - new ArrayBlockingQueue<>( - IoTDBDescriptor.getInstance().getConfig().getPipeTaskThreadCount()), - new IoTThreadFactory(ThreadName.PIPE_TERMINATE_EXECUTION_POOL.getName()), - ThreadName.PIPE_TERMINATE_EXECUTION_POOL.getName()); + private static final ExecutorService terminateExecutor = createTerminateExecutor(); + + private static ExecutorService createTerminateExecutor() { + final WrappedThreadPoolExecutor executor = + new WrappedThreadPoolExecutor( + TERMINATE_EXECUTOR_THREAD_COUNT, + TERMINATE_EXECUTOR_THREAD_COUNT, + 60L, + TimeUnit.SECONDS, + new ArrayBlockingQueue<>(TERMINATE_EXECUTOR_QUEUE_SIZE), + new IoTThreadFactory(ThreadName.PIPE_TERMINATE_EXECUTION_POOL.getName()), + ThreadName.PIPE_TERMINATE_EXECUTION_POOL.getName()); + executor.allowCoreThreadTimeOut(true); + return executor; + } public PipeTerminateEvent( final String pipeName, From 5ec7ee21e3be47c3cd8bb6a14572554e7d11476e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 13 May 2026 14:09:51 +0800 Subject: [PATCH 045/102] [To dev/1.3] Load: Harden LOAD TSFILE source path validation (#17624) (#17654) * Load: Harden LOAD TSFILE source path validation (#17624) * Load pri * sp * MAINTAIN * rollback * Add * change * canonical * line * Pre * Update LoadTsFileStatementTest.java * Update LoadTsFileStatementTest.java * Update IoTDBDescriptor.java --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 83 +++++++++++++ .../apache/iotdb/db/conf/IoTDBDescriptor.java | 30 +++++ .../protocol/legacy/loader/TsFileLoader.java | 2 +- .../thrift/IoTDBDataNodeReceiver.java | 3 +- .../plan/analyze/load/LoadTsFileAnalyzer.java | 2 +- .../scheduler/load/LoadTsFileScheduler.java | 2 +- .../statement/crud/LoadTsFileStatement.java | 70 ++++++++++- .../load/active/ActiveLoadTsFileLoader.java | 3 +- .../crud/LoadTsFileStatementTest.java | 111 ++++++++++++++++++ .../conf/iotdb-system.properties.template | 11 ++ 10 files changed, 307 insertions(+), 10 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatementTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index d3f13d1439232..07c73b43d87f9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -61,8 +61,10 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.lang.reflect.Field; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -324,6 +326,14 @@ public class IoTDBConfig { tierDataDirs[0][0] + File.separator + IoTDBConstant.LOAD_TSFILE_FOLDER_NAME }; + private String[] loadTsFileAllowedDirs = new String[0]; + + private CanonicalPaths loadTsFileDirCanonicalPaths = canonicalPaths(loadTsFileDirs); + + private CanonicalPaths loadTsFileAllowedDirCanonicalPaths = canonicalPaths(loadTsFileAllowedDirs); + + private boolean loadTsFileSourcePathCheckEnabled = false; + /** Strategy of multiple directories. */ private String multiDirStrategyClassName = null; @@ -1394,6 +1404,10 @@ private void formulateFolders() { for (int i = 0; i < loadActiveListeningDirs.length; i++) { loadActiveListeningDirs[i] = addDataHomeDir(loadActiveListeningDirs[i]); } + for (int i = 0; i < loadTsFileAllowedDirs.length; i++) { + loadTsFileAllowedDirs[i] = addDataHomeDir(loadTsFileAllowedDirs[i]); + } + loadTsFileAllowedDirCanonicalPaths = canonicalPaths(loadTsFileAllowedDirs); loadActiveListeningPipeDir = addDataHomeDir(loadActiveListeningPipeDir); loadActiveListeningFailDir = addDataHomeDir(loadActiveListeningFailDir); udfDir = addDataHomeDir(udfDir); @@ -1589,6 +1603,36 @@ public String[] getLoadTsFileDirs() { return this.loadTsFileDirs; } + public String[] getLoadTsFileAllowedDirs() { + return this.loadTsFileAllowedDirs.length == 0 + ? getLoadTsFileDirs() + : this.loadTsFileAllowedDirs; + } + + public Path[] getLoadTsFileAllowedDirCanonicalPaths() throws FileNotFoundException { + return (this.loadTsFileAllowedDirs.length == 0 + ? this.loadTsFileDirCanonicalPaths + : this.loadTsFileAllowedDirCanonicalPaths) + .getPaths(); + } + + public boolean isLoadTsFileSourcePathCheckEnabled() { + return loadTsFileSourcePathCheckEnabled; + } + + public void setLoadTsFileSourcePathCheckEnabled(boolean loadTsFileSourcePathCheckEnabled) { + this.loadTsFileSourcePathCheckEnabled = loadTsFileSourcePathCheckEnabled; + } + + public void setLoadTsFileAllowedDirs(String[] loadTsFileAllowedDirs) { + final String[] newLoadTsFileAllowedDirs = new String[loadTsFileAllowedDirs.length]; + for (int i = 0; i < loadTsFileAllowedDirs.length; i++) { + newLoadTsFileAllowedDirs[i] = addDataHomeDir(loadTsFileAllowedDirs[i]); + } + this.loadTsFileAllowedDirs = newLoadTsFileAllowedDirs; + this.loadTsFileAllowedDirCanonicalPaths = canonicalPaths(newLoadTsFileAllowedDirs); + } + public void formulateLoadTsFileDirs(String[][] tierDataDirs) { if (tierDataDirs.length < 1) { logger.warn("No data directory is set. loadTsFileDirs is kept as the default value."); @@ -1606,6 +1650,45 @@ public void formulateLoadTsFileDirs(String[][] tierDataDirs) { // or the newLoadTsFileDirs will be used in the middle of the process // and cause the undefined behavior. this.loadTsFileDirs = newLoadTsFileDirs; + this.loadTsFileDirCanonicalPaths = canonicalPaths(newLoadTsFileDirs); + } + + private static CanonicalPaths canonicalPaths(final String[] dirs) { + final Path[] paths = new Path[dirs.length]; + for (int i = 0; i < dirs.length; i++) { + try { + paths[i] = new File(dirs[i]).getCanonicalFile().toPath(); + } catch (final IOException e) { + return new CanonicalPaths( + String.format( + "Failed to resolve canonical path for Load TsFile allowed directory %s: %s", + dirs[i], e.getMessage())); + } + } + return new CanonicalPaths(paths); + } + + private static class CanonicalPaths { + + private final Path[] paths; + private final String errorMessage; + + private CanonicalPaths(final Path[] paths) { + this.paths = paths; + this.errorMessage = null; + } + + private CanonicalPaths(final String errorMessage) { + this.paths = new Path[0]; + this.errorMessage = errorMessage; + } + + private Path[] getPaths() throws FileNotFoundException { + if (errorMessage != null) { + throw new FileNotFoundException(errorMessage); + } + return paths; + } } public String getSchemaDir() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index b01fca1a88f82..b380a31eb6486 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -2449,6 +2449,17 @@ private void loadLoadTsFileProps(TrimProperties properties) throws IOException { properties.getProperty( "load_write_throughput_bytes_per_second", String.valueOf(conf.getLoadWriteThroughputBytesPerSecond())))); + + conf.setLoadTsFileAllowedDirs( + Arrays.stream(properties.getProperty("load_tsfile_allowed_dirs", "").trim().split(",")) + .filter(dir -> !dir.isEmpty()) + .toArray(String[]::new)); + conf.setLoadTsFileSourcePathCheckEnabled( + Boolean.parseBoolean( + properties.getProperty( + "load_tsfile_source_path_check_enable", + Boolean.toString(conf.isLoadTsFileSourcePathCheckEnabled())))); + conf.setLoadTabletConversionThresholdBytes( Long.parseLong( properties.getProperty( @@ -2560,6 +2571,25 @@ private void loadLoadTsFileHotModifiedProp(TrimProperties properties) throws IOE ConfigurationFileUtils.getConfigurationDefaultValue( "load_write_throughput_bytes_per_second")))); + conf.setLoadTsFileAllowedDirs( + Arrays.stream( + properties + .getProperty( + "load_tsfile_allowed_dirs", + Optional.ofNullable( + ConfigurationFileUtils.getConfigurationDefaultValue( + "load_tsfile_allowed_dirs")) + .orElse("")) + .trim() + .split(",")) + .filter(dir -> !dir.isEmpty()) + .toArray(String[]::new)); + conf.setLoadTsFileSourcePathCheckEnabled( + Boolean.parseBoolean( + properties.getProperty( + "load_tsfile_source_path_check_enable", + Boolean.toString(conf.isLoadTsFileSourcePathCheckEnabled())))); + conf.setLoadActiveListeningEnable( Boolean.parseBoolean( properties.getProperty( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/loader/TsFileLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/loader/TsFileLoader.java index f88510a0ca85d..8459e22f1eed3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/loader/TsFileLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/loader/TsFileLoader.java @@ -54,7 +54,7 @@ public TsFileLoader(File tsFile, String database) { @Override public void load() { try { - LoadTsFileStatement statement = new LoadTsFileStatement(tsFile.getAbsolutePath()); + LoadTsFileStatement statement = LoadTsFileStatement.createUnchecked(tsFile.getAbsolutePath()); statement.setDeleteAfterLoad(true); statement.setConvertOnTypeMismatch(true); statement.setDatabaseLevel(parseSgLevel()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index 334c3530414f3..58d4c29eddc57 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -490,8 +490,7 @@ private TSStatus loadTsFileAsync(final List absolutePaths) throws IOExce } private TSStatus loadTsFileSync(final String fileAbsolutePath) throws FileNotFoundException { - final LoadTsFileStatement statement = new LoadTsFileStatement(fileAbsolutePath); - + final LoadTsFileStatement statement = LoadTsFileStatement.createUnchecked(fileAbsolutePath); statement.setDeleteAfterLoad(true); statement.setConvertOnTypeMismatch(true); statement.setVerifySchema(validateTsFile.get()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java index 5a346fdf8e09d..b6d4fb0043b79 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java @@ -372,7 +372,7 @@ private boolean handleSingleMiniFile(final int i) throws FileNotFoundException { final TSStatus status = loadTsFileDataTypeConverter .convertForTreeModel( - new LoadTsFileStatement(tsFiles.get(i).getPath()) + LoadTsFileStatement.createUnchecked(tsFiles.get(i).getPath()) .setDeleteAfterLoad(isDeleteAfterLoad) .setConvertOnTypeMismatch(isConvertOnTypeMismatch)) .orElse(null); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java index 4a85b6d0ebc0a..5cafe3b838461 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java @@ -551,7 +551,7 @@ private void convertFailedTsFilesToTabletsAndRetry() { final TSStatus status = loadTsFileDataTypeConverter .convertForTreeModel( - new LoadTsFileStatement(filePath) + LoadTsFileStatement.createUnchecked(filePath) .setDeleteAfterLoad(failedNode.isDeleteAfterLoad()) .setConvertOnTypeMismatch(true)) .orElse(null); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java index 3b10d3733e1b3..6e74ceed206ec 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.statement.Statement; import org.apache.iotdb.db.queryengine.plan.statement.StatementType; @@ -34,7 +35,9 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.file.Path; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -56,6 +59,15 @@ public class LoadTsFileStatement extends Statement { private List writePointCountList; public LoadTsFileStatement(String filePath) throws FileNotFoundException { + this(filePath, true); + } + + public static LoadTsFileStatement createUnchecked(String filePath) throws FileNotFoundException { + return new LoadTsFileStatement(filePath, false); + } + + private LoadTsFileStatement(String filePath, boolean validateSourcePath) + throws FileNotFoundException { this.file = new File(filePath).getAbsoluteFile(); this.databaseLevel = IoTDBDescriptor.getInstance().getConfig().getDefaultDatabaseLevel(); this.verifySchema = true; @@ -65,13 +77,22 @@ public LoadTsFileStatement(String filePath) throws FileNotFoundException { IoTDBDescriptor.getInstance().getConfig().getLoadTabletConversionThresholdBytes(); this.autoCreateDatabase = IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled(); - this.tsFiles = processTsFile(file); + this.tsFiles = processTsFile(file, validateSourcePath); this.resources = new ArrayList<>(); this.writePointCountList = new ArrayList<>(); this.statementType = StatementType.MULTI_BATCH_INSERT; } public static List processTsFile(final File file) throws FileNotFoundException { + return processTsFile(file, true); + } + + public static List processTsFile(final File file, final boolean validateSourcePath) + throws FileNotFoundException { + if (validateSourcePath) { + validateLoadSourcePath(file); + } + final List tsFiles = new ArrayList<>(); if (file.isFile()) { tsFiles.add(file); @@ -82,7 +103,7 @@ public static List processTsFile(final File file) throws FileNotFoundExcep "Can not find %s on this machine, notice that load can only handle files on this machine.", file.getPath())); } - tsFiles.addAll(findAllTsFile(file)); + tsFiles.addAll(findAllTsFile(file, validateSourcePath)); } sortTsFiles(tsFiles); return tsFiles; @@ -101,7 +122,8 @@ protected LoadTsFileStatement() { this.statementType = StatementType.MULTI_BATCH_INSERT; } - private static List findAllTsFile(File file) { + private static List findAllTsFile(File file, boolean validateSourcePath) + throws FileNotFoundException { final File[] files = file.listFiles(); if (files == null) { return Collections.emptyList(); @@ -109,15 +131,55 @@ private static List findAllTsFile(File file) { final List tsFiles = new ArrayList<>(); for (File nowFile : files) { + if (validateSourcePath) { + validateLoadSourcePath(nowFile); + } if (nowFile.getName().endsWith(TsFileConstant.TSFILE_SUFFIX)) { tsFiles.add(nowFile); } else if (nowFile.isDirectory()) { - tsFiles.addAll(findAllTsFile(nowFile)); + tsFiles.addAll(findAllTsFile(nowFile, validateSourcePath)); } } return tsFiles; } + public static void validateLoadSourcePath(final String filePath) throws FileNotFoundException { + validateLoadSourcePath(new File(filePath)); + } + + private static void validateLoadSourcePath(final File file) throws FileNotFoundException { + final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + if (!config.isLoadTsFileSourcePathCheckEnabled()) { + return; + } + + final Path sourcePath = canonicalPath(file); + final String[] allowedDirs = config.getLoadTsFileAllowedDirs(); + final Path[] allowedDirCanonicalPaths = config.getLoadTsFileAllowedDirCanonicalPaths(); + + for (final Path allowedDirCanonicalPath : allowedDirCanonicalPaths) { + if (sourcePath.startsWith(allowedDirCanonicalPath)) { + return; + } + } + + throw new FileNotFoundException( + String.format( + "Load TsFile source path %s is outside allowed directories %s.", + sourcePath, Arrays.toString(allowedDirs))); + } + + private static Path canonicalPath(final File file) throws FileNotFoundException { + try { + return file.getCanonicalFile().toPath(); + } catch (final IOException e) { + throw new FileNotFoundException( + String.format( + "Failed to resolve canonical path for Load TsFile source %s: %s", + file.getPath(), e.getMessage())); + } + } + private static void sortTsFiles(List files) { files.sort( (o1, o2) -> { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java index 8a1f305e4f28c..a228ef6c6514c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadTsFileLoader.java @@ -218,7 +218,8 @@ private TSStatus loadTsFile( final ActiveLoadPendingQueue.ActiveLoadEntry entry, final IClientSession session) throws FileNotFoundException { final File tsFile = new File(entry.getFile()); - final LoadTsFileStatement statement = new LoadTsFileStatement(entry.getFile()); + final LoadTsFileStatement statement = + LoadTsFileStatement.createUnchecked(tsFile.getAbsolutePath()); statement.setDeleteAfterLoad(true); statement.setAutoCreateDatabase( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatementTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatementTest.java new file mode 100644 index 0000000000000..acf608abf806a --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatementTest.java @@ -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. + */ + +package org.apache.iotdb.db.queryengine.plan.statement.crud; + +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Comparator; +import java.util.stream.Stream; + +public class LoadTsFileStatementTest { + + @Test + public void testLoadSourcePathMustBeInAllowedDirs() throws Exception { + final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + final String[] originalAllowedDirs = config.getLoadTsFileAllowedDirs().clone(); + final boolean originalCheckEnabled = config.isLoadTsFileSourcePathCheckEnabled(); + final Path allowedDir = Files.createTempDirectory("load-tsfile-allowed"); + final Path deniedDir = Files.createTempDirectory("load-tsfile-denied"); + + try { + config.setLoadTsFileSourcePathCheckEnabled(true); + config.setLoadTsFileAllowedDirs(new String[] {allowedDir.toString()}); + final Path deniedTsFile = Files.createFile(deniedDir.resolve("denied.tsfile")); + final Path traversalTsFile = + allowedDir.resolve("..").resolve(deniedDir.getFileName()).resolve("denied.tsfile"); + + assertLoadSourcePathRejected(deniedTsFile); + assertLoadSourcePathRejected(traversalTsFile); + } finally { + config.setLoadTsFileAllowedDirs(originalAllowedDirs); + config.setLoadTsFileSourcePathCheckEnabled(originalCheckEnabled); + deleteRecursively(allowedDir); + deleteRecursively(deniedDir); + } + } + + @Test + public void testLoadSourcePathCheckCanBeDisabled() throws Exception { + final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + final String[] originalAllowedDirs = config.getLoadTsFileAllowedDirs().clone(); + final boolean originalCheckEnabled = config.isLoadTsFileSourcePathCheckEnabled(); + final Path allowedDir = Files.createTempDirectory("load-tsfile-allowed"); + final Path deniedDir = Files.createTempDirectory("load-tsfile-denied"); + + try { + config.setLoadTsFileSourcePathCheckEnabled(false); + config.setLoadTsFileAllowedDirs(new String[] {allowedDir.toString()}); + final Path deniedTsFile = Files.createFile(deniedDir.resolve("denied.tsfile")); + + new LoadTsFileStatement(deniedTsFile.toString()); + } finally { + config.setLoadTsFileAllowedDirs(originalAllowedDirs); + config.setLoadTsFileSourcePathCheckEnabled(originalCheckEnabled); + deleteRecursively(allowedDir); + deleteRecursively(deniedDir); + } + } + + private static void assertLoadSourcePathRejected(final Path sourcePath) { + try { + new LoadTsFileStatement(sourcePath.toString()); + Assert.fail("Expected disallowed LOAD TSFILE source path to be rejected."); + } catch (final FileNotFoundException e) { + Assert.assertTrue(e.getMessage().contains("outside allowed directories")); + } + } + + private static void deleteRecursively(final Path path) throws IOException { + if (path == null || !Files.exists(path)) { + return; + } + + try (final Stream pathStream = Files.walk(path)) { + pathStream + .sorted(Comparator.reverseOrder()) + .forEach( + currentPath -> { + try { + Files.deleteIfExists(currentPath); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + } +} diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 5447bcf5fd4ad..741a4c846cdee 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -2033,6 +2033,17 @@ load_clean_up_task_execution_delay_time_seconds=1800 # Datatype: int load_write_throughput_bytes_per_second=-1 +# Whether the load_tsfile supports path allowed dirs check. +# effectiveMode: hot_reload +# Datatype: String +load_tsfile_source_path_check_enable=false + +# Comma-separated list of directories from which user-issued LOAD TSFILE statements can read. +# If empty, IoTDB only permits LOAD sources under the internal load TsFile directories. +# effectiveMode: hot_reload +# Datatype: String +load_tsfile_allowed_dirs= + # Whether to enable the active listening mode for tsfile loading. # effectiveMode: hot_reload # Datatype: Boolean From e2ea8719da6a8a8ae1b259cf574b9afc3bebdec7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 13 May 2026 15:29:13 +0800 Subject: [PATCH 046/102] [To dev/1.3] Pipe: Fixed the OOM bug of parser for large aligned pages (#17639) (#17646) * Pipe: Fixed the OOM bug of parser for large aligned pages (#17639) * chunk * shop * Update TsFileInsertionEventScanParser.java * Fix * cp-ger * Update TsFileInsertionDataContainerTest.java * Update TsFileInsertionDataContainerTest.java --- .../AlignedSinglePageWholeChunkReader.java | 53 ++++++- .../scan/EstimatedMemoryChunkReader.java | 25 +++ .../scan/SinglePageWholeChunkReader.java | 27 +++- .../TsFileInsertionScanDataContainer.java | 124 +++++++++++++-- .../TsFileInsertionDataContainerTest.java | 144 ++++++++++++++++++ 5 files changed, 348 insertions(+), 25 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/EstimatedMemoryChunkReader.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/AlignedSinglePageWholeChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/AlignedSinglePageWholeChunkReader.java index 740a1523d279f..a9de04cbe6463 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/AlignedSinglePageWholeChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/AlignedSinglePageWholeChunkReader.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.event.common.tsfile.container.scan; +import org.apache.tsfile.compress.IUnCompressor; import org.apache.tsfile.encoding.decoder.Decoder; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.header.ChunkHeader; @@ -29,6 +30,7 @@ import org.apache.tsfile.read.reader.chunk.AbstractChunkReader; import org.apache.tsfile.read.reader.chunk.ChunkReader; import org.apache.tsfile.read.reader.page.AlignedPageReader; +import org.apache.tsfile.read.reader.page.LazyLoadPageData; import java.io.IOException; import java.io.Serializable; @@ -40,7 +42,8 @@ * The {@link AlignedSinglePageWholeChunkReader} is used to read a whole single page aligned chunk * with need to pass in the statistics. */ -public class AlignedSinglePageWholeChunkReader extends AbstractChunkReader { +public class AlignedSinglePageWholeChunkReader extends AbstractChunkReader + implements EstimatedMemoryChunkReader { // chunk header of the time column private final ChunkHeader timeChunkHeader; @@ -53,12 +56,15 @@ public class AlignedSinglePageWholeChunkReader extends AbstractChunkReader { private final List valueChunkDataBufferList = new ArrayList<>(); // deleted intervals of all the sub sensors private final List> valueDeleteIntervalsList = new ArrayList<>(); + private final long pageEstimatedMemoryUsageInBytes; public AlignedSinglePageWholeChunkReader(Chunk timeChunk, List valueChunkList) throws IOException { super(Long.MIN_VALUE, null); this.timeChunkHeader = timeChunk.getHeader(); this.timeChunkDataBuffer = timeChunk.getData(); + this.pageEstimatedMemoryUsageInBytes = + calculatePageEstimatedMemoryUsageInBytes(timeChunk, valueChunkList); valueChunkList.forEach( chunk -> { @@ -124,7 +130,7 @@ private AlignedPageReader constructAlignedPageReader( ChunkReader.deserializePageData(timePageHeader, timeChunkDataBuffer, timeChunkHeader); List valuePageHeaderList = new ArrayList<>(); - List valuePageDataList = new ArrayList<>(); + LazyLoadPageData[] valuePageDataArray = new LazyLoadPageData[rawValuePageHeaderList.size()]; List valueDataTypeList = new ArrayList<>(); List valueDecoderList = new ArrayList<>(); @@ -135,15 +141,21 @@ private AlignedPageReader constructAlignedPageReader( if (valuePageHeader == null || valuePageHeader.getUncompressedSize() == 0) { // Empty Page valuePageHeaderList.add(null); - valuePageDataList.add(null); + valuePageDataArray[i] = null; valueDataTypeList.add(null); valueDecoderList.add(null); } else { ChunkHeader valueChunkHeader = valueChunkHeaderList.get(i); + int currentPagePosition = valueChunkDataBufferList.get(i).position(); + valueChunkDataBufferList + .get(i) + .position(currentPagePosition + valuePageHeader.getCompressedSize()); valuePageHeaderList.add(valuePageHeader); - valuePageDataList.add( - ChunkReader.deserializePageData( - valuePageHeader, valueChunkDataBufferList.get(i), valueChunkHeader)); + valuePageDataArray[i] = + new LazyLoadPageData( + valueChunkDataBufferList.get(i).array(), + currentPagePosition, + IUnCompressor.getUnCompressor(valueChunkHeader.getCompressionType())); valueDataTypeList.add(valueChunkHeader.getDataType()); valueDecoderList.add( Decoder.getDecoderByType( @@ -160,11 +172,38 @@ private AlignedPageReader constructAlignedPageReader( timePageData, defaultTimeDecoder, valuePageHeaderList, - valuePageDataList, + valuePageDataArray, valueDataTypeList, valueDecoderList, queryFilter); alignedPageReader.setDeleteIntervalList(valueDeleteIntervalsList); return alignedPageReader; } + + @Override + public long getCurrentPageEstimatedMemoryUsageInBytes() { + return pageEstimatedMemoryUsageInBytes; + } + + public static long calculatePageEstimatedMemoryUsageInBytes( + final Chunk timeChunk, final List valueChunkList) throws IOException { + final ByteBuffer timeChunkDataBuffer = timeChunk.getData().duplicate(); + long estimatedMemoryUsageInBytes = + PageHeader.deserializeFrom(timeChunkDataBuffer, (Statistics) null) + .getUncompressedSize(); + + for (final Chunk valueChunk : valueChunkList) { + if (valueChunk == null) { + continue; + } + + final ByteBuffer valueChunkDataBuffer = valueChunk.getData().duplicate(); + estimatedMemoryUsageInBytes += + PageHeader.deserializeFrom( + valueChunkDataBuffer, (Statistics) null) + .getUncompressedSize(); + } + + return estimatedMemoryUsageInBytes; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/EstimatedMemoryChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/EstimatedMemoryChunkReader.java new file mode 100644 index 0000000000000..dc1f2501bdf7b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/EstimatedMemoryChunkReader.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.event.common.tsfile.container.scan; + +interface EstimatedMemoryChunkReader { + + long getCurrentPageEstimatedMemoryUsageInBytes(); +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/SinglePageWholeChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/SinglePageWholeChunkReader.java index 3f1aadbba0ccb..ade5001290348 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/SinglePageWholeChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/SinglePageWholeChunkReader.java @@ -26,22 +26,25 @@ import org.apache.tsfile.file.metadata.statistics.Statistics; import org.apache.tsfile.read.common.Chunk; import org.apache.tsfile.read.reader.chunk.AbstractChunkReader; +import org.apache.tsfile.read.reader.page.LazyLoadPageData; import org.apache.tsfile.read.reader.page.PageReader; import java.io.IOException; import java.io.Serializable; import java.nio.ByteBuffer; -public class SinglePageWholeChunkReader extends AbstractChunkReader { +public class SinglePageWholeChunkReader extends AbstractChunkReader + implements EstimatedMemoryChunkReader { private final ChunkHeader chunkHeader; private final ByteBuffer chunkDataBuffer; + private final long pageEstimatedMemoryUsageInBytes; public SinglePageWholeChunkReader(Chunk chunk) throws IOException { super(Long.MIN_VALUE, null); this.chunkHeader = chunk.getHeader(); this.chunkDataBuffer = chunk.getData(); - + this.pageEstimatedMemoryUsageInBytes = calculatePageEstimatedMemoryUsageInBytes(chunk); initAllPageReaders(); } @@ -56,15 +59,33 @@ private void initAllPageReaders() throws IOException { } private PageReader constructPageReader(PageHeader pageHeader) throws IOException { + final int currentPagePosition = chunkDataBuffer.position(); + chunkDataBuffer.position(currentPagePosition + pageHeader.getCompressedSize()); return new PageReader( pageHeader, - deserializePageData(pageHeader, chunkDataBuffer, chunkHeader), + new LazyLoadPageData( + chunkDataBuffer.array(), + currentPagePosition, + IUnCompressor.getUnCompressor(chunkHeader.getCompressionType())), chunkHeader.getDataType(), Decoder.getDecoderByType(chunkHeader.getEncodingType(), chunkHeader.getDataType()), defaultTimeDecoder, null); } + @Override + public long getCurrentPageEstimatedMemoryUsageInBytes() { + return pageEstimatedMemoryUsageInBytes; + } + + public static long calculatePageEstimatedMemoryUsageInBytes(final Chunk chunk) + throws IOException { + final ByteBuffer chunkDataBuffer = chunk.getData().duplicate(); + final PageHeader pageHeader = + PageHeader.deserializeFrom(chunkDataBuffer, (Statistics) null); + return pageHeader.getUncompressedSize(); + } + ///////////////////////////////////////////////////////////////////////////////////////////////// // util methods ///////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java index 07b91cd68af7c..9366d0f62dfe2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java @@ -98,10 +98,11 @@ public class TsFileInsertionScanDataContainer extends TsFileInsertionDataContain // Cached time chunk private final List timeChunkList = new ArrayList<>(); private final List isMultiPageList = new ArrayList<>(); + private final List timeChunkPageMemorySizeList = new ArrayList<>(); private final Map measurementIndexMap = new HashMap<>(); private int lastIndex = -1; - private ChunkHeader firstChunkHeader4NextSequentialValueChunks; + private Chunk firstChunk4NextSequentialValueChunks; private byte lastMarker = Byte.MIN_VALUE; @@ -343,6 +344,7 @@ private void prepareData() throws IOException { } do { + resizePageDataMemoryForCurrentPageIfNeeded(); data = chunkReader.nextPageData(); long size = PipeMemoryWeightUtil.calculateBatchDataRamBytesUsed(data); if (allocatedMemoryBlockForBatchData.getMemoryUsageInBytes() < size) { @@ -352,6 +354,23 @@ private void prepareData() throws IOException { } while (!data.hasCurrent()); } + private void resizePageDataMemoryForCurrentPageIfNeeded() { + if (!(chunkReader instanceof EstimatedMemoryChunkReader)) { + return; + } + + final long estimatedMemoryUsageInBytes = + ((EstimatedMemoryChunkReader) chunkReader).getCurrentPageEstimatedMemoryUsageInBytes(); + resizePageDataMemoryIfNeeded(estimatedMemoryUsageInBytes); + } + + private void resizePageDataMemoryIfNeeded(final long estimatedMemoryUsageInBytes) { + if (allocatedMemoryBlockForBatchData.getMemoryUsageInBytes() < estimatedMemoryUsageInBytes) { + PipeDataNodeResourceManager.memory() + .forceResize(allocatedMemoryBlockForBatchData, estimatedMemoryUsageInBytes); + } + } + private boolean putValueToColumns(final BatchData data, final Tablet tablet, final int rowIndex) { final Object[] columns = tablet.values; boolean isNeedFillTime = false; @@ -439,6 +458,7 @@ private boolean putValueToColumns(final BatchData data, final Tablet tablet, fin private void moveToNextChunkReader() throws IOException, IllegalStateException { ChunkHeader chunkHeader; long valueChunkSize = 0; + long valueChunkPageMemorySize = 0; final List valueChunkList = new ArrayList<>(); currentMeasurements.clear(); modsInfos.clear(); @@ -449,7 +469,12 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { } byte marker; - while ((marker = lastMarker != Byte.MIN_VALUE ? lastMarker : tsFileSequenceReader.readMarker()) + while ((marker = + lastMarker != Byte.MIN_VALUE + ? lastMarker + : Objects.nonNull(firstChunk4NextSequentialValueChunks) + ? toValueChunkMarker(firstChunk4NextSequentialValueChunks.getHeader()) + : tsFileSequenceReader.readMarker()) != MetaMarker.SEPARATOR) { lastMarker = Byte.MIN_VALUE; switch (marker) { @@ -473,10 +498,17 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { if ((chunkHeader.getChunkType() & TsFileConstant.TIME_COLUMN_MASK) == TsFileConstant.TIME_COLUMN_MASK) { - timeChunkList.add( + final Chunk timeChunk = new Chunk( - chunkHeader, tsFileSequenceReader.readChunk(-1, chunkHeader.getDataSize()))); - isMultiPageList.add(marker == MetaMarker.TIME_CHUNK_HEADER); + chunkHeader, tsFileSequenceReader.readChunk(-1, chunkHeader.getDataSize())); + final boolean isMultiPage = marker == MetaMarker.TIME_CHUNK_HEADER; + timeChunkList.add(timeChunk); + isMultiPageList.add(isMultiPage); + timeChunkPageMemorySizeList.add( + isMultiPage + ? 0 + : SinglePageWholeChunkReader.calculatePageEstimatedMemoryUsageInBytes( + timeChunk)); break; } @@ -540,8 +572,10 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { case MetaMarker.VALUE_CHUNK_HEADER: case MetaMarker.ONLY_ONE_PAGE_VALUE_CHUNK_HEADER: { - if (Objects.isNull(firstChunkHeader4NextSequentialValueChunks)) { - long currentChunkHeaderOffset = tsFileSequenceReader.position() - 1; + Chunk chunk; + long currentValueChunkPageMemorySize = 0; + if (Objects.isNull(firstChunk4NextSequentialValueChunks)) { + final long currentChunkHeaderOffset = tsFileSequenceReader.position() - 1; chunkHeader = tsFileSequenceReader.readChunkHeader(marker); final long nextMarkerOffset = @@ -592,40 +626,56 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { if (chunkHeader.getDataSize() == 0) { break; } + chunk = + new Chunk( + chunkHeader, tsFileSequenceReader.readChunk(-1, chunkHeader.getDataSize())); + currentValueChunkPageMemorySize = + calculatePageMemorySizeIfSinglePageValueChunk(chunk); boolean needReturn = false; final long timeChunkSize = lastIndex >= 0 ? PipeMemoryWeightUtil.calculateChunkRamBytesUsed( timeChunkList.get(lastIndex)) : 0; + final long timeChunkPageMemorySize = + lastIndex >= 0 ? timeChunkPageMemorySizeList.get(lastIndex) : 0; if (lastIndex >= 0) { if (valueIndex != lastIndex) { needReturn = recordAlignedChunk(valueChunkList, marker); } else { final long chunkSize = timeChunkSize + valueChunkSize; + final long pageMemorySize = timeChunkPageMemorySize + valueChunkPageMemorySize; if (chunkSize + chunkHeader.getDataSize() - > allocatedMemoryBlockForChunk.getMemoryUsageInBytes()) { + > allocatedMemoryBlockForChunk.getMemoryUsageInBytes() + || timeChunkPageMemorySize > 0 + && currentValueChunkPageMemorySize > 0 + && pageMemorySize + currentValueChunkPageMemorySize + > getPageDataMemoryLimitInBytes()) { needReturn = recordAlignedChunk(valueChunkList, marker); } } } lastIndex = valueIndex; if (needReturn) { - firstChunkHeader4NextSequentialValueChunks = chunkHeader; + firstChunk4NextSequentialValueChunks = chunk; return; } resizeChunkMemoryBlockIfFirstValueChunkExceedsLimit(valueChunkList, chunkHeader); + resizePageDataMemoryBlockIfFirstValueChunkExceedsLimit( + valueChunkList, currentValueChunkPageMemorySize); } else { - chunkHeader = firstChunkHeader4NextSequentialValueChunks; - firstChunkHeader4NextSequentialValueChunks = null; + chunk = firstChunk4NextSequentialValueChunks; + chunkHeader = chunk.getHeader(); + firstChunk4NextSequentialValueChunks = null; + currentValueChunkPageMemorySize = + calculatePageMemorySizeIfSinglePageValueChunk(chunk); resizeChunkMemoryBlockIfFirstValueChunkExceedsLimit(valueChunkList, chunkHeader); + resizePageDataMemoryBlockIfFirstValueChunkExceedsLimit( + valueChunkList, currentValueChunkPageMemorySize); } - Chunk chunk = - new Chunk( - chunkHeader, tsFileSequenceReader.readChunk(-1, chunkHeader.getDataSize())); - valueChunkSize += chunkHeader.getDataSize(); + valueChunkPageMemorySize += currentValueChunkPageMemorySize; valueChunkList.add(chunk); currentMeasurements.add( new MeasurementSchema(chunkHeader.getMeasurementID(), chunkHeader.getDataType())); @@ -649,6 +699,7 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { lastIndex = -1; timeChunkList.clear(); isMultiPageList.clear(); + timeChunkPageMemorySizeList.clear(); measurementIndexMap.clear(); currentDevice = pattern.mayOverlapWithDevice(deviceID) ? deviceID : null; @@ -670,12 +721,21 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { } } + private long getPageDataMemoryLimitInBytes() { + return PipeConfig.getInstance().getPipeMaxReaderChunkSize(); + } + private boolean recordAlignedChunk(final List valueChunkList, final byte marker) throws IOException { if (!valueChunkList.isEmpty()) { final Chunk timeChunk = timeChunkList.get(lastIndex); timeChunk.getData().rewind(); currentIsMultiPage = isMultiPageList.get(lastIndex); + if (!currentIsMultiPage) { + resizePageDataMemoryIfNeeded( + AlignedSinglePageWholeChunkReader.calculatePageEstimatedMemoryUsageInBytes( + timeChunk, valueChunkList)); + } chunkReader = currentIsMultiPage ? new AlignedChunkReader(timeChunk, valueChunkList, filter) @@ -701,6 +761,40 @@ private void resizeChunkMemoryBlockIfFirstValueChunkExceedsLimit( } } + private void resizePageDataMemoryBlockIfFirstValueChunkExceedsLimit( + final List valueChunkList, final long valueChunkPageMemorySize) { + if (!valueChunkList.isEmpty() || lastIndex < 0 || valueChunkPageMemorySize <= 0) { + return; + } + + final long timeChunkPageMemorySize = timeChunkPageMemorySizeList.get(lastIndex); + if (timeChunkPageMemorySize <= 0) { + return; + } + + final long pageMemorySize = timeChunkPageMemorySize + valueChunkPageMemorySize; + if (pageMemorySize > getPageDataMemoryLimitInBytes()) { + PipeDataNodeResourceManager.memory() + .forceResize(allocatedMemoryBlockForBatchData, pageMemorySize); + } + } + + private long calculatePageMemorySizeIfSinglePageValueChunk(final Chunk chunk) throws IOException { + return isSinglePageValueChunk(chunk.getHeader()) + ? SinglePageWholeChunkReader.calculatePageEstimatedMemoryUsageInBytes(chunk) + : 0; + } + + private boolean isSinglePageValueChunk(final ChunkHeader chunkHeader) { + return (chunkHeader.getChunkType() & 0x3F) == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER; + } + + private byte toValueChunkMarker(final ChunkHeader chunkHeader) { + return isSinglePageValueChunk(chunkHeader) + ? MetaMarker.ONLY_ONE_PAGE_VALUE_CHUNK_HEADER + : MetaMarker.VALUE_CHUNK_HEADER; + } + @Override public void close() { super.close(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java index 51fc1f3a55617..7fe514b277e63 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java @@ -26,20 +26,29 @@ import org.apache.iotdb.commons.pipe.datastructure.pattern.PrefixPipePattern; import org.apache.iotdb.db.pipe.event.common.tsfile.container.TsFileInsertionDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.container.query.TsFileInsertionQueryDataContainer; +import org.apache.iotdb.db.pipe.event.common.tsfile.container.scan.AlignedSinglePageWholeChunkReader; import org.apache.iotdb.db.pipe.event.common.tsfile.container.scan.TsFileInsertionScanDataContainer; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionTestFileWriter; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; import org.apache.iotdb.pipe.api.access.Row; import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.file.MetaMarker; +import org.apache.tsfile.file.metadata.AlignedChunkMetadata; +import org.apache.tsfile.file.metadata.ChunkMetadata; +import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.file.metadata.PlainDeviceID; import org.apache.tsfile.file.metadata.enums.CompressionType; import org.apache.tsfile.file.metadata.enums.TSEncoding; import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.read.common.Chunk; import org.apache.tsfile.read.common.Path; import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.utils.Binary; @@ -121,11 +130,96 @@ public void testScanContainer() throws Exception { System.out.println(System.currentTimeMillis() - startTime); } + @Test + public void testScanParserSplitAlignedSinglePageChunkByEstimatedPageMemory() throws Exception { + final long originalPipeMaxReaderChunkSize = + CommonDescriptor.getInstance().getConfig().getPipeMaxReaderChunkSize(); + final int originalPageSizeInByte = + TSFileDescriptor.getInstance().getConfig().getPageSizeInByte(); + final int originalMaxNumberOfPointsInPage = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); + + try { + TSFileDescriptor.getInstance().getConfig().setPageSizeInByte(64 * 1024); + TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(10000); + + final int measurementCount = 16; + final int rowCount = 64; + final List schemaList = new ArrayList<>(); + for (int i = 0; i < measurementCount; ++i) { + schemaList.add( + new MeasurementSchema( + "s" + i, TSDataType.STRING, TSEncoding.PLAIN, CompressionType.LZ4)); + } + + alignedTsFile = new File("aligned-single-page-high-compression.tsfile"); + final Tablet tablet = new Tablet("root.sg.d", schemaList, rowCount); + final Binary value = + new Binary(new String(new char[512]).replace('\0', 'a'), TSFileConfig.STRING_CHARSET); + for (int row = 0; row < rowCount; ++row) { + tablet.addTimestamp(row, row); + for (int measurementIndex = 0; measurementIndex < measurementCount; ++measurementIndex) { + tablet.addValue("s" + measurementIndex, row, value); + } + } + tablet.rowSize = rowCount; + + try (final TsFileWriter writer = new TsFileWriter(alignedTsFile)) { + writer.registerAlignedTimeseries(new Path("root.sg.d"), schemaList); + writer.writeAligned(tablet); + } + + CommonDescriptor.getInstance() + .getConfig() + .setPipeMaxReaderChunkSize( + calculatePipeMaxReaderChunkSizeForSinglePageAlignedChunk(alignedTsFile)); + + int tabletCount = 0; + int maxMeasurementCount = 0; + int pointCount = 0; + try (final TsFileInsertionScanDataContainer parser = + new TsFileInsertionScanDataContainer( + alignedTsFile, + new PrefixPipePattern("root"), + Long.MIN_VALUE, + Long.MAX_VALUE, + null, + null, + false)) { + for (final Pair tabletWithIsAligned : parser.toTabletWithIsAligneds()) { + Assert.assertTrue(tabletWithIsAligned.getRight()); + final Tablet parsedTablet = tabletWithIsAligned.getLeft(); + tabletCount++; + maxMeasurementCount = Math.max(maxMeasurementCount, parsedTablet.getSchemas().size()); + pointCount += getNonNullSize(parsedTablet); + } + } + + Assert.assertTrue(tabletCount > 1); + Assert.assertTrue(maxMeasurementCount < measurementCount); + Assert.assertEquals(measurementCount * rowCount, pointCount); + } finally { + CommonDescriptor.getInstance() + .getConfig() + .setPipeMaxReaderChunkSize(originalPipeMaxReaderChunkSize); + TSFileDescriptor.getInstance().getConfig().setPageSizeInByte(originalPageSizeInByte); + TSFileDescriptor.getInstance() + .getConfig() + .setMaxNumberOfPointsInPage(originalMaxNumberOfPointsInPage); + } + } + @Test public void testScanParserResizesChunkMemoryForFirstAlignedValueChunk() throws Exception { final long originalPipeMaxReaderChunkSize = PipeConfig.getInstance().getPipeMaxReaderChunkSize(); + final int originalPipeDataStructureTabletSizeInBytes = + PipeConfig.getInstance().getPipeDataStructureTabletSizeInBytes(); + final int configuredBatchMemorySize = 1024 * 1024; CommonDescriptor.getInstance().getConfig().setPipeMaxReaderChunkSize(0); + CommonDescriptor.getInstance() + .getConfig() + .setPipeDataStructureTabletSizeInBytes(configuredBatchMemorySize); alignedTsFile = new File("single-aligned-value-chunk.tsfile"); final List schemaList = new ArrayList<>(); @@ -154,11 +248,18 @@ public void testScanParserResizesChunkMemoryForFirstAlignedValueChunk() throws E null, false)) { Assert.assertTrue(getAllocatedChunkMemory(parser).getMemoryUsageInBytes() > 0); + Assert.assertTrue(getAllocatedBatchDataMemory(parser).getMemoryUsageInBytes() > 0); + Assert.assertTrue( + getAllocatedBatchDataMemory(parser).getMemoryUsageInBytes() + < configuredBatchMemorySize); } } finally { CommonDescriptor.getInstance() .getConfig() .setPipeMaxReaderChunkSize(originalPipeMaxReaderChunkSize); + CommonDescriptor.getInstance() + .getConfig() + .setPipeDataStructureTabletSizeInBytes(originalPipeDataStructureTabletSizeInBytes); } } @@ -696,4 +797,47 @@ private PipeMemoryBlock getAllocatedChunkMemory(final TsFileInsertionScanDataCon field.setAccessible(true); return (PipeMemoryBlock) field.get(parser); } + + private PipeMemoryBlock getAllocatedBatchDataMemory(final TsFileInsertionScanDataContainer parser) + throws NoSuchFieldException, IllegalAccessException { + final Field field = + TsFileInsertionScanDataContainer.class.getDeclaredField("allocatedMemoryBlockForBatchData"); + field.setAccessible(true); + return (PipeMemoryBlock) field.get(parser); + } + + private long calculatePipeMaxReaderChunkSizeForSinglePageAlignedChunk(final File tsFile) + throws Exception { + try (final TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + final List deviceIDList = reader.getAllDevices(); + Assert.assertEquals(1, deviceIDList.size()); + final IDeviceID deviceID = deviceIDList.get(0); + final List alignedChunkMetadataList = + reader.getAlignedChunkMetadata(deviceID); + Assert.assertEquals(1, alignedChunkMetadataList.size()); + + final AlignedChunkMetadata alignedChunkMetadata = alignedChunkMetadataList.get(0); + final Chunk timeChunk = + reader.readMemChunk((ChunkMetadata) alignedChunkMetadata.getTimeChunkMetadata()); + Assert.assertEquals( + MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER, timeChunk.getHeader().getChunkType() & 0x3F); + + final List valueChunkList = new ArrayList<>(); + long chunkSizeLimit = PipeMemoryWeightUtil.calculateChunkRamBytesUsed(timeChunk); + for (final IChunkMetadata valueChunkMetadata : + alignedChunkMetadata.getValueChunkMetadataList()) { + final Chunk valueChunk = reader.readMemChunk((ChunkMetadata) valueChunkMetadata); + Assert.assertEquals( + MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER, valueChunk.getHeader().getChunkType() & 0x3F); + valueChunkList.add(valueChunk); + chunkSizeLimit += valueChunk.getHeader().getDataSize(); + } + + final long estimatedPageMemorySize = + AlignedSinglePageWholeChunkReader.calculatePageEstimatedMemoryUsageInBytes( + timeChunk, valueChunkList); + Assert.assertTrue(estimatedPageMemorySize > chunkSizeLimit); + return chunkSizeLimit; + } + } } From f241857393683afc19cc98760ac81207ecb228d7 Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Wed, 13 May 2026 20:41:43 +0800 Subject: [PATCH 047/102] [to dev/1.3] Fixed concurrency issues caused by write and flush sorting during query execution --- .../dataregion/flush/MemTableFlushTask.java | 1 + .../memtable/AbstractWritableMemChunk.java | 43 ++++++++- .../memtable/AlignedWritableMemChunk.java | 88 ++++++++++++------- .../memtable/IWritableMemChunk.java | 2 + .../dataregion/memtable/WritableMemChunk.java | 38 ++++---- .../db/utils/datastructure/AlignedTVList.java | 10 +++ .../db/utils/datastructure/BinaryTVList.java | 9 ++ .../db/utils/datastructure/BooleanTVList.java | 9 ++ .../db/utils/datastructure/DoubleTVList.java | 9 ++ .../db/utils/datastructure/FloatTVList.java | 9 ++ .../db/utils/datastructure/IntTVList.java | 9 ++ .../db/utils/datastructure/LongTVList.java | 9 ++ .../iotdb/db/utils/datastructure/TVList.java | 2 + .../memtable/PrimitiveMemTableTest.java | 45 ++++++++++ 14 files changed, 227 insertions(+), 56 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/MemTableFlushTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/MemTableFlushTask.java index 48efe55c223d3..0c5ccc9ac1bb1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/MemTableFlushTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/MemTableFlushTask.java @@ -278,6 +278,7 @@ public void run() { times = new long[MAX_NUMBER_OF_POINTS_IN_PAGE]; } writableMemChunk.encode(ioTaskQueue, encodeInfo, times); + writableMemChunk.releaseTemporaryTvListForFlush(); long subTaskTime = System.currentTimeMillis() - starTime; WRITING_METRICS.recordFlushSubTaskCost(WritingMetrics.ENCODING_TASK, subTaskTime); memSerializeTime += subTaskTime; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java index 1b9dbbcae60d7..5167ea96b56a8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java @@ -45,6 +45,8 @@ public abstract class AbstractWritableMemChunk implements IWritableMemChunk { protected static long RETRY_INTERVAL_MS = 100L; protected static long MAX_WAIT_QUERY_MS = 60 * 1000L; + protected TVList workingListForFlush; + /** * Release the TVList if there is no query on it. Otherwise, it should set the first query as the * owner. TVList is released until all queries finish. If it throws memory-not-enough exception @@ -194,7 +196,46 @@ public abstract void writeAlignedTablet( public abstract IMeasurementSchema getSchema(); @Override - public abstract void sortTvListForFlush(); + public void sortTvListForFlush() { + TVList workingList = getWorkingTVList(); + if (workingList.isSorted()) { + workingListForFlush = workingList; + return; + } + + /* + * Concurrency background: + * + * A query may start earlier and record the current row count (rows) of the TVList as its visible range. + * After that, new unseq writes may arrive and immediately trigger a flush, which will sort the TVList. + * + * During sorting, the underlying indices array of the TVList may be reordered. + * If the query continues to use the previously recorded rows as its upper bound, + * it may convert a logical index to a physical index via the updated indices array. + * + * In this case, the converted physical index may exceed the previously visible + * rows range, leading to invalid access or unexpected behavior. + * + * To avoid this issue, when there are active queries on the working TVList, we must + * clone the times and indices before sorting, so that the flush sort does not mutate + * the data structures that concurrent queries rely on. + */ + boolean needCloneTimesAndIndicesInWorkingTVList; + workingList.lockQueryList(); + try { + needCloneTimesAndIndicesInWorkingTVList = !workingList.getQueryContextSet().isEmpty(); + } finally { + workingList.unlockQueryList(); + } + workingListForFlush = + needCloneTimesAndIndicesInWorkingTVList ? workingList.cloneForFlushSort() : workingList; + workingListForFlush.sort(); + } + + @Override + public void releaseTemporaryTvListForFlush() { + workingListForFlush = null; + } @Override public abstract int delete(long lowerBound, long upperBound); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java index 2fca3ca398a51..06bd5404be0f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedWritableMemChunk.java @@ -63,6 +63,7 @@ public class AlignedWritableMemChunk extends AbstractWritableMemChunk { private final Map measurementIndexMap; private List dataTypes; private final List schemaList; + // Note: Use AbstractWritableMemChunk.workingListForFlush instead of list in FlushTask private AlignedTVList list; private List sortedList; private long sortedRowCount = 0; @@ -273,13 +274,6 @@ public long getMinTime() { return minTime; } - @Override - public synchronized void sortTvListForFlush() { - if (!list.isSorted()) { - list.sort(); - } - } - @Override public int delete(long lowerBound, long upperBound) { int deletedNumber = list.delete(lowerBound, upperBound); @@ -322,7 +316,8 @@ public void encodeWorkingAlignedTVList( BlockingQueue ioTaskQueue, long maxNumberOfPointsInChunk, int maxNumberOfPointsInPage) { - BitMap allValueColDeletedMap = list.getAllValueColDeletedMap(); + AlignedTVList alignedWorkingListForFlush = (AlignedTVList) workingListForFlush; + BitMap allValueColDeletedMap = alignedWorkingListForFlush.getAllValueColDeletedMap(); boolean[] timeDuplicateInfo = null; @@ -334,8 +329,10 @@ public void encodeWorkingAlignedTVList( int pointNumInPage = 0; int pointNumInChunk = 0; - for (int sortedRowIndex = 0; sortedRowIndex < list.rowCount(); sortedRowIndex++) { - long time = list.getTime(sortedRowIndex); + for (int sortedRowIndex = 0; + sortedRowIndex < alignedWorkingListForFlush.rowCount(); + sortedRowIndex++) { + long time = alignedWorkingListForFlush.getTime(sortedRowIndex); if (pointNumInPage == 0) { pageRange.add(sortedRowIndex); } @@ -356,14 +353,16 @@ public void encodeWorkingAlignedTVList( } int nextRowIndex = sortedRowIndex + 1; - while (nextRowIndex < list.rowCount() + while (nextRowIndex < alignedWorkingListForFlush.rowCount() && allValueColDeletedMap != null - && allValueColDeletedMap.isMarked(list.getValueIndex(nextRowIndex))) { + && allValueColDeletedMap.isMarked( + alignedWorkingListForFlush.getValueIndex(nextRowIndex))) { nextRowIndex++; } - if (nextRowIndex != list.rowCount() && time == list.getTime(nextRowIndex)) { + if (nextRowIndex != alignedWorkingListForFlush.rowCount() + && time == alignedWorkingListForFlush.getTime(nextRowIndex)) { if (Objects.isNull(timeDuplicateInfo)) { - timeDuplicateInfo = new boolean[list.rowCount()]; + timeDuplicateInfo = new boolean[alignedWorkingListForFlush.rowCount()]; } timeDuplicateInfo[sortedRowIndex] = true; } @@ -371,7 +370,7 @@ public void encodeWorkingAlignedTVList( } if (pointNumInPage != 0) { - pageRange.add(list.rowCount() - 1); + pageRange.add(alignedWorkingListForFlush.rowCount() - 1); } if (pointNumInChunk != 0) { chunkRange.add(pageRange); @@ -387,7 +386,8 @@ private void handleEncoding( boolean[] timeDuplicateInfo, BitMap allValueColDeletedMap, int maxNumberOfPointsInPage) { - List dataTypes = list.getTsDataTypes(); + AlignedTVList alignedWorkingListForFlush = (AlignedTVList) workingListForFlush; + List dataTypes = alignedWorkingListForFlush.getTsDataTypes(); Pair[] lastValidPointIndexForTimeDupCheck = new Pair[dataTypes.size()]; for (List pageRange : chunkRange) { AlignedChunkWriterImpl alignedChunkWriter = new AlignedChunkWriterImpl(schemaList); @@ -404,16 +404,18 @@ private void handleEncoding( sortedRowIndex++) { // skip empty row if (allValueColDeletedMap != null - && allValueColDeletedMap.isMarked(list.getValueIndex(sortedRowIndex))) { + && allValueColDeletedMap.isMarked( + alignedWorkingListForFlush.getValueIndex(sortedRowIndex))) { continue; } // skip time duplicated rows - long time = list.getTime(sortedRowIndex); + long time = alignedWorkingListForFlush.getTime(sortedRowIndex); if (Objects.nonNull(timeDuplicateInfo)) { - if (!list.isNullValue(list.getValueIndex(sortedRowIndex), columnIndex)) { + if (!alignedWorkingListForFlush.isNullValue( + alignedWorkingListForFlush.getValueIndex(sortedRowIndex), columnIndex)) { lastValidPointIndexForTimeDupCheck[columnIndex].left = time; lastValidPointIndexForTimeDupCheck[columnIndex].right = - list.getValueIndex(sortedRowIndex); + alignedWorkingListForFlush.getValueIndex(sortedRowIndex); } if (timeDuplicateInfo[sortedRowIndex]) { continue; @@ -434,41 +436,55 @@ private void handleEncoding( && (time == lastValidPointIndexForTimeDupCheck[columnIndex].left)) { originRowIndex = lastValidPointIndexForTimeDupCheck[columnIndex].right; } else { - originRowIndex = list.getValueIndex(sortedRowIndex); + originRowIndex = alignedWorkingListForFlush.getValueIndex(sortedRowIndex); } - boolean isNull = list.isNullValue(originRowIndex, columnIndex); + boolean isNull = alignedWorkingListForFlush.isNullValue(originRowIndex, columnIndex); switch (tsDataType) { case BOOLEAN: alignedChunkWriter.writeByColumn( time, - !isNull && list.getBooleanByValueIndex(originRowIndex, columnIndex), + !isNull + && alignedWorkingListForFlush.getBooleanByValueIndex( + originRowIndex, columnIndex), isNull); break; case INT32: case DATE: alignedChunkWriter.writeByColumn( time, - isNull ? 0 : list.getIntByValueIndex(originRowIndex, columnIndex), + isNull + ? 0 + : alignedWorkingListForFlush.getIntByValueIndex( + originRowIndex, columnIndex), isNull); break; case INT64: case TIMESTAMP: alignedChunkWriter.writeByColumn( time, - isNull ? 0 : list.getLongByValueIndex(originRowIndex, columnIndex), + isNull + ? 0 + : alignedWorkingListForFlush.getLongByValueIndex( + originRowIndex, columnIndex), isNull); break; case FLOAT: alignedChunkWriter.writeByColumn( time, - isNull ? 0 : list.getFloatByValueIndex(originRowIndex, columnIndex), + isNull + ? 0 + : alignedWorkingListForFlush.getFloatByValueIndex( + originRowIndex, columnIndex), isNull); break; case DOUBLE: alignedChunkWriter.writeByColumn( time, - isNull ? 0 : list.getDoubleByValueIndex(originRowIndex, columnIndex), + isNull + ? 0 + : alignedWorkingListForFlush.getDoubleByValueIndex( + originRowIndex, columnIndex), isNull); break; case TEXT: @@ -476,7 +492,10 @@ private void handleEncoding( case BLOB: alignedChunkWriter.writeByColumn( time, - isNull ? null : list.getBinaryByValueIndex(originRowIndex, columnIndex), + isNull + ? null + : alignedWorkingListForFlush.getBinaryByValueIndex( + originRowIndex, columnIndex), isNull); break; default: @@ -486,18 +505,20 @@ private void handleEncoding( alignedChunkWriter.nextColumn(); } - long[] times = new long[Math.min(maxNumberOfPointsInPage, list.rowCount())]; + long[] times = + new long[Math.min(maxNumberOfPointsInPage, alignedWorkingListForFlush.rowCount())]; int pointsInPage = 0; for (int sortedRowIndex = pageRange.get(pageNum * 2); sortedRowIndex <= pageRange.get(pageNum * 2 + 1); sortedRowIndex++) { // skip empty row if (allValueColDeletedMap != null - && allValueColDeletedMap.isMarked(list.getValueIndex(sortedRowIndex))) { + && allValueColDeletedMap.isMarked( + alignedWorkingListForFlush.getValueIndex(sortedRowIndex))) { continue; } if (Objects.isNull(timeDuplicateInfo) || !timeDuplicateInfo[sortedRowIndex]) { - times[pointsInPage++] = list.getTime(sortedRowIndex); + times[pointsInPage++] = alignedWorkingListForFlush.getTime(sortedRowIndex); } } alignedChunkWriter.write(times, pointsInPage, 0); @@ -513,8 +534,7 @@ private void handleEncoding( } @Override - public synchronized void encode( - BlockingQueue ioTaskQueue, BatchEncodeInfo encodeInfo, long[] times) { + public void encode(BlockingQueue ioTaskQueue, BatchEncodeInfo encodeInfo, long[] times) { if (TVLIST_SORT_THRESHOLD == 0) { encodeWorkingAlignedTVList( ioTaskQueue, encodeInfo.maxNumberOfPointsInChunk, encodeInfo.maxNumberOfPointsInPage); @@ -525,7 +545,7 @@ public synchronized void encode( // create MergeSortAlignedTVListIterator. List alignedTvLists = new ArrayList<>(sortedList); - alignedTvLists.add(list); + alignedTvLists.add((AlignedTVList) workingListForFlush); List columnIndexList = buildColumnIndexList(schemaList); MemPointIterator timeValuePairIterator = MemPointIteratorFactory.create( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index 18ca5201bb049..994045024375f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -94,6 +94,8 @@ void writeAlignedTablet( */ void sortTvListForFlush(); + void releaseTemporaryTvListForFlush(); + default long getMaxTime() { return Long.MAX_VALUE; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java index fd616ac94fa4a..90e96597b0cd2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunk.java @@ -54,6 +54,7 @@ public class WritableMemChunk extends AbstractWritableMemChunk { private IMeasurementSchema schema; + // Note: Use AbstractWritableMemChunk.workingListForFlush instead of list in FlushTask private TVList list; private List sortedList; private long sortedRowCount = 0; @@ -241,13 +242,6 @@ public void putAlignedTablet(long[] t, Object[] v, BitMap[] bitMaps, int start, throw new UnSupportedDataTypeException(UNSUPPORTED_TYPE + schema.getType()); } - @Override - public synchronized void sortTvListForFlush() { - if (!list.isSorted()) { - list.sort(); - } - } - @Override public TVList getWorkingTVList() { return list; @@ -374,50 +368,53 @@ public void encodeWorkingTVList( ChunkWriterImpl chunkWriterImpl = createIChunkWriter(); long dataSizeInCurrentChunk = 0; int pointNumInCurrentChunk = 0; - for (int sortedRowIndex = 0; sortedRowIndex < list.rowCount(); sortedRowIndex++) { - if (list.isNullValue(list.getValueIndex(sortedRowIndex))) { + for (int sortedRowIndex = 0; + sortedRowIndex < workingListForFlush.rowCount(); + sortedRowIndex++) { + if (workingListForFlush.isNullValue(workingListForFlush.getValueIndex(sortedRowIndex))) { continue; } - long time = list.getTime(sortedRowIndex); + long time = workingListForFlush.getTime(sortedRowIndex); // skip duplicated data - if ((sortedRowIndex + 1 < list.rowCount() && (time == list.getTime(sortedRowIndex + 1)))) { + if ((sortedRowIndex + 1 < workingListForFlush.rowCount() + && (time == workingListForFlush.getTime(sortedRowIndex + 1)))) { continue; } // store last point for SDT - if (sortedRowIndex + 1 == list.rowCount()) { + if (sortedRowIndex + 1 == workingListForFlush.rowCount()) { chunkWriterImpl.setLastPoint(true); } switch (tsDataType) { case BOOLEAN: - chunkWriterImpl.write(time, list.getBoolean(sortedRowIndex)); + chunkWriterImpl.write(time, workingListForFlush.getBoolean(sortedRowIndex)); dataSizeInCurrentChunk += 8L + 1L; break; case INT32: case DATE: - chunkWriterImpl.write(time, list.getInt(sortedRowIndex)); + chunkWriterImpl.write(time, workingListForFlush.getInt(sortedRowIndex)); dataSizeInCurrentChunk += 8L + 4L; break; case INT64: case TIMESTAMP: - chunkWriterImpl.write(time, list.getLong(sortedRowIndex)); + chunkWriterImpl.write(time, workingListForFlush.getLong(sortedRowIndex)); dataSizeInCurrentChunk += 8L + 8L; break; case FLOAT: - chunkWriterImpl.write(time, list.getFloat(sortedRowIndex)); + chunkWriterImpl.write(time, workingListForFlush.getFloat(sortedRowIndex)); dataSizeInCurrentChunk += 8L + 4L; break; case DOUBLE: - chunkWriterImpl.write(time, list.getDouble(sortedRowIndex)); + chunkWriterImpl.write(time, workingListForFlush.getDouble(sortedRowIndex)); dataSizeInCurrentChunk += 8L + 8L; break; case TEXT: case BLOB: case STRING: - Binary value = list.getBinary(sortedRowIndex); + Binary value = workingListForFlush.getBinary(sortedRowIndex); chunkWriterImpl.write(time, value); dataSizeInCurrentChunk += 8L + getBinarySize(value); break; @@ -452,8 +449,7 @@ public void encodeWorkingTVList( } @Override - public synchronized void encode( - BlockingQueue ioTaskQueue, BatchEncodeInfo encodeInfo, long[] times) { + public void encode(BlockingQueue ioTaskQueue, BatchEncodeInfo encodeInfo, long[] times) { if (TVLIST_SORT_THRESHOLD == 0) { encodeWorkingTVList( ioTaskQueue, encodeInfo.maxNumberOfPointsInChunk, encodeInfo.targetChunkSize); @@ -467,7 +463,7 @@ public synchronized void encode( // create MultiTvListIterator. It need not handle float/double precision here. List tvLists = new ArrayList<>(sortedList); - tvLists.add(list); + tvLists.add(workingListForFlush); MemPointIterator timeValuePairIterator = MemPointIteratorFactory.create( schema.getType(), tvLists, encodeInfo.maxNumberOfPointsInPage); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java index 615bfcaf4cab0..b99e784faadb0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/AlignedTVList.java @@ -137,6 +137,16 @@ public TVList getTvListByColumnIndex(List columnIndex, List return alignedTvList; } + @Override + public synchronized AlignedTVList cloneForFlushSort() { + AlignedTVList cloneList = AlignedTVList.newAlignedList(new ArrayList<>(dataTypes)); + cloneAs(cloneList); + cloneList.memoryBinaryChunkSize = this.memoryBinaryChunkSize; + cloneList.values = this.values; + cloneList.bitMaps = this.bitMaps; + return cloneList; + } + @Override public synchronized AlignedTVList clone() { AlignedTVList cloneList = AlignedTVList.newAlignedList(new ArrayList<>(dataTypes)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java index dc4ff5529d45b..b274bc8ef6586 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BinaryTVList.java @@ -63,6 +63,15 @@ public static BinaryTVList newList() { } } + @Override + public synchronized TVList cloneForFlushSort() { + BinaryTVList cloneList = BinaryTVList.newList(); + cloneAs(cloneList); + cloneList.bitMap = this.bitMap; + cloneList.values = this.values; + return cloneList; + } + @Override public synchronized BinaryTVList clone() { BinaryTVList cloneList = BinaryTVList.newList(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java index b8eb0e508bfe8..5e3461acb2c8e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/BooleanTVList.java @@ -62,6 +62,15 @@ public static BooleanTVList newList() { } } + @Override + public synchronized TVList cloneForFlushSort() { + BooleanTVList cloneList = BooleanTVList.newList(); + cloneAs(cloneList); + cloneList.bitMap = this.bitMap; + cloneList.values = this.values; + return cloneList; + } + @Override public synchronized BooleanTVList clone() { BooleanTVList cloneList = BooleanTVList.newList(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java index f61995ef0628a..753ca2020a89f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/DoubleTVList.java @@ -63,6 +63,15 @@ public static DoubleTVList newList() { } } + @Override + public synchronized TVList cloneForFlushSort() { + DoubleTVList cloneList = DoubleTVList.newList(); + cloneAs(cloneList); + cloneList.bitMap = this.bitMap; + cloneList.values = this.values; + return cloneList; + } + @Override public synchronized DoubleTVList clone() { DoubleTVList cloneList = DoubleTVList.newList(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java index 3623fa49a3efe..517dc208211fb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/FloatTVList.java @@ -63,6 +63,15 @@ public static FloatTVList newList() { } } + @Override + public synchronized TVList cloneForFlushSort() { + FloatTVList cloneList = FloatTVList.newList(); + cloneAs(cloneList); + cloneList.bitMap = this.bitMap; + cloneList.values = this.values; + return cloneList; + } + @Override public synchronized FloatTVList clone() { FloatTVList cloneList = FloatTVList.newList(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java index 758cd64053bc5..8593400fa79c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/IntTVList.java @@ -62,6 +62,15 @@ public static IntTVList newList() { } } + @Override + public synchronized TVList cloneForFlushSort() { + IntTVList cloneList = IntTVList.newList(); + cloneAs(cloneList); + cloneList.bitMap = this.bitMap; + cloneList.values = this.values; + return cloneList; + } + @Override public synchronized IntTVList clone() { IntTVList cloneList = IntTVList.newList(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java index 7b4bd8d82d264..544d2cb7dd0eb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/LongTVList.java @@ -62,6 +62,15 @@ public static LongTVList newList() { } } + @Override + public synchronized TVList cloneForFlushSort() { + LongTVList cloneList = LongTVList.newList(); + cloneAs(cloneList); + cloneList.bitMap = this.bitMap; + cloneList.values = this.values; + return cloneList; + } + @Override public synchronized LongTVList clone() { LongTVList cloneList = LongTVList.newList(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java index b4fd0d52221e1..d21ecc83e0f1b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java @@ -548,6 +548,8 @@ public long getVersion() { protected abstract void expandValues(); + public abstract TVList cloneForFlushSort(); + @Override public abstract TVList clone(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java index ff476c60e4e5d..b366a48ab7797 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java @@ -177,6 +177,51 @@ public void testWriteDuringPrepareTVListAndActualQueryExecution() } } + @Test + public void testWriteAndFlushSortDuringQuerySortTVListAndActualQueryExecution() + throws QueryProcessException, IOException, IllegalPathException { + + PrimitiveMemTable memTable = new PrimitiveMemTable("root.test", "0"); + List measurementSchemas = + Arrays.asList( + new MeasurementSchema("s1", TSDataType.INT32), + new MeasurementSchema("s2", TSDataType.INT32), + new MeasurementSchema("s3", TSDataType.INT32)); + for (int i = 1000; i < 2000; i++) { + memTable.writeAlignedRow( + new PlainDeviceID("root.test.d1"), measurementSchemas, i, new Object[] {i, i, i}); + } + for (int i = 100; i < 200; i++) { + memTable.writeAlignedRow( + new PlainDeviceID("root.test.d1"), measurementSchemas, i, new Object[] {i, i, i}); + } + MeasurementPath path = new MeasurementPath("root.test.d1.s1", TSDataType.INT32); + memTable.delete(path, path.getDevicePath(), 150, 160); + path = new MeasurementPath("root.test.d1.s2", TSDataType.INT32); + memTable.delete(path, path.getDevicePath(), 150, 160); + path = new MeasurementPath("root.test.d1.s3", TSDataType.INT32); + memTable.delete(path, path.getDevicePath(), 150, 160); + ResourceByPathUtils resourcesByPathUtils = + ResourceByPathUtils.getResourceInstance( + new AlignedPath("root.test.d1", Arrays.asList("s1", "s2", "s3"), measurementSchemas)); + ReadOnlyMemChunk readOnlyMemChunk = + resourcesByPathUtils.getReadOnlyMemChunkFromMemTable( + new QueryContext(), memTable, null, Long.MAX_VALUE, null); + + for (int i = 1; i <= 50; i++) { + memTable.writeAlignedRow( + new PlainDeviceID("root.test.d1"), measurementSchemas, i, new Object[] {i, i, i}); + } + memTable.getWritableMemChunk(new PlainDeviceID("root.test.d1"), "").sortTvListForFlush(); + + readOnlyMemChunk.sortTvLists(); + + MemPointIterator memPointIterator = readOnlyMemChunk.createMemPointIterator(Ordering.ASC, null); + while (memPointIterator.hasNextBatch()) { + memPointIterator.nextBatch(); + } + } + @Test public void memSeriesToStringTest() throws IOException { TSDataType dataType = TSDataType.INT32; From 193271555b9926a1ef0819cd56a6e9fa0066a5fd Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 14 May 2026 21:46:03 +0800 Subject: [PATCH 048/102] [To dev/1.3] Fixed the bug that the unclosed file may be degraded when directly deleted (#17024) (#17660) --- .../db/storageengine/dataregion/tsfile/TsFileResource.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 8571fd2e91528..9f344c1994bec 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -639,11 +639,13 @@ public void removeModFile() throws IOException { * file physically. */ public boolean remove() { - forceMarkDeleted(); // To release the memory occupied by pipe if held by it // Note that pipe can safely handle the case that the time index does not exist isEmpty(); - degradeTimeIndex(); + if (getStatus() != TsFileResourceStatus.UNCLOSED) { + degradeTimeIndex(); + } + forceMarkDeleted(); try { fsFactory.deleteIfExists(file); fsFactory.deleteIfExists( From 4593bd471d4918264624057da32c9758c14d151a Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 15 May 2026 13:18:32 +0800 Subject: [PATCH 049/102] Pipe: Do not wait for sealed tsFiles close at realtime (#17671) (#17681) --- .../tsfile/PipeTsFileInsertionEvent.java | 56 ++++++++++++- .../event/PipeTsFileInsertionEventTest.java | 80 +++++++++++++++++++ 2 files changed, 133 insertions(+), 3 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index f099301a63cdd..53f0b16826ab3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -78,6 +78,10 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent protected final boolean isGeneratedByPipeConsensus; protected final boolean isGeneratedByHistoricalExtractor; + // Realtime TsFile events are created after TsFileProcessor#endFile(), so the file is already + // immutable even if TsFileResource status is still UNCLOSED. + private final boolean isTsFileSealed; + protected final AtomicBoolean isClosed; protected final AtomicReference dataContainer; @@ -91,7 +95,18 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent public PipeTsFileInsertionEvent(final TsFileResource resource, final boolean isLoaded) { // The modFile must be copied before the event is assigned to the listening pipes this( - resource, null, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); + resource, + null, + true, + isLoaded, + false, + null, + 0, + null, + null, + Long.MIN_VALUE, + Long.MAX_VALUE, + true); } public PipeTsFileInsertionEvent( @@ -106,7 +121,36 @@ public PipeTsFileInsertionEvent( final PipePattern pattern, final long startTime, final long endTime) { - super(pipeName, creationTime, pipeTaskMeta, pattern, startTime, endTime); + this( + resource, + tsFile, + isWithMod, + isLoaded, + isGeneratedByHistoricalExtractor, + pipeName, + creationTime, + pipeTaskMeta, + pattern, + startTime, + endTime, + false); + } + + private PipeTsFileInsertionEvent( + final TsFileResource resource, + final File tsFile, + final boolean isWithMod, + final boolean isLoaded, + final boolean isGeneratedByHistoricalExtractor, + final String pipeName, + final long creationTime, + final PipeTaskMeta pipeTaskMeta, + final PipePattern pipePattern, + final long startTime, + final long endTime, + final boolean isTsFileSealed) { + super(pipeName, creationTime, pipeTaskMeta, pipePattern, startTime, endTime); + this.resource = resource; // For events created at assigner or historical extractor, the tsFile is get from the resource @@ -123,6 +167,7 @@ public PipeTsFileInsertionEvent( this.isGeneratedByPipe = resource.isGeneratedByPipe(); this.isGeneratedByPipeConsensus = resource.isGeneratedByPipeConsensus(); this.isGeneratedByHistoricalExtractor = isGeneratedByHistoricalExtractor; + this.isTsFileSealed = isTsFileSealed; this.tableNames = tableNames; this.dataContainer = new AtomicReference<>(null); @@ -180,6 +225,10 @@ public boolean waitForTsFileClose() throws InterruptedException { return true; } + if (isTsFileSealed) { + return !resource.isEmpty(); + } + if (!isClosed.get()) { isClosed.set(resource.isClosed()); @@ -359,7 +408,8 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep pipeTaskMeta, pattern, startTime, - endTime); + endTime, + isTsFileSealed); } @Override diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java new file mode 100644 index 0000000000000..fd824d2d25712 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.event; + +import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.DeviceTimeIndex; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.ITimeIndex; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.PlainDeviceID; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; + +public class PipeTsFileInsertionEventTest { + + @Test(timeout = 5000) + public void testRealtimeEventCanSkipWaitingForClosedStatusAfterTsFileSealed() throws Exception { + final File tempDir = Files.createTempDirectory("pipeTsFileSealed").toFile(); + + try { + final TsFileResource resource = + createNonEmptyTsFileResource(tempDir, "realtime.tsfile", 1L, 1); + Assert.assertFalse(resource.isClosed()); + Assert.assertFalse(resource.isEmpty()); + + final PipeTsFileInsertionEvent sourceEvent = new PipeTsFileInsertionEvent(resource, false); + Assert.assertTrue(sourceEvent.waitForTsFileClose()); + + final PipeTsFileInsertionEvent copiedEvent = + sourceEvent.shallowCopySelfAndBindPipeTaskMetaForProgressReport( + "pipe", 1L, null, null, Long.MIN_VALUE, Long.MAX_VALUE); + Assert.assertTrue(copiedEvent.waitForTsFileClose()); + + copiedEvent.close(); + sourceEvent.close(); + } finally { + FileUtils.deleteFileOrDirectory(tempDir); + } + } + + private TsFileResource createNonEmptyTsFileResource( + final File tempDir, final String fileName, final long flushOrderId, final int dataRegionId) + throws IOException { + final File file = new File(tempDir, fileName); + Assert.assertTrue(file.createNewFile()); + + final TsFileResource resource = new TsFileResource(file); + resource.updateProgressIndex(new SimpleProgressIndex(1, flushOrderId)); + final ITimeIndex timeIndex = new DeviceTimeIndex(); + final IDeviceID deviceID = new PlainDeviceID("root.db.d" + dataRegionId); + timeIndex.putStartTime(deviceID, 1); + timeIndex.putEndTime(deviceID, 1); + resource.setTimeIndex(timeIndex); + return resource; + } +} From e05d0bdaf0d61e165a925ca11797c9276cf760e3 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 15 May 2026 13:18:47 +0800 Subject: [PATCH 050/102] Pipe: Implemented slice logic for async sink (#17668) (#17679) * sl * chew --- .../PipeTransferTabletBatchEventHandler.java | 2 +- ...eTransferTabletInsertNodeEventHandler.java | 2 +- .../PipeTransferTabletRawEventHandler.java | 2 +- .../handler/PipeTransferTrackableHandler.java | 127 ++++++++++ .../handler/PipeTransferTsFileHandler.java | 2 +- .../PipeTransferTrackableHandlerTest.java | 221 ++++++++++++++++++ .../AsyncPipeDataTransferServiceClient.java | 4 + .../pipe/sink/client/IoTDBSyncClient.java | 31 +-- .../common/PipeTransferSliceReqBuilder.java | 73 ++++++ .../PipeTransferSliceReqBuilderTest.java | 106 +++++++++ 10 files changed, 542 insertions(+), 28 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandlerTest.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferSliceReqBuilder.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferSliceReqBuilderTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java index 8bcb9d47009c1..d5ee15c6367ba 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletBatchEventHandler.java @@ -129,7 +129,7 @@ protected void onErrorInternal(final Exception exception) { protected void doTransfer( final AsyncPipeDataTransferServiceClient client, final TPipeTransferReq req) throws TException { - client.pipeTransfer(req, this); + transferWithOptionalRequestSlicing(client, req); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java index 912a1e724f748..56d1ce41b029c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletInsertNodeEventHandler.java @@ -41,7 +41,7 @@ public PipeTransferTabletInsertNodeEventHandler( protected void doTransfer( final AsyncPipeDataTransferServiceClient client, final TPipeTransferReq req) throws TException { - client.pipeTransfer(req, this); + transferWithOptionalRequestSlicing(client, req); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletRawEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletRawEventHandler.java index b64e446827aff..eb4677de358e9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletRawEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTabletRawEventHandler.java @@ -40,7 +40,7 @@ public PipeTransferTabletRawEventHandler( protected void doTransfer( final AsyncPipeDataTransferServiceClient client, final TPipeTransferReq req) throws TException { - client.pipeTransfer(req, this); + transferWithOptionalRequestSlicing(client, req); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java index a8b4a3b7a79a7..a0e6ad73fe7ca 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java @@ -21,7 +21,11 @@ import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.common.PipeTransferSliceReqBuilder; import org.apache.iotdb.db.pipe.sink.protocol.thrift.async.IoTDBDataRegionAsyncSink; +import org.apache.iotdb.pipe.api.exception.PipeConnectionException; +import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; @@ -126,8 +130,131 @@ protected abstract void doTransfer( final AsyncPipeDataTransferServiceClient client, final TPipeTransferReq req) throws TException; + protected final void transferWithOptionalRequestSlicing( + final AsyncPipeDataTransferServiceClient client, final TPipeTransferReq req) + throws TException { + final int bodySizeLimit = PipeTransferSliceReqBuilder.getBodySizeLimit(); + if (!PipeTransferSliceReqBuilder.shouldSlice(req, bodySizeLimit)) { + client.pipeTransfer(req, this); + return; + } + + LOGGER.warn( + "The body size of the request is too large. The request will be sliced. Origin req: {}-{}. " + + "Request body size: {}, threshold: {}", + req.getVersion(), + req.getType(), + req.body.limit(), + bodySizeLimit); + + final int sliceCount = PipeTransferSliceReqBuilder.getSliceCount(req, bodySizeLimit); + final boolean shouldReturnSelf = client.shouldReturnSelf(); + try { + transferSlicedRequest( + client, + req, + shouldReturnSelf, + PipeTransferSliceReqBuilder.nextSliceOrderId(), + 0, + sliceCount, + bodySizeLimit); + } catch (final Exception e) { + fallbackToWholeRequest(client, req, shouldReturnSelf, e); + } + } + public abstract void clearEventsReferenceCount(); + private void transferSlicedRequest( + final AsyncPipeDataTransferServiceClient client, + final TPipeTransferReq originalReq, + final boolean shouldReturnSelf, + final int sliceOrderId, + final int sliceIndex, + final int sliceCount, + final int bodySizeLimit) + throws Exception { + client.setShouldReturnSelf(shouldReturnSelf && sliceIndex == sliceCount - 1); + client.pipeTransfer( + PipeTransferSliceReqBuilder.buildSliceReq( + originalReq, sliceOrderId, sliceIndex, sliceCount, bodySizeLimit), + new AsyncMethodCallback() { + @Override + public void onComplete(final TPipeTransferResp response) { + if (sink.isClosed() || sliceIndex == sliceCount - 1) { + PipeTransferTrackableHandler.this.onComplete(response); + return; + } + + if (response == null) { + fallbackToWholeRequest( + client, + originalReq, + shouldReturnSelf, + new PipeException("TPipeTransferResp is null when transferring slice.")); + return; + } + + if (response.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + fallbackToWholeRequest( + client, + originalReq, + shouldReturnSelf, + new PipeConnectionException( + String.format( + "Failed to transfer slice. Origin req: %s-%s, slice index: %d, slice count: %d. Reason: %s", + originalReq.getVersion(), + originalReq.getType(), + sliceIndex, + sliceCount, + response.getStatus()))); + return; + } + + try { + transferSlicedRequest( + client, + originalReq, + shouldReturnSelf, + sliceOrderId, + sliceIndex + 1, + sliceCount, + bodySizeLimit); + } catch (final Exception e) { + fallbackToWholeRequest(client, originalReq, shouldReturnSelf, e); + } + } + + @Override + public void onError(final Exception exception) { + if (sink.isClosed() || sliceIndex == sliceCount - 1) { + PipeTransferTrackableHandler.this.onError(exception); + return; + } + fallbackToWholeRequest(client, originalReq, shouldReturnSelf, exception); + } + }); + } + + private void fallbackToWholeRequest( + final AsyncPipeDataTransferServiceClient client, + final TPipeTransferReq originalReq, + final boolean shouldReturnSelf, + final Exception exception) { + LOGGER.warn( + "Failed to transfer slice. Origin req: {}-{}. Retry the whole transfer.", + originalReq.getVersion(), + originalReq.getType(), + exception); + + try { + client.setShouldReturnSelf(shouldReturnSelf); + client.pipeTransfer(originalReq, this); + } catch (final Exception e) { + PipeTransferTrackableHandler.this.onError(e); + } + } + public void closeClient() { if (Objects.isNull(client)) { return; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java index 6742199ac1043..c79a09ec2397b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java @@ -441,7 +441,7 @@ protected void doTransfer( return; } - client.pipeTransfer(req, this); + transferWithOptionalRequestSlicing(client, req); } @Override diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandlerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandlerTest.java new file mode 100644 index 0000000000000..60b6923508548 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandlerTest.java @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.sink.protocol.thrift.async.handler; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.conf.CommonConfig; +import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.IoTDBSinkRequestVersion; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeRequestType; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferSliceReq; +import org.apache.iotdb.db.pipe.sink.protocol.thrift.async.IoTDBDataRegionAsyncSink; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; + +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class PipeTransferTrackableHandlerTest { + + private final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig(); + + private int originalRequestSliceThresholdBytes; + + @Before + public void setUp() { + originalRequestSliceThresholdBytes = commonConfig.getPipeSinkRequestSliceThresholdBytes(); + commonConfig.setPipeSinkRequestSliceThresholdBytes(4); + } + + @After + public void tearDown() { + commonConfig.setPipeSinkRequestSliceThresholdBytes(originalRequestSliceThresholdBytes); + } + + @Test + public void testLargeRequestWillBeSlicedForAsyncTransfer() throws Exception { + final IoTDBDataRegionAsyncSink sink = Mockito.mock(IoTDBDataRegionAsyncSink.class); + final AsyncPipeDataTransferServiceClient client = + Mockito.mock(AsyncPipeDataTransferServiceClient.class); + Mockito.when(client.shouldReturnSelf()).thenReturn(true); + + final List transferredRequests = new ArrayList<>(); + Mockito.doAnswer( + invocation -> { + final TPipeTransferReq req = invocation.getArgument(0); + final AsyncMethodCallback callback = invocation.getArgument(1); + transferredRequests.add(req); + callback.onComplete(successResp()); + return null; + }) + .when(client) + .pipeTransfer(Mockito.any(TPipeTransferReq.class), Mockito.any()); + + final TestPipeTransferTrackableHandler handler = new TestPipeTransferTrackableHandler(sink); + final TPipeTransferReq originalReq = createReq(10); + + handler.transfer(client, originalReq); + + Assert.assertEquals(3, transferredRequests.size()); + Assert.assertEquals(1, handler.completeCount); + Assert.assertEquals(0, handler.errorCount); + + final PipeTransferSliceReq firstSlice = + PipeTransferSliceReq.fromTPipeTransferReq(transferredRequests.get(0)); + final PipeTransferSliceReq secondSlice = + PipeTransferSliceReq.fromTPipeTransferReq(transferredRequests.get(1)); + final PipeTransferSliceReq thirdSlice = + PipeTransferSliceReq.fromTPipeTransferReq(transferredRequests.get(2)); + + Assert.assertEquals( + PipeRequestType.TRANSFER_SLICE.getType(), transferredRequests.get(0).getType()); + Assert.assertEquals(firstSlice.getOrderId(), secondSlice.getOrderId()); + Assert.assertEquals(firstSlice.getOrderId(), thirdSlice.getOrderId()); + Assert.assertEquals(originalReq.getType(), firstSlice.getOriginReqType()); + Assert.assertEquals(10, firstSlice.getOriginBodySize()); + Assert.assertEquals(3, firstSlice.getSliceCount()); + Assert.assertEquals(0, firstSlice.getSliceIndex()); + Assert.assertEquals(1, secondSlice.getSliceIndex()); + Assert.assertEquals(2, thirdSlice.getSliceIndex()); + Assert.assertEquals(4, firstSlice.getSliceBody().length); + Assert.assertEquals(4, secondSlice.getSliceBody().length); + Assert.assertEquals(2, thirdSlice.getSliceBody().length); + + final ArgumentCaptor shouldReturnSelfCaptor = ArgumentCaptor.forClass(Boolean.class); + Mockito.verify(client, Mockito.times(3)).setShouldReturnSelf(shouldReturnSelfCaptor.capture()); + Assert.assertEquals(Arrays.asList(false, false, true), shouldReturnSelfCaptor.getAllValues()); + } + + @Test + public void testLargeRequestFallsBackToWholeRequestWhenSliceTransferFails() throws Exception { + final IoTDBDataRegionAsyncSink sink = Mockito.mock(IoTDBDataRegionAsyncSink.class); + final AsyncPipeDataTransferServiceClient client = + Mockito.mock(AsyncPipeDataTransferServiceClient.class); + Mockito.when(client.shouldReturnSelf()).thenReturn(true); + + final List transferredRequests = new ArrayList<>(); + Mockito.doAnswer( + invocation -> { + final TPipeTransferReq req = invocation.getArgument(0); + final AsyncMethodCallback callback = invocation.getArgument(1); + transferredRequests.add(req); + if (req.getType() == PipeRequestType.TRANSFER_SLICE.getType()) { + callback.onComplete(failedResp()); + } else { + callback.onComplete(successResp()); + } + return null; + }) + .when(client) + .pipeTransfer(Mockito.any(TPipeTransferReq.class), Mockito.any()); + + final TestPipeTransferTrackableHandler handler = new TestPipeTransferTrackableHandler(sink); + final TPipeTransferReq originalReq = createReq(10); + + handler.transfer(client, originalReq); + + Assert.assertEquals(2, transferredRequests.size()); + Assert.assertEquals( + PipeRequestType.TRANSFER_SLICE.getType(), transferredRequests.get(0).getType()); + Assert.assertEquals(originalReq.getType(), transferredRequests.get(1).getType()); + Assert.assertEquals(originalReq.getVersion(), transferredRequests.get(1).getVersion()); + Assert.assertArrayEquals(originalReq.getBody(), transferredRequests.get(1).getBody()); + Assert.assertEquals(1, handler.completeCount); + Assert.assertEquals(0, handler.errorCount); + } + + private static TPipeTransferReq createReq(final int bodySize) { + final byte[] body = new byte[bodySize]; + for (int i = 0; i < body.length; ++i) { + body[i] = (byte) i; + } + + final TPipeTransferReq req = new TPipeTransferReq(); + req.version = IoTDBSinkRequestVersion.VERSION_1.getVersion(); + req.type = (short) 123; + req.body = ByteBuffer.wrap(body); + return req; + } + + private static TPipeTransferResp successResp() { + final TPipeTransferResp resp = new TPipeTransferResp(); + resp.setStatus(new TSStatus().setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + return resp; + } + + private static TPipeTransferResp failedResp() { + final TPipeTransferResp resp = new TPipeTransferResp(); + resp.setStatus( + new TSStatus().setCode(TSStatusCode.PIPE_TRANSFER_SLICE_OUT_OF_ORDER.getStatusCode())); + return resp; + } + + private static class TestPipeTransferTrackableHandler extends PipeTransferTrackableHandler { + + private int completeCount; + private int errorCount; + + private TestPipeTransferTrackableHandler(final IoTDBDataRegionAsyncSink sink) { + super(sink); + } + + private void transfer( + final AsyncPipeDataTransferServiceClient client, final TPipeTransferReq req) + throws TException { + tryTransfer(client, req); + } + + @Override + protected boolean onCompleteInternal(final TPipeTransferResp response) { + completeCount++; + return true; + } + + @Override + protected void onErrorInternal(final Exception exception) { + errorCount++; + } + + @Override + protected void doTransfer( + final AsyncPipeDataTransferServiceClient client, final TPipeTransferReq req) + throws TException { + transferWithOptionalRequestSlicing(client, req); + } + + @Override + public void clearEventsReferenceCount() { + // Do nothing + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java index 41e8c59511a6f..2cdfbd865c817 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/client/async/AsyncPipeDataTransferServiceClient.java @@ -135,6 +135,10 @@ public void setShouldReturnSelf(final boolean shouldReturnSelf) { this.shouldReturnSelf.set(shouldReturnSelf); } + public boolean shouldReturnSelf() { + return shouldReturnSelf.get(); + } + public void setTimeoutDynamically(final int timeout) { try { ((TNonblockingSocket) ___transport).setTimeout(timeout); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBSyncClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBSyncClient.java index b7f42295e6cc3..1ad5d0a855f23 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBSyncClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/client/IoTDBSyncClient.java @@ -22,9 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.property.ThriftClientProperty; -import org.apache.iotdb.commons.pipe.config.PipeConfig; -import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.IoTDBSinkRequestVersion; -import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferSliceReq; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.common.PipeTransferSliceReqBuilder; import org.apache.iotdb.pipe.api.exception.PipeConnectionException; import org.apache.iotdb.rpc.DeepCopyRpcTransportFactory; import org.apache.iotdb.rpc.TSStatusCode; @@ -39,15 +37,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.atomic.AtomicInteger; - public class IoTDBSyncClient extends IClientRPCService.Client implements ThriftClient, AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBSyncClient.class); - private static final AtomicInteger SLICE_ORDER_ID_GENERATOR = new AtomicInteger(0); - private final String ipAddress; private final int port; private final TEndPoint endPoint; @@ -100,9 +94,8 @@ public void setTimeout(int timeout) { @Override public TPipeTransferResp pipeTransfer(final TPipeTransferReq req) throws TException { - final int bodySizeLimit = PipeConfig.getInstance().getPipeSinkRequestSliceThresholdBytes(); - if (req.getVersion() != IoTDBSinkRequestVersion.VERSION_1.getVersion() - || req.body.limit() < bodySizeLimit) { + final int bodySizeLimit = PipeTransferSliceReqBuilder.getBodySizeLimit(); + if (!PipeTransferSliceReqBuilder.shouldSlice(req, bodySizeLimit)) { return super.pipeTransfer(req); } @@ -115,23 +108,13 @@ public TPipeTransferResp pipeTransfer(final TPipeTransferReq req) throws TExcept bodySizeLimit); try { - final int sliceOrderId = SLICE_ORDER_ID_GENERATOR.getAndIncrement(); - // Slice the buffer to avoid the buffer being too large - final int sliceCount = - req.body.limit() / bodySizeLimit + (req.body.limit() % bodySizeLimit == 0 ? 0 : 1); + final int sliceOrderId = PipeTransferSliceReqBuilder.nextSliceOrderId(); + final int sliceCount = PipeTransferSliceReqBuilder.getSliceCount(req, bodySizeLimit); for (int i = 0; i < sliceCount; ++i) { - final int startIndexInBody = i * bodySizeLimit; - final int endIndexInBody = Math.min((i + 1) * bodySizeLimit, req.body.limit()); final TPipeTransferResp sliceResp = super.pipeTransfer( - PipeTransferSliceReq.toTPipeTransferReq( - sliceOrderId, - req.getType(), - i, - sliceCount, - req.body.duplicate(), - startIndexInBody, - endIndexInBody)); + PipeTransferSliceReqBuilder.buildSliceReq( + req, sliceOrderId, i, sliceCount, bodySizeLimit)); if (i == sliceCount - 1) { return sliceResp; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferSliceReqBuilder.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferSliceReqBuilder.java new file mode 100644 index 0000000000000..b108d6f1d3ab6 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferSliceReqBuilder.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.sink.payload.thrift.common; + +import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.IoTDBSinkRequestVersion; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferSliceReq; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; + +public final class PipeTransferSliceReqBuilder { + + private static final AtomicInteger SLICE_ORDER_ID_GENERATOR = new AtomicInteger(0); + + private PipeTransferSliceReqBuilder() { + // Utility class + } + + public static int getBodySizeLimit() { + return PipeConfig.getInstance().getPipeSinkRequestSliceThresholdBytes(); + } + + public static boolean shouldSlice(final TPipeTransferReq req, final int bodySizeLimit) { + return req.getVersion() == IoTDBSinkRequestVersion.VERSION_1.getVersion() + && req.body.limit() >= bodySizeLimit; + } + + public static int nextSliceOrderId() { + return SLICE_ORDER_ID_GENERATOR.getAndIncrement(); + } + + public static int getSliceCount(final TPipeTransferReq req, final int bodySizeLimit) { + return req.body.limit() / bodySizeLimit + (req.body.limit() % bodySizeLimit == 0 ? 0 : 1); + } + + public static PipeTransferSliceReq buildSliceReq( + final TPipeTransferReq originalReq, + final int sliceOrderId, + final int sliceIndex, + final int sliceCount, + final int bodySizeLimit) + throws IOException { + final int startIndexInBody = sliceIndex * bodySizeLimit; + final int endIndexInBody = Math.min((sliceIndex + 1) * bodySizeLimit, originalReq.body.limit()); + return PipeTransferSliceReq.toTPipeTransferReq( + sliceOrderId, + originalReq.getType(), + sliceIndex, + sliceCount, + originalReq.body.duplicate(), + startIndexInBody, + endIndexInBody); + } +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferSliceReqBuilderTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferSliceReqBuilderTest.java new file mode 100644 index 0000000000000..290ce3979807d --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/sink/payload/thrift/common/PipeTransferSliceReqBuilderTest.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.sink.payload.thrift.common; + +import org.apache.iotdb.commons.conf.CommonConfig; +import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.IoTDBSinkRequestVersion; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeTransferSliceReq; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +public class PipeTransferSliceReqBuilderTest { + + private final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig(); + + private int originalRequestSliceThresholdBytes; + + @Before + public void setUp() { + originalRequestSliceThresholdBytes = commonConfig.getPipeSinkRequestSliceThresholdBytes(); + commonConfig.setPipeSinkRequestSliceThresholdBytes(4); + } + + @After + public void tearDown() { + commonConfig.setPipeSinkRequestSliceThresholdBytes(originalRequestSliceThresholdBytes); + } + + @Test + public void testBuildSliceReq() throws Exception { + final TPipeTransferReq req = createReq(IoTDBSinkRequestVersion.VERSION_1.getVersion(), 10); + final int bodySizeLimit = PipeTransferSliceReqBuilder.getBodySizeLimit(); + + Assert.assertTrue(PipeTransferSliceReqBuilder.shouldSlice(req, bodySizeLimit)); + Assert.assertEquals(3, PipeTransferSliceReqBuilder.getSliceCount(req, bodySizeLimit)); + + final PipeTransferSliceReq firstSlice = + PipeTransferSliceReqBuilder.buildSliceReq(req, 123, 0, 3, bodySizeLimit); + final PipeTransferSliceReq secondSlice = + PipeTransferSliceReqBuilder.buildSliceReq(req, 123, 1, 3, bodySizeLimit); + final PipeTransferSliceReq thirdSlice = + PipeTransferSliceReqBuilder.buildSliceReq(req, 123, 2, 3, bodySizeLimit); + + Assert.assertArrayEquals(new byte[] {0, 1, 2, 3}, firstSlice.getSliceBody()); + Assert.assertArrayEquals(new byte[] {4, 5, 6, 7}, secondSlice.getSliceBody()); + Assert.assertArrayEquals(new byte[] {8, 9}, thirdSlice.getSliceBody()); + Assert.assertEquals(0, firstSlice.getSliceIndex()); + Assert.assertEquals(1, secondSlice.getSliceIndex()); + Assert.assertEquals(2, thirdSlice.getSliceIndex()); + Assert.assertEquals(3, firstSlice.getSliceCount()); + Assert.assertEquals(req.getType(), firstSlice.getOriginReqType()); + Assert.assertEquals(10, firstSlice.getOriginBodySize()); + } + + @Test + public void testShouldSliceOnlyForVersion1RequestsAboveThreshold() { + final int bodySizeLimit = PipeTransferSliceReqBuilder.getBodySizeLimit(); + + Assert.assertFalse( + PipeTransferSliceReqBuilder.shouldSlice( + createReq(IoTDBSinkRequestVersion.VERSION_1.getVersion(), 3), bodySizeLimit)); + Assert.assertFalse( + PipeTransferSliceReqBuilder.shouldSlice( + createReq((byte) (IoTDBSinkRequestVersion.VERSION_1.getVersion() + 1), 10), + bodySizeLimit)); + Assert.assertTrue( + PipeTransferSliceReqBuilder.shouldSlice( + createReq(IoTDBSinkRequestVersion.VERSION_1.getVersion(), 4), bodySizeLimit)); + } + + private static TPipeTransferReq createReq(final byte version, final int bodySize) { + final byte[] body = new byte[bodySize]; + for (int i = 0; i < body.length; ++i) { + body[i] = (byte) i; + } + + final TPipeTransferReq req = new TPipeTransferReq(); + req.version = version; + req.type = (short) 123; + req.body = ByteBuffer.wrap(body); + return req; + } +} From fea9a2f67a6fd44473f9bb66e574a79ec23fe675 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 15 May 2026 16:37:25 +0800 Subject: [PATCH 051/102] [To dev/1.3] Pipe: Fixed multiple leakages of InsertNodeMemoryEstimator (#17657) (#17685) * Pipe: Fixed multiple leakages of InsertNodeMemoryEstimator (#17657) * Fix * sptls * Update InsertNodeMemoryEstimator.java --- .../memory/InsertNodeMemoryEstimator.java | 358 +++++++++++------- .../memory/InsertNodeMemoryEstimatorTest.java | 265 +++++++++++++ 2 files changed, 490 insertions(+), 133 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimatorTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java index c29eb15371ce2..9616777b021f2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimator.java @@ -26,6 +26,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertMultiTabletsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; @@ -44,10 +45,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; public class InsertNodeMemoryEstimator { @@ -58,7 +63,6 @@ public class InsertNodeMemoryEstimator { private static final String INSERT_ROWS_NODE = "InsertRowsNode"; private static final String INSERT_ROWS_OF_ONE_DEVICE_NODE = "InsertRowsOfOneDeviceNode"; private static final String INSERT_MULTI_TABLETS_NODE = "InsertMultiTabletsNode"; - private static final String RELATIONAL_INSERT_ROW_NODE = "RelationalInsertRowNode"; private static final long NUM_BYTES_OBJECT_REF = RamUsageEstimator.NUM_BYTES_OBJECT_REF; private static final long NUM_BYTES_OBJECT_HEADER = RamUsageEstimator.NUM_BYTES_OBJECT_HEADER; @@ -158,174 +162,104 @@ public static long sizeOf(final InsertNode insertNode) { // =============================InsertNode================================== private static long calculateFullInsertNodeSize(final InsertNode node) { + return calculateFullInsertNodeSize(node, null); + } + + private static long calculateFullInsertNodeSize( + final InsertNode node, final Set deduplicatedObjects) { long size = 0; + // PlanNodeId + size += sizeOfPlanNodeId(node.getPlanNodeId(), deduplicatedObjects); // PartialPath - size += sizeOfPartialPath(node.getDevicePath()); + size += sizeOfPartialPath(node.getDevicePath(), deduplicatedObjects); // MeasurementSchemas - size += sizeOfMeasurementSchemas(node.getMeasurementSchemas()); + size += sizeOfMeasurementSchemas(node.getMeasurementSchemas(), deduplicatedObjects); // Measurement - size += sizeOfStringArray(node.getMeasurements()); + size += sizeOfStringArray(node.getMeasurements(), deduplicatedObjects); // dataTypes - size += RamUsageEstimator.shallowSizeOf(node.getDataTypes()); + size += sizeOfShallowObject(node.getDataTypes(), deduplicatedObjects); // deviceID if (node.isDeviceIDExists()) { - size += sizeOfIDeviceID(node.getDeviceID()); + size += sizeOfIDeviceID(node.getDeviceID(), deduplicatedObjects); } // dataRegionReplicaSet - size += sizeOfTRegionReplicaSet(node.getRegionReplicaSet()); + size += sizeOfTRegionReplicaSet(node.getRegionReplicaSet(), deduplicatedObjects); // progressIndex - size += sizeOfProgressIndex(node.getProgressIndex()); - return size; - } - - private static long calculateInsertNodeSizeExcludingSchemas(final InsertNode node) { - // Measurement - long size = 2 * RamUsageEstimator.shallowSizeOf(node.getMeasurementSchemas()); - // dataTypes - size += RamUsageEstimator.shallowSizeOf(node.getDataTypes()); - // deviceID - if (node.isDeviceIDExists()) { - size += sizeOfIDeviceID(node.getDeviceID()); - } - // dataRegionReplicaSet - size += sizeOfTRegionReplicaSet(node.getRegionReplicaSet()); - // progressIndex - size += sizeOfProgressIndex(node.getProgressIndex()); + size += sizeOfProgressIndex(node.getProgressIndex(), deduplicatedObjects); return size; } private static long sizeOfInsertTabletNode(final InsertTabletNode node) { - long size = INSERT_TABLET_NODE_SIZE; - size += calculateFullInsertNodeSize(node); - size += RamUsageEstimator.sizeOf(node.getTimes()); - size += sizeOfBitMapArray(node.getBitMaps()); - size += sizeOfColumns(node.getColumns(), node.getMeasurementSchemas()); - final List range = node.getRange(); - if (range != null) { - size += NUM_BYTES_OBJECT_HEADER + SIZE_OF_INT * range.size(); - } - return size; + return sizeOfInsertTabletNode(node, newDeduplicatedObjectSet()); } - private static long calculateInsertTabletNodeSizeExcludingSchemas(final InsertTabletNode node) { + private static long sizeOfInsertTabletNode( + final InsertTabletNode node, final Set deduplicatedObjects) { long size = INSERT_TABLET_NODE_SIZE; - - size += calculateInsertNodeSizeExcludingSchemas(node); - + size += calculateFullInsertNodeSize(node, deduplicatedObjects); size += RamUsageEstimator.sizeOf(node.getTimes()); - size += sizeOfBitMapArray(node.getBitMaps()); - size += sizeOfColumns(node.getColumns(), node.getMeasurementSchemas()); - - final List range = node.getRange(); - if (range != null) { - size += NUM_BYTES_OBJECT_HEADER + SIZE_OF_INT * range.size(); - } + size += sizeOfIntegerList(node.getRange()); return size; } private static long sizeOfInsertRowNode(final InsertRowNode node) { - long size = INSERT_ROW_NODE_SIZE; - size += calculateFullInsertNodeSize(node); - size += sizeOfValues(node.getValues(), node.getMeasurementSchemas()); - return size; + return sizeOfInsertRowNode(node, newDeduplicatedObjectSet()); } - private static long calculateInsertRowNodeExcludingSchemas(final InsertRowNode node) { + private static long sizeOfInsertRowNode( + final InsertRowNode node, final Set deduplicatedObjects) { long size = INSERT_ROW_NODE_SIZE; - size += calculateInsertNodeSizeExcludingSchemas(node); + size += calculateFullInsertNodeSize(node, deduplicatedObjects); size += sizeOfValues(node.getValues(), node.getMeasurementSchemas()); return size; } private static long sizeOfInsertRowsNode(final InsertRowsNode node) { + final Set deduplicatedObjects = newDeduplicatedObjectSet(); long size = INSERT_ROWS_NODE_SIZE; - size += calculateFullInsertNodeSize(node); - final List rows = node.getInsertRowNodeList(); - final List indexList = node.getInsertRowNodeIndexList(); - if (rows != null && !rows.isEmpty()) { - // InsertRowNodeList - size += NUM_BYTES_OBJECT_HEADER; - size += - (calculateInsertRowNodeExcludingSchemas(rows.get(0)) + NUM_BYTES_OBJECT_REF) - * rows.size(); - size += sizeOfPartialPath(rows.get(0).getDevicePath()); - size += sizeOfMeasurementSchemas(rows.get(0).getMeasurementSchemas()); - // InsertRowNodeIndexList - size += NUM_BYTES_OBJECT_HEADER; - size += (long) indexList.size() * (SIZE_OF_INT + NUM_BYTES_OBJECT_REF); - } + size += calculateFullInsertNodeSize(node, deduplicatedObjects); + size += sizeOfInsertRowNodeList(node.getInsertRowNodeList(), deduplicatedObjects); + size += sizeOfIntegerList(node.getInsertRowNodeIndexList()); + size += sizeOfResults(node.getResults()); return size; } private static long sizeOfInsertRowsOfOneDeviceNode(final InsertRowsOfOneDeviceNode node) { + final Set deduplicatedObjects = newDeduplicatedObjectSet(); long size = INSERT_ROWS_OF_ONE_DEVICE_NODE_SIZE; - size += calculateFullInsertNodeSize(node); - final List rows = node.getInsertRowNodeList(); - final List indexList = node.getInsertRowNodeIndexList(); - if (rows != null && !rows.isEmpty()) { - // InsertRowNodeList - size += NUM_BYTES_OBJECT_HEADER; - size += - (calculateInsertRowNodeExcludingSchemas(rows.get(0)) + NUM_BYTES_OBJECT_REF) - * rows.size(); - size += sizeOfPartialPath(rows.get(0).getDevicePath()); - size += sizeOfMeasurementSchemas(rows.get(0).getMeasurementSchemas()); - // InsertRowNodeIndexList - size += NUM_BYTES_OBJECT_HEADER; - size += (long) indexList.size() * (SIZE_OF_INT + NUM_BYTES_OBJECT_REF); - } - // results - size += NUM_BYTES_OBJECT_HEADER; - for (Map.Entry entry : node.getResults().entrySet()) { - size += - Integer.BYTES - + sizeOfTSStatus(entry.getValue()) - + RamUsageEstimator.HASHTABLE_RAM_BYTES_PER_ENTRY; - } + size += calculateFullInsertNodeSize(node, deduplicatedObjects); + size += sizeOfInsertRowNodeList(node.getInsertRowNodeList(), deduplicatedObjects); + size += sizeOfIntegerList(node.getInsertRowNodeIndexList()); + size += sizeOfResults(node.getResults()); return size; } private static long sizeOfInsertMultiTabletsNode(final InsertMultiTabletsNode node) { + final Set deduplicatedObjects = newDeduplicatedObjectSet(); long size = INSERT_MULTI_TABLETS_NODE_SIZE; - size += calculateFullInsertNodeSize(node); - // dataTypes - size += RamUsageEstimator.shallowSizeOf(node.getDataTypes()); - - final List rows = node.getInsertTabletNodeList(); - final List indexList = node.getParentInsertTabletNodeIndexList(); - if (rows != null && !rows.isEmpty()) { - // InsertTabletNodeList - size += NUM_BYTES_OBJECT_HEADER; - size += - (calculateInsertTabletNodeSizeExcludingSchemas(rows.get(0)) + NUM_BYTES_OBJECT_REF) - * rows.size(); - size += sizeOfPartialPath(rows.get(0).getDevicePath()); - size += sizeOfMeasurementSchemas(rows.get(0).getMeasurementSchemas()); - // ParentInsertTabletNodeIndexList - size += NUM_BYTES_OBJECT_HEADER; - size += (long) indexList.size() * (SIZE_OF_INT + NUM_BYTES_OBJECT_REF); - } - // results - if (node.getResults() != null) { - size += NUM_BYTES_OBJECT_HEADER; - for (Map.Entry entry : node.getResults().entrySet()) { - size += - Integer.BYTES - + sizeOfTSStatus(entry.getValue()) - + RamUsageEstimator.HASHTABLE_RAM_BYTES_PER_ENTRY; - } - } + size += calculateFullInsertNodeSize(node, deduplicatedObjects); + size += sizeOfInsertTabletNodeList(node.getInsertTabletNodeList(), deduplicatedObjects); + size += sizeOfIntegerList(node.getParentInsertTabletNodeIndexList()); + size += sizeOfResults(node.getResults()); return size; } // ============================Device And Measurement=================================== public static long sizeOfPartialPath(final PartialPath partialPath) { + return sizeOfPartialPath(partialPath, null); + } + + private static long sizeOfPartialPath( + final PartialPath partialPath, final Set deduplicatedObjects) { if (partialPath == null) { return 0L; } + if (!shouldCountObject(partialPath, deduplicatedObjects)) { + return 0L; + } long size = PARTIAL_PATH_SIZE; final String[] nodes = partialPath.getNodes(); if (nodes != null) { @@ -338,22 +272,38 @@ public static long sizeOfPartialPath(final PartialPath partialPath) { } public static long sizeOfMeasurementSchemas(final MeasurementSchema[] measurementSchemas) { + return sizeOfMeasurementSchemas(measurementSchemas, null); + } + + private static long sizeOfMeasurementSchemas( + final MeasurementSchema[] measurementSchemas, final Set deduplicatedObjects) { if (measurementSchemas == null) { return 0L; } + if (!shouldCountObject(measurementSchemas, deduplicatedObjects)) { + return 0L; + } long size = RamUsageEstimator.alignObjectSize( NUM_BYTES_ARRAY_HEADER + NUM_BYTES_OBJECT_REF * measurementSchemas.length); for (MeasurementSchema measurementSchema : measurementSchemas) { - size += sizeOfMeasurementSchema(measurementSchema); + size += sizeOfMeasurementSchema(measurementSchema, deduplicatedObjects); } return size; } - private static long sizeOfMeasurementSchema(final MeasurementSchema measurementSchema) { + public static long sizeOfMeasurementSchema(final MeasurementSchema measurementSchema) { + return sizeOfMeasurementSchema(measurementSchema, null); + } + + private static long sizeOfMeasurementSchema( + final MeasurementSchema measurementSchema, final Set deduplicatedObjects) { if (measurementSchema == null) { return 0L; } + if (!shouldCountObject(measurementSchema, deduplicatedObjects)) { + return 0L; + } // Header + primitive + reference long size = MEASUREMENT_SCHEMA_SIZE; // measurementId @@ -361,7 +311,7 @@ private static long sizeOfMeasurementSchema(final MeasurementSchema measurementS // props final Map props = measurementSchema.getProps(); if (props != null) { - size += NUM_BYTES_OBJECT_HEADER; + size += RamUsageEstimator.shallowSizeOf(props); for (Map.Entry entry : props.entrySet()) { size += RamUsageEstimator.sizeOf(entry.getKey()) @@ -373,16 +323,31 @@ private static long sizeOfMeasurementSchema(final MeasurementSchema measurementS return size; } - private static long sizeOfIDeviceID(final IDeviceID deviceID) { - return Objects.nonNull(deviceID) ? deviceID.ramBytesUsed() : 0L; + public static long sizeOfIDeviceID(final IDeviceID deviceID) { + return sizeOfIDeviceID(deviceID, null); + } + + private static long sizeOfIDeviceID( + final IDeviceID deviceID, final Set deduplicatedObjects) { + return Objects.nonNull(deviceID) && shouldCountObject(deviceID, deduplicatedObjects) + ? deviceID.ramBytesUsed() + : 0L; } // =============================Thrift================================== private static long sizeOfTRegionReplicaSet(final TRegionReplicaSet tRegionReplicaSet) { + return sizeOfTRegionReplicaSet(tRegionReplicaSet, null); + } + + private static long sizeOfTRegionReplicaSet( + final TRegionReplicaSet tRegionReplicaSet, final Set deduplicatedObjects) { if (tRegionReplicaSet == null) { return 0L; } + if (!shouldCountObject(tRegionReplicaSet, deduplicatedObjects)) { + return 0L; + } // Memory alignment of basic types and reference types in structures long size = T_REGION_REPLICA_SET_SIZE; // Memory calculation in reference type, cannot get exact value, roughly estimate @@ -390,9 +355,9 @@ private static long sizeOfTRegionReplicaSet(final TRegionReplicaSet tRegionRepli size += sizeOfTConsensusGroupId(); } if (tRegionReplicaSet.isSetDataNodeLocations()) { - size += NUM_BYTES_OBJECT_HEADER; + size += sizeOfObjectList(tRegionReplicaSet.getDataNodeLocations()); for (TDataNodeLocation tDataNodeLocation : tRegionReplicaSet.getDataNodeLocations()) { - size += sizeOfTDataNodeLocation(tDataNodeLocation); + size += sizeOfTDataNodeLocation(tDataNodeLocation, deduplicatedObjects); } } return size; @@ -403,25 +368,39 @@ private static long sizeOfTConsensusGroupId() { return T_CONSENSUS_GROUP_ID_SIZE; } - private static long sizeOfTDataNodeLocation(final TDataNodeLocation tDataNodeLocation) { + private static long sizeOfTDataNodeLocation( + final TDataNodeLocation tDataNodeLocation, final Set deduplicatedObjects) { if (tDataNodeLocation == null) { return 0L; } + if (!shouldCountObject(tDataNodeLocation, deduplicatedObjects)) { + return 0L; + } long size = T_DATA_NODE_LOCATION_SIZE; - size += sizeOfTEndPoint(tDataNodeLocation.getClientRpcEndPoint()); - size += sizeOfTEndPoint(tDataNodeLocation.getInternalEndPoint()); - size += sizeOfTEndPoint(tDataNodeLocation.getMPPDataExchangeEndPoint()); - size += sizeOfTEndPoint(tDataNodeLocation.getDataRegionConsensusEndPoint()); - size += sizeOfTEndPoint(tDataNodeLocation.getSchemaRegionConsensusEndPoint()); + size += sizeOfTEndPoint(tDataNodeLocation.getClientRpcEndPoint(), deduplicatedObjects); + size += sizeOfTEndPoint(tDataNodeLocation.getInternalEndPoint(), deduplicatedObjects); + size += sizeOfTEndPoint(tDataNodeLocation.getMPPDataExchangeEndPoint(), deduplicatedObjects); + size += + sizeOfTEndPoint(tDataNodeLocation.getDataRegionConsensusEndPoint(), deduplicatedObjects); + size += + sizeOfTEndPoint(tDataNodeLocation.getSchemaRegionConsensusEndPoint(), deduplicatedObjects); return size; } private static long sizeOfTEndPoint(final TEndPoint tEndPoint) { + return sizeOfTEndPoint(tEndPoint, null); + } + + private static long sizeOfTEndPoint( + final TEndPoint tEndPoint, final Set deduplicatedObjects) { if (tEndPoint == null) { return 0L; } + if (!shouldCountObject(tEndPoint, deduplicatedObjects)) { + return 0L; + } // objectHeader + ip + port long size = T_END_POINT_SIZE; @@ -430,18 +409,32 @@ private static long sizeOfTEndPoint(final TEndPoint tEndPoint) { } private static long sizeOfTSStatus(final TSStatus tSStatus) { + return sizeOfTSStatus(tSStatus, null); + } + + private static long sizeOfTSStatus( + final TSStatus tSStatus, final Set deduplicatedObjects) { if (tSStatus == null) { return 0L; } + if (!shouldCountObject(tSStatus, deduplicatedObjects)) { + return 0L; + } long size = TS_STATUS_SIZE; // message if (tSStatus.isSetMessage()) { size += RamUsageEstimator.sizeOf(tSStatus.message); } - // ignore subStatus + // subStatus + if (tSStatus.getSubStatus() != null) { + size += sizeOfObjectList(tSStatus.getSubStatus()); + for (TSStatus subStatus : tSStatus.getSubStatus()) { + size += sizeOfTSStatus(subStatus, deduplicatedObjects); + } + } // redirectNode if (tSStatus.isSetRedirectNode()) { - size += sizeOfTEndPoint(tSStatus.redirectNode); + size += sizeOfTEndPoint(tSStatus.redirectNode, deduplicatedObjects); } return size; } @@ -449,7 +442,14 @@ private static long sizeOfTSStatus(final TSStatus tSStatus) { // =============================ProgressIndex================================== private static long sizeOfProgressIndex(final ProgressIndex progressIndex) { - return Objects.nonNull(progressIndex) ? progressIndex.ramBytesUsed() : 0L; + return sizeOfProgressIndex(progressIndex, null); + } + + private static long sizeOfProgressIndex( + final ProgressIndex progressIndex, final Set deduplicatedObjects) { + return Objects.nonNull(progressIndex) && shouldCountObject(progressIndex, deduplicatedObjects) + ? progressIndex.ramBytesUsed() + : 0L; } // =============================Write================================== @@ -617,4 +617,96 @@ public static long sizeOfValues( } return size; } + + private static long sizeOfPlanNodeId( + final PlanNodeId planNodeId, final Set deduplicatedObjects) { + return planNodeId != null && shouldCountObject(planNodeId, deduplicatedObjects) + ? planNodeId.ramBytesUsed() + : 0L; + } + + private static long sizeOfStringArray( + final String[] strings, final Set deduplicatedObjects) { + return strings != null && shouldCountObject(strings, deduplicatedObjects) + ? RamUsageEstimator.sizeOf(strings) + : 0L; + } + + private static long sizeOfShallowObject( + final Object object, final Set deduplicatedObjects) { + return object != null && shouldCountObject(object, deduplicatedObjects) + ? RamUsageEstimator.shallowSizeOf(object) + : 0L; + } + + private static long sizeOfInsertRowNodeList( + final List rows, final Set deduplicatedObjects) { + if (rows == null) { + return 0L; + } + long size = sizeOfObjectList(rows); + for (InsertRowNode row : rows) { + size += sizeOfInsertRowNode(row, deduplicatedObjects); + } + return size; + } + + private static long sizeOfInsertTabletNodeList( + final List tablets, final Set deduplicatedObjects) { + if (tablets == null) { + return 0L; + } + long size = sizeOfObjectList(tablets); + for (InsertTabletNode tablet : tablets) { + size += sizeOfInsertTabletNode(tablet, deduplicatedObjects); + } + return size; + } + + private static long sizeOfObjectList(final List list) { + if (list == null) { + return 0L; + } + long size = RamUsageEstimator.shallowSizeOf(list); + if (list instanceof ArrayList) { + size += + RamUsageEstimator.alignObjectSize( + NUM_BYTES_ARRAY_HEADER + NUM_BYTES_OBJECT_REF * list.size()); + } + return size; + } + + private static long sizeOfIntegerList(final List integers) { + if (integers == null) { + return 0L; + } + long size = sizeOfObjectList(integers); + for (Integer ignored : integers) { + size += SIZE_OF_INT; + } + return size; + } + + private static long sizeOfResults(final Map results) { + if (results == null) { + return 0L; + } + long size = RamUsageEstimator.shallowSizeOf(results); + for (Map.Entry entry : results.entrySet()) { + size += + SIZE_OF_INT + + sizeOfTSStatus(entry.getValue()) + + RamUsageEstimator.HASHTABLE_RAM_BYTES_PER_ENTRY; + } + return size; + } + + private static Set newDeduplicatedObjectSet() { + return Collections.newSetFromMap(new IdentityHashMap<>()); + } + + private static boolean shouldCountObject( + final Object object, final Set deduplicatedObjects) { + return object != null && (deduplicatedObjects == null || deduplicatedObjects.add(object)); + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimatorTest.java new file mode 100644 index 0000000000000..58b2756b16f6b --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/memory/InsertNodeMemoryEstimatorTest.java @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.resource.memory; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertMultiTabletsNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsOfOneDeviceNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class InsertNodeMemoryEstimatorTest { + + @Test + public void testInsertRowsNodeLaterRowSizeIsEstimated() throws IllegalPathException { + InsertRowNode firstRow = + createTextInsertRowNode("child-1", "root.sg.d1", new String[] {"s1"}, new String[] {"v1"}); + InsertRowNode smallSecondRow = + createTextInsertRowNode("child-2", "root.sg.d2", new String[] {"s1"}, new String[] {"v2"}); + InsertRowNode largeSecondRow = + createTextInsertRowNode( + "child-3", + "root.sg.device_with_a_longer_path_segment", + new String[] {"s1", "measurement_with_a_longer_name", "s3"}, + new String[] {"v2", repeatedString("payload", 32), repeatedString("payload", 48)}); + + long baselineSize = + InsertNodeMemoryEstimator.sizeOf(createInsertRowsNode("parent", firstRow, smallSecondRow)); + long largerNodeSize = + InsertNodeMemoryEstimator.sizeOf(createInsertRowsNode("parent", firstRow, largeSecondRow)); + + Assert.assertTrue(largerNodeSize > baselineSize); + } + + @Test + public void testInsertRowsNodeResultsAreEstimated() throws IllegalPathException { + InsertRowsNode node = + createInsertRowsNode( + "parent", + createTextInsertRowNode( + "child-1", "root.sg.d1", new String[] {"s1"}, new String[] {"v1"}), + createTextInsertRowNode( + "child-2", "root.sg.d2", new String[] {"s1"}, new String[] {"v2"})); + + long sizeWithoutResults = InsertNodeMemoryEstimator.sizeOf(node); + + TSStatus statusWithoutSubStatus = createStatus("outer-message"); + node.getResults().put(1, statusWithoutSubStatus); + long sizeWithResults = InsertNodeMemoryEstimator.sizeOf(node); + + TSStatus statusWithSubStatus = createStatus("outer-message"); + List subStatusList = new ArrayList<>(); + subStatusList.add(createStatus(repeatedString("inner-message", 16))); + statusWithSubStatus.setSubStatus(subStatusList); + node.getResults().put(1, statusWithSubStatus); + long sizeWithSubStatus = InsertNodeMemoryEstimator.sizeOf(node); + + Assert.assertTrue(sizeWithResults > sizeWithoutResults); + Assert.assertTrue(sizeWithSubStatus > sizeWithResults); + } + + @Test + public void testInsertRowsOfOneDeviceNodeLaterRowSizeIsEstimated() throws IllegalPathException { + InsertRowNode firstRow = + createTextInsertRowNode( + "child-1", "root.sg.d1", new String[] {"s1", "s2"}, new String[] {"v1", "v2"}); + InsertRowNode smallSecondRow = + createTextInsertRowNode( + "child-2", "root.sg.d1", new String[] {"s1", "s2"}, new String[] {"v3", "v4"}); + InsertRowNode largeSecondRow = + createTextInsertRowNode( + "child-3", + "root.sg.d1", + new String[] {"s1", "s2"}, + new String[] {repeatedString("payload", 32), repeatedString("payload", 48)}); + + long baselineSize = + InsertNodeMemoryEstimator.sizeOf( + createInsertRowsOfOneDeviceNode("parent", firstRow, smallSecondRow)); + long largerNodeSize = + InsertNodeMemoryEstimator.sizeOf( + createInsertRowsOfOneDeviceNode("parent", firstRow, largeSecondRow)); + + Assert.assertTrue(largerNodeSize > baselineSize); + } + + @Test + public void testInsertMultiTabletsNodeLaterTabletSizeIsEstimated() throws IllegalPathException { + InsertTabletNode firstTablet = createTextInsertTabletNode("child-1", "root.sg.d1", 1, 1, 2); + InsertTabletNode smallSecondTablet = + createTextInsertTabletNode("child-2", "root.sg.d2", 1, 1, 2); + InsertTabletNode largeSecondTablet = + createTextInsertTabletNode("child-3", "root.sg.d3", 3, 8, 16); + + long baselineSize = + InsertNodeMemoryEstimator.sizeOf( + createInsertMultiTabletsNode("parent", firstTablet, smallSecondTablet)); + long largerNodeSize = + InsertNodeMemoryEstimator.sizeOf( + createInsertMultiTabletsNode("parent", firstTablet, largeSecondTablet)); + + Assert.assertTrue(largerNodeSize > baselineSize); + } + + @Test + public void testPlanNodeIdIsEstimated() throws IllegalPathException { + InsertRowNode shortPlanNodeIdRow = + createTextInsertRowNode("id", "root.sg.d1", new String[] {"s1"}, new String[] {"v1"}); + InsertRowNode longPlanNodeIdRow = + createTextInsertRowNode( + repeatedString("plan-node-id", 12), + "root.sg.d1", + new String[] {"s1"}, + new String[] {"v1"}); + + Assert.assertTrue( + InsertNodeMemoryEstimator.sizeOf(longPlanNodeIdRow) + > InsertNodeMemoryEstimator.sizeOf(shortPlanNodeIdRow)); + } + + private static InsertRowsNode createInsertRowsNode( + String planNodeId, InsertRowNode... insertRowNodes) { + InsertRowsNode node = new InsertRowsNode(new PlanNodeId(planNodeId)); + for (int i = 0; i < insertRowNodes.length; i++) { + node.addOneInsertRowNode(insertRowNodes[i], i); + } + node.setDevicePath(insertRowNodes[0].getDevicePath()); + node.setMeasurementSchemas(insertRowNodes[0].getMeasurementSchemas()); + node.setMeasurements(insertRowNodes[0].getMeasurements()); + node.setDataTypes(insertRowNodes[0].getDataTypes()); + return node; + } + + private static InsertRowsOfOneDeviceNode createInsertRowsOfOneDeviceNode( + String planNodeId, InsertRowNode... insertRowNodes) { + InsertRowsOfOneDeviceNode node = new InsertRowsOfOneDeviceNode(new PlanNodeId(planNodeId)); + List rows = new ArrayList<>(Arrays.asList(insertRowNodes)); + List indexes = new ArrayList<>(); + for (int i = 0; i < insertRowNodes.length; i++) { + indexes.add(i); + } + node.setInsertRowNodeList(rows); + node.setInsertRowNodeIndexList(indexes); + return node; + } + + private static InsertMultiTabletsNode createInsertMultiTabletsNode( + String planNodeId, InsertTabletNode... insertTabletNodes) { + InsertMultiTabletsNode node = new InsertMultiTabletsNode(new PlanNodeId(planNodeId)); + for (int i = 0; i < insertTabletNodes.length; i++) { + node.addInsertTabletNode(insertTabletNodes[i], i); + } + return node; + } + + private static InsertRowNode createTextInsertRowNode( + String planNodeId, String devicePath, String[] measurements, String[] values) + throws IllegalPathException { + TSDataType[] dataTypes = new TSDataType[measurements.length]; + MeasurementSchema[] measurementSchemas = new MeasurementSchema[measurements.length]; + Object[] rowValues = new Object[measurements.length]; + for (int i = 0; i < measurements.length; i++) { + dataTypes[i] = TSDataType.TEXT; + measurementSchemas[i] = new MeasurementSchema(measurements[i], TSDataType.TEXT); + rowValues[i] = new Binary(values[i], TSFileConfig.STRING_CHARSET); + } + return new InsertRowNode( + new PlanNodeId(planNodeId), + new PartialPath(devicePath), + false, + measurements, + dataTypes, + measurementSchemas, + 1L, + rowValues, + false); + } + + private static InsertTabletNode createTextInsertTabletNode( + String planNodeId, String devicePath, int measurementCount, int rowCount, int repeatCount) + throws IllegalPathException { + String[] measurements = new String[measurementCount]; + TSDataType[] dataTypes = new TSDataType[measurementCount]; + MeasurementSchema[] measurementSchemas = new MeasurementSchema[measurementCount]; + Object[] columns = new Object[measurementCount]; + for (int measurementIndex = 0; measurementIndex < measurementCount; measurementIndex++) { + measurements[measurementIndex] = "s" + measurementIndex; + dataTypes[measurementIndex] = TSDataType.TEXT; + measurementSchemas[measurementIndex] = + new MeasurementSchema(measurements[measurementIndex], TSDataType.TEXT); + Binary[] values = new Binary[rowCount]; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + values[rowIndex] = + new Binary( + repeatedString("value-" + measurementIndex + "-" + rowIndex, repeatCount), + TSFileConfig.STRING_CHARSET); + } + columns[measurementIndex] = values; + } + + long[] times = new long[rowCount]; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + times[rowIndex] = rowIndex; + } + + return new InsertTabletNode( + new PlanNodeId(planNodeId), + new PartialPath(devicePath), + false, + measurements, + dataTypes, + measurementSchemas, + times, + null, + columns, + rowCount); + } + + private static TSStatus createStatus(String message) { + TSStatus status = new TSStatus(); + status.setCode(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); + status.setMessage(message); + return status; + } + + private static String repeatedString(String unit, int repeatCount) { + StringBuilder builder = new StringBuilder(unit.length() * repeatCount); + for (int i = 0; i < repeatCount; i++) { + builder.append(unit); + } + return builder.toString(); + } +} From 1d3578026d8e19d5801f4d47fe0a9e03c141678b Mon Sep 17 00:00:00 2001 From: shuwenwei <55970239+shuwenwei@users.noreply.github.com> Date: Tue, 19 May 2026 14:27:43 +0800 Subject: [PATCH 052/102] [to dev/1.3] Fix query reuse of flushing memtable TVList (#17709) --- .../execution/fragment/QueryContext.java | 4 +- .../utils/ResourceByPathUtils.java | 40 ++++++++++++- .../dataregion/flush/MemTableFlushTask.java | 1 - .../memtable/AbstractWritableMemChunk.java | 20 ++++++- .../memtable/AlignedReadOnlyMemChunk.java | 2 +- .../memtable/IWritableMemChunk.java | 3 + .../dataregion/memtable/ReadOnlyMemChunk.java | 2 +- .../FragmentInstanceExecutionTest.java | 6 +- .../memtable/PrimitiveMemTableTest.java | 59 ++++++++++++++++++- 9 files changed, 123 insertions(+), 14 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java index 8d62e20776679..d1b235ff7b100 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/fragment/QueryContext.java @@ -217,7 +217,7 @@ public void setQueryStatistics(QueryStatistics queryStatistics) { this.queryStatistics = queryStatistics; } - public void addTVListToSet(Map tvListMap) { - tvListSet.addAll(tvListMap.keySet()); + public void addTVListToSet(Set set) { + tvListSet.addAll(set); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java index c64d32c1e327b..fa2f603d6facc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/utils/ResourceByPathUtils.java @@ -158,10 +158,46 @@ protected Map prepareTvListMapForQuery( } if (!isWorkMemTable) { + /* + * 1. Q1 queries this TVList while it is still in the working memtable and records a smaller + * visible row count. + * 2. Later writes append out-of-order rows to the same TVList, then FLUSH moves the + * memtable to the flushing list. + * 3. Q2 queries the flushing memtable. If Q2 directly reuses the original mutable TVList, + * Q2's query-side sort may reorder the indices in place. + * 4. Q1 continues to read with its old row count and the reordered indices. The converted + * value index can exceed Q1's bitmap range and cause out-of-bound access. + * + * Therefore, this flushing branch can reuse the original list only when it is already + * sorted or no active query is using it. Otherwise, Q2 should read from + * workingListForFlush. + */ + boolean canUseListDirectly = list.isSorted() || list.getQueryContextSet().isEmpty(); LOGGER.debug( "Flushing MemTable - add current query context to mutable TVList's query list"); - list.getQueryContextSet().add(context); - tvListQueryMap.put(list, list.rowCount()); + if (canUseListDirectly) { + list.getQueryContextSet().add(context); + tvListQueryMap.put(list, list.rowCount()); + } else { + TVList workingListForFlushSort = memChunk.initWorkingListForFlushIfNecessary(list, true); + /* + * The query will read from workingListForFlushSort, but cloneForFlushSort() only clones + * times and indices. The value arrays and bitmaps are still shared with the original + * list. + * + * Therefore, this query must also hold the original list until it finishes. Adding + * context to list.getQueryContextSet() lets flush/query cleanup see that the original + * list is still in use. Adding list to context.tvListSet makes + * releaseTVListOwnedByQuery() remove this context from the original list later. + * + * Do not put the original list into tvListQueryMap here. The actual read path must use + * workingListForFlushSort to avoid sorting the original list in place. + */ + list.getQueryContextSet().add(context); + context.addTVListToSet(Collections.singleton(list)); + workingListForFlushSort.getQueryContextSet().add(context); + tvListQueryMap.put(workingListForFlushSort, workingListForFlushSort.rowCount()); + } } else { if (list.isSorted() || list.getQueryContextSet().isEmpty()) { LOGGER.debug( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/MemTableFlushTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/MemTableFlushTask.java index 0c5ccc9ac1bb1..48efe55c223d3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/MemTableFlushTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/MemTableFlushTask.java @@ -278,7 +278,6 @@ public void run() { times = new long[MAX_NUMBER_OF_POINTS_IN_PAGE]; } writableMemChunk.encode(ioTaskQueue, encodeInfo, times); - writableMemChunk.releaseTemporaryTvListForFlush(); long subTaskTime = System.currentTimeMillis() - starTime; WRITING_METRICS.recordFlushSubTaskCost(WritingMetrics.ENCODING_TASK, subTaskTime); memSerializeTime += subTaskTime; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java index 5167ea96b56a8..6c773942fb72b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractWritableMemChunk.java @@ -206,8 +206,9 @@ public void sortTvListForFlush() { /* * Concurrency background: * - * A query may start earlier and record the current row count (rows) of the TVList as its visible range. - * After that, new unseq writes may arrive and immediately trigger a flush, which will sort the TVList. + * A query may start earlier and record the current row count (rows) of the TVList as its + * visible range. After that, new unseq writes may arrive and immediately trigger a flush, which + * will sort the TVList. * * During sorting, the underlying indices array of the TVList may be reordered. * If the query continues to use the previously recorded rows as its upper bound, @@ -219,6 +220,9 @@ public void sortTvListForFlush() { * To avoid this issue, when there are active queries on the working TVList, we must * clone the times and indices before sorting, so that the flush sort does not mutate * the data structures that concurrent queries rely on. + * + * Flushing-memtable queries may also reuse workingListForFlush instead of the original working + * TVList for the same reason. */ boolean needCloneTimesAndIndicesInWorkingTVList; workingList.lockQueryList(); @@ -228,7 +232,7 @@ public void sortTvListForFlush() { workingList.unlockQueryList(); } workingListForFlush = - needCloneTimesAndIndicesInWorkingTVList ? workingList.cloneForFlushSort() : workingList; + initWorkingListForFlushIfNecessary(workingList, needCloneTimesAndIndicesInWorkingTVList); workingListForFlush.sort(); } @@ -267,4 +271,14 @@ public abstract void encode( @Override public abstract int serializedSize(); + + @Override + public synchronized TVList initWorkingListForFlushIfNecessary( + TVList workingList, boolean needCloneTimesAndIndicesInWorkingTVList) { + if (workingListForFlush == null) { + workingListForFlush = + needCloneTimesAndIndicesInWorkingTVList ? workingList.cloneForFlushSort() : workingList; + } + return workingListForFlush; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java index 30976fb6790f1..bb2ee311d3047 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AlignedReadOnlyMemChunk.java @@ -104,7 +104,7 @@ public AlignedReadOnlyMemChunk( this.valueStatisticsList = new ArrayList<>(); this.alignedTvListQueryMap = alignedTvListQueryMap; this.columnIndexList = columnIndexList; - this.context.addTVListToSet(alignedTvListQueryMap); + this.context.addTVListToSet(alignedTvListQueryMap.keySet()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java index 994045024375f..376136e1d122e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IWritableMemChunk.java @@ -126,4 +126,7 @@ default long getMinTime() { TVList getWorkingTVList(); void setWorkingTVList(TVList list); + + TVList initWorkingListForFlushIfNecessary( + TVList workingList, boolean needCloneTimesAndIndicesInWorkingTVList); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java index 9e46740a759c8..c0a71bf7edcdc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/ReadOnlyMemChunk.java @@ -127,7 +127,7 @@ public ReadOnlyMemChunk( this.deletionList = deletionList; this.tvListQueryMap = tvListQueryMap; this.pageStatisticsList = new ArrayList<>(); - this.context.addTVListToSet(tvListQueryMap); + this.context.addTVListToSet(tvListQueryMap.keySet()); } public void sortTvLists() { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecutionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecutionTest.java index 22e5c360ae3c5..cfc7f887dcfd3 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecutionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/fragment/FragmentInstanceExecutionTest.java @@ -44,7 +44,7 @@ import org.apache.iotdb.db.utils.datastructure.AlignedTVList; import org.apache.iotdb.db.utils.datastructure.TVList; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.enums.CompressionType; import org.apache.tsfile.file.metadata.enums.TSEncoding; @@ -133,13 +133,13 @@ public void testTVListOwnerTransfer() throws InterruptedException { FragmentInstanceExecution execution1 = createFragmentInstanceExecution(1, instanceNotificationExecutor); FragmentInstanceContext fragmentInstanceContext1 = execution1.getFragmentInstanceContext(); - fragmentInstanceContext1.addTVListToSet(ImmutableMap.of(tvList, 0)); + fragmentInstanceContext1.addTVListToSet(ImmutableSet.of(tvList)); tvList.getQueryContextSet().add(fragmentInstanceContext1); FragmentInstanceExecution execution2 = createFragmentInstanceExecution(2, instanceNotificationExecutor); FragmentInstanceContext fragmentInstanceContext2 = execution2.getFragmentInstanceContext(); - fragmentInstanceContext2.addTVListToSet(ImmutableMap.of(tvList, 0)); + fragmentInstanceContext2.addTVListToSet(ImmutableSet.of(tvList)); tvList.getQueryContextSet().add(fragmentInstanceContext2); // mock flush's behavior diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java index b366a48ab7797..b9259db0a0fed 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/memtable/PrimitiveMemTableTest.java @@ -222,6 +222,63 @@ public void testWriteAndFlushSortDuringQuerySortTVListAndActualQueryExecution() } } + @Test + public void testFlushingQueryDoesNotSortWorkingTVListUsedByPreviousQuery() + throws QueryProcessException, IOException, IllegalPathException { + + PrimitiveMemTable memTable = new PrimitiveMemTable("root.test", "0"); + List measurementSchemas = + Arrays.asList( + new MeasurementSchema("s1", TSDataType.INT32), + new MeasurementSchema("s2", TSDataType.INT32), + new MeasurementSchema("s3", TSDataType.INT32)); + PlainDeviceID deviceID = new PlainDeviceID("root.test.d1"); + for (int i = 1000; i < 2000; i++) { + memTable.writeAlignedRow(deviceID, measurementSchemas, i, new Object[] {i, i, i}); + } + + ResourceByPathUtils resourcesByPathUtils = + ResourceByPathUtils.getResourceInstance( + new AlignedPath("root.test.d1", Arrays.asList("s1", "s2", "s3"), measurementSchemas)); + AlignedReadOnlyMemChunk firstQueryMemChunk = + (AlignedReadOnlyMemChunk) + resourcesByPathUtils.getReadOnlyMemChunkFromMemTable( + new QueryContext(1), memTable, null, Long.MAX_VALUE, null); + TVList originalWorkingList = memTable.getWritableMemChunk(deviceID, "").getWorkingTVList(); + Assert.assertSame( + originalWorkingList, + firstQueryMemChunk.getAligendTvListQueryMap().keySet().iterator().next()); + + for (int i = 1; i <= 50; i++) { + memTable.writeAlignedRow(deviceID, measurementSchemas, i, new Object[] {i, i, i}); + } + MeasurementPath path = new MeasurementPath("root.test.d1.s1", TSDataType.INT32); + memTable.delete(path, path.getDevicePath(), 1, 10); + path = new MeasurementPath("root.test.d1.s2", TSDataType.INT32); + memTable.delete(path, path.getDevicePath(), 1, 10); + path = new MeasurementPath("root.test.d1.s3", TSDataType.INT32); + memTable.delete(path, path.getDevicePath(), 1, 10); + Assert.assertFalse(originalWorkingList.isSorted()); + + AlignedReadOnlyMemChunk flushingQueryMemChunk = + (AlignedReadOnlyMemChunk) + resourcesByPathUtils.getReadOnlyMemChunkFromMemTable( + new QueryContext(2), memTable, new ArrayList<>(), Long.MAX_VALUE, null); + TVList flushingQueryList = + flushingQueryMemChunk.getAligendTvListQueryMap().keySet().iterator().next(); + Assert.assertNotSame(originalWorkingList, flushingQueryList); + + flushingQueryMemChunk.sortTvLists(); + Assert.assertFalse(originalWorkingList.isSorted()); + + firstQueryMemChunk.sortTvLists(); + MemPointIterator memPointIterator = + firstQueryMemChunk.createMemPointIterator(Ordering.ASC, null); + while (memPointIterator.hasNextBatch()) { + memPointIterator.nextBatch(); + } + } + @Test public void memSeriesToStringTest() throws IOException { TSDataType dataType = TSDataType.INT32; @@ -743,7 +800,7 @@ public void testReleaseWithNotEnoughMemory() throws CpuNotEnoughException { list.getQueryContextSet().add(queryContext); Map tvlistMap = new HashMap<>(); tvlistMap.put(list, 100); - queryContext.addTVListToSet(tvlistMap); + queryContext.addTVListToSet(tvlistMap.keySet()); // fragment instance execution IDriverScheduler scheduler = Mockito.mock(IDriverScheduler.class); From 8657782e4c8bbfea602156005ed80ece0f29cc62 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 20 May 2026 18:06:17 +0800 Subject: [PATCH 053/102] Pipe: Optimized logger for temporarily out of memory exception & Do not stop pipe for "Waited for memory to parse TsFile" (#17542) (#17728) --- .../processor/PipeProcessorSubtask.java | 8 +++---- .../tsfile/PipeTsFileInsertionEvent.java | 21 ++++++++++++------- 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/processor/PipeProcessorSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/processor/PipeProcessorSubtask.java index ca5a8d0f4dba7..193693c5a95bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/processor/PipeProcessorSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/processor/PipeProcessorSubtask.java @@ -191,15 +191,15 @@ protected boolean executeOnce() throws Exception { } catch (final PipeRuntimeOutOfMemoryCriticalException e) { PipeLogger.log( LOGGER::info, - e, - "Temporarily out of memory in pipe event processing, will wait for the memory to release."); + "Temporarily out of memory in pipe event processing, will wait for the memory to release. Message: %s", + e.getMessage()); return false; } catch (final Exception e) { if (ExceptionUtils.getRootCause(e) instanceof PipeRuntimeOutOfMemoryCriticalException) { PipeLogger.log( LOGGER::info, - e, - "Temporarily out of memory in pipe event processing, will wait for the memory to release."); + "Temporarily out of memory in pipe event processing, will wait for the memory to release. Message: %s", + e.getMessage()); return false; } if (!isClosed.get()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 53f0b16826ab3..8ffbc9f2f9bde 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -26,6 +26,7 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.commons.pipe.resource.log.PipeLogger; import org.apache.iotdb.commons.pipe.resource.ref.PipePhantomReferenceManager.PipeEventResource; import org.apache.iotdb.db.pipe.event.ReferenceTrackableEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; @@ -495,8 +496,7 @@ public void consumeTabletInsertionEventsWithRetry( callerName, getTsFile(), tabletEventCount, - retryCount, - e); + retryCount); } else if (LOGGER.isDebugEnabled()) { LOGGER.debug( "{}: failed to allocate memory for parsing TsFile {}, tablet event no. {}, retry count is {}, will keep retrying.", @@ -542,7 +542,11 @@ public Iterable toTabletInsertionEvents(final long timeout "Interrupted when waiting for closing TsFile %s.", resource.getTsFilePath()) : String.format( "Parse TsFile %s error. Because: %s", resource.getTsFilePath(), e.getMessage()); - LOGGER.warn(errorMsg, e); + if (e instanceof PipeRuntimeOutOfMemoryCriticalException) { + PipeLogger.log(LOGGER::warn, errorMsg); + } else { + PipeLogger.log(LOGGER::warn, e, errorMsg); + } throw new PipeException(errorMsg); } } @@ -566,28 +570,29 @@ private void waitForResourceEnough4Parsing(final long timeoutMs) throws Interrup final double waitTimeSeconds = (currentTime - startTime) / 1000.0; if (elapsedRecordTimeSeconds > 10.0) { LOGGER.info( - "Wait for resource enough for parsing {} for {} seconds.", + "Wait for memory enough for parsing {} for {} seconds.", resource != null ? resource.getTsFilePath() : "tsfile", waitTimeSeconds); lastRecordTime = currentTime; } else if (LOGGER.isDebugEnabled()) { LOGGER.debug( - "Wait for resource enough for parsing {} for {} seconds.", + "Wait for memory enough for parsing {} for {} seconds.", resource != null ? resource.getTsFilePath() : "tsfile", waitTimeSeconds); } if (waitTimeSeconds * 1000 > timeoutMs) { // should contain 'TimeoutException' in exception message - throw new PipeException( - String.format("TimeoutException: Waited %s seconds", waitTimeSeconds)); + throw new PipeRuntimeOutOfMemoryCriticalException( + String.format( + "TimeoutException: Waited %s seconds for memory to parse TsFile", waitTimeSeconds)); } } final long currentTime = System.currentTimeMillis(); final double waitTimeSeconds = (currentTime - startTime) / 1000.0; LOGGER.info( - "Wait for resource enough for parsing {} for {} seconds.", + "Wait for memory enough for parsing {} for {} seconds.", resource != null ? resource.getTsFilePath() : "tsfile", waitTimeSeconds); } From f06fbeea01f656d619cfbbee98e94afc328e929a Mon Sep 17 00:00:00 2001 From: Weihao Li <60659567+Wei-hao-Li@users.noreply.github.com> Date: Sun, 24 May 2026 09:00:29 +0800 Subject: [PATCH 054/102] [To dev/1.3] Fix forced Exchange branch isolation in distributed align-by-device planning --- .../IoTDBAlignByDeviceWildcardIT.java | 94 +++++++++++++++++++ .../distribution/DistributionPlanner.java | 24 +++-- .../distribution/ExchangeNodeAdder.java | 83 +++++++++++++--- .../plan/node/process/ExchangeNode.java | 18 +++- 4 files changed, 195 insertions(+), 24 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/db/it/aggregation/IoTDBAlignByDeviceWildcardIT.java diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/aggregation/IoTDBAlignByDeviceWildcardIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/aggregation/IoTDBAlignByDeviceWildcardIT.java new file mode 100644 index 0000000000000..e2bb07f23bee8 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/aggregation/IoTDBAlignByDeviceWildcardIT.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.it.aggregation; + +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.LocalStandaloneIT; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import static org.apache.iotdb.db.it.utils.TestUtils.prepareData; +import static org.apache.iotdb.db.it.utils.TestUtils.resultSetEqualTest; + +@RunWith(IoTDBTestRunner.class) +@Category({LocalStandaloneIT.class}) +public class IoTDBAlignByDeviceWildcardIT { + + private static final String[] SQL_LIST = + new String[] { + "CREATE DATABASE root.min", + "CREATE TIMESERIES root.min.d1.s1 WITH DATATYPE=INT32, ENCODING=PLAIN", + "CREATE TIMESERIES root.min.d1.s2 WITH DATATYPE=INT32, ENCODING=PLAIN", + "CREATE TIMESERIES root.min.d2.s1 WITH DATATYPE=INT32, ENCODING=PLAIN", + "CREATE TIMESERIES root.min.d2.s2 WITH DATATYPE=INT32, ENCODING=PLAIN", + "INSERT INTO root.min.d1(time, s1, s2) VALUES(1, 1, 1)", + "INSERT INTO root.min.d1(time, s1, s2) VALUES(2, 1, 1)", + "FLUSH", + "INSERT INTO root.min.d2(time, s1, s2) VALUES(5, 1, 1)", + "FLUSH" + }; + + @BeforeClass + public static void setUp() throws Exception { + EnvFactory.getEnv() + .getConfig() + .getCommonConfig() + .setDefaultDataRegionGroupNumPerDatabase(1) + .setTimePartitionInterval(1) + .setEnableSeqSpaceCompaction(false) + .setEnableUnseqSpaceCompaction(false) + .setEnableCrossSpaceCompaction(false); + EnvFactory.getEnv().initClusterEnvironment(); + prepareData(SQL_LIST); + } + + @AfterClass + public static void tearDown() throws Exception { + // EnvFactory.getEnv().cleanClusterEnvironment(); + } + + @Test + public void testWildcardAlignByDeviceWithTimePartitionSplit() { + String sql = + "SELECT count(s1) FROM root.min.** " + + "WHERE s2 is not null and s1 is not null " + + "GROUP BY([1, 6), 1ms) ALIGN BY DEVICE"; + String[] expectedHeader = new String[] {"Time", "Device", "count(s1)"}; + String[] expectedRows = + new String[] { + "1,root.min.d1,1,", + "2,root.min.d1,1,", + "3,root.min.d1,0,", + "4,root.min.d1,0,", + "5,root.min.d1,0,", + "1,root.min.d2,0,", + "2,root.min.d2,0,", + "3,root.min.d2,0,", + "4,root.min.d2,0,", + "5,root.min.d2,1,", + }; + resultSetEqualTest(sql, expectedHeader, expectedRows); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/DistributionPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/DistributionPlanner.java index c2ca7011039c9..1f1bb0c3080fa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/DistributionPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/DistributionPlanner.java @@ -145,13 +145,23 @@ private void adjustUpStreamHelper( ExchangeNode exchangeNode = (ExchangeNode) child; TRegionReplicaSet regionOfChild = context.getNodeDistribution(exchangeNode.getChild().getPlanNodeId()).getRegion(); - MultiChildrenSinkNode newChild = - memo.computeIfAbsent( - regionOfChild, - tRegionReplicaSet -> - needShuffleSinkNode - ? new ShuffleSinkNode(context.queryContext.getQueryId().genPlanNodeId()) - : new IdentitySinkNode(context.queryContext.getQueryId().genPlanNodeId())); + MultiChildrenSinkNode newChild; + if (exchangeNode.isForcedExchange()) { + // Keep forced exchange branch isolated: do not merge into shared sink memo. + newChild = + needShuffleSinkNode + ? new ShuffleSinkNode(context.queryContext.getQueryId().genPlanNodeId()) + : new IdentitySinkNode(context.queryContext.getQueryId().genPlanNodeId()); + } else { + newChild = + memo.computeIfAbsent( + regionOfChild, + tRegionReplicaSet -> + needShuffleSinkNode + ? new ShuffleSinkNode(context.queryContext.getQueryId().genPlanNodeId()) + : new IdentitySinkNode( + context.queryContext.getQueryId().genPlanNodeId())); + } newChild.addChild(exchangeNode.getChild()); newChild.addDownStreamChannelLocation( new DownStreamChannelLocation(exchangeNode.getPlanNodeId().toString())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/ExchangeNodeAdder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/ExchangeNodeAdder.java index f390d90c842ed..eb1490a6fc24d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/ExchangeNodeAdder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/ExchangeNodeAdder.java @@ -69,17 +69,17 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SeriesScanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.source.SourceNode; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import static com.google.common.collect.ImmutableList.toImmutableList; - public class ExchangeNodeAdder extends PlanVisitor { private final Analysis analysis; + private boolean containsInnerTimeJoinInCurrentSubtree = false; public ExchangeNodeAdder(Analysis analysis) { this.analysis = analysis; @@ -92,10 +92,7 @@ public PlanNode visitPlan(PlanNode node, NodeGroupContext context) { return node; } // Visit all the children of current node - List children = - node.getChildren().stream() - .map(child -> child.accept(this, context)) - .collect(toImmutableList()); + List children = visitChildrenAndRecordInnerTimeJoin(node.getChildren(), context); // Calculate the node distribution info according to its children @@ -215,7 +212,13 @@ private PlanNode processNoChildSourceNode(SourceNode node, NodeGroupContext cont @Override public PlanNode visitDeviceView(DeviceViewNode node, NodeGroupContext context) { - return processMultiChildNode(node, context); + List visitedChildren = + visitChildrenAndRecordInnerTimeJoin(node.getChildren(), context); + // Force Exchange for multi-child DeviceView if any child subtree contains InnerTimeJoin. + if (node.getChildren().size() > 1 && containsInnerTimeJoinInCurrentSubtree) { + return processMultiChildNodeWithForcedExchange(node, context, visitedChildren); + } + return processMultiChildNode(node, context, visitedChildren); } @Override @@ -236,7 +239,13 @@ public PlanNode visitSingleDeviceView(SingleDeviceViewNode node, NodeGroupContex @Override public PlanNode visitMergeSort(MergeSortNode node, NodeGroupContext context) { - return processMultiChildNode(node, context); + List visitedChildren = + visitChildrenAndRecordInnerTimeJoin(node.getChildren(), context); + // Force Exchange if any child subtree contains InnerTimeJoin. + if (containsInnerTimeJoinInCurrentSubtree) { + return processMultiChildNodeWithForcedExchange(node, context, visitedChildren); + } + return processMultiChildNode(node, context, visitedChildren); } @Override @@ -428,11 +437,14 @@ private PlanNode processMultiChildNode(MultiChildProcessNode node, NodeGroupCont return processMultiChildNodeByLocation(node, context); } - MultiChildProcessNode newNode = (MultiChildProcessNode) node.clone(); List visitedChildren = - node.getChildren().stream() - .map(child -> visit(child, context)) - .collect(Collectors.toList()); + visitChildrenAndRecordInnerTimeJoin(node.getChildren(), context); + return processMultiChildNode(node, context, visitedChildren); + } + + private PlanNode processMultiChildNode( + MultiChildProcessNode node, NodeGroupContext context, List visitedChildren) { + MultiChildProcessNode newNode = (MultiChildProcessNode) node.clone(); // DataRegion in which node locates TRegionReplicaSet dataRegion; @@ -546,13 +558,47 @@ private PlanNode processTopKNode( } private ExchangeNode genExchangeNode(NodeGroupContext context, PlanNode child) { + return genExchangeNode(context, child, false); + } + + private ExchangeNode genExchangeNode( + NodeGroupContext context, PlanNode child, boolean forcedExchange) { ExchangeNode exchangeNode = new ExchangeNode(context.queryContext.getQueryId().genPlanNodeId()); exchangeNode.setChild(child); exchangeNode.setOutputColumnNames(child.getOutputColumnNames()); + exchangeNode.setForcedExchange(forcedExchange); context.hasExchangeNode = true; return exchangeNode; } + private PlanNode processMultiChildNodeWithForcedExchange( + MultiChildProcessNode node, NodeGroupContext context, List visitedChildren) { + MultiChildProcessNode newNode = (MultiChildProcessNode) node.clone(); + for (PlanNode child : visitedChildren) { + newNode.addChild(genExchangeNode(context, child, true)); + } + context.putNodeDistribution( + newNode.getPlanNodeId(), + new NodeDistribution( + NodeDistributionType.SAME_WITH_SOME_CHILD, context.getMostlyUsedDataRegion())); + return newNode; + } + + private List visitChildrenAndRecordInnerTimeJoin( + List children, NodeGroupContext context) { + List result = new ArrayList<>(children.size()); + boolean originalTimeJoin = containsInnerTimeJoinInCurrentSubtree; + boolean hasInnerTimeJoinInChildren = false; + for (PlanNode child : children) { + containsInnerTimeJoinInCurrentSubtree = false; + PlanNode visitedChild = visit(child, context); + hasInnerTimeJoinInChildren |= containsInnerTimeJoinInCurrentSubtree; + result.add(visitedChild); + } + containsInnerTimeJoinInCurrentSubtree = originalTimeJoin || hasInnerTimeJoinInChildren; + return result; + } + @Override public PlanNode visitSlidingWindowAggregation( SlidingWindowAggregationNode node, NodeGroupContext context) { @@ -584,9 +630,9 @@ private TRegionReplicaSet calculateDataRegionByChildren( if (region == null && context.getNodeDistribution(child.getPlanNodeId()).getType() == NodeDistributionType.SAME_WITH_ALL_CHILDREN) { - return calculateSchemaRegionByChildren(child.getChildren(), context); + region = calculateSchemaRegionByChildren(child.getChildren(), context); } - return region; + return region == null ? DataPartition.NOT_ASSIGNED : region; }, Collectors.counting())); @@ -644,6 +690,13 @@ private boolean nodeDistributionIsSame(List children, NodeGroupContext } public PlanNode visit(PlanNode node, NodeGroupContext context) { - return node.accept(this, context); + boolean originalTimeJoin = containsInnerTimeJoinInCurrentSubtree; + containsInnerTimeJoinInCurrentSubtree = false; + PlanNode visitedNode = node.accept(this, context); + containsInnerTimeJoinInCurrentSubtree = + originalTimeJoin + || containsInnerTimeJoinInCurrentSubtree + || node instanceof InnerTimeJoinNode; + return visitedNode; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/process/ExchangeNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/process/ExchangeNode.java index a8cfa04c41043..9a1eef8ae1411 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/process/ExchangeNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/process/ExchangeNode.java @@ -48,6 +48,9 @@ public class ExchangeNode extends SingleChildProcessNode { /** Exchange needs to know which child of IdentitySinkNode/ShuffleSinkNode it matches */ private int indexOfUpstreamSinkHandle = 0; + /** Planner-only flag: this exchange is forced and should keep independent upstream sink. */ + private transient boolean forcedExchange = false; + public ExchangeNode(PlanNodeId id) { super(id); } @@ -72,6 +75,7 @@ public PlanNode clone() { ExchangeNode node = new ExchangeNode(getPlanNodeId()); node.setOutputColumnNames(outputColumnNames); node.setIndexOfUpstreamSinkHandle(indexOfUpstreamSinkHandle); + node.setForcedExchange(forcedExchange); return node; } @@ -162,6 +166,14 @@ public void setIndexOfUpstreamSinkHandle(int indexOfUpstreamSinkHandle) { this.indexOfUpstreamSinkHandle = indexOfUpstreamSinkHandle; } + public boolean isForcedExchange() { + return forcedExchange; + } + + public void setForcedExchange(boolean forcedExchange) { + this.forcedExchange = forcedExchange; + } + public TEndPoint getUpstreamEndpoint() { return upstreamEndpoint; } @@ -188,11 +200,13 @@ public boolean equals(Object o) { ExchangeNode that = (ExchangeNode) o; return Objects.equals(upstreamEndpoint, that.upstreamEndpoint) && Objects.equals(upstreamInstanceId, that.upstreamInstanceId) - && Objects.equals(upstreamPlanNodeId, that.upstreamPlanNodeId); + && Objects.equals(upstreamPlanNodeId, that.upstreamPlanNodeId) + && forcedExchange == that.forcedExchange; } @Override public int hashCode() { - return Objects.hash(super.hashCode(), upstreamEndpoint, upstreamInstanceId, upstreamPlanNodeId); + return Objects.hash( + super.hashCode(), upstreamEndpoint, upstreamInstanceId, upstreamPlanNodeId, forcedExchange); } } From 29ae788e59c2b7bb1ab148ffb567610fcf6014a4 Mon Sep 17 00:00:00 2001 From: Weihao Li <60659567+Wei-hao-Li@users.noreply.github.com> Date: Sun, 24 May 2026 09:00:58 +0800 Subject: [PATCH 055/102] [To dev/1.3] Make IoTDBLastQueryAliasITs query ordered --- .../iotdb/db/it/last/IoTDBLastQueryAliasIT.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/last/IoTDBLastQueryAliasIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/last/IoTDBLastQueryAliasIT.java index 4772804f38480..3e83a4ab5a290 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/last/IoTDBLastQueryAliasIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/last/IoTDBLastQueryAliasIT.java @@ -92,9 +92,13 @@ public void nonAlignedTest() { "1,root.test.d1.alias1,3,INT32,", }; resultSetEqualTest( - "select last alias3,alias2,alias1 from root.test.d1", expectedHeader, retArray); + "select last alias3,alias2,alias1 from root.test.d1 order by timeseries desc", + expectedHeader, + retArray); resultSetEqualTest( - "select last alias3,alias2,alias1 from root.test.d1", expectedHeader, retArray); + "select last alias3,alias2,alias1 from root.test.d1 order by timeseries desc", + expectedHeader, + retArray); retArray = new String[] { @@ -123,9 +127,13 @@ public void alignedTest() { "2,root.test.d2.alias1,4,INT32,", }; resultSetEqualTest( - "select last alias3,alias2,alias1 from root.test.d2", expectedHeader, retArray); + "select last alias3,alias2,alias1 from root.test.d2 order by timeseries desc", + expectedHeader, + retArray); resultSetEqualTest( - "select last alias3,alias2,alias1 from root.test.d2", expectedHeader, retArray); + "select last alias3,alias2,alias1 from root.test.d2 order by timeseries desc", + expectedHeader, + retArray); retArray = new String[] { From 3d535db48a4dd4ab9303d32cf7d984b2999c4d84 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 25 May 2026 09:55:28 +0800 Subject: [PATCH 056/102] Fix fast last empty cache handling in 1.3 (#17743) --- .../iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java index 2ae50d3605718..f8bd094b69ee4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java @@ -1008,7 +1008,8 @@ public TSExecuteStatementResp executeFastLastDataQueryForOneDeviceV2( if (timeValuePair == null) { allCached = false; break; - } else if (timeValuePair.getValue() == null) { + } else if (timeValuePair == DeviceLastCache.EMPTY_TIME_VALUE_PAIR + || timeValuePair.getValue() == null) { // there is no data for this sensor if (!canUseNullEntry) { allCached = false; From ac090721589c1cedb4c1edbb64ef582ce9ca045d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 25 May 2026 15:34:24 +0800 Subject: [PATCH 057/102] Pipe: Implemented batch logic for air-gap sink (#17667) (#17727) * air-gap-batch * fix --- .../task/subtask/sink/PipeSinkSubtask.java | 4 + .../airgap/IoTDBDataRegionAirGapSink.java | 301 ++++++++++++++++-- .../airgap/IoTDBDataRegionAirGapSinkTest.java | 171 ++++++++++ .../pipe/sink/protocol/IoTDBAirGapSink.java | 18 +- 4 files changed, 459 insertions(+), 35 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSinkTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java index ae48fcb277861..1e7c50f389e51 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java @@ -33,6 +33,7 @@ import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.metric.schema.PipeSchemaRegionSinkMetrics; import org.apache.iotdb.db.pipe.metric.sink.PipeDataRegionSinkMetrics; +import org.apache.iotdb.db.pipe.sink.protocol.airgap.IoTDBDataRegionAirGapSink; import org.apache.iotdb.db.pipe.sink.protocol.thrift.async.IoTDBDataRegionAsyncSink; import org.apache.iotdb.db.pipe.sink.protocol.thrift.sync.IoTDBDataRegionSyncSink; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; @@ -300,6 +301,9 @@ public int getBatchSize() { if (outputPipeConnector instanceof IoTDBDataRegionSyncSink) { return ((IoTDBDataRegionSyncSink) outputPipeConnector).getBatchSize(); } + if (outputPipeConnector instanceof IoTDBDataRegionAirGapSink) { + return ((IoTDBDataRegionAirGapSink) outputPipeConnector).getBatchSize(); + } return 0; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java index 6396c710abd01..13bcb537ae1ed 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.sink.limiter.TsFileSendRateLimiter; +import org.apache.iotdb.commons.utils.RetryUtils; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; @@ -30,6 +31,10 @@ import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.metric.overview.PipeResourceMetrics; import org.apache.iotdb.db.pipe.metric.sink.PipeDataRegionSinkMetrics; +import org.apache.iotdb.db.pipe.sink.payload.evolvable.batch.PipeTabletEventBatch; +import org.apache.iotdb.db.pipe.sink.payload.evolvable.batch.PipeTabletEventPlainBatch; +import org.apache.iotdb.db.pipe.sink.payload.evolvable.batch.PipeTabletEventTsFileBatch; +import org.apache.iotdb.db.pipe.sink.payload.evolvable.batch.PipeTransferBatchReqBuilder; import org.apache.iotdb.db.pipe.sink.payload.evolvable.request.PipeTransferTabletInsertNodeReq; import org.apache.iotdb.db.pipe.sink.payload.evolvable.request.PipeTransferTabletRawReq; import org.apache.iotdb.db.pipe.sink.payload.evolvable.request.PipeTransferTsFilePieceReq; @@ -38,6 +43,7 @@ import org.apache.iotdb.db.pipe.sink.payload.evolvable.request.PipeTransferTsFileSealWithModReq; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.storageengine.dataregion.wal.exception.WALPipeException; +import org.apache.iotdb.metrics.type.Histogram; import org.apache.iotdb.pipe.api.customizer.configuration.PipeConnectorRuntimeConfiguration; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.event.Event; @@ -46,13 +52,26 @@ import org.apache.iotdb.pipe.api.exception.PipeConnectionException; import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; +import org.apache.commons.io.FileUtils; +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.DataOutputStream; import java.io.File; import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.file.NoSuchFileException; import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.Objects; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_ENABLE_SEND_TSFILE_LIMIT; @@ -63,6 +82,7 @@ public class IoTDBDataRegionAirGapSink extends IoTDBDataNodeAirGapSink { private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBDataRegionAirGapSink.class); + private PipeTransferBatchReqBuilder tabletBatchBuilder; private boolean enableSendTsFileLimit; @Override @@ -71,6 +91,10 @@ public void customize( throws Exception { super.customize(parameters, configuration); + if (isTabletBatchModeEnabled) { + tabletBatchBuilder = new PipeTransferBatchReqBuilder(parameters); + } + enableSendTsFileLimit = parameters.getBooleanOrDefault( Arrays.asList(SINK_ENABLE_SEND_TSFILE_LIMIT, CONNECTOR_ENABLE_SEND_TSFILE_LIMIT), @@ -97,7 +121,10 @@ public void transfer(final TabletInsertionEvent tabletInsertionEvent) throws Exc // When receiver encountered packet loss, the transfer will time out // We need to restore the transfer quickly by retry under this circumstance socket.setSoTimeout(PIPE_CONFIG.getPipeAirGapSinkTabletTimeoutMs()); - if (tabletInsertionEvent instanceof PipeInsertNodeTabletInsertionEvent) { + if (isTabletBatchModeEnabled) { + tabletBatchBuilder.onEvent(tabletInsertionEvent); + doTransferWrapper(socket); + } else if (tabletInsertionEvent instanceof PipeInsertNodeTabletInsertionEvent) { doTransferWrapper(socket, (PipeInsertNodeTabletInsertionEvent) tabletInsertionEvent); } else { doTransferWrapper(socket, (PipeRawTabletInsertionEvent) tabletInsertionEvent); @@ -136,6 +163,9 @@ public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exc final AirGapSocket socket = sockets.get(socketIndex); try { + if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) { + doTransferWrapper(socket); + } doTransferWrapper(socket, (PipeTsFileInsertionEvent) tsFileInsertionEvent); } catch (final IOException e) { isSocketAlive.set(socketIndex, false); @@ -156,8 +186,18 @@ public void transfer(final Event event) throws Exception { try { if (event instanceof PipeSchemaRegionWritePlanEvent) { + if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) { + doTransferWrapper(socket); + } doTransferWrapper(socket, (PipeSchemaRegionWritePlanEvent) event); - } else if (!(event instanceof PipeHeartbeatEvent || event instanceof PipeTerminateEvent)) { + return; + } + + if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) { + doTransferWrapper(socket); + } + + if (!(event instanceof PipeHeartbeatEvent || event instanceof PipeTerminateEvent)) { LOGGER.warn( "IoTDBDataRegionAirGapConnector does not support transferring generic event: {}.", event); @@ -173,6 +213,63 @@ public void transfer(final Event event) throws Exception { } } + private void doTransferWrapper(final AirGapSocket socket) + throws IOException, WriteProcessException { + for (final Pair nonEmptyAndShouldEmitBatch : + tabletBatchBuilder.getAllNonEmptyAndShouldEmitBatches()) { + final PipeTabletEventBatch batch = nonEmptyAndShouldEmitBatch.getRight(); + if (batch instanceof PipeTabletEventPlainBatch) { + doTransfer(socket, (PipeTabletEventPlainBatch) batch); + } else if (batch instanceof PipeTabletEventTsFileBatch) { + doTransfer(socket, (PipeTabletEventTsFileBatch) batch); + } else { + LOGGER.warn("Unsupported batch type {}.", batch.getClass()); + } + batch.decreaseEventsReferenceCount(IoTDBDataRegionAirGapSink.class.getName(), true); + batch.onSuccess(); + } + } + + private void doTransfer( + final AirGapSocket socket, final PipeTabletEventPlainBatch batchToTransfer) + throws IOException { + if (!sendBatch( + socket, + toTPipeTransferBytes(batchToTransfer.toTPipeTransferReq()), + batchToTransfer.getPipe2BytesAccumulated())) { + final String errorMessage = + String.format("Transfer PipeTransferTabletBatchReq error. Socket: %s.", socket); + receiverStatusHandler.handle( + new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) + .setMessage(errorMessage), + errorMessage, + batchToTransfer.deepCopyEvents().toString()); + } + } + + private void doTransfer( + final AirGapSocket socket, final PipeTabletEventTsFileBatch batchToTransfer) + throws IOException, WriteProcessException { + final List sealedFiles = batchToTransfer.sealTsFiles(); + final Map, Double> pipe2WeightMap = batchToTransfer.deepCopyPipe2WeightMap(); + + for (final File tsFile : sealedFiles) { + doTransfer(pipe2WeightMap, socket, tsFile, null, tsFile.getName()); + try { + RetryUtils.retryOnException( + () -> { + FileUtils.delete(tsFile); + return null; + }); + } catch (final NoSuchFileException e) { + LOGGER.info("The file {} is not found, may already be deleted.", tsFile); + } catch (final Exception e) { + LOGGER.warn( + "Failed to delete batch file {}, this file should be deleted manually later", tsFile); + } + } + } + private void doTransferWrapper( final AirGapSocket socket, final PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletInsertionEvent) @@ -271,50 +368,152 @@ private void doTransferWrapper( private void doTransfer( final AirGapSocket socket, final PipeTsFileInsertionEvent pipeTsFileInsertionEvent) throws PipeException, IOException { - final String pipeName = pipeTsFileInsertionEvent.getPipeName(); - final long creationTime = pipeTsFileInsertionEvent.getCreationTime(); - final File tsFile = pipeTsFileInsertionEvent.getTsFile(); + doTransfer( + Collections.singletonMap( + new Pair<>( + pipeTsFileInsertionEvent.getPipeName(), pipeTsFileInsertionEvent.getCreationTime()), + 1.0), + socket, + pipeTsFileInsertionEvent.getTsFile(), + pipeTsFileInsertionEvent.isWithMod() && supportModsIfIsDataNodeReceiver + ? pipeTsFileInsertionEvent.getModFile() + : null, + pipeTsFileInsertionEvent.toString()); + } + + private void doTransfer( + final Map, Double> pipe2WeightMap, + final AirGapSocket socket, + final File tsFile, + final File modFile, + final String receiverStatusContext) + throws PipeException, IOException { final String errorMessage = String.format("Seal file %s error. Socket %s.", tsFile, socket); - // 1. Transfer file piece by piece, and mod if needed - if (pipeTsFileInsertionEvent.isWithMod() && supportModsIfIsDataNodeReceiver) { - final File modFile = pipeTsFileInsertionEvent.getModFile(); - transferFilePieces(pipeName, creationTime, modFile, socket, true); - transferFilePieces(pipeName, creationTime, tsFile, socket, true); - // 2. Transfer file seal signal with mod, which means the file is transferred completely - if (!send( - pipeName, - creationTime, + if (Objects.nonNull(modFile)) { + transferFilePieces(pipe2WeightMap, modFile, socket, true); + transferFilePieces(pipe2WeightMap, tsFile, socket, true); + if (!sendWeighted( socket, PipeTransferTsFileSealWithModReq.toTPipeTransferBytes( - modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length()))) { + modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length()), + pipe2WeightMap)) { receiverStatusHandler.handle( new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(errorMessage), errorMessage, - pipeTsFileInsertionEvent.toString()); + receiverStatusContext); } else { LOGGER.info("Successfully transferred file {}.", tsFile); } } else { - transferFilePieces(pipeName, creationTime, tsFile, socket, false); - // 2. Transfer file seal signal without mod, which means the file is transferred completely - if (!send( - pipeName, - creationTime, + transferFilePieces(pipe2WeightMap, tsFile, socket, false); + if (!sendWeighted( socket, - PipeTransferTsFileSealReq.toTPipeTransferBytes(tsFile.getName(), tsFile.length()))) { + PipeTransferTsFileSealReq.toTPipeTransferBytes(tsFile.getName(), tsFile.length()), + pipe2WeightMap)) { receiverStatusHandler.handle( new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) .setMessage(errorMessage), errorMessage, - pipeTsFileInsertionEvent.toString()); + receiverStatusContext); } else { LOGGER.info("Successfully transferred file {}.", tsFile); } } } + private void transferFilePieces( + final Map, Double> pipe2WeightMap, + final File file, + final AirGapSocket socket, + final boolean isMultiFile) + throws PipeException, IOException { + final int readFileBufferSize = PIPE_CONFIG.getPipeSinkReadFileBufferSize(); + final byte[] readBuffer = new byte[readFileBufferSize]; + long position = 0; + try (final RandomAccessFile reader = new RandomAccessFile(file, "r")) { + while (true) { + mayLimitRateAndRecordIO(readFileBufferSize); + final int readLength = reader.read(readBuffer); + if (readLength == -1) { + break; + } + + final byte[] payload = + readLength == readFileBufferSize + ? readBuffer + : Arrays.copyOfRange(readBuffer, 0, readLength); + if (!sendWeighted( + socket, + isMultiFile + ? getTransferMultiFilePieceBytes(file.getName(), position, payload) + : getTransferSingleFilePieceBytes(file.getName(), position, payload), + pipe2WeightMap)) { + final String errorMessage = + String.format("Transfer file %s error. Socket %s.", file, socket); + receiverStatusHandler.handle( + new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) + .setMessage(errorMessage), + errorMessage, + file.toString()); + } else { + position += readLength; + } + } + } + } + + private boolean sendBatch( + final AirGapSocket socket, + byte[] bytes, + final Map, Long> pipe2BytesAccumulated) + throws IOException { + final long uncompressedSize = bytes.length; + bytes = compressIfNeeded(bytes); + + final double compressionRatio = + uncompressedSize == 0 ? 1 : (double) bytes.length / uncompressedSize; + for (final Map.Entry, Long> entry : pipe2BytesAccumulated.entrySet()) { + rateLimitIfNeeded( + entry.getKey().getLeft(), + entry.getKey().getRight(), + socket.getEndPoint(), + (long) (entry.getValue() * compressionRatio)); + } + return sendBytes(socket, bytes); + } + + private boolean sendWeighted( + final AirGapSocket socket, byte[] bytes, final Map, Double> pipe2WeightMap) + throws IOException { + bytes = compressIfNeeded(bytes); + + for (final Map.Entry, Double> entry : pipe2WeightMap.entrySet()) { + rateLimitIfNeeded( + entry.getKey().getLeft(), + entry.getKey().getRight(), + socket.getEndPoint(), + (long) (bytes.length * entry.getValue())); + } + return sendBytes(socket, bytes); + } + + private byte[] toTPipeTransferBytes(final TPipeTransferReq req) throws IOException { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + ReadWriteIOUtils.write(req.version, outputStream); + ReadWriteIOUtils.write(req.type, outputStream); + + final ByteBuffer bodyBuffer = req.body.duplicate(); + final byte[] body = new byte[bodyBuffer.remaining()]; + bodyBuffer.get(body); + outputStream.write(body); + + return byteArrayOutputStream.toByteArray(); + } + } + @Override protected void mayLimitRateAndRecordIO(final long requiredBytes) { PipeResourceMetrics.getInstance().recordDiskIO(requiredBytes); @@ -343,4 +542,60 @@ protected byte[] compressIfNeeded(final byte[] reqInBytes) throws IOException { } return super.compressIfNeeded(reqInBytes); } + + @Override + public synchronized void discardEventsOfPipe( + final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + if (Objects.nonNull(tabletBatchBuilder)) { + tabletBatchBuilder.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); + } + } + + public int getBatchSize() { + return Objects.nonNull(tabletBatchBuilder) ? tabletBatchBuilder.size() : 0; + } + + @Override + public void close() { + if (tabletBatchBuilder != null) { + tabletBatchBuilder.close(); + } + + super.close(); + } + + @Override + public void setTabletBatchSizeHistogram(Histogram tabletBatchSizeHistogram) { + if (tabletBatchBuilder != null) { + tabletBatchBuilder.setTabletBatchSizeHistogram(tabletBatchSizeHistogram); + } + } + + @Override + public void setTsFileBatchSizeHistogram(Histogram tsFileBatchSizeHistogram) { + if (tabletBatchBuilder != null) { + tabletBatchBuilder.setTsFileBatchSizeHistogram(tsFileBatchSizeHistogram); + } + } + + @Override + public void setTabletBatchTimeIntervalHistogram(Histogram tabletBatchTimeIntervalHistogram) { + if (tabletBatchBuilder != null) { + tabletBatchBuilder.setTabletBatchTimeIntervalHistogram(tabletBatchTimeIntervalHistogram); + } + } + + @Override + public void setTsFileBatchTimeIntervalHistogram(Histogram tsFileBatchTimeIntervalHistogram) { + if (tabletBatchBuilder != null) { + tabletBatchBuilder.setTsFileBatchTimeIntervalHistogram(tsFileBatchTimeIntervalHistogram); + } + } + + @Override + public void setBatchEventSizeHistogram(Histogram eventSizeHistogram) { + if (tabletBatchBuilder != null) { + tabletBatchBuilder.setEventSizeHistogram(eventSizeHistogram); + } + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSinkTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSinkTest.java new file mode 100644 index 0000000000000..ef947f13c281b --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSinkTest.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.sink.protocol.airgap; + +import org.apache.iotdb.commons.pipe.agent.plugin.builtin.BuiltinPipePlugin; +import org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant; +import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; +import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskSinkRuntimeEnvironment; +import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeRequestType; +import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.sink.payload.evolvable.request.PipeTransferTabletBatchReq; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class IoTDBDataRegionAirGapSinkTest { + + @Test + public void testTransferTabletBatchOverAirGap() throws Exception { + try (final RecordingIoTDBDataRegionAirGapSink sink = new RecordingIoTDBDataRegionAirGapSink()) { + final PipeParameters parameters = buildParameters(false); + sink.validate(new PipeParameterValidator(parameters)); + sink.customize( + parameters, + new PipeTaskRuntimeConfiguration(new PipeTaskSinkRuntimeEnvironment("pipe", 1L, 1))); + sink.prepareSocket(); + + sink.transfer(createPipeRawTabletInsertionEvent("pipe", 1L, 1L)); + sink.transfer(createPipeRawTabletInsertionEvent("pipe", 1L, 2L)); + + Thread.sleep(300L); + sink.transfer(new PipeHeartbeatEvent("1", false)); + + Assert.assertEquals(1, sink.sentRequests.size()); + + final TPipeTransferReq req = toTPipeTransferReq(sink.sentRequests.get(0)); + Assert.assertEquals(PipeRequestType.TRANSFER_TABLET_BATCH.getType(), req.type); + + final PipeTransferTabletBatchReq batchReq = + PipeTransferTabletBatchReq.fromTPipeTransferReq(req); + Assert.assertEquals(2, batchReq.getTabletReqs().size()); + } + } + + @Test + public void testTransferTsFileBatchOverAirGap() throws Exception { + try (final RecordingIoTDBDataRegionAirGapSink sink = new RecordingIoTDBDataRegionAirGapSink()) { + final PipeParameters parameters = buildParameters(true); + sink.validate(new PipeParameterValidator(parameters)); + sink.customize( + parameters, + new PipeTaskRuntimeConfiguration(new PipeTaskSinkRuntimeEnvironment("pipe", 1L, 1))); + sink.prepareSocket(); + + sink.transfer(createPipeRawTabletInsertionEvent("pipe", 1L, 1L)); + sink.transfer(createPipeRawTabletInsertionEvent("pipe", 1L, 2L)); + + Thread.sleep(300L); + sink.transfer(new PipeHeartbeatEvent("1", false)); + + final List requestTypes = new ArrayList<>(); + for (final byte[] requestBytes : sink.sentRequests) { + requestTypes.add(toTPipeTransferReq(requestBytes).type); + } + + Assert.assertTrue(requestTypes.contains(PipeRequestType.TRANSFER_TS_FILE_PIECE.getType())); + Assert.assertTrue(requestTypes.contains(PipeRequestType.TRANSFER_TS_FILE_SEAL.getType())); + Assert.assertFalse(requestTypes.contains(PipeRequestType.TRANSFER_TABLET_RAW.getType())); + Assert.assertFalse(requestTypes.contains(PipeRequestType.TRANSFER_TABLET_BATCH.getType())); + } + } + + private PipeParameters buildParameters(final boolean useTsFileBatch) { + final Map attributes = new HashMap<>(); + attributes.put( + PipeSinkConstant.CONNECTOR_KEY, + BuiltinPipePlugin.IOTDB_AIR_GAP_CONNECTOR.getPipePluginName()); + attributes.put(PipeSinkConstant.CONNECTOR_IOTDB_NODE_URLS_KEY, "127.0.0.1:6668"); + attributes.put(PipeSinkConstant.CONNECTOR_IOTDB_BATCH_DELAY_MS_KEY, "200"); + attributes.put(PipeSinkConstant.CONNECTOR_IOTDB_BATCH_SIZE_KEY, "1048576"); + if (useTsFileBatch) { + attributes.put(PipeSinkConstant.CONNECTOR_FORMAT_KEY, "tsfile"); + } + return new PipeParameters(attributes); + } + + private PipeRawTabletInsertionEvent createPipeRawTabletInsertionEvent( + final String pipeName, final long creationTime, final long value) { + final List schemaList = + Arrays.asList(new MeasurementSchema("s1", TSDataType.INT64)); + final Tablet tablet = new Tablet("root.db.d1", schemaList, 1); + tablet.addTimestamp(0, value); + tablet.addValue("s1", 0, value); + return new PipeRawTabletInsertionEvent( + tablet, false, pipeName, creationTime, null, null, false); + } + + private static TPipeTransferReq toTPipeTransferReq(final byte[] requestBytes) { + final ByteBuffer buffer = ByteBuffer.wrap(requestBytes); + + final TPipeTransferReq req = new TPipeTransferReq(); + req.version = ReadWriteIOUtils.readByte(buffer); + req.type = ReadWriteIOUtils.readShort(buffer); + req.body = buffer.slice(); + return req; + } + + private static class RecordingIoTDBDataRegionAirGapSink extends IoTDBDataRegionAirGapSink { + + private final List sentRequests = new ArrayList<>(); + + private void prepareSocket() { + sockets.set(0, new TestingAirGapSocket()); + } + + @Override + protected int nextSocketIndex() { + return 0; + } + + @Override + protected boolean sendBytes(final AirGapSocket socket, final byte[] bytes) { + sentRequests.add(Arrays.copyOf(bytes, bytes.length)); + return true; + } + + private static class TestingAirGapSocket extends AirGapSocket { + + private TestingAirGapSocket() { + super("127.0.0.1", 6668); + } + + @Override + public synchronized void setSoTimeout(final int timeout) { + // No-op for unit test. + } + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBAirGapSink.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBAirGapSink.java index 763bfa644ead7..498baeda5822f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBAirGapSink.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/IoTDBAirGapSink.java @@ -104,14 +104,6 @@ public void customize( final PipeParameters parameters, final PipeConnectorRuntimeConfiguration configuration) throws Exception { super.customize(parameters, configuration); - - if (isTabletBatchModeEnabled) { - LOGGER.warn( - "Batch mode is enabled by the given parameters. " - + "IoTDBAirGapConnector does not support batch mode. " - + "Disable batch mode."); - } - for (int i = 0; i < nodeUrls.size(); i++) { isSocketAlive.add(false); sockets.add(null); @@ -323,15 +315,17 @@ protected int nextSocketIndex() { protected boolean send( final String pipeName, final long creationTime, final AirGapSocket socket, byte[] bytes) throws IOException { + bytes = compressIfNeeded(bytes); + rateLimitIfNeeded(pipeName, creationTime, socket.getEndPoint(), bytes.length); + return sendBytes(socket, bytes); + } + + protected boolean sendBytes(final AirGapSocket socket, byte[] bytes) throws IOException { if (!socket.isConnected()) { throw new SocketException( String.format("Socket %s is closed, will try to handshake", socket)); } - bytes = compressIfNeeded(bytes); - - rateLimitIfNeeded(pipeName, creationTime, socket.getEndPoint(), bytes.length); - final BufferedOutputStream outputStream = new BufferedOutputStream(socket.getOutputStream()); bytes = enrichWithLengthAndChecksum(bytes); outputStream.write(eLanguageEnable ? enrichWithELanguage(bytes) : bytes); From afcc36f0c765e86ce0afbccf82f3d6b19c756735 Mon Sep 17 00:00:00 2001 From: Haonan Date: Mon, 25 May 2026 17:28:00 +0800 Subject: [PATCH 058/102] Upgrade Netty for CVE fixes (#17760) --- LICENSE-binary | 22 +++++++++++----------- pom.xml | 2 +- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index f3fadab3af857..e64118ab4394f 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -241,17 +241,17 @@ io.dropwizard.metrics:metrics-core:4.2.19 io.dropwizard.metrics:metrics-jvm:3.2.2 com.librato.metrics:metrics-librato:5.1.0 com.github.moquette-io.moquette:moquette-broker:0.18 -io.netty:netty-buffer:4.1.126.Final -io.netty:netty-codec:4.1.126.Final -io.netty:netty-codec-http:4.1.126.Final -io.netty:netty-codec-mqtt:4.1.126.Final -io.netty:netty-common:4.1.126.Final -io.netty:netty-handler:4.1.126.Final -io.netty:netty-resolver:4.1.126.Final -io.netty:netty-transport:4.1.126.Final -io.netty:netty-transport-native-epoll:4.1.126.Final:linux-aarch_64 -io.netty:netty-transport-native-epoll:4.1.126.Final:linux-x86_64 -io.netty:netty-transport-native-unix-common:4.1.126.Final +io.netty:netty-buffer:4.1.134.Final +io.netty:netty-codec:4.1.134.Final +io.netty:netty-codec-http:4.1.134.Final +io.netty:netty-codec-mqtt:4.1.134.Final +io.netty:netty-common:4.1.134.Final +io.netty:netty-handler:4.1.134.Final +io.netty:netty-resolver:4.1.134.Final +io.netty:netty-transport:4.1.134.Final +io.netty:netty-transport-native-epoll:4.1.134.Final:linux-aarch_64 +io.netty:netty-transport-native-epoll:4.1.134.Final:linux-x86_64 +io.netty:netty-transport-native-unix-common:4.1.134.Final org.osgi:org.osgi.core:7.0.0 org.osgi:osgi.cmpn:7.0.0 org.ops4j.pax.jdbc:pax-jdbc-common:1.5.6 diff --git a/pom.xml b/pom.xml index cbe907c71b934..a83a66e135c5a 100644 --- a/pom.xml +++ b/pom.xml @@ -126,7 +126,7 @@ 0.18.0 - 4.1.126.Final + 4.1.134.Final 6.6.0 7.0.0 From 7721cab039215b892c718d1a575e9f6ee71f8b46 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 27 May 2026 15:47:21 +0800 Subject: [PATCH 059/102] Pipe: Do not listen to tsFiles when no sources need (#17669) (#17761) * assigner * Update PipeDataRegionAssigner.java * fix (cherry picked from commit 906b86f784bc60b515b3d51347b18478b103803c) --- .../assigner/PipeDataRegionAssigner.java | 48 ++++++++++- .../PipeInsertionDataNodeListener.java | 47 ++++------- .../pipe/source/PipeRealtimeExtractTest.java | 81 +++++++++++++++++++ 3 files changed, 139 insertions(+), 37 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java index 0b4eb547144af..2375726e42772 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -56,6 +56,9 @@ public class PipeDataRegionAssigner implements Closeable { private final String dataRegionId; + private volatile int listenToTsFileSourceCount = 0; + private volatile int listenToInsertNodeSourceCount = 0; + private final PipeEventCounter eventCounter = new PipeDataRegionEventCounter(); public String getDataRegionId() { @@ -194,12 +197,34 @@ private void assignToSource( }); } - public void startAssignTo(final PipeRealtimeDataRegionSource extractor) { - matcher.register(extractor); + public synchronized void startAssignTo(final PipeRealtimeDataRegionSource source) { + matcher.register(source); + if (source.isNeedListenToTsFile()) { + listenToTsFileSourceCount++; + } + if (source.isNeedListenToInsertNode()) { + listenToInsertNodeSourceCount++; + } + logSourceAssignmentChange("registered", source); + } + + public synchronized void stopAssignTo(final PipeRealtimeDataRegionSource source) { + matcher.deregister(source); + if (source.isNeedListenToTsFile()) { + listenToTsFileSourceCount--; + } + if (source.isNeedListenToInsertNode()) { + listenToInsertNodeSourceCount--; + } + logSourceAssignmentChange("deregistered", source); } - public void stopAssignTo(final PipeRealtimeDataRegionSource extractor) { - matcher.deregister(extractor); + public boolean shouldListenToTsFile() { + return listenToTsFileSourceCount > 0; + } + + public boolean shouldListenToInsertNode() { + return listenToInsertNodeSourceCount > 0; } public boolean notMoreSourceNeededToBeAssigned() { @@ -236,4 +261,19 @@ public int getTsFileInsertionEventCount() { public int getPipeHeartbeatEventCount() { return eventCounter.getPipeHeartbeatEventCount(); } + + private void logSourceAssignmentChange( + final String action, final PipeRealtimeDataRegionSource source) { + LOGGER.info( + "Pipe {}@{} {} realtime source on data region {} (listenToTsFile={}, listenToInsertNode={}, registeredSourceCount={}, tsFileSourceCount={}, insertNodeSourceCount={}).", + source.getPipeName(), + source.getCreationTime(), + action, + dataRegionId, + source.isNeedListenToTsFile(), + source.isNeedListenToInsertNode(), + matcher.getRegisterCount(), + listenToTsFileSourceCount, + listenToInsertNodeSourceCount); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 882d4aff0d8eb..ad3586df830e9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -30,7 +30,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicInteger; /** * PipeInsertionEventListener is a singleton in each data node. @@ -48,23 +47,20 @@ public class PipeInsertionDataNodeListener { private final ConcurrentMap dataRegionId2Assigner = new ConcurrentHashMap<>(); - private final AtomicInteger listenToTsFileSourceCount = new AtomicInteger(0); - private final AtomicInteger listenToInsertNodeSourceCount = new AtomicInteger(0); - //////////////////////////// start & stop //////////////////////////// public synchronized void startListenAndAssign( final String dataRegionId, final PipeRealtimeDataRegionSource source) { - dataRegionId2Assigner - .computeIfAbsent(dataRegionId, o -> new PipeDataRegionAssigner(dataRegionId)) - .startAssignTo(source); - - if (source.isNeedListenToTsFile()) { - listenToTsFileSourceCount.incrementAndGet(); - } - if (source.isNeedListenToInsertNode()) { - listenToInsertNodeSourceCount.incrementAndGet(); - } + // Keep registration inside compute so the assigner is fully started before it becomes visible + // to concurrent listeners. + dataRegionId2Assigner.compute( + dataRegionId, + (id, assigner) -> { + final PipeDataRegionAssigner actualAssigner = + assigner == null ? new PipeDataRegionAssigner(dataRegionId) : assigner; + actualAssigner.startAssignTo(source); + return actualAssigner; + }); } public synchronized void stopListenAndAssign( @@ -79,13 +75,6 @@ public synchronized void stopListenAndAssign( assigner.stopAssignTo(source); - if (source.isNeedListenToTsFile()) { - listenToTsFileSourceCount.decrementAndGet(); - } - if (source.isNeedListenToInsertNode()) { - listenToInsertNodeSourceCount.decrementAndGet(); - } - if (assigner.notMoreSourceNeededToBeAssigned()) { // The removed assigner will is the same as the one referenced by the variable `assigner` dataRegionId2Assigner.remove(dataRegionId); @@ -104,14 +93,10 @@ public synchronized void stopListenAndAssign( public void listenToTsFile( final String dataRegionId, final TsFileResource tsFileResource, final boolean isLoaded) { - // We don't judge whether listenToTsFileSourceCount.get() == 0 here on purpose - // because sources may use tsfile events when some exceptions occur in the - // insert nodes listening process. - final PipeDataRegionAssigner assigner = dataRegionId2Assigner.get(dataRegionId); - // only events from registered data region will be extracted - if (assigner == null) { + // only events from registered data region with tsfile listeners will be extracted + if (assigner == null || !assigner.shouldListenToTsFile()) { return; } @@ -121,14 +106,10 @@ public void listenToTsFile( public void listenToInsertNode( final String dataRegionId, final InsertNode insertNode, final TsFileResource tsFileResource) { - if (listenToInsertNodeSourceCount.get() == 0) { - return; - } - final PipeDataRegionAssigner assigner = dataRegionId2Assigner.get(dataRegionId); - // only events from registered data region will be extracted - if (assigner == null) { + // only events from registered data region with insert listeners will be extracted + if (assigner == null || !assigner.shouldListenToInsertNode()) { return; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/PipeRealtimeExtractTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/PipeRealtimeExtractTest.java index 9e5b42bb93c4d..2cba4c8eebcaa 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/PipeRealtimeExtractTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/PipeRealtimeExtractTest.java @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskSourceRuntimeEnvironment; import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.source.dataregion.realtime.PipeRealtimeDataRegionHybridSource; import org.apache.iotdb.db.pipe.source.dataregion.realtime.PipeRealtimeDataRegionLogSource; import org.apache.iotdb.db.pipe.source.dataregion.realtime.PipeRealtimeDataRegionSource; @@ -39,6 +40,7 @@ import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; +import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.enums.TSDataType; @@ -63,6 +65,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; public class PipeRealtimeExtractTest { @@ -268,6 +271,52 @@ public void testRealtimeExtractProcess() { } } + @Test + public void testListenToTsFileSkipsAssignerWithoutTsFileSource() throws Exception { + try (final NoTsFileRealtimeDataRegionSource extractor = + new NoTsFileRealtimeDataRegionSource()) { + final PipeParameters parameters = + new PipeParameters( + new HashMap() { + { + put(PipeSourceConstant.EXTRACTOR_PATTERN_KEY, pattern1); + } + }); + final PipeTaskRuntimeConfiguration configuration = + new PipeTaskRuntimeConfiguration( + new PipeTaskSourceRuntimeEnvironment( + "1", + 1, + Integer.parseInt(dataRegion1), + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1))); + + extractor.validate(new PipeParameterValidator(parameters)); + extractor.customize(parameters, configuration); + extractor.start(); + + final File dataRegionDir = + new File(tsFileDir.getPath() + File.separator + dataRegion1 + File.separator + "0"); + final boolean ignored = dataRegionDir.mkdirs(); + final File tsFile = new File(dataRegionDir, "0-0-0-0.tsfile"); + Assert.assertTrue(tsFile.createNewFile()); + + final TsFileResource resource = new TsFileResource(tsFile); + resource.updateStartTime( + new PlainDeviceID(String.join(TsFileConstant.PATH_SEPARATOR, device)), 0); + resource.close(); + + PipeInsertionDataNodeListener.getInstance().listenToTsFile(dataRegion1, resource, false); + + final long deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(1); + while (System.currentTimeMillis() < deadline + && extractor.getObservedTsFileEventCount() == 0) { + TimeUnit.MILLISECONDS.sleep(10); + } + + Assert.assertEquals(0, extractor.getObservedTsFileEventCount()); + } + } + private Future write2DataRegion( final int writeNum, final String dataRegionId, final int startNum) { final File dataRegionDir = @@ -351,4 +400,36 @@ private Future listen( } }); } + + private static class NoTsFileRealtimeDataRegionSource extends PipeRealtimeDataRegionSource { + + private final AtomicInteger observedTsFileEventCount = new AtomicInteger(0); + + @Override + public Event supply() { + return null; + } + + @Override + protected void doExtract(final PipeRealtimeEvent event) { + if (event.getEvent() instanceof TsFileInsertionEvent) { + observedTsFileEventCount.incrementAndGet(); + } + event.decreaseReferenceCount(NoTsFileRealtimeDataRegionSource.class.getName(), false); + } + + @Override + public boolean isNeedListenToTsFile() { + return false; + } + + @Override + public boolean isNeedListenToInsertNode() { + return false; + } + + private int getObservedTsFileEventCount() { + return observedTsFileEventCount.get(); + } + } } From b061ffd003a6254a9f05a7b1cb4212033c7ea95e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 27 May 2026 15:47:38 +0800 Subject: [PATCH 060/102] [To dev/1.3] Add historical transfer summary logs (#17763) Backport 18ea0e9aadc2c39a1fcbf8e34c559ae142b71221 (#17717) to dev/1.3. --- .../heartbeat/PipeHeartbeatParser.java | 13 ++ .../task/connection/PipeEventCollector.java | 9 + .../common/terminate/PipeTerminateEvent.java | 179 ++++++++++++++++++ .../PipeHistoricalDataRegionTsFileSource.java | 16 ++ .../receiver/PipeReceiverStatusHandler.java | 29 +++ 5 files changed, 246 insertions(+) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParser.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParser.java index 6dc11ddd3f3bd..8cafa2094c446 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParser.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatParser.java @@ -154,11 +154,24 @@ private void parseHeartbeatAndSaveMetaChangeLocally( if (Boolean.TRUE.equals(isPipeCompletedFromAgent)) { temporaryMeta.markDataNodeCompleted(nodeId); + LOGGER.info( + "Detected historical pipe completion report from DataNode {} for pipe {}. remainingEventCount: {}, remainingTime: {}, completedDataNodes: {}", + nodeId, + staticMeta.getPipeName(), + pipeHeartbeat.getRemainingEventCount(staticMeta), + pipeHeartbeat.getRemainingTime(staticMeta), + temporaryMeta.getCompletedDataNodeIds()); final Set uncompletedDataNodeIds = configManager.getNodeManager().getRegisteredDataNodeLocations().keySet(); uncompletedDataNodeIds.removeAll(temporaryMeta.getCompletedDataNodeIds()); if (uncompletedDataNodeIds.isEmpty()) { + LOGGER.info( + "All DataNodes reported historical pipe {} completed. globalRemainingEventCount: {}, globalRemainingTime: {}, staticMeta: {}", + staticMeta.getPipeName(), + temporaryMeta.getGlobalRemainingEvents(), + temporaryMeta.getGlobalRemainingTime(), + staticMeta); pipeTaskInfo.get().removePipeMeta(staticMeta.getPipeName()); LOGGER.info( "Detected completion of pipe {}, static meta: {}, remove it.", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java index 387d4ff7ec621..95e8196ad3865 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java @@ -29,6 +29,7 @@ import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.source.schemaregion.IoTDBSchemaRegionSource; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; @@ -131,12 +132,20 @@ private void parseAndCollectEvent(final PipeTsFileInsertionEvent sourceEvent) th if (skipParsing || !forceTabletFormat && canSkipParsing4TsFileEvent(sourceEvent)) { collectEvent(sourceEvent); + if (sourceEvent.isGeneratedByHistoricalExtractor()) { + PipeTerminateEvent.markHistoricalTsFileUnsplit( + sourceEvent.getPipeName(), sourceEvent.getCreationTime(), regionId); + } return; } try { sourceEvent.consumeTabletInsertionEventsWithRetry( this::collectParsedRawTableEvent, "PipeEventCollector::parseAndCollectEvent"); + if (sourceEvent.isGeneratedByHistoricalExtractor()) { + PipeTerminateEvent.markHistoricalTsFileSplit( + sourceEvent.getPipeName(), sourceEvent.getCreationTime(), regionId); + } } finally { sourceEvent.close(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java index 4bf79a3df030c..f57c016e246a4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java @@ -32,9 +32,16 @@ import org.apache.iotdb.db.pipe.agent.task.PipeDataNodeTask; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; /** * The {@link PipeTerminateEvent} is an {@link EnrichedEvent} that controls the termination of pipe, @@ -44,6 +51,8 @@ */ public class PipeTerminateEvent extends EnrichedEvent { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeTerminateEvent.class); + private final int dataRegionId; private final boolean shouldMark; @@ -57,6 +66,9 @@ public class PipeTerminateEvent extends EnrichedEvent { // Do not use call run policy to avoid deadlock private static final ExecutorService terminateExecutor = createTerminateExecutor(); + private static final ConcurrentMap + HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP = new ConcurrentHashMap<>(); + private static ExecutorService createTerminateExecutor() { final WrappedThreadPoolExecutor executor = new WrappedThreadPoolExecutor( @@ -128,6 +140,18 @@ public boolean mayEventPathsOverlappedWithPattern() { } public void markCompleted() { + final HistoricalTransferSummary summary = + snapshotAndClearHistoricalTransferSummary(pipeName, creationTime, dataRegionId); + if (Objects.nonNull(summary)) { + LOGGER.info( + "Pipe {}@{}: terminate event committed for historical transfer. creationTime: {}, shouldMark: {}. {}", + pipeName, + dataRegionId, + creationTime, + shouldMark, + summary.toReportMessage()); + } + // To avoid deadlock if (shouldMark) { terminateExecutor.submit( @@ -142,4 +166,159 @@ public String toString() { + " - " + super.toString(); } + + public static void initializeHistoricalTransferSummary( + final String pipeName, + final long creationTime, + final int dataRegionId, + final long extractedHistoricalTsFileCount, + final long extractedHistoricalDeletionCount) { + HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP + .computeIfAbsent( + new HistoricalTransferKey(pipeName, creationTime, dataRegionId), + ignored -> new HistoricalTransferSummaryCounter()) + .initialize(extractedHistoricalTsFileCount, extractedHistoricalDeletionCount); + } + + public static void markHistoricalTsFileSkipped( + final String pipeName, final long creationTime, final int dataRegionId) { + getOrCreateHistoricalTransferSummaryCounter(pipeName, creationTime, dataRegionId) + .skippedHistoricalTsFileCount + .incrementAndGet(); + } + + public static void markHistoricalTsFileSplit( + final String pipeName, final long creationTime, final int dataRegionId) { + getOrCreateHistoricalTransferSummaryCounter(pipeName, creationTime, dataRegionId) + .splitHistoricalTsFileCount + .incrementAndGet(); + } + + public static void markHistoricalTsFileUnsplit( + final String pipeName, final long creationTime, final int dataRegionId) { + getOrCreateHistoricalTransferSummaryCounter(pipeName, creationTime, dataRegionId) + .unsplitHistoricalTsFileCount + .incrementAndGet(); + } + + public static HistoricalTransferSummary snapshotHistoricalTransferSummary( + final String pipeName, final long creationTime, final int dataRegionId) { + final HistoricalTransferSummaryCounter counter = + HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP.get( + new HistoricalTransferKey(pipeName, creationTime, dataRegionId)); + return Objects.nonNull(counter) ? counter.snapshot() : null; + } + + public static void clearHistoricalTransferSummary( + final String pipeName, final long creationTime, final int dataRegionId) { + HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP.remove( + new HistoricalTransferKey(pipeName, creationTime, dataRegionId)); + } + + private static HistoricalTransferSummary snapshotAndClearHistoricalTransferSummary( + final String pipeName, final long creationTime, final int dataRegionId) { + final HistoricalTransferSummaryCounter counter = + HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP.remove( + new HistoricalTransferKey(pipeName, creationTime, dataRegionId)); + return Objects.nonNull(counter) ? counter.snapshot() : null; + } + + private static HistoricalTransferSummaryCounter getOrCreateHistoricalTransferSummaryCounter( + final String pipeName, final long creationTime, final int dataRegionId) { + return HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP.computeIfAbsent( + new HistoricalTransferKey(pipeName, creationTime, dataRegionId), + ignored -> new HistoricalTransferSummaryCounter()); + } + + public static final class HistoricalTransferSummary { + + private final long extractedHistoricalTsFileCount; + private final long skippedHistoricalTsFileCount; + private final long splitHistoricalTsFileCount; + private final long unsplitHistoricalTsFileCount; + private final long extractedHistoricalDeletionCount; + + private HistoricalTransferSummary( + final long extractedHistoricalTsFileCount, + final long skippedHistoricalTsFileCount, + final long splitHistoricalTsFileCount, + final long unsplitHistoricalTsFileCount, + final long extractedHistoricalDeletionCount) { + this.extractedHistoricalTsFileCount = extractedHistoricalTsFileCount; + this.skippedHistoricalTsFileCount = skippedHistoricalTsFileCount; + this.splitHistoricalTsFileCount = splitHistoricalTsFileCount; + this.unsplitHistoricalTsFileCount = unsplitHistoricalTsFileCount; + this.extractedHistoricalDeletionCount = extractedHistoricalDeletionCount; + } + + public String toReportMessage() { + return String.format( + "historical summary: extractedTsFileCount=%s, skippedTsFileCount=%s, splitTsFileCount=%s, unsplitTsFileCount=%s, deletionCount=%s", + extractedHistoricalTsFileCount, + skippedHistoricalTsFileCount, + splitHistoricalTsFileCount, + unsplitHistoricalTsFileCount, + extractedHistoricalDeletionCount); + } + } + + private static final class HistoricalTransferSummaryCounter { + + private final AtomicLong extractedHistoricalTsFileCount = new AtomicLong(0); + private final AtomicLong skippedHistoricalTsFileCount = new AtomicLong(0); + private final AtomicLong splitHistoricalTsFileCount = new AtomicLong(0); + private final AtomicLong unsplitHistoricalTsFileCount = new AtomicLong(0); + private final AtomicLong extractedHistoricalDeletionCount = new AtomicLong(0); + + private void initialize( + final long extractedHistoricalTsFileCount, final long extractedHistoricalDeletionCount) { + this.extractedHistoricalTsFileCount.set(extractedHistoricalTsFileCount); + this.skippedHistoricalTsFileCount.set(0); + this.splitHistoricalTsFileCount.set(0); + this.unsplitHistoricalTsFileCount.set(0); + this.extractedHistoricalDeletionCount.set(extractedHistoricalDeletionCount); + } + + private HistoricalTransferSummary snapshot() { + return new HistoricalTransferSummary( + extractedHistoricalTsFileCount.get(), + skippedHistoricalTsFileCount.get(), + splitHistoricalTsFileCount.get(), + unsplitHistoricalTsFileCount.get(), + extractedHistoricalDeletionCount.get()); + } + } + + private static final class HistoricalTransferKey { + + private final String pipeName; + private final long creationTime; + private final int dataRegionId; + + private HistoricalTransferKey( + final String pipeName, final long creationTime, final int dataRegionId) { + this.pipeName = pipeName; + this.creationTime = creationTime; + this.dataRegionId = dataRegionId; + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof HistoricalTransferKey)) { + return false; + } + final HistoricalTransferKey that = (HistoricalTransferKey) obj; + return creationTime == that.creationTime + && dataRegionId == that.dataRegionId + && Objects.equals(pipeName, that.pipeName); + } + + @Override + public int hashCode() { + return Objects.hash(pipeName, creationTime, dataRegionId); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileSource.java index fd960584c9f8c..54597cb1bcdac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileSource.java @@ -441,6 +441,8 @@ && mayTsFileResourceOverlappedWithPattern(resource))) ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) : o1.getMaxProgressIndex().topologicalCompareTo(o2.getMaxProgressIndex())); pendingQueue = new ArrayDeque<>(originalResourceList); + PipeTerminateEvent.initializeHistoricalTransferSummary( + pipeName, creationTime, dataRegionId, filteredTsFileResources.size(), 0); LOGGER.info( "Pipe {}@{}: finish to extract historical TsFile, extracted sequence file count {}/{}, " @@ -537,6 +539,17 @@ public synchronized Event supply() { final TsFileResource resource = pendingQueue.poll(); if (resource == null) { + final PipeTerminateEvent.HistoricalTransferSummary historicalTransferSummary = + PipeTerminateEvent.snapshotHistoricalTransferSummary( + pipeName, creationTime, dataRegionId); + if (Objects.nonNull(historicalTransferSummary)) { + LOGGER.info( + "Pipe {}@{}: historical source has supplied all events, emitting terminate event. {}", + pipeName, + dataRegionId, + historicalTransferSummary.toReportMessage()); + } + final PipeTerminateEvent terminateEvent = new PipeTerminateEvent( pipeName, @@ -632,6 +645,9 @@ public int getPendingQueueSize() { @Override public synchronized void close() { + if (!isTerminateSignalSent) { + PipeTerminateEvent.clearHistoricalTransferSummary(pipeName, creationTime, dataRegionId); + } if (Objects.nonNull(pendingQueue)) { pendingQueue.forEach( resource -> { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java index d48a9fdbcb2d5..604ad044dd6af 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/PipeReceiverStatusHandler.java @@ -46,6 +46,7 @@ public class PipeReceiverStatusHandler { private static final String NO_PERMISSION = "No permission"; private static final String UNCLASSIFIED_EXCEPTION = "Unclassified exception"; private static final String NO_PERMISSION_STR = "No permissions for this operation"; + private static final int MAX_RECORD_MESSAGE_LENGTH_IN_LOG = 2048; private final boolean isRetryAllowedWhenConflictOccurs; private final long retryMaxMillisWhenConflictOccurs; @@ -134,6 +135,7 @@ public void handle( "User conflict exception: will be ignored because retry is not allowed. event: {}. status: {}", shouldRecordIgnoredDataWhenConflictOccurs ? recordMessage : "not recorded", status); + logDiscardedUserConflictData("retry is not allowed", recordMessage, status); return; } @@ -147,6 +149,7 @@ public void handle( "User conflict exception: retry timeout. will be ignored. event: {}. status: {}", shouldRecordIgnoredDataWhenConflictOccurs ? recordMessage : "not recorded", status); + logDiscardedUserConflictData("retry timeout", recordMessage, status); resetExceptionStatus(); return; } @@ -252,6 +255,32 @@ private static String getNoPermission(final boolean noPermission) { return noPermission ? NO_PERMISSION : UNCLASSIFIED_EXCEPTION; } + private void logDiscardedUserConflictData( + final String reason, final String recordMessage, final TSStatus status) { + if (!LOGGER.isWarnEnabled()) { + return; + } + + LOGGER.warn( + "User conflict exception: discarded data info because {}. data: {}. receiver message: {}. status: {}", + reason, + summarizeRecordMessage(recordMessage), + status.getMessage(), + status); + } + + private String summarizeRecordMessage(final String recordMessage) { + if (Objects.isNull(recordMessage) || recordMessage.isEmpty()) { + return ""; + } + + final String normalizedRecordMessage = + recordMessage.replace('\r', ' ').replace('\n', ' ').trim(); + return normalizedRecordMessage.length() <= MAX_RECORD_MESSAGE_LENGTH_IN_LOG + ? normalizedRecordMessage + : normalizedRecordMessage.substring(0, MAX_RECORD_MESSAGE_LENGTH_IN_LOG) + "...(truncated)"; + } + private void recordExceptionStatusIfNecessary(final String message) { if (!Objects.equals(exceptionRecordedMessage.get(), message)) { exceptionFirstEncounteredTime.set(System.currentTimeMillis()); From 7cc3851892aad611cd14b0b000729cc8c5fa2c7e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 27 May 2026 15:48:02 +0800 Subject: [PATCH 061/102] Pipe: Stablized the tree model IoTDBPipeClusterIT (#17729) (#17762) * Update IoTDBPipeClusterIT.java * Fix --- .../it/autocreate/IoTDBPipeClusterIT.java | 183 ++++++++++++++---- 1 file changed, 149 insertions(+), 34 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java index 9617de89cd70a..96ec056cd6cf5 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java @@ -19,11 +19,13 @@ package org.apache.iotdb.pipe.it.autocreate; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient; import org.apache.iotdb.commons.cluster.RegionRoleType; import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq; +import org.apache.iotdb.confignode.rpc.thrift.TRegionInfo; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeInfo; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq; @@ -49,18 +51,22 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import static org.awaitility.Awaitility.await; import static org.junit.Assert.fail; @RunWith(IoTDBTestRunner.class) @Category({MultiClusterIT2AutoCreateSchema.class}) public class IoTDBPipeClusterIT extends AbstractPipeDualAutoIT { + private static final double SYNC_LAG_DELTA = 0.001; + @Override @Before public void setUp() { @@ -77,6 +83,10 @@ public void setUp() { .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS) .setPipeMemoryManagementEnabled(false) .setIsPipeEnableMemoryCheck(false); + senderEnv + .getConfig() + .getDataNodeConfig() + .setMetricReporterType(Collections.singletonList("PROMETHEUS")); receiverEnv .getConfig() @@ -89,6 +99,10 @@ public void setUp() { .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS) .setPipeMemoryManagementEnabled(false) .setIsPipeEnableMemoryCheck(false); + receiverEnv + .getConfig() + .getDataNodeConfig() + .setMetricReporterType(Collections.singletonList("PROMETHEUS")); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); @@ -302,40 +316,7 @@ public void testPipeAfterDataRegionLeaderStop() throws Exception { TestUtils.executeNonQueries( senderEnv, Arrays.asList("insert into root.db.d1(time,s1) values (1,1)", "flush"), null); - final AtomicInteger leaderPort = new AtomicInteger(-1); - final TShowRegionResp showRegionResp = client.showRegion(new TShowRegionReq()); - showRegionResp - .getRegionInfoList() - .forEach( - regionInfo -> { - if (RegionRoleType.Leader.getRoleType().equals(regionInfo.getRoleType())) { - leaderPort.set(regionInfo.getClientRpcPort()); - } - }); - - int leaderIndex = -1; - for (int i = 0; i < 3; ++i) { - if (senderEnv.getDataNodeWrapper(i).getPort() == leaderPort.get()) { - leaderIndex = i; - try { - senderEnv.shutdownDataNode(i); - } catch (final Throwable e) { - e.printStackTrace(); - return; - } - try { - TimeUnit.SECONDS.sleep(1); - } catch (final InterruptedException ignored) { - } - try { - senderEnv.startDataNode(i); - ((AbstractEnv) senderEnv).checkClusterStatusWithoutUnknown(); - } catch (final Throwable e) { - e.printStackTrace(); - return; - } - } - } + final int leaderIndex = restartTreeDataRegionLeader(client, "root.db"); if (leaderIndex == -1) { // ensure the leader is stopped fail(); } @@ -352,6 +333,7 @@ public void testPipeAfterDataRegionLeaderStop() throws Exception { "select count(*) from root.db.d1", "count(root.db.d1.s1),", Collections.singleton("2,")); + waitForTreeDataRegionReplicationComplete(Collections.singletonList("root.db")); } try { @@ -397,6 +379,139 @@ public void testPipeAfterDataRegionLeaderStop() throws Exception { } } + private int restartTreeDataRegionLeader( + final SyncConfigNodeIServiceClient client, final String database) throws TException { + final List leaderRegionInfoList = + showTreeDataRegionLeaders(Collections.singletonList(database), client); + if (leaderRegionInfoList.isEmpty()) { + return -1; + } + + final TRegionInfo targetRegionInfo = + leaderRegionInfoList.stream() + .min(Comparator.comparingInt(regionInfo -> regionInfo.getConsensusGroupId().getId())) + .orElse(null); + if (targetRegionInfo == null) { + return -1; + } + + final int leaderPort = targetRegionInfo.getClientRpcPort(); + for (int i = 0; i < senderEnv.getDataNodeWrapperList().size(); ++i) { + if (senderEnv.getDataNodeWrapper(i).getPort() != leaderPort) { + continue; + } + + try { + senderEnv.shutdownDataNode(i); + } catch (final Throwable e) { + e.printStackTrace(); + return -1; + } + + try { + TimeUnit.SECONDS.sleep(1); + } catch (final InterruptedException ignored) { + Thread.currentThread().interrupt(); + return -1; + } + + try { + senderEnv.startDataNode(i); + ((AbstractEnv) senderEnv).checkClusterStatusWithoutUnknown(); + } catch (final Throwable e) { + e.printStackTrace(); + return -1; + } + return i; + } + return -1; + } + + private void waitForTreeDataRegionReplicationComplete(final List databases) { + await() + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> { + try (final SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + final List leaderRegionInfoList = + showTreeDataRegionLeaders(databases, client); + Assert.assertFalse( + "No tree DataRegion leader found for databases " + databases, + leaderRegionInfoList.isEmpty()); + + for (final TRegionInfo regionInfo : leaderRegionInfoList) { + final DataNodeWrapper leaderNode = + findDataNodeWrapperByPort(regionInfo.getClientRpcPort()); + final String metricsUrl = + "http://" + + leaderNode.getIp() + + ":" + + leaderNode.getMetricPort() + + "/metrics"; + final String metricsContent = senderEnv.getUrlContent(metricsUrl, null); + Assert.assertNotNull( + "Failed to fetch metrics from leader DataNode at " + metricsUrl, + metricsContent); + assertSyncLagIsZero(metricsContent, buildDataRegionTag(regionInfo), metricsUrl); + } + } + }); + } + + private List showTreeDataRegionLeaders( + final List databases, final SyncConfigNodeIServiceClient client) throws TException { + final TShowRegionResp showRegionResp = + client.showRegion( + new TShowRegionReq() + .setConsensusGroupType(TConsensusGroupType.DataRegion) + .setDatabases(databases)); + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), showRegionResp.getStatus().getCode()); + final List result = new ArrayList<>(); + for (final TRegionInfo regionInfo : showRegionResp.getRegionInfoList()) { + if (RegionRoleType.Leader.getRoleType().equals(regionInfo.getRoleType())) { + result.add(regionInfo); + } + } + return result; + } + + private DataNodeWrapper findDataNodeWrapperByPort(final int port) { + for (final DataNodeWrapper dataNodeWrapper : senderEnv.getDataNodeWrapperList()) { + if (dataNodeWrapper.getPort() == port) { + return dataNodeWrapper; + } + } + fail("Failed to find DataNodeWrapper for client rpc port " + port); + return null; + } + + private String buildDataRegionTag(final TRegionInfo regionInfo) { + return "DataRegion[" + regionInfo.getConsensusGroupId().getId() + "]"; + } + + private void assertSyncLagIsZero( + final String metricsContent, final String dataRegionTag, final String metricsUrl) { + for (final String line : metricsContent.split("\\R")) { + if (!line.startsWith("iot_consensus{") + || !line.contains("type=\"syncLag\"") + || !line.contains("region=\"" + dataRegionTag + "\"")) { + continue; + } + final int lastSpaceIndex = line.lastIndexOf(' '); + Assert.assertTrue("Malformed syncLag metric line: " + line, lastSpaceIndex > 0); + Assert.assertEquals( + "Expected syncLag of " + dataRegionTag + " to be 0 at " + metricsUrl + " but got " + line, + 0.0, + Double.parseDouble(line.substring(lastSpaceIndex + 1)), + SYNC_LAG_DELTA); + return; + } + fail("No syncLag metric found for " + dataRegionTag + " at " + metricsUrl); + } + @Test public void testPipeAfterRegisterNewDataNode() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); From c7a8f6e1011f2ded370d4a067cbd1dc9ac941046 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 27 May 2026 15:49:26 +0800 Subject: [PATCH 062/102] Fixed TTL problems (#17735) (#17758) (cherry picked from commit cc108e78ec81b2b7d003305b88218fa3327f581e) --- .../iotdb/confignode/manager/TTLManager.java | 4 + .../iotdb/confignode/persistence/TTLInfo.java | 33 +- .../impl/schema/SetTTLProcedure.java | 241 +++++++++++-- .../procedure/state/schema/SetTTLState.java | 4 +- .../confignode/persistence/TTLInfoTest.java | 68 +++- .../impl/schema/SetTTLProcedureTest.java | 327 ++++++++++++++++++ 6 files changed, 635 insertions(+), 42 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TTLManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TTLManager.java index 4cf14ac4cba75..c4d57c2f85e30 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TTLManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TTLManager.java @@ -125,6 +125,10 @@ public Map getAllTTL() { return ((ShowTTLResp) showTTL(new ShowTTLPlan())).getPathTTLMap(); } + public long getTTL(final String[] pathPattern) { + return ttlInfo.getTTL(pathPattern); + } + public int getTTLCount() { return ttlInfo.getTTLCount(); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/TTLInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/TTLInfo.java index d0c6b6da0134d..566c2b0715a90 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/TTLInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/TTLInfo.java @@ -68,13 +68,15 @@ public TSStatus setTTL(SetTTLPlan plan) { try { // check ttl rule capacity final int tTlRuleCapacity = CommonDescriptor.getInstance().getConfig().getTTlRuleCapacity(); - if (getTTLCount() >= tTlRuleCapacity) { + final int newTTLRuleCount = calculateNewTTLRuleCount(plan); + final int requestedTTLRuleCount = ttlCache.getTtlCount() + newTTLRuleCount; + if (newTTLRuleCount > 0 && requestedTTLRuleCount > tTlRuleCapacity) { TSStatus errorStatus = new TSStatus(TSStatusCode.OVERSIZE_TTL.getStatusCode()); errorStatus.setMessage( String.format( - "The number of TTL rules has reached the limit (%d). Please delete " - + "some existing rules first.", - tTlRuleCapacity)); + "The number of TTL rules has reached the limit " + + "(capacity: %d, requested total: %d). Please delete some existing rules first.", + tTlRuleCapacity, requestedTTLRuleCount)); return errorStatus; } ttlCache.setTTL(plan.getPathPattern(), plan.getTTL()); @@ -90,6 +92,20 @@ public TSStatus setTTL(SetTTLPlan plan) { return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS); } + private int calculateNewTTLRuleCount(SetTTLPlan plan) { + int newTTLRuleCount = isNewTTLRule(plan.getPathPattern()) ? 1 : 0; + if (plan.isDataBase()) { + String[] pathNodes = Arrays.copyOf(plan.getPathPattern(), plan.getPathPattern().length + 1); + pathNodes[pathNodes.length - 1] = IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD; + newTTLRuleCount += isNewTTLRule(pathNodes) ? 1 : 0; + } + return newTTLRuleCount; + } + + private boolean isNewTTLRule(String[] pathNodes) { + return ttlCache.getLastNodeTTL(pathNodes) == TTLCache.NULL_TTL; + } + /** Only used for upgrading from database level ttl to device level ttl. */ public void setTTL(Map databaseTTLMap) throws IllegalPathException { lock.writeLock().lock(); @@ -157,6 +173,15 @@ public int getTTLCount() { } } + public long getTTL(final String[] pathPattern) { + lock.readLock().lock(); + try { + return ttlCache.getLastNodeTTL(pathPattern); + } finally { + lock.readLock().unlock(); + } + } + /** * Get the maximum ttl of the corresponding database level. * diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java index 3e1af7dd1f55b..510690a8d6f91 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java @@ -22,8 +22,10 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; +import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.schema.ttl.TTLCache; import org.apache.iotdb.confignode.client.async.CnToDnAsyncRequestType; import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; @@ -45,14 +47,21 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collections; import java.util.Map; import java.util.Objects; public class SetTTLProcedure extends StateMachineProcedure { private static final Logger LOGGER = LoggerFactory.getLogger(SetTTLProcedure.class); + // Distinguishes no previous TTL from TTLCache.NULL_TTL, the explicit unset marker for rollback. + private static final long TTL_NOT_EXIST = Long.MIN_VALUE; + private static final int ROLLBACK_STATE_BYTES = Byte.BYTES + Long.BYTES * 2; private SetTTLPlan plan; + private long previousTTL = TTL_NOT_EXIST; + private long previousDatabaseWildcardTTL = TTL_NOT_EXIST; + private boolean previousTTLStateCaptured = false; public SetTTLProcedure(final boolean isGeneratedByPipe) { super(isGeneratedByPipe); @@ -69,6 +78,10 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, SetTTLState state) long startTime = System.currentTimeMillis(); try { switch (state) { + case CAPTURE_PREVIOUS_TTL: + capturePreviousTTLState(env); + setNextState(SetTTLState.SET_CONFIGNODE_TTL); + return Flow.HAS_MORE_STATE; case SET_CONFIGNODE_TTL: setConfigNodeTTL(env); return Flow.HAS_MORE_STATE; @@ -83,18 +96,13 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, SetTTLState state) } } - private void setConfigNodeTTL(ConfigNodeProcedureEnv env) { - TSStatus res; - try { - res = - env.getConfigManager() - .getConsensusManager() - .write(isGeneratedByPipe ? new PipeEnrichedPlan(this.plan) : this.plan); - } catch (ConsensusException e) { - LOGGER.warn("Failed in the write API executing the consensus layer due to: ", e); - res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); - res.setMessage(e.getMessage()); + void setConfigNodeTTL(final ConfigNodeProcedureEnv env) { + if (!previousTTLStateCaptured) { + capturePreviousTTLState(env); + setNextState(SetTTLState.SET_CONFIGNODE_TTL); + return; } + final TSStatus res = writeConfigNodePlan(env, plan); if (res.code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { LOGGER.info("Failed to execute plan {} because {}", plan, res.message); setFailure(new ProcedureException(new IoTDBException(res))); @@ -103,34 +111,175 @@ private void setConfigNodeTTL(ConfigNodeProcedureEnv env) { } } - private void updateDataNodeTTL(ConfigNodeProcedureEnv env) { - Map dataNodeLocationMap = + void updateDataNodeTTL(final ConfigNodeProcedureEnv env) { + final Map dataNodeLocationMap = env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); - DataNodeAsyncRequestContext clientHandler = - new DataNodeAsyncRequestContext<>( - CnToDnAsyncRequestType.SET_TTL, - new TSetTTLReq( - Collections.singletonList(String.join(".", plan.getPathPattern())), - plan.getTTL(), - plan.isDataBase()), - dataNodeLocationMap); + final DataNodeAsyncRequestContext clientHandler = + sendTTLRequest( + dataNodeLocationMap, + buildSetTTLReq(plan.getPathPattern(), plan.getTTL(), plan.isDataBase())); + if (hasFailedDataNode(clientHandler)) { + LOGGER.error("Failed to update ttl cache of dataNode."); + setFailure(new ProcedureException(new MetadataException("Update dataNode ttl cache failed"))); + } + } + + private void capturePreviousTTLState(final ConfigNodeProcedureEnv env) { + if (previousTTLStateCaptured) { + return; + } + previousTTL = getTTLOrDefault(env, plan.getPathPattern()); + if (plan.isDataBase()) { + previousDatabaseWildcardTTL = + getTTLOrDefault(env, getDatabaseWildcardPathPattern(plan.getPathPattern())); + } + previousTTLStateCaptured = true; + } + + TSStatus writeConfigNodePlan(final ConfigNodeProcedureEnv env, final SetTTLPlan setTTLPlan) { + try { + return env.getConfigManager() + .getConsensusManager() + .write(isGeneratedByPipe ? new PipeEnrichedPlan(setTTLPlan) : setTTLPlan); + } catch (ConsensusException e) { + LOGGER.warn("Failed in the write API executing the consensus layer due to: ", e); + final TSStatus res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); + res.setMessage(e.getMessage()); + return res; + } + } + + DataNodeAsyncRequestContext sendTTLRequest( + final Map dataNodeLocationMap, final TSetTTLReq req) { + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>(CnToDnAsyncRequestType.SET_TTL, req, dataNodeLocationMap); CnToDnInternalServiceAsyncRequestManager.getInstance().sendAsyncRequestWithRetry(clientHandler); - Map statusMap = clientHandler.getResponseMap(); - for (TSStatus status : statusMap.values()) { - // all dataNodes must clear the related schemaengine cache + return clientHandler; + } + + private TSetTTLReq buildSetTTLReq( + final String[] pathPattern, final long ttl, final boolean isDataBase) { + return new TSetTTLReq( + Collections.singletonList(String.join(".", pathPattern)), ttl, isDataBase); + } + + private boolean hasFailedDataNode( + final DataNodeAsyncRequestContext clientHandler) { + if (!clientHandler.getRequestIndices().isEmpty()) { + return true; + } + for (TSStatus status : clientHandler.getResponseMap().values()) { if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - LOGGER.error("Failed to update ttl cache of dataNode."); - setFailure( - new ProcedureException(new MetadataException("Update dataNode ttl cache failed"))); - return; + return true; } } + return false; } + private long getTTLOrDefault(final ConfigNodeProcedureEnv env, final String[] pathPattern) { + final long ttl = env.getConfigManager().getTTLManager().getTTL(pathPattern); + return ttl == TTLCache.NULL_TTL ? TTL_NOT_EXIST : ttl; + } + + private String[] getDatabaseWildcardPathPattern(final String[] pathPattern) { + final String[] pathNodes = Arrays.copyOf(pathPattern, pathPattern.length + 1); + pathNodes[pathNodes.length - 1] = IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD; + return pathNodes; + } + + private void rollbackConfigNodeTTL(final ConfigNodeProcedureEnv env) throws ProcedureException { + restoreTTLOnConfigNode(env, plan.getPathPattern(), previousTTL); + if (plan.isDataBase()) { + restoreTTLOnConfigNode( + env, getDatabaseWildcardPathPattern(plan.getPathPattern()), previousDatabaseWildcardTTL); + } + } + + private void restoreTTLOnConfigNode( + final ConfigNodeProcedureEnv env, final String[] pathPattern, final long ttl) + throws ProcedureException { + // TTL_NOT_EXIST means the original ttl was absent; NULL_TTL asks the executor to unset it. + final SetTTLPlan rollbackPlan = + new SetTTLPlan(pathPattern, ttl == TTL_NOT_EXIST ? TTLCache.NULL_TTL : ttl); + // Database rollback restores the database path and db.** separately, so avoid auto-expansion. + rollbackPlan.setDataBase(false); + final TSStatus status = writeConfigNodePlan(env, rollbackPlan); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + throw new ProcedureException( + new MetadataException( + "Rollback ConfigNode ttl failed for " + + String.join(".", pathPattern) + + ": " + + status.getMessage())); + } + } + + private void rollbackDataNodeTTL(final ConfigNodeProcedureEnv env) throws ProcedureException { + final Map dataNodeLocationMap = + env.getConfigManager().getNodeManager().getRegisteredDataNodeLocations(); + restoreTTLOnDataNodes(dataNodeLocationMap, plan.getPathPattern(), previousTTL); + if (plan.isDataBase()) { + restoreTTLOnDataNodes( + dataNodeLocationMap, + getDatabaseWildcardPathPattern(plan.getPathPattern()), + previousDatabaseWildcardTTL); + } + } + + private void restoreTTLOnDataNodes( + final Map dataNodeLocationMap, + final String[] pathPattern, + final long ttl) + throws ProcedureException { + if (dataNodeLocationMap.isEmpty()) { + return; + } + final DataNodeAsyncRequestContext clientHandler = + sendTTLRequest( + dataNodeLocationMap, + buildSetTTLReq(pathPattern, ttl == TTL_NOT_EXIST ? TTLCache.NULL_TTL : ttl, false)); + if (hasFailedDataNode(clientHandler)) { + throw new ProcedureException( + new MetadataException( + "Rollback dataNode ttl cache failed for " + String.join(".", pathPattern))); + } + } + + /** + * Best-effort rollback: restore both sides, throw the earliest failure, and suppress later ones. + */ @Override - protected void rollbackState( - ConfigNodeProcedureEnv configNodeProcedureEnv, SetTTLState setTTLState) - throws IOException, InterruptedException, ProcedureException {} + protected void rollbackState(final ConfigNodeProcedureEnv env, final SetTTLState setTTLState) + throws IOException, InterruptedException, ProcedureException { + if (setTTLState != SetTTLState.UPDATE_DATANODE_CACHE || !previousTTLStateCaptured) { + return; + } + ProcedureException rollbackFailure = null; + try { + rollbackConfigNodeTTL(env); + } catch (ProcedureException e) { + LOGGER.error("Failed to rollback ConfigNode ttl state.", e); + rollbackFailure = e; + } + try { + rollbackDataNodeTTL(env); + } catch (ProcedureException e) { + LOGGER.error("Failed to rollback DataNode ttl cache.", e); + if (rollbackFailure == null) { + rollbackFailure = e; + } else { + rollbackFailure.addSuppressed(e); + } + } + if (rollbackFailure != null) { + throw rollbackFailure; + } + } + + @Override + protected boolean isRollbackSupported(final SetTTLState state) { + return state == SetTTLState.UPDATE_DATANODE_CACHE; + } @Override protected SetTTLState getState(int stateId) { @@ -144,7 +293,7 @@ protected int getStateId(SetTTLState setTTLState) { @Override protected SetTTLState getInitialState() { - return SetTTLState.SET_CONFIGNODE_TTL; + return SetTTLState.CAPTURE_PREVIOUS_TTL; } @Override @@ -155,14 +304,25 @@ public void serialize(DataOutputStream stream) throws IOException { : ProcedureType.SET_TTL_PROCEDURE.getTypeCode()); super.serialize(stream); ReadWriteIOUtils.write(plan.serializeToByteBuffer(), stream); + stream.writeBoolean(previousTTLStateCaptured); + stream.writeLong(previousTTL); + stream.writeLong(previousDatabaseWildcardTTL); } @Override public void deserialize(ByteBuffer byteBuffer) { super.deserialize(byteBuffer); try { - ReadWriteIOUtils.readInt(byteBuffer); + final int length = ReadWriteIOUtils.readInt(byteBuffer); + final int position = byteBuffer.position(); this.plan = (SetTTLPlan) ConfigPhysicalPlan.Factory.create(byteBuffer); + // The serialized plan buffer may include padding; skip to the actual payload end. + byteBuffer.position(position + length); + if (byteBuffer.remaining() >= ROLLBACK_STATE_BYTES) { + this.previousTTLStateCaptured = byteBuffer.get() != 0; + this.previousTTL = byteBuffer.getLong(); + this.previousDatabaseWildcardTTL = byteBuffer.getLong(); + } } catch (IOException e) { LOGGER.error("IO error when deserialize setTTL plan.", e); } @@ -176,12 +336,21 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - return this.plan.equals(((SetTTLProcedure) o).plan) - && this.isGeneratedByPipe == (((SetTTLProcedure) o).isGeneratedByPipe); + final SetTTLProcedure that = (SetTTLProcedure) o; + return this.isGeneratedByPipe == that.isGeneratedByPipe + && this.previousTTL == that.previousTTL + && this.previousDatabaseWildcardTTL == that.previousDatabaseWildcardTTL + && this.previousTTLStateCaptured == that.previousTTLStateCaptured + && this.plan.equals(that.plan); } @Override public int hashCode() { - return Objects.hash(plan, isGeneratedByPipe); + return Objects.hash( + plan, + isGeneratedByPipe, + previousTTL, + previousDatabaseWildcardTTL, + previousTTLStateCaptured); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/SetTTLState.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/SetTTLState.java index fbdc026fc709b..4dd3063ea3f62 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/SetTTLState.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/schema/SetTTLState.java @@ -19,6 +19,8 @@ package org.apache.iotdb.confignode.procedure.state.schema; public enum SetTTLState { + // Keep existing state ordinals stable for persisted procedures. SET_CONFIGNODE_TTL, - UPDATE_DATANODE_CACHE + UPDATE_DATANODE_CACHE, + CAPTURE_PREVIOUS_TTL } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/TTLInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/TTLInfoTest.java index d5deb1d2b2c1e..977f1dde53ca1 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/TTLInfoTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/TTLInfoTest.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.schema.ttl.TTLCache; import org.apache.iotdb.confignode.consensus.request.read.ttl.ShowTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; import org.apache.iotdb.confignode.consensus.response.ttl.ShowTTLResp; @@ -50,6 +51,7 @@ public class TTLInfoTest { private final File snapshotDir = new File(BASE_OUTPUT_PATH, "ttlInfo-snapshot"); private final long ttl = 123435565323L; private long[] originTTLArr; + private int originTTlRuleCapacity; @Before public void setup() throws IOException { @@ -57,6 +59,7 @@ public void setup() throws IOException { snapshotDir.mkdirs(); } originTTLArr = CommonDescriptor.getInstance().getConfig().getTierTTLInMs(); + originTTlRuleCapacity = CommonDescriptor.getInstance().getConfig().getTTlRuleCapacity(); long[] ttlArr = new long[2]; ttlArr[0] = 10000000L; ttlArr[1] = ttl; @@ -70,6 +73,7 @@ public void tearDown() throws IOException { FileUtils.deleteDirectory(snapshotDir); } CommonDescriptor.getInstance().getConfig().setTierTTLInMs(originTTLArr); + CommonDescriptor.getInstance().getConfig().setTTlRuleCapacity(originTTlRuleCapacity); } @Test @@ -208,6 +212,17 @@ public void testSetAndUnsetTTL() throws IllegalPathException { Assert.assertEquals(4, ttlInfo.getTTLCount()); } + @Test + public void testGetTTLReturnsExactPathTTL() throws IllegalPathException { + PartialPath path = new PartialPath("root.test.db1.**"); + ttlInfo.setTTL(new SetTTLPlan(Arrays.asList(path.getNodes()), 121322323L)); + + Assert.assertEquals(121322323L, ttlInfo.getTTL(path.getNodes())); + Assert.assertEquals( + TTLCache.NULL_TTL, ttlInfo.getTTL(new PartialPath("root.test.db1").getNodes())); + Assert.assertEquals(Long.MAX_VALUE, ttlInfo.getTTL(new PartialPath("root.**").getNodes())); + } + @Test public void testUnsetNonExistTTL() throws IllegalPathException { assertEquals( @@ -241,10 +256,61 @@ public void testTooManyTTL() { final TSStatus status = ttlInfo.setTTL(setTTLPlan); assertEquals(TSStatusCode.OVERSIZE_TTL.getStatusCode(), status.code); assertEquals( - "The number of TTL rules has reached the limit (1000). Please delete some existing rules first.", + "The number of TTL rules has reached the limit " + + "(capacity: 1000, requested total: 1001). Please delete some existing rules first.", status.message); } + @Test + public void testUpdateExistingTTLWhenCapacityIsReached() { + CommonDescriptor.getInstance().getConfig().setTTlRuleCapacity(2); + + SetTTLPlan setTTLPlan = + new SetTTLPlan(PathNodesGenerator.splitPathToNodes("root.sg1.d1.**"), 1000); + assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), ttlInfo.setTTL(setTTLPlan).code); + assertEquals(2, ttlInfo.getTTLCount()); + + setTTLPlan = new SetTTLPlan(PathNodesGenerator.splitPathToNodes("root.sg1.d1.**"), 2000); + assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), ttlInfo.setTTL(setTTLPlan).code); + assertEquals(2, ttlInfo.getTTLCount()); + assertEquals( + Long.valueOf(2000), + ttlInfo.showTTL(new ShowTTLPlan()).getPathTTLMap().get("root.sg1.d1.**")); + } + + @Test + public void testUpdateExistingTTLWhenCurrentStateIsAlreadyOversize() { + SetTTLPlan setTTLPlan = + new SetTTLPlan(PathNodesGenerator.splitPathToNodes("root.sg1.d1.**"), 1000); + assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), ttlInfo.setTTL(setTTLPlan).code); + assertEquals(2, ttlInfo.getTTLCount()); + + CommonDescriptor.getInstance().getConfig().setTTlRuleCapacity(1); + + setTTLPlan = new SetTTLPlan(PathNodesGenerator.splitPathToNodes("root.sg1.d1.**"), 2000); + assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), ttlInfo.setTTL(setTTLPlan).code); + assertEquals(2, ttlInfo.getTTLCount()); + assertEquals( + Long.valueOf(2000), + ttlInfo.showTTL(new ShowTTLPlan()).getPathTTLMap().get("root.sg1.d1.**")); + } + + @Test + public void testDatabaseTTLShouldReserveTwoSlots() { + CommonDescriptor.getInstance().getConfig().setTTlRuleCapacity(2); + + SetTTLPlan setTTLPlan = new SetTTLPlan(PathNodesGenerator.splitPathToNodes("root.sg1"), 1000); + setTTLPlan.setDataBase(true); + + final TSStatus status = ttlInfo.setTTL(setTTLPlan); + assertEquals(TSStatusCode.OVERSIZE_TTL.getStatusCode(), status.code); + assertEquals(1, ttlInfo.getTTLCount()); + assertEquals(1, ttlInfo.showTTL(new ShowTTLPlan()).getPathTTLMap().size()); + assertEquals( + Long.valueOf(Long.MAX_VALUE), + ttlInfo.showTTL(new ShowTTLPlan()).getPathTTLMap().get("root.**")); + } + @Test public void testSnapshot() throws TException, IOException, IllegalPathException { // set ttl diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedureTest.java index 5042eb1dd0f13..cb09c23659c39 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedureTest.java @@ -19,19 +19,42 @@ package org.apache.iotdb.confignode.procedure.impl.schema; +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.common.rpc.thrift.TSetTTLReq; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.schema.ttl.TTLCache; +import org.apache.iotdb.confignode.client.async.CnToDnAsyncRequestType; +import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.TTLManager; +import org.apache.iotdb.confignode.manager.node.NodeManager; +import org.apache.iotdb.confignode.procedure.Procedure; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; +import org.apache.iotdb.confignode.procedure.state.ProcedureState; +import org.apache.iotdb.confignode.procedure.state.schema.SetTTLState; import org.apache.iotdb.confignode.procedure.store.ProcedureFactory; +import org.apache.iotdb.confignode.procedure.store.ProcedureType; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.tsfile.utils.PublicBAOS; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.junit.Assert; import org.junit.Test; +import org.mockito.Mockito; import java.io.DataOutputStream; import java.io.IOException; +import java.lang.reflect.Field; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; public class SetTTLProcedureTest { @@ -65,4 +88,308 @@ public void serializeDeserializeTest() throws IOException, IllegalPathException buffer.clear(); byteArrayOutputStream.reset(); } + + @Test + public void serializeDeserializeTestWithCapturedRollbackState() throws Exception { + final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); + + final SetTTLPlan setTTLPlan = + new SetTTLPlan(Arrays.asList(new PartialPath("root.db").getNodes()), 2000L); + setTTLPlan.setDataBase(true); + final TestingSetTTLProcedure procedure = new TestingSetTTLProcedure(setTTLPlan); + + final Map ttlMap = new HashMap<>(); + ttlMap.put("root.**", Long.MAX_VALUE); + ttlMap.put("root.db", 500L); + ttlMap.put("root.db.**", 600L); + + procedure.executeFromState(mockProcedureEnv(ttlMap), SetTTLState.CAPTURE_PREVIOUS_TTL); + + procedure.serialize(outputStream); + final ByteBuffer buffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + final SetTTLProcedure deserializedProcedure = + (SetTTLProcedure) ProcedureFactory.getInstance().create(buffer); + assertSerializedProcedure( + deserializedProcedure, "root.db", 2000L, true, true, 500L, 600L, false); + } + + @Test + public void deserializeOldFormatWithoutRollbackStateTest() throws Exception { + final SetTTLPlan setTTLPlan = + new SetTTLPlan(Arrays.asList(new PartialPath("root.db").getNodes()), 2000L); + setTTLPlan.setDataBase(true); + + final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); + writeOldFormatProcedure(outputStream, setTTLPlan); + + final ByteBuffer buffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + final SetTTLProcedure deserializedProcedure = + (SetTTLProcedure) ProcedureFactory.getInstance().create(buffer); + + assertSerializedProcedure( + deserializedProcedure, + "root.db", + 2000L, + true, + false, + Long.MIN_VALUE, + Long.MIN_VALUE, + false); + } + + @Test + public void setConfigNodeTTLShouldNotWriteBeforePreviousStateIsCaptured() throws Exception { + final SetTTLPlan setTTLPlan = + new SetTTLPlan(Arrays.asList(new PartialPath("root.db").getNodes()), 2000L); + setTTLPlan.setDataBase(true); + final TestingSetTTLProcedure procedure = new TestingSetTTLProcedure(setTTLPlan); + + final Map ttlMap = new HashMap<>(); + ttlMap.put("root.**", Long.MAX_VALUE); + ttlMap.put("root.db", 500L); + ttlMap.put("root.db.**", 600L); + + procedure.executeFromState(mockProcedureEnv(ttlMap), SetTTLState.SET_CONFIGNODE_TTL); + + Assert.assertTrue(procedure.getWrittenPlans().isEmpty()); + assertSerializedProcedure(procedure, "root.db", 2000L, true, true, 500L, 600L, false); + + procedure.executeFromState(mockProcedureEnv(ttlMap), SetTTLState.SET_CONFIGNODE_TTL); + + Assert.assertEquals(1, procedure.getWrittenPlans().size()); + assertPlan(procedure.getWrittenPlans().get(0), "root.db", 2000L, true); + } + + @Test + public void rollbackStateShouldUnsetNewTTLWhenPreviousStateDidNotExist() throws Exception { + final SetTTLPlan setTTLPlan = + new SetTTLPlan(Arrays.asList(new PartialPath("root.test.sg1.**").getNodes()), 1000L); + final TestingSetTTLProcedure procedure = new TestingSetTTLProcedure(setTTLPlan); + procedure.failFirstDataNodeUpdateForTest(); + + final ConfigNodeProcedureEnv env = + mockProcedureEnv(Collections.singletonMap("root.**", Long.MAX_VALUE)); + + procedure.executeFromState(env, SetTTLState.CAPTURE_PREVIOUS_TTL); + procedure.executeFromState(env, SetTTLState.SET_CONFIGNODE_TTL); + procedure.executeFromState(env, SetTTLState.UPDATE_DATANODE_CACHE); + Assert.assertTrue(procedure.isFailed()); + + procedure.rollbackState(env, SetTTLState.UPDATE_DATANODE_CACHE); + + Assert.assertEquals(2, procedure.getWrittenPlans().size()); + assertPlan(procedure.getWrittenPlans().get(0), "root.test.sg1.**", 1000L, false); + assertPlan(procedure.getWrittenPlans().get(1), "root.test.sg1.**", -1L, false); + + Assert.assertEquals(2, procedure.getRequests().size()); + assertRequest(procedure.getRequests().get(0), "root.test.sg1.**", 1000L, false); + assertRequest(procedure.getRequests().get(1), "root.test.sg1.**", -1L, false); + } + + @Test + public void rollbackStateShouldRestoreDatabaseWildcardTTLSeparately() throws Exception { + final SetTTLPlan setTTLPlan = + new SetTTLPlan(Arrays.asList(new PartialPath("root.db").getNodes()), 2000L); + setTTLPlan.setDataBase(true); + final TestingSetTTLProcedure procedure = new TestingSetTTLProcedure(setTTLPlan); + procedure.failFirstDataNodeUpdateForTest(); + + final Map ttlMap = new HashMap<>(); + ttlMap.put("root.**", Long.MAX_VALUE); + ttlMap.put("root.db", 500L); + ttlMap.put("root.db.**", 600L); + final ConfigNodeProcedureEnv env = mockProcedureEnv(ttlMap); + + procedure.executeFromState(env, SetTTLState.CAPTURE_PREVIOUS_TTL); + procedure.executeFromState(env, SetTTLState.SET_CONFIGNODE_TTL); + procedure.executeFromState(env, SetTTLState.UPDATE_DATANODE_CACHE); + Assert.assertTrue(procedure.isFailed()); + + procedure.rollbackState(env, SetTTLState.UPDATE_DATANODE_CACHE); + + Assert.assertEquals(3, procedure.getWrittenPlans().size()); + assertPlan(procedure.getWrittenPlans().get(0), "root.db", 2000L, true); + assertPlan(procedure.getWrittenPlans().get(1), "root.db", 500L, false); + assertPlan(procedure.getWrittenPlans().get(2), "root.db.**", 600L, false); + + Assert.assertEquals(3, procedure.getRequests().size()); + assertRequest(procedure.getRequests().get(0), "root.db", 2000L, true); + assertRequest(procedure.getRequests().get(1), "root.db", 500L, false); + assertRequest(procedure.getRequests().get(2), "root.db.**", 600L, false); + } + + private ConfigNodeProcedureEnv mockProcedureEnv(final Map ttlMap) { + final ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + final ConfigManager configManager = Mockito.mock(ConfigManager.class); + final TTLManager ttlManager = Mockito.mock(TTLManager.class); + final NodeManager nodeManager = Mockito.mock(NodeManager.class); + + final TDataNodeLocation dataNodeLocation = new TDataNodeLocation(); + dataNodeLocation.setDataNodeId(1); + + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getTTLManager()).thenReturn(ttlManager); + Mockito.when(ttlManager.getTTL(Mockito.any(String[].class))) + .thenAnswer( + invocation -> { + final String[] pathPattern = invocation.getArgument(0); + return ttlMap.getOrDefault(String.join(".", pathPattern), TTLCache.NULL_TTL); + }); + Mockito.when(configManager.getNodeManager()).thenReturn(nodeManager); + Mockito.when(nodeManager.getRegisteredDataNodeLocations()) + .thenReturn(Collections.singletonMap(1, dataNodeLocation)); + return env; + } + + private void assertPlan( + final SetTTLPlan plan, final String path, final long ttl, final boolean isDataBase) { + Assert.assertEquals(path, String.join(".", plan.getPathPattern())); + Assert.assertEquals(ttl, plan.getTTL()); + Assert.assertEquals(isDataBase, plan.isDataBase()); + } + + private void assertRequest( + final TSetTTLReq req, final String path, final long ttl, final boolean isDataBase) { + Assert.assertEquals(Collections.singletonList(path), req.getPathPattern()); + Assert.assertEquals(ttl, req.getTTL()); + Assert.assertEquals(isDataBase, req.isDataBase); + } + + private void writeOldFormatProcedure(final DataOutputStream stream, final SetTTLPlan plan) + throws IOException { + stream.writeShort(ProcedureType.SET_TTL_PROCEDURE.getTypeCode()); + // Procedure fields. + stream.writeLong(Procedure.NO_PROC_ID); + stream.writeInt(ProcedureState.INITIALIZING.ordinal()); + stream.writeLong(0L); + stream.writeLong(0L); + stream.writeLong(Procedure.NO_PROC_ID); + stream.writeLong(Procedure.NO_TIMEOUT); + stream.writeInt(-1); // no stack indexes + stream.write((byte) 0); // no exception + stream.writeInt(-1); // no result + stream.write((byte) 0); // no lock + // StateMachineProcedure fields. + stream.writeInt(0); // no states + ReadWriteIOUtils.write(plan.serializeToByteBuffer(), stream); + } + + private void assertSerializedProcedure( + final SetTTLProcedure procedure, + final String path, + final long ttl, + final boolean isDataBase, + final boolean previousTTLStateCaptured, + final long previousTTL, + final long previousDatabaseWildcardTTL, + final boolean isGeneratedByPipe) + throws Exception { + final Field planField = findField(SetTTLProcedure.class, "plan"); + planField.setAccessible(true); + assertPlan((SetTTLPlan) planField.get(procedure), path, ttl, isDataBase); + + final Field previousTTLStateCapturedField = + findField(SetTTLProcedure.class, "previousTTLStateCaptured"); + previousTTLStateCapturedField.setAccessible(true); + Assert.assertEquals(previousTTLStateCaptured, previousTTLStateCapturedField.get(procedure)); + + final Field previousTTLField = findField(SetTTLProcedure.class, "previousTTL"); + previousTTLField.setAccessible(true); + Assert.assertEquals(previousTTL, previousTTLField.get(procedure)); + + final Field previousDatabaseWildcardTTLField = + findField(SetTTLProcedure.class, "previousDatabaseWildcardTTL"); + previousDatabaseWildcardTTLField.setAccessible(true); + Assert.assertEquals( + previousDatabaseWildcardTTL, previousDatabaseWildcardTTLField.get(procedure)); + + final Field isGeneratedByPipeField = findField(SetTTLProcedure.class, "isGeneratedByPipe"); + isGeneratedByPipeField.setAccessible(true); + Assert.assertEquals(isGeneratedByPipe, isGeneratedByPipeField.get(procedure)); + } + + private Field findField(final Class clazz, final String fieldName) + throws NoSuchFieldException { + Class current = clazz; + while (current != null) { + try { + return current.getDeclaredField(fieldName); + } catch (NoSuchFieldException e) { + current = current.getSuperclass(); + } + } + throw new NoSuchFieldException(fieldName); + } + + private static class TestingSetTTLProcedure extends SetTTLProcedure { + + private final List requests = new ArrayList<>(); + private final List writtenPlans = new ArrayList<>(); + private boolean failFirstDataNodeUpdate = false; + private int requestCount = 0; + + private TestingSetTTLProcedure(final SetTTLPlan plan) { + super(plan, false); + } + + private void failFirstDataNodeUpdateForTest() { + failFirstDataNodeUpdate = true; + } + + private List getRequests() { + return requests; + } + + private List getWrittenPlans() { + return writtenPlans; + } + + @Override + TSStatus writeConfigNodePlan(final ConfigNodeProcedureEnv env, final SetTTLPlan setTTLPlan) { + writtenPlans.add(copyPlan(setTTLPlan)); + return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } + + @Override + DataNodeAsyncRequestContext sendTTLRequest( + final Map dataNodeLocationMap, final TSetTTLReq req) { + requests.add(copyRequest(req)); + + final DataNodeAsyncRequestContext clientHandler = + new DataNodeAsyncRequestContext<>( + CnToDnAsyncRequestType.SET_TTL, copyRequest(req), dataNodeLocationMap); + final List requestIds = new ArrayList<>(clientHandler.getNodeLocationMap().keySet()); + final boolean shouldFail = failFirstDataNodeUpdate && requestCount++ == 0; + + for (Integer requestId : requestIds) { + clientHandler + .getResponseMap() + .put( + requestId, + new TSStatus( + shouldFail + ? TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode() + : TSStatusCode.SUCCESS_STATUS.getStatusCode())); + if (!shouldFail) { + clientHandler.getNodeLocationMap().remove(requestId); + } + } + return clientHandler; + } + + private SetTTLPlan copyPlan(final SetTTLPlan plan) { + final SetTTLPlan copiedPlan = + new SetTTLPlan(Arrays.asList(plan.getPathPattern()), plan.getTTL()); + copiedPlan.setDataBase(plan.isDataBase()); + return copiedPlan; + } + + private TSetTTLReq copyRequest(final TSetTTLReq req) { + return new TSetTTLReq(new ArrayList<>(req.getPathPattern()), req.getTTL(), req.isDataBase); + } + } } From 17e73ee71265a67abc60293bd50a6033c9c09aed Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 27 May 2026 16:36:00 +0800 Subject: [PATCH 063/102] [To dev/1.3] [Pipe] Optimize memory usage (#17775) --- .../db/pipe/event/common/row/PipeRow.java | 5 +- .../event/common/row/PipeRowCollector.java | 25 +- .../tablet/PipeRawTabletInsertionEvent.java | 1 + .../event/common/tablet/PipeTabletUtils.java | 245 ++++++++++++++++++ .../tablet/TabletInsertionDataContainer.java | 58 ++--- .../TsFileInsertionQueryDataContainer.java | 5 +- ...sFileInsertionQueryDataTabletIterator.java | 45 ++-- .../TsFileInsertionScanDataContainer.java | 105 +++++--- .../resource/memory/PipeMemoryWeightUtil.java | 12 +- .../batch/PipeTabletEventTsFileBatch.java | 5 +- .../request/PipeTransferTabletBatchReq.java | 6 +- .../request/PipeTransferTabletRawReq.java | 19 +- .../protocol/opcua/server/OpcUaNameSpace.java | 5 +- .../pipe/sink/util/PipeTabletEventSorter.java | 8 + .../plan/node/write/InsertTabletNode.java | 21 +- .../statement/crud/InsertTabletStatement.java | 3 +- .../rescon/quotas/DefaultOperationQuota.java | 28 +- .../apache/iotdb/db/utils/BitMapUtils.java | 46 ++++ .../event/PipeTabletInsertionEventTest.java | 36 ++- .../common/tablet/PipeTabletUtilsTest.java | 72 +++++ .../sink/PipeDataNodeThriftRequestTest.java | 40 +++ .../node/write/WritePlanNodeSplitTest.java | 10 + .../quotas/DefaultOperationQuotaTest.java | 64 +++++ 23 files changed, 727 insertions(+), 137 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeTabletUtils.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/BitMapUtils.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeTabletUtilsTest.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/rescon/quotas/DefaultOperationQuotaTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRow.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRow.java index 33bf0a5925c39..a8579b32bb771 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRow.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRow.java @@ -152,7 +152,10 @@ public Type getDataType(final int columnIndex) { @Override public boolean isNull(final int columnIndex) { - return bitMaps[columnIndex].isMarked(rowIndex); + return bitMaps != null + && columnIndex < bitMaps.length + && bitMaps[columnIndex] != null + && bitMaps[columnIndex].isMarked(rowIndex); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java index 42560f23a5c59..c26b05f6756b3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletEventConverter; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.pipe.api.access.Row; import org.apache.iotdb.pipe.api.collector.RowCollector; @@ -66,27 +67,26 @@ public void collectRow(Row row) { Pair rowCountAndMemorySize = PipeMemoryWeightUtil.calculateTabletRowCountAndMemory(pipeRow); tablet = new Tablet(deviceId, measurementSchemaList, rowCountAndMemorySize.getLeft()); - tablet.initBitMaps(); isAligned = pipeRow.isAligned(); } final int rowIndex = tablet.rowSize; - tablet.addTimestamp(rowIndex, row.getTime()); + PipeTabletUtils.putTimestamp(tablet, rowIndex, row.getTime()); for (int i = 0; i < row.size(); i++) { final Object value = row.getObject(i); - if (value instanceof org.apache.iotdb.pipe.api.type.Binary) { - tablet.addValue( - measurementSchemaArray[i].getMeasurementId(), - rowIndex, - PipeBinaryTransformer.transformToBinary((org.apache.iotdb.pipe.api.type.Binary) value)); - } else { - tablet.addValue(measurementSchemaArray[i].getMeasurementId(), rowIndex, value); - } + PipeTabletUtils.putValue( + tablet, + rowIndex, + i, + measurementSchemaArray[i].getType(), + value instanceof org.apache.iotdb.pipe.api.type.Binary + ? PipeBinaryTransformer.transformToBinary( + (org.apache.iotdb.pipe.api.type.Binary) value) + : value); if (row.isNull(i)) { - tablet.bitMaps[i].mark(rowIndex); + PipeTabletUtils.markNullValue(tablet, rowIndex, i); } } - tablet.rowSize++; if (tablet.rowSize == tablet.getMaxRowNumber()) { collectTabletInsertionEvent(); @@ -95,6 +95,7 @@ public void collectRow(Row row) { private void collectTabletInsertionEvent() { if (tablet != null) { + PipeTabletUtils.compactBitMaps(tablet); tabletInsertionEventList.add( new PipeRawTabletInsertionEvent( tablet, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java index d322291934ffe..261e3c7a8b324 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java @@ -323,6 +323,7 @@ public boolean isAligned() { public Tablet convertToTablet() { if (!shouldParseTimeOrPattern()) { + PipeTabletUtils.compactBitMaps(tablet); return tablet; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeTabletUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeTabletUtils.java new file mode 100644 index 0000000000000..0a6b073b5b6ca --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeTabletUtils.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.event.common.tablet; + +import org.apache.iotdb.db.utils.BitMapUtils; + +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.write.UnSupportedDataTypeException; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.MeasurementSchema; + +import java.time.LocalDate; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public final class PipeTabletUtils { + + private PipeTabletUtils() {} + + public static final class TabletStringInternPool { + + private final Map internedStrings = new HashMap<>(); + + public String intern(final String value) { + if (Objects.isNull(value)) { + return null; + } + + final String internedValue = internedStrings.get(value); + if (Objects.nonNull(internedValue)) { + return internedValue; + } + + internedStrings.put(value, value); + return value; + } + + public void intern(final String[] values) { + if (Objects.isNull(values)) { + return; + } + + for (int i = 0; i < values.length; ++i) { + values[i] = intern(values[i]); + } + } + + public void intern(final List values) { + if (Objects.isNull(values)) { + return; + } + + for (int i = 0; i < values.size(); ++i) { + values.set(i, intern(values.get(i))); + } + } + + public Tablet intern(final Tablet tablet) { + if (Objects.isNull(tablet)) { + return null; + } + + tablet.setDeviceId(intern(tablet.deviceId)); + internMeasurementSchemas(tablet.getSchemas()); + return tablet; + } + + public void internMeasurementSchemas(final List schemas) { + if (Objects.isNull(schemas)) { + return; + } + + for (final MeasurementSchema schema : schemas) { + intern(schema); + } + } + + public MeasurementSchema intern(final MeasurementSchema schema) { + if (Objects.isNull(schema)) { + return null; + } + + schema.setMeasurementId(intern(schema.getMeasurementId())); + schema.setProps(intern(schema.getProps())); + return schema; + } + + private Map intern(final Map props) { + if (Objects.isNull(props) || props.isEmpty()) { + return props; + } + + final Map internedProps = new HashMap<>(props.size()); + for (final Map.Entry entry : props.entrySet()) { + internedProps.put(intern(entry.getKey()), intern(entry.getValue())); + } + return internedProps; + } + } + + public static Tablet internTablet( + final Tablet tablet, final TabletStringInternPool tabletStringInternPool) { + return Objects.nonNull(tabletStringInternPool) ? tabletStringInternPool.intern(tablet) : tablet; + } + + public static void compactBitMaps(final Tablet tablet) { + if (Objects.isNull(tablet)) { + return; + } + tablet.bitMaps = compactBitMaps(tablet.bitMaps, tablet.rowSize); + } + + public static BitMap[] compactBitMaps(final BitMap[] bitMaps, final int rowCount) { + return BitMapUtils.compactBitMaps(bitMaps, rowCount); + } + + public static BitMap[] copyBitMapsOrCreateEmpty(final Tablet tablet) { + final BitMap[] bitMaps = tablet.bitMaps; + return Objects.nonNull(bitMaps) + ? Arrays.copyOf(bitMaps, bitMaps.length) + : new BitMap[getColumnCount(tablet)]; + } + + public static void markNullValue(final Tablet tablet, final int rowIndex, final int columnIndex) { + final BitMap[] bitMaps = ensureBitMaps(tablet, columnIndex + 1); + if (Objects.isNull(bitMaps[columnIndex])) { + bitMaps[columnIndex] = new BitMap(tablet.getMaxRowNumber()); + } + bitMaps[columnIndex].mark(rowIndex); + } + + public static void putTimestamp(final Tablet tablet, final int rowIndex, final long timestamp) { + tablet.timestamps[rowIndex] = timestamp; + tablet.rowSize = Math.max(tablet.rowSize, rowIndex + 1); + } + + public static void putValue( + final Tablet tablet, + final int rowIndex, + final int columnIndex, + final TSDataType dataType, + final Object value) { + switch (dataType) { + case BOOLEAN: + ((boolean[]) tablet.values[columnIndex])[rowIndex] = (Boolean) value; + break; + case INT32: + ((int[]) tablet.values[columnIndex])[rowIndex] = (Integer) value; + break; + case DATE: + ((LocalDate[]) tablet.values[columnIndex])[rowIndex] = (LocalDate) value; + break; + case INT64: + case TIMESTAMP: + ((long[]) tablet.values[columnIndex])[rowIndex] = (Long) value; + break; + case FLOAT: + ((float[]) tablet.values[columnIndex])[rowIndex] = (Float) value; + break; + case DOUBLE: + ((double[]) tablet.values[columnIndex])[rowIndex] = (Double) value; + break; + case TEXT: + case BLOB: + case STRING: + ((Binary[]) tablet.values[columnIndex])[rowIndex] = toBinary(value); + break; + default: + throw new UnSupportedDataTypeException("Unsupported data type: " + dataType); + } + unmarkNullValue(tablet, rowIndex, columnIndex); + } + + private static void unmarkNullValue( + final Tablet tablet, final int rowIndex, final int columnIndex) { + final BitMap[] bitMaps = tablet.bitMaps; + if (Objects.nonNull(bitMaps) + && columnIndex < bitMaps.length + && Objects.nonNull(bitMaps[columnIndex])) { + bitMaps[columnIndex].unmark(rowIndex); + } + } + + private static BitMap[] ensureBitMaps(final Tablet tablet, final int minColumnCount) { + final int columnCount = Math.max(getColumnCount(tablet), minColumnCount); + BitMap[] bitMaps = tablet.bitMaps; + if (Objects.isNull(bitMaps)) { + bitMaps = new BitMap[columnCount]; + tablet.bitMaps = bitMaps; + } else if (bitMaps.length < columnCount) { + final BitMap[] expandedBitMaps = new BitMap[columnCount]; + System.arraycopy(bitMaps, 0, expandedBitMaps, 0, bitMaps.length); + bitMaps = expandedBitMaps; + tablet.bitMaps = bitMaps; + } + return bitMaps; + } + + private static int getColumnCount(final Tablet tablet) { + if (Objects.nonNull(tablet.getSchemas())) { + return tablet.getSchemas().size(); + } + return Objects.nonNull(tablet.values) ? tablet.values.length : 0; + } + + private static Binary toBinary(final Object value) { + if (Objects.isNull(value)) { + return Binary.EMPTY_VALUE; + } + if (value instanceof Binary) { + return (Binary) value; + } + if (value instanceof byte[]) { + return new Binary((byte[]) value); + } + if (value instanceof String) { + return new Binary(((String) value).getBytes(TSFileConfig.STRING_CHARSET)); + } + throw new IllegalArgumentException( + String.format("Expected Binary, byte[] or String, but was %s.", value.getClass())); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java index 133dbb5bff8a0..d8c2bccaa97c9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java @@ -198,6 +198,7 @@ private void parse(final InsertRowNode insertRowNode, final PipePattern pattern) } this.rowCount = this.timestampColumn.length; + this.nullValueColumnBitmaps = PipeTabletUtils.compactBitMaps(nullValueColumnBitmaps, rowCount); if (this.rowCount == 0 && LOGGER.isDebugEnabled()) { LOGGER.debug( "InsertRowNode({}) is parsed to zero rows according to the pattern({}) and time range [{}, {}], the corresponding source event({}) will be ignored.", @@ -217,7 +218,6 @@ private void parse(final InsertTabletNode insertTabletNode, final PipePattern pa this.isAligned = insertTabletNode.isAligned(); final long[] originTimestampColumn = insertTabletNode.getTimes(); - final int originRowSize = originTimestampColumn.length; final List rowIndexList = generateRowIndexList(originTimestampColumn); this.timestampColumn = rowIndexList.stream().mapToLong(i -> originTimestampColumn[i]).toArray(); @@ -243,18 +243,7 @@ private void parse(final InsertTabletNode insertTabletNode, final PipePattern pa final String[] originColumnNameStringList = insertTabletNode.getMeasurements(); final TSDataType[] originValueColumnTypes = insertTabletNode.getDataTypes(); final Object[] originValueColumns = insertTabletNode.getColumns(); - final BitMap[] originBitMapList = - (insertTabletNode.getBitMaps() == null - ? IntStream.range(0, originColumnSize) - .boxed() - .map(o -> new BitMap(originRowSize)) - .toArray(BitMap[]::new) - : insertTabletNode.getBitMaps()); - for (int i = 0; i < originBitMapList.length; i++) { - if (originBitMapList[i] == null) { - originBitMapList[i] = new BitMap(originRowSize); - } - } + final BitMap[] originBitMapList = insertTabletNode.getBitMaps(); for (int i = 0; i < originColumnIndex2FilteredColumnIndexMapperList.length; i++) { if (originColumnIndex2FilteredColumnIndexMapperList[i] != null) { @@ -277,7 +266,7 @@ private void parse(final InsertTabletNode insertTabletNode, final PipePattern pa originValueColumns[i], rowIndexList, false, - originBitMapList[i], + getBitMap(originBitMapList, i), bitMap); } this.nullValueColumnBitmaps[filteredColumnIndex] = bitMap; @@ -285,6 +274,7 @@ private void parse(final InsertTabletNode insertTabletNode, final PipePattern pa } this.rowCount = this.timestampColumn.length; + this.nullValueColumnBitmaps = PipeTabletUtils.compactBitMaps(nullValueColumnBitmaps, rowCount); if (rowCount == 0 && LOGGER.isDebugEnabled()) { LOGGER.debug( "InsertTabletNode({}) is parsed to zero rows according to the pattern({}) and time range [{}, {}], the corresponding source event({}) will be ignored.", @@ -338,18 +328,7 @@ private void parse(final Tablet tablet, final boolean isAligned, final PipePatte } final Object[] originValueColumns = tablet.values; // we do not reduce value columns here by origin row size - final BitMap[] originBitMapList = - tablet.bitMaps == null - ? IntStream.range(0, originColumnSize) - .boxed() - .map(o -> new BitMap(tablet.getMaxRowNumber())) - .toArray(BitMap[]::new) - : tablet.bitMaps; // We do not reduce bitmaps here by origin row size - for (int i = 0; i < originBitMapList.length; i++) { - if (originBitMapList[i] == null) { - originBitMapList[i] = new BitMap(tablet.getMaxRowNumber()); - } - } + final BitMap[] originBitMapList = tablet.bitMaps; for (int i = 0; i < originColumnIndex2FilteredColumnIndexMapperList.length; i++) { if (originColumnIndex2FilteredColumnIndexMapperList[i] != null) { @@ -372,7 +351,7 @@ private void parse(final Tablet tablet, final boolean isAligned, final PipePatte originValueColumns[i], rowIndexList, false, - originBitMapList[i], + getBitMap(originBitMapList, i), bitMap); } this.nullValueColumnBitmaps[filteredColumnIndex] = bitMap; @@ -380,6 +359,7 @@ private void parse(final Tablet tablet, final boolean isAligned, final PipePatte } this.rowCount = this.timestampColumn.length; + this.nullValueColumnBitmaps = PipeTabletUtils.compactBitMaps(nullValueColumnBitmaps, rowCount); if (this.rowCount == 0 && LOGGER.isDebugEnabled()) { LOGGER.debug( "Tablet({}) is parsed to zero rows according to the pattern({}) and time range [{}, {}], the corresponding source event({}) will be ignored.", @@ -471,7 +451,7 @@ private static Object filterValueColumnsByRowIndexList( : (int[]) originValueColumn; final int[] valueColumns = new int[rowIndexList.size()]; for (int i = 0; i < rowIndexList.size(); ++i) { - if (originNullValueColumnBitmap.isMarked(rowIndexList.get(i))) { + if (isNullValue(originNullValueColumnBitmap, rowIndexList.get(i))) { valueColumns[i] = 0; nullValueColumnBitmap.mark(i); } else { @@ -493,7 +473,7 @@ private static Object filterValueColumnsByRowIndexList( : (LocalDate[]) originValueColumn; for (int i = 0; i < rowIndexList.size(); ++i) { - if (originNullValueColumnBitmap.isMarked(rowIndexList.get(i))) { + if (isNullValue(originNullValueColumnBitmap, rowIndexList.get(i))) { valueColumns[i] = EMPTY_LOCALDATE; nullValueColumnBitmap.mark(i); } else { @@ -507,7 +487,7 @@ private static Object filterValueColumnsByRowIndexList( ? new int[] {(int) originValueColumn} : (int[]) originValueColumn; for (int i = 0; i < rowIndexList.size(); ++i) { - if (originNullValueColumnBitmap.isMarked(rowIndexList.get(i))) { + if (isNullValue(originNullValueColumnBitmap, rowIndexList.get(i))) { valueColumns[i] = EMPTY_LOCALDATE; nullValueColumnBitmap.mark(i); } else { @@ -527,7 +507,7 @@ private static Object filterValueColumnsByRowIndexList( : (long[]) originValueColumn; final long[] valueColumns = new long[rowIndexList.size()]; for (int i = 0; i < rowIndexList.size(); ++i) { - if (originNullValueColumnBitmap.isMarked(rowIndexList.get(i))) { + if (isNullValue(originNullValueColumnBitmap, rowIndexList.get(i))) { valueColumns[i] = 0L; nullValueColumnBitmap.mark(i); } else { @@ -544,7 +524,7 @@ private static Object filterValueColumnsByRowIndexList( : (float[]) originValueColumn; final float[] valueColumns = new float[rowIndexList.size()]; for (int i = 0; i < rowIndexList.size(); ++i) { - if (originNullValueColumnBitmap.isMarked(rowIndexList.get(i))) { + if (isNullValue(originNullValueColumnBitmap, rowIndexList.get(i))) { valueColumns[i] = 0F; nullValueColumnBitmap.mark(i); } else { @@ -561,7 +541,7 @@ private static Object filterValueColumnsByRowIndexList( : (double[]) originValueColumn; final double[] valueColumns = new double[rowIndexList.size()]; for (int i = 0; i < rowIndexList.size(); ++i) { - if (originNullValueColumnBitmap.isMarked(rowIndexList.get(i))) { + if (isNullValue(originNullValueColumnBitmap, rowIndexList.get(i))) { valueColumns[i] = 0D; nullValueColumnBitmap.mark(i); } else { @@ -578,7 +558,7 @@ private static Object filterValueColumnsByRowIndexList( : (boolean[]) originValueColumn; final boolean[] valueColumns = new boolean[rowIndexList.size()]; for (int i = 0; i < rowIndexList.size(); ++i) { - if (originNullValueColumnBitmap.isMarked(rowIndexList.get(i))) { + if (isNullValue(originNullValueColumnBitmap, rowIndexList.get(i))) { valueColumns[i] = false; nullValueColumnBitmap.mark(i); } else { @@ -599,7 +579,7 @@ private static Object filterValueColumnsByRowIndexList( for (int i = 0; i < rowIndexList.size(); ++i) { if (Objects.isNull(binaryValueColumns[rowIndexList.get(i)]) || Objects.isNull(binaryValueColumns[rowIndexList.get(i)].getValues()) - || originNullValueColumnBitmap.isMarked(rowIndexList.get(i))) { + || isNullValue(originNullValueColumnBitmap, rowIndexList.get(i))) { valueColumns[i] = Binary.EMPTY_VALUE; nullValueColumnBitmap.mark(i); } else { @@ -659,6 +639,14 @@ private void fillNullValue( } } + private static BitMap getBitMap(final BitMap[] bitMaps, final int index) { + return Objects.nonNull(bitMaps) && index < bitMaps.length ? bitMaps[index] : null; + } + + private static boolean isNullValue(final BitMap bitMap, final int rowIndex) { + return Objects.nonNull(bitMap) && bitMap.isMarked(rowIndex); + } + //////////////////////////// process //////////////////////////// public List processRowByRow(final BiConsumer consumer) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java index 4353e4984a259..e1fa58f5a0f68 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataContainer.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils.TabletStringInternPool; import org.apache.iotdb.db.pipe.event.common.tsfile.container.TsFileInsertionDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.parser.util.ModsOperationUtil; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; @@ -70,6 +71,7 @@ public class TsFileInsertionQueryDataContainer extends TsFileInsertionDataContai private final Iterator>> deviceMeasurementsMapIterator; private final Map deviceIsAlignedMap; private final Map measurementDataTypeMap; + private final TabletStringInternPool tabletStringInternPool = new TabletStringInternPool(); @TestOnly public TsFileInsertionQueryDataContainer( @@ -385,7 +387,8 @@ public boolean hasNext() { entry.getValue(), timeFilterExpression, allocatedMemoryBlockForTablet, - currentModifications); + currentModifications, + tabletStringInternPool); } catch (final Exception e) { close(); throw new PipeException( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataTabletIterator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataTabletIterator.java index e16c7113da354..2e81f4aa335a0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataTabletIterator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/query/TsFileInsertionQueryDataTabletIterator.java @@ -20,6 +20,8 @@ package org.apache.iotdb.db.pipe.event.common.tsfile.container.query; import org.apache.iotdb.commons.path.PatternTreeMap; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils.TabletStringInternPool; import org.apache.iotdb.db.pipe.event.common.tsfile.parser.util.ModsOperationUtil; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; @@ -37,6 +39,7 @@ import org.apache.tsfile.read.expression.IExpression; import org.apache.tsfile.read.expression.QueryExpression; import org.apache.tsfile.read.query.dataset.QueryDataSet; +import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.MeasurementSchema; @@ -57,6 +60,7 @@ public class TsFileInsertionQueryDataTabletIterator implements Iterator private final String deviceId; private final List measurements; + private final List schemas; private final IExpression timeFilterExpression; @@ -76,20 +80,28 @@ public class TsFileInsertionQueryDataTabletIterator implements Iterator final List measurements, final IExpression timeFilterExpression, final PipeMemoryBlock allocatedBlockForTablet, - final PatternTreeMap currentModifications) + final PatternTreeMap currentModifications, + final TabletStringInternPool tabletStringInternPool) throws IOException { this.tsFileReader = tsFileReader; this.measurementDataTypeMap = measurementDataTypeMap; - this.deviceId = deviceId; + this.deviceId = tabletStringInternPool.intern(deviceId); this.measurements = measurements.stream() .filter( measurement -> // time column in aligned time-series should not be a query column measurement != null && !measurement.isEmpty()) + .map(tabletStringInternPool::intern) .sorted() .collect(Collectors.toList()); + this.schemas = new ArrayList<>(); + for (final String measurement : this.measurements) { + final TSDataType dataType = + measurementDataTypeMap.get(this.deviceId + TsFileConstant.PATH_SEPARATOR + measurement); + schemas.add(new MeasurementSchema(measurement, dataType)); + } this.timeFilterExpression = timeFilterExpression; @@ -99,7 +111,7 @@ public class TsFileInsertionQueryDataTabletIterator implements Iterator this.measurementModsList = ModsOperationUtil.initializeMeasurementMods( - deviceId, this.measurements, currentModifications); + this.deviceId, this.measurements, currentModifications); } private QueryDataSet buildQueryDataSet() throws IOException { @@ -133,18 +145,9 @@ public Tablet next() { } private Tablet buildNextTablet() throws IOException { - final List schemas = new ArrayList<>(); - for (final String measurement : measurements) { - final TSDataType dataType = - measurementDataTypeMap.get(deviceId + TsFileConstant.PATH_SEPARATOR + measurement); - schemas.add(new MeasurementSchema(measurement, dataType)); - } - Tablet tablet = null; if (!queryDataSet.hasNext()) { - tablet = new Tablet(deviceId, schemas, 1); - tablet.initBitMaps(); - return tablet; + return new Tablet(deviceId, schemas, 1); } boolean isFirstRow = true; @@ -156,7 +159,6 @@ private Tablet buildNextTablet() throws IOException { Pair rowCountAndMemorySize = PipeMemoryWeightUtil.calculateTabletRowCountAndMemory(rowRecord); tablet = new Tablet(deviceId, schemas, rowCountAndMemorySize.getLeft()); - tablet.initBitMaps(); if (allocatedBlockForTablet.getMemoryUsageInBytes() < rowCountAndMemorySize.getRight()) { PipeDataNodeResourceManager.memory() .forceResize(allocatedBlockForTablet, rowCountAndMemorySize.getRight()); @@ -172,27 +174,30 @@ private Tablet buildNextTablet() throws IOException { final int fieldSize = fields.size(); for (int i = 0; i < fieldSize; i++) { final Field field = fields.get(i); - final String measurement = measurements.get(i); + final TSDataType dataType = schemas.get(i).getType(); // Check if this value is deleted by mods if (field == null || ModsOperationUtil.isDelete(rowRecord.getTimestamp(), measurementModsList.get(i))) { - tablet.bitMaps[i].mark(rowIndex); + if (dataType != null && dataType.isBinary()) { + PipeTabletUtils.putValue(tablet, rowIndex, i, dataType, Binary.EMPTY_VALUE); + } + PipeTabletUtils.markNullValue(tablet, rowIndex, i); } else { - tablet.addValue(measurement, rowIndex, field.getObjectValue(schemas.get(i).getType())); + PipeTabletUtils.putValue( + tablet, rowIndex, i, dataType, field.getObjectValue(schemas.get(i).getType())); isNeedFillTime = true; } } if (isNeedFillTime) { - tablet.addTimestamp(rowIndex, rowRecord.getTimestamp()); + PipeTabletUtils.putTimestamp(tablet, rowIndex, rowRecord.getTimestamp()); } - tablet.rowSize++; - if (tablet.rowSize == tablet.getMaxRowNumber()) { break; } } + PipeTabletUtils.compactBitMaps(tablet); return tablet; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java index 9366d0f62dfe2..e903c7340e4f5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java @@ -26,6 +26,8 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils.TabletStringInternPool; import org.apache.iotdb.db.pipe.event.common.tsfile.container.TsFileInsertionDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.parser.util.ModsOperationUtil; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; @@ -93,6 +95,7 @@ public class TsFileInsertionScanDataContainer extends TsFileInsertionDataContain private String currentDevice; private boolean currentIsAligned; private final List currentMeasurements = new ArrayList<>(); + private final TabletStringInternPool tabletStringInternPool = new TabletStringInternPool(); private final List modsInfos = new ArrayList<>(); // Cached time chunk @@ -272,7 +275,6 @@ private Tablet getNextTablet() { if (!data.hasCurrent()) { tablet = new Tablet(currentDevice, currentMeasurements, 1); - tablet.initBitMaps(); // Ignore the memory cost of tablet PipeDataNodeResourceManager.memory().forceResize(allocatedMemoryBlockForTablet, 0); return tablet; @@ -288,7 +290,6 @@ private Tablet getNextTablet() { PipeMemoryWeightUtil.calculateTabletRowCountAndMemory(data); tablet = new Tablet(currentDevice, currentMeasurements, rowCountAndMemorySize.getLeft()); - tablet.initBitMaps(); if (allocatedMemoryBlockForTablet.getMemoryUsageInBytes() < rowCountAndMemorySize.getRight()) { PipeDataNodeResourceManager.memory() @@ -300,10 +301,8 @@ private Tablet getNextTablet() { final int rowIndex = tablet.rowSize; if (putValueToColumns(data, tablet, rowIndex)) { - tablet.addTimestamp(rowIndex, data.currentTime()); + PipeTabletUtils.putTimestamp(tablet, rowIndex, data.currentTime()); } - - tablet.rowSize++; } data.next(); @@ -318,13 +317,13 @@ private Tablet getNextTablet() { if (tablet == null) { tablet = new Tablet(currentDevice, currentMeasurements, 1); - tablet.initBitMaps(); } // Switch chunk reader iff current chunk is all consumed if (!data.hasCurrent()) { prepareData(); } + PipeTabletUtils.compactBitMaps(tablet); return tablet; } catch (final Exception e) { close(); @@ -372,81 +371,100 @@ private void resizePageDataMemoryIfNeeded(final long estimatedMemoryUsageInBytes } private boolean putValueToColumns(final BatchData data, final Tablet tablet, final int rowIndex) { - final Object[] columns = tablet.values; boolean isNeedFillTime = false; if (data.getDataType() == TSDataType.VECTOR) { - for (int i = 0; i < columns.length; ++i) { + for (int i = 0; i < tablet.getSchemas().size(); ++i) { final TsPrimitiveType primitiveType = data.getVector()[i]; + final TSDataType type = tablet.getSchemas().get(i).getType(); if (Objects.isNull(primitiveType) || ModsOperationUtil.isDelete(data.currentTime(), modsInfos.get(i))) { - tablet.bitMaps[i].mark(rowIndex); - final TSDataType type = tablet.getSchemas().get(i).getType(); if (type == TSDataType.TEXT || type == TSDataType.BLOB || type == TSDataType.STRING) { - ((Binary[]) columns[i])[rowIndex] = Binary.EMPTY_VALUE; - } - if (type == TSDataType.DATE) { - ((LocalDate[]) columns[i])[rowIndex] = EMPTY_DATE; + PipeTabletUtils.putValue(tablet, rowIndex, i, type, Binary.EMPTY_VALUE); } + PipeTabletUtils.markNullValue(tablet, rowIndex, i); continue; } isNeedFillTime = true; - switch (tablet.getSchemas().get(i).getType()) { + switch (type) { case BOOLEAN: - ((boolean[]) columns[i])[rowIndex] = primitiveType.getBoolean(); + PipeTabletUtils.putValue(tablet, rowIndex, i, type, primitiveType.getBoolean()); break; case INT32: - ((int[]) columns[i])[rowIndex] = primitiveType.getInt(); + PipeTabletUtils.putValue(tablet, rowIndex, i, type, primitiveType.getInt()); break; case DATE: - ((LocalDate[]) columns[i])[rowIndex] = - DateUtils.parseIntToLocalDate(primitiveType.getInt()); + PipeTabletUtils.putValue( + tablet, rowIndex, i, type, DateUtils.parseIntToLocalDate(primitiveType.getInt())); break; case INT64: case TIMESTAMP: - ((long[]) columns[i])[rowIndex] = primitiveType.getLong(); + PipeTabletUtils.putValue(tablet, rowIndex, i, type, primitiveType.getLong()); break; case FLOAT: - ((float[]) columns[i])[rowIndex] = primitiveType.getFloat(); + PipeTabletUtils.putValue(tablet, rowIndex, i, type, primitiveType.getFloat()); break; case DOUBLE: - ((double[]) columns[i])[rowIndex] = primitiveType.getDouble(); + PipeTabletUtils.putValue(tablet, rowIndex, i, type, primitiveType.getDouble()); break; case TEXT: case BLOB: case STRING: - ((Binary[]) columns[i])[rowIndex] = primitiveType.getBinary(); + final Binary binary = primitiveType.getBinary(); + PipeTabletUtils.putValue( + tablet, + rowIndex, + i, + type, + Objects.isNull(binary) || Objects.isNull(binary.getValues()) + ? Binary.EMPTY_VALUE + : binary); break; default: throw new UnSupportedDataTypeException("UnSupported" + primitiveType.getDataType()); } } } else { + if (!modsInfos.isEmpty() + && ModsOperationUtil.isDelete(data.currentTime(), modsInfos.get(0))) { + return false; + } + isNeedFillTime = true; - switch (tablet.getSchemas().get(0).getType()) { + final TSDataType type = tablet.getSchemas().get(0).getType(); + switch (type) { case BOOLEAN: - ((boolean[]) columns[0])[rowIndex] = data.getBoolean(); + PipeTabletUtils.putValue(tablet, rowIndex, 0, type, data.getBoolean()); break; case INT32: - ((int[]) columns[0])[rowIndex] = data.getInt(); + PipeTabletUtils.putValue(tablet, rowIndex, 0, type, data.getInt()); break; case DATE: - ((LocalDate[]) columns[0])[rowIndex] = DateUtils.parseIntToLocalDate(data.getInt()); + PipeTabletUtils.putValue( + tablet, rowIndex, 0, type, DateUtils.parseIntToLocalDate(data.getInt())); break; case INT64: case TIMESTAMP: - ((long[]) columns[0])[rowIndex] = data.getLong(); + PipeTabletUtils.putValue(tablet, rowIndex, 0, type, data.getLong()); break; case FLOAT: - ((float[]) columns[0])[rowIndex] = data.getFloat(); + PipeTabletUtils.putValue(tablet, rowIndex, 0, type, data.getFloat()); break; case DOUBLE: - ((double[]) columns[0])[rowIndex] = data.getDouble(); + PipeTabletUtils.putValue(tablet, rowIndex, 0, type, data.getDouble()); break; case TEXT: case BLOB: case STRING: - ((Binary[]) columns[0])[rowIndex] = data.getBinary(); + final Binary binary = data.getBinary(); + PipeTabletUtils.putValue( + tablet, + rowIndex, + 0, + type, + Objects.isNull(binary) || Objects.isNull(binary.getValues()) + ? Binary.EMPTY_VALUE + : binary); break; default: throw new UnSupportedDataTypeException("UnSupported" + data.getDataType()); @@ -560,13 +578,13 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { ? new ChunkReader(chunk, filter) : new SinglePageWholeChunkReader(chunk); currentIsAligned = false; + final String measurementID = + tabletStringInternPool.intern(chunkHeader.getMeasurementID()); currentMeasurements.add( - new MeasurementSchema(chunkHeader.getMeasurementID(), chunkHeader.getDataType())); + new MeasurementSchema(measurementID, chunkHeader.getDataType())); modsInfos.addAll( ModsOperationUtil.initializeMeasurementMods( - currentDevice, - Collections.singletonList(chunkHeader.getMeasurementID()), - currentModifications)); + currentDevice, Collections.singletonList(measurementID), currentModifications)); return; } case MetaMarker.VALUE_CHUNK_HEADER: @@ -615,9 +633,11 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { } // Increase value index + final String measurementID = + tabletStringInternPool.intern(chunkHeader.getMeasurementID()); final int valueIndex = measurementIndexMap.compute( - chunkHeader.getMeasurementID(), + measurementID, (measurement, index) -> Objects.nonNull(index) ? index + 1 : 0); // Emit when encountered non-sequential value chunk, or the chunk size exceeds @@ -677,13 +697,13 @@ > getPageDataMemoryLimitInBytes()) { valueChunkSize += chunkHeader.getDataSize(); valueChunkPageMemorySize += currentValueChunkPageMemorySize; valueChunkList.add(chunk); + final String measurementID = + tabletStringInternPool.intern(chunkHeader.getMeasurementID()); currentMeasurements.add( - new MeasurementSchema(chunkHeader.getMeasurementID(), chunkHeader.getDataType())); + new MeasurementSchema(measurementID, chunkHeader.getDataType())); modsInfos.addAll( ModsOperationUtil.initializeMeasurementMods( - currentDevice, - Collections.singletonList(chunkHeader.getMeasurementID()), - currentModifications)); + currentDevice, Collections.singletonList(measurementID), currentModifications)); break; } case MetaMarker.CHUNK_GROUP_HEADER: @@ -702,7 +722,10 @@ > getPageDataMemoryLimitInBytes()) { timeChunkPageMemorySizeList.clear(); measurementIndexMap.clear(); - currentDevice = pattern.mayOverlapWithDevice(deviceID) ? deviceID : null; + currentDevice = + pattern.mayOverlapWithDevice(deviceID) + ? tabletStringInternPool.intern(deviceID) + : null; break; } case MetaMarker.OPERATION_INDEX_RANGE: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java index a707f554c5184..a22522666c2ff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java @@ -179,14 +179,14 @@ private static Pair calculateTabletRowCountAndMemoryBySize( return new Pair<>(1, 0); } - // Calculate row number according to the max size of a pipe tablet. - // "-100" is the estimated size of other data structures in a pipe tablet. + // Calculate row number according to the max size of a pipe tablet. "100" is the estimated size + // of other data structures in a pipe tablet. // "*8" converts bytes to bits, because the bitmap size is 1 bit per schema. - // Here we estimate the max use of int sizeLimit = - Math.min( - IoTDBDescriptor.getInstance().getConfig().getPipeDataStructureTabletSizeInBytes(), - (int) (inputNum * rowBytesUsed * 1.2)); + (int) + Math.min( + IoTDBDescriptor.getInstance().getConfig().getPipeDataStructureTabletSizeInBytes(), + Math.min(Integer.MAX_VALUE, 100 + inputNum * (double) rowBytesUsed * 1.2)); int rowNumber = 8 * (sizeLimit - 100) / (8 * rowBytesUsed + schemaCount); rowNumber = Math.max(1, rowNumber); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index 36ca10daa725a..2e69b23927743 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.db.pipe.sink.util.PipeTabletEventSorter; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; @@ -361,7 +362,7 @@ private Tablet tryBestToAggregateTablets( // Aggregate the current tablet's data aggregatedSchemas.addAll(tablet.getSchemas()); aggregatedValues.addAll(Arrays.asList(tablet.values)); - aggregatedBitMaps.addAll(Arrays.asList(tablet.bitMaps)); + aggregatedBitMaps.addAll(Arrays.asList(PipeTabletUtils.copyBitMapsOrCreateEmpty(tablet))); // Remove the aggregated tablet tablets.pollFirst(); } else { @@ -563,7 +564,7 @@ private void writeTabletsIntoOneFile( .map(schema -> (MeasurementSchema) schema) .toArray(MeasurementSchema[]::new); Object[] values = Arrays.copyOf(tablet.values, tablet.values.length); - BitMap[] bitMaps = Arrays.copyOf(tablet.bitMaps, tablet.bitMaps.length); + BitMap[] bitMaps = PipeTabletUtils.copyBitMapsOrCreateEmpty(tablet); // convert date value to int refer to // org.apache.iotdb.db.storageengine.dataregion.memtable.WritableMemChunk.writeNonAlignedTablet diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTabletBatchReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTabletBatchReq.java index 94a838ee0ad69..266894060dcb8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTabletBatchReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTabletBatchReq.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.IoTDBSinkRequestVersion; import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeRequestType; import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils.TabletStringInternPool; import org.apache.iotdb.db.queryengine.plan.planner.plan.PlanFragment; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertBaseStatement; @@ -34,7 +35,6 @@ import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.PublicBAOS; import org.apache.tsfile.utils.ReadWriteIOUtils; -import org.apache.tsfile.write.record.Tablet; import java.io.DataOutputStream; import java.io.IOException; @@ -130,6 +130,7 @@ public static PipeTransferTabletBatchReq toTPipeTransferReq( public static PipeTransferTabletBatchReq fromTPipeTransferReq( final TPipeTransferReq transferReq) { final PipeTransferTabletBatchReq batchReq = new PipeTransferTabletBatchReq(); + final TabletStringInternPool tabletStringInternPool = new TabletStringInternPool(); // Binary req, for rolling upgrading ReadWriteIOUtils.readInt(transferReq.body); @@ -144,8 +145,7 @@ public static PipeTransferTabletBatchReq fromTPipeTransferReq( size = ReadWriteIOUtils.readInt(transferReq.body); for (int i = 0; i < size; ++i) { batchReq.tabletReqs.add( - PipeTransferTabletRawReq.toTPipeTransferRawReq( - Tablet.deserialize(transferReq.body), ReadWriteIOUtils.readBool(transferReq.body))); + PipeTransferTabletRawReq.toTPipeTransferRawReq(transferReq.body, tabletStringInternPool)); } batchReq.version = transferReq.version; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTabletRawReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTabletRawReq.java index 47bf4d4489700..60619fd426816 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTabletRawReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTabletRawReq.java @@ -22,6 +22,8 @@ import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.IoTDBSinkRequestVersion; import org.apache.iotdb.commons.pipe.sink.payload.thrift.request.PipeRequestType; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils.TabletStringInternPool; import org.apache.iotdb.db.pipe.sink.util.PipeTabletEventSorter; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; @@ -82,6 +84,17 @@ public static PipeTransferTabletRawReq toTPipeTransferRawReq( return tabletReq; } + public static PipeTransferTabletRawReq toTPipeTransferRawReq( + final ByteBuffer buffer, final TabletStringInternPool tabletStringInternPool) { + final PipeTransferTabletRawReq tabletReq = new PipeTransferTabletRawReq(); + + tabletReq.tablet = + PipeTabletUtils.internTablet(Tablet.deserialize(buffer), tabletStringInternPool); + tabletReq.isAligned = ReadWriteIOUtils.readBool(buffer); + + return tabletReq; + } + /////////////////////////////// Thrift /////////////////////////////// public static PipeTransferTabletRawReq toTPipeTransferReq( @@ -105,10 +118,8 @@ public static PipeTransferTabletRawReq toTPipeTransferReq( } public static PipeTransferTabletRawReq fromTPipeTransferReq(final TPipeTransferReq transferReq) { - final PipeTransferTabletRawReq tabletReq = new PipeTransferTabletRawReq(); - - tabletReq.tablet = Tablet.deserialize(transferReq.body); - tabletReq.isAligned = ReadWriteIOUtils.readBool(transferReq.body); + final PipeTransferTabletRawReq tabletReq = + toTPipeTransferRawReq(transferReq.body, new TabletStringInternPool()); tabletReq.version = transferReq.version; tabletReq.type = transferReq.type; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaNameSpace.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaNameSpace.java index 3a42ec8796916..713a87b2e350f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaNameSpace.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/opcua/server/OpcUaNameSpace.java @@ -374,7 +374,10 @@ private void transferTabletForPubSubModel(final Tablet tablet) throws UaExceptio for (int rowIndex = 0; rowIndex < tablet.rowSize; ++rowIndex) { // Filter null value - if (tablet.bitMaps[columnIndex].isMarked(rowIndex)) { + if (tablet.bitMaps != null + && columnIndex < tablet.bitMaps.length + && tablet.bitMaps[columnIndex] != null + && tablet.bitMaps[columnIndex].isMarked(rowIndex)) { continue; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/util/PipeTabletEventSorter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/util/PipeTabletEventSorter.java index 17e7a7c13e263..50beb6405e96e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/util/PipeTabletEventSorter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/util/PipeTabletEventSorter.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.pipe.sink.util; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils; + import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.BitMap; @@ -129,6 +131,7 @@ private void deduplicateTimestamps() { // Col: [6, 1] private void sortAndMayDeduplicateValuesAndBitMaps() { int columnIndex = 0; + boolean bitMapsModified = false; for (int i = 0, size = tablet.getSchemas().size(); i < size; i++) { final IMeasurementSchema schema = tablet.getSchemas().get(i); if (schema != null) { @@ -145,10 +148,15 @@ private void sortAndMayDeduplicateValuesAndBitMaps() { if (tablet.bitMaps != null && tablet.bitMaps[columnIndex] != null) { tablet.bitMaps[columnIndex] = deDuplicatedBitMap; + bitMapsModified = true; } columnIndex++; } } + + if (bitMapsModified) { + tablet.bitMaps = PipeTabletUtils.compactBitMaps(tablet.bitMaps, deDuplicatedSize); + } } private Object reorderValueListAndBitMap( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java index faacc10eccded..6a1e9bf1bb48c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java @@ -37,6 +37,7 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; +import org.apache.iotdb.db.utils.BitMapUtils; import org.apache.iotdb.db.utils.QueryDataSetUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -277,7 +278,7 @@ public List splitByPartition(IAnalysis analysis) { long[] subTimes = new long[count]; int destLoc = 0; Object[] values = initTabletValues(dataTypes.length, count, dataTypes); - BitMap[] bitMaps = this.bitMaps == null ? null : initBitmaps(dataTypes.length, count); + BitMap[] bitMaps = initBitmapsForSplit(dataTypes.length, count); System.arraycopy(times, start, subTimes, destLoc, end - start); for (int k = 0; k < values.length; k++) { if (dataTypes[k] != null) { @@ -302,6 +303,7 @@ public List splitByPartition(IAnalysis analysis) { subNode.setFailedMeasurementNumber(getFailedMeasurementNumber()); subNode.setRange(locs); subNode.setDataRegionReplicaSet(entry.getKey()); + subNode.bitMaps = BitMapUtils.compactBitMaps(subNode.bitMaps, subNode.rowCount); result.add(subNode); } } @@ -366,6 +368,23 @@ private BitMap[] initBitmaps(int columnSize, int rowSize) { return bitMaps; } + protected BitMap[] initBitmapsForSplit(int columnSize, int rowSize) { + if (this.bitMaps == null) { + return null; + } + + final int sourceRowCount = rowCount > 0 ? rowCount : times == null ? 0 : times.length; + final BitMap[] splitBitMaps = new BitMap[columnSize]; + boolean hasBitMap = false; + for (int i = 0; i < columnSize && i < this.bitMaps.length; ++i) { + if (this.bitMaps[i] != null && !this.bitMaps[i].isAllUnmarked()) { + splitBitMaps[i] = new BitMap(rowSize); + hasBitMap = true; + } + } + return hasBitMap ? splitBitMaps : null; + } + @Override public void markFailedMeasurement(int index) { if (measurements[index] == null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java index 2e393678c5d52..525bb48f0dde7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java @@ -34,6 +34,7 @@ import org.apache.iotdb.db.queryengine.plan.analyze.schema.ISchemaValidation; import org.apache.iotdb.db.queryengine.plan.statement.StatementType; import org.apache.iotdb.db.queryengine.plan.statement.StatementVisitor; +import org.apache.iotdb.db.utils.BitMapUtils; import org.apache.iotdb.db.utils.CommonUtils; import org.apache.tsfile.enums.TSDataType; @@ -326,7 +327,7 @@ public List getSplitList() { statement.setMeasurementSchemas(measurementSchemas); statement.setDataTypes(dataTypes); if (this.bitMaps != null) { - statement.setBitMaps(copiedBitMaps); + statement.setBitMaps(BitMapUtils.compactBitMaps(copiedBitMaps, rowCount)); } statement.setFailedMeasurementIndex2Info(failedMeasurementIndex2Info); insertTabletStatementList.add(statement); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/quotas/DefaultOperationQuota.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/quotas/DefaultOperationQuota.java index e4f1170ddb36c..3da4db4b4083a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/quotas/DefaultOperationQuota.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/rescon/quotas/DefaultOperationQuota.java @@ -116,9 +116,7 @@ protected void updateEstimateConsumeQuota(int numWrites, int numReads, Statement case BATCH_INSERT: // InsertTabletStatement InsertTabletStatement insertTabletStatement = (InsertTabletStatement) s; - for (BitMap bitMap : insertTabletStatement.getBitMaps()) { - avgSize += bitMap.getSize(); - } + avgSize += calculationWrite(insertTabletStatement.getBitMaps()); break; case BATCH_INSERT_ONE_DEVICE: // InsertRowsOfOneDeviceStatement @@ -151,10 +149,12 @@ protected void updateEstimateConsumeQuota(int numWrites, int numReads, Statement for (int i = 0; i < insertMultiTabletsStatement.getInsertTabletStatementList().size(); i++) { - for (BitMap bitMap : - insertMultiTabletsStatement.getInsertTabletStatementList().get(i).getBitMaps()) { - avgSize += bitMap.getSize(); - } + avgSize += + calculationWrite( + insertMultiTabletsStatement + .getInsertTabletStatementList() + .get(i) + .getBitMaps()); } } break; @@ -178,6 +178,20 @@ private long calculationWrite(Object[] values) { return size; } + private long calculationWrite(BitMap[] bitMaps) { + if (bitMaps == null) { + return 0; + } + + long size = 0; + for (BitMap bitMap : bitMaps) { + if (bitMap != null) { + size += bitMap.getSize(); + } + } + return size; + } + private long estimateConsume(int numReqs, long avgSize) { if (numReqs > 0) { return avgSize * numReqs; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/BitMapUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/BitMapUtils.java new file mode 100644 index 0000000000000..ba30c8847b4b5 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/BitMapUtils.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.utils; + +import org.apache.tsfile.utils.BitMap; + +import java.util.Objects; + +public final class BitMapUtils { + + private BitMapUtils() {} + + public static BitMap[] compactBitMaps(final BitMap[] bitMaps, final int rowCount) { + if (Objects.isNull(bitMaps)) { + return null; + } + + boolean hasMarkedBitMap = false; + for (int i = 0; i < bitMaps.length; ++i) { + if (Objects.nonNull(bitMaps[i]) && bitMaps[i].isAllUnmarked()) { + bitMaps[i] = null; + } + if (Objects.nonNull(bitMaps[i])) { + hasMarkedBitMap = true; + } + } + return hasMarkedBitMap ? bitMaps : null; + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTabletInsertionEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTabletInsertionEventTest.java index 4a7d77eab8d29..308c5458dc9b9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTabletInsertionEventTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTabletInsertionEventTest.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.pipe.datastructure.pattern.PrefixPipePattern; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeTabletUtils; import org.apache.iotdb.db.pipe.event.common.tablet.TabletInsertionDataContainer; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; @@ -217,7 +218,7 @@ private void createTablet() { tabletForInsertRowNode.values = values; tabletForInsertRowNode.timestamps = new long[] {times[0]}; tabletForInsertRowNode.rowSize = 1; - tabletForInsertRowNode.bitMaps = bitMapsForInsertRowNode; + tabletForInsertRowNode.bitMaps = PipeTabletUtils.compactBitMaps(bitMapsForInsertRowNode, 1); // create tablet for insertTabletNode BitMap[] bitMapsForInsertTabletNode = new BitMap[schemas.length]; @@ -253,7 +254,8 @@ private void createTablet() { tabletForInsertTabletNode.values = values; tabletForInsertTabletNode.timestamps = times; tabletForInsertTabletNode.rowSize = times.length; - tabletForInsertTabletNode.bitMaps = bitMapsForInsertTabletNode; + tabletForInsertTabletNode.bitMaps = + PipeTabletUtils.compactBitMaps(bitMapsForInsertTabletNode, times.length); } @Test @@ -318,6 +320,36 @@ public void convertToAlignedTabletForTest() { Assert.assertTrue(isAligned4); } + @Test + public void convertToTabletSkipsUnnecessaryBitMapsForTest() throws Exception { + final BitMap[] bitMaps = new BitMap[schemas.length]; + bitMaps[0] = new BitMap(times.length); + bitMaps[1] = new BitMap(times.length); + bitMaps[1].mark(1); + + final InsertTabletNode nodeWithSparseColumn = + new InsertTabletNode( + new PlanNodeId("plannode bitmap"), + new PartialPath(deviceId), + false, + measurementIds, + dataTypes, + schemas, + times, + bitMaps, + insertTabletNode.getColumns(), + times.length); + + final Tablet tablet = + new TabletInsertionDataContainer(nodeWithSparseColumn, new PrefixPipePattern(pattern)) + .convertToTablet(); + + Assert.assertNotNull(tablet.bitMaps); + Assert.assertNull(tablet.bitMaps[0]); + Assert.assertNotNull(tablet.bitMaps[1]); + Assert.assertTrue(tablet.bitMaps[1].isMarked(1)); + } + @Test public void convertToTabletWithFilteredRowsForTest() { TabletInsertionDataContainer container1 = diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeTabletUtilsTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeTabletUtilsTest.java new file mode 100644 index 0000000000000..8bf32bd066ffb --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeTabletUtilsTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.event.common.tablet; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.BitMap; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; + +public class PipeTabletUtilsTest { + + @Test + public void testPutValueUnmarksReusedNullRow() { + final List schemas = + Arrays.asList( + new MeasurementSchema("s1", TSDataType.FLOAT), + new MeasurementSchema("s2", TSDataType.FLOAT)); + final Tablet tablet = new Tablet("root.sg.d1", schemas, 2); + + PipeTabletUtils.markNullValue(tablet, 0, 0); + PipeTabletUtils.markNullValue(tablet, 0, 1); + + PipeTabletUtils.putValue(tablet, 0, 0, TSDataType.FLOAT, 1.0f); + PipeTabletUtils.putTimestamp(tablet, 0, 1L); + PipeTabletUtils.compactBitMaps(tablet); + + Assert.assertNull(tablet.bitMaps[0]); + Assert.assertTrue(tablet.bitMaps[1].isMarked(0)); + } + + @Test + public void testCopyBitMapsOrCreateEmptyWithNullBitMaps() { + final List schemas = + Arrays.asList( + new MeasurementSchema("s1", TSDataType.FLOAT), + new MeasurementSchema("s2", TSDataType.FLOAT)); + final Tablet tablet = new Tablet("root.sg.d1", schemas, 2); + tablet.addTimestamp(0, 1L); + tablet.addValue("s1", 0, 1.0f); + tablet.addValue("s2", 0, 2.0f); + + Assert.assertNull(tablet.bitMaps); + + final BitMap[] bitMaps = PipeTabletUtils.copyBitMapsOrCreateEmpty(tablet); + + Assert.assertEquals(schemas.size(), bitMaps.length); + Assert.assertNull(bitMaps[0]); + Assert.assertNull(bitMaps[1]); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeDataNodeThriftRequestTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeDataNodeThriftRequestTest.java index ee9b7218dab35..4e4d11aacbf53 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeDataNodeThriftRequestTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/PipeDataNodeThriftRequestTest.java @@ -321,6 +321,46 @@ public void testPipeTransferTabletBatchReq() throws IOException { Assert.assertFalse(deserializedReq.getTabletReqs().get(0).getIsAligned()); } + @Test + public void testPipeTransferTabletBatchReqInternsRepeatedMeasurementNames() throws IOException { + final List tabletBuffers = new ArrayList<>(); + tabletBuffers.add( + serializeTablet(createSingleValueTablet(new String("root.sg.d"), new String("s1")), false)); + tabletBuffers.add( + serializeTablet(createSingleValueTablet(new String("root.sg.d"), new String("s1")), false)); + + final PipeTransferTabletBatchReq deserializedReq = + PipeTransferTabletBatchReq.fromTPipeTransferReq( + PipeTransferTabletBatchReq.toTPipeTransferReq(Collections.emptyList(), tabletBuffers)); + final Tablet firstTablet = deserializedReq.getTabletReqs().get(0).getTablet(); + final Tablet secondTablet = deserializedReq.getTabletReqs().get(1).getTablet(); + + Assert.assertSame(firstTablet.deviceId, secondTablet.deviceId); + Assert.assertSame( + firstTablet.getSchemas().get(0).getMeasurementId(), + secondTablet.getSchemas().get(0).getMeasurementId()); + } + + private static Tablet createSingleValueTablet(final String deviceId, final String measurement) { + final List schemaList = new ArrayList<>(); + schemaList.add(new MeasurementSchema(measurement, TSDataType.INT32)); + final Tablet tablet = new Tablet(deviceId, schemaList, 1); + tablet.addTimestamp(0, 1); + tablet.addValue(measurement, 0, 1); + tablet.rowSize = 1; + return tablet; + } + + private static ByteBuffer serializeTablet(final Tablet tablet, final boolean isAligned) + throws IOException { + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + tablet.serialize(outputStream); + ReadWriteIOUtils.write(isAligned, outputStream); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + } + @Test public void testPipeTransferFilePieceReq() throws IOException { final byte[] body = "testPipeTransferFilePieceReq".getBytes(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/WritePlanNodeSplitTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/WritePlanNodeSplitTest.java index a93a22b6e9f4a..305a3197cfebf 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/WritePlanNodeSplitTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/WritePlanNodeSplitTest.java @@ -44,6 +44,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.utils.BitMap; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -201,6 +202,9 @@ public void testSplitInsertTablet() throws IllegalPathException { insertTabletNode.setColumns( new Object[] {new int[] {-20, -10, 10, 20, 30, 40, 50, 60, 70, 80, 90, 100}}); insertTabletNode.setRowCount(insertTabletNode.getTimes().length); + final BitMap[] bitMaps = new BitMap[] {new BitMap(insertTabletNode.getRowCount())}; + bitMaps[0].mark(2); + insertTabletNode.setBitMaps(bitMaps); DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDevicePath(insertTabletNode.getDevicePath().getFullPath()); @@ -219,6 +223,12 @@ public void testSplitInsertTablet() throws IllegalPathException { Assert.assertEquals(tabletNode.getTimes().length, 2); TConsensusGroupId regionId = tabletNode.getDataRegionReplicaSet().getRegionId(); Assert.assertEquals(getRegionIdByTime(tabletNode.getMinTime()), regionId.getId()); + if (tabletNode.getTimes()[0] == 1) { + Assert.assertNotNull(tabletNode.getBitMaps()); + Assert.assertTrue(tabletNode.getBitMaps()[0].isMarked(0)); + } else { + Assert.assertNull(tabletNode.getBitMaps()); + } } insertTabletNode = new InsertTabletNode(new PlanNodeId("plan node 2")); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/rescon/quotas/DefaultOperationQuotaTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/rescon/quotas/DefaultOperationQuotaTest.java new file mode 100644 index 0000000000000..5f11039eeb312 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/rescon/quotas/DefaultOperationQuotaTest.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.rescon.quotas; + +import org.apache.iotdb.common.rpc.thrift.TTimedQuota; +import org.apache.iotdb.common.rpc.thrift.ThrottleType; +import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertMultiTabletsStatement; +import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; + +import org.apache.tsfile.utils.BitMap; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.EnumMap; +import java.util.Map; + +public class DefaultOperationQuotaTest { + + @Test + public void testCheckQuotaWithNullAndSparseBitMaps() throws Exception { + final DefaultOperationQuota quota = new DefaultOperationQuota(createQuotaLimiter()); + + final InsertTabletStatement tabletWithoutBitMaps = new InsertTabletStatement(); + tabletWithoutBitMaps.setBitMaps(null); + quota.checkQuota(1, 0, tabletWithoutBitMaps); + + final InsertTabletStatement tabletWithSparseBitMaps = new InsertTabletStatement(); + final BitMap bitMap = new BitMap(8); + bitMap.mark(0); + tabletWithSparseBitMaps.setBitMaps(new BitMap[] {null, bitMap}); + quota.checkQuota(1, 0, tabletWithSparseBitMaps); + + final InsertMultiTabletsStatement multiTabletsStatement = new InsertMultiTabletsStatement(); + multiTabletsStatement.setInsertTabletStatementList( + Arrays.asList(tabletWithoutBitMaps, tabletWithSparseBitMaps)); + quota.checkQuota(1, 0, multiTabletsStatement); + } + + private static QuotaLimiter createQuotaLimiter() { + final Map quotas = new EnumMap<>(ThrottleType.class); + for (final ThrottleType throttleType : ThrottleType.values()) { + quotas.put(throttleType, new TTimedQuota(60_000L, 1_000_000_000L)); + } + return QuotaLimiter.fromThrottle(Collections.unmodifiableMap(quotas)); + } +} From 93275a6e975e46831f4f83dc1ac1bc4b168e2a0b Mon Sep 17 00:00:00 2001 From: Hongzhi Gao <761417898@qq.com> Date: Thu, 28 May 2026 11:45:51 +0800 Subject: [PATCH 064/102] fix(client-cpp): ignore 0.0.0.0 in write redirection --- iotdb-client/client-cpp/src/main/Session.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-client/client-cpp/src/main/Session.cpp b/iotdb-client/client-cpp/src/main/Session.cpp index 6e8d37e6199ea..cb7dbc581d9ee 100644 --- a/iotdb-client/client-cpp/src/main/Session.cpp +++ b/iotdb-client/client-cpp/src/main/Session.cpp @@ -1780,7 +1780,7 @@ void Session::handleQueryRedirection(TEndPoint endPoint) { void Session::handleRedirection(const std::string& deviceId, TEndPoint endPoint) { if (!enableRedirection_) return; - if (endPoint.ip == "127.0.0.1") return; + if (endPoint.ip == "0.0.0.0") return; deviceIdToEndpoint[deviceId] = endPoint; shared_ptr newConnection; @@ -1803,7 +1803,7 @@ void Session::handleRedirection(const std::string& deviceId, TEndPoint endPoint) void Session::handleRedirection(const std::shared_ptr& deviceId, TEndPoint endPoint) { if (!enableRedirection_) return; - if (endPoint.ip == "127.0.0.1") return; + if (endPoint.ip == "0.0.0.0") return; tableModelDeviceIdToEndpoint[deviceId] = endPoint; shared_ptr newConnection; From 0f253d8e65b8b1cea7bb108bc2e859af54106580 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 28 May 2026 15:52:27 +0800 Subject: [PATCH 065/102] Degraded the UT logs (#17782) * Degraded the UT logs (cherry picked from commit 65cd197600d3502160ced4a9f0c25ef7552011ab) * Update TsFileInsertionDataContainer.java --- .../TsFileInsertionDataContainer.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java index f335cff10354a..1eb620e84bea5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainer.java @@ -97,16 +97,18 @@ protected TsFileInsertionDataContainer( .forceAllocateForTabletWithRetry( IoTDBDescriptor.getInstance().getConfig().getPipeDataStructureTabletSizeInBytes()); - LOGGER.info( - "TsFile {} has initialized {}, pipeName: {}, creation time: {}, pattern: {}, startTime: {}, endTime: {}, withMod: {}", - tsFile, - getClass().getSimpleName(), - pipeName, - creationTime, - pattern, - startTime, - endTime, - isWithMod); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "TsFile {} has initialized {}, pipeName: {}, creation time: {}, pattern: {}, startTime: {}, endTime: {}, withMod: {}", + tsFile, + getClass().getSimpleName(), + pipeName, + creationTime, + pattern, + startTime, + endTime, + isWithMod); + } } /** From ca52520f4823397fa00eeba8ea1ecd5c4aba3345 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 28 May 2026 17:16:14 +0800 Subject: [PATCH 066/102] [To dev/1.3] Fix pipe drop event discard with restart-aware committer keys (#17748) (#17778) --- .../task/connection/PipeEventCollector.java | 5 +- .../PipeRealtimePriorityBlockingQueue.java | 10 ++-- .../task/subtask/sink/PipeSinkSubtask.java | 25 +++++---- .../sink/PipeSinkSubtaskLifeCycle.java | 9 ++-- .../subtask/sink/PipeSinkSubtaskManager.java | 6 ++- .../evolvable/batch/PipeTabletEventBatch.java | 17 +++++-- .../batch/PipeTransferBatchReqBuilder.java | 11 ++-- .../airgap/IoTDBDataRegionAirGapSink.java | 8 ++- .../async/IoTDBDataRegionAsyncSink.java | 41 +++++++-------- .../thrift/sync/IoTDBDataRegionSyncSink.java | 8 ++- .../websocket/WebSocketConnectorServer.java | 51 +++++++++---------- .../protocol/websocket/WebSocketSink.java | 8 +++ .../SubscriptionSinkSubtaskLifeCycle.java | 4 +- .../SubscriptionSinkSubtaskManager.java | 7 ++- .../subtask/sink/PipeSinkSubtaskTest.java | 6 ++- .../task/connection/BlockingPendingQueue.java | 39 +++++++++----- .../task/progress/PipeEventCommitManager.java | 5 ++ .../PipeConnectorWithEventDiscard.java | 7 +++ 18 files changed, 171 insertions(+), 96 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java index 95e8196ad3865..f0cc462161287 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java @@ -202,7 +202,10 @@ private void collectEvent(final Event event) { enrichedEvent.setRebootTimes(PipeDataNodeAgent.runtime().getRebootTimes()); if (enrichedEvent.getPipeName() != null - && pendingQueue.isPipeDropped(enrichedEvent.getPipeName(), creationTime, regionId)) { + && (pendingQueue.isEventFromDroppedPipe(enrichedEvent) + || (enrichedEvent.getCommitterKey() == null + && pendingQueue.isPipeDropped( + enrichedEvent.getPipeName(), creationTime, regionId)))) { enrichedEvent.clearReferenceCount(PipeEventCollector.class.getName()); return; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeRealtimePriorityBlockingQueue.java index f972bba0e6ede..e35763c30123a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeRealtimePriorityBlockingQueue.java @@ -360,12 +360,16 @@ public void discardAllEvents() { @Override public void discardEventsOfPipe( final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { - super.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); + discardEventsOfPipe(new CommitterKey(pipeNameToDrop, creationTimeToDrop, regionId, -1)); + } + + @Override + public void discardEventsOfPipe(final CommitterKey committerKey) { + super.discardEventsOfPipe(committerKey); tsfileInsertEventDeque.removeIf( event -> { if (event instanceof EnrichedEvent - && isEventFromPipe( - ((EnrichedEvent) event), pipeNameToDrop, creationTimeToDrop, regionId)) { + && isEventFromPipe((EnrichedEvent) event, committerKey)) { if (((EnrichedEvent) event) .clearReferenceCount(PipeRealtimePriorityBlockingQueue.class.getName())) { eventCounter.decreaseEventCount(event); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java index 1e7c50f389e51..c855eb5714013 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtask.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.agent.task.subtask.PipeAbstractSinkSubtask; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; @@ -201,10 +202,9 @@ public void close() { * When a pipe is dropped, the connector maybe reused and will not be closed. So we just discard * its queued events in the output pipe connector. */ - public void discardEventsOfPipe( - final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + public void discardEventsOfPipe(final CommitterKey committerKey) { // Try to remove the events as much as possible - inputPendingQueue.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); + inputPendingQueue.discardEventsOfPipe(committerKey); try { increaseHighPriorityTaskCount(); @@ -217,9 +217,7 @@ public void discardEventsOfPipe( // use a new thread to stop all the pipes, we will not encounter deadlock here. Or else we // will. if (lastEvent instanceof EnrichedEvent - && pipeNameToDrop.equals(((EnrichedEvent) lastEvent).getPipeName()) - && creationTimeToDrop == ((EnrichedEvent) lastEvent).getCreationTime() - && regionId == ((EnrichedEvent) lastEvent).getRegionId()) { + && isEventFromPipe((EnrichedEvent) lastEvent, committerKey)) { // Do not clear the last event's reference counts because it may be on transferring lastEvent = null; // Submit self to avoid that the lastEvent has been retried "max times" times and has @@ -241,9 +239,7 @@ public void discardEventsOfPipe( // clear the lastExceptionEvent. It's safe to potentially clear it twice because we have the // "nonnull" detection. if (lastExceptionEvent instanceof EnrichedEvent - && pipeNameToDrop.equals(((EnrichedEvent) lastExceptionEvent).getPipeName()) - && creationTimeToDrop == ((EnrichedEvent) lastExceptionEvent).getCreationTime() - && regionId == ((EnrichedEvent) lastExceptionEvent).getRegionId()) { + && isEventFromPipe((EnrichedEvent) lastExceptionEvent, committerKey)) { clearReferenceCountAndReleaseLastExceptionEvent(); } } @@ -252,11 +248,18 @@ public void discardEventsOfPipe( } if (outputPipeConnector instanceof PipeConnectorWithEventDiscard) { - ((PipeConnectorWithEventDiscard) outputPipeConnector) - .discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); + ((PipeConnectorWithEventDiscard) outputPipeConnector).discardEventsOfPipe(committerKey); } } + private static boolean isEventFromPipe( + final EnrichedEvent event, final CommitterKey committerKey) { + return committerKey.getPipeName().equals(event.getPipeName()) + && committerKey.getCreationTime() == event.getCreationTime() + && committerKey.getRegionId() == event.getRegionId() + && (committerKey.getRestartTimes() < 0 || committerKey.equals(event.getCommitterKey())); + } + //////////////////////////// APIs provided for metric framework //////////////////////////// public String getAttributeSortedString() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskLifeCycle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskLifeCycle.java index 1780f5a87efa8..61a064e1637a4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskLifeCycle.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskLifeCycle.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.agent.task.subtask.sink; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.db.pipe.agent.task.execution.PipeSinkSubtaskExecutor; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.pipe.api.event.Event; @@ -86,19 +87,17 @@ public synchronized void register() { * Otherwise, the {@link PipeSinkSubtaskLifeCycle#runningTaskCount} might be inconsistent with the * {@link PipeSinkSubtaskLifeCycle#registeredTaskCount} because of parallel connector scheduling. * - * @param pipeNameToDeregister pipe name - * @param regionId region id + * @param committerKey committer key of the pipe task to deregister * @return {@code true} if the {@link PipeSinkSubtask} is out of life cycle, indicating that the * {@link PipeSinkSubtask} should never be used again * @throws IllegalStateException if {@link PipeSinkSubtaskLifeCycle#registeredTaskCount} <= 0 */ - public synchronized boolean deregister( - final String pipeNameToDeregister, final long creationTimeToDeregister, final int regionId) { + public synchronized boolean deregister(final CommitterKey committerKey) { if (registeredTaskCount <= 0) { throw new IllegalStateException("registeredTaskCount <= 0"); } - subtask.discardEventsOfPipe(pipeNameToDeregister, creationTimeToDeregister, regionId); + subtask.discardEventsOfPipe(committerKey); try { if (registeredTaskCount > 1) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskManager.java index d7f81c12dbc98..817471c785ab6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskManager.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.consensus.DataRegionId; import org.apache.iotdb.commons.pipe.agent.plugin.builtin.BuiltinPipePlugin; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.agent.task.progress.PipeEventCommitManager; import org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; @@ -209,7 +210,10 @@ public synchronized void deregister( // Shall not be empty final PipeSinkSubtaskExecutor executor = lifeCycles.get(0).executor; - lifeCycles.removeIf(o -> o.deregister(pipeName, creationTime, regionId)); + final CommitterKey committerKey = + PipeEventCommitManager.getInstance().getCommitterKey(pipeName, creationTime, regionId); + + lifeCycles.removeIf(o -> o.deregister(committerKey)); if (lifeCycles.isEmpty()) { attributeSortedString2SubtaskLifeCycleMap.remove(attributeSortedString); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventBatch.java index c44e12a4bbf20..7058b88575015 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTabletEventBatch.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.sink.payload.evolvable.batch; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; @@ -156,11 +157,13 @@ public synchronized void close() { */ public synchronized void discardEventsOfPipe( final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + discardEventsOfPipe(new CommitterKey(pipeNameToDrop, creationTimeToDrop, regionId, -1)); + } + + public synchronized void discardEventsOfPipe(final CommitterKey committerKey) { events.removeIf( event -> { - if (pipeNameToDrop.equals(event.getPipeName()) - && creationTimeToDrop == event.getCreationTime() - && regionId == event.getRegionId()) { + if (isEventFromPipe(event, committerKey)) { event.clearReferenceCount(IoTDBDataRegionAsyncSink.class.getName()); return true; } @@ -168,6 +171,14 @@ public synchronized void discardEventsOfPipe( }); } + private static boolean isEventFromPipe( + final EnrichedEvent event, final CommitterKey committerKey) { + return committerKey.getPipeName().equals(event.getPipeName()) + && committerKey.getCreationTime() == event.getCreationTime() + && committerKey.getRegionId() == event.getRegionId() + && (committerKey.getRestartTimes() < 0 || committerKey.equals(event.getCommitterKey())); + } + public synchronized void decreaseEventsReferenceCount( final String holderMessage, final boolean shouldReport) { events.forEach(event -> event.decreaseReferenceCount(holderMessage, shouldReport)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java index ac5f568f1c6af..45264138596ff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.sink.payload.evolvable.batch; import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; @@ -197,10 +198,12 @@ public boolean isEmpty() { public synchronized void discardEventsOfPipe( final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { - defaultBatch.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); - endPointToBatch - .values() - .forEach(batch -> batch.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId)); + discardEventsOfPipe(new CommitterKey(pipeNameToDrop, creationTimeToDrop, regionId, -1)); + } + + public synchronized void discardEventsOfPipe(final CommitterKey committerKey) { + defaultBatch.discardEventsOfPipe(committerKey); + endPointToBatch.values().forEach(batch -> batch.discardEventsOfPipe(committerKey)); } public int size() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java index 13bcb537ae1ed..81e745dc6a7ff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.sink.protocol.airgap; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.sink.limiter.TsFileSendRateLimiter; import org.apache.iotdb.commons.utils.RetryUtils; @@ -546,8 +547,13 @@ protected byte[] compressIfNeeded(final byte[] reqInBytes) throws IOException { @Override public synchronized void discardEventsOfPipe( final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + discardEventsOfPipe(new CommitterKey(pipeNameToDrop, creationTimeToDrop, regionId, -1)); + } + + @Override + public synchronized void discardEventsOfPipe(final CommitterKey committerKey) { if (Objects.nonNull(tabletBatchBuilder)) { - tabletBatchBuilder.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); + tabletBatchBuilder.discardEventsOfPipe(committerKey); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java index fe3d44bedb488..69dcb922e4a24 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java @@ -22,8 +22,8 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.config.PipeConfig; -import org.apache.iotdb.commons.pipe.datastructure.Triple; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.resource.log.PipeLogger; import org.apache.iotdb.commons.pipe.sink.protocol.IoTDBSink; @@ -123,9 +123,7 @@ public class IoTDBDataRegionAsyncSink extends IoTDBSink { private final Map pendingHandlers = new ConcurrentHashMap<>(); - // Pipe name, creation time, region id - private final Set> droppedPipeTaskKeys = - ConcurrentHashMap.newKeySet(); + private final Set droppedPipeTaskKeys = ConcurrentHashMap.newKeySet(); private boolean enableSendTsFileLimit; private volatile boolean isConnectionException; @@ -722,16 +720,20 @@ public boolean isEnableSendTsFileLimit() { @Override public synchronized void discardEventsOfPipe( final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { - droppedPipeTaskKeys.add(new Triple<>(pipeNameToDrop, creationTimeToDrop, regionId)); + discardEventsOfPipe(new CommitterKey(pipeNameToDrop, creationTimeToDrop, regionId, -1)); + } + + @Override + public synchronized void discardEventsOfPipe(final CommitterKey committerKey) { + droppedPipeTaskKeys.add(committerKey); if (isTabletBatchModeEnabled && Objects.nonNull(tabletBatchBuilder)) { - tabletBatchBuilder.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); + tabletBatchBuilder.discardEventsOfPipe(committerKey); } retryEventQueue.removeIf( event -> { if (event instanceof EnrichedEvent - && isDroppedPipe( - (EnrichedEvent) event, pipeNameToDrop, creationTimeToDrop, regionId)) { + && isDroppedPipe((EnrichedEvent) event, committerKey)) { ((EnrichedEvent) event).clearReferenceCount(IoTDBDataRegionAsyncSink.class.getName()); retryEventQueueEventCounter.decreaseEventCount(event); return true; @@ -742,8 +744,7 @@ && isDroppedPipe( retryTsFileQueue.removeIf( event -> { if (event instanceof EnrichedEvent - && isDroppedPipe( - (EnrichedEvent) event, pipeNameToDrop, creationTimeToDrop, regionId)) { + && isDroppedPipe((EnrichedEvent) event, committerKey)) { ((EnrichedEvent) event).clearReferenceCount(IoTDBDataRegionAsyncSink.class.getName()); retryEventQueueEventCounter.decreaseEventCount(event); return true; @@ -845,18 +846,14 @@ public void setTransferTsFileCounter(AtomicInteger transferTsFileCounter) { } private boolean isDroppedPipe(final EnrichedEvent event) { - return droppedPipeTaskKeys.contains( - new Triple<>(event.getPipeName(), event.getCreationTime(), event.getRegionId())); - } - - private static boolean isDroppedPipe( - final EnrichedEvent event, - final String pipeNameToDrop, - final long creationTimeToDrop, - final int regionId) { - return pipeNameToDrop.equals(event.getPipeName()) - && creationTimeToDrop == event.getCreationTime() - && regionId == event.getRegionId(); + return droppedPipeTaskKeys.stream().anyMatch(key -> isDroppedPipe(event, key)); + } + + private static boolean isDroppedPipe(final EnrichedEvent event, final CommitterKey committerKey) { + return committerKey.getPipeName().equals(event.getPipeName()) + && committerKey.getCreationTime() == event.getCreationTime() + && committerKey.getRegionId() == event.getRegionId() + && (committerKey.getRestartTimes() < 0 || committerKey.equals(event.getCommitterKey())); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java index 552b8cf1cae1d..ef3d59f0d2a54 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.sink.client.IoTDBSyncClient; import org.apache.iotdb.commons.pipe.sink.limiter.TsFileSendRateLimiter; @@ -523,8 +524,13 @@ public TPipeTransferReq compressIfNeeded(final TPipeTransferReq req) throws IOEx @Override public synchronized void discardEventsOfPipe( final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { + discardEventsOfPipe(new CommitterKey(pipeNameToDrop, creationTimeToDrop, regionId, -1)); + } + + @Override + public synchronized void discardEventsOfPipe(final CommitterKey committerKey) { if (Objects.nonNull(tabletBatchBuilder)) { - tabletBatchBuilder.discardEventsOfPipe(pipeNameToDrop, creationTimeToDrop, regionId); + tabletBatchBuilder.discardEventsOfPipe(committerKey); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketConnectorServer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketConnectorServer.java index 2a8b5c8c3c07d..0ecd1ad6e340b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketConnectorServer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketConnectorServer.java @@ -19,7 +19,7 @@ package org.apache.iotdb.db.pipe.sink.protocol.websocket; -import org.apache.iotdb.commons.pipe.datastructure.Triple; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.pipe.api.event.Event; @@ -59,9 +59,7 @@ public class WebSocketConnectorServer extends WebSocketServer { private final ConcurrentHashMap> eventsWaitingForAck = new ConcurrentHashMap<>(); - // Pipe name, creation time, region id - private final Set> droppedPipeTaskKeys = - ConcurrentHashMap.newKeySet(); + private final Set droppedPipeTaskKeys = ConcurrentHashMap.newKeySet(); private final BidiMap router = new DualTreeBidiMap(null, Comparator.comparing(Object::hashCode)) {}; @@ -117,33 +115,33 @@ public synchronized void unregister(WebSocketSink connector) { .forEach((eventId, eventWrapper) -> discardEvent(eventWrapper.event)); } - droppedPipeTaskKeys.removeIf(key -> key.getFirst().equals(pipeName)); + droppedPipeTaskKeys.removeIf(key -> key.getPipeName().equals(pipeName)); } public synchronized void discardEventsOfPipe( final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { - droppedPipeTaskKeys.add(new Triple<>(pipeNameToDrop, creationTimeToDrop, regionId)); + discardEventsOfPipe(new CommitterKey(pipeNameToDrop, creationTimeToDrop, regionId, -1)); + } + + public synchronized void discardEventsOfPipe(final CommitterKey committerKey) { + droppedPipeTaskKeys.add(committerKey); final PriorityBlockingQueue eventTransferQueue = - eventsWaitingForTransfer.get(pipeNameToDrop); + eventsWaitingForTransfer.get(committerKey.getPipeName()); if (eventTransferQueue != null) { eventTransferQueue.removeIf( - eventWrapper -> - discardIfMatches(eventWrapper.event, pipeNameToDrop, creationTimeToDrop, regionId)); + eventWrapper -> discardIfMatches(eventWrapper.event, committerKey)); synchronized (eventTransferQueue) { eventTransferQueue.notifyAll(); } } final ConcurrentHashMap eventId2EventMap = - eventsWaitingForAck.get(pipeNameToDrop); + eventsWaitingForAck.get(committerKey.getPipeName()); if (eventId2EventMap != null) { eventId2EventMap .entrySet() - .removeIf( - entry -> - discardIfMatches( - entry.getValue().event, pipeNameToDrop, creationTimeToDrop, regionId)); + .removeIf(entry -> discardIfMatches(entry.getValue().event, committerKey)); } } @@ -515,19 +513,13 @@ public EventWaitingForAck(WebSocketSink connector, Event event) { } } - private boolean discardIfMatches( - final Event event, - final String pipeNameToDrop, - final long creationTimeToDrop, - final int regionId) { + private boolean discardIfMatches(final Event event, final CommitterKey committerKey) { if (!(event instanceof EnrichedEvent)) { return false; } final EnrichedEvent enrichedEvent = (EnrichedEvent) event; - if (!pipeNameToDrop.equals(enrichedEvent.getPipeName()) - || creationTimeToDrop != enrichedEvent.getCreationTime() - || regionId != enrichedEvent.getRegionId()) { + if (!isEventFromPipe(enrichedEvent, committerKey)) { return false; } @@ -537,11 +529,16 @@ private boolean discardIfMatches( private boolean isDroppedPipe(final Event event) { return event instanceof EnrichedEvent - && droppedPipeTaskKeys.contains( - new Triple<>( - ((EnrichedEvent) event).getPipeName(), - ((EnrichedEvent) event).getCreationTime(), - ((EnrichedEvent) event).getRegionId())); + && droppedPipeTaskKeys.stream() + .anyMatch(key -> isEventFromPipe((EnrichedEvent) event, key)); + } + + private static boolean isEventFromPipe( + final EnrichedEvent event, final CommitterKey committerKey) { + return committerKey.getPipeName().equals(event.getPipeName()) + && committerKey.getCreationTime() == event.getCreationTime() + && committerKey.getRegionId() == event.getRegionId() + && (committerKey.getRestartTimes() < 0 || committerKey.equals(event.getCommitterKey())); } private boolean isQueueAvailable( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketSink.java index 40fccc12c999f..7841e0199b250 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/websocket/WebSocketSink.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.sink.protocol.websocket; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.sink.protocol.PipeConnectorWithEventDiscard; @@ -173,6 +174,13 @@ public void discardEventsOfPipe( } } + @Override + public void discardEventsOfPipe(final CommitterKey committerKey) { + if (server != null) { + server.discardEventsOfPipe(committerKey); + } + } + public void commit(EnrichedEvent enrichedEvent) { Optional.ofNullable(enrichedEvent) .ifPresent(event -> event.decreaseReferenceCount(WebSocketSink.class.getName(), true)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskLifeCycle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskLifeCycle.java index af871feaa7e0d..c24098f44fca6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskLifeCycle.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskLifeCycle.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.subscription.task.subtask; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.db.pipe.agent.task.execution.PipeSinkSubtaskExecutor; import org.apache.iotdb.db.pipe.agent.task.subtask.sink.PipeSinkSubtask; import org.apache.iotdb.db.pipe.agent.task.subtask.sink.PipeSinkSubtaskLifeCycle; @@ -63,8 +64,7 @@ public synchronized void register() { } @Override - public synchronized boolean deregister( - final String pipeNameToDeregister, final long creationTimeToDeregister, final int regionId) { + public synchronized boolean deregister(final CommitterKey committerKey) { if (registeredTaskCount <= 0) { throw new IllegalStateException("registeredTaskCount <= 0"); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskManager.java index f4547673eaa42..1c081888b3577 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/subtask/SubscriptionSinkSubtaskManager.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.pipe.agent.plugin.builtin.BuiltinPipePlugin; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.agent.task.progress.PipeEventCommitManager; import org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; @@ -167,7 +168,11 @@ public synchronized void deregister( final PipeSinkSubtaskLifeCycle lifeCycle = attributeSortedString2SubtaskLifeCycleMap.get(attributeSortedString); - if (lifeCycle.deregister(pipeName, creationTime, regionId)) { + + final CommitterKey committerKey = + PipeEventCommitManager.getInstance().getCommitterKey(pipeName, creationTime, regionId); + + if (lifeCycle.deregister(committerKey)) { attributeSortedString2SubtaskLifeCycleMap.remove(attributeSortedString); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskTest.java index ddfc699721b92..2a15fb9ea181b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/agent/task/subtask/sink/PipeSinkSubtaskTest.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.agent.task.subtask.sink; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.sink.protocol.PipeConnectorWithEventDiscard; import org.apache.iotdb.pipe.api.PipeConnector; @@ -51,9 +52,10 @@ public void testDiscardEventsOfPipeDelegatesToConnector() { connector)); try { - subtask.discardEventsOfPipe("pipe", 1L, 1); + final CommitterKey committerKey = new CommitterKey("pipe", 1L, 1, -1); + subtask.discardEventsOfPipe(committerKey); - verify((PipeConnectorWithEventDiscard) connector).discardEventsOfPipe("pipe", 1L, 1); + verify((PipeConnectorWithEventDiscard) connector).discardEventsOfPipe(committerKey); } finally { subtask.close(); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java index 8d920121363a3..c7b91f36d222b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java @@ -19,8 +19,8 @@ package org.apache.iotdb.commons.pipe.agent.task.connection; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.config.PipeConfig; -import org.apache.iotdb.commons.pipe.datastructure.Triple; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.metric.PipeEventCounter; import org.apache.iotdb.pipe.api.event.Event; @@ -47,9 +47,7 @@ public abstract class BlockingPendingQueue { protected final AtomicBoolean isClosed = new AtomicBoolean(false); - // Pipe name, creation time, region id - protected final Set> droppedPipeTaskKeys = - ConcurrentHashMap.newKeySet(); + protected final Set droppedPipeTaskKeys = ConcurrentHashMap.newKeySet(); protected BlockingPendingQueue( final BlockingQueue pendingQueue, final PipeEventCounter eventCounter) { @@ -138,12 +136,15 @@ public void discardAllEvents() { public void discardEventsOfPipe( final String pipeNameToDrop, final long creationTimeToDrop, final int regionId) { - droppedPipeTaskKeys.add(new Triple<>(pipeNameToDrop, creationTimeToDrop, regionId)); + discardEventsOfPipe(new CommitterKey(pipeNameToDrop, creationTimeToDrop, regionId, -1)); + } + + public void discardEventsOfPipe(final CommitterKey committerKey) { + droppedPipeTaskKeys.add(committerKey); pendingQueue.removeIf( event -> { if (event instanceof EnrichedEvent - && isEventFromPipe( - ((EnrichedEvent) event), pipeNameToDrop, creationTimeToDrop, regionId)) { + && isEventFromPipe((EnrichedEvent) event, committerKey)) { if (((EnrichedEvent) event).clearReferenceCount(BlockingPendingQueue.class.getName())) { eventCounter.decreaseEventCount(event); } @@ -191,16 +192,30 @@ protected static boolean isEventFromPipe( && regionId == event.getRegionId(); } + protected static boolean isEventFromPipe( + final EnrichedEvent event, final CommitterKey committerKey) { + return committerKey.getPipeName().equals(event.getPipeName()) + && committerKey.getCreationTime() == event.getCreationTime() + && committerKey.getRegionId() == event.getRegionId() + && (committerKey.getRestartTimes() < 0 || committerKey.equals(event.getCommitterKey())); + } + protected boolean isEventFromDroppedPipe(final E event) { return event instanceof EnrichedEvent && ((EnrichedEvent) event).getPipeName() != null - && isPipeDropped( - ((EnrichedEvent) event).getPipeName(), - ((EnrichedEvent) event).getCreationTime(), - ((EnrichedEvent) event).getRegionId()); + && isEventFromDroppedPipe((EnrichedEvent) event); + } + + public boolean isEventFromDroppedPipe(final EnrichedEvent event) { + return droppedPipeTaskKeys.stream().anyMatch(key -> isEventFromPipe(event, key)); } public boolean isPipeDropped(final String pipeName, final long creationTime, final int regionId) { - return droppedPipeTaskKeys.contains(new Triple<>(pipeName, creationTime, regionId)); + return droppedPipeTaskKeys.stream() + .anyMatch( + key -> + key.getPipeName().equals(pipeName) + && key.getCreationTime() == creationTime + && key.getRegionId() == regionId); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitManager.java index 7056b052a3ee1..9e1653a25167d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitManager.java @@ -169,6 +169,11 @@ private boolean commitSingleId( return true; } + public CommitterKey getCommitterKey( + final String pipeName, final long creationTime, final int regionId) { + return generateCommitterKey(pipeName, creationTime, regionId); + } + private CommitterKey generateCommitterKey( final String pipeName, final long creationTime, final int regionId) { return taskAgent.getCommitterKey( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/PipeConnectorWithEventDiscard.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/PipeConnectorWithEventDiscard.java index ab4dbcf90750f..4ffc0c25ed244 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/PipeConnectorWithEventDiscard.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/sink/protocol/PipeConnectorWithEventDiscard.java @@ -19,7 +19,14 @@ package org.apache.iotdb.commons.pipe.sink.protocol; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; + public interface PipeConnectorWithEventDiscard { void discardEventsOfPipe(String pipeName, long creationTime, int regionId); + + default void discardEventsOfPipe(final CommitterKey committerKey) { + discardEventsOfPipe( + committerKey.getPipeName(), committerKey.getCreationTime(), committerKey.getRegionId()); + } } From 5d4277a49a1c17af753dc767604c6791bab393e3 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 28 May 2026 18:43:02 +0800 Subject: [PATCH 067/102] Pipe: Removed the useless ban on async file transfer for non realtime-first pipes (#17785) --- .../sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java index 69dcb922e4a24..aaef2ae435f85 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java @@ -431,7 +431,7 @@ private void transfer(final PipeTransferTsFileHandler pipeTransferTsFileHandler) }, transferTsFileClientManager.getExecutor()); - if (PipeConfig.getInstance().isTransferTsFileSync() || !isRealtimeFirst) { + if (PipeConfig.getInstance().isTransferTsFileSync()) { try { completableFuture.get(); } catch (final Exception e) { From ddeb23b53cef249d9b2931da866063e489dd382b Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 1 Jun 2026 16:08:55 +0800 Subject: [PATCH 068/102] [To dev/1.3] Pipe Log: Reduce repeatable logs (#17700) (#17793) Backport the non-i18n log reduction part of b616502aec0b700d0b7f3a1577e9ecee1edc365b. --- .../IoTDBDataNodeAsyncClientManager.java | 18 +++++++++-------- .../PipeConsensusTabletBatchEventHandler.java | 14 ++++++++----- ...eConsensusTabletInsertionEventHandler.java | 20 +++++++++++++------ ...eConsensusTsFileInsertionEventHandler.java | 10 ++++++---- .../async/IoTDBDataRegionAsyncSink.java | 20 ++++++++++--------- .../handler/PipeTransferTrackableHandler.java | 4 +++- .../handler/PipeTransferTsFileHandler.java | 13 +++++++----- 7 files changed, 61 insertions(+), 38 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeAsyncClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeAsyncClientManager.java index 435f890f3a2e2..e2f6dbec5693a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeAsyncClientManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/client/IoTDBDataNodeAsyncClientManager.java @@ -193,11 +193,12 @@ public AsyncPipeDataTransferServiceClient borrowClient(final TEndPoint endPoint) return client; } } catch (final Exception e) { - LOGGER.warn( - "failed to borrow client {}:{} for cached leader.", + PipeLogger.log( + LOGGER::warn, + e, + "Failed to borrow client %s:%s for cached leader.", endPoint.getIp(), - endPoint.getPort(), - e); + endPoint.getPort()); } return borrowClient(); @@ -342,11 +343,12 @@ public void onError(final Exception e) { client.close(); client.invalidateAll(); } catch (final Exception e) { - LOGGER.warn( - "Failed to close client {}:{} after handshake failure when the manager is closed.", + PipeLogger.log( + LOGGER::warn, + e, + "Failed to close client %s:%s after handshake failure when the manager is closed.", targetNodeUrl.getIp(), - targetNodeUrl.getPort(), - e); + targetNodeUrl.getPort()); } } client.setShouldReturnSelf(true); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java index 56d2c9ad0626f..32481f43bef1b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.commons.pipe.resource.log.PipeLogger; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferReq; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferResp; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; @@ -116,17 +117,20 @@ public void onComplete(final TPipeConsensusBatchTransferResp response) { @Override public void onError(final Exception exception) { - LOGGER.warn( - "PipeConsensus: Failed to transfer TabletInsertionEvent batch. Total failed events: {}, related pipe names: {}", - events.size(), + final Object pipeNames = events.stream() .map( event -> event instanceof EnrichedEvent ? ((EnrichedEvent) event).getPipeName() : "UNKNOWN") - .collect(Collectors.toSet()), - exception); + .collect(Collectors.toSet()); + PipeLogger.log( + LOGGER::warn, + exception, + "PipeConsensus: Failed to transfer TabletInsertionEvent batch. Total failed events: %s, related pipe names: %s", + events.size(), + pipeNames); connector.addFailureEventsToRetryQueue(events); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java index 9d027e711f3f4..91c083fe3c626 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.commons.pipe.resource.log.PipeLogger; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; import org.apache.iotdb.db.pipe.consensus.PipeConsensusSinkMetrics; @@ -106,14 +107,21 @@ public void onComplete(TPipeConsensusTransferResp response) { @Override public void onError(Exception exception) { - LOGGER.warn( - "Failed to transfer TabletInsertionEvent {} (committer key={}, commit id={}).", + final Object eventReportMessage = event instanceof EnrichedEvent ? ((EnrichedEvent) event).coreReportMessage() - : event.toString(), - event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitterKey() : null, - event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitId() : null, - exception); + : event.toString(); + final Object committerKey = + event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitterKey() : null; + final Object commitId = + event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitId() : null; + PipeLogger.log( + LOGGER::warn, + exception, + "Failed to transfer TabletInsertionEvent %s (committer key=%s, commit id=%s).", + eventReportMessage, + committerKey, + commitId); connector.addFailureEventToRetryQueue(event); metric.recordRetryCounter(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java index 22b55239e197d..3496c0cf85755 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java @@ -23,6 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.async.AsyncPipeConsensusServiceClient; import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.commons.pipe.resource.log.PipeLogger; import org.apache.iotdb.commons.pipe.sink.payload.pipeconsensus.response.PipeConsensusTransferFilePieceResp; import org.apache.iotdb.consensus.pipe.thrift.TCommitId; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; @@ -273,12 +274,13 @@ public void onComplete(final TPipeConsensusTransferResp response) { @Override public void onError(final Exception exception) { - LOGGER.warn( - "Failed to transfer TsFileInsertionEvent {} (committer key {}, commit id {}).", + PipeLogger.log( + LOGGER::warn, + exception, + "Failed to transfer TsFileInsertionEvent %s (committer key %s, commit id %s).", tsFile, event.getCommitterKey(), - event.getCommitId(), - exception); + event.getCommitId()); try { if (reader != null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java index aaef2ae435f85..9d0c1563c7883 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java @@ -258,7 +258,7 @@ private void transferInBatchWithoutCheck( false)); } } catch (final Exception e) { - LOGGER.warn("Failed to transfer tsfile batch ({}).", sealedFiles, e); + PipeLogger.log(LOGGER::warn, e, "Failed to transfer tsfile batch (%s).", sealedFiles); if (eventsHadBeenAddedToRetryQueue.compareAndSet(false, true)) { addFailureEventsToRetryQueue(events, e); } @@ -437,17 +437,19 @@ private void transfer(final PipeTransferTsFileHandler pipeTransferTsFileHandler) } catch (final Exception e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); - LOGGER.warn( - "Transfer tsfile event {} asynchronously was interrupted.", - pipeTransferTsFileHandler.getTsFile(), - e); + PipeLogger.log( + LOGGER::warn, + e, + "Transfer tsfile event %s asynchronously was interrupted.", + pipeTransferTsFileHandler.getTsFile()); } pipeTransferTsFileHandler.onError(e); - LOGGER.warn( - "Failed to transfer tsfile event {} asynchronously.", - pipeTransferTsFileHandler.getTsFile(), - e); + PipeLogger.log( + LOGGER::warn, + e, + "Failed to transfer tsfile event %s asynchronously.", + pipeTransferTsFileHandler.getTsFile()); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java index a0e6ad73fe7ca..40b05066a93f7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTrackableHandler.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.client.ThriftClient; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.pipe.resource.log.PipeLogger; import org.apache.iotdb.commons.pipe.sink.payload.thrift.common.PipeTransferSliceReqBuilder; import org.apache.iotdb.db.pipe.sink.protocol.thrift.async.IoTDBDataRegionAsyncSink; import org.apache.iotdb.pipe.api.exception.PipeConnectionException; @@ -105,7 +106,8 @@ protected boolean tryTransfer( client.returnSelf( (e) -> { if (e instanceof IllegalStateException) { - LOGGER.info( + PipeLogger.log( + LOGGER::info, "Illegal state when return the client to object pool, maybe the pool is already cleared. Will ignore."); return true; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java index c79a09ec2397b..8d9648f52921d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java @@ -157,8 +157,9 @@ public void transfer( this.client = client; if (client == null) { - LOGGER.warn( - "Client has been returned to the pool. Current handler status is {}. Will not transfer {}.", + PipeLogger.log( + LOGGER::warn, + "Client has been returned to the pool. Current handler status is %s. Will not transfer %s.", sink.isClosed() ? "CLOSED" : "NOT CLOSED", tsFile); return; @@ -420,7 +421,8 @@ private void returnClientIfNecessary() { client.returnSelf( (e) -> { if (e instanceof IllegalStateException) { - LOGGER.info( + PipeLogger.log( + LOGGER::info, "Illegal state when return the client to object pool, maybe the pool is already cleared. Will ignore."); return true; } @@ -434,8 +436,9 @@ protected void doTransfer( final AsyncPipeDataTransferServiceClient client, final TPipeTransferReq req) throws TException { if (client == null) { - LOGGER.warn( - "Client has been returned to the pool. Current handler status is {}. Will not transfer {}.", + PipeLogger.log( + LOGGER::warn, + "Client has been returned to the pool. Current handler status is %s. Will not transfer %s.", sink.isClosed() ? "CLOSED" : "NOT CLOSED", tsFile); return; From 0507f0b1516a05daf288384c47ca23c95623adc8 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 1 Jun 2026 18:50:31 +0800 Subject: [PATCH 069/102] Load: Stablized the IoTDBLoadTsFileWithModIT (#17787) --- .../iotdb/db/it/IoTDBLoadTsFileWithModIT.java | 249 ++++++++++++++++++ 1 file changed, 249 insertions(+) create mode 100644 integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileWithModIT.java diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileWithModIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileWithModIT.java new file mode 100644 index 0000000000000..4e8e4c8a629a7 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileWithModIT.java @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.it; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.db.exception.DataRegionException; +import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; +import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; +import org.apache.iotdb.db.storageengine.dataregion.wal.recover.file.SealedTsFileRecoverPerformer; +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.ClusterIT; +import org.apache.iotdb.itbase.category.LocalStandaloneIT; +import org.apache.iotdb.itbase.exception.InconsistentDataException; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.exception.write.WriteProcessException; +import org.apache.tsfile.read.common.Path; +import org.apache.tsfile.write.TsFileWriter; +import org.apache.tsfile.write.record.Tablet; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Collections; + +@RunWith(IoTDBTestRunner.class) +@Category({LocalStandaloneIT.class, ClusterIT.class}) +public class IoTDBLoadTsFileWithModIT { + private static File tmpDir; + + @BeforeClass + public static void setUp() throws Exception { + tmpDir = new File(Files.createTempDirectory("load").toUri()); + EnvFactory.getEnv().initClusterEnvironment(); + } + + @AfterClass + public static void tearDown() throws Exception { + try { + if (tmpDir != null && tmpDir.exists()) { + File[] files = tmpDir.listFiles(); + if (files != null) { + for (File file : files) { + try { + Files.delete(file.toPath()); + } catch (IOException ignored) { + // ignore + } + } + } + try { + Files.delete(tmpDir.toPath()); + } catch (IOException ignored) { + // ignore + } + } + } finally { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + } + + private static void clearTmpDir() { + if (tmpDir == null || !tmpDir.exists()) { + return; + } + File[] files = tmpDir.listFiles(); + if (files == null) { + return; + } + for (File file : files) { + try { + Files.delete(file.toPath()); + } catch (IOException ignored) { + // ignore + } + } + } + + private static void executeQuietly(Statement statement, String sql) { + try { + statement.execute(sql); + } catch (SQLException ignored) { + // ignore + } + } + + private static void cleanupTestDb() { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + executeQuietly(statement, "DELETE DATABASE root.test.**"); + executeQuietly(statement, "DELETE DATABASE root.test"); + } catch (SQLException ignored) { + // ignore + } + } + + private void generateFileWithModFile() + throws IOException, WriteProcessException, IllegalPathException, DataRegionException { + TsFileResource resource = generateFile(); + try (ModificationFile modFile = ModificationFile.getNormalMods(resource)) { + modFile.write(new Deletion(new MeasurementPath("root.test.d1.de.s1"), Long.MAX_VALUE, 1, 2)); + } + } + + private TsFileResource generateFile() + throws WriteProcessException, IOException, DataRegionException { + File tsfile = new File(tmpDir, "1-1-0-0.tsfile"); + try (TsFileWriter writer = new TsFileWriter(tsfile)) { + writer.registerAlignedTimeseries( + new Path("root.test.d1.de"), + Collections.singletonList(new MeasurementSchema("s1", TSDataType.BOOLEAN))); + Tablet tablet = + new Tablet( + "root.test.d1.de", + Collections.singletonList(new MeasurementSchema("s1", TSDataType.BOOLEAN))); + for (int i = 0; i < 5; i++) { + int rowIndex = tablet.rowSize++; + tablet.addTimestamp(rowIndex, i); + tablet.addValue("s1", rowIndex, true); + } + writer.writeAligned(tablet); + } + // generate resource file + TsFileResource resource = new TsFileResource(tsfile); + try (SealedTsFileRecoverPerformer performer = new SealedTsFileRecoverPerformer(resource)) { + performer.recover(); + } + resource.setStatusForTest(TsFileResourceStatus.NORMAL); + resource.deserialize(); + return resource; + } + + @Test + public void testWithNewModFile() + throws SQLException, + IOException, + DataRegionException, + WriteProcessException, + IllegalPathException { + try { + generateFileWithModFile(); + try (final Connection connection = EnvFactory.getEnv().getConnection(); + final Statement statement = connection.createStatement()) { + + statement.execute(String.format("load \'%s\'", tmpDir.getAbsolutePath())); + + try (final ResultSet resultSet = + statement.executeQuery("select count(s1) as c from root.test.d1.de")) { + Assert.assertTrue(resultSet.next()); + Assert.assertEquals(3, resultSet.getLong("c")); + } + } + } finally { + clearTmpDir(); + cleanupTestDb(); + } + } + + @Test + public void testWithNewModFileAndLoadAttributes() + throws SQLException, + IOException, + DataRegionException, + WriteProcessException, + IllegalPathException { + try { + generateFileWithModFile(); + final String databaseName = "root.test.d1"; + + try (final Connection connection = EnvFactory.getEnv().getConnection(); + final Statement statement = connection.createStatement()) { + + statement.execute( + String.format( + "load \'%s\' with (" + + "'database-level'='2'," + + "'verify'='true'," + + "'on-success'='none'," + + "'async'='true')", + tmpDir.getAbsolutePath())); + + boolean databaseFound = false; + for (int i = 0; i < 10; i++) { + try (final ResultSet resultSet = statement.executeQuery("show databases")) { + while (resultSet.next()) { + final String currentDatabase = resultSet.getString(1); + if (databaseName.equalsIgnoreCase(currentDatabase)) { + databaseFound = true; + break; + } + } + } catch (InconsistentDataException ignored) { + // Async load propagates the new database metadata to different DataNodes at + // slightly different times, so "show databases" may be inconsistent transiently. + } + + if (databaseFound) { + break; + } + + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + Assert.assertTrue( + "The `database-level` parameter is not working; the generated database does not contain 'root.test.d1'.", + databaseFound); + } + } finally { + clearTmpDir(); + cleanupTestDb(); + } + } +} From e94f0e8ed11b7a5c95f888f76a26e703aa3c03b1 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 2 Jun 2026 15:08:54 +0800 Subject: [PATCH 070/102] [dev/1.3] Pipe: Harden legacy pipe file transfer validation and access checks (#17791) * Pipe: Harden legacy pipe file transfer validation and access checks (#17741) * Fix * fix * Fix legacy pipe receiver test FileUtils import --- .../IoTDBLegacyPipeReceiverSecurityIT.java | 112 ++++++++++++++++ .../legacy/IoTDBLegacyPipeReceiverAgent.java | 50 +++++-- .../protocol/legacy/IoTDBLegacyPipeSink.java | 26 ++++ .../thrift/impl/ClientRPCServiceImpl.java | 54 ++++++-- .../IoTDBLegacyPipeReceiverAgentTest.java | 123 ++++++++++++++++++ 5 files changed, 344 insertions(+), 21 deletions(-) create mode 100644 integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBLegacyPipeReceiverSecurityIT.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/IoTDBLegacyPipeReceiverAgentTest.java diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBLegacyPipeReceiverSecurityIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBLegacyPipeReceiverSecurityIT.java new file mode 100644 index 0000000000000..51e1e211535e8 --- /dev/null +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBLegacyPipeReceiverSecurityIT.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.pipe.it.single; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.property.ThriftClientProperty; +import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.commons.pipe.sink.client.IoTDBSyncClient; +import org.apache.iotdb.it.env.EnvFactory; +import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; +import org.apache.iotdb.it.framework.IoTDBTestRunner; +import org.apache.iotdb.itbase.category.LocalStandaloneIT; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.TSCloseSessionReq; +import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq; +import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp; +import org.apache.iotdb.service.rpc.thrift.TSProtocolVersion; +import org.apache.iotdb.service.rpc.thrift.TSyncIdentityInfo; +import org.apache.iotdb.service.rpc.thrift.TSyncTransportMetaInfo; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.File; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.ZoneId; + +@RunWith(IoTDBTestRunner.class) +@Category({LocalStandaloneIT.class}) +public class IoTDBLegacyPipeReceiverSecurityIT { + + @BeforeClass + public static void setUp() { + EnvFactory.getEnv().initClusterEnvironment(); + } + + @AfterClass + public static void tearDown() { + EnvFactory.getEnv().cleanClusterEnvironment(); + } + + @Test + public void testRejectPathTraversalFileNameInLegacyTransportFile() throws Exception { + final DataNodeWrapper dataNode = EnvFactory.getEnv().getDataNodeWrapper(0); + + try (final IoTDBSyncClient client = + new IoTDBSyncClient( + new ThriftClientProperty.Builder().build(), + dataNode.getIp(), + dataNode.getPort(), + false, + null, + null)) { + final TSOpenSessionResp openSessionResp = client.openSession(createOpenSessionReq()); + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), openSessionResp.getStatus().getCode()); + + try { + final TSStatus handshakeStatus = + client.handshake( + new TSyncIdentityInfo( + "pathTraversalPipe", System.currentTimeMillis(), "UNKNOWN", "")); + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), handshakeStatus.getCode()); + + final String maliciousFileName = + ".." + File.separator + ".." + File.separator + "pwned.tsfile"; + final TSStatus status = + client.sendFile( + new TSyncTransportMetaInfo(maliciousFileName, 0), + ByteBuffer.wrap("pwned".getBytes(StandardCharsets.UTF_8))); + + Assert.assertEquals(TSStatusCode.SYNC_FILE_ERROR.getStatusCode(), status.getCode()); + Assert.assertTrue(status.getMessage().contains("Illegal fileName")); + } finally { + client.closeSession(new TSCloseSessionReq(openSessionResp.getSessionId())); + } + } + } + + private TSOpenSessionReq createOpenSessionReq() { + final TSOpenSessionReq req = new TSOpenSessionReq(); + req.setClient_protocol(TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V3); + req.setUsername("root"); + req.setPassword("root"); + req.setZoneId(ZoneId.systemDefault().toString()); + req.putToConfiguration("version", IoTDBConstant.ClientVersion.V_1_0.toString()); + req.putToConfiguration("sql_dialect", "tree"); + return req; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/IoTDBLegacyPipeReceiverAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/IoTDBLegacyPipeReceiverAgent.java index ae7ab22308a09..cc85e2f4f100e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/IoTDBLegacyPipeReceiverAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/IoTDBLegacyPipeReceiverAgent.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.pipe.receiver.PipeReceiverFilePathUtils; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -51,6 +52,7 @@ import java.io.IOException; import java.io.RandomAccessFile; import java.nio.ByteBuffer; +import java.nio.file.Paths; import java.time.ZoneId; import java.util.Map; import java.util.Objects; @@ -249,10 +251,12 @@ private SyncIdentityInfo getCurrentSyncIdentityInfo() { * @param tsFilePipeData pipeData * @param fileDir path of file data dir */ - private void handleTsFilePipeData(TsFilePipeData tsFilePipeData, String fileDir) { - String tsFileName = tsFilePipeData.getTsFileName(); - File dir = new File(fileDir); - File[] targetFiles = + private void handleTsFilePipeData(final TsFilePipeData tsFilePipeData, final String fileDir) + throws IOException { + final String tsFileName = tsFilePipeData.getTsFileName(); + final File tsFile = resolveFileInFileDataDir(fileDir, tsFileName); + final File dir = tsFile.getParentFile(); + final File[] targetFiles = dir.listFiles((dir1, name) -> name.startsWith(tsFileName) && name.endsWith(PATCH_SUFFIX)); if (targetFiles != null) { for (File targetFile : targetFiles) { @@ -289,13 +293,21 @@ public TSStatus transportFile(TSyncTransportMetaInfo metaInfo, ByteBuffer buff) LOGGER.debug( "Invoke transportData method from client ip = {}", identityInfo.getRemoteAddress()); - String fileDir = getFileDataDir(identityInfo); - String fileName = metaInfo.fileName; - long startIndex = metaInfo.startIndex; - File file = new File(fileDir, fileName + PATCH_SUFFIX); + final String fileDir = getFileDataDir(identityInfo); + final String fileName = metaInfo.fileName; + final long startIndex = metaInfo.startIndex; + final File file; + final File fileWithoutPatch; + try { + fileWithoutPatch = resolveFileInFileDataDir(fileDir, fileName); + file = resolveFileInFileDataDir(fileDir, fileName + PATCH_SUFFIX); + } catch (final IOException e) { + LOGGER.warn(e.getMessage()); + return RpcUtils.getStatus(TSStatusCode.SYNC_FILE_ERROR, e.getMessage()); + } // step2. check startIndex - IndexCheckResult result = checkStartIndexValid(new File(fileDir, fileName), startIndex); + final IndexCheckResult result = checkStartIndexValid(fileWithoutPatch, startIndex); if (!result.isResult()) { return RpcUtils.getStatus(TSStatusCode.SYNC_FILE_REDIRECTION_ERROR, result.getIndex()); } @@ -307,9 +319,9 @@ public TSStatus transportFile(TSyncTransportMetaInfo metaInfo, ByteBuffer buff) byte[] byteArray = new byte[length]; buff.get(byteArray); randomAccessFile.write(byteArray); - recordStartIndex(new File(fileDir, fileName), startIndex + length); + recordStartIndex(fileWithoutPatch, startIndex + length); LOGGER.debug("Sync {} start at {} to {} is done.", fileName, startIndex, startIndex + length); - } catch (IOException e) { + } catch (final IOException e) { LOGGER.error(e.getMessage()); return RpcUtils.getStatus(TSStatusCode.SYNC_FILE_ERROR, e.getMessage()); } @@ -317,7 +329,21 @@ public TSStatus transportFile(TSyncTransportMetaInfo metaInfo, ByteBuffer buff) return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS, ""); } - private IndexCheckResult checkStartIndexValid(File file, long startIndex) { + private static File resolveFileInFileDataDir(final String fileDir, final String fileName) + throws IOException { + if (StringUtils.isEmpty(fileName)) { + throw new IOException("Illegal fileName: " + fileName); + } + + final String illegalError = FileUtils.getIllegalError4Directory(fileName); + if (Objects.nonNull(illegalError)) { + throw new IOException("Illegal fileName: " + fileName + ", " + illegalError); + } + + return PipeReceiverFilePathUtils.resolveFilePath(Paths.get(fileDir), fileName).toFile(); + } + + private IndexCheckResult checkStartIndexValid(final File file, final long startIndex) { // get local index from memory map long localIndex = getCurrentFileStartIndex(file.getAbsolutePath()); // get local index from file diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/legacy/IoTDBLegacyPipeSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/legacy/IoTDBLegacyPipeSink.java index 4328c758d3934..0a090f99b8df6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/legacy/IoTDBLegacyPipeSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/legacy/IoTDBLegacyPipeSink.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.client.property.ThriftClientProperty; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.consensus.DataRegionId; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; import org.apache.iotdb.commons.pipe.config.PipeConfig; @@ -50,6 +51,9 @@ import org.apache.iotdb.rpc.IoTDBConnectionException; import org.apache.iotdb.rpc.StatementExecutionException; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.TSOpenSessionReq; +import org.apache.iotdb.service.rpc.thrift.TSOpenSessionResp; +import org.apache.iotdb.service.rpc.thrift.TSProtocolVersion; import org.apache.iotdb.service.rpc.thrift.TSyncIdentityInfo; import org.apache.iotdb.service.rpc.thrift.TSyncTransportMetaInfo; import org.apache.iotdb.session.pool.SessionPool; @@ -64,6 +68,7 @@ import java.io.IOException; import java.io.RandomAccessFile; import java.nio.ByteBuffer; +import java.time.ZoneId; import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -224,6 +229,7 @@ public void handshake() throws Exception { useSSL, trustStore, trustStorePwd); + openClientSession(); final TSyncIdentityInfo identityInfo = new TSyncIdentityInfo( pipeName, System.currentTimeMillis(), syncConnectorVersion, databaseName); @@ -254,6 +260,26 @@ public void handshake() throws Exception { .build(); } + private void openClientSession() throws TException { + final TSOpenSessionReq openSessionReq = new TSOpenSessionReq(); + openSessionReq.setClient_protocol(TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V3); + openSessionReq.setUsername(user); + openSessionReq.setPassword(password); + openSessionReq.setZoneId(ZoneId.systemDefault().toString()); + openSessionReq.putToConfiguration("version", IoTDBConstant.ClientVersion.V_1_0.toString()); + openSessionReq.putToConfiguration("sql_dialect", "tree"); + + final TSOpenSessionResp openSessionResp = client.openSession(openSessionReq); + if (openSessionResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + final String errorMsg = + String.format( + "Failed to login to receiver %s:%s for legacy pipe transfer because %s", + ipAddress, port, openSessionResp.getStatus().getMessage()); + LOGGER.warn(errorMsg); + throw new PipeRuntimeCriticalException(errorMsg); + } + } + @Override public void heartbeat() throws Exception { // do nothing diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java index f8bd094b69ee4..3d34def409f64 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/ClientRPCServiceImpl.java @@ -25,6 +25,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TShowConfigurationResp; import org.apache.iotdb.common.rpc.thrift.TShowConfigurationTemplateResp; +import org.apache.iotdb.commons.auth.entity.PrivilegeType; import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; @@ -2746,24 +2747,59 @@ public TSStatus createTimeseriesUsingSchemaTemplate(TCreateTimeseriesUsingSchema @Override public TSStatus handshake(final TSyncIdentityInfo info) throws TException { - return PipeDataNodeAgent.receiver() - .legacy() - .handshake( - info, - SESSION_MANAGER.getCurrSession().getClientAddress(), - partitionFetcher, - schemaFetcher); + try { + final TSStatus status = checkLegacyPipeReceiverPermission(); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return status; + } + return PipeDataNodeAgent.receiver() + .legacy() + .handshake( + info, + SESSION_MANAGER.getCurrSession().getClientAddress(), + partitionFetcher, + schemaFetcher); + } finally { + SESSION_MANAGER.updateIdleTime(); + } } @Override public TSStatus sendPipeData(final ByteBuffer buff) throws TException { - return PipeDataNodeAgent.receiver().legacy().transportPipeData(buff); + try { + final TSStatus status = checkLegacyPipeReceiverPermission(); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return status; + } + return PipeDataNodeAgent.receiver().legacy().transportPipeData(buff); + } finally { + SESSION_MANAGER.updateIdleTime(); + } } @Override public TSStatus sendFile(final TSyncTransportMetaInfo metaInfo, final ByteBuffer buff) throws TException { - return PipeDataNodeAgent.receiver().legacy().transportFile(metaInfo, buff); + try { + final TSStatus status = checkLegacyPipeReceiverPermission(); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return status; + } + return PipeDataNodeAgent.receiver().legacy().transportFile(metaInfo, buff); + } finally { + SESSION_MANAGER.updateIdleTime(); + } + } + + private TSStatus checkLegacyPipeReceiverPermission() { + final IClientSession clientSession = SESSION_MANAGER.getCurrSessionAndUpdateIdleTime(); + if (!SESSION_MANAGER.checkLogin(clientSession)) { + return getNotLoggedInStatus(); + } + return AuthorityChecker.getTSStatus( + AuthorityChecker.checkSystemPermission( + clientSession.getUsername(), PrivilegeType.USE_PIPE.ordinal()), + PrivilegeType.USE_PIPE); } @Override diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/IoTDBLegacyPipeReceiverAgentTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/IoTDBLegacyPipeReceiverAgentTest.java new file mode 100644 index 0000000000000..eaf21aef1fde8 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/legacy/IoTDBLegacyPipeReceiverAgentTest.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.receiver.protocol.legacy; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.db.pipe.sink.payload.legacy.TsFilePipeData; +import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.iotdb.service.rpc.thrift.TSyncIdentityInfo; +import org.apache.iotdb.service.rpc.thrift.TSyncTransportMetaInfo; + +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; + +public class IoTDBLegacyPipeReceiverAgentTest { + + private static final String PIPE_NAME = "poc"; + private static final long CREATE_TIME = 1700000000000L; + private static final String REMOTE_ADDRESS = "127.0.0.1"; + + private String originalSyncDir; + private Path syncDir; + private IoTDBLegacyPipeReceiverAgent agent; + + @Before + public void setUp() throws Exception { + originalSyncDir = CommonDescriptor.getInstance().getConfig().getSyncDir(); + syncDir = Files.createTempDirectory("legacy-pipe-receiver"); + CommonDescriptor.getInstance().getConfig().setSyncDir(syncDir.toString()); + + agent = new IoTDBLegacyPipeReceiverAgent(); + final TSStatus status = + agent.handshake( + new TSyncIdentityInfo(PIPE_NAME, CREATE_TIME, "UNKNOWN", ""), + REMOTE_ADDRESS, + null, + null); + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + } + + @After + public void tearDown() throws Exception { + if (agent != null) { + agent.handleClientExit(); + } + CommonDescriptor.getInstance().getConfig().setSyncDir(originalSyncDir); + if (syncDir != null) { + FileUtils.deleteDirectory(syncDir.toFile()); + } + } + + @Test + public void testTransportFileRejectsPathTraversal() throws Exception { + final String traversal = + ".." + File.separator + ".." + File.separator + ".." + File.separator + "pwned"; + + final TSStatus status = + agent.transportFile( + new TSyncTransportMetaInfo(traversal, 0), + ByteBuffer.wrap("pwned".getBytes(StandardCharsets.UTF_8))); + + Assert.assertEquals(TSStatusCode.SYNC_FILE_ERROR.getStatusCode(), status.getCode()); + Assert.assertTrue(status.getMessage().contains("Illegal fileName")); + Assert.assertFalse(Files.exists(syncDir.resolve("pwned.patch"))); + } + + @Test + public void testTransportFileWritesPlainFileUnderFileDataDir() throws Exception { + final String fileName = "1-2-3-4.tsfile"; + final byte[] payload = "iotdb".getBytes(StandardCharsets.UTF_8); + + final TSStatus status = + agent.transportFile(new TSyncTransportMetaInfo(fileName, 0), ByteBuffer.wrap(payload)); + + Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); + final Path patchFile = getFileDataDir().resolve(fileName + ".patch"); + Assert.assertArrayEquals(payload, Files.readAllBytes(patchFile)); + } + + @Test + public void testTransportPipeDataRejectsPathTraversalTsFileName() throws Exception { + final String traversal = ".." + File.separator + "evil.tsfile"; + + final TSStatus status = + agent.transportPipeData(ByteBuffer.wrap(new TsFilePipeData("", traversal, -1).serialize())); + + Assert.assertEquals(TSStatusCode.PIPESERVER_ERROR.getStatusCode(), status.getCode()); + Assert.assertTrue(status.getMessage().contains("Illegal fileName")); + } + + private Path getFileDataDir() { + return syncDir + .resolve("receiver") + .resolve(String.format("%s-%d-%s", PIPE_NAME, CREATE_TIME, REMOTE_ADDRESS)) + .resolve("file-data"); + } +} From 3c959c399e25b92c4e54050a367b6c171b8322dc Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 2 Jun 2026 15:09:17 +0800 Subject: [PATCH 071/102] Excluded system & audit from COUNT TIMESERIES and included views (#17703) (#17792) (cherry picked from commit 312d3841a9715f3ad24394ff38bb5ceccfb5183a) --- .../db/it/schema/IoTDBMetadataFetchIT.java | 28 ++++ .../regionscan/IoTDBActiveSchemaQueryIT.java | 34 +++++ .../queryengine/common/TimeseriesContext.java | 106 +++++++++++++- .../schema/CountGroupByLevelScanOperator.java | 15 +- .../operator/schema/SchemaCountOperator.java | 4 + .../operator/schema/source/ISchemaSource.java | 13 ++ .../schema/source/SchemaSourceFactory.java | 4 +- .../schema/source/TimeSeriesSchemaSource.java | 20 +++ .../ActiveTimeSeriesRegionScanOperator.java | 15 +- .../plan/analyze/AnalyzeVisitor.java | 136 ++++++++++++++--- .../plan/planner/OperatorTreeGenerator.java | 33 ++++- .../schema/SchemaCountOperatorTest.java | 113 +++++++++++++++ .../source/TimeSeriesSchemaSourceTest.java | 137 ++++++++++++++++++ 13 files changed, 626 insertions(+), 32 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSourceTest.java diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java index 2f1d4e5bed406..4d0c873664d1f 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/IoTDBMetadataFetchIT.java @@ -26,6 +26,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runners.Parameterized; @@ -494,6 +495,26 @@ public void showCountTimeSeries() throws SQLException { } } + @Test + @Ignore + public void showCountTimeSeriesExcludeInternalDatabaseAndIncludeView() throws SQLException { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + final long baseVisibleCount = queryCount(statement, "COUNT TIMESERIES root.ln*.**"); + statement.execute("CREATE DATABASE root.count_it"); + statement.execute( + "CREATE TIMESERIES root.count_it.src.s1 WITH DATATYPE = INT32, ENCODING = PLAIN"); + statement.execute( + "CREATE TIMESERIES root.count_it.src.s2 WITH DATATYPE = INT32, ENCODING = PLAIN"); + statement.execute("CREATE VIEW root.count_it.dst.v1 AS SELECT s1 FROM root.count_it.src;"); + + final long localCount = queryCount(statement, "COUNT TIMESERIES root.count_it.**"); + assertEquals(3L, localCount); + assertEquals( + baseVisibleCount + localCount, queryCount(statement, "COUNT TIMESERIES root.**")); + } + } + @Test public void showCountTimeSeriesWithTag() throws SQLException { try (Connection connection = EnvFactory.getEnv().getConnection(); @@ -865,4 +886,11 @@ public void showDeadbandInfo() throws SQLException { } } } + + private long queryCount(final Statement statement, final String sql) throws SQLException { + try (ResultSet resultSet = statement.executeQuery(sql)) { + Assert.assertTrue(resultSet.next()); + return resultSet.getLong(1); + } + } } diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/regionscan/IoTDBActiveSchemaQueryIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/regionscan/IoTDBActiveSchemaQueryIT.java index 853ef3c87ae00..98713b94767bb 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/regionscan/IoTDBActiveSchemaQueryIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/regionscan/IoTDBActiveSchemaQueryIT.java @@ -27,6 +27,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runners.Parameterized; @@ -235,6 +236,39 @@ public void testShowTimeSeries() { } } + @Test + @Ignore + public void testCountTimeSeriesWithTimeConditionIncludesView() { + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + statement.execute("CREATE DATABASE root.view_count"); + statement.execute( + "CREATE TIMESERIES root.view_count.src.s1 WITH DATATYPE = INT32, ENCODING = PLAIN"); + statement.execute( + "CREATE TIMESERIES root.view_count.src.s2 WITH DATATYPE = INT32, ENCODING = PLAIN"); + statement.execute("CREATE VIEW root.view_count.dst.v1 AS SELECT s1 FROM root.view_count.src"); + + checkResultSet( + statement, + "count timeseries root.view_count.**", + new HashSet<>(Collections.singletonList("3,"))); + + statement.execute("insert into root.view_count.src(timestamp,s1) values(1,1)"); + + checkResultSet( + statement, + "count timeseries root.view_count.** where time>0", + new HashSet<>(Collections.singletonList("2,"))); + checkResultSet( + statement, + "count timeseries root.view_count.dst.** where time>0", + new HashSet<>(Collections.singletonList("1,"))); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } + } + @Test public void testShowDevices() { try (Connection connection = EnvFactory.getEnv().getConnection(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/TimeseriesContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/TimeseriesContext.java index 9225b97fc70ae..aaf8b51787db5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/TimeseriesContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/TimeseriesContext.java @@ -28,7 +28,10 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashSet; import java.util.Objects; +import java.util.Set; import static org.apache.iotdb.db.queryengine.execution.operator.schema.source.TimeSeriesSchemaSource.mapToString; @@ -42,8 +45,17 @@ public class TimeseriesContext { private final String deadband; private final String deadbandParameters; + private final int activeCountMultiplier; + private final Set activeLogicalViewCountSet; public TimeseriesContext(IMeasurementSchemaInfo schemaInfo) { + this(schemaInfo, 1, Collections.emptySet()); + } + + public TimeseriesContext( + IMeasurementSchemaInfo schemaInfo, + int activeCountMultiplier, + Set activeLogicalViewCountSet) { this.dataType = schemaInfo.getSchema().getType().toString(); this.encoding = schemaInfo.getSchema().getEncodingType().toString(); this.compression = schemaInfo.getSchema().getCompressor().toString(); @@ -54,6 +66,8 @@ public TimeseriesContext(IMeasurementSchemaInfo schemaInfo) { MetaUtils.parseDeadbandInfo(schemaInfo.getSchema().getProps()); this.deadband = deadbandInfo.left; this.deadbandParameters = deadbandInfo.right; + this.activeCountMultiplier = activeCountMultiplier; + this.activeLogicalViewCountSet = new HashSet<>(activeLogicalViewCountSet); } public String getDataType() { @@ -88,6 +102,14 @@ public String getDeadband() { return deadband; } + public int getActiveCountMultiplier() { + return activeCountMultiplier; + } + + public Set getActiveLogicalViewCountSet() { + return activeLogicalViewCountSet; + } + public TimeseriesContext( String dataType, String alias, @@ -97,6 +119,30 @@ public TimeseriesContext( String attributes, String deadband, String deadbandParameters) { + this( + dataType, + alias, + encoding, + compression, + tags, + attributes, + deadband, + deadbandParameters, + 1, + Collections.emptySet()); + } + + public TimeseriesContext( + String dataType, + String alias, + String encoding, + String compression, + String tags, + String attributes, + String deadband, + String deadbandParameters, + int activeCountMultiplier, + Set activeLogicalViewCountSet) { this.dataType = dataType; this.alias = alias; this.encoding = encoding; @@ -105,6 +151,24 @@ public TimeseriesContext( this.attributes = attributes; this.deadband = deadband; this.deadbandParameters = deadbandParameters; + this.activeCountMultiplier = activeCountMultiplier; + this.activeLogicalViewCountSet = new HashSet<>(activeLogicalViewCountSet); + } + + public TimeseriesContext mergeActiveCount(TimeseriesContext that) { + Set mergedActiveLogicalViewCountSet = new HashSet<>(activeLogicalViewCountSet); + mergedActiveLogicalViewCountSet.addAll(that.activeLogicalViewCountSet); + return new TimeseriesContext( + dataType, + alias, + encoding, + compression, + tags, + attributes, + deadband, + deadbandParameters, + activeCountMultiplier + that.activeCountMultiplier, + mergedActiveLogicalViewCountSet); } public void serializeAttributes(ByteBuffer byteBuffer) { @@ -116,6 +180,11 @@ public void serializeAttributes(ByteBuffer byteBuffer) { ReadWriteIOUtils.write(attributes, byteBuffer); ReadWriteIOUtils.write(deadband, byteBuffer); ReadWriteIOUtils.write(deadbandParameters, byteBuffer); + ReadWriteIOUtils.write(activeCountMultiplier, byteBuffer); + ReadWriteIOUtils.write(activeLogicalViewCountSet.size(), byteBuffer); + for (String logicalView : activeLogicalViewCountSet) { + ReadWriteIOUtils.write(logicalView, byteBuffer); + } } public void serializeAttributes(DataOutputStream stream) throws IOException { @@ -127,6 +196,11 @@ public void serializeAttributes(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(attributes, stream); ReadWriteIOUtils.write(deadband, stream); ReadWriteIOUtils.write(deadbandParameters, stream); + ReadWriteIOUtils.write(activeCountMultiplier, stream); + ReadWriteIOUtils.write(activeLogicalViewCountSet.size(), stream); + for (String logicalView : activeLogicalViewCountSet) { + ReadWriteIOUtils.write(logicalView, stream); + } } public static TimeseriesContext deserialize(ByteBuffer buffer) { @@ -138,8 +212,23 @@ public static TimeseriesContext deserialize(ByteBuffer buffer) { String attributes = ReadWriteIOUtils.readString(buffer); String deadband = ReadWriteIOUtils.readString(buffer); String deadbandParameters = ReadWriteIOUtils.readString(buffer); + int activeCountMultiplier = ReadWriteIOUtils.readInt(buffer); + int activeLogicalViewCountSetSize = ReadWriteIOUtils.readInt(buffer); + Set activeLogicalViewCountSet = new HashSet<>(); + for (int i = 0; i < activeLogicalViewCountSetSize; i++) { + activeLogicalViewCountSet.add(ReadWriteIOUtils.readString(buffer)); + } return new TimeseriesContext( - dataType, alias, encoding, compression, tags, attributes, deadband, deadbandParameters); + dataType, + alias, + encoding, + compression, + tags, + attributes, + deadband, + deadbandParameters, + activeCountMultiplier, + activeLogicalViewCountSet); } @Override @@ -159,13 +248,24 @@ public boolean equals(Object obj) { && Objects.equals(tags, that.tags) && Objects.equals(attributes, that.attributes) && Objects.equals(deadband, that.deadband) - && Objects.equals(deadbandParameters, that.deadbandParameters); + && Objects.equals(deadbandParameters, that.deadbandParameters) + && activeCountMultiplier == that.activeCountMultiplier + && Objects.equals(activeLogicalViewCountSet, that.activeLogicalViewCountSet); return res; } @Override public int hashCode() { return Objects.hash( - dataType, alias, encoding, compression, tags, attributes, deadband, deadbandParameters); + dataType, + alias, + encoding, + compression, + tags, + attributes, + deadband, + deadbandParameters, + activeCountMultiplier, + activeLogicalViewCountSet); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/CountGroupByLevelScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/CountGroupByLevelScanOperator.java index 084935f68b02a..8d46f25327c45 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/CountGroupByLevelScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/CountGroupByLevelScanOperator.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.schema.source.ISchemaSource; import org.apache.iotdb.db.queryengine.execution.operator.source.SourceOperator; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion; import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ISchemaInfo; import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.reader.ISchemaReader; @@ -95,6 +96,10 @@ public OperatorContext getOperatorContext() { return operatorContext; } + private ISchemaRegion getSchemaRegion() { + return ((SchemaDriverContext) operatorContext.getDriverContext()).getSchemaRegion(); + } + @Override public ListenableFuture isBlocked() { if (isBlocked == null) { @@ -109,6 +114,11 @@ public ListenableFuture isBlocked() { */ private ListenableFuture tryGetNext() { if (schemaReader == null) { + if (schemaSource.shouldSkipSchemaRegion(getSchemaRegion())) { + next = null; + isFinished = true; + return NOT_BLOCKED; + } schemaReader = createTimeSeriesReader(); } while (true) { @@ -172,15 +182,14 @@ public TsBlock next() throws Exception { @Override public boolean hasNext() throws Exception { isBlocked().get(); // wait for the next TsBlock - if (!schemaReader.isSuccess()) { + if (schemaReader != null && !schemaReader.isSuccess()) { throw new SchemaExecutionException(schemaReader.getFailure()); } return next != null; } public ISchemaReader createTimeSeriesReader() { - return schemaSource.getSchemaReader( - ((SchemaDriverContext) operatorContext.getDriverContext()).getSchemaRegion()); + return schemaSource.getSchemaReader(getSchemaRegion()); } private TsBlock constructTsBlockAndClearMap(Map countMap) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaCountOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaCountOperator.java index 91c7ebda26b01..9e7e935dd330c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaCountOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaCountOperator.java @@ -93,6 +93,10 @@ public ListenableFuture isBlocked() { */ private ListenableFuture tryGetNext() { ISchemaRegion schemaRegion = getSchemaRegion(); + if (schemaSource.shouldSkipSchemaRegion(schemaRegion)) { + next = constructTsBlock(0); + return NOT_BLOCKED; + } if (schemaSource.hasSchemaStatistic(schemaRegion)) { next = constructTsBlock(schemaSource.getSchemaStatistic(schemaRegion)); return NOT_BLOCKED; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/ISchemaSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/ISchemaSource.java index 2a80e1071e535..573c7e35f062a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/ISchemaSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/ISchemaSource.java @@ -24,6 +24,7 @@ import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ISchemaInfo; import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.reader.ISchemaReader; +import org.apache.tsfile.common.conf.TSFileDescriptor; import org.apache.tsfile.read.common.block.TsBlockBuilder; import java.util.List; @@ -52,4 +53,16 @@ public interface ISchemaSource { boolean hasSchemaStatistic(ISchemaRegion schemaRegion); long getSchemaStatistic(ISchemaRegion schemaRegion); + + default boolean shouldSkipSchemaRegion(ISchemaRegion schemaRegion) { + return false; + } + + default boolean checkRegionDatabaseIncluded(ISchemaRegion schemaRegion) { + return true; + } + + default long getMaxMemory(ISchemaRegion schemaRegion) { + return TSFileDescriptor.getInstance().getConfig().getMaxTsBlockSizeInBytes(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java index 7fe1084257957..8a0e4abf3f7fe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java @@ -44,7 +44,7 @@ public static ISchemaSource getTimeSeriesSchemaCountSourc Map templateMap, PathPatternTree scope) { return new TimeSeriesSchemaSource( - pathPattern, isPrefixMatch, 0, 0, schemaFilter, templateMap, false, scope); + pathPattern, isPrefixMatch, 0, 0, schemaFilter, templateMap, false, true, scope); } // show time series @@ -57,7 +57,7 @@ public static ISchemaSource getTimeSeriesSchemaScanSource Map templateMap, PathPatternTree scope) { return new TimeSeriesSchemaSource( - pathPattern, isPrefixMatch, limit, offset, schemaFilter, templateMap, true, scope); + pathPattern, isPrefixMatch, limit, offset, schemaFilter, templateMap, true, false, scope); } // count device diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java index 47b73ccb0e5cd..cc47a9361b6fd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java @@ -54,6 +54,7 @@ public class TimeSeriesSchemaSource implements ISchemaSource templateMap; private final boolean needViewDetail; + private final boolean excludeInternalDatabase; TimeSeriesSchemaSource( PartialPath pathPattern, @@ -63,6 +64,7 @@ public class TimeSeriesSchemaSource implements ISchemaSource templateMap, boolean needViewDetail, + boolean excludeInternalDatabase, PathPatternTree scope) { this.pathPattern = pathPattern; this.isPrefixMatch = isPrefixMatch; @@ -71,6 +73,7 @@ public class TimeSeriesSchemaSource implements ISchemaSource map) { if (map == null || map.isEmpty()) { return null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ActiveTimeSeriesRegionScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ActiveTimeSeriesRegionScanOperator.java index a9cbea64ddbe3..264bea0dd0e0b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ActiveTimeSeriesRegionScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ActiveTimeSeriesRegionScanOperator.java @@ -36,13 +36,16 @@ import org.apache.tsfile.utils.RamUsageEstimator; import java.io.IOException; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; public class ActiveTimeSeriesRegionScanOperator extends AbstractRegionScanDataSourceOperator { // Timeseries which need to be checked. private final Map> timeSeriesToSchemasInfo; + private final Set countedLogicalViews; private static final Binary VIEW_TYPE = new Binary("BASE".getBytes()); private final Binary dataBaseName; private static final long INSTANCE_SIZE = @@ -61,6 +64,7 @@ public ActiveTimeSeriesRegionScanOperator( this.operatorContext = operatorContext; this.sourceId = sourceId; this.timeSeriesToSchemasInfo = timeSeriesToSchemasInfo; + this.countedLogicalViews = new HashSet<>(); this.regionScanUtil = new RegionScanForActiveTimeSeriesUtil(timeFilter, ttlCache); this.dataBaseName = new Binary( @@ -102,7 +106,16 @@ protected void updateActiveData() { if (outputCount) { for (Map.Entry> entry : activeTimeSeries.entrySet()) { List timeSeriesList = entry.getValue(); - count += timeSeriesList.size(); + Map timeSeriesInfo = timeSeriesToSchemasInfo.get(entry.getKey()); + for (String timeSeries : timeSeriesList) { + TimeseriesContext schemaInfo = timeSeriesInfo.get(timeSeries); + count += schemaInfo.getActiveCountMultiplier(); + for (String logicalView : schemaInfo.getActiveLogicalViewCountSet()) { + if (countedLogicalViews.add(logicalView)) { + count++; + } + } + } removeTimeseriesListFromDevice(entry.getKey(), timeSeriesList); } return; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index a28ca79c9567b..b4d056c1ff7d1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -3112,7 +3112,8 @@ private boolean analyzeTimeseriesRegionScan( PathPatternTree patternTree, Analysis analysis, MPPQueryContext context, - PathPatternTree authorityScope) + PathPatternTree authorityScope, + boolean includeLogicalView) throws IllegalPathException { analyzeGlobalTimeConditionInShowMetaData(timeCondition, analysis); context.generateGlobalTimeFilter(analysis); @@ -3124,7 +3125,15 @@ private boolean analyzeTimeseriesRegionScan( analysis.setFinishQueryAfterAnalyze(true); return false; } - removeLogicViewMeasurement(schemaTree); + List deviceSchemaInfoList; + if (includeLogicalView) { + deviceSchemaInfoList = schemaTree.getMatchedDevices(ALL_MATCH_PATTERN); + updateSchemaTreeByViews(analysis, schemaTree, context); + } else { + removeLogicViewMeasurement(schemaTree); + deviceSchemaInfoList = schemaTree.getMatchedDevices(ALL_MATCH_PATTERN); + } + Map>> deviceToTimeseriesContext = new HashMap<>(); /** @@ -3132,38 +3141,56 @@ private boolean analyzeTimeseriesRegionScan( * as a normal node, not a device+templateId. This means that all nodes are what we need.). We * can use ALL_MATCH_PATTERN to get result. */ - List deviceSchemaInfoList = schemaTree.getMatchedDevices(ALL_MATCH_PATTERN); Set deviceSet = new HashSet<>(); for (DeviceSchemaInfo deviceSchemaInfo : deviceSchemaInfoList) { boolean isAligned = deviceSchemaInfo.isAligned(); PartialPath devicePath = deviceSchemaInfo.getDevicePath(); - deviceSet.add(devicePath.getFullPath()); if (isAligned) { List measurementList = new ArrayList<>(); List schemaList = new ArrayList<>(); List timeseriesContextList = new ArrayList<>(); for (IMeasurementSchemaInfo measurementSchemaInfo : deviceSchemaInfo.getMeasurementSchemaInfoList()) { + if (includeLogicalView && measurementSchemaInfo.isLogicalView()) { + addLogicalViewSourcesForActiveCount( + devicePath, + measurementSchemaInfo, + schemaTree, + deviceToTimeseriesContext, + deviceSet); + continue; + } schemaList.add(measurementSchemaInfo.getSchema()); measurementList.add(measurementSchemaInfo.getName()); timeseriesContextList.add(new TimeseriesContext(measurementSchemaInfo)); } - AlignedPath alignedPath = - new AlignedPath(devicePath.getNodes(), measurementList, schemaList); - deviceToTimeseriesContext - .computeIfAbsent(devicePath, k -> new HashMap<>()) - .put(alignedPath, timeseriesContextList); + if (!measurementList.isEmpty()) { + deviceSet.add(devicePath.getFullPath()); + AlignedPath alignedPath = + new AlignedPath(devicePath.getNodes(), measurementList, schemaList); + deviceToTimeseriesContext + .computeIfAbsent(devicePath, k -> new HashMap<>()) + .put(alignedPath, timeseriesContextList); + } } else { for (IMeasurementSchemaInfo measurementSchemaInfo : deviceSchemaInfo.getMeasurementSchemaInfoList()) { - MeasurementPath measurementPath = - new MeasurementPath( - devicePath.concatNode(measurementSchemaInfo.getName()).getNodes()); - deviceToTimeseriesContext - .computeIfAbsent(devicePath, k -> new HashMap<>()) - .put( - measurementPath, - Collections.singletonList(new TimeseriesContext(measurementSchemaInfo))); + if (includeLogicalView && measurementSchemaInfo.isLogicalView()) { + addLogicalViewSourcesForActiveCount( + devicePath, + measurementSchemaInfo, + schemaTree, + deviceToTimeseriesContext, + deviceSet); + } else { + addPhysicalTimeseriesForActiveCount( + devicePath, + measurementSchemaInfo, + false, + new TimeseriesContext(measurementSchemaInfo), + deviceToTimeseriesContext, + deviceSet); + } } } } @@ -3175,6 +3202,75 @@ private boolean analyzeTimeseriesRegionScan( return true; } + private void addLogicalViewSourcesForActiveCount( + PartialPath viewDevicePath, + IMeasurementSchemaInfo viewSchemaInfo, + ISchemaTree schemaTree, + Map>> deviceToTimeseriesContext, + Set deviceSet) { + LogicalViewSchema logicalViewSchema = viewSchemaInfo.getSchemaAsLogicalViewSchema(); + if (logicalViewSchema == null) { + return; + } + + String viewPath = viewDevicePath.concatNode(viewSchemaInfo.getName()).getFullPath(); + for (PartialPath sourcePath : getSourcePaths(logicalViewSchema.getExpression())) { + if (sourcePath.getNodeLength() <= 1) { + continue; + } + PartialPath sourceDevicePath = + new PartialPath(Arrays.copyOf(sourcePath.getNodes(), sourcePath.getNodeLength() - 1)); + DeviceSchemaInfo sourceDeviceSchemaInfo = + schemaTree.searchDeviceSchemaInfo( + sourceDevicePath, Collections.singletonList(sourcePath.getMeasurement())); + if (sourceDeviceSchemaInfo == null + || sourceDeviceSchemaInfo.getMeasurementSchemaInfoList().isEmpty()) { + continue; + } + + IMeasurementSchemaInfo sourceSchemaInfo = + sourceDeviceSchemaInfo.getMeasurementSchemaInfoList().get(0); + if (sourceSchemaInfo == null || sourceSchemaInfo.isLogicalView()) { + continue; + } + + addPhysicalTimeseriesForActiveCount( + sourceDevicePath, + sourceSchemaInfo, + sourceDeviceSchemaInfo.isAligned(), + new TimeseriesContext(sourceSchemaInfo, 0, Collections.singleton(viewPath)), + deviceToTimeseriesContext, + deviceSet); + } + } + + private void addPhysicalTimeseriesForActiveCount( + PartialPath devicePath, + IMeasurementSchemaInfo measurementSchemaInfo, + boolean isAligned, + TimeseriesContext timeseriesContext, + Map>> deviceToTimeseriesContext, + Set deviceSet) { + deviceSet.add(devicePath.getFullPath()); + PartialPath timeseriesPath = + isAligned + ? new AlignedPath( + devicePath.getNodes(), + Collections.singletonList(measurementSchemaInfo.getName()), + Collections.singletonList(measurementSchemaInfo.getSchema())) + : new MeasurementPath( + devicePath.concatNode(measurementSchemaInfo.getName()).getNodes()); + Map> timeseriesContextMap = + deviceToTimeseriesContext.computeIfAbsent(devicePath, k -> new HashMap<>()); + List existingContextList = timeseriesContextMap.get(timeseriesPath); + if (existingContextList == null) { + timeseriesContextMap.put( + timeseriesPath, new ArrayList<>(Collections.singletonList(timeseriesContext))); + } else { + existingContextList.set(0, existingContextList.get(0).mergeActiveCount(timeseriesContext)); + } + } + @Override public Analysis visitShowTimeSeries( ShowTimeSeriesStatement showTimeSeriesStatement, MPPQueryContext context) { @@ -3193,7 +3289,8 @@ public Analysis visitShowTimeSeries( patternTree, analysis, context, - showTimeSeriesStatement.getAuthorityScope()); + showTimeSeriesStatement.getAuthorityScope(), + false); if (!hasSchema) { analysis.setRespDatasetHeader(DatasetHeaderFactory.getShowTimeSeriesHeader()); return analysis; @@ -3437,7 +3534,8 @@ public Analysis visitCountTimeSeries( patternTree, analysis, context, - countTimeSeriesStatement.getAuthorityScope()); + countTimeSeriesStatement.getAuthorityScope(), + true); if (!hasSchema) { analysis.setRespDatasetHeader(DatasetHeaderFactory.getCountTimeSeriesHeader()); return analysis; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java index 0649f808c4ffd..c93912c27ebb4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java @@ -3781,9 +3781,17 @@ private static Map getTimeseriesSchemaInfoMap( Map timeseriesSchemaInfoMap = new HashMap<>(); for (Map.Entry> entry : entryMap.getValue().entrySet()) { PartialPath path = entry.getKey(); - context.addPath(path); if (path instanceof MeasurementPath) { - timeseriesSchemaInfoMap.put(path.getMeasurement(), entry.getValue().get(0)); + String measurement = path.getMeasurement(); + TimeseriesContext timeseriesContext = entry.getValue().get(0); + TimeseriesContext existingContext = timeseriesSchemaInfoMap.get(measurement); + if (existingContext == null) { + timeseriesSchemaInfoMap.put(measurement, timeseriesContext); + context.addPath(path); + } else { + timeseriesSchemaInfoMap.put( + measurement, existingContext.mergeActiveCount(timeseriesContext)); + } } else if (path instanceof AlignedPath) { AlignedPath alignedPath = (AlignedPath) path; List measurementList = alignedPath.getMeasurementList(); @@ -3791,8 +3799,25 @@ private static Map getTimeseriesSchemaInfoMap( throw new IllegalArgumentException( "The size of measurementList and timeseriesSchemaInfoList should be equal in aligned path."); } - for (int i = 0; i < measurementList.size(); i++) { - timeseriesSchemaInfoMap.put(measurementList.get(i), entry.getValue().get(i)); + int size = measurementList.size(); + List schemaList = new ArrayList<>(size); + List newMeasurementList = new ArrayList<>(size); + List alignedSchemaList = alignedPath.getSchemaList(); + for (int i = 0; i < size; i++) { + String measurement = measurementList.get(i); + TimeseriesContext timeseriesContext = entry.getValue().get(i); + TimeseriesContext existingContext = timeseriesSchemaInfoMap.get(measurement); + if (existingContext == null) { + timeseriesSchemaInfoMap.put(measurement, timeseriesContext); + newMeasurementList.add(measurement); + schemaList.add(alignedSchemaList.get(i)); + } else { + timeseriesSchemaInfoMap.put( + measurement, existingContext.mergeActiveCount(timeseriesContext)); + } + } + if (!newMeasurementList.isEmpty()) { + context.addPath(new AlignedPath(path.getNodes(), newMeasurementList, schemaList)); } } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaCountOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaCountOperatorTest.java index 3bdd80a3c1b53..c5e8335ebf566 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaCountOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/SchemaCountOperatorTest.java @@ -114,6 +114,82 @@ public void testSchemaCountOperator() throws Exception { } } + @Test + public void testSchemaCountOperatorSkipSchemaRegion() throws Exception { + ExecutorService instanceNotificationExecutor = + IoTDBThreadPoolFactory.newFixedThreadPool(1, "test-instance-notification"); + try { + QueryId queryId = new QueryId("stub_query"); + FragmentInstanceId instanceId = + new FragmentInstanceId(new PlanFragmentId(queryId, 0), "stub-instance"); + FragmentInstanceStateMachine stateMachine = + new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor); + FragmentInstanceContext fragmentInstanceContext = + createFragmentInstanceContext(instanceId, stateMachine); + DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0); + PlanNodeId planNodeId = queryId.genPlanNodeId(); + ISchemaRegion schemaRegion = Mockito.mock(ISchemaRegion.class); + OperatorContext operatorContext = + driverContext.addOperatorContext( + 1, planNodeId, SchemaCountOperator.class.getSimpleName()); + operatorContext.setDriverContext( + new SchemaDriverContext(fragmentInstanceContext, schemaRegion, 0)); + ISchemaSource schemaSource = Mockito.mock(ISchemaSource.class); + Mockito.when(schemaSource.shouldSkipSchemaRegion(schemaRegion)).thenReturn(true); + + SchemaCountOperator schemaCountOperator = + new SchemaCountOperator<>( + planNodeId, driverContext.getOperatorContexts().get(0), schemaSource); + + assertTrue(schemaCountOperator.hasNext()); + TsBlock tsBlock = schemaCountOperator.next(); + assertEquals(0, tsBlock.getColumn(0).getLong(0)); + assertTrue(schemaCountOperator.isFinished()); + Mockito.verify(schemaSource, Mockito.never()).getSchemaReader(schemaRegion); + } finally { + instanceNotificationExecutor.shutdown(); + } + } + + @Test + public void testSchemaCountOperatorUseSchemaStatistic() throws Exception { + ExecutorService instanceNotificationExecutor = + IoTDBThreadPoolFactory.newFixedThreadPool(1, "test-instance-notification"); + try { + QueryId queryId = new QueryId("stub_query"); + FragmentInstanceId instanceId = + new FragmentInstanceId(new PlanFragmentId(queryId, 0), "stub-instance"); + FragmentInstanceStateMachine stateMachine = + new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor); + FragmentInstanceContext fragmentInstanceContext = + createFragmentInstanceContext(instanceId, stateMachine); + DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0); + PlanNodeId planNodeId = queryId.genPlanNodeId(); + ISchemaRegion schemaRegion = Mockito.mock(ISchemaRegion.class); + OperatorContext operatorContext = + driverContext.addOperatorContext( + 1, planNodeId, SchemaCountOperator.class.getSimpleName()); + operatorContext.setDriverContext( + new SchemaDriverContext(fragmentInstanceContext, schemaRegion, 0)); + ISchemaSource schemaSource = Mockito.mock(ISchemaSource.class); + Mockito.when(schemaSource.hasSchemaStatistic(schemaRegion)).thenReturn(true); + Mockito.when(schemaSource.getSchemaStatistic(schemaRegion)).thenReturn(7L); + Mockito.when(schemaSource.checkRegionDatabaseIncluded(schemaRegion)).thenReturn(true); + + SchemaCountOperator schemaCountOperator = + new SchemaCountOperator<>( + planNodeId, driverContext.getOperatorContexts().get(0), schemaSource); + + assertTrue(schemaCountOperator.hasNext()); + TsBlock tsBlock = schemaCountOperator.next(); + assertEquals(7, tsBlock.getColumn(0).getLong(0)); + Mockito.verify(schemaSource).getSchemaStatistic(schemaRegion); + Mockito.verify(schemaSource, Mockito.never()).getSchemaReader(schemaRegion); + } finally { + instanceNotificationExecutor.shutdown(); + } + } + @Test public void testLevelTimeSeriesCountOperator() { ExecutorService instanceNotificationExecutor = @@ -185,6 +261,43 @@ public void testLevelTimeSeriesCountOperator() { } } + @Test + public void testLevelTimeSeriesCountOperatorSkipSchemaRegion() { + ExecutorService instanceNotificationExecutor = + IoTDBThreadPoolFactory.newFixedThreadPool(1, "test-instance-notification"); + try { + QueryId queryId = new QueryId("stub_query"); + FragmentInstanceId instanceId = + new FragmentInstanceId(new PlanFragmentId(queryId, 0), "stub-instance"); + FragmentInstanceStateMachine stateMachine = + new FragmentInstanceStateMachine(instanceId, instanceNotificationExecutor); + FragmentInstanceContext fragmentInstanceContext = + createFragmentInstanceContext(instanceId, stateMachine); + DriverContext driverContext = new DriverContext(fragmentInstanceContext, 0); + PlanNodeId planNodeId = queryId.genPlanNodeId(); + OperatorContext operatorContext = + driverContext.addOperatorContext( + 1, planNodeId, CountGroupByLevelScanOperator.class.getSimpleName()); + ISchemaRegion schemaRegion = Mockito.mock(ISchemaRegion.class); + operatorContext.setDriverContext( + new SchemaDriverContext(fragmentInstanceContext, schemaRegion, 0)); + ISchemaSource schemaSource = Mockito.mock(ISchemaSource.class); + Mockito.when(schemaSource.shouldSkipSchemaRegion(schemaRegion)).thenReturn(true); + + CountGroupByLevelScanOperator timeSeriesCountOperator = + new CountGroupByLevelScanOperator<>( + planNodeId, driverContext.getOperatorContexts().get(0), 1, schemaSource); + + assertTrue(collectResult(timeSeriesCountOperator).isEmpty()); + Mockito.verify(schemaSource, Mockito.never()).getSchemaReader(schemaRegion); + } catch (Exception e) { + e.printStackTrace(); + fail(); + } finally { + instanceNotificationExecutor.shutdown(); + } + } + private List collectResult(CountGroupByLevelScanOperator operator) throws Exception { List tsBlocks = new ArrayList<>(); while (operator.hasNext()) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSourceTest.java new file mode 100644 index 0000000000000..b7becf649c970 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSourceTest.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.execution.operator.schema.source; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.schema.SchemaConstant; +import org.apache.iotdb.db.schemaengine.rescon.ISchemaRegionStatistics; +import org.apache.iotdb.db.schemaengine.schemaregion.ISchemaRegion; +import org.apache.iotdb.db.schemaengine.schemaregion.read.resp.info.ITimeSeriesSchemaInfo; + +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TimeSeriesSchemaSourceTest { + + @Test + public void testCountSourceSkipsImplicitInternalDatabases() throws Exception { + final ISchemaSource countSource = + SchemaSourceFactory.getTimeSeriesSchemaCountSource( + new PartialPath("root.**"), + false, + null, + Collections.emptyMap(), + SchemaConstant.ALL_MATCH_SCOPE); + + assertTrue( + countSource.shouldSkipSchemaRegion(mockSchemaRegion(SchemaConstant.SYSTEM_DATABASE))); + assertFalse(countSource.shouldSkipSchemaRegion(mockSchemaRegion("root.sg"))); + } + + @Test + public void testCountSourceKeepsExplicitInternalDatabaseQueries() throws Exception { + final ISchemaSource systemCountSource = + SchemaSourceFactory.getTimeSeriesSchemaCountSource( + new PartialPath("root.__system.**"), + false, + null, + Collections.emptyMap(), + SchemaConstant.ALL_MATCH_SCOPE); + assertFalse( + systemCountSource.shouldSkipSchemaRegion(mockSchemaRegion(SchemaConstant.SYSTEM_DATABASE))); + } + + @Test + public void testCountSourceSkipsWildcardSecondNodeForInternalDatabases() throws Exception { + final ISchemaSource countSource = + SchemaSourceFactory.getTimeSeriesSchemaCountSource( + new PartialPath("root.*.**"), + false, + null, + Collections.emptyMap(), + SchemaConstant.ALL_MATCH_SCOPE); + + assertTrue( + countSource.shouldSkipSchemaRegion(mockSchemaRegion(SchemaConstant.SYSTEM_DATABASE))); + assertFalse(countSource.shouldSkipSchemaRegion(mockSchemaRegion("root.sg"))); + } + + @Test + public void testCountSourceKeepsExactInternalDatabaseQueries() throws Exception { + final ISchemaSource systemCountSource = + SchemaSourceFactory.getTimeSeriesSchemaCountSource( + new PartialPath("root.__system"), + false, + null, + Collections.emptyMap(), + SchemaConstant.ALL_MATCH_SCOPE); + assertFalse( + systemCountSource.shouldSkipSchemaRegion(mockSchemaRegion(SchemaConstant.SYSTEM_DATABASE))); + } + + @Test + public void testShowSourceDoesNotSkipInternalDatabases() throws Exception { + final ISchemaSource showSource = + SchemaSourceFactory.getTimeSeriesSchemaScanSource( + new PartialPath("root.**"), + false, + 0, + 0, + null, + Collections.emptyMap(), + SchemaConstant.ALL_MATCH_SCOPE); + + assertFalse( + showSource.shouldSkipSchemaRegion(mockSchemaRegion(SchemaConstant.SYSTEM_DATABASE))); + } + + @Test + public void testCountStatisticIncludesView() throws Exception { + final ISchemaSource countSource = + SchemaSourceFactory.getTimeSeriesSchemaCountSource( + new PartialPath("root.sg.**"), + false, + null, + Collections.emptyMap(), + SchemaConstant.ALL_MATCH_SCOPE); + final ISchemaRegion schemaRegion = mockSchemaRegion("root.sg"); + final ISchemaRegionStatistics schemaRegionStatistics = + Mockito.mock(ISchemaRegionStatistics.class); + + Mockito.when(schemaRegion.getSchemaRegionStatistics()).thenReturn(schemaRegionStatistics); + Mockito.when(schemaRegionStatistics.getSeriesNumber(true)).thenReturn(5L); + + assertEquals(5L, countSource.getSchemaStatistic(schemaRegion)); + Mockito.verify(schemaRegionStatistics).getSeriesNumber(true); + Mockito.verify(schemaRegionStatistics, Mockito.never()).getSeriesNumber(false); + } + + private ISchemaRegion mockSchemaRegion(final String database) { + final ISchemaRegion schemaRegion = Mockito.mock(ISchemaRegion.class); + Mockito.when(schemaRegion.getDatabaseFullPath()).thenReturn(database); + return schemaRegion; + } +} From 13faa155d32fe4e27fab86edd8b673942dadf026 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 2 Jun 2026 18:00:42 +0800 Subject: [PATCH 072/102] Added flags to mark whether a device has device descendants to optimize query like select xx from xxx.** (#17672) (#17802) * flag- * Fix * Address device descendant flag review comments (cherry picked from commit a565a06825317e5f39393e619fb44e3fbbb88a9b) --- .../impl/mem/MTreeBelowSGMemoryImpl.java | 130 +++++++-- .../mtree/impl/mem/mnode/IMemMNode.java | 30 ++- .../impl/mem/mnode/basic/BasicMNode.java | 34 ++- .../impl/pbtree/MTreeBelowSGCachedImpl.java | 83 +++++- .../mtree/impl/pbtree/mnode/ICachedMNode.java | 19 ++ .../pbtree/mnode/basic/CachedBasicMNode.java | 40 ++- .../mtree/traverser/Traverser.java | 102 +++++++ .../traverser/basic/MeasurementTraverser.java | 5 + .../schemaRegion/SchemaRegionBasicTest.java | 22 ++ .../impl/mem/MTreeBelowSGMemoryImplTest.java | 253 ++++++++++++++++++ .../pbtree/MTreeBelowSGCachedImplTest.java | 217 +++++++++++++++ .../common/AbstractAboveDatabaseMNode.java | 4 + .../schema/node/utils/IMNodeIterator.java | 2 +- .../schema/tree/AbstractTreeVisitor.java | 4 + 14 files changed, 908 insertions(+), 37 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImplTest.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImplTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java index b6f1221f7013f..a82250e229986 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImpl.java @@ -174,6 +174,59 @@ public synchronized boolean createSnapshot(File snapshotDir) { return store.createSnapshot(snapshotDir); } + private void applySubtreeMeasurementDelta(IMemMNode startNode, final long delta) { + if (delta == 0 || startNode == null) { + return; + } + IMemMNode current = startNode; + while (current != null) { + current.setSubtreeMeasurementCount(current.getSubtreeMeasurementCount() + delta); + current = current.getParent(); + } + } + + private IDeviceMNode setToEntityAndUpdateFlags(final IMemMNode node) { + final boolean wasDevice = node.isDevice(); + final IDeviceMNode deviceMNode = store.setToEntity(node); + if (!wasDevice) { + markAncestorsHavingDeviceDescendant(node); + } + return deviceMNode; + } + + private void markAncestorsHavingDeviceDescendant(final IMemMNode deviceNode) { + IMemMNode current = deviceNode.getParent(); + while (current != null && !current.hasDeviceDescendant()) { + current.setHasDeviceDescendant(true); + current = current.getParent(); + } + } + + private boolean hasDeviceDescendantInChildren(final IMemMNode node) { + try (final IMNodeIterator iterator = store.getChildrenIterator(node)) { + while (iterator.hasNext()) { + final IMemMNode child = iterator.next(); + if (child.isDevice() || child.hasDeviceDescendant()) { + return true; + } + } + return false; + } + } + + private void refreshAncestorsHavingDeviceDescendant(IMemMNode startNode) { + IMemMNode current = startNode; + while (current != null) { + current.setHasDeviceDescendant(hasDeviceDescendantInChildren(current)); + current = current.getParent(); + } + } + + private long getTemplateMeasurementCount(final int templateId) { + final Template template = ClusterTemplateManager.getInstance().getTemplate(templateId); + return template == null ? 0L : template.getMeasurementNumber(); + } + public static MTreeBelowSGMemoryImpl loadFromSnapshot( File snapshotDir, String storageGroupFullPath, @@ -184,13 +237,16 @@ public static MTreeBelowSGMemoryImpl loadFromSnapshot( Function, Map> tagGetter, Function, Map> attributeGetter) throws IOException, IllegalPathException { - return new MTreeBelowSGMemoryImpl( - new PartialPath(storageGroupFullPath), - MemMTreeStore.loadFromSnapshot( - snapshotDir, measurementProcess, deviceProcess, regionStatistics, metric), - tagGetter, - attributeGetter, - regionStatistics); + final MTreeBelowSGMemoryImpl mtree = + new MTreeBelowSGMemoryImpl( + new PartialPath(storageGroupFullPath), + MemMTreeStore.loadFromSnapshot( + snapshotDir, measurementProcess, deviceProcess, regionStatistics, metric), + tagGetter, + attributeGetter, + regionStatistics); + mtree.rebuildSubtreeMeasurementCount(); + return mtree; } // endregion @@ -268,7 +324,7 @@ public IMeasurementMNode createTimeSeries( if (device.isDevice()) { entityMNode = device.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(device); + entityMNode = setToEntityAndUpdateFlags(device); } // create a non-aligned time series @@ -290,6 +346,7 @@ public IMeasurementMNode createTimeSeries( entityMNode.addAlias(alias, measurementMNode); } + applySubtreeMeasurementDelta(measurementMNode.getAsMNode(), 1L); return measurementMNode; } } @@ -360,7 +417,7 @@ public List> createAlignedTimeSeries( if (device.isDevice()) { entityMNode = device.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(device); + entityMNode = setToEntityAndUpdateFlags(device); entityMNode.setAligned(true); } @@ -386,6 +443,7 @@ public List> createAlignedTimeSeries( if (aliasList != null && aliasList.get(i) != null) { entityMNode.addAlias(aliasList.get(i), measurementMNode); } + applySubtreeMeasurementDelta(measurementMNode.getAsMNode(), 1L); measurementMNodeList.add(measurementMNode); } return measurementMNodeList; @@ -539,6 +597,7 @@ public IMeasurementMNode deleteTimeseries(PartialPath path) throws Me if (deletedNode.getAlias() != null) { parent.getAsDeviceMNode().deleteAliasChild(deletedNode.getAlias()); } + applySubtreeMeasurementDelta(parent, -1L); } deleteEmptyInternalMNode(parent.getAsDeviceMNode()); return deletedNode; @@ -551,14 +610,15 @@ public void deleteEmptyInternalMNode(IDeviceMNode entityMNode) { boolean hasMeasurement = false; boolean hasNonViewMeasurement = false; IMemMNode child; - IMNodeIterator iterator = store.getChildrenIterator(curNode); - while (iterator.hasNext()) { - child = iterator.next(); - if (child.isMeasurement()) { - hasMeasurement = true; - if (!child.getAsMeasurementMNode().isLogicalView()) { - hasNonViewMeasurement = true; - break; + try (final IMNodeIterator iterator = store.getChildrenIterator(curNode)) { + while (iterator.hasNext()) { + child = iterator.next(); + if (child.isMeasurement()) { + hasMeasurement = true; + if (!child.getAsMeasurementMNode().isLogicalView()) { + hasNonViewMeasurement = true; + break; + } } } } @@ -567,6 +627,7 @@ public void deleteEmptyInternalMNode(IDeviceMNode entityMNode) { synchronized (this) { curNode = store.setToInternal(entityMNode); } + refreshAncestorsHavingDeviceDescendant(curNode.getParent()); } else if (!hasNonViewMeasurement) { // has some measurement but they are all logical view entityMNode.setAligned(null); @@ -889,7 +950,7 @@ public void activateTemplate(PartialPath activatePath, Template template) if (cur.isDevice()) { entityMNode = cur.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(cur); + entityMNode = setToEntityAndUpdateFlags(cur); } } @@ -907,6 +968,7 @@ public void activateTemplate(PartialPath activatePath, Template template) entityMNode.setUseTemplate(true); entityMNode.setSchemaTemplateId(template.getId()); regionStatistics.activateTemplate(template.getId()); + applySubtreeMeasurementDelta(entityMNode.getAsMNode(), (long) template.getMeasurementNumber()); } public Map> constructSchemaBlackListWithTemplate( @@ -968,6 +1030,8 @@ protected void updateEntity(IDeviceMNode node) { resultTemplateSetInfo.put( node.getPartialPath(), Collections.singletonList(node.getSchemaTemplateId())); regionStatistics.deactivateTemplate(node.getSchemaTemplateId()); + applySubtreeMeasurementDelta( + node.getAsMNode(), -getTemplateMeasurementCount(node.getSchemaTemplateId())); node.deactivateTemplate(); deleteEmptyInternalMNode(node); } @@ -991,7 +1055,7 @@ public void activateTemplateWithoutCheck( if (cur.isDevice()) { entityMNode = cur.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(cur); + entityMNode = setToEntityAndUpdateFlags(cur); } if (!entityMNode.isAligned()) { @@ -1000,6 +1064,7 @@ public void activateTemplateWithoutCheck( entityMNode.setUseTemplate(true); entityMNode.setSchemaTemplateId(templateId); regionStatistics.activateTemplate(templateId); + applySubtreeMeasurementDelta(entityMNode.getAsMNode(), getTemplateMeasurementCount(templateId)); } public long countPathsUsingTemplate(PartialPath pathPattern, int templateId) @@ -1011,6 +1076,30 @@ public long countPathsUsingTemplate(PartialPath pathPattern, int templateId) } } + public void rebuildSubtreeMeasurementCount() { + rebuildSubtreeMeasurementCountFromNode(rootNode); + } + + private long rebuildSubtreeMeasurementCountFromNode(final IMemMNode node) { + long count = node.isMeasurement() ? 1L : 0L; + boolean hasDeviceDescendant = false; + try (final IMNodeIterator iterator = store.getChildrenIterator(node)) { + while (iterator.hasNext()) { + final IMemMNode child = iterator.next(); + count += rebuildSubtreeMeasurementCountFromNode(child); + if (child.isDevice() || child.hasDeviceDescendant()) { + hasDeviceDescendant = true; + } + } + } + if (node.isDevice() && node.getAsDeviceMNode().isUseTemplate()) { + count += getTemplateMeasurementCount(node.getAsDeviceMNode().getSchemaTemplateId()); + } + node.setSubtreeMeasurementCount(count); + node.setHasDeviceDescendant(hasDeviceDescendant); + return count; + } + // endregion // region Interfaces for schema reader @@ -1278,7 +1367,7 @@ public IMeasurementMNode createLogicalView( if (device.isDevice()) { entityMNode = device.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(device); + entityMNode = setToEntityAndUpdateFlags(device); // this parent has no measurement before. The leafName is his first child who is a logical // view. entityMNode.setAligned(null); @@ -1287,6 +1376,7 @@ public IMeasurementMNode createLogicalView( measurementMNode.setParent(entityMNode.getAsMNode()); store.addChild(entityMNode.getAsMNode(), leafName, measurementMNode.getAsMNode()); + applySubtreeMeasurementDelta(measurementMNode.getAsMNode(), 1L); return measurementMNode; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/IMemMNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/IMemMNode.java index d3d055928b14a..257d7caeef8ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/IMemMNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/IMemMNode.java @@ -19,5 +19,33 @@ package org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.mem.mnode; import org.apache.iotdb.commons.schema.node.IMNode; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.mem.mnode.basic.BasicMNode; -public interface IMemMNode extends IMNode {} +public interface IMemMNode extends IMNode { + + BasicMNode getBasicMNode(); + + /** + * The count of measurement nodes contained in the subtree rooted at this node. The counter is + * maintained in memory only. + */ + default long getSubtreeMeasurementCount() { + return getBasicMNode().getSubtreeMeasurementCount(); + } + + default void setSubtreeMeasurementCount(final long subtreeMeasurementCount) { + getBasicMNode().setSubtreeMeasurementCount(subtreeMeasurementCount); + } + + /** + * Whether there is any device node in the subtree rooted at this node, excluding the node itself. + * This flag is maintained in memory only. + */ + default boolean hasDeviceDescendant() { + return getBasicMNode().hasDeviceDescendant(); + } + + default void setHasDeviceDescendant(final boolean hasDeviceDescendant) { + getBasicMNode().setHasDeviceDescendant(hasDeviceDescendant); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/basic/BasicMNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/basic/BasicMNode.java index ccacef6c7ebe8..e70885c7d03e4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/basic/BasicMNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/mnode/basic/BasicMNode.java @@ -46,6 +46,12 @@ public class BasicMNode implements IMemMNode { private IMemMNode parent; private final BasicMNodeInfo basicMNodeInfo; + /** Cached count of measurements in this node's subtree, rebuilt on restart. */ + private long subtreeMeasurementCount = 0L; + + /** Cached flag showing whether there is any device in the subtree below this node. */ + private boolean hasDeviceDescendant = false; + /** from root to this node, only be set when used once for InternalMNode */ private String fullPath; @@ -55,6 +61,11 @@ public BasicMNode(IMemMNode parent, String name) { this.basicMNodeInfo = new BasicMNodeInfo(name); } + @Override + public BasicMNode getBasicMNode() { + return this; + } + @Override public String getName() { return basicMNodeInfo.getName(); @@ -98,6 +109,25 @@ public void setFullPath(String fullPath) { this.fullPath = fullPath; } + public long getSubtreeMeasurementCount() { + return subtreeMeasurementCount; + } + + @Override + public void setSubtreeMeasurementCount(final long subtreeMeasurementCount) { + this.subtreeMeasurementCount = subtreeMeasurementCount; + } + + @Override + public boolean hasDeviceDescendant() { + return hasDeviceDescendant; + } + + @Override + public void setHasDeviceDescendant(final boolean hasDeviceDescendant) { + this.hasDeviceDescendant = hasDeviceDescendant; + } + @Override public PartialPath getPartialPath() { List detachedPath = new ArrayList<>(); @@ -224,6 +254,8 @@ public R accept(MNodeVisitor visitor, C context) { *
  • basicMNodeInfo reference, 8B *
  • parent reference, 8B *
  • fullPath reference, 8B + *
  • subtreeMeasurementCount, 8B + *
  • hasDeviceDescendant, 1B * *
  • MapEntry in parent *
      @@ -235,7 +267,7 @@ public R accept(MNodeVisitor visitor, C context) { */ @Override public int estimateSize() { - return 8 + 8 + 8 + 8 + 8 + 8 + 28 + basicMNodeInfo.estimateSize(); + return 8 + 8 + 8 + 8 + 8 + 1 + 8 + 8 + 28 + basicMNodeInfo.estimateSize(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java index f79a8c2e05743..829d242187e61 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImpl.java @@ -132,6 +132,68 @@ public class MTreeBelowSGCachedImpl { private final int levelOfDB; private final CachedSchemaRegionStatistics regionStatistics; + private IDeviceMNode setToEntityAndUpdateFlags(final ICachedMNode node) + throws MetadataException { + final boolean wasDevice = node.isDevice(); + if (!node.isDeviceDescendantComputed()) { + node.setHasDeviceDescendant(hasDeviceDescendantInChildren(node)); + node.setDeviceDescendantComputed(true); + } + final IDeviceMNode deviceMNode = store.setToEntity(node); + if (!wasDevice) { + markAncestorsHavingDeviceDescendant(node); + } + return deviceMNode; + } + + private void markAncestorsHavingDeviceDescendant(final ICachedMNode deviceNode) { + ICachedMNode current = deviceNode.getParent(); + while (current != null && !current.hasDeviceDescendant()) { + current.setHasDeviceDescendant(true); + current.setDeviceDescendantComputed(true); + current = current.getParent(); + } + } + + private boolean hasDeviceDescendantInChildren(final ICachedMNode node) throws MetadataException { + try (final IMNodeIterator iterator = store.getChildrenIterator(node)) { + while (iterator.hasNext()) { + final ICachedMNode child = iterator.next(); + try { + if (child.isDevice() || hasDeviceDescendant(child)) { + return true; + } + } finally { + unPinMNode(child); + } + } + return false; + } + } + + private boolean hasDeviceDescendant(final ICachedMNode node) throws MetadataException { + if (node.isMeasurement()) { + node.setHasDeviceDescendant(false); + node.setDeviceDescendantComputed(true); + return false; + } + if (!node.isDeviceDescendantComputed()) { + node.setHasDeviceDescendant(hasDeviceDescendantInChildren(node)); + node.setDeviceDescendantComputed(true); + } + return node.hasDeviceDescendant(); + } + + private void refreshAncestorsHavingDeviceDescendant(ICachedMNode startNode) + throws MetadataException { + ICachedMNode current = startNode; + while (current != null) { + current.setHasDeviceDescendant(hasDeviceDescendantInChildren(current)); + current.setDeviceDescendantComputed(true); + current = current.getParent(); + } + } + // region MTree initialization, clear and serialization public MTreeBelowSGCachedImpl( PartialPath storageGroupPath, @@ -348,7 +410,7 @@ public IMeasurementMNode createTimeSeriesWithPinnedReturn( if (device.isDevice()) { entityMNode = device.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(device); + entityMNode = setToEntityAndUpdateFlags(device); device = entityMNode.getAsMNode(); } @@ -453,7 +515,7 @@ public List> createAlignedTimeSeries( if (device.isDevice()) { entityMNode = device.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(device); + entityMNode = setToEntityAndUpdateFlags(device); entityMNode.setAligned(true); device = entityMNode.getAsMNode(); } @@ -674,8 +736,7 @@ private void deleteAndUnpinEmptyInternalMNode(IDeviceMNode entityM boolean hasMeasurement = false; boolean hasNonViewMeasurement = false; ICachedMNode child; - IMNodeIterator iterator = store.getChildrenIterator(curNode); - try { + try (final IMNodeIterator iterator = store.getChildrenIterator(curNode)) { while (iterator.hasNext()) { child = iterator.next(); unPinMNode(child); @@ -687,12 +748,11 @@ private void deleteAndUnpinEmptyInternalMNode(IDeviceMNode entityM } } } - } finally { - iterator.close(); } if (!hasMeasurement) { curNode = store.setToInternal(entityMNode); + refreshAncestorsHavingDeviceDescendant(curNode.getParent()); } else if (!hasNonViewMeasurement) { // has some measurement but they are all logical view store.updateMNode(entityMNode.getAsMNode(), o -> o.getAsDeviceMNode().setAligned(null)); @@ -719,14 +779,11 @@ private void deleteAndUnpinEmptyInternalMNode(IDeviceMNode entityM } private boolean isEmptyInternalMNode(ICachedMNode node) throws MetadataException { - IMNodeIterator iterator = store.getChildrenIterator(node); - try { + try (final IMNodeIterator iterator = store.getChildrenIterator(node)) { return !IoTDBConstant.PATH_ROOT.equals(node.getName()) && !node.isMeasurement() && !(node.isDevice() && node.getAsDeviceMNode().isUseTemplate()) && !iterator.hasNext(); - } finally { - iterator.close(); } } @@ -1064,7 +1121,7 @@ public void createLogicalView(PartialPath path, ViewExpression viewExpression) if (device.isDevice()) { entityMNode = device.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(device); + entityMNode = setToEntityAndUpdateFlags(device); // this parent has no measurement before. The leafName is his first child who is a // logical // view. @@ -1186,7 +1243,7 @@ public void activateTemplate(PartialPath activatePath, Template template) if (cur.isDevice()) { entityMNode = cur.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(cur); + entityMNode = setToEntityAndUpdateFlags(cur); } if (entityMNode.isUseTemplate()) { @@ -1232,7 +1289,7 @@ public void activateTemplateWithoutCheck( if (cur.isDevice()) { entityMNode = cur.getAsDeviceMNode(); } else { - entityMNode = store.setToEntity(cur); + entityMNode = setToEntityAndUpdateFlags(cur); } store.updateMNode( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/ICachedMNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/ICachedMNode.java index 489b1a642aad9..244e077013c57 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/ICachedMNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/ICachedMNode.java @@ -22,8 +22,27 @@ import org.apache.iotdb.commons.schema.node.IMNode; import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.pbtree.lock.LockEntry; import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.pbtree.memory.cache.CacheEntry; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.pbtree.mnode.basic.CachedBasicMNode; public interface ICachedMNode extends IMNode { + CachedBasicMNode getBasicMNode(); + + default boolean hasDeviceDescendant() { + return getBasicMNode().hasDeviceDescendant(); + } + + default void setHasDeviceDescendant(final boolean hasDeviceDescendant) { + getBasicMNode().setHasDeviceDescendant(hasDeviceDescendant); + } + + default boolean isDeviceDescendantComputed() { + return getBasicMNode().isDeviceDescendantComputed(); + } + + default void setDeviceDescendantComputed(final boolean deviceDescendantComputed) { + getBasicMNode().setDeviceDescendantComputed(deviceDescendantComputed); + } + CacheEntry getCacheEntry(); void setCacheEntry(CacheEntry cacheEntry); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/basic/CachedBasicMNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/basic/CachedBasicMNode.java index d34fde474abee..14bb9f3609b78 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/basic/CachedBasicMNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/mnode/basic/CachedBasicMNode.java @@ -47,6 +47,17 @@ public class CachedBasicMNode implements ICachedMNode { private ICachedMNode parent; private final CacheMNodeInfo cacheMNodeInfo; + /** Cached flag showing whether there is any device in the subtree below this node. */ + private boolean hasDeviceDescendant = false; + + /** + * Whether {@link #hasDeviceDescendant} is trusted for the current in-memory node instance. + * + *

      This state is intentionally not persisted. A node reloaded from PBTree can lazily recompute + * it when the wildcard-suffix optimization needs it. + */ + private boolean deviceDescendantComputed = false; + /** from root to this node, only be set when used once for InternalMNode */ private String fullPath; @@ -56,6 +67,11 @@ public CachedBasicMNode(ICachedMNode parent, String name) { this.cacheMNodeInfo = new CacheMNodeInfo(name); } + @Override + public CachedBasicMNode getBasicMNode() { + return this; + } + @Override public String getName() { return cacheMNodeInfo.getName(); @@ -99,6 +115,26 @@ public void setFullPath(String fullPath) { this.fullPath = fullPath; } + @Override + public boolean hasDeviceDescendant() { + return hasDeviceDescendant; + } + + @Override + public void setHasDeviceDescendant(final boolean hasDeviceDescendant) { + this.hasDeviceDescendant = hasDeviceDescendant; + } + + @Override + public boolean isDeviceDescendantComputed() { + return deviceDescendantComputed; + } + + @Override + public void setDeviceDescendantComputed(final boolean deviceDescendantComputed) { + this.deviceDescendantComputed = deviceDescendantComputed; + } + @Override public PartialPath getPartialPath() { List detachedPath = new ArrayList<>(); @@ -245,6 +281,8 @@ public void setLockEntry(LockEntry lockEntry) { *

    1. basicMNodeInfo reference, 8B *
    2. parent reference, 8B *
    3. fullPath reference, 8B + *
    4. hasDeviceDescendant, 1B + *
    5. deviceDescendantComputed, 1B *
    *
  • MapEntry in parent *
      @@ -256,7 +294,7 @@ public void setLockEntry(LockEntry lockEntry) { */ @Override public int estimateSize() { - return 8 + 8 + 8 + 8 + 8 + 8 + 28 + cacheMNodeInfo.estimateSize(); + return 8 + 8 + 8 + 8 + 1 + 1 + 8 + 8 + 28 + cacheMNodeInfo.estimateSize(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java index f1ac33d99ed60..6c9e23e9597ed 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java @@ -25,15 +25,18 @@ import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.path.fa.IFAState; import org.apache.iotdb.commons.path.fa.IFATransition; +import org.apache.iotdb.commons.path.fa.match.IStateMatchInfo; import org.apache.iotdb.commons.schema.node.IMNode; import org.apache.iotdb.commons.schema.node.role.IDeviceMNode; import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory; import org.apache.iotdb.commons.schema.node.utils.IMNodeIterator; import org.apache.iotdb.commons.schema.tree.AbstractTreeVisitor; import org.apache.iotdb.db.schemaengine.schemaregion.mtree.IMTreeStore; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.mem.mnode.IMemMNode; import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.mem.mnode.iterator.MNodeIterator; import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.pbtree.ReentrantReadOnlyCachedMTreeStore; import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.pbtree.memory.ReleaseFlushMonitor; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.pbtree.mnode.ICachedMNode; import org.apache.iotdb.db.schemaengine.schemaregion.utils.MNodeUtils; import org.apache.iotdb.db.schemaengine.template.Template; @@ -41,9 +44,11 @@ import org.slf4j.LoggerFactory; import java.util.Iterator; +import java.util.LinkedHashSet; import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.Set; import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_ROOT; import static org.apache.iotdb.commons.schema.SchemaConstant.NON_TEMPLATE; @@ -248,10 +253,107 @@ protected Iterator getChildrenIterator(N parent) throws MetadataException { if (parent.isAboveDatabase()) { return new MNodeIterator<>(parent.getChildren().values().iterator()); } else { + final Iterator optimizedIterator = + shouldUseLeafDeviceMeasurementOptimization() + ? getOptimizedChildrenIterator(parent) + : null; + if (optimizedIterator != null) { + return optimizedIterator; + } return store.getTraverserIterator(parent, templateMap, skipPreDeletedSchema); } } + protected boolean shouldUseLeafDeviceMeasurementOptimization() { + return false; + } + + @SuppressWarnings("unchecked") + private Iterator getOptimizedChildrenIterator(final N parent) throws MetadataException { + if (!parent.isDevice()) { + return null; + } + + if (parent instanceof IMemMNode) { + if (((IMemMNode) parent).hasDeviceDescendant()) { + return null; + } + } else if (parent instanceof ICachedMNode) { + if (hasDeviceDescendant((ICachedMNode) parent)) { + return null; + } + } else { + return null; + } + + final IStateMatchInfo stateMatchInfo = getCurrentStateMatchInfo(); + final Set candidateNames = new LinkedHashSet<>(); + + for (int i = 0; i < stateMatchInfo.getMatchedStateSize(); i++) { + final IFAState matchedState = stateMatchInfo.getMatchedState(i); + + for (final IFATransition transition : + patternFA.getPreciseMatchTransition(matchedState).values()) { + if (patternFA.getNextState(matchedState, transition).isFinal()) { + candidateNames.add(transition.getAcceptEvent()); + } + } + + final Iterator fuzzyTransitionIterator = + patternFA.getFuzzyMatchTransitionIterator(matchedState); + while (fuzzyTransitionIterator.hasNext()) { + if (patternFA.getNextState(matchedState, fuzzyTransitionIterator.next()).isFinal()) { + return null; + } + } + } + + if (candidateNames.isEmpty()) { + return null; + } + + // For leaf devices, `**.measurement` does not need to enumerate every measurement child. + try { + return getChildrenIterator(parent, candidateNames.iterator()); + } catch (final MetadataException e) { + throw e; + } catch (final Exception e) { + throw new MetadataException(e.getMessage(), e); + } + } + + @SuppressWarnings("unchecked") + private boolean hasDeviceDescendant(final ICachedMNode node) throws MetadataException { + if (node.isMeasurement()) { + node.setHasDeviceDescendant(false); + node.setDeviceDescendantComputed(true); + return false; + } + if (!node.isDeviceDescendantComputed()) { + node.setHasDeviceDescendant(hasDeviceDescendantInChildren(node)); + node.setDeviceDescendantComputed(true); + } + return node.hasDeviceDescendant(); + } + + @SuppressWarnings("unchecked") + private boolean hasDeviceDescendantInChildren(final ICachedMNode node) throws MetadataException { + final IMTreeStore cachedStore = (IMTreeStore) store; + try (final IMNodeIterator iterator = cachedStore.getChildrenIterator(node)) { + while (iterator.hasNext()) { + final ICachedMNode child = iterator.next(); + try { + if (child.isDevice() || hasDeviceDescendant(child)) { + return true; + } + } finally { + cachedStore.unPin(child); + } + } + return false; + } + } + @Override protected void releaseNodeIterator(Iterator nodeIterator) { if (nodeIterator instanceof IMNodeIterator) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MeasurementTraverser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MeasurementTraverser.java index 4996e3f0c8b36..490145d5371fe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MeasurementTraverser.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/basic/MeasurementTraverser.java @@ -84,4 +84,9 @@ protected boolean shouldVisitSubtreeOfFullMatchedNode(N node) { protected boolean shouldVisitSubtreeOfInternalMatchedNode(N node) { return !node.isMeasurement(); } + + @Override + protected boolean shouldUseLeafDeviceMeasurementOptimization() { + return true; + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionBasicTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionBasicTest.java index 667ec4bc9b47b..913b5b151e8c1 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionBasicTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaRegionBasicTest.java @@ -1185,6 +1185,28 @@ public void testGetMatchedDevicesWithSpecialPattern() throws Exception { Assert.assertEquals(expectedPathList, actualPathList); } + @Test + public void testShowTimeseriesWildcardSuffixWithNestedAndLeafDevices() throws Exception { + final ISchemaRegion schemaRegion = getSchemaRegion("root.test", 0); + + SchemaRegionTestUtil.createSimpleTimeSeriesByList( + schemaRegion, + Arrays.asList( + "root.test.d1.s1", "root.test.d1.a.d2.s1", "root.test.d3.s1", "root.test.d4.s2")); + + final List result = + SchemaRegionTestUtil.showTimeseries(schemaRegion, new PartialPath("root.test.**.s1")); + final Set expectedPathList = + new HashSet<>(Arrays.asList("root.test.d1.s1", "root.test.d1.a.d2.s1", "root.test.d3.s1")); + Assert.assertEquals(expectedPathList.size(), result.size()); + + final Set actualPathList = new HashSet<>(); + for (final ITimeSeriesSchemaInfo timeSeriesSchemaInfo : result) { + actualPathList.add(timeSeriesSchemaInfo.getFullPath()); + } + Assert.assertEquals(expectedPathList, actualPathList); + } + @Test public void testGetMatchedDevicesWithSpecialPattern2() throws Exception { final ISchemaRegion schemaRegion = getSchemaRegion("root.test", 0); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImplTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImplTest.java new file mode 100644 index 0000000000000..1dfa4a056bbb1 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/mem/MTreeBelowSGMemoryImplTest.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.mem; + +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.schema.SchemaConstant; +import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode; +import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory; +import org.apache.iotdb.commons.schema.node.utils.IMNodeIterator; +import org.apache.iotdb.db.exception.metadata.PathNotExistException; +import org.apache.iotdb.db.schemaengine.metric.SchemaRegionMemMetric; +import org.apache.iotdb.db.schemaengine.rescon.MemSchemaEngineStatistics; +import org.apache.iotdb.db.schemaengine.rescon.MemSchemaRegionStatistics; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.mem.mnode.IMemMNode; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.loader.MNodeFactoryLoader; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.collector.MeasurementCollector; +import org.apache.iotdb.db.schemaengine.template.Template; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class MTreeBelowSGMemoryImplTest { + + @Test + public void testDeviceDescendantFlagIsMaintainedAcrossCreateDeleteAndRebuild() throws Exception { + final MTreeBelowSGMemoryImpl mtree = newMTree(); + + mtree.createTimeSeries( + new MeasurementPath("root.sg.a.b.s1"), + TSDataType.BOOLEAN, + TSEncoding.PLAIN, + CompressionType.SNAPPY, + null, + null, + false, + null); + + IMemMNode database = mtree.getNodeByPath(new PartialPath("root.sg")); + IMemMNode aNode = mtree.getNodeByPath(new PartialPath("root.sg.a")); + IMemMNode bNode = mtree.getNodeByPath(new PartialPath("root.sg.a.b")); + + Assert.assertTrue(database.hasDeviceDescendant()); + Assert.assertFalse(aNode.isDevice()); + Assert.assertTrue(aNode.hasDeviceDescendant()); + Assert.assertTrue(bNode.isDevice()); + Assert.assertFalse(bNode.hasDeviceDescendant()); + + mtree.createTimeSeries( + new MeasurementPath("root.sg.a.s0"), + TSDataType.BOOLEAN, + TSEncoding.PLAIN, + CompressionType.SNAPPY, + null, + null, + false, + null); + mtree.createTimeSeries( + new MeasurementPath("root.sg.c.d.s2"), + TSDataType.BOOLEAN, + TSEncoding.PLAIN, + CompressionType.SNAPPY, + null, + null, + false, + null); + + aNode = mtree.getNodeByPath(new PartialPath("root.sg.a")); + IMemMNode cNode = mtree.getNodeByPath(new PartialPath("root.sg.c")); + Assert.assertTrue(aNode.isDevice()); + Assert.assertTrue(aNode.hasDeviceDescendant()); + Assert.assertTrue(cNode.hasDeviceDescendant()); + + database.setHasDeviceDescendant(false); + aNode.setHasDeviceDescendant(false); + bNode.setHasDeviceDescendant(true); + cNode.setHasDeviceDescendant(false); + mtree.rebuildSubtreeMeasurementCount(); + + database = mtree.getNodeByPath(new PartialPath("root.sg")); + aNode = mtree.getNodeByPath(new PartialPath("root.sg.a")); + bNode = mtree.getNodeByPath(new PartialPath("root.sg.a.b")); + cNode = mtree.getNodeByPath(new PartialPath("root.sg.c")); + Assert.assertTrue(database.hasDeviceDescendant()); + Assert.assertTrue(aNode.hasDeviceDescendant()); + Assert.assertFalse(bNode.hasDeviceDescendant()); + Assert.assertTrue(cNode.hasDeviceDescendant()); + + mtree.deleteTimeseries(new PartialPath("root.sg.a.s0")); + aNode = mtree.getNodeByPath(new PartialPath("root.sg.a")); + Assert.assertFalse(aNode.isDevice()); + Assert.assertTrue(aNode.hasDeviceDescendant()); + + mtree.deleteTimeseries(new PartialPath("root.sg.a.b.s1")); + database = mtree.getNodeByPath(new PartialPath("root.sg")); + Assert.assertTrue(database.hasDeviceDescendant()); + assertPathNotExist(mtree, new PartialPath("root.sg.a")); + + mtree.deleteTimeseries(new PartialPath("root.sg.c.d.s2")); + database = mtree.getNodeByPath(new PartialPath("root.sg")); + Assert.assertFalse(database.hasDeviceDescendant()); + assertPathNotExist(mtree, new PartialPath("root.sg.c")); + } + + @Test + public void testLeafDeviceWildcardSuffixUsesDirectMeasurementLookup() throws Exception { + final PartialPath databasePath = new PartialPath("root.sg"); + final MemSchemaRegionStatistics regionStatistics = newRegionStatistics(); + final CountingMemMTreeStore store = + new CountingMemMTreeStore( + databasePath, regionStatistics, new SchemaRegionMemMetric(regionStatistics, "root.sg")); + final IMNodeFactory nodeFactory = + MNodeFactoryLoader.getInstance().getMemMNodeIMNodeFactory(); + + final IMemMNode databaseMNode = store.getRoot(); + final IMemMNode rootNode = store.generatePrefix(databasePath); + final IMemMNode deviceNode = + store.addChild(databaseMNode, "d1", nodeFactory.createInternalMNode(databaseMNode, "d1")); + store.setToEntity(deviceNode); + + for (int i = 0; i < 64; i++) { + final String measurement = i == 0 ? "target" : "s" + i; + final IMeasurementMNode measurementMNode = + nodeFactory.createMeasurementMNode( + deviceNode.getAsDeviceMNode(), + measurement, + new MeasurementSchema( + measurement, TSDataType.BOOLEAN, TSEncoding.PLAIN, CompressionType.SNAPPY), + null); + store.addChild(deviceNode, measurement, measurementMNode.getAsMNode()); + } + + store.watchParent(deviceNode); + final List matchedPaths = new ArrayList<>(); + try (final MeasurementCollector collector = + new MeasurementCollector( + rootNode, + new PartialPath("root.sg.**.target"), + store, + false, + SchemaConstant.ALL_MATCH_SCOPE) { + @Override + protected Void collectMeasurement(final IMeasurementMNode node) { + matchedPaths.add(getCurrentMeasurementPathInTraverse(node).getFullPath()); + return null; + } + }) { + collector.traverse(); + } + + Assert.assertEquals(Collections.singletonList("root.sg.d1.target"), matchedPaths); + Assert.assertEquals(0, store.getWatchedTraverserIteratorCount()); + Assert.assertEquals(1, store.getWatchedChildLookupCount()); + } + + private static MTreeBelowSGMemoryImpl newMTree() throws Exception { + final MemSchemaRegionStatistics regionStatistics = newRegionStatistics(); + return new MTreeBelowSGMemoryImpl( + new PartialPath("root.sg"), + node -> Collections.emptyMap(), + node -> Collections.emptyMap(), + regionStatistics, + new SchemaRegionMemMetric(regionStatistics, "root.sg")); + } + + private static MemSchemaRegionStatistics newRegionStatistics() { + return new MemSchemaRegionStatistics(0, new MemSchemaEngineStatistics()); + } + + private static void assertPathNotExist(final MTreeBelowSGMemoryImpl mtree, final PartialPath path) + throws MetadataException { + try { + mtree.getNodeByPath(path); + Assert.fail("Expected path not exist: " + path.getFullPath()); + } catch (final PathNotExistException ignored) { + // expected + } + } + + private static class CountingMemMTreeStore extends MemMTreeStore { + private IMemMNode watchedParent; + private int watchedTraverserIteratorCount; + private int watchedChildLookupCount; + + private CountingMemMTreeStore( + final PartialPath rootPath, + final MemSchemaRegionStatistics regionStatistics, + final SchemaRegionMemMetric metric) { + super(rootPath, regionStatistics, metric); + } + + private void watchParent(final IMemMNode parent) { + this.watchedParent = parent; + this.watchedTraverserIteratorCount = 0; + this.watchedChildLookupCount = 0; + } + + private int getWatchedTraverserIteratorCount() { + return watchedTraverserIteratorCount; + } + + private int getWatchedChildLookupCount() { + return watchedChildLookupCount; + } + + @Override + public IMemMNode getChild(final IMemMNode parent, final String name) { + if (parent == watchedParent) { + watchedChildLookupCount++; + } + return super.getChild(parent, name); + } + + @Override + public IMNodeIterator getTraverserIterator( + final IMemMNode parent, + final Map templateMap, + final boolean skipPreDeletedSchema) + throws MetadataException { + if (parent == watchedParent) { + watchedTraverserIteratorCount++; + } + return super.getTraverserIterator(parent, templateMap, skipPreDeletedSchema); + } + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImplTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImplTest.java new file mode 100644 index 0000000000000..65a6f31a75da7 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/MTreeBelowSGCachedImplTest.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.pbtree; + +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.schema.SchemaConstant; +import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.metadata.PathNotExistException; +import org.apache.iotdb.db.schemaengine.metric.SchemaRegionCachedMetric; +import org.apache.iotdb.db.schemaengine.rescon.CachedSchemaEngineStatistics; +import org.apache.iotdb.db.schemaengine.rescon.CachedSchemaRegionStatistics; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.pbtree.memory.ReleaseFlushMonitor; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.impl.pbtree.mnode.ICachedMNode; +import org.apache.iotdb.db.schemaengine.schemaregion.mtree.traverser.collector.MeasurementCollector; + +import org.apache.commons.io.FileUtils; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class MTreeBelowSGCachedImplTest { + + private static final Field STORE_FIELD; + private static final Field ROOT_NODE_FIELD; + + static { + try { + STORE_FIELD = MTreeBelowSGCachedImpl.class.getDeclaredField("store"); + STORE_FIELD.setAccessible(true); + ROOT_NODE_FIELD = MTreeBelowSGCachedImpl.class.getDeclaredField("rootNode"); + ROOT_NODE_FIELD.setAccessible(true); + } catch (final NoSuchFieldException e) { + throw new ExceptionInInitializerError(e); + } + } + + private final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + private int rawCachedMNodeSize; + private MTreeBelowSGCachedImpl mtree; + + @Before + public void setUp() { + rawCachedMNodeSize = config.getCachedMNodeSizeInPBTreeMode(); + config.setCachedMNodeSizeInPBTreeMode(10000); + ReleaseFlushMonitor.getInstance().clear(); + ReleaseFlushMonitor.getInstance().init(new CachedSchemaEngineStatistics()); + } + + @After + public void tearDown() throws Exception { + if (mtree != null) { + mtree.clear(); + mtree = null; + } + ReleaseFlushMonitor.getInstance().clear(); + FileUtils.deleteDirectory(new File(config.getSchemaDir())); + config.setCachedMNodeSizeInPBTreeMode(rawCachedMNodeSize); + } + + @Test + public void testDeviceDescendantFlagIsMaintainedAcrossCreateDeleteAndLazyRecompute() + throws Exception { + mtree = newMTree(); + + createBooleanTimeSeries("root.sg.a.b.s1"); + + ICachedMNode database = mtree.getNodeByPath(new PartialPath("root.sg")); + ICachedMNode aNode = mtree.getNodeByPath(new PartialPath("root.sg.a")); + ICachedMNode bNode = mtree.getNodeByPath(new PartialPath("root.sg.a.b")); + + Assert.assertTrue(database.hasDeviceDescendant()); + Assert.assertTrue(database.isDeviceDescendantComputed()); + Assert.assertFalse(aNode.isDevice()); + Assert.assertTrue(aNode.hasDeviceDescendant()); + Assert.assertTrue(aNode.isDeviceDescendantComputed()); + Assert.assertTrue(bNode.isDevice()); + Assert.assertFalse(bNode.hasDeviceDescendant()); + Assert.assertTrue(bNode.isDeviceDescendantComputed()); + mtree.unPinMNode(database); + mtree.unPinMNode(aNode); + mtree.unPinMNode(bNode); + + createBooleanTimeSeries("root.sg.a.s0"); + createBooleanTimeSeries("root.sg.c.d.s2"); + + aNode = mtree.getNodeByPath(new PartialPath("root.sg.a")); + ICachedMNode cNode = mtree.getNodeByPath(new PartialPath("root.sg.c")); + Assert.assertTrue(aNode.isDevice()); + Assert.assertTrue(aNode.hasDeviceDescendant()); + Assert.assertTrue(aNode.isDeviceDescendantComputed()); + Assert.assertTrue(cNode.hasDeviceDescendant()); + Assert.assertTrue(cNode.isDeviceDescendantComputed()); + + aNode.setHasDeviceDescendant(false); + aNode.setDeviceDescendantComputed(false); + mtree.unPinMNode(aNode); + cNode.setHasDeviceDescendant(false); + cNode.setDeviceDescendantComputed(false); + mtree.unPinMNode(cNode); + + Assert.assertEquals( + Collections.singletonList("root.sg.a.b.s1"), + collectMeasurementPaths(new PartialPath("root.sg.a.**.s1"))); + + aNode = mtree.getNodeByPath(new PartialPath("root.sg.a")); + Assert.assertTrue(aNode.hasDeviceDescendant()); + Assert.assertTrue(aNode.isDeviceDescendantComputed()); + mtree.unPinMNode(aNode); + cNode = mtree.getNodeByPath(new PartialPath("root.sg.c")); + Assert.assertFalse(cNode.isDeviceDescendantComputed()); + mtree.unPinMNode(cNode); + + mtree.deleteTimeseries(new PartialPath("root.sg.a.s0")); + aNode = mtree.getNodeByPath(new PartialPath("root.sg.a")); + Assert.assertFalse(aNode.isDevice()); + Assert.assertTrue(aNode.hasDeviceDescendant()); + Assert.assertTrue(aNode.isDeviceDescendantComputed()); + mtree.unPinMNode(aNode); + + mtree.deleteTimeseries(new PartialPath("root.sg.a.b.s1")); + database = mtree.getNodeByPath(new PartialPath("root.sg")); + Assert.assertTrue(database.hasDeviceDescendant()); + Assert.assertTrue(database.isDeviceDescendantComputed()); + mtree.unPinMNode(database); + assertPathNotExist(new PartialPath("root.sg.a")); + + mtree.deleteTimeseries(new PartialPath("root.sg.c.d.s2")); + database = mtree.getNodeByPath(new PartialPath("root.sg")); + Assert.assertFalse(database.hasDeviceDescendant()); + Assert.assertTrue(database.isDeviceDescendantComputed()); + mtree.unPinMNode(database); + assertPathNotExist(new PartialPath("root.sg.c")); + } + + private MTreeBelowSGCachedImpl newMTree() throws Exception { + final CachedSchemaRegionStatistics regionStatistics = + new CachedSchemaRegionStatistics(0, new CachedSchemaEngineStatistics()); + return new MTreeBelowSGCachedImpl( + new PartialPath("root.sg"), + node -> Collections.emptyMap(), + node -> Collections.emptyMap(), + () -> {}, + node -> {}, + node -> {}, + 0, + regionStatistics, + new SchemaRegionCachedMetric(regionStatistics, "root.sg")); + } + + private void createBooleanTimeSeries(final String path) throws MetadataException { + mtree.createTimeSeries( + new MeasurementPath(path), + TSDataType.BOOLEAN, + TSEncoding.PLAIN, + CompressionType.SNAPPY, + null, + null); + } + + private List collectMeasurementPaths(final PartialPath pattern) throws Exception { + final ICachedMNode rootNode = (ICachedMNode) ROOT_NODE_FIELD.get(mtree); + final CachedMTreeStore store = (CachedMTreeStore) STORE_FIELD.get(mtree); + final List matchedPaths = new ArrayList<>(); + try (MeasurementCollector collector = + new MeasurementCollector( + rootNode, pattern, store, false, SchemaConstant.ALL_MATCH_SCOPE) { + @Override + protected Void collectMeasurement(final IMeasurementMNode node) { + matchedPaths.add(getCurrentMeasurementPathInTraverse(node).getFullPath()); + return null; + } + }) { + collector.traverse(); + } + return matchedPaths; + } + + private void assertPathNotExist(final PartialPath path) throws MetadataException { + try { + mtree.getNodeByPath(path); + Assert.fail("Expected path not exist: " + path.getFullPath()); + } catch (final PathNotExistException ignored) { + // expected + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractAboveDatabaseMNode.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractAboveDatabaseMNode.java index ab0592480e5e2..fc379e34e5b02 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractAboveDatabaseMNode.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/common/AbstractAboveDatabaseMNode.java @@ -37,6 +37,10 @@ public AbstractAboveDatabaseMNode(BasicNode basicMNode) { this.basicMNode = basicMNode; } + public BasicNode getBasicMNode() { + return basicMNode; + } + @Override public String getName() { return basicMNode.getName(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeIterator.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeIterator.java index 75fd9d40c67dc..d1f410c546015 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeIterator.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/node/utils/IMNodeIterator.java @@ -22,7 +22,7 @@ import java.util.Iterator; -public interface IMNodeIterator> extends Iterator { +public interface IMNodeIterator> extends Iterator, AutoCloseable { void skipTemplateChildren(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java index caa8019ef8717..28b192e99cae1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/tree/AbstractTreeVisitor.java @@ -413,6 +413,10 @@ protected Iterator getCurrentChildrenIterator() { } } + protected final IStateMatchInfo getCurrentStateMatchInfo() { + return currentStateMatchInfo; + } + // Release a child node. protected void releaseNode(N node) {} From 7a258736ea5a8a2005303ba37600dfe9008d288f Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 3 Jun 2026 10:43:31 +0800 Subject: [PATCH 073/102] Pipe: Fixed the bug that drop pipe may stuck when disruptor is interrupted or ring buffer full (#17673) (#17818) * Fix interrupt * spotless (cherry picked from commit b614d6d16b5ab2ed50216028e4467b165004f6fb) --- .../realtime/assigner/DisruptorQueue.java | 19 ++++++- .../assigner/PipeDataRegionAssigner.java | 41 ++++++++++++-- .../disruptor/BatchEventProcessor.java | 11 ++-- .../disruptor/MultiProducerSequencer.java | 21 +++++++ .../realtime/disruptor/RingBuffer.java | 22 +++++++- .../disruptor/DisruptorShutdownTest.java | 56 +++++++++++++++++++ 6 files changed, 157 insertions(+), 13 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java index 2019eba85603b..d3fc92a677a23 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/DisruptorQueue.java @@ -87,16 +87,29 @@ public DisruptorQueue( } public void publish(final PipeRealtimeEvent event) { + publishOrDrop(event); + } + + public boolean publishOrDrop(final PipeRealtimeEvent event) { final EnrichedEvent innerEvent = event.getEvent(); if (innerEvent instanceof PipeHeartbeatEvent) { ((PipeHeartbeatEvent) innerEvent).recordDisruptorSize(ringBuffer); } - ringBuffer.publishEvent((container, sequence, o) -> container.setEvent(event), event); - mayPrintExceedingLog(); + final boolean published = + ringBuffer.publishEvent( + (container, sequence, o) -> container.setEvent(event), event, this::isClosed); + if (published) { + mayPrintExceedingLog(); + } + return published; } - public void shutdown() { + public void closeInput() { isClosed = true; + } + + public void shutdown() { + closeInput(); // use shutdown instead of halt to ensure all published events have been handled disruptor.shutdown(); allocatedMemoryBlock.close(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java index 2375726e42772..01dcbb8926509 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -60,6 +60,7 @@ public class PipeDataRegionAssigner implements Closeable { private volatile int listenToInsertNodeSourceCount = 0; private final PipeEventCounter eventCounter = new PipeDataRegionEventCounter(); + private int inFlightPublishCount = 0; public String getDataRegionId() { return dataRegionId; @@ -87,14 +88,28 @@ public void publishToAssign(final PipeRealtimeEvent event) { ((PipeHeartbeatEvent) innerEvent).onPublished(); } - // use synchronized here for completely preventing reference count leaks under extreme thread - // scheduling when closing synchronized (this) { - if (!disruptor.isClosed()) { - disruptor.publish(event); - } else { + if (disruptor.isClosed()) { onAssignedHook(event); + return; } + inFlightPublishCount++; + } + + boolean isPublished = false; + try { + isPublished = disruptor.publishOrDrop(event); + } finally { + synchronized (this) { + inFlightPublishCount--; + if (inFlightPublishCount == 0) { + notifyAll(); + } + } + } + + if (!isPublished) { + onAssignedHook(event); } } @@ -241,9 +256,25 @@ public boolean notMoreSourceNeededToBeAssigned() { public synchronized void close() { PipeAssignerMetrics.getInstance().deregister(dataRegionId); + boolean interrupted = false; + disruptor.closeInput(); + while (inFlightPublishCount > 0) { + try { + wait(); + } catch (final InterruptedException e) { + interrupted = true; + LOGGER.warn( + "Interrupted while waiting for in-flight publishes to finish when closing assigner on data region {}.", + dataRegionId); + } + } + final long startTime = System.currentTimeMillis(); disruptor.shutdown(); matcher.clear(); + if (interrupted) { + Thread.currentThread().interrupt(); + } LOGGER.info( "Pipe: Assigner on data region {} shutdown internal disruptor within {} ms", dataRegionId, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/BatchEventProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/BatchEventProcessor.java index d0432821cf77c..6a3b2cc62fd93 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/BatchEventProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/BatchEventProcessor.java @@ -80,11 +80,14 @@ public void run() { nextSequence = processAvailableEvents(nextSequence, availableSequence); } catch (final InterruptedException ex) { - if (running) { - Thread.currentThread().interrupt(); - LOGGER.info("Processor interrupted"); + if (!running) { + break; } - break; + // A transient interrupt should not permanently stop the consumer thread. Otherwise the + // gating sequence will stop advancing and producers may block forever on a full ring + // buffer, making the later close path appear stuck. + Thread.interrupted(); + LOGGER.warn("Processor interrupted unexpectedly, continue running"); } catch (final Throwable ex) { exceptionHandler.handleEventException(ex, nextSequence, ringBuffer.get(nextSequence)); sequence.set(nextSequence); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/MultiProducerSequencer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/MultiProducerSequencer.java index d40ed96839870..79d205d670ce7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/MultiProducerSequencer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/MultiProducerSequencer.java @@ -21,6 +21,7 @@ import java.util.concurrent.atomic.AtomicIntegerArray; import java.util.concurrent.locks.LockSupport; +import java.util.function.BooleanSupplier; /** * Multi-producer sequencer for coordinating concurrent publishers @@ -108,14 +109,31 @@ public MultiProducerSequencer(int bufferSize, Sequence[] gatingSequences) { * @return highest claimed sequence number */ public long next(int n) { + return next(n, () -> false); + } + + /** + * Claim next n sequences for publishing, or abort if the caller is closing. + * + * @param n number of sequences to claim + * @param abortCondition returns {@code true} if the claim should be abandoned + * @return highest claimed sequence number, or {@link Sequence#INITIAL_VALUE} if aborted + */ + public long next(final int n, final BooleanSupplier abortCondition) { if (n < 1) { throw new IllegalArgumentException("n must be > 0"); } + final BooleanSupplier effectiveAbortCondition = + abortCondition != null ? abortCondition : () -> false; long current; long next; do { + if (effectiveAbortCondition.getAsBoolean()) { + return Sequence.INITIAL_VALUE; + } + current = cursor.get(); next = current + n; @@ -126,6 +144,9 @@ public long next(int n) { long gatingSequence = Sequence.getMinimumSequence(gatingSequences, current); if (wrapPoint > gatingSequence) { + if (effectiveAbortCondition.getAsBoolean()) { + return Sequence.INITIAL_VALUE; + } LockSupport.parkNanos(1); continue; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/RingBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/RingBuffer.java index 2af784b603d6c..25f7fe7d1b91f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/RingBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/RingBuffer.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.pipe.source.dataregion.realtime.disruptor; +import java.util.function.BooleanSupplier; + /** * Left-hand side padding for cache line alignment * @@ -205,8 +207,26 @@ public void publish(long lo, long hi) { * @param argument type */ public void publishEvent(EventTranslator translator, A arg0) { - final long sequence = sequencer.next(1); + publishEvent(translator, arg0, () -> false); + } + + /** + * Publish event using a translator function, or abort if the caller is closing. + * + * @param translator function to populate the event + * @param arg0 argument passed to translator + * @param abortCondition returns {@code true} if the publish should be abandoned + * @param argument type + * @return {@code true} if the event is published, {@code false} if the publish is aborted + */ + public boolean publishEvent( + final EventTranslator translator, final A arg0, final BooleanSupplier abortCondition) { + final long sequence = sequencer.next(1, abortCondition); + if (sequence == Sequence.INITIAL_VALUE) { + return false; + } translateAndPublish(translator, sequence, arg0); + return true; } /** diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/DisruptorShutdownTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/DisruptorShutdownTest.java index 3fd40c4d4f2b5..ef57ea625aef6 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/DisruptorShutdownTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/disruptor/DisruptorShutdownTest.java @@ -25,6 +25,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -113,6 +114,61 @@ public void testDisruptorShutdownInterruptsWaitingProcessor() throws Exception { Assert.assertFalse(processorThread.isAlive()); } + @Test + public void testUnexpectedInterruptDoesNotStopProcessor() throws Exception { + final AtomicReference processorThreadReference = new AtomicReference<>(); + final ThreadFactory threadFactory = + runnable -> { + final Thread thread = new Thread(runnable, "pipe-disruptor-unexpected-interrupt-test"); + processorThreadReference.set(thread); + return thread; + }; + + final CountDownLatch handled = new CountDownLatch(1); + final Disruptor disruptor = new Disruptor<>(TestEvent::new, 32, threadFactory); + final RingBuffer ringBuffer = + disruptor.handleEventsWith((event, sequence, endOfBatch) -> handled.countDown()).start(); + + final Thread processorThread = processorThreadReference.get(); + Assert.assertNotNull(processorThread); + + TimeUnit.MILLISECONDS.sleep(50); + processorThread.interrupt(); + + ringBuffer.publishEvent((event, sequence, value) -> event.value = value, 1); + Assert.assertTrue(handled.await(5, TimeUnit.SECONDS)); + Assert.assertTrue(processorThread.isAlive()); + + disruptor.shutdown(); + Assert.assertFalse(processorThread.isAlive()); + } + + @Test + public void testPublishEventCanAbortWhenClosingWhileBufferIsFull() throws Exception { + final RingBuffer ringBuffer = RingBuffer.createMultiProducer(TestEvent::new, 1); + final Sequence gatingSequence = new Sequence(); + ringBuffer.addGatingSequences(gatingSequence); + ringBuffer.publishEvent((event, sequence, value) -> event.value = value, 1); + + final AtomicBoolean isClosed = new AtomicBoolean(false); + final AtomicBoolean published = new AtomicBoolean(true); + final Thread publisherThread = + new Thread( + () -> + published.set( + ringBuffer.publishEvent( + (event, sequence, value) -> event.value = value, 2, isClosed::get)), + "pipe-disruptor-publish-abort-test"); + + publisherThread.start(); + TimeUnit.MILLISECONDS.sleep(50); + isClosed.set(true); + publisherThread.join(TimeUnit.SECONDS.toMillis(5)); + + Assert.assertFalse(publisherThread.isAlive()); + Assert.assertFalse(published.get()); + } + private static class TestEvent { private int value; } From 24be79aba9c973a4cb11fdb8947cdf32da12ea8e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 3 Jun 2026 10:57:35 +0800 Subject: [PATCH 074/102] Fix CQ recovery gap and stale callback contamination (#17734) (#17820) * fix * sp * Fix CQ local schedule cancellation --- iotdb-core/confignode/pom.xml | 4 - .../consensus/request/read/cq/ShowCQPlan.java | 13 ++ .../request/write/cq/ActiveCQPlan.java | 20 +-- .../consensus/request/write/cq/AddCQPlan.java | 20 +-- .../request/write/cq/DropCQPlan.java | 20 +-- .../write/cq/UpdateCQLastExecTimePlan.java | 23 ++-- .../confignode/manager/cq/CQManager.java | 102 ++++++++++++++- .../confignode/manager/cq/CQScheduleTask.java | 58 +++++++-- .../confignode/persistence/cq/CQInfo.java | 76 +++++++----- .../executor/ConfigPlanExecutor.java | 3 +- .../procedure/impl/cq/CreateCQProcedure.java | 80 ++++++++++-- .../request/ConfigPhysicalPlanSerDeTest.java | 8 +- .../iotdb/confignode/cq/CQManagerTest.java | 107 ++++++++++++++++ .../confignode/persistence/CQInfoTest.java | 64 +++++++++- .../procedure/impl/CreateCQProcedureTest.java | 26 ++++ .../cq/CreateCQProcedureRecoveryTest.java | 117 ++++++++++++++++++ 16 files changed, 635 insertions(+), 106 deletions(-) create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java diff --git a/iotdb-core/confignode/pom.xml b/iotdb-core/confignode/pom.xml index fb22fd5b091f7..a7cd4b6298b5f 100644 --- a/iotdb-core/confignode/pom.xml +++ b/iotdb-core/confignode/pom.xml @@ -137,10 +137,6 @@ org.apache.commons commons-lang3 - - commons-codec - commons-codec - org.apache.thrift libthrift diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java index 5217849deb488..c28838d556b48 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java @@ -21,11 +21,24 @@ import org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan; +import java.util.Optional; + import static org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType.SHOW_CQ; public class ShowCQPlan extends ConfigPhysicalReadPlan { + private final String cqId; + public ShowCQPlan() { + this(null); + } + + public ShowCQPlan(String cqId) { super(SHOW_CQ); + this.cqId = cqId; + } + + public Optional getCqId() { + return Optional.ofNullable(cqId); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java index e488ac2566921..4ca33b054a93f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java @@ -35,39 +35,39 @@ public class ActiveCQPlan extends ConfigPhysicalPlan { private String cqId; - private String md5; + private String cqToken; public ActiveCQPlan() { super(ACTIVE_CQ); } - public ActiveCQPlan(String cqId, String md5) { + public ActiveCQPlan(String cqId, String cqToken) { super(ACTIVE_CQ); Validate.notNull(cqId); - Validate.notNull(md5); + Validate.notNull(cqToken); this.cqId = cqId; - this.md5 = md5; + this.cqToken = cqToken; } public String getCqId() { return cqId; } - public String getMd5() { - return md5; + public String getCqToken() { + return cqToken; } @Override protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(cqId, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { cqId = ReadWriteIOUtils.readString(buffer); - md5 = ReadWriteIOUtils.readString(buffer); + cqToken = ReadWriteIOUtils.readString(buffer); } @Override @@ -82,11 +82,11 @@ public boolean equals(Object o) { return false; } ActiveCQPlan that = (ActiveCQPlan) o; - return cqId.equals(that.cqId) && md5.equals(that.md5); + return cqId.equals(that.cqId) && cqToken.equals(that.cqToken); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), cqId, md5); + return Objects.hash(super.hashCode(), cqId, cqToken); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java index 0aae9e2e974ca..721c83d3d2f76 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java @@ -37,7 +37,7 @@ public class AddCQPlan extends ConfigPhysicalPlan { private TCreateCQReq req; - private String md5; + private String cqToken; private long firstExecutionTime; @@ -45,12 +45,12 @@ public AddCQPlan() { super(ADD_CQ); } - public AddCQPlan(TCreateCQReq req, String md5, long firstExecutionTime) { + public AddCQPlan(TCreateCQReq req, String cqToken, long firstExecutionTime) { super(ADD_CQ); Validate.notNull(req); - Validate.notNull(md5); + Validate.notNull(cqToken); this.req = req; - this.md5 = md5; + this.cqToken = cqToken; this.firstExecutionTime = firstExecutionTime; } @@ -58,8 +58,8 @@ public TCreateCQReq getReq() { return req; } - public String getMd5() { - return md5; + public String getCqToken() { + return cqToken; } public long getFirstExecutionTime() { @@ -70,14 +70,14 @@ public long getFirstExecutionTime() { protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ThriftCommonsSerDeUtils.serializeTCreateCQReq(req, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); ReadWriteIOUtils.write(firstExecutionTime, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { req = ThriftCommonsSerDeUtils.deserializeTCreateCQReq(buffer); - md5 = ReadWriteIOUtils.readString(buffer); + cqToken = ReadWriteIOUtils.readString(buffer); firstExecutionTime = ReadWriteIOUtils.readLong(buffer); } @@ -95,11 +95,11 @@ public boolean equals(Object o) { AddCQPlan addCQPlan = (AddCQPlan) o; return firstExecutionTime == addCQPlan.firstExecutionTime && Objects.equals(req, addCQPlan.req) - && Objects.equals(md5, addCQPlan.md5); + && Objects.equals(cqToken, addCQPlan.cqToken); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), req, md5, firstExecutionTime); + return Objects.hash(super.hashCode(), req, cqToken, firstExecutionTime); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java index 5c901362997f8..f22561c0b807d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java @@ -37,7 +37,7 @@ public class DropCQPlan extends ConfigPhysicalPlan { private String cqId; // may be null in user call of drop CQ - private String md5; + private String cqToken; public DropCQPlan() { super(DROP_CQ); @@ -49,33 +49,33 @@ public DropCQPlan(String cqId) { this.cqId = cqId; } - public DropCQPlan(String cqId, String md5) { + public DropCQPlan(String cqId, String cqToken) { super(DROP_CQ); Validate.notNull(cqId); - Validate.notNull(md5); + Validate.notNull(cqToken); this.cqId = cqId; - this.md5 = md5; + this.cqToken = cqToken; } public String getCqId() { return cqId; } - public Optional getMd5() { - return Optional.ofNullable(md5); + public Optional getCqToken() { + return Optional.ofNullable(cqToken); } @Override protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(cqId, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { cqId = ReadWriteIOUtils.readString(buffer); - md5 = ReadWriteIOUtils.readString(buffer); + cqToken = ReadWriteIOUtils.readString(buffer); } @Override @@ -90,11 +90,11 @@ public boolean equals(Object o) { return false; } DropCQPlan that = (DropCQPlan) o; - return cqId.equals(that.cqId) && Objects.equals(md5, that.md5); + return cqId.equals(that.cqId) && Objects.equals(cqToken, that.cqToken); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), cqId, md5); + return Objects.hash(super.hashCode(), cqId, cqToken); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java index 51a8358a04c28..392f6006ba56e 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java @@ -37,20 +37,19 @@ public class UpdateCQLastExecTimePlan extends ConfigPhysicalPlan { private long executionTime; - // may be null in user call of drop CQ - private String md5; + private String cqToken; public UpdateCQLastExecTimePlan() { super(UPDATE_CQ_LAST_EXEC_TIME); } - public UpdateCQLastExecTimePlan(String cqId, long executionTime, String md5) { + public UpdateCQLastExecTimePlan(String cqId, long executionTime, String cqToken) { super(UPDATE_CQ_LAST_EXEC_TIME); Validate.notNull(cqId); - Validate.notNull(md5); + Validate.notNull(cqToken); this.cqId = cqId; this.executionTime = executionTime; - this.md5 = md5; + this.cqToken = cqToken; } public String getCqId() { @@ -61,8 +60,8 @@ public long getExecutionTime() { return executionTime; } - public String getMd5() { - return md5; + public String getCqToken() { + return cqToken; } @Override @@ -70,14 +69,14 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(cqId, stream); ReadWriteIOUtils.write(executionTime, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { cqId = ReadWriteIOUtils.readString(buffer); executionTime = ReadWriteIOUtils.readLong(buffer); - md5 = ReadWriteIOUtils.readString(buffer); + cqToken = ReadWriteIOUtils.readString(buffer); } @Override @@ -92,11 +91,13 @@ public boolean equals(Object o) { return false; } UpdateCQLastExecTimePlan that = (UpdateCQLastExecTimePlan) o; - return executionTime == that.executionTime && cqId.equals(that.cqId) && md5.equals(that.md5); + return executionTime == that.executionTime + && cqId.equals(that.cqId) + && cqToken.equals(that.cqToken); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), cqId, executionTime, md5); + return Objects.hash(super.hashCode(), cqId, executionTime, cqToken); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java index 5726b3ce82698..a90a21f285ff2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java @@ -42,7 +42,10 @@ import java.util.Collections; import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -56,11 +59,15 @@ public class CQManager { private final ReadWriteLock lock; + // Key: CQ id. Value: the local task and the metadata token it owns. + private final ConcurrentMap locallyScheduledCQs; + private ScheduledExecutorService executor; public CQManager(ConfigManager configManager) { this.configManager = configManager; this.lock = new ReentrantReadWriteLock(); + this.locallyScheduledCQs = new ConcurrentHashMap<>(); this.executor = IoTDBThreadPoolFactory.newScheduledThreadPool( CONF.getCqSubmitThread(), ThreadName.CQ_SCHEDULER.getName()); @@ -77,14 +84,21 @@ public TSStatus createCQ(TCreateCQReq req) { } public TSStatus dropCQ(TDropCQReq req) { + lock.readLock().lock(); try { - return configManager.getConsensusManager().write(new DropCQPlan(req.cqId)); + TSStatus status = configManager.getConsensusManager().write(new DropCQPlan(req.cqId)); + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + cancelLocallyScheduledCQ(req.cqId); + } + return status; } catch (ConsensusException e) { LOGGER.warn("Unexpected error happened while dropping cq {}: ", req.cqId, e); // consensus layer related errors TSStatus res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); res.setMessage(e.getMessage()); return res; + } finally { + lock.readLock().unlock(); } } @@ -117,6 +131,7 @@ public void startCQScheduler() { try { // 1. shutdown previous cq schedule thread pool try { + cancelAllLocallyScheduledCQs(); if (executor != null) { executor.shutdown(); } @@ -154,7 +169,15 @@ public void startCQScheduler() { for (CQInfo.CQEntry entry : allCQs) { if (entry.getState() == CQState.ACTIVE) { CQScheduleTask cqScheduleTask = new CQScheduleTask(entry, executor, configManager); - cqScheduleTask.submitSelf(); + if (!markCQLocallyScheduled(entry.getCqId(), entry.getCqToken(), cqScheduleTask)) { + continue; + } + try { + cqScheduleTask.submitSelf(); + } catch (RuntimeException e) { + unmarkCQLocallyScheduled(entry.getCqId(), entry.getCqToken()); + throw e; + } } } } @@ -174,6 +197,7 @@ public void stopCQScheduler() { try { previous = executor; executor = null; + cancelAllLocallyScheduledCQs(); } finally { lock.writeLock().unlock(); } @@ -181,4 +205,78 @@ public void stopCQScheduler() { previous.shutdown(); } } + + public boolean markCQLocallyScheduled(String cqId, String cqToken, CQScheduleTask task) { + AtomicBoolean shouldSchedule = new AtomicBoolean(false); + LocallyScheduledCQ schedule = new LocallyScheduledCQ(cqToken, task); + lock.readLock().lock(); + try { + locallyScheduledCQs.compute( + cqId, + (ignored, previousSchedule) -> { + if (previousSchedule != null && previousSchedule.hasToken(cqToken)) { + return previousSchedule; + } + if (previousSchedule != null) { + previousSchedule.cancel(); + } + shouldSchedule.set(true); + return schedule; + }); + if (!shouldSchedule.get()) { + task.cancel(); + } + return shouldSchedule.get(); + } finally { + lock.readLock().unlock(); + } + } + + public void unmarkCQLocallyScheduled(String cqId, String cqToken) { + lock.readLock().lock(); + try { + locallyScheduledCQs.computeIfPresent( + cqId, + (ignored, schedule) -> { + if (schedule.hasToken(cqToken)) { + schedule.cancel(); + return null; + } + return schedule; + }); + } finally { + lock.readLock().unlock(); + } + } + + private void cancelLocallyScheduledCQ(String cqId) { + LocallyScheduledCQ schedule = locallyScheduledCQs.remove(cqId); + if (schedule != null) { + schedule.cancel(); + } + } + + private void cancelAllLocallyScheduledCQs() { + locallyScheduledCQs.values().forEach(LocallyScheduledCQ::cancel); + locallyScheduledCQs.clear(); + } + + private static class LocallyScheduledCQ { + + private final String cqToken; + private final CQScheduleTask task; + + private LocallyScheduledCQ(String cqToken, CQScheduleTask task) { + this.cqToken = cqToken; + this.task = task; + } + + private boolean hasToken(String cqToken) { + return this.cqToken.equals(cqToken); + } + + private void cancel() { + task.cancel(); + } + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java index 6125edc1ef7bd..d85d0f12a3040 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java @@ -39,7 +39,10 @@ import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; public class CQScheduleTask implements Runnable { @@ -69,7 +72,7 @@ public class CQScheduleTask implements Runnable { private final long endTimeOffset; private final TimeoutPolicy timeoutPolicy; private final String queryBody; - private final String md5; + private final String cqToken; private final String zoneId; @@ -81,12 +84,15 @@ public class CQScheduleTask implements Runnable { private final long retryWaitTimeInMS; + private final AtomicBoolean cancelled; + private final AtomicReference> scheduledFuture; + private long executionTime; public CQScheduleTask( TCreateCQReq req, long firstExecutionTime, - String md5, + String cqToken, ScheduledExecutorService executor, ConfigManager configManager) { this( @@ -96,7 +102,7 @@ public CQScheduleTask( req.endTimeOffset, TimeoutPolicy.deserialize(req.timeoutPolicy), req.queryBody, - md5, + cqToken, req.zoneId, req.username, executor, @@ -113,7 +119,7 @@ public CQScheduleTask( entry.getEndTimeOffset(), entry.getTimeoutPolicy(), entry.getQueryBody(), - entry.getMd5(), + entry.getCqToken(), entry.getZoneId(), entry.getUsername(), executor, @@ -129,7 +135,7 @@ public CQScheduleTask( long endTimeOffset, TimeoutPolicy timeoutPolicy, String queryBody, - String md5, + String cqToken, String zoneId, String username, ScheduledExecutorService executor, @@ -141,12 +147,14 @@ public CQScheduleTask( this.endTimeOffset = endTimeOffset; this.timeoutPolicy = timeoutPolicy; this.queryBody = queryBody; - this.md5 = md5; + this.cqToken = cqToken; this.zoneId = zoneId; this.username = username; this.executor = executor; this.configManager = configManager; this.retryWaitTimeInMS = Math.min(DEFAULT_RETRY_WAIT_TIME_IN_MS, everyInterval / FACTOR); + this.cancelled = new AtomicBoolean(false); + this.scheduledFuture = new AtomicReference<>(); this.executionTime = executionTime; } @@ -165,6 +173,9 @@ public static long getFirstExecutionTime(long boundaryTime, long everyInterval, @Override public void run() { + if (cancelled.get()) { + return; + } long startTime = executionTime - startTimeOffset; long endTime = executionTime - endTimeOffset; @@ -177,6 +188,9 @@ public void run() { submitSelf(retryWaitTimeInMS, TimeUnit.MILLISECONDS); } } else { + if (cancelled.get()) { + return; + } LOGGER.info( "[StartExecuteCQ] execute CQ {} on DataNode[{}], time range is [{}, {}), current time is {}", cqId, @@ -206,12 +220,32 @@ public void submitSelf() { } private void submitSelf(long delay, TimeUnit unit) { - executor.schedule(this, delay, unit); + if (cancelled.get()) { + return; + } + ScheduledFuture newFuture = executor.schedule(this, delay, unit); + ScheduledFuture previousFuture = scheduledFuture.getAndSet(newFuture); + if (previousFuture != null) { + previousFuture.cancel(false); + } + if (cancelled.get() && scheduledFuture.compareAndSet(newFuture, null)) { + newFuture.cancel(false); + } + } + + public void cancel() { + cancelled.set(true); + ScheduledFuture currentFuture = scheduledFuture.getAndSet(null); + if (currentFuture != null) { + currentFuture.cancel(false); + } } private boolean needSubmit() { // current node is still leader and thread pool is not shut down. - return configManager.getConsensusManager().isLeader() && !executor.isShutdown(); + return !cancelled.get() + && configManager.getConsensusManager().isLeader() + && !executor.isShutdown(); } private class AsyncExecuteCQCallback implements AsyncMethodCallback { @@ -238,6 +272,9 @@ private void updateExecutionTime() { @Override public void onComplete(TSStatus response) { + if (cancelled.get()) { + return; + } if (response.code == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { LOGGER.info( @@ -251,7 +288,7 @@ public void onComplete(TSStatus response) { result = configManager .getConsensusManager() - .write(new UpdateCQLastExecTimePlan(cqId, executionTime, md5)); + .write(new UpdateCQLastExecTimePlan(cqId, executionTime, cqToken)); } catch (ConsensusException e) { result = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); result.setMessage(e.getMessage()); @@ -291,6 +328,9 @@ public void onComplete(TSStatus response) { @Override public void onError(Exception exception) { + if (cancelled.get()) { + return; + } LOGGER.warn("Execute CQ {} failed", cqId, exception); if (needSubmit()) { submitSelf(retryWaitTimeInMS, TimeUnit.MILLISECONDS); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java index 4c1225ef48a05..b24f1ca9aa528 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.cq.CQState; import org.apache.iotdb.commons.cq.TimeoutPolicy; import org.apache.iotdb.commons.snapshot.SnapshotProcessor; +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.ActiveCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.DropCQPlan; @@ -44,7 +45,9 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -61,7 +64,7 @@ public class CQInfo implements SnapshotProcessor { private static final String CQ_NOT_EXIST_FORMAT = "CQ %s doesn't exist."; - private static final String MD5_NOT_MATCH_FORMAT = "MD5 of CQ %s doesn't match"; + private static final String CQ_TOKEN_NOT_MATCH_FORMAT = "Token of CQ %s doesn't match"; private final Map cqMap; @@ -91,7 +94,7 @@ public TSStatus addCQ(AddCQPlan plan) { CQEntry cqEntry = new CQEntry( plan.getReq(), - plan.getMd5(), + plan.getCqToken(), plan.getFirstExecutionTime() - plan.getReq().everyInterval); cqMap.put(cqId, cqEntry); res.code = TSStatusCode.SUCCESS_STATUS.getStatusCode(); @@ -105,13 +108,13 @@ public TSStatus addCQ(AddCQPlan plan) { /** * Drop the CQ whose ID is same as cqId in plan. * - * @return SUCCESS_STATUS if there is CQ whose ID and md5 is same as cqId in plan, + * @return SUCCESS_STATUS if there is CQ whose ID and token is same as cqId in plan, * otherwise NO_SUCH_CQ. */ public TSStatus dropCQ(DropCQPlan plan) { TSStatus res = new TSStatus(); String cqId = plan.getCqId(); - Optional md5 = plan.getMd5(); + Optional cqToken = plan.getCqToken(); lock.writeLock().lock(); try { CQEntry cqEntry = cqMap.get(cqId); @@ -119,10 +122,10 @@ public TSStatus dropCQ(DropCQPlan plan) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); res.message = String.format(CQ_NOT_EXIST_FORMAT, cqId); LOGGER.warn("Drop CQ {} failed, because it doesn't exist.", cqId); - } else if ((md5.isPresent() && !md5.get().equals(cqEntry.md5))) { + } else if ((cqToken.isPresent() && !cqToken.get().equals(cqEntry.cqToken))) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); - res.message = String.format(MD5_NOT_MATCH_FORMAT, cqId); - LOGGER.warn("Drop CQ {} failed, because its MD5 doesn't match.", cqId); + res.message = String.format(CQ_TOKEN_NOT_MATCH_FORMAT, cqId); + LOGGER.warn("Drop CQ {} failed, because its token doesn't match.", cqId); } else { cqMap.remove(cqId); res.code = TSStatusCode.SUCCESS_STATUS.getStatusCode(); @@ -135,11 +138,24 @@ public TSStatus dropCQ(DropCQPlan plan) { } public ShowCQResp showCQ() { + return showCQ(new ShowCQPlan()); + } + + public ShowCQResp showCQ(ShowCQPlan plan) { lock.readLock().lock(); try { - return new ShowCQResp( - new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), - cqMap.values().stream().map(CQEntry::new).collect(Collectors.toList())); + Optional cqId = plan.getCqId(); + List cqList; + if (cqId.isPresent()) { + CQEntry cqEntry = cqMap.get(cqId.get()); + cqList = + cqEntry == null + ? Collections.emptyList() + : Collections.singletonList(new CQEntry(cqEntry)); + } else { + cqList = cqMap.values().stream().map(CQEntry::new).collect(Collectors.toList()); + } + return new ShowCQResp(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), cqList); } finally { lock.readLock().unlock(); } @@ -153,16 +169,16 @@ public ShowCQResp showCQ() { public TSStatus activeCQ(ActiveCQPlan plan) { TSStatus res = new TSStatus(); String cqId = plan.getCqId(); - String md5 = plan.getMd5(); + String cqToken = plan.getCqToken(); lock.writeLock().lock(); try { CQEntry cqEntry = cqMap.get(cqId); if (cqEntry == null) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); res.message = String.format(CQ_NOT_EXIST_FORMAT, cqId); - } else if (!md5.equals(cqEntry.md5)) { + } else if (!cqToken.equals(cqEntry.cqToken)) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); - res.message = String.format(MD5_NOT_MATCH_FORMAT, cqId); + res.message = String.format(CQ_TOKEN_NOT_MATCH_FORMAT, cqId); } else if (cqEntry.state == CQState.ACTIVE) { res.code = TSStatusCode.CQ_ALREADY_ACTIVE.getStatusCode(); res.message = String.format("CQ %s has already been active", cqId); @@ -180,22 +196,22 @@ public TSStatus activeCQ(ActiveCQPlan plan) { * Update the last execution time of the corresponding CQ. * * @return SUCCESS_STATUS if successfully updated, or NO_SUCH_CQ if 1. the CQ doesn't exist; or 2. - * md5 is different. or CQ_UPDATE_LAST_EXEC_TIME_FAILED 3. original lastExecutionTime >= + * token is different. or CQ_UPDATE_LAST_EXEC_TIME_FAILED 3. original lastExecutionTime >= * current lastExecutionTime; */ public TSStatus updateCQLastExecutionTime(UpdateCQLastExecTimePlan plan) { TSStatus res = new TSStatus(); String cqId = plan.getCqId(); - String md5 = plan.getMd5(); + String cqToken = plan.getCqToken(); lock.writeLock().lock(); try { CQEntry cqEntry = cqMap.get(cqId); if (cqEntry == null) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); res.message = String.format(CQ_NOT_EXIST_FORMAT, cqId); - } else if (!md5.equals(cqEntry.md5)) { + } else if (!cqToken.equals(cqEntry.cqToken)) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); - res.message = String.format(MD5_NOT_MATCH_FORMAT, cqId); + res.message = String.format(CQ_TOKEN_NOT_MATCH_FORMAT, cqId); } else if (cqEntry.lastExecutionTime >= plan.getExecutionTime()) { res.code = TSStatusCode.CQ_UPDATE_LAST_EXEC_TIME_ERROR.getStatusCode(); res.message = @@ -299,7 +315,7 @@ public static class CQEntry { private final TimeoutPolicy timeoutPolicy; private final String queryBody; private final String sql; - private final String md5; + private final String cqToken; private final String zoneId; @@ -308,7 +324,7 @@ public static class CQEntry { private CQState state; private long lastExecutionTime; - private CQEntry(TCreateCQReq req, String md5, long lastExecutionTime) { + private CQEntry(TCreateCQReq req, String cqToken, long lastExecutionTime) { this( req.cqId, req.everyInterval, @@ -318,7 +334,7 @@ private CQEntry(TCreateCQReq req, String md5, long lastExecutionTime) { TimeoutPolicy.deserialize(req.timeoutPolicy), req.queryBody, req.sql, - md5, + cqToken, req.zoneId, req.username, CQState.INACTIVE, @@ -335,7 +351,7 @@ private CQEntry(CQEntry other) { other.timeoutPolicy, other.queryBody, other.sql, - other.md5, + other.cqToken, other.zoneId, other.username, other.state, @@ -352,7 +368,7 @@ private CQEntry( TimeoutPolicy timeoutPolicy, String queryBody, String sql, - String md5, + String cqToken, String zoneId, String username, CQState state, @@ -365,7 +381,7 @@ private CQEntry( this.timeoutPolicy = timeoutPolicy; this.queryBody = queryBody; this.sql = sql; - this.md5 = md5; + this.cqToken = cqToken; this.zoneId = zoneId; this.username = username; this.state = state; @@ -381,7 +397,7 @@ private void serialize(OutputStream stream) throws IOException { ReadWriteIOUtils.write(timeoutPolicy.getType(), stream); ReadWriteIOUtils.write(queryBody, stream); ReadWriteIOUtils.write(sql, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); ReadWriteIOUtils.write(zoneId, stream); ReadWriteIOUtils.write(username, stream); ReadWriteIOUtils.write(state.getType(), stream); @@ -397,7 +413,7 @@ private static CQEntry deserialize(InputStream stream) throws IOException { TimeoutPolicy timeoutPolicy = TimeoutPolicy.deserialize(ReadWriteIOUtils.readByte(stream)); String queryBody = ReadWriteIOUtils.readString(stream); String sql = ReadWriteIOUtils.readString(stream); - String md5 = ReadWriteIOUtils.readString(stream); + String cqToken = ReadWriteIOUtils.readString(stream); String zoneId = ReadWriteIOUtils.readString(stream); String username = ReadWriteIOUtils.readString(stream); CQState state = CQState.deserialize(ReadWriteIOUtils.readByte(stream)); @@ -411,7 +427,7 @@ private static CQEntry deserialize(InputStream stream) throws IOException { timeoutPolicy, queryBody, sql, - md5, + cqToken, zoneId, username, state, @@ -450,8 +466,8 @@ public String getSql() { return sql; } - public String getMd5() { - return md5; + public String getCqToken() { + return cqToken; } public CQState getState() { @@ -488,7 +504,7 @@ public boolean equals(Object o) { && timeoutPolicy == cqEntry.timeoutPolicy && Objects.equals(queryBody, cqEntry.queryBody) && Objects.equals(sql, cqEntry.sql) - && Objects.equals(md5, cqEntry.md5) + && Objects.equals(cqToken, cqEntry.cqToken) && Objects.equals(zoneId, cqEntry.zoneId) && Objects.equals(username, cqEntry.username) && state == cqEntry.state; @@ -505,7 +521,7 @@ public int hashCode() { timeoutPolicy, queryBody, sql, - md5, + cqToken, zoneId, username, state, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index 09017a841c73c..60ec748fb5478 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -30,6 +30,7 @@ import org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan; import org.apache.iotdb.confignode.consensus.request.read.ainode.GetAINodeConfigurationPlan; import org.apache.iotdb.confignode.consensus.request.read.auth.AuthorReadPlan; +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.read.database.CountDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.database.GetDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.datanode.GetDataNodeConfigurationPlan; @@ -322,7 +323,7 @@ public DataSet executeQueryPlan(final ConfigPhysicalReadPlan req) case GetSeriesSlotList: return partitionInfo.getSeriesSlotList((GetSeriesSlotListPlan) req); case SHOW_CQ: - return cqInfo.showCQ(); + return cqInfo.showCQ((ShowCQPlan) req); case GetFunctionTable: return udfInfo.getUDFTable(); case GetFunctionJar: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java index 3da60c5ad847b..af7f968e8a57c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java @@ -22,10 +22,14 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils; +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.ActiveCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.DropCQPlan; +import org.apache.iotdb.confignode.consensus.response.cq.ShowCQResp; +import org.apache.iotdb.confignode.manager.cq.CQManager; import org.apache.iotdb.confignode.manager.cq.CQScheduleTask; +import org.apache.iotdb.confignode.persistence.cq.CQInfo; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.impl.node.AbstractNodeProcedure; @@ -35,7 +39,6 @@ import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.commons.codec.digest.DigestUtils; import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +47,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Objects; +import java.util.Optional; +import java.util.UUID; import java.util.concurrent.ScheduledExecutorService; import static org.apache.iotdb.confignode.procedure.state.cq.CreateCQState.INACTIVE; @@ -59,7 +64,7 @@ public class CreateCQProcedure extends AbstractNodeProcedure { private TCreateCQReq req; - private String md5; + private String cqToken; private long firstExecutionTime; @@ -74,7 +79,7 @@ public CreateCQProcedure(ScheduledExecutorService executor) { public CreateCQProcedure(TCreateCQReq req, ScheduledExecutorService executor) { super(); this.req = req; - this.md5 = DigestUtils.md2Hex(req.cqId); + this.cqToken = generateCQToken(); this.executor = executor; this.firstExecutionTime = CQScheduleTask.getFirstExecutionTime(req.boundaryTime, req.everyInterval); @@ -90,12 +95,16 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, CreateCQState state) addCQ(env); return Flow.HAS_MORE_STATE; case INACTIVE: - CQScheduleTask cqScheduleTask = - new CQScheduleTask(req, firstExecutionTime, md5, executor, env.getConfigManager()); - cqScheduleTask.submitSelf(); + submitScheduleTask( + env, + new CQScheduleTask( + req, firstExecutionTime, cqToken, executor, env.getConfigManager())); setNextState(SCHEDULED); break; case SCHEDULED: + if (isStateDeserialized()) { + recoverScheduledTask(env); + } activeCQ(env); return Flow.NO_MORE_STATE; default: @@ -125,7 +134,7 @@ private void addCQ(ConfigNodeProcedureEnv env) { res = env.getConfigManager() .getConsensusManager() - .write(new AddCQPlan(req, md5, firstExecutionTime)); + .write(new AddCQPlan(req, cqToken, firstExecutionTime)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -146,7 +155,7 @@ private void addCQ(ConfigNodeProcedureEnv env) { private void activeCQ(ConfigNodeProcedureEnv env) { TSStatus res; try { - res = env.getConfigManager().getConsensusManager().write(new ActiveCQPlan(req.cqId, md5)); + res = env.getConfigManager().getConsensusManager().write(new ActiveCQPlan(req.cqId, cqToken)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -164,6 +173,42 @@ private void activeCQ(ConfigNodeProcedureEnv env) { } } + void recoverScheduledTask(ConfigNodeProcedureEnv env) throws ConsensusException { + Optional cqEntry = getCurrentCQEntry(env); + if (!cqEntry.isPresent()) { + LOGGER.info( + "Skip recovering the schedule task of CQ {} because its metadata is unavailable.", + req.cqId); + return; + } + submitScheduleTask(env, new CQScheduleTask(cqEntry.get(), executor, env.getConfigManager())); + } + + Optional getCurrentCQEntry(ConfigNodeProcedureEnv env) throws ConsensusException { + ShowCQResp response = + (ShowCQResp) env.getConfigManager().getConsensusManager().read(new ShowCQPlan(req.cqId)); + return response.getCqList().stream() + .filter(entry -> cqToken.equals(entry.getCqToken())) + .findFirst(); + } + + private static String generateCQToken() { + return UUID.randomUUID().toString(); + } + + private void submitScheduleTask(ConfigNodeProcedureEnv env, CQScheduleTask cqScheduleTask) { + CQManager cqManager = env.getConfigManager().getCQManager(); + if (!cqManager.markCQLocallyScheduled(req.cqId, cqToken, cqScheduleTask)) { + return; + } + try { + cqScheduleTask.submitSelf(); + } catch (RuntimeException e) { + cqManager.unmarkCQLocallyScheduled(req.cqId, cqToken); + throw e; + } + } + @Override protected void rollbackState(ConfigNodeProcedureEnv env, CreateCQState state) throws IOException, InterruptedException, ProcedureException { @@ -176,7 +221,8 @@ protected void rollbackState(ConfigNodeProcedureEnv env, CreateCQState state) LOGGER.info("Start [INACTIVE] rollback of CQ {}", req.cqId); TSStatus res; try { - res = env.getConfigManager().getConsensusManager().write(new DropCQPlan(req.cqId, md5)); + res = + env.getConfigManager().getConsensusManager().write(new DropCQPlan(req.cqId, cqToken)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -227,7 +273,7 @@ public void serialize(DataOutputStream stream) throws IOException { stream.writeShort(ProcedureType.CREATE_CQ_PROCEDURE.getTypeCode()); super.serialize(stream); ThriftCommonsSerDeUtils.serializeTCreateCQReq(req, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); ReadWriteIOUtils.write(firstExecutionTime, stream); } @@ -235,7 +281,7 @@ public void serialize(DataOutputStream stream) throws IOException { public void deserialize(ByteBuffer byteBuffer) { super.deserialize(byteBuffer); this.req = ThriftCommonsSerDeUtils.deserializeTCreateCQReq(byteBuffer); - this.md5 = ReadWriteIOUtils.readString(byteBuffer); + this.cqToken = ReadWriteIOUtils.readString(byteBuffer); this.firstExecutionTime = ReadWriteIOUtils.readLong(byteBuffer); } @@ -254,7 +300,7 @@ && getCycles() == that.getCycles() && isGeneratedByPipe == that.isGeneratedByPipe && firstExecutionTime == that.firstExecutionTime && Objects.equals(req, that.req) - && Objects.equals(md5, that.md5); + && Objects.equals(cqToken, that.cqToken); } @Override @@ -265,7 +311,15 @@ public int hashCode() { getCycles(), isGeneratedByPipe, req, - md5, + cqToken, firstExecutionTime); } + + public String getCqId() { + return req == null ? null : req.getCqId(); + } + + public String getCqToken() { + return cqToken; + } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java index 3b0554baa11fa..1c73fe323cec3 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java @@ -1213,7 +1213,7 @@ public void UpdateTriggerStateInTablePlanTest() throws IOException { @Test public void ActiveCQPlanTest() throws IOException { - ActiveCQPlan activeCQPlan0 = new ActiveCQPlan("testCq", "testCq_md5"); + ActiveCQPlan activeCQPlan0 = new ActiveCQPlan("testCq", "testCqToken"); ActiveCQPlan activeCQPlan1 = (ActiveCQPlan) ConfigPhysicalPlan.Factory.create(activeCQPlan0.serializeToByteBuffer()); @@ -1236,7 +1236,7 @@ public void AddCQPlanTest() throws IOException { "create cq testCq1 BEGIN select s1 into root.backup.d1.s1 from root.sg.d1 END", "Asia", "root"), - "testCq1_md5", + "testCq1Token", executionTime); AddCQPlan addCQPlan1 = (AddCQPlan) ConfigPhysicalPlan.Factory.create(addCQPlan0.serializeToByteBuffer()); @@ -1251,7 +1251,7 @@ public void DropCQPlanTest() throws IOException { (DropCQPlan) ConfigPhysicalPlan.Factory.create(dropCQPlan0.serializeToByteBuffer()); Assert.assertEquals(dropCQPlan0, dropCQPlan1); - dropCQPlan0 = new DropCQPlan("testCq1", "testCq1_md5"); + dropCQPlan0 = new DropCQPlan("testCq1", "testCq1Token"); dropCQPlan1 = (DropCQPlan) ConfigPhysicalPlan.Factory.create(dropCQPlan0.serializeToByteBuffer()); Assert.assertEquals(dropCQPlan0, dropCQPlan1); @@ -1260,7 +1260,7 @@ public void DropCQPlanTest() throws IOException { @Test public void UpdateCQLastExecTimePlanTest() throws IOException { UpdateCQLastExecTimePlan updateCQLastExecTimePlan0 = - new UpdateCQLastExecTimePlan("testCq", System.currentTimeMillis(), "testCq_md5"); + new UpdateCQLastExecTimePlan("testCq", System.currentTimeMillis(), "testCqToken"); UpdateCQLastExecTimePlan updateCQLastExecTimePlan1 = (UpdateCQLastExecTimePlan) ConfigPhysicalPlan.Factory.create(updateCQLastExecTimePlan0.serializeToByteBuffer()); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java new file mode 100644 index 0000000000000..a0bc5a523ba70 --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iotdb.confignode.cq; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.cq.TimeoutPolicy; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; +import org.apache.iotdb.confignode.manager.cq.CQManager; +import org.apache.iotdb.confignode.manager.cq.CQScheduleTask; +import org.apache.iotdb.confignode.rpc.thrift.TDropCQReq; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertTrue; + +public class CQManagerTest { + + @SuppressWarnings("unchecked") + @Test + public void dropCQShouldCancelLocallyScheduledTask() throws Exception { + ConfigManager configManager = Mockito.mock(ConfigManager.class); + ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); + Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); + Mockito.when(consensusManager.write(Mockito.any())) + .thenReturn(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + CQManager cqManager = new CQManager(configManager); + ScheduledFuture future = Mockito.mock(ScheduledFuture.class); + CQScheduleTask task = newScheduledTask(configManager, future, "token"); + + try { + assertTrue(cqManager.markCQLocallyScheduled("testCq", "token", task)); + task.submitSelf(); + cqManager.dropCQ(new TDropCQReq("testCq")); + + Mockito.verify(future).cancel(false); + } finally { + cqManager.stopCQScheduler(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void newTokenShouldCancelPreviousLocallyScheduledTask() { + ConfigManager configManager = Mockito.mock(ConfigManager.class); + CQManager cqManager = new CQManager(configManager); + ScheduledFuture previousFuture = Mockito.mock(ScheduledFuture.class); + CQScheduleTask previousTask = newScheduledTask(configManager, previousFuture, "previousToken"); + ScheduledFuture currentFuture = Mockito.mock(ScheduledFuture.class); + CQScheduleTask currentTask = newScheduledTask(configManager, currentFuture, "currentToken"); + + try { + assertTrue(cqManager.markCQLocallyScheduled("testCq", "previousToken", previousTask)); + previousTask.submitSelf(); + assertTrue(cqManager.markCQLocallyScheduled("testCq", "currentToken", currentTask)); + + Mockito.verify(previousFuture).cancel(false); + } finally { + cqManager.stopCQScheduler(); + } + } + + @SuppressWarnings("unchecked") + private CQScheduleTask newScheduledTask( + ConfigManager configManager, ScheduledFuture scheduledFuture, String cqToken) { + ScheduledExecutorService executor = Mockito.mock(ScheduledExecutorService.class); + Mockito.when( + executor.schedule( + Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class))) + .thenReturn((ScheduledFuture) scheduledFuture); + return new CQScheduleTask( + "testCq", + 1000, + 0, + 1000, + TimeoutPolicy.BLOCKED, + "select s1 into root.backup.d1.s1 from root.sg.d1", + cqToken, + "Asia", + "root", + executor, + configManager, + System.currentTimeMillis() + 10_000); + } +} diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java index bfadd3d05e8df..8bf2d12cd5941 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java @@ -18,9 +18,14 @@ */ package org.apache.iotdb.confignode.persistence; +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; +import org.apache.iotdb.confignode.consensus.request.write.cq.DropCQPlan; +import org.apache.iotdb.confignode.consensus.request.write.cq.UpdateCQLastExecTimePlan; +import org.apache.iotdb.confignode.consensus.response.cq.ShowCQResp; import org.apache.iotdb.confignode.persistence.cq.CQInfo; import org.apache.iotdb.confignode.rpc.thrift.TCreateCQReq; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.commons.io.FileUtils; import org.apache.thrift.TException; @@ -70,7 +75,7 @@ public void testSnapshot() throws TException, IOException { "create cq testCq1 BEGIN select s1 into root.backup.d1.s1 from root.sg.d1 END", "Asia", "root"), - "testCq1_md5", + "testCq1Token", executionTime); cqInfo.addCQ(addCQPlan); @@ -89,7 +94,7 @@ public void testSnapshot() throws TException, IOException { "create cq testCq2 BEGIN select s1 into root.backup.d2.s1 from root.sg.d2 END", "Asia", "root"), - "testCq2_md5", + "testCq2Token", executionTime); cqInfo.addCQ(addCQPlan); @@ -99,4 +104,59 @@ public void testSnapshot() throws TException, IOException { Assert.assertEquals(cqInfo, actualCQInfo); } + + @Test + public void testOldCallbackCannotTouchRecreatedCQ() throws Exception { + long executionTime = System.currentTimeMillis(); + TCreateCQReq req = + new TCreateCQReq( + "testCq3", + 1000, + 0, + 1000, + 0, + (byte) 0, + "select s1 into root.backup.d3.s1 from root.sg.d3", + "create cq testCq3 BEGIN select s1 into root.backup.d3.s1 from root.sg.d3 END", + "Asia", + "root"); + + cqInfo.addCQ(new AddCQPlan(req, "oldToken", executionTime)); + cqInfo.dropCQ(new DropCQPlan("testCq3")); + cqInfo.addCQ(new AddCQPlan(req, "newToken", executionTime)); + + Assert.assertEquals( + TSStatusCode.NO_SUCH_CQ.getStatusCode(), + cqInfo.updateCQLastExecutionTime( + new UpdateCQLastExecTimePlan("testCq3", executionTime + 1000, "oldToken")) + .code); + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), + cqInfo.updateCQLastExecutionTime( + new UpdateCQLastExecTimePlan("testCq3", executionTime + 1000, "newToken")) + .code); + } + + @Test + public void testShowCQCanFilterByCQId() throws Exception { + long executionTime = System.currentTimeMillis(); + TCreateCQReq req = + new TCreateCQReq( + "testCq4", + 1000, + 0, + 1000, + 0, + (byte) 0, + "select s1 into root.backup.d4.s1 from root.sg.d4", + "create cq testCq4 BEGIN select s1 into root.backup.d4.s1 from root.sg.d4 END", + "Asia", + "root"); + cqInfo.addCQ(new AddCQPlan(req, "testCq4Token", executionTime)); + + ShowCQResp showCQResp = cqInfo.showCQ(new ShowCQPlan("testCq4")); + + Assert.assertEquals(1, showCQResp.getCqList().size()); + Assert.assertEquals("testCq4", showCQResp.getCqList().get(0).getCqId()); + } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java index d0e92b3281666..3e7fd2052ad53 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java @@ -36,10 +36,36 @@ import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.fail; public class CreateCQProcedureTest { + @Test + public void tokenShouldBeUniqueForSameCQId() { + ScheduledExecutorService executor = Executors.newScheduledThreadPool(1); + try { + TCreateCQReq req = + new TCreateCQReq( + "testCq1", + 1000, + 0, + 1000, + 0, + (byte) 0, + "select s1 into root.backup.d1(s1) from root.sg.d1", + "create cq testCq1 BEGIN select s1 into root.backup.d1(s1) from root.sg.d1 END", + "Asia", + "root"); + CreateCQProcedure createCQProcedure1 = new CreateCQProcedure(req, executor); + CreateCQProcedure createCQProcedure2 = new CreateCQProcedure(req, executor); + + assertNotEquals(createCQProcedure1.getCqToken(), createCQProcedure2.getCqToken()); + } finally { + executor.shutdown(); + } + } + @Test public void serializeDeserializeTest() { diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java new file mode 100644 index 0000000000000..a90e282494f0e --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.impl.cq; + +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; +import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; +import org.apache.iotdb.confignode.manager.cq.CQManager; +import org.apache.iotdb.confignode.manager.cq.CQScheduleTask; +import org.apache.iotdb.confignode.persistence.cq.CQInfo; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; +import org.apache.iotdb.confignode.rpc.thrift.TCreateCQReq; + +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +public class CreateCQProcedureRecoveryTest { + + private TCreateCQReq newCreateCQReq() { + return new TCreateCQReq( + "testCq1", + 1000, + 0, + 1000, + 0, + (byte) 0, + "select s1 into root.backup.d1.s1 from root.sg.d1", + "create cq testCq1 BEGIN select s1 into root.backup.d1.s1 from root.sg.d1 END", + "Asia", + "root"); + } + + @SuppressWarnings("unchecked") + @Test + public void recoverScheduledTaskShouldResubmitFromLatestMetadata() throws Exception { + ScheduledExecutorService executor = Mockito.mock(ScheduledExecutorService.class); + Mockito.when( + executor.schedule( + Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class))) + .thenReturn(Mockito.mock(ScheduledFuture.class)); + + ConfigManager configManager = Mockito.mock(ConfigManager.class); + ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); + CQManager cqManager = Mockito.mock(CQManager.class); + ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); + Mockito.when(configManager.getCQManager()).thenReturn(cqManager); + Mockito.when( + cqManager.markCQLocallyScheduled( + Mockito.anyString(), Mockito.anyString(), Mockito.any(CQScheduleTask.class))) + .thenReturn(true); + + TCreateCQReq req = newCreateCQReq(); + CreateCQProcedure procedure = new CreateCQProcedure(req, executor); + + CQInfo cqInfo = new CQInfo(); + cqInfo.addCQ(new AddCQPlan(req, procedure.getCqToken(), System.currentTimeMillis() + 10_000)); + Mockito.when(consensusManager.read(Mockito.any(ShowCQPlan.class))).thenReturn(cqInfo.showCQ()); + + procedure.recoverScheduledTask(env); + + Mockito.verify(executor) + .schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class)); + } + + @SuppressWarnings("unchecked") + @Test + public void recoverScheduledTaskShouldSkipDuplicatedLocalSchedule() throws Exception { + ScheduledExecutorService executor = Mockito.mock(ScheduledExecutorService.class); + ConfigManager configManager = Mockito.mock(ConfigManager.class); + ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); + CQManager cqManager = Mockito.mock(CQManager.class); + ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); + Mockito.when(configManager.getCQManager()).thenReturn(cqManager); + Mockito.when( + cqManager.markCQLocallyScheduled( + Mockito.anyString(), Mockito.anyString(), Mockito.any(CQScheduleTask.class))) + .thenReturn(false); + + TCreateCQReq req = newCreateCQReq(); + CreateCQProcedure procedure = new CreateCQProcedure(req, executor); + + CQInfo cqInfo = new CQInfo(); + cqInfo.addCQ(new AddCQPlan(req, procedure.getCqToken(), System.currentTimeMillis() + 10_000)); + Mockito.when(consensusManager.read(Mockito.any(ShowCQPlan.class))).thenReturn(cqInfo.showCQ()); + + procedure.recoverScheduledTask(env); + + Mockito.verify(executor, Mockito.never()) + .schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class)); + } +} From b0033afcaf55585166c99bd8a31d90991fa1f5ae Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 3 Jun 2026 11:30:47 +0800 Subject: [PATCH 075/102] Optimized the overall performance of IoTDB & Fixed the NPE in LimitOperatorTest (#17664) (#17819) * Opt * Update UnclosedFileScanHandleImpl.java * Update StorageEngine.java * Update ClosedFileScanHandleImpl.java * column index * spt * Address performance review comments * fix (cherry picked from commit 89730b14397d675f2255dc2a8e8069168d18c7c3) --- .../client-cpp/src/main/SessionDataSet.cpp | 26 +-- .../apache/iotdb/isession/SessionDataSet.java | 39 ++-- .../apache/iotdb/rpc/IoTDBJDBCDataSet.java | 47 +++-- .../org/apache/iotdb/rpc/IoTDBRpcDataSet.java | 8 + .../org/apache/iotdb/session/Session.java | 130 ++++++++----- .../cache/schema/DeviceSchemaCache.java | 2 +- .../iotdb/db/schemaengine/SchemaEngine.java | 17 +- .../iotdb/db/storageengine/StorageEngine.java | 16 +- .../impl/ClosedFileScanHandleImpl.java | 45 +++-- .../impl/UnclosedFileScanHandleImpl.java | 83 +++++--- .../dataregion/tsfile/TsFileManager.java | 21 +- .../iotdb/db/utils/ModificationUtils.java | 19 +- .../iotdb/db/utils/QueryDataSetUtils.java | 179 ++++-------------- .../cache/DataNodeSchemaCacheTest.java | 37 ++++ .../execution/operator/LimitOperatorTest.java | 3 + 15 files changed, 363 insertions(+), 309 deletions(-) diff --git a/iotdb-client/client-cpp/src/main/SessionDataSet.cpp b/iotdb-client/client-cpp/src/main/SessionDataSet.cpp index 30e11fa1efb22..fe28714419a83 100644 --- a/iotdb-client/client-cpp/src/main/SessionDataSet.cpp +++ b/iotdb-client/client-cpp/src/main/SessionDataSet.cpp @@ -211,39 +211,41 @@ const std::vector& SessionDataSet::DataIterator::getColumnTypeList( shared_ptr SessionDataSet::constructRowRecordFromValueArray() { std::vector outFields; - for (int i = iotdbRpcDataSet_->getValueColumnStartIndex(); i < iotdbRpcDataSet_->getColumnSize(); i++) { + const int32_t valueColumnStartIndex = iotdbRpcDataSet_->getValueColumnStartIndex(); + const int32_t columnSize = iotdbRpcDataSet_->getColumnSize(); + outFields.reserve(columnSize - valueColumnStartIndex); + for (int32_t columnIndex = valueColumnStartIndex + 1; columnIndex <= columnSize; ++columnIndex) { Field field; - std::string columnName = iotdbRpcDataSet_->getColumnNameList().at(i); - if (!iotdbRpcDataSet_->isNullByColumnName(columnName)) { - TSDataType::TSDataType dataType = iotdbRpcDataSet_->getDataType(columnName); + if (!iotdbRpcDataSet_->isNullByIndex(columnIndex)) { + TSDataType::TSDataType dataType = iotdbRpcDataSet_->getDataTypeByIndex(columnIndex); field.dataType = dataType; switch (dataType) { case TSDataType::BOOLEAN: - field.boolV = iotdbRpcDataSet_->getBoolean(columnName); + field.boolV = iotdbRpcDataSet_->getBooleanByIndex(columnIndex); break; case TSDataType::INT32: - field.intV = iotdbRpcDataSet_->getInt(columnName); + field.intV = iotdbRpcDataSet_->getIntByIndex(columnIndex); break; case TSDataType::DATE: - field.dateV = parseIntToDate(iotdbRpcDataSet_->getInt(columnName)); + field.dateV = iotdbRpcDataSet_->getDateByIndex(columnIndex); break; case TSDataType::INT64: case TSDataType::TIMESTAMP: - field.longV = iotdbRpcDataSet_->getLong(columnName); + field.longV = iotdbRpcDataSet_->getLongByIndex(columnIndex); break; case TSDataType::FLOAT: - field.floatV = iotdbRpcDataSet_->getFloat(columnName); + field.floatV = iotdbRpcDataSet_->getFloatByIndex(columnIndex); break; case TSDataType::DOUBLE: - field.doubleV = iotdbRpcDataSet_->getDouble(columnName); + field.doubleV = iotdbRpcDataSet_->getDoubleByIndex(columnIndex); break; case TSDataType::TEXT: case TSDataType::BLOB: case TSDataType::STRING: - field.stringV = iotdbRpcDataSet_->getBinary(columnName)->getStringValue(); + field.stringV = iotdbRpcDataSet_->getBinaryByIndex(columnIndex)->getStringValue(); break; default: - throw new UnSupportedDataTypeException("Data type %s is not supported." + dataType); + throw UnSupportedDataTypeException("Data type %s is not supported." + dataType); } } outFields.emplace_back(field); diff --git a/iotdb-client/isession/src/main/java/org/apache/iotdb/isession/SessionDataSet.java b/iotdb-client/isession/src/main/java/org/apache/iotdb/isession/SessionDataSet.java index 822c464ddcadf..fc671b83926dd 100644 --- a/iotdb-client/isession/src/main/java/org/apache/iotdb/isession/SessionDataSet.java +++ b/iotdb-client/isession/src/main/java/org/apache/iotdb/isession/SessionDataSet.java @@ -40,8 +40,6 @@ import java.util.List; import java.util.Map; -import static org.apache.iotdb.rpc.IoTDBRpcDataSet.START_INDEX; - public class SessionDataSet implements ISessionDataSet { private final IoTDBRpcDataSet ioTDBRpcDataSet; @@ -176,56 +174,45 @@ public boolean hasNext() throws StatementExecutionException, IoTDBConnectionExce } private RowRecord constructRowRecordFromValueArray() throws StatementExecutionException { - List outFields = new ArrayList<>(); - for (int i = 0; i < ioTDBRpcDataSet.columnSize; i++) { + int valueColumnStartIndex = ioTDBRpcDataSet.getValueColumnStartIndex(); + int columnSize = ioTDBRpcDataSet.getColumnSize(); + List outFields = new ArrayList<>(columnSize - valueColumnStartIndex); + for (int columnIndex = valueColumnStartIndex + 1; columnIndex <= columnSize; columnIndex++) { Field field; - - int index = i + 1; - int datasetColumnIndex = i + START_INDEX; - if (ioTDBRpcDataSet.ignoreTimeStamp) { - index--; - datasetColumnIndex--; - } - int loc = - ioTDBRpcDataSet.columnOrdinalMap.get(ioTDBRpcDataSet.columnNameList.get(index)) - - START_INDEX; - - if (!ioTDBRpcDataSet.isNull(datasetColumnIndex)) { - TSDataType dataType = ioTDBRpcDataSet.columnTypeDeduplicatedList.get(loc); + if (!ioTDBRpcDataSet.isNull(columnIndex)) { + TSDataType dataType = ioTDBRpcDataSet.getDataType(columnIndex); field = new Field(dataType); switch (dataType) { case BOOLEAN: - boolean booleanValue = ioTDBRpcDataSet.getBoolean(datasetColumnIndex); + boolean booleanValue = ioTDBRpcDataSet.getBoolean(columnIndex); field.setBoolV(booleanValue); break; case INT32: case DATE: - int intValue = ioTDBRpcDataSet.getInt(datasetColumnIndex); + int intValue = ioTDBRpcDataSet.getInt(columnIndex); field.setIntV(intValue); break; case INT64: case TIMESTAMP: - long longValue = ioTDBRpcDataSet.getLong(datasetColumnIndex); + long longValue = ioTDBRpcDataSet.getLong(columnIndex); field.setLongV(longValue); break; case FLOAT: - float floatValue = ioTDBRpcDataSet.getFloat(datasetColumnIndex); + float floatValue = ioTDBRpcDataSet.getFloat(columnIndex); field.setFloatV(floatValue); break; case DOUBLE: - double doubleValue = ioTDBRpcDataSet.getDouble(datasetColumnIndex); + double doubleValue = ioTDBRpcDataSet.getDouble(columnIndex); field.setDoubleV(doubleValue); break; case TEXT: case BLOB: case STRING: - field.setBinaryV(ioTDBRpcDataSet.getBinary(datasetColumnIndex)); + field.setBinaryV(ioTDBRpcDataSet.getBinary(columnIndex)); break; default: throw new UnSupportedDataTypeException( - String.format( - "Data type %s is not supported.", - ioTDBRpcDataSet.columnTypeDeduplicatedList.get(i))); + String.format("Data type %s is not supported.", dataType)); } } else { field = new Field(null); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBJDBCDataSet.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBJDBCDataSet.java index 5bba6d0ea718d..254c6d685982f 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBJDBCDataSet.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBJDBCDataSet.java @@ -127,20 +127,15 @@ public IoTDBJDBCDataSet( // deduplicate and map if (columnNameIndex != null) { - int deduplicatedColumnSize = (int) columnNameIndex.values().stream().distinct().count(); - this.columnTypeDeduplicatedList = new ArrayList<>(deduplicatedColumnSize); - for (int i = 0; i < deduplicatedColumnSize; i++) { - columnTypeDeduplicatedList.add(null); - } + this.columnTypeDeduplicatedList = + initDeduplicatedColumnTypes(getDeduplicatedColumnSize(columnNameIndex)); for (int i = 0; i < columnNameList.size(); i++) { String name = columnNameList.get(i); this.columnNameList.add(name); this.columnTypeList.add(columnTypeList.get(i)); if (!columnOrdinalMap.containsKey(name)) { int index = columnNameIndex.get(name); - if (!columnOrdinalMap.containsValue(index + START_INDEX)) { - columnTypeDeduplicatedList.set(index, TSDataType.valueOf(columnTypeList.get(i))); - } + setColumnTypeIfAbsent(columnTypeDeduplicatedList, index, columnTypeList.get(i)); columnOrdinalMap.put(name, index + START_INDEX); } } @@ -241,11 +236,8 @@ public IoTDBJDBCDataSet( // deduplicate and map if (columnNameIndex != null) { - int deduplicatedColumnSize = (int) columnNameIndex.values().stream().distinct().count(); - this.columnTypeDeduplicatedList = new ArrayList<>(deduplicatedColumnSize); - for (int i = 0; i < deduplicatedColumnSize; i++) { - columnTypeDeduplicatedList.add(null); - } + this.columnTypeDeduplicatedList = + initDeduplicatedColumnTypes(getDeduplicatedColumnSize(columnNameIndex)); for (int i = 0; i < columnNameList.size(); i++) { String name = ""; if (sgList != null @@ -261,9 +253,7 @@ public IoTDBJDBCDataSet( // "Time".equals(name) -> to allow the Time column appear in value columns if (!columnOrdinalMap.containsKey(name) || "Time".equals(name)) { int index = columnNameIndex.get(name); - if (!columnOrdinalMap.containsValue(index + START_INDEX)) { - columnTypeDeduplicatedList.set(index, TSDataType.valueOf(columnTypeList.get(i))); - } + setColumnTypeIfAbsent(columnTypeDeduplicatedList, index, columnTypeList.get(i)); columnOrdinalMap.put(name, index + START_INDEX); } } @@ -318,6 +308,31 @@ public IoTDBJDBCDataSet( this.emptyResultSet = (queryDataSet == null || !queryDataSet.time.hasRemaining()); } + private static int getDeduplicatedColumnSize(Map columnNameIndex) { + int deduplicatedColumnSize = 0; + for (Integer index : columnNameIndex.values()) { + if (index != null && index + 1 > deduplicatedColumnSize) { + deduplicatedColumnSize = index + 1; + } + } + return deduplicatedColumnSize; + } + + private static List initDeduplicatedColumnTypes(int deduplicatedColumnSize) { + List columnTypes = new ArrayList<>(deduplicatedColumnSize); + for (int i = 0; i < deduplicatedColumnSize; i++) { + columnTypes.add(null); + } + return columnTypes; + } + + private static void setColumnTypeIfAbsent( + List columnTypeDeduplicatedList, int index, String columnType) { + if (columnTypeDeduplicatedList.get(index) == null) { + columnTypeDeduplicatedList.set(index, TSDataType.valueOf(columnType)); + } + } + public void close() throws StatementExecutionException, TException { if (isClosed) { return; diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBRpcDataSet.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBRpcDataSet.java index 8fc484fd00c0a..2ff01502fb311 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBRpcDataSet.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/IoTDBRpcDataSet.java @@ -634,4 +634,12 @@ public void checkRecord() throws StatementExecutionException { throw new StatementExecutionException("No record remains"); } } + + public int getValueColumnStartIndex() { + return ignoreTimeStamp ? 0 : 1; + } + + public int getColumnSize() { + return columnNameList.size(); + } } diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java index 461aef3624511..c8d26f0cefe68 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/Session.java @@ -788,13 +788,12 @@ private TSCreateAlignedTimeseriesReq getTSCreateAlignedTimeseriesReq( TSCreateAlignedTimeseriesReq request = new TSCreateAlignedTimeseriesReq(); request.setPrefixPath(prefixPath); request.setMeasurements(measurements); - request.setDataTypes(dataTypes.stream().map(TSDataType::ordinal).collect(Collectors.toList())); - request.setEncodings(encodings.stream().map(TSEncoding::ordinal).collect(Collectors.toList())); - request.setCompressors( - compressors.stream().map(i -> (int) i.serialize()).collect(Collectors.toList())); - request.setMeasurementAlias(measurementAliasList); - request.setTagsList(tagsList); - request.setAttributesList(attributesList); + request.setDataTypes(toDataTypeOrdinals(dataTypes)); + request.setEncodings(toEncodingOrdinals(encodings)); + request.setCompressors(toCompressionOrdinals(compressors)); + request.setMeasurementAlias(replaceNullStrings(measurementAliasList)); + request.setTagsList(replaceNullMaps(tagsList)); + request.setAttributesList(replaceNullMaps(attributesList)); return request; } @@ -834,29 +833,14 @@ private TSCreateMultiTimeseriesReq genTSCreateMultiTimeseriesReq( TSCreateMultiTimeseriesReq request = new TSCreateMultiTimeseriesReq(); request.setPaths(paths); - - List dataTypeOrdinals = new ArrayList<>(dataTypes.size()); - for (TSDataType dataType : dataTypes) { - dataTypeOrdinals.add(dataType.ordinal()); - } - request.setDataTypes(dataTypeOrdinals); - - List encodingOrdinals = new ArrayList<>(dataTypes.size()); - for (TSEncoding encoding : encodings) { - encodingOrdinals.add(encoding.ordinal()); - } - request.setEncodings(encodingOrdinals); - - List compressionOrdinals = new ArrayList<>(paths.size()); - for (CompressionType compression : compressors) { - compressionOrdinals.add((int) compression.serialize()); - } - request.setCompressors(compressionOrdinals); + request.setDataTypes(toDataTypeOrdinals(dataTypes)); + request.setEncodings(toEncodingOrdinals(encodings)); + request.setCompressors(toCompressionOrdinals(compressors)); request.setPropsList(propsList); - request.setTagsList(tagsList); - request.setAttributesList(attributesList); - request.setMeasurementAliasList(measurementAliasList); + request.setTagsList(replaceNullMaps(tagsList)); + request.setAttributesList(replaceNullMaps(attributesList)); + request.setMeasurementAliasList(replaceNullStrings(measurementAliasList)); return request; } @@ -1675,19 +1659,23 @@ private boolean filterNullValueAndMeasurement( List measurementsList, List types, List valuesList) { - Map nullMap = new HashMap<>(); + Map nullMap = logger.isInfoEnabled() ? new HashMap<>() : null; for (int i = valuesList.size() - 1; i >= 0; i--) { if (valuesList.get(i) == null) { - nullMap.put(measurementsList.get(i), valuesList.get(i)); + if (nullMap != null) { + nullMap.put(measurementsList.get(i), valuesList.get(i)); + } valuesList.remove(i); measurementsList.remove(i); types.remove(i); } } if (valuesList.isEmpty()) { - logger.info("All values of the {} are null,null values are {}", deviceId, nullMap); + if (nullMap != null) { + logger.info("All values of the {} are null,null values are {}", deviceId, nullMap); + } return true; - } else { + } else if (nullMap != null) { logger.info("Some values of {} are null,null values are {}", deviceId, nullMap); } return false; @@ -1733,18 +1721,22 @@ private void filterNullValueAndMeasurementWithStringType( */ private boolean filterNullValueAndMeasurementWithStringType( List valuesList, String deviceId, List measurementsList) { - Map nullMap = new HashMap<>(); + Map nullMap = logger.isInfoEnabled() ? new HashMap<>() : null; for (int i = valuesList.size() - 1; i >= 0; i--) { if (valuesList.get(i) == null) { - nullMap.put(measurementsList.get(i), valuesList.get(i)); + if (nullMap != null) { + nullMap.put(measurementsList.get(i), valuesList.get(i)); + } valuesList.remove(i); measurementsList.remove(i); } } if (valuesList.isEmpty()) { - logger.info("All values of the {} are null,null values are {}", deviceId, nullMap); + if (nullMap != null) { + logger.info("All values of the {} are null,null values are {}", deviceId, nullMap); + } return true; - } else { + } else if (nullMap != null) { logger.info("Some values of {} are null,null values are {}", deviceId, nullMap); } return false; @@ -2481,7 +2473,57 @@ private TSInsertStringRecordsOfOneDeviceReq genTSInsertStringRecordsOfOneDeviceR * @return ordered list */ private static List sortList(List source, Integer[] index) { - return Arrays.stream(index).map(source::get).collect(Collectors.toList()); + List sortedList = new ArrayList<>(index.length); + for (int position : index) { + sortedList.add(source.get(position)); + } + return sortedList; + } + + private static List toDataTypeOrdinals(List dataTypes) { + List ordinals = new ArrayList<>(dataTypes.size()); + for (TSDataType dataType : dataTypes) { + ordinals.add(dataType.ordinal()); + } + return ordinals; + } + + private static List toEncodingOrdinals(List encodings) { + List ordinals = new ArrayList<>(encodings.size()); + for (TSEncoding encoding : encodings) { + ordinals.add(encoding.ordinal()); + } + return ordinals; + } + + private static List toCompressionOrdinals(List compressors) { + List ordinals = new ArrayList<>(compressors.size()); + for (CompressionType compression : compressors) { + ordinals.add((int) compression.serialize()); + } + return ordinals; + } + + private static List replaceNullStrings(List values) { + if (values == null) { + return null; + } + List replacedValues = new ArrayList<>(values.size()); + for (String value : values) { + replacedValues.add(value != null ? value : ""); + } + return replacedValues; + } + + private static List> replaceNullMaps(List> values) { + if (values == null) { + return null; + } + List> replacedValues = new ArrayList<>(values.size()); + for (Map value : values) { + replacedValues.add(value != null ? value : new HashMap<>()); + } + return replacedValues; } private List objectValuesListToByteBufferList( @@ -3523,10 +3565,9 @@ public void addAlignedMeasurementsInTemplate( TSAppendSchemaTemplateReq req = new TSAppendSchemaTemplateReq(); req.setName(templateName); req.setMeasurements(measurementsPath); - req.setDataTypes(dataTypes.stream().map(TSDataType::ordinal).collect(Collectors.toList())); - req.setEncodings(encodings.stream().map(TSEncoding::ordinal).collect(Collectors.toList())); - req.setCompressors( - compressors.stream().map(i -> (int) i.serialize()).collect(Collectors.toList())); + req.setDataTypes(toDataTypeOrdinals(dataTypes)); + req.setEncodings(toEncodingOrdinals(encodings)); + req.setCompressors(toCompressionOrdinals(compressors)); req.setIsAligned(true); defaultSessionConnection.appendSchemaTemplate(req); } @@ -3569,10 +3610,9 @@ public void addUnalignedMeasurementsInTemplate( TSAppendSchemaTemplateReq req = new TSAppendSchemaTemplateReq(); req.setName(templateName); req.setMeasurements(measurementsPath); - req.setDataTypes(dataTypes.stream().map(TSDataType::ordinal).collect(Collectors.toList())); - req.setEncodings(encodings.stream().map(TSEncoding::ordinal).collect(Collectors.toList())); - req.setCompressors( - compressors.stream().map(i -> (int) i.serialize()).collect(Collectors.toList())); + req.setDataTypes(toDataTypeOrdinals(dataTypes)); + req.setEncodings(toEncodingOrdinals(encodings)); + req.setCompressors(toCompressionOrdinals(compressors)); req.setIsAligned(false); defaultSessionConnection.appendSchemaTemplate(req); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/schema/DeviceSchemaCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/schema/DeviceSchemaCache.java index ecbb2d3aea25a..9bcb020b72fc8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/schema/DeviceSchemaCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/schema/DeviceSchemaCache.java @@ -165,7 +165,7 @@ void invalidateLastCache(final PartialPath devicePath, final String measurement) // the un-related paths being cleared, like "root.*.b.c.**" affects // "root.*.d.c.**", thereby lower the query performance. dualKeyCache.update( - cachedDeviceID -> cachedDeviceID.matchFullPath(devicePath), updateFunction); + cachedDeviceID -> devicePath.matchFullPath(cachedDeviceID), updateFunction); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/SchemaEngine.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/SchemaEngine.java index 9b766b4c5140e..1e185e40cda15 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/SchemaEngine.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/SchemaEngine.java @@ -55,6 +55,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -365,13 +366,15 @@ public int getSchemaRegionNumber() { return schemaRegionMap.size(); } - public Map countDeviceNumBySchemaRegion(List schemaIds) { - Map deviceNum = new HashMap<>(); + public Map countDeviceNumBySchemaRegion(final List schemaIds) { + final Map deviceNum = new HashMap<>(); + final Collection targetSchemaIds = + schemaIds.size() > 1 ? new HashSet<>(schemaIds) : schemaIds; schemaRegionMap.entrySet().stream() .filter( entry -> - schemaIds.contains(entry.getKey().getId()) + targetSchemaIds.contains(entry.getKey().getId()) && SchemaRegionConsensusImpl.getInstance().isLeader(entry.getKey())) .forEach( entry -> @@ -381,12 +384,14 @@ public Map countDeviceNumBySchemaRegion(List schemaIds) return deviceNum; } - public Map countTimeSeriesNumBySchemaRegion(List schemaIds) { - Map timeSeriesNum = new HashMap<>(); + public Map countTimeSeriesNumBySchemaRegion(final List schemaIds) { + final Map timeSeriesNum = new HashMap<>(); + final Collection targetSchemaIds = + schemaIds.size() > 1 ? new HashSet<>(schemaIds) : schemaIds; schemaRegionMap.entrySet().stream() .filter( entry -> - schemaIds.contains(entry.getKey().getId()) + targetSchemaIds.contains(entry.getKey().getId()) && SchemaRegionConsensusImpl.getInstance().isLeader(entry.getKey())) .forEach( entry -> diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java index c8439e0218399..1c772cdb20b47 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java @@ -90,7 +90,9 @@ import java.net.URL; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -1038,12 +1040,14 @@ private void stopTimedServiceAndThrow(ScheduledExecutorService pool, String pool public void getDiskSizeByDataRegion( Map dataRegionDisk, List dataRegionIds) { - dataRegionMap.forEach( - (dataRegionId, dataRegion) -> { - if (dataRegionIds.contains(dataRegionId.getId())) { - dataRegionDisk.put(dataRegionId.getId(), dataRegion.countRegionDiskSize()); - } - }); + final Collection targetDataRegionIds = + dataRegionIds.size() > 1 ? new HashSet<>(dataRegionIds) : dataRegionIds; + for (Integer dataRegionId : targetDataRegionIds) { + final DataRegion dataRegion = dataRegionMap.get(new DataRegionId(dataRegionId)); + if (dataRegion != null) { + dataRegionDisk.put(dataRegionId, dataRegion.countRegionDiskSize()); + } + } } public static File getDataRegionSystemDir(String dataBaseName, String dataRegionId) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/ClosedFileScanHandleImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/ClosedFileScanHandleImpl.java index 055b8517b48e8..1552b7d81fb0a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/ClosedFileScanHandleImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/ClosedFileScanHandleImpl.java @@ -53,11 +53,11 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; public class ClosedFileScanHandleImpl implements IFileScanHandle { @@ -65,13 +65,16 @@ public class ClosedFileScanHandleImpl implements IFileScanHandle { private final TsFileResource tsFileResource; private final QueryContext queryContext; private PatternTreeMap curFileMods = null; + // Used to cache the device-level modifications + private final Map> deviceToDeletionRanges; // Used to cache the modifications of each timeseries private final Map>> deviceToModifications; public ClosedFileScanHandleImpl(TsFileResource tsFileResource, QueryContext context) { this.tsFileResource = tsFileResource; this.queryContext = context; - this.deviceToModifications = new HashMap<>(); + this.deviceToDeletionRanges = new ConcurrentHashMap<>(); + this.deviceToModifications = new ConcurrentHashMap<>(); } @Override @@ -89,14 +92,26 @@ public boolean isDeviceTimeDeleted(IDeviceID deviceID, long timestamp) curFileMods != null ? curFileMods : queryContext.loadAllModificationsFromDisk(tsFileResource); - List modifications = queryContext.getPathModifications(curFileMods, deviceID); - List timeRangeList = - modifications.stream() - .filter(Deletion.class::isInstance) - .map(Deletion.class::cast) - .map(Deletion::getTimeRange) - .collect(Collectors.toList()); - return ModificationUtils.isPointDeletedWithoutOrderedRange(timestamp, timeRangeList); + List timeRangeList = deviceToDeletionRanges.get(deviceID); + if (timeRangeList == null) { + final List computedTimeRangeList = + getMergedTimeRanges(queryContext.getPathModifications(curFileMods, deviceID)); + final List existingTimeRangeList = + deviceToDeletionRanges.putIfAbsent(deviceID, computedTimeRangeList); + timeRangeList = existingTimeRangeList == null ? computedTimeRangeList : existingTimeRangeList; + } + return ModificationUtils.isPointDeleted(timestamp, timeRangeList); + } + + private static List getMergedTimeRanges(List modifications) { + List timeRangeList = new ArrayList<>(modifications.size()); + for (Modification modification : modifications) { + if (modification instanceof Deletion) { + timeRangeList.add(((Deletion) modification).getTimeRange()); + } + } + TimeRange.sortAndMerge(timeRangeList); + return timeRangeList; } @Override @@ -114,15 +129,9 @@ public boolean isTimeSeriesTimeDeleted(IDeviceID deviceID, String timeSeriesName : queryContext.loadAllModificationsFromDisk(tsFileResource); List modifications = queryContext.getPathModifications(curFileMods, new PartialPath(deviceID, timeSeriesName)); - List timeRangeList = - modifications.stream() - .filter(Deletion.class::isInstance) - .map(Deletion.class::cast) - .map(Deletion::getTimeRange) - .collect(Collectors.toList()); - TimeRange.sortAndMerge(timeRangeList); + List timeRangeList = getMergedTimeRanges(modifications); deviceToModifications - .computeIfAbsent(deviceID, k -> new HashMap<>()) + .computeIfAbsent(deviceID, k -> new ConcurrentHashMap<>()) .put(timeSeriesName, timeRangeList); return ModificationUtils.isPointDeleted(timestamp, timeRangeList); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/UnclosedFileScanHandleImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/UnclosedFileScanHandleImpl.java index 20ae0958a87d4..ef74c0e43369f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/UnclosedFileScanHandleImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/filescan/impl/UnclosedFileScanHandleImpl.java @@ -29,6 +29,7 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.DeviceTimeIndex; import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.ITimeIndex; import org.apache.iotdb.db.storageengine.dataregion.utils.TsFileDeviceStartEndTimeIterator; +import org.apache.iotdb.db.utils.ModificationUtils; import org.apache.tsfile.file.metadata.AlignedChunkMetadata; import org.apache.tsfile.file.metadata.IChunkMetadata; @@ -39,15 +40,19 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; public class UnclosedFileScanHandleImpl implements IFileScanHandle { private final TsFileResource tsFileResource; private final Map>> deviceToChunkMetadataMap; private final Map>> deviceToMemChunkHandleMap; + private final Map> deviceToDeletionRanges; + private final Map>> deviceToTimeSeriesDeletionRanges; public UnclosedFileScanHandleImpl( Map>> deviceToChunkMetadataMap, @@ -56,6 +61,8 @@ public UnclosedFileScanHandleImpl( this.deviceToChunkMetadataMap = deviceToChunkMetadataMap; this.deviceToMemChunkHandleMap = deviceToMemChunkHandleMap; this.tsFileResource = tsFileResource; + this.deviceToDeletionRanges = new ConcurrentHashMap<>(); + this.deviceToTimeSeriesDeletionRanges = new ConcurrentHashMap<>(); } @Override @@ -68,19 +75,9 @@ public TsFileDeviceStartEndTimeIterator getDeviceStartEndTimeIterator() throws I @Override public boolean isDeviceTimeDeleted(IDeviceID deviceID, long timeArray) { - Map> chunkMetadataMap = deviceToChunkMetadataMap.get(deviceID); - for (List chunkMetadataList : chunkMetadataMap.values()) { - for (IChunkMetadata chunkMetadata : chunkMetadataList) { - if (chunkMetadata.getDeleteIntervalList() != null) { - for (TimeRange deleteInterval : chunkMetadata.getDeleteIntervalList()) { - if (deleteInterval.contains(timeArray)) { - return true; - } - } - } - } - } - return false; + List deletionRanges = + deviceToDeletionRanges.computeIfAbsent(deviceID, this::collectDeviceDeletionRanges); + return ModificationUtils.isPointDeleted(timeArray, deletionRanges); } @Override @@ -121,19 +118,13 @@ public Iterator getAllDeviceChunkMetaData() throws @Override public boolean isTimeSeriesTimeDeleted( IDeviceID deviceID, String timeSeriesName, long timestamp) { - List chunkMetadataList = - deviceToChunkMetadataMap.get(deviceID).get(timeSeriesName); - // check if timestamp is deleted by deleteInterval - for (IChunkMetadata chunkMetadata : chunkMetadataList) { - if (chunkMetadata.getDeleteIntervalList() != null) { - for (TimeRange deleteInterval : chunkMetadata.getDeleteIntervalList()) { - if (deleteInterval.contains(timestamp)) { - return true; - } - } - } - } - return false; + Map> timeSeriesDeletionRanges = + deviceToTimeSeriesDeletionRanges.computeIfAbsent( + deviceID, key -> new ConcurrentHashMap<>()); + List deletionRanges = + timeSeriesDeletionRanges.computeIfAbsent( + timeSeriesName, key -> collectTimeSeriesDeletionRanges(deviceID, key)); + return ModificationUtils.isPointDeleted(timestamp, deletionRanges); } @Override @@ -167,4 +158,44 @@ public boolean isDeleted() { public TsFileResource getTsResource() { return tsFileResource; } + + private List collectDeviceDeletionRanges(IDeviceID deviceID) { + Map> chunkMetadataMap = deviceToChunkMetadataMap.get(deviceID); + if (chunkMetadataMap == null || chunkMetadataMap.isEmpty()) { + return Collections.emptyList(); + } + List deletionRanges = new ArrayList<>(); + for (List chunkMetadataList : chunkMetadataMap.values()) { + appendDeletionRanges(deletionRanges, chunkMetadataList); + } + TimeRange.sortAndMerge(deletionRanges); + return deletionRanges; + } + + private List collectTimeSeriesDeletionRanges( + IDeviceID deviceID, String timeSeriesName) { + Map> chunkMetadataMap = deviceToChunkMetadataMap.get(deviceID); + if (chunkMetadataMap == null) { + return Collections.emptyList(); + } + List chunkMetadataList = chunkMetadataMap.get(timeSeriesName); + if (chunkMetadataList == null || chunkMetadataList.isEmpty()) { + return Collections.emptyList(); + } + List deletionRanges = new ArrayList<>(); + appendDeletionRanges(deletionRanges, chunkMetadataList); + TimeRange.sortAndMerge(deletionRanges); + return deletionRanges; + } + + private void appendDeletionRanges( + List deletionRanges, List chunkMetadataList) { + for (IChunkMetadata chunkMetadata : chunkMetadataList) { + if (chunkMetadata.getDeleteIntervalList() != null) { + for (TimeRange deletionRange : chunkMetadata.getDeleteIntervalList()) { + deletionRanges.add(new TimeRange(deletionRange.getMin(), deletionRange.getMax())); + } + } + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java index 1c83153499147..be7052536ed53 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java @@ -191,14 +191,7 @@ public Iterator getIterator(boolean sequence) { public void remove(TsFileResource tsFileResource, boolean sequence) { writeLock("remove"); try { - Map selectedMap = sequence ? sequenceFiles : unsequenceFiles; - for (Map.Entry entry : selectedMap.entrySet()) { - if (entry.getValue().contains(tsFileResource)) { - entry.getValue().remove(tsFileResource); - TsFileResourceManager.getInstance().removeTsFileResource(tsFileResource); - break; - } - } + removeFromPartitionFileList(tsFileResource, sequence); } finally { writeUnlock(); } @@ -208,10 +201,18 @@ public void removeAll(List tsFileResourceList, boolean sequence) writeLock("removeAll"); try { for (TsFileResource resource : tsFileResourceList) { - remove(resource, sequence); + removeFromPartitionFileList(resource, sequence); } } finally { - writeLock("removeAll"); + writeUnlock(); + } + } + + private void removeFromPartitionFileList(TsFileResource tsFileResource, boolean sequence) { + Map selectedMap = sequence ? sequenceFiles : unsequenceFiles; + TsFileResourceList tsFileResources = selectedMap.get(tsFileResource.getTimePartition()); + if (tsFileResources != null && tsFileResources.remove(tsFileResource)) { + TsFileResourceManager.getInstance().removeTsFileResource(tsFileResource); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ModificationUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ModificationUtils.java index a9f28e24e0a35..982ac2a3e2223 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ModificationUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/ModificationUtils.java @@ -79,7 +79,11 @@ public static void modifyChunkMetaData( if (range.contains(metaData.getStartTime(), metaData.getEndTime())) { return true; } else { - if (range.overlaps(new TimeRange(metaData.getStartTime(), metaData.getEndTime()))) { + if (overlap( + metaData.getStartTime(), + metaData.getEndTime(), + range.getMin(), + range.getMax())) { metaData.setModified(true); } } @@ -135,9 +139,11 @@ public static void modifyAlignedChunkMetaData( currentRemoved = true; break; } else { - if (range.overlaps( - new TimeRange( - valueChunkMetadata.getStartTime(), valueChunkMetadata.getEndTime()))) { + if (overlap( + valueChunkMetadata.getStartTime(), + valueChunkMetadata.getEndTime(), + range.getMin(), + range.getMax())) { valueChunkMetadata.setModified(true); modified = true; } @@ -209,6 +215,11 @@ public static boolean isPointDeleted(long timestamp, List deletionLis return isPointDeleted(timestamp, deletionList, deleteCursor); } + // Both ranges are closed. + public static boolean overlap(long startA, long endA, long startB, long endB) { + return endB >= startA && startB <= endA; + } + /** * Check whether the device with start time and end time is completely deleted by mods or not. * There are some slight differences from that in {@link SettleSelectorImpl}. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java index dec3864353cc0..90e405ebdea91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/QueryDataSetUtils.java @@ -30,15 +30,14 @@ import org.apache.tsfile.utils.BitMap; import org.apache.tsfile.utils.BytesUtils; import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.PublicBAOS; import org.apache.tsfile.write.UnSupportedDataTypeException; -import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.LinkedList; import java.util.List; import java.util.Optional; @@ -57,14 +56,13 @@ public static Pair convertTsBlockByFetchSize( // indicate whether it is a null int columnNumWithTime = columnNum * 2 + 1; DataOutputStream[] dataOutputStreams = new DataOutputStream[columnNumWithTime]; - ByteArrayOutputStream[] byteArrayOutputStreams = new ByteArrayOutputStream[columnNumWithTime]; + PublicBAOS[] byteArrayOutputStreams = new PublicBAOS[columnNumWithTime]; for (int i = 0; i < columnNumWithTime; i++) { - byteArrayOutputStreams[i] = new ByteArrayOutputStream(); + byteArrayOutputStreams[i] = new PublicBAOS(); dataOutputStreams[i] = new DataOutputStream(byteArrayOutputStreams[i]); } int rowCount = 0; - int[] valueOccupation = new int[columnNum]; // used to record a bitmap for every 8 points int[] bitmaps = new int[columnNum]; @@ -77,14 +75,7 @@ public static Pair convertTsBlockByFetchSize( TsBlock tsBlock = optionalTsBlock.get(); if (!tsBlock.isEmpty()) { int currentCount = tsBlock.getPositionCount(); - serializeTsBlock( - rowCount, - currentCount, - tsBlock, - columnNum, - dataOutputStreams, - valueOccupation, - bitmaps); + serializeTsBlock(rowCount, currentCount, tsBlock, columnNum, dataOutputStreams, bitmaps); rowCount += currentCount; } } @@ -93,9 +84,9 @@ public static Pair convertTsBlockByFetchSize( TSQueryDataSet tsQueryDataSet = new TSQueryDataSet(); - fillTimeColumn(rowCount, byteArrayOutputStreams, tsQueryDataSet); + fillTimeColumn(byteArrayOutputStreams, tsQueryDataSet); - fillValueColumnsAndBitMaps(rowCount, byteArrayOutputStreams, valueOccupation, tsQueryDataSet); + fillValueColumnsAndBitMaps(byteArrayOutputStreams, tsQueryDataSet); return new Pair<>(tsQueryDataSet, finished); } @@ -109,14 +100,13 @@ public static TSQueryDataSet convertTsBlockByFetchSize(List tsBlocks) int columnNum = 1; int columnNumWithTime = columnNum * 2 + 1; DataOutputStream[] dataOutputStreams = new DataOutputStream[columnNumWithTime]; - ByteArrayOutputStream[] byteArrayOutputStreams = new ByteArrayOutputStream[columnNumWithTime]; + PublicBAOS[] byteArrayOutputStreams = new PublicBAOS[columnNumWithTime]; for (int i = 0; i < columnNumWithTime; i++) { - byteArrayOutputStreams[i] = new ByteArrayOutputStream(); + byteArrayOutputStreams[i] = new PublicBAOS(); dataOutputStreams[i] = new DataOutputStream(byteArrayOutputStreams[i]); } int rowCount = 0; - int[] valueOccupation = new int[columnNum]; // used to record a bitmap for every 8 points int[] bitmaps = new int[columnNum]; @@ -150,7 +140,6 @@ public static TSQueryDataSet convertTsBlockByFetchSize(List tsBlocks) } else { bitmaps[k] = (bitmaps[k] << 1) | FLAG; dataOutputStream.writeInt(column.getInt(i)); - valueOccupation[k] += 4; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[k]); @@ -168,7 +157,6 @@ public static TSQueryDataSet convertTsBlockByFetchSize(List tsBlocks) } else { bitmaps[k] = (bitmaps[k] << 1) | FLAG; dataOutputStream.writeLong(column.getLong(i)); - valueOccupation[k] += 8; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[k]); @@ -185,7 +173,6 @@ public static TSQueryDataSet convertTsBlockByFetchSize(List tsBlocks) } else { bitmaps[k] = (bitmaps[k] << 1) | FLAG; dataOutputStream.writeFloat(column.getFloat(i)); - valueOccupation[k] += 4; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[k]); @@ -202,7 +189,6 @@ public static TSQueryDataSet convertTsBlockByFetchSize(List tsBlocks) } else { bitmaps[k] = (bitmaps[k] << 1) | FLAG; dataOutputStream.writeDouble(column.getDouble(i)); - valueOccupation[k] += 8; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[k]); @@ -219,7 +205,6 @@ public static TSQueryDataSet convertTsBlockByFetchSize(List tsBlocks) } else { bitmaps[k] = (bitmaps[k] << 1) | FLAG; dataOutputStream.writeBoolean(column.getBoolean(i)); - valueOccupation[k] += 1; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[k]); @@ -240,7 +225,6 @@ public static TSQueryDataSet convertTsBlockByFetchSize(List tsBlocks) Binary binary = column.getBinary(i); dataOutputStream.writeInt(binary.getLength()); dataOutputStream.write(binary.getValues()); - valueOccupation[k] = valueOccupation[k] + 4 + binary.getLength(); } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[k]); @@ -268,27 +252,14 @@ public static TSQueryDataSet convertTsBlockByFetchSize(List tsBlocks) } // calculate the time buffer size - int timeOccupation = rowCount * 8; - ByteBuffer timeBuffer = ByteBuffer.allocate(timeOccupation); - timeBuffer.put(byteArrayOutputStreams[0].toByteArray()); - timeBuffer.flip(); - tsQueryDataSet.setTime(timeBuffer); + tsQueryDataSet.setTime(wrapBuffer(byteArrayOutputStreams[0])); // calculate the bitmap buffer size - int bitmapOccupation = (rowCount + 7) / 8; - - List bitmapList = new LinkedList<>(); - List valueList = new LinkedList<>(); + List bitmapList = new ArrayList<>(columnNum); + List valueList = new ArrayList<>(columnNum); for (int i = 1; i < byteArrayOutputStreams.length; i += 2) { - ByteBuffer valueBuffer = ByteBuffer.allocate(valueOccupation[(i - 1) / 2]); - valueBuffer.put(byteArrayOutputStreams[i].toByteArray()); - valueBuffer.flip(); - valueList.add(valueBuffer); - - ByteBuffer bitmapBuffer = ByteBuffer.allocate(bitmapOccupation); - bitmapBuffer.put(byteArrayOutputStreams[i + 1].toByteArray()); - bitmapBuffer.flip(); - bitmapList.add(bitmapBuffer); + valueList.add(wrapBuffer(byteArrayOutputStreams[i])); + bitmapList.add(wrapBuffer(byteArrayOutputStreams[i + 1])); } tsQueryDataSet.setBitmapList(bitmapList); tsQueryDataSet.setValueList(valueList); @@ -301,7 +272,6 @@ private static void serializeTsBlock( TsBlock tsBlock, int columnNum, DataOutputStream[] dataOutputStreams, - int[] valueOccupation, int[] bitmaps) throws IOException { // serialize time column @@ -321,67 +291,27 @@ private static void serializeTsBlock( switch (type) { case INT32: case DATE: - doWithInt32Column( - rowCount, - column, - bitmaps, - k, - dataOutputStream, - valueOccupation, - dataBitmapOutputStream); + doWithInt32Column(rowCount, column, bitmaps, k, dataOutputStream, dataBitmapOutputStream); break; case INT64: case TIMESTAMP: - doWithInt64Column( - rowCount, - column, - bitmaps, - k, - dataOutputStream, - valueOccupation, - dataBitmapOutputStream); + doWithInt64Column(rowCount, column, bitmaps, k, dataOutputStream, dataBitmapOutputStream); break; case FLOAT: - doWithFloatColumn( - rowCount, - column, - bitmaps, - k, - dataOutputStream, - valueOccupation, - dataBitmapOutputStream); + doWithFloatColumn(rowCount, column, bitmaps, k, dataOutputStream, dataBitmapOutputStream); break; case DOUBLE: doWithDoubleColumn( - rowCount, - column, - bitmaps, - k, - dataOutputStream, - valueOccupation, - dataBitmapOutputStream); + rowCount, column, bitmaps, k, dataOutputStream, dataBitmapOutputStream); break; case BOOLEAN: doWithBooleanColumn( - rowCount, - column, - bitmaps, - k, - dataOutputStream, - valueOccupation, - dataBitmapOutputStream); + rowCount, column, bitmaps, k, dataOutputStream, dataBitmapOutputStream); break; case TEXT: case BLOB: case STRING: - doWithTextColumn( - rowCount, - column, - bitmaps, - k, - dataOutputStream, - valueOccupation, - dataBitmapOutputStream); + doWithTextColumn(rowCount, column, bitmaps, k, dataOutputStream, dataBitmapOutputStream); break; default: throw new UnSupportedDataTypeException( @@ -396,7 +326,6 @@ private static void doWithInt32Column( int[] bitmaps, int columnIndex, DataOutputStream dataOutputStream, - int[] valueOccupation, DataOutputStream dataBitmapOutputStream) throws IOException { for (int i = 0, size = column.getPositionCount(); i < size; i++) { @@ -406,7 +335,6 @@ private static void doWithInt32Column( } else { bitmaps[columnIndex] = (bitmaps[columnIndex] << 1) | FLAG; dataOutputStream.writeInt(column.getInt(i)); - valueOccupation[columnIndex] += 4; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[columnIndex]); @@ -422,7 +350,6 @@ private static void doWithInt64Column( int[] bitmaps, int columnIndex, DataOutputStream dataOutputStream, - int[] valueOccupation, DataOutputStream dataBitmapOutputStream) throws IOException { for (int i = 0, size = column.getPositionCount(); i < size; i++) { @@ -432,7 +359,6 @@ private static void doWithInt64Column( } else { bitmaps[columnIndex] = (bitmaps[columnIndex] << 1) | FLAG; dataOutputStream.writeLong(column.getLong(i)); - valueOccupation[columnIndex] += 8; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[columnIndex]); @@ -448,7 +374,6 @@ private static void doWithFloatColumn( int[] bitmaps, int columnIndex, DataOutputStream dataOutputStream, - int[] valueOccupation, DataOutputStream dataBitmapOutputStream) throws IOException { for (int i = 0, size = column.getPositionCount(); i < size; i++) { @@ -458,7 +383,6 @@ private static void doWithFloatColumn( } else { bitmaps[columnIndex] = (bitmaps[columnIndex] << 1) | FLAG; dataOutputStream.writeFloat(column.getFloat(i)); - valueOccupation[columnIndex] += 4; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[columnIndex]); @@ -474,7 +398,6 @@ private static void doWithDoubleColumn( int[] bitmaps, int columnIndex, DataOutputStream dataOutputStream, - int[] valueOccupation, DataOutputStream dataBitmapOutputStream) throws IOException { for (int i = 0, size = column.getPositionCount(); i < size; i++) { @@ -484,7 +407,6 @@ private static void doWithDoubleColumn( } else { bitmaps[columnIndex] = (bitmaps[columnIndex] << 1) | FLAG; dataOutputStream.writeDouble(column.getDouble(i)); - valueOccupation[columnIndex] += 8; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[columnIndex]); @@ -500,7 +422,6 @@ private static void doWithBooleanColumn( int[] bitmaps, int columnIndex, DataOutputStream dataOutputStream, - int[] valueOccupation, DataOutputStream dataBitmapOutputStream) throws IOException { for (int i = 0, size = column.getPositionCount(); i < size; i++) { @@ -510,7 +431,6 @@ private static void doWithBooleanColumn( } else { bitmaps[columnIndex] = (bitmaps[columnIndex] << 1) | FLAG; dataOutputStream.writeBoolean(column.getBoolean(i)); - valueOccupation[columnIndex] += 1; } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[columnIndex]); @@ -526,7 +446,6 @@ private static void doWithTextColumn( int[] bitmaps, int columnIndex, DataOutputStream dataOutputStream, - int[] valueOccupation, DataOutputStream dataBitmapOutputStream) throws IOException { for (int i = 0, size = column.getPositionCount(); i < size; i++) { @@ -538,7 +457,6 @@ private static void doWithTextColumn( Binary binary = column.getBinary(i); dataOutputStream.writeInt(binary.getLength()); dataOutputStream.write(binary.getValues()); - valueOccupation[columnIndex] = valueOccupation[columnIndex] + 4 + binary.getLength(); } if (rowCount != 0 && rowCount % 8 == 0) { dataBitmapOutputStream.writeByte(bitmaps[columnIndex]); @@ -562,40 +480,27 @@ private static void fillRemainingBitMap( } private static void fillTimeColumn( - int rowCount, ByteArrayOutputStream[] byteArrayOutputStreams, TSQueryDataSet tsQueryDataSet) { - // calculate the time buffer size - int timeOccupation = rowCount * 8; - ByteBuffer timeBuffer = ByteBuffer.allocate(timeOccupation); - timeBuffer.put(byteArrayOutputStreams[0].toByteArray()); - timeBuffer.flip(); - tsQueryDataSet.setTime(timeBuffer); + PublicBAOS[] byteArrayOutputStreams, TSQueryDataSet tsQueryDataSet) { + tsQueryDataSet.setTime(wrapBuffer(byteArrayOutputStreams[0])); } private static void fillValueColumnsAndBitMaps( - int rowCount, - ByteArrayOutputStream[] byteArrayOutputStreams, - int[] valueOccupation, - TSQueryDataSet tsQueryDataSet) { - // calculate the bitmap buffer size - int bitmapOccupation = (rowCount + 7) / 8; - - List bitmapList = new LinkedList<>(); - List valueList = new LinkedList<>(); + PublicBAOS[] byteArrayOutputStreams, TSQueryDataSet tsQueryDataSet) { + int columnNum = byteArrayOutputStreams.length / 2; + List bitmapList = new ArrayList<>(columnNum); + List valueList = new ArrayList<>(columnNum); for (int i = 1; i < byteArrayOutputStreams.length; i += 2) { - ByteBuffer valueBuffer = ByteBuffer.allocate(valueOccupation[(i - 1) / 2]); - valueBuffer.put(byteArrayOutputStreams[i].toByteArray()); - valueBuffer.flip(); - valueList.add(valueBuffer); - - ByteBuffer bitmapBuffer = ByteBuffer.allocate(bitmapOccupation); - bitmapBuffer.put(byteArrayOutputStreams[i + 1].toByteArray()); - bitmapBuffer.flip(); - bitmapList.add(bitmapBuffer); + valueList.add(wrapBuffer(byteArrayOutputStreams[i])); + bitmapList.add(wrapBuffer(byteArrayOutputStreams[i + 1])); } tsQueryDataSet.setBitmapList(bitmapList); tsQueryDataSet.setValueList(valueList); } + private static ByteBuffer wrapBuffer(PublicBAOS outputStream) { + return ByteBuffer.wrap(outputStream.getBuf(), 0, outputStream.size()); + } + /** * To fetch required amounts of data and combine them through List * @@ -654,10 +559,8 @@ public static Optional readBitMapsFromBuffer(ByteBuffer buffer, int co for (int i = 0; i < columns; i++) { boolean hasBitMap = BytesUtils.byteToBool(buffer.get()); if (hasBitMap) { - byte[] bytes = new byte[size / Byte.SIZE + 1]; - for (int j = 0; j < bytes.length; j++) { - bytes[j] = buffer.get(); - } + byte[] bytes = new byte[getBitmapByteSize(size)]; + buffer.get(bytes); bitMaps[i] = new BitMap(size, bytes); } } @@ -673,16 +576,18 @@ public static Optional readBitMapsFromStream( for (int i = 0; i < columns; i++) { boolean hasBitMap = BytesUtils.byteToBool(stream.readByte()); if (hasBitMap) { - byte[] bytes = new byte[size / Byte.SIZE + 1]; - for (int j = 0; j < bytes.length; j++) { - bytes[j] = stream.readByte(); - } + byte[] bytes = new byte[getBitmapByteSize(size)]; + stream.readFully(bytes); bitMaps[i] = new BitMap(size, bytes); } } return Optional.of(bitMaps); } + private static int getBitmapByteSize(int size) { + return size / Byte.SIZE + 1; + } + public static Object[] readTabletValuesFromBuffer( ByteBuffer buffer, List types, int columns, int size) { TSDataType[] dataTypes = new TSDataType[types.size()]; @@ -850,11 +755,7 @@ private static void parseTextColumn( for (int index = 0; index < size; index++) { int binarySize = stream.readInt(); byte[] binaryValue = new byte[binarySize]; - int actualReadSize = stream.read(binaryValue); - if (actualReadSize != binarySize) { - throw new IllegalStateException( - "Expect to read " + binarySize + " bytes, actually read " + actualReadSize + "bytes."); - } + stream.readFully(binaryValue); binaryValues[index] = new Binary(binaryValue); } values[columnIndex] = binaryValues; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/cache/DataNodeSchemaCacheTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/cache/DataNodeSchemaCacheTest.java index 144f2396ef7fd..bf15d216e67cb 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/cache/DataNodeSchemaCacheTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/cache/DataNodeSchemaCacheTest.java @@ -267,6 +267,43 @@ public void testUpdateLastCache() throws IllegalPathException { Assert.assertEquals(0, dataNodeSchemaCache.getDeviceSchemaCache().getMemoryUsage()); } + @Test + public void testInvalidateLastCacheByWildcardDevicePath() throws IllegalPathException { + final MeasurementSchema s0 = new MeasurementSchema("s0", TSDataType.INT32); + final PartialPath device0 = new PartialPath("root.sg1.d1"); + final PartialPath device1 = new PartialPath("root.sg2.d1"); + final MeasurementPath path0 = new MeasurementPath(device0.concatNode("s0"), s0); + final MeasurementPath path1 = new MeasurementPath(device1.concatNode("s0"), s0); + final TimeValuePair tv0 = new TimeValuePair(0L, new TsPrimitiveType.TsInt(0)); + + updateLastCache("root.sg1", device0, path0, s0, tv0); + updateLastCache("root.sg2", device1, path1, s0, tv0); + + Assert.assertEquals(tv0, dataNodeSchemaCache.getLastCache(path0)); + Assert.assertEquals(tv0, dataNodeSchemaCache.getLastCache(path1)); + + dataNodeSchemaCache.invalidateLastCache(new MeasurementPath("root.sg1.*.s0")); + + Assert.assertNull(dataNodeSchemaCache.getLastCache(path0)); + Assert.assertEquals(tv0, dataNodeSchemaCache.getLastCache(path1)); + } + + private void updateLastCache( + final String database, + final PartialPath devicePath, + final MeasurementPath path, + final MeasurementSchema schema, + final TimeValuePair timeValuePair) { + dataNodeSchemaCache.declareLastCache(database, path); + dataNodeSchemaCache.updateLastCacheIfExists( + database, + devicePath, + new String[] {path.getMeasurement()}, + new TimeValuePair[] {timeValuePair}, + false, + new MeasurementSchema[] {schema}); + } + @Test public void testPut() throws Exception { final ClusterSchemaTree clusterSchemaTree = new ClusterSchemaTree(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/LimitOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/LimitOperatorTest.java index 31dda8933581d..4b4a5329f8a43 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/LimitOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/LimitOperatorTest.java @@ -157,6 +157,9 @@ public void batchTest() throws Exception { int count = 0; while (limitOperator.isBlocked().isDone() && limitOperator.hasNext()) { TsBlock tsBlock = limitOperator.next(); + if (tsBlock == null) { + continue; + } assertEquals(2, tsBlock.getValueColumnCount()); assertTrue(tsBlock.getColumn(0) instanceof IntColumn); assertTrue(tsBlock.getColumn(1) instanceof IntColumn); From 5e1587f67e3a115daa301268c466ca2bc0dcee4f Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 3 Jun 2026 12:24:13 +0800 Subject: [PATCH 076/102] Revert "Fix CQ recovery gap and stale callback contamination (#17734) (#17820)" (#17826) This reverts commit 24be79aba9c973a4cb11fdb8947cdf32da12ea8e. --- iotdb-core/confignode/pom.xml | 4 + .../consensus/request/read/cq/ShowCQPlan.java | 13 -- .../request/write/cq/ActiveCQPlan.java | 20 +-- .../consensus/request/write/cq/AddCQPlan.java | 20 +-- .../request/write/cq/DropCQPlan.java | 20 +-- .../write/cq/UpdateCQLastExecTimePlan.java | 23 ++-- .../confignode/manager/cq/CQManager.java | 102 +-------------- .../confignode/manager/cq/CQScheduleTask.java | 58 ++------- .../confignode/persistence/cq/CQInfo.java | 76 +++++------- .../executor/ConfigPlanExecutor.java | 3 +- .../procedure/impl/cq/CreateCQProcedure.java | 80 ++---------- .../request/ConfigPhysicalPlanSerDeTest.java | 8 +- .../iotdb/confignode/cq/CQManagerTest.java | 107 ---------------- .../confignode/persistence/CQInfoTest.java | 64 +--------- .../procedure/impl/CreateCQProcedureTest.java | 26 ---- .../cq/CreateCQProcedureRecoveryTest.java | 117 ------------------ 16 files changed, 106 insertions(+), 635 deletions(-) delete mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java delete mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java diff --git a/iotdb-core/confignode/pom.xml b/iotdb-core/confignode/pom.xml index a7cd4b6298b5f..fb22fd5b091f7 100644 --- a/iotdb-core/confignode/pom.xml +++ b/iotdb-core/confignode/pom.xml @@ -137,6 +137,10 @@ org.apache.commons commons-lang3 + + commons-codec + commons-codec + org.apache.thrift libthrift diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java index c28838d556b48..5217849deb488 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java @@ -21,24 +21,11 @@ import org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan; -import java.util.Optional; - import static org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType.SHOW_CQ; public class ShowCQPlan extends ConfigPhysicalReadPlan { - private final String cqId; - public ShowCQPlan() { - this(null); - } - - public ShowCQPlan(String cqId) { super(SHOW_CQ); - this.cqId = cqId; - } - - public Optional getCqId() { - return Optional.ofNullable(cqId); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java index 4ca33b054a93f..e488ac2566921 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java @@ -35,39 +35,39 @@ public class ActiveCQPlan extends ConfigPhysicalPlan { private String cqId; - private String cqToken; + private String md5; public ActiveCQPlan() { super(ACTIVE_CQ); } - public ActiveCQPlan(String cqId, String cqToken) { + public ActiveCQPlan(String cqId, String md5) { super(ACTIVE_CQ); Validate.notNull(cqId); - Validate.notNull(cqToken); + Validate.notNull(md5); this.cqId = cqId; - this.cqToken = cqToken; + this.md5 = md5; } public String getCqId() { return cqId; } - public String getCqToken() { - return cqToken; + public String getMd5() { + return md5; } @Override protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(cqId, stream); - ReadWriteIOUtils.write(cqToken, stream); + ReadWriteIOUtils.write(md5, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { cqId = ReadWriteIOUtils.readString(buffer); - cqToken = ReadWriteIOUtils.readString(buffer); + md5 = ReadWriteIOUtils.readString(buffer); } @Override @@ -82,11 +82,11 @@ public boolean equals(Object o) { return false; } ActiveCQPlan that = (ActiveCQPlan) o; - return cqId.equals(that.cqId) && cqToken.equals(that.cqToken); + return cqId.equals(that.cqId) && md5.equals(that.md5); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), cqId, cqToken); + return Objects.hash(super.hashCode(), cqId, md5); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java index 721c83d3d2f76..0aae9e2e974ca 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java @@ -37,7 +37,7 @@ public class AddCQPlan extends ConfigPhysicalPlan { private TCreateCQReq req; - private String cqToken; + private String md5; private long firstExecutionTime; @@ -45,12 +45,12 @@ public AddCQPlan() { super(ADD_CQ); } - public AddCQPlan(TCreateCQReq req, String cqToken, long firstExecutionTime) { + public AddCQPlan(TCreateCQReq req, String md5, long firstExecutionTime) { super(ADD_CQ); Validate.notNull(req); - Validate.notNull(cqToken); + Validate.notNull(md5); this.req = req; - this.cqToken = cqToken; + this.md5 = md5; this.firstExecutionTime = firstExecutionTime; } @@ -58,8 +58,8 @@ public TCreateCQReq getReq() { return req; } - public String getCqToken() { - return cqToken; + public String getMd5() { + return md5; } public long getFirstExecutionTime() { @@ -70,14 +70,14 @@ public long getFirstExecutionTime() { protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ThriftCommonsSerDeUtils.serializeTCreateCQReq(req, stream); - ReadWriteIOUtils.write(cqToken, stream); + ReadWriteIOUtils.write(md5, stream); ReadWriteIOUtils.write(firstExecutionTime, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { req = ThriftCommonsSerDeUtils.deserializeTCreateCQReq(buffer); - cqToken = ReadWriteIOUtils.readString(buffer); + md5 = ReadWriteIOUtils.readString(buffer); firstExecutionTime = ReadWriteIOUtils.readLong(buffer); } @@ -95,11 +95,11 @@ public boolean equals(Object o) { AddCQPlan addCQPlan = (AddCQPlan) o; return firstExecutionTime == addCQPlan.firstExecutionTime && Objects.equals(req, addCQPlan.req) - && Objects.equals(cqToken, addCQPlan.cqToken); + && Objects.equals(md5, addCQPlan.md5); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), req, cqToken, firstExecutionTime); + return Objects.hash(super.hashCode(), req, md5, firstExecutionTime); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java index f22561c0b807d..5c901362997f8 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java @@ -37,7 +37,7 @@ public class DropCQPlan extends ConfigPhysicalPlan { private String cqId; // may be null in user call of drop CQ - private String cqToken; + private String md5; public DropCQPlan() { super(DROP_CQ); @@ -49,33 +49,33 @@ public DropCQPlan(String cqId) { this.cqId = cqId; } - public DropCQPlan(String cqId, String cqToken) { + public DropCQPlan(String cqId, String md5) { super(DROP_CQ); Validate.notNull(cqId); - Validate.notNull(cqToken); + Validate.notNull(md5); this.cqId = cqId; - this.cqToken = cqToken; + this.md5 = md5; } public String getCqId() { return cqId; } - public Optional getCqToken() { - return Optional.ofNullable(cqToken); + public Optional getMd5() { + return Optional.ofNullable(md5); } @Override protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(cqId, stream); - ReadWriteIOUtils.write(cqToken, stream); + ReadWriteIOUtils.write(md5, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { cqId = ReadWriteIOUtils.readString(buffer); - cqToken = ReadWriteIOUtils.readString(buffer); + md5 = ReadWriteIOUtils.readString(buffer); } @Override @@ -90,11 +90,11 @@ public boolean equals(Object o) { return false; } DropCQPlan that = (DropCQPlan) o; - return cqId.equals(that.cqId) && Objects.equals(cqToken, that.cqToken); + return cqId.equals(that.cqId) && Objects.equals(md5, that.md5); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), cqId, cqToken); + return Objects.hash(super.hashCode(), cqId, md5); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java index 392f6006ba56e..51a8358a04c28 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java @@ -37,19 +37,20 @@ public class UpdateCQLastExecTimePlan extends ConfigPhysicalPlan { private long executionTime; - private String cqToken; + // may be null in user call of drop CQ + private String md5; public UpdateCQLastExecTimePlan() { super(UPDATE_CQ_LAST_EXEC_TIME); } - public UpdateCQLastExecTimePlan(String cqId, long executionTime, String cqToken) { + public UpdateCQLastExecTimePlan(String cqId, long executionTime, String md5) { super(UPDATE_CQ_LAST_EXEC_TIME); Validate.notNull(cqId); - Validate.notNull(cqToken); + Validate.notNull(md5); this.cqId = cqId; this.executionTime = executionTime; - this.cqToken = cqToken; + this.md5 = md5; } public String getCqId() { @@ -60,8 +61,8 @@ public long getExecutionTime() { return executionTime; } - public String getCqToken() { - return cqToken; + public String getMd5() { + return md5; } @Override @@ -69,14 +70,14 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(cqId, stream); ReadWriteIOUtils.write(executionTime, stream); - ReadWriteIOUtils.write(cqToken, stream); + ReadWriteIOUtils.write(md5, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { cqId = ReadWriteIOUtils.readString(buffer); executionTime = ReadWriteIOUtils.readLong(buffer); - cqToken = ReadWriteIOUtils.readString(buffer); + md5 = ReadWriteIOUtils.readString(buffer); } @Override @@ -91,13 +92,11 @@ public boolean equals(Object o) { return false; } UpdateCQLastExecTimePlan that = (UpdateCQLastExecTimePlan) o; - return executionTime == that.executionTime - && cqId.equals(that.cqId) - && cqToken.equals(that.cqToken); + return executionTime == that.executionTime && cqId.equals(that.cqId) && md5.equals(that.md5); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), cqId, executionTime, cqToken); + return Objects.hash(super.hashCode(), cqId, executionTime, md5); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java index a90a21f285ff2..5726b3ce82698 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java @@ -42,10 +42,7 @@ import java.util.Collections; import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -59,15 +56,11 @@ public class CQManager { private final ReadWriteLock lock; - // Key: CQ id. Value: the local task and the metadata token it owns. - private final ConcurrentMap locallyScheduledCQs; - private ScheduledExecutorService executor; public CQManager(ConfigManager configManager) { this.configManager = configManager; this.lock = new ReentrantReadWriteLock(); - this.locallyScheduledCQs = new ConcurrentHashMap<>(); this.executor = IoTDBThreadPoolFactory.newScheduledThreadPool( CONF.getCqSubmitThread(), ThreadName.CQ_SCHEDULER.getName()); @@ -84,21 +77,14 @@ public TSStatus createCQ(TCreateCQReq req) { } public TSStatus dropCQ(TDropCQReq req) { - lock.readLock().lock(); try { - TSStatus status = configManager.getConsensusManager().write(new DropCQPlan(req.cqId)); - if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - cancelLocallyScheduledCQ(req.cqId); - } - return status; + return configManager.getConsensusManager().write(new DropCQPlan(req.cqId)); } catch (ConsensusException e) { LOGGER.warn("Unexpected error happened while dropping cq {}: ", req.cqId, e); // consensus layer related errors TSStatus res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); res.setMessage(e.getMessage()); return res; - } finally { - lock.readLock().unlock(); } } @@ -131,7 +117,6 @@ public void startCQScheduler() { try { // 1. shutdown previous cq schedule thread pool try { - cancelAllLocallyScheduledCQs(); if (executor != null) { executor.shutdown(); } @@ -169,15 +154,7 @@ public void startCQScheduler() { for (CQInfo.CQEntry entry : allCQs) { if (entry.getState() == CQState.ACTIVE) { CQScheduleTask cqScheduleTask = new CQScheduleTask(entry, executor, configManager); - if (!markCQLocallyScheduled(entry.getCqId(), entry.getCqToken(), cqScheduleTask)) { - continue; - } - try { - cqScheduleTask.submitSelf(); - } catch (RuntimeException e) { - unmarkCQLocallyScheduled(entry.getCqId(), entry.getCqToken()); - throw e; - } + cqScheduleTask.submitSelf(); } } } @@ -197,7 +174,6 @@ public void stopCQScheduler() { try { previous = executor; executor = null; - cancelAllLocallyScheduledCQs(); } finally { lock.writeLock().unlock(); } @@ -205,78 +181,4 @@ public void stopCQScheduler() { previous.shutdown(); } } - - public boolean markCQLocallyScheduled(String cqId, String cqToken, CQScheduleTask task) { - AtomicBoolean shouldSchedule = new AtomicBoolean(false); - LocallyScheduledCQ schedule = new LocallyScheduledCQ(cqToken, task); - lock.readLock().lock(); - try { - locallyScheduledCQs.compute( - cqId, - (ignored, previousSchedule) -> { - if (previousSchedule != null && previousSchedule.hasToken(cqToken)) { - return previousSchedule; - } - if (previousSchedule != null) { - previousSchedule.cancel(); - } - shouldSchedule.set(true); - return schedule; - }); - if (!shouldSchedule.get()) { - task.cancel(); - } - return shouldSchedule.get(); - } finally { - lock.readLock().unlock(); - } - } - - public void unmarkCQLocallyScheduled(String cqId, String cqToken) { - lock.readLock().lock(); - try { - locallyScheduledCQs.computeIfPresent( - cqId, - (ignored, schedule) -> { - if (schedule.hasToken(cqToken)) { - schedule.cancel(); - return null; - } - return schedule; - }); - } finally { - lock.readLock().unlock(); - } - } - - private void cancelLocallyScheduledCQ(String cqId) { - LocallyScheduledCQ schedule = locallyScheduledCQs.remove(cqId); - if (schedule != null) { - schedule.cancel(); - } - } - - private void cancelAllLocallyScheduledCQs() { - locallyScheduledCQs.values().forEach(LocallyScheduledCQ::cancel); - locallyScheduledCQs.clear(); - } - - private static class LocallyScheduledCQ { - - private final String cqToken; - private final CQScheduleTask task; - - private LocallyScheduledCQ(String cqToken, CQScheduleTask task) { - this.cqToken = cqToken; - this.task = task; - } - - private boolean hasToken(String cqToken) { - return this.cqToken.equals(cqToken); - } - - private void cancel() { - task.cancel(); - } - } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java index d85d0f12a3040..6125edc1ef7bd 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java @@ -39,10 +39,7 @@ import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; public class CQScheduleTask implements Runnable { @@ -72,7 +69,7 @@ public class CQScheduleTask implements Runnable { private final long endTimeOffset; private final TimeoutPolicy timeoutPolicy; private final String queryBody; - private final String cqToken; + private final String md5; private final String zoneId; @@ -84,15 +81,12 @@ public class CQScheduleTask implements Runnable { private final long retryWaitTimeInMS; - private final AtomicBoolean cancelled; - private final AtomicReference> scheduledFuture; - private long executionTime; public CQScheduleTask( TCreateCQReq req, long firstExecutionTime, - String cqToken, + String md5, ScheduledExecutorService executor, ConfigManager configManager) { this( @@ -102,7 +96,7 @@ public CQScheduleTask( req.endTimeOffset, TimeoutPolicy.deserialize(req.timeoutPolicy), req.queryBody, - cqToken, + md5, req.zoneId, req.username, executor, @@ -119,7 +113,7 @@ public CQScheduleTask( entry.getEndTimeOffset(), entry.getTimeoutPolicy(), entry.getQueryBody(), - entry.getCqToken(), + entry.getMd5(), entry.getZoneId(), entry.getUsername(), executor, @@ -135,7 +129,7 @@ public CQScheduleTask( long endTimeOffset, TimeoutPolicy timeoutPolicy, String queryBody, - String cqToken, + String md5, String zoneId, String username, ScheduledExecutorService executor, @@ -147,14 +141,12 @@ public CQScheduleTask( this.endTimeOffset = endTimeOffset; this.timeoutPolicy = timeoutPolicy; this.queryBody = queryBody; - this.cqToken = cqToken; + this.md5 = md5; this.zoneId = zoneId; this.username = username; this.executor = executor; this.configManager = configManager; this.retryWaitTimeInMS = Math.min(DEFAULT_RETRY_WAIT_TIME_IN_MS, everyInterval / FACTOR); - this.cancelled = new AtomicBoolean(false); - this.scheduledFuture = new AtomicReference<>(); this.executionTime = executionTime; } @@ -173,9 +165,6 @@ public static long getFirstExecutionTime(long boundaryTime, long everyInterval, @Override public void run() { - if (cancelled.get()) { - return; - } long startTime = executionTime - startTimeOffset; long endTime = executionTime - endTimeOffset; @@ -188,9 +177,6 @@ public void run() { submitSelf(retryWaitTimeInMS, TimeUnit.MILLISECONDS); } } else { - if (cancelled.get()) { - return; - } LOGGER.info( "[StartExecuteCQ] execute CQ {} on DataNode[{}], time range is [{}, {}), current time is {}", cqId, @@ -220,32 +206,12 @@ public void submitSelf() { } private void submitSelf(long delay, TimeUnit unit) { - if (cancelled.get()) { - return; - } - ScheduledFuture newFuture = executor.schedule(this, delay, unit); - ScheduledFuture previousFuture = scheduledFuture.getAndSet(newFuture); - if (previousFuture != null) { - previousFuture.cancel(false); - } - if (cancelled.get() && scheduledFuture.compareAndSet(newFuture, null)) { - newFuture.cancel(false); - } - } - - public void cancel() { - cancelled.set(true); - ScheduledFuture currentFuture = scheduledFuture.getAndSet(null); - if (currentFuture != null) { - currentFuture.cancel(false); - } + executor.schedule(this, delay, unit); } private boolean needSubmit() { // current node is still leader and thread pool is not shut down. - return !cancelled.get() - && configManager.getConsensusManager().isLeader() - && !executor.isShutdown(); + return configManager.getConsensusManager().isLeader() && !executor.isShutdown(); } private class AsyncExecuteCQCallback implements AsyncMethodCallback { @@ -272,9 +238,6 @@ private void updateExecutionTime() { @Override public void onComplete(TSStatus response) { - if (cancelled.get()) { - return; - } if (response.code == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { LOGGER.info( @@ -288,7 +251,7 @@ public void onComplete(TSStatus response) { result = configManager .getConsensusManager() - .write(new UpdateCQLastExecTimePlan(cqId, executionTime, cqToken)); + .write(new UpdateCQLastExecTimePlan(cqId, executionTime, md5)); } catch (ConsensusException e) { result = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); result.setMessage(e.getMessage()); @@ -328,9 +291,6 @@ public void onComplete(TSStatus response) { @Override public void onError(Exception exception) { - if (cancelled.get()) { - return; - } LOGGER.warn("Execute CQ {} failed", cqId, exception); if (needSubmit()) { submitSelf(retryWaitTimeInMS, TimeUnit.MILLISECONDS); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java index b24f1ca9aa528..4c1225ef48a05 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java @@ -23,7 +23,6 @@ import org.apache.iotdb.commons.cq.CQState; import org.apache.iotdb.commons.cq.TimeoutPolicy; import org.apache.iotdb.commons.snapshot.SnapshotProcessor; -import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.ActiveCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.DropCQPlan; @@ -45,9 +44,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -64,7 +61,7 @@ public class CQInfo implements SnapshotProcessor { private static final String CQ_NOT_EXIST_FORMAT = "CQ %s doesn't exist."; - private static final String CQ_TOKEN_NOT_MATCH_FORMAT = "Token of CQ %s doesn't match"; + private static final String MD5_NOT_MATCH_FORMAT = "MD5 of CQ %s doesn't match"; private final Map cqMap; @@ -94,7 +91,7 @@ public TSStatus addCQ(AddCQPlan plan) { CQEntry cqEntry = new CQEntry( plan.getReq(), - plan.getCqToken(), + plan.getMd5(), plan.getFirstExecutionTime() - plan.getReq().everyInterval); cqMap.put(cqId, cqEntry); res.code = TSStatusCode.SUCCESS_STATUS.getStatusCode(); @@ -108,13 +105,13 @@ public TSStatus addCQ(AddCQPlan plan) { /** * Drop the CQ whose ID is same as cqId in plan. * - * @return SUCCESS_STATUS if there is CQ whose ID and token is same as cqId in plan, + * @return SUCCESS_STATUS if there is CQ whose ID and md5 is same as cqId in plan, * otherwise NO_SUCH_CQ. */ public TSStatus dropCQ(DropCQPlan plan) { TSStatus res = new TSStatus(); String cqId = plan.getCqId(); - Optional cqToken = plan.getCqToken(); + Optional md5 = plan.getMd5(); lock.writeLock().lock(); try { CQEntry cqEntry = cqMap.get(cqId); @@ -122,10 +119,10 @@ public TSStatus dropCQ(DropCQPlan plan) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); res.message = String.format(CQ_NOT_EXIST_FORMAT, cqId); LOGGER.warn("Drop CQ {} failed, because it doesn't exist.", cqId); - } else if ((cqToken.isPresent() && !cqToken.get().equals(cqEntry.cqToken))) { + } else if ((md5.isPresent() && !md5.get().equals(cqEntry.md5))) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); - res.message = String.format(CQ_TOKEN_NOT_MATCH_FORMAT, cqId); - LOGGER.warn("Drop CQ {} failed, because its token doesn't match.", cqId); + res.message = String.format(MD5_NOT_MATCH_FORMAT, cqId); + LOGGER.warn("Drop CQ {} failed, because its MD5 doesn't match.", cqId); } else { cqMap.remove(cqId); res.code = TSStatusCode.SUCCESS_STATUS.getStatusCode(); @@ -138,24 +135,11 @@ public TSStatus dropCQ(DropCQPlan plan) { } public ShowCQResp showCQ() { - return showCQ(new ShowCQPlan()); - } - - public ShowCQResp showCQ(ShowCQPlan plan) { lock.readLock().lock(); try { - Optional cqId = plan.getCqId(); - List cqList; - if (cqId.isPresent()) { - CQEntry cqEntry = cqMap.get(cqId.get()); - cqList = - cqEntry == null - ? Collections.emptyList() - : Collections.singletonList(new CQEntry(cqEntry)); - } else { - cqList = cqMap.values().stream().map(CQEntry::new).collect(Collectors.toList()); - } - return new ShowCQResp(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), cqList); + return new ShowCQResp( + new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), + cqMap.values().stream().map(CQEntry::new).collect(Collectors.toList())); } finally { lock.readLock().unlock(); } @@ -169,16 +153,16 @@ public ShowCQResp showCQ(ShowCQPlan plan) { public TSStatus activeCQ(ActiveCQPlan plan) { TSStatus res = new TSStatus(); String cqId = plan.getCqId(); - String cqToken = plan.getCqToken(); + String md5 = plan.getMd5(); lock.writeLock().lock(); try { CQEntry cqEntry = cqMap.get(cqId); if (cqEntry == null) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); res.message = String.format(CQ_NOT_EXIST_FORMAT, cqId); - } else if (!cqToken.equals(cqEntry.cqToken)) { + } else if (!md5.equals(cqEntry.md5)) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); - res.message = String.format(CQ_TOKEN_NOT_MATCH_FORMAT, cqId); + res.message = String.format(MD5_NOT_MATCH_FORMAT, cqId); } else if (cqEntry.state == CQState.ACTIVE) { res.code = TSStatusCode.CQ_ALREADY_ACTIVE.getStatusCode(); res.message = String.format("CQ %s has already been active", cqId); @@ -196,22 +180,22 @@ public TSStatus activeCQ(ActiveCQPlan plan) { * Update the last execution time of the corresponding CQ. * * @return SUCCESS_STATUS if successfully updated, or NO_SUCH_CQ if 1. the CQ doesn't exist; or 2. - * token is different. or CQ_UPDATE_LAST_EXEC_TIME_FAILED 3. original lastExecutionTime >= + * md5 is different. or CQ_UPDATE_LAST_EXEC_TIME_FAILED 3. original lastExecutionTime >= * current lastExecutionTime; */ public TSStatus updateCQLastExecutionTime(UpdateCQLastExecTimePlan plan) { TSStatus res = new TSStatus(); String cqId = plan.getCqId(); - String cqToken = plan.getCqToken(); + String md5 = plan.getMd5(); lock.writeLock().lock(); try { CQEntry cqEntry = cqMap.get(cqId); if (cqEntry == null) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); res.message = String.format(CQ_NOT_EXIST_FORMAT, cqId); - } else if (!cqToken.equals(cqEntry.cqToken)) { + } else if (!md5.equals(cqEntry.md5)) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); - res.message = String.format(CQ_TOKEN_NOT_MATCH_FORMAT, cqId); + res.message = String.format(MD5_NOT_MATCH_FORMAT, cqId); } else if (cqEntry.lastExecutionTime >= plan.getExecutionTime()) { res.code = TSStatusCode.CQ_UPDATE_LAST_EXEC_TIME_ERROR.getStatusCode(); res.message = @@ -315,7 +299,7 @@ public static class CQEntry { private final TimeoutPolicy timeoutPolicy; private final String queryBody; private final String sql; - private final String cqToken; + private final String md5; private final String zoneId; @@ -324,7 +308,7 @@ public static class CQEntry { private CQState state; private long lastExecutionTime; - private CQEntry(TCreateCQReq req, String cqToken, long lastExecutionTime) { + private CQEntry(TCreateCQReq req, String md5, long lastExecutionTime) { this( req.cqId, req.everyInterval, @@ -334,7 +318,7 @@ private CQEntry(TCreateCQReq req, String cqToken, long lastExecutionTime) { TimeoutPolicy.deserialize(req.timeoutPolicy), req.queryBody, req.sql, - cqToken, + md5, req.zoneId, req.username, CQState.INACTIVE, @@ -351,7 +335,7 @@ private CQEntry(CQEntry other) { other.timeoutPolicy, other.queryBody, other.sql, - other.cqToken, + other.md5, other.zoneId, other.username, other.state, @@ -368,7 +352,7 @@ private CQEntry( TimeoutPolicy timeoutPolicy, String queryBody, String sql, - String cqToken, + String md5, String zoneId, String username, CQState state, @@ -381,7 +365,7 @@ private CQEntry( this.timeoutPolicy = timeoutPolicy; this.queryBody = queryBody; this.sql = sql; - this.cqToken = cqToken; + this.md5 = md5; this.zoneId = zoneId; this.username = username; this.state = state; @@ -397,7 +381,7 @@ private void serialize(OutputStream stream) throws IOException { ReadWriteIOUtils.write(timeoutPolicy.getType(), stream); ReadWriteIOUtils.write(queryBody, stream); ReadWriteIOUtils.write(sql, stream); - ReadWriteIOUtils.write(cqToken, stream); + ReadWriteIOUtils.write(md5, stream); ReadWriteIOUtils.write(zoneId, stream); ReadWriteIOUtils.write(username, stream); ReadWriteIOUtils.write(state.getType(), stream); @@ -413,7 +397,7 @@ private static CQEntry deserialize(InputStream stream) throws IOException { TimeoutPolicy timeoutPolicy = TimeoutPolicy.deserialize(ReadWriteIOUtils.readByte(stream)); String queryBody = ReadWriteIOUtils.readString(stream); String sql = ReadWriteIOUtils.readString(stream); - String cqToken = ReadWriteIOUtils.readString(stream); + String md5 = ReadWriteIOUtils.readString(stream); String zoneId = ReadWriteIOUtils.readString(stream); String username = ReadWriteIOUtils.readString(stream); CQState state = CQState.deserialize(ReadWriteIOUtils.readByte(stream)); @@ -427,7 +411,7 @@ private static CQEntry deserialize(InputStream stream) throws IOException { timeoutPolicy, queryBody, sql, - cqToken, + md5, zoneId, username, state, @@ -466,8 +450,8 @@ public String getSql() { return sql; } - public String getCqToken() { - return cqToken; + public String getMd5() { + return md5; } public CQState getState() { @@ -504,7 +488,7 @@ public boolean equals(Object o) { && timeoutPolicy == cqEntry.timeoutPolicy && Objects.equals(queryBody, cqEntry.queryBody) && Objects.equals(sql, cqEntry.sql) - && Objects.equals(cqToken, cqEntry.cqToken) + && Objects.equals(md5, cqEntry.md5) && Objects.equals(zoneId, cqEntry.zoneId) && Objects.equals(username, cqEntry.username) && state == cqEntry.state; @@ -521,7 +505,7 @@ public int hashCode() { timeoutPolicy, queryBody, sql, - cqToken, + md5, zoneId, username, state, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index 60ec748fb5478..09017a841c73c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -30,7 +30,6 @@ import org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan; import org.apache.iotdb.confignode.consensus.request.read.ainode.GetAINodeConfigurationPlan; import org.apache.iotdb.confignode.consensus.request.read.auth.AuthorReadPlan; -import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.read.database.CountDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.database.GetDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.datanode.GetDataNodeConfigurationPlan; @@ -323,7 +322,7 @@ public DataSet executeQueryPlan(final ConfigPhysicalReadPlan req) case GetSeriesSlotList: return partitionInfo.getSeriesSlotList((GetSeriesSlotListPlan) req); case SHOW_CQ: - return cqInfo.showCQ((ShowCQPlan) req); + return cqInfo.showCQ(); case GetFunctionTable: return udfInfo.getUDFTable(); case GetFunctionJar: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java index af7f968e8a57c..3da60c5ad847b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java @@ -22,14 +22,10 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils; -import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.ActiveCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.DropCQPlan; -import org.apache.iotdb.confignode.consensus.response.cq.ShowCQResp; -import org.apache.iotdb.confignode.manager.cq.CQManager; import org.apache.iotdb.confignode.manager.cq.CQScheduleTask; -import org.apache.iotdb.confignode.persistence.cq.CQInfo; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.impl.node.AbstractNodeProcedure; @@ -39,6 +35,7 @@ import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.rpc.TSStatusCode; +import org.apache.commons.codec.digest.DigestUtils; import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,8 +44,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Objects; -import java.util.Optional; -import java.util.UUID; import java.util.concurrent.ScheduledExecutorService; import static org.apache.iotdb.confignode.procedure.state.cq.CreateCQState.INACTIVE; @@ -64,7 +59,7 @@ public class CreateCQProcedure extends AbstractNodeProcedure { private TCreateCQReq req; - private String cqToken; + private String md5; private long firstExecutionTime; @@ -79,7 +74,7 @@ public CreateCQProcedure(ScheduledExecutorService executor) { public CreateCQProcedure(TCreateCQReq req, ScheduledExecutorService executor) { super(); this.req = req; - this.cqToken = generateCQToken(); + this.md5 = DigestUtils.md2Hex(req.cqId); this.executor = executor; this.firstExecutionTime = CQScheduleTask.getFirstExecutionTime(req.boundaryTime, req.everyInterval); @@ -95,16 +90,12 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, CreateCQState state) addCQ(env); return Flow.HAS_MORE_STATE; case INACTIVE: - submitScheduleTask( - env, - new CQScheduleTask( - req, firstExecutionTime, cqToken, executor, env.getConfigManager())); + CQScheduleTask cqScheduleTask = + new CQScheduleTask(req, firstExecutionTime, md5, executor, env.getConfigManager()); + cqScheduleTask.submitSelf(); setNextState(SCHEDULED); break; case SCHEDULED: - if (isStateDeserialized()) { - recoverScheduledTask(env); - } activeCQ(env); return Flow.NO_MORE_STATE; default: @@ -134,7 +125,7 @@ private void addCQ(ConfigNodeProcedureEnv env) { res = env.getConfigManager() .getConsensusManager() - .write(new AddCQPlan(req, cqToken, firstExecutionTime)); + .write(new AddCQPlan(req, md5, firstExecutionTime)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -155,7 +146,7 @@ private void addCQ(ConfigNodeProcedureEnv env) { private void activeCQ(ConfigNodeProcedureEnv env) { TSStatus res; try { - res = env.getConfigManager().getConsensusManager().write(new ActiveCQPlan(req.cqId, cqToken)); + res = env.getConfigManager().getConsensusManager().write(new ActiveCQPlan(req.cqId, md5)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -173,42 +164,6 @@ private void activeCQ(ConfigNodeProcedureEnv env) { } } - void recoverScheduledTask(ConfigNodeProcedureEnv env) throws ConsensusException { - Optional cqEntry = getCurrentCQEntry(env); - if (!cqEntry.isPresent()) { - LOGGER.info( - "Skip recovering the schedule task of CQ {} because its metadata is unavailable.", - req.cqId); - return; - } - submitScheduleTask(env, new CQScheduleTask(cqEntry.get(), executor, env.getConfigManager())); - } - - Optional getCurrentCQEntry(ConfigNodeProcedureEnv env) throws ConsensusException { - ShowCQResp response = - (ShowCQResp) env.getConfigManager().getConsensusManager().read(new ShowCQPlan(req.cqId)); - return response.getCqList().stream() - .filter(entry -> cqToken.equals(entry.getCqToken())) - .findFirst(); - } - - private static String generateCQToken() { - return UUID.randomUUID().toString(); - } - - private void submitScheduleTask(ConfigNodeProcedureEnv env, CQScheduleTask cqScheduleTask) { - CQManager cqManager = env.getConfigManager().getCQManager(); - if (!cqManager.markCQLocallyScheduled(req.cqId, cqToken, cqScheduleTask)) { - return; - } - try { - cqScheduleTask.submitSelf(); - } catch (RuntimeException e) { - cqManager.unmarkCQLocallyScheduled(req.cqId, cqToken); - throw e; - } - } - @Override protected void rollbackState(ConfigNodeProcedureEnv env, CreateCQState state) throws IOException, InterruptedException, ProcedureException { @@ -221,8 +176,7 @@ protected void rollbackState(ConfigNodeProcedureEnv env, CreateCQState state) LOGGER.info("Start [INACTIVE] rollback of CQ {}", req.cqId); TSStatus res; try { - res = - env.getConfigManager().getConsensusManager().write(new DropCQPlan(req.cqId, cqToken)); + res = env.getConfigManager().getConsensusManager().write(new DropCQPlan(req.cqId, md5)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -273,7 +227,7 @@ public void serialize(DataOutputStream stream) throws IOException { stream.writeShort(ProcedureType.CREATE_CQ_PROCEDURE.getTypeCode()); super.serialize(stream); ThriftCommonsSerDeUtils.serializeTCreateCQReq(req, stream); - ReadWriteIOUtils.write(cqToken, stream); + ReadWriteIOUtils.write(md5, stream); ReadWriteIOUtils.write(firstExecutionTime, stream); } @@ -281,7 +235,7 @@ public void serialize(DataOutputStream stream) throws IOException { public void deserialize(ByteBuffer byteBuffer) { super.deserialize(byteBuffer); this.req = ThriftCommonsSerDeUtils.deserializeTCreateCQReq(byteBuffer); - this.cqToken = ReadWriteIOUtils.readString(byteBuffer); + this.md5 = ReadWriteIOUtils.readString(byteBuffer); this.firstExecutionTime = ReadWriteIOUtils.readLong(byteBuffer); } @@ -300,7 +254,7 @@ && getCycles() == that.getCycles() && isGeneratedByPipe == that.isGeneratedByPipe && firstExecutionTime == that.firstExecutionTime && Objects.equals(req, that.req) - && Objects.equals(cqToken, that.cqToken); + && Objects.equals(md5, that.md5); } @Override @@ -311,15 +265,7 @@ public int hashCode() { getCycles(), isGeneratedByPipe, req, - cqToken, + md5, firstExecutionTime); } - - public String getCqId() { - return req == null ? null : req.getCqId(); - } - - public String getCqToken() { - return cqToken; - } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java index 1c73fe323cec3..3b0554baa11fa 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java @@ -1213,7 +1213,7 @@ public void UpdateTriggerStateInTablePlanTest() throws IOException { @Test public void ActiveCQPlanTest() throws IOException { - ActiveCQPlan activeCQPlan0 = new ActiveCQPlan("testCq", "testCqToken"); + ActiveCQPlan activeCQPlan0 = new ActiveCQPlan("testCq", "testCq_md5"); ActiveCQPlan activeCQPlan1 = (ActiveCQPlan) ConfigPhysicalPlan.Factory.create(activeCQPlan0.serializeToByteBuffer()); @@ -1236,7 +1236,7 @@ public void AddCQPlanTest() throws IOException { "create cq testCq1 BEGIN select s1 into root.backup.d1.s1 from root.sg.d1 END", "Asia", "root"), - "testCq1Token", + "testCq1_md5", executionTime); AddCQPlan addCQPlan1 = (AddCQPlan) ConfigPhysicalPlan.Factory.create(addCQPlan0.serializeToByteBuffer()); @@ -1251,7 +1251,7 @@ public void DropCQPlanTest() throws IOException { (DropCQPlan) ConfigPhysicalPlan.Factory.create(dropCQPlan0.serializeToByteBuffer()); Assert.assertEquals(dropCQPlan0, dropCQPlan1); - dropCQPlan0 = new DropCQPlan("testCq1", "testCq1Token"); + dropCQPlan0 = new DropCQPlan("testCq1", "testCq1_md5"); dropCQPlan1 = (DropCQPlan) ConfigPhysicalPlan.Factory.create(dropCQPlan0.serializeToByteBuffer()); Assert.assertEquals(dropCQPlan0, dropCQPlan1); @@ -1260,7 +1260,7 @@ public void DropCQPlanTest() throws IOException { @Test public void UpdateCQLastExecTimePlanTest() throws IOException { UpdateCQLastExecTimePlan updateCQLastExecTimePlan0 = - new UpdateCQLastExecTimePlan("testCq", System.currentTimeMillis(), "testCqToken"); + new UpdateCQLastExecTimePlan("testCq", System.currentTimeMillis(), "testCq_md5"); UpdateCQLastExecTimePlan updateCQLastExecTimePlan1 = (UpdateCQLastExecTimePlan) ConfigPhysicalPlan.Factory.create(updateCQLastExecTimePlan0.serializeToByteBuffer()); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java deleted file mode 100644 index a0bc5a523ba70..0000000000000 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iotdb.confignode.cq; - -import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.cq.TimeoutPolicy; -import org.apache.iotdb.confignode.manager.ConfigManager; -import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; -import org.apache.iotdb.confignode.manager.cq.CQManager; -import org.apache.iotdb.confignode.manager.cq.CQScheduleTask; -import org.apache.iotdb.confignode.rpc.thrift.TDropCQReq; -import org.apache.iotdb.rpc.TSStatusCode; - -import org.junit.Test; -import org.mockito.Mockito; - -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; - -import static org.junit.Assert.assertTrue; - -public class CQManagerTest { - - @SuppressWarnings("unchecked") - @Test - public void dropCQShouldCancelLocallyScheduledTask() throws Exception { - ConfigManager configManager = Mockito.mock(ConfigManager.class); - ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); - Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); - Mockito.when(consensusManager.write(Mockito.any())) - .thenReturn(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); - CQManager cqManager = new CQManager(configManager); - ScheduledFuture future = Mockito.mock(ScheduledFuture.class); - CQScheduleTask task = newScheduledTask(configManager, future, "token"); - - try { - assertTrue(cqManager.markCQLocallyScheduled("testCq", "token", task)); - task.submitSelf(); - cqManager.dropCQ(new TDropCQReq("testCq")); - - Mockito.verify(future).cancel(false); - } finally { - cqManager.stopCQScheduler(); - } - } - - @SuppressWarnings("unchecked") - @Test - public void newTokenShouldCancelPreviousLocallyScheduledTask() { - ConfigManager configManager = Mockito.mock(ConfigManager.class); - CQManager cqManager = new CQManager(configManager); - ScheduledFuture previousFuture = Mockito.mock(ScheduledFuture.class); - CQScheduleTask previousTask = newScheduledTask(configManager, previousFuture, "previousToken"); - ScheduledFuture currentFuture = Mockito.mock(ScheduledFuture.class); - CQScheduleTask currentTask = newScheduledTask(configManager, currentFuture, "currentToken"); - - try { - assertTrue(cqManager.markCQLocallyScheduled("testCq", "previousToken", previousTask)); - previousTask.submitSelf(); - assertTrue(cqManager.markCQLocallyScheduled("testCq", "currentToken", currentTask)); - - Mockito.verify(previousFuture).cancel(false); - } finally { - cqManager.stopCQScheduler(); - } - } - - @SuppressWarnings("unchecked") - private CQScheduleTask newScheduledTask( - ConfigManager configManager, ScheduledFuture scheduledFuture, String cqToken) { - ScheduledExecutorService executor = Mockito.mock(ScheduledExecutorService.class); - Mockito.when( - executor.schedule( - Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class))) - .thenReturn((ScheduledFuture) scheduledFuture); - return new CQScheduleTask( - "testCq", - 1000, - 0, - 1000, - TimeoutPolicy.BLOCKED, - "select s1 into root.backup.d1.s1 from root.sg.d1", - cqToken, - "Asia", - "root", - executor, - configManager, - System.currentTimeMillis() + 10_000); - } -} diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java index 8bf2d12cd5941..bfadd3d05e8df 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java @@ -18,14 +18,9 @@ */ package org.apache.iotdb.confignode.persistence; -import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; -import org.apache.iotdb.confignode.consensus.request.write.cq.DropCQPlan; -import org.apache.iotdb.confignode.consensus.request.write.cq.UpdateCQLastExecTimePlan; -import org.apache.iotdb.confignode.consensus.response.cq.ShowCQResp; import org.apache.iotdb.confignode.persistence.cq.CQInfo; import org.apache.iotdb.confignode.rpc.thrift.TCreateCQReq; -import org.apache.iotdb.rpc.TSStatusCode; import org.apache.commons.io.FileUtils; import org.apache.thrift.TException; @@ -75,7 +70,7 @@ public void testSnapshot() throws TException, IOException { "create cq testCq1 BEGIN select s1 into root.backup.d1.s1 from root.sg.d1 END", "Asia", "root"), - "testCq1Token", + "testCq1_md5", executionTime); cqInfo.addCQ(addCQPlan); @@ -94,7 +89,7 @@ public void testSnapshot() throws TException, IOException { "create cq testCq2 BEGIN select s1 into root.backup.d2.s1 from root.sg.d2 END", "Asia", "root"), - "testCq2Token", + "testCq2_md5", executionTime); cqInfo.addCQ(addCQPlan); @@ -104,59 +99,4 @@ public void testSnapshot() throws TException, IOException { Assert.assertEquals(cqInfo, actualCQInfo); } - - @Test - public void testOldCallbackCannotTouchRecreatedCQ() throws Exception { - long executionTime = System.currentTimeMillis(); - TCreateCQReq req = - new TCreateCQReq( - "testCq3", - 1000, - 0, - 1000, - 0, - (byte) 0, - "select s1 into root.backup.d3.s1 from root.sg.d3", - "create cq testCq3 BEGIN select s1 into root.backup.d3.s1 from root.sg.d3 END", - "Asia", - "root"); - - cqInfo.addCQ(new AddCQPlan(req, "oldToken", executionTime)); - cqInfo.dropCQ(new DropCQPlan("testCq3")); - cqInfo.addCQ(new AddCQPlan(req, "newToken", executionTime)); - - Assert.assertEquals( - TSStatusCode.NO_SUCH_CQ.getStatusCode(), - cqInfo.updateCQLastExecutionTime( - new UpdateCQLastExecTimePlan("testCq3", executionTime + 1000, "oldToken")) - .code); - Assert.assertEquals( - TSStatusCode.SUCCESS_STATUS.getStatusCode(), - cqInfo.updateCQLastExecutionTime( - new UpdateCQLastExecTimePlan("testCq3", executionTime + 1000, "newToken")) - .code); - } - - @Test - public void testShowCQCanFilterByCQId() throws Exception { - long executionTime = System.currentTimeMillis(); - TCreateCQReq req = - new TCreateCQReq( - "testCq4", - 1000, - 0, - 1000, - 0, - (byte) 0, - "select s1 into root.backup.d4.s1 from root.sg.d4", - "create cq testCq4 BEGIN select s1 into root.backup.d4.s1 from root.sg.d4 END", - "Asia", - "root"); - cqInfo.addCQ(new AddCQPlan(req, "testCq4Token", executionTime)); - - ShowCQResp showCQResp = cqInfo.showCQ(new ShowCQPlan("testCq4")); - - Assert.assertEquals(1, showCQResp.getCqList().size()); - Assert.assertEquals("testCq4", showCQResp.getCqList().get(0).getCqId()); - } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java index 3e7fd2052ad53..d0e92b3281666 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java @@ -36,36 +36,10 @@ import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.fail; public class CreateCQProcedureTest { - @Test - public void tokenShouldBeUniqueForSameCQId() { - ScheduledExecutorService executor = Executors.newScheduledThreadPool(1); - try { - TCreateCQReq req = - new TCreateCQReq( - "testCq1", - 1000, - 0, - 1000, - 0, - (byte) 0, - "select s1 into root.backup.d1(s1) from root.sg.d1", - "create cq testCq1 BEGIN select s1 into root.backup.d1(s1) from root.sg.d1 END", - "Asia", - "root"); - CreateCQProcedure createCQProcedure1 = new CreateCQProcedure(req, executor); - CreateCQProcedure createCQProcedure2 = new CreateCQProcedure(req, executor); - - assertNotEquals(createCQProcedure1.getCqToken(), createCQProcedure2.getCqToken()); - } finally { - executor.shutdown(); - } - } - @Test public void serializeDeserializeTest() { diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java deleted file mode 100644 index a90e282494f0e..0000000000000 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.confignode.procedure.impl.cq; - -import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; -import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; -import org.apache.iotdb.confignode.manager.ConfigManager; -import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; -import org.apache.iotdb.confignode.manager.cq.CQManager; -import org.apache.iotdb.confignode.manager.cq.CQScheduleTask; -import org.apache.iotdb.confignode.persistence.cq.CQInfo; -import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; -import org.apache.iotdb.confignode.rpc.thrift.TCreateCQReq; - -import org.junit.Test; -import org.mockito.Mockito; - -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; - -public class CreateCQProcedureRecoveryTest { - - private TCreateCQReq newCreateCQReq() { - return new TCreateCQReq( - "testCq1", - 1000, - 0, - 1000, - 0, - (byte) 0, - "select s1 into root.backup.d1.s1 from root.sg.d1", - "create cq testCq1 BEGIN select s1 into root.backup.d1.s1 from root.sg.d1 END", - "Asia", - "root"); - } - - @SuppressWarnings("unchecked") - @Test - public void recoverScheduledTaskShouldResubmitFromLatestMetadata() throws Exception { - ScheduledExecutorService executor = Mockito.mock(ScheduledExecutorService.class); - Mockito.when( - executor.schedule( - Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class))) - .thenReturn(Mockito.mock(ScheduledFuture.class)); - - ConfigManager configManager = Mockito.mock(ConfigManager.class); - ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); - CQManager cqManager = Mockito.mock(CQManager.class); - ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); - Mockito.when(env.getConfigManager()).thenReturn(configManager); - Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); - Mockito.when(configManager.getCQManager()).thenReturn(cqManager); - Mockito.when( - cqManager.markCQLocallyScheduled( - Mockito.anyString(), Mockito.anyString(), Mockito.any(CQScheduleTask.class))) - .thenReturn(true); - - TCreateCQReq req = newCreateCQReq(); - CreateCQProcedure procedure = new CreateCQProcedure(req, executor); - - CQInfo cqInfo = new CQInfo(); - cqInfo.addCQ(new AddCQPlan(req, procedure.getCqToken(), System.currentTimeMillis() + 10_000)); - Mockito.when(consensusManager.read(Mockito.any(ShowCQPlan.class))).thenReturn(cqInfo.showCQ()); - - procedure.recoverScheduledTask(env); - - Mockito.verify(executor) - .schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class)); - } - - @SuppressWarnings("unchecked") - @Test - public void recoverScheduledTaskShouldSkipDuplicatedLocalSchedule() throws Exception { - ScheduledExecutorService executor = Mockito.mock(ScheduledExecutorService.class); - ConfigManager configManager = Mockito.mock(ConfigManager.class); - ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); - CQManager cqManager = Mockito.mock(CQManager.class); - ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); - Mockito.when(env.getConfigManager()).thenReturn(configManager); - Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); - Mockito.when(configManager.getCQManager()).thenReturn(cqManager); - Mockito.when( - cqManager.markCQLocallyScheduled( - Mockito.anyString(), Mockito.anyString(), Mockito.any(CQScheduleTask.class))) - .thenReturn(false); - - TCreateCQReq req = newCreateCQReq(); - CreateCQProcedure procedure = new CreateCQProcedure(req, executor); - - CQInfo cqInfo = new CQInfo(); - cqInfo.addCQ(new AddCQPlan(req, procedure.getCqToken(), System.currentTimeMillis() + 10_000)); - Mockito.when(consensusManager.read(Mockito.any(ShowCQPlan.class))).thenReturn(cqInfo.showCQ()); - - procedure.recoverScheduledTask(env); - - Mockito.verify(executor, Mockito.never()) - .schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class)); - } -} From cb8fd4ceb00fa5428ca9c931f22b7ab5c3c1b126 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 3 Jun 2026 14:43:57 +0800 Subject: [PATCH 077/102] Reapply "Fix CQ recovery gap and stale callback contamination (#17734) (#17820)" (#17826) (#17829) This reverts commit 5e1587f67e3a115daa301268c466ca2bc0dcee4f. --- iotdb-core/confignode/pom.xml | 4 - .../consensus/request/read/cq/ShowCQPlan.java | 13 ++ .../request/write/cq/ActiveCQPlan.java | 20 +-- .../consensus/request/write/cq/AddCQPlan.java | 20 +-- .../request/write/cq/DropCQPlan.java | 20 +-- .../write/cq/UpdateCQLastExecTimePlan.java | 23 ++-- .../confignode/manager/cq/CQManager.java | 102 ++++++++++++++- .../confignode/manager/cq/CQScheduleTask.java | 58 +++++++-- .../confignode/persistence/cq/CQInfo.java | 76 +++++++----- .../executor/ConfigPlanExecutor.java | 3 +- .../procedure/impl/cq/CreateCQProcedure.java | 80 ++++++++++-- .../request/ConfigPhysicalPlanSerDeTest.java | 8 +- .../iotdb/confignode/cq/CQManagerTest.java | 107 ++++++++++++++++ .../confignode/persistence/CQInfoTest.java | 64 +++++++++- .../procedure/impl/CreateCQProcedureTest.java | 26 ++++ .../cq/CreateCQProcedureRecoveryTest.java | 117 ++++++++++++++++++ 16 files changed, 635 insertions(+), 106 deletions(-) create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java diff --git a/iotdb-core/confignode/pom.xml b/iotdb-core/confignode/pom.xml index fb22fd5b091f7..a7cd4b6298b5f 100644 --- a/iotdb-core/confignode/pom.xml +++ b/iotdb-core/confignode/pom.xml @@ -137,10 +137,6 @@ org.apache.commons commons-lang3 - - commons-codec - commons-codec - org.apache.thrift libthrift diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java index 5217849deb488..c28838d556b48 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/read/cq/ShowCQPlan.java @@ -21,11 +21,24 @@ import org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan; +import java.util.Optional; + import static org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType.SHOW_CQ; public class ShowCQPlan extends ConfigPhysicalReadPlan { + private final String cqId; + public ShowCQPlan() { + this(null); + } + + public ShowCQPlan(String cqId) { super(SHOW_CQ); + this.cqId = cqId; + } + + public Optional getCqId() { + return Optional.ofNullable(cqId); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java index e488ac2566921..4ca33b054a93f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/ActiveCQPlan.java @@ -35,39 +35,39 @@ public class ActiveCQPlan extends ConfigPhysicalPlan { private String cqId; - private String md5; + private String cqToken; public ActiveCQPlan() { super(ACTIVE_CQ); } - public ActiveCQPlan(String cqId, String md5) { + public ActiveCQPlan(String cqId, String cqToken) { super(ACTIVE_CQ); Validate.notNull(cqId); - Validate.notNull(md5); + Validate.notNull(cqToken); this.cqId = cqId; - this.md5 = md5; + this.cqToken = cqToken; } public String getCqId() { return cqId; } - public String getMd5() { - return md5; + public String getCqToken() { + return cqToken; } @Override protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(cqId, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { cqId = ReadWriteIOUtils.readString(buffer); - md5 = ReadWriteIOUtils.readString(buffer); + cqToken = ReadWriteIOUtils.readString(buffer); } @Override @@ -82,11 +82,11 @@ public boolean equals(Object o) { return false; } ActiveCQPlan that = (ActiveCQPlan) o; - return cqId.equals(that.cqId) && md5.equals(that.md5); + return cqId.equals(that.cqId) && cqToken.equals(that.cqToken); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), cqId, md5); + return Objects.hash(super.hashCode(), cqId, cqToken); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java index 0aae9e2e974ca..721c83d3d2f76 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/AddCQPlan.java @@ -37,7 +37,7 @@ public class AddCQPlan extends ConfigPhysicalPlan { private TCreateCQReq req; - private String md5; + private String cqToken; private long firstExecutionTime; @@ -45,12 +45,12 @@ public AddCQPlan() { super(ADD_CQ); } - public AddCQPlan(TCreateCQReq req, String md5, long firstExecutionTime) { + public AddCQPlan(TCreateCQReq req, String cqToken, long firstExecutionTime) { super(ADD_CQ); Validate.notNull(req); - Validate.notNull(md5); + Validate.notNull(cqToken); this.req = req; - this.md5 = md5; + this.cqToken = cqToken; this.firstExecutionTime = firstExecutionTime; } @@ -58,8 +58,8 @@ public TCreateCQReq getReq() { return req; } - public String getMd5() { - return md5; + public String getCqToken() { + return cqToken; } public long getFirstExecutionTime() { @@ -70,14 +70,14 @@ public long getFirstExecutionTime() { protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ThriftCommonsSerDeUtils.serializeTCreateCQReq(req, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); ReadWriteIOUtils.write(firstExecutionTime, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { req = ThriftCommonsSerDeUtils.deserializeTCreateCQReq(buffer); - md5 = ReadWriteIOUtils.readString(buffer); + cqToken = ReadWriteIOUtils.readString(buffer); firstExecutionTime = ReadWriteIOUtils.readLong(buffer); } @@ -95,11 +95,11 @@ public boolean equals(Object o) { AddCQPlan addCQPlan = (AddCQPlan) o; return firstExecutionTime == addCQPlan.firstExecutionTime && Objects.equals(req, addCQPlan.req) - && Objects.equals(md5, addCQPlan.md5); + && Objects.equals(cqToken, addCQPlan.cqToken); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), req, md5, firstExecutionTime); + return Objects.hash(super.hashCode(), req, cqToken, firstExecutionTime); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java index 5c901362997f8..f22561c0b807d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/DropCQPlan.java @@ -37,7 +37,7 @@ public class DropCQPlan extends ConfigPhysicalPlan { private String cqId; // may be null in user call of drop CQ - private String md5; + private String cqToken; public DropCQPlan() { super(DROP_CQ); @@ -49,33 +49,33 @@ public DropCQPlan(String cqId) { this.cqId = cqId; } - public DropCQPlan(String cqId, String md5) { + public DropCQPlan(String cqId, String cqToken) { super(DROP_CQ); Validate.notNull(cqId); - Validate.notNull(md5); + Validate.notNull(cqToken); this.cqId = cqId; - this.md5 = md5; + this.cqToken = cqToken; } public String getCqId() { return cqId; } - public Optional getMd5() { - return Optional.ofNullable(md5); + public Optional getCqToken() { + return Optional.ofNullable(cqToken); } @Override protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(cqId, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { cqId = ReadWriteIOUtils.readString(buffer); - md5 = ReadWriteIOUtils.readString(buffer); + cqToken = ReadWriteIOUtils.readString(buffer); } @Override @@ -90,11 +90,11 @@ public boolean equals(Object o) { return false; } DropCQPlan that = (DropCQPlan) o; - return cqId.equals(that.cqId) && Objects.equals(md5, that.md5); + return cqId.equals(that.cqId) && Objects.equals(cqToken, that.cqToken); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), cqId, md5); + return Objects.hash(super.hashCode(), cqId, cqToken); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java index 51a8358a04c28..392f6006ba56e 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/cq/UpdateCQLastExecTimePlan.java @@ -37,20 +37,19 @@ public class UpdateCQLastExecTimePlan extends ConfigPhysicalPlan { private long executionTime; - // may be null in user call of drop CQ - private String md5; + private String cqToken; public UpdateCQLastExecTimePlan() { super(UPDATE_CQ_LAST_EXEC_TIME); } - public UpdateCQLastExecTimePlan(String cqId, long executionTime, String md5) { + public UpdateCQLastExecTimePlan(String cqId, long executionTime, String cqToken) { super(UPDATE_CQ_LAST_EXEC_TIME); Validate.notNull(cqId); - Validate.notNull(md5); + Validate.notNull(cqToken); this.cqId = cqId; this.executionTime = executionTime; - this.md5 = md5; + this.cqToken = cqToken; } public String getCqId() { @@ -61,8 +60,8 @@ public long getExecutionTime() { return executionTime; } - public String getMd5() { - return md5; + public String getCqToken() { + return cqToken; } @Override @@ -70,14 +69,14 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ReadWriteIOUtils.write(cqId, stream); ReadWriteIOUtils.write(executionTime, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { cqId = ReadWriteIOUtils.readString(buffer); executionTime = ReadWriteIOUtils.readLong(buffer); - md5 = ReadWriteIOUtils.readString(buffer); + cqToken = ReadWriteIOUtils.readString(buffer); } @Override @@ -92,11 +91,13 @@ public boolean equals(Object o) { return false; } UpdateCQLastExecTimePlan that = (UpdateCQLastExecTimePlan) o; - return executionTime == that.executionTime && cqId.equals(that.cqId) && md5.equals(that.md5); + return executionTime == that.executionTime + && cqId.equals(that.cqId) + && cqToken.equals(that.cqToken); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), cqId, executionTime, md5); + return Objects.hash(super.hashCode(), cqId, executionTime, cqToken); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java index 5726b3ce82698..a90a21f285ff2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQManager.java @@ -42,7 +42,10 @@ import java.util.Collections; import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -56,11 +59,15 @@ public class CQManager { private final ReadWriteLock lock; + // Key: CQ id. Value: the local task and the metadata token it owns. + private final ConcurrentMap locallyScheduledCQs; + private ScheduledExecutorService executor; public CQManager(ConfigManager configManager) { this.configManager = configManager; this.lock = new ReentrantReadWriteLock(); + this.locallyScheduledCQs = new ConcurrentHashMap<>(); this.executor = IoTDBThreadPoolFactory.newScheduledThreadPool( CONF.getCqSubmitThread(), ThreadName.CQ_SCHEDULER.getName()); @@ -77,14 +84,21 @@ public TSStatus createCQ(TCreateCQReq req) { } public TSStatus dropCQ(TDropCQReq req) { + lock.readLock().lock(); try { - return configManager.getConsensusManager().write(new DropCQPlan(req.cqId)); + TSStatus status = configManager.getConsensusManager().write(new DropCQPlan(req.cqId)); + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + cancelLocallyScheduledCQ(req.cqId); + } + return status; } catch (ConsensusException e) { LOGGER.warn("Unexpected error happened while dropping cq {}: ", req.cqId, e); // consensus layer related errors TSStatus res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); res.setMessage(e.getMessage()); return res; + } finally { + lock.readLock().unlock(); } } @@ -117,6 +131,7 @@ public void startCQScheduler() { try { // 1. shutdown previous cq schedule thread pool try { + cancelAllLocallyScheduledCQs(); if (executor != null) { executor.shutdown(); } @@ -154,7 +169,15 @@ public void startCQScheduler() { for (CQInfo.CQEntry entry : allCQs) { if (entry.getState() == CQState.ACTIVE) { CQScheduleTask cqScheduleTask = new CQScheduleTask(entry, executor, configManager); - cqScheduleTask.submitSelf(); + if (!markCQLocallyScheduled(entry.getCqId(), entry.getCqToken(), cqScheduleTask)) { + continue; + } + try { + cqScheduleTask.submitSelf(); + } catch (RuntimeException e) { + unmarkCQLocallyScheduled(entry.getCqId(), entry.getCqToken()); + throw e; + } } } } @@ -174,6 +197,7 @@ public void stopCQScheduler() { try { previous = executor; executor = null; + cancelAllLocallyScheduledCQs(); } finally { lock.writeLock().unlock(); } @@ -181,4 +205,78 @@ public void stopCQScheduler() { previous.shutdown(); } } + + public boolean markCQLocallyScheduled(String cqId, String cqToken, CQScheduleTask task) { + AtomicBoolean shouldSchedule = new AtomicBoolean(false); + LocallyScheduledCQ schedule = new LocallyScheduledCQ(cqToken, task); + lock.readLock().lock(); + try { + locallyScheduledCQs.compute( + cqId, + (ignored, previousSchedule) -> { + if (previousSchedule != null && previousSchedule.hasToken(cqToken)) { + return previousSchedule; + } + if (previousSchedule != null) { + previousSchedule.cancel(); + } + shouldSchedule.set(true); + return schedule; + }); + if (!shouldSchedule.get()) { + task.cancel(); + } + return shouldSchedule.get(); + } finally { + lock.readLock().unlock(); + } + } + + public void unmarkCQLocallyScheduled(String cqId, String cqToken) { + lock.readLock().lock(); + try { + locallyScheduledCQs.computeIfPresent( + cqId, + (ignored, schedule) -> { + if (schedule.hasToken(cqToken)) { + schedule.cancel(); + return null; + } + return schedule; + }); + } finally { + lock.readLock().unlock(); + } + } + + private void cancelLocallyScheduledCQ(String cqId) { + LocallyScheduledCQ schedule = locallyScheduledCQs.remove(cqId); + if (schedule != null) { + schedule.cancel(); + } + } + + private void cancelAllLocallyScheduledCQs() { + locallyScheduledCQs.values().forEach(LocallyScheduledCQ::cancel); + locallyScheduledCQs.clear(); + } + + private static class LocallyScheduledCQ { + + private final String cqToken; + private final CQScheduleTask task; + + private LocallyScheduledCQ(String cqToken, CQScheduleTask task) { + this.cqToken = cqToken; + this.task = task; + } + + private boolean hasToken(String cqToken) { + return this.cqToken.equals(cqToken); + } + + private void cancel() { + task.cancel(); + } + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java index 6125edc1ef7bd..d85d0f12a3040 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/cq/CQScheduleTask.java @@ -39,7 +39,10 @@ import java.util.Optional; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; public class CQScheduleTask implements Runnable { @@ -69,7 +72,7 @@ public class CQScheduleTask implements Runnable { private final long endTimeOffset; private final TimeoutPolicy timeoutPolicy; private final String queryBody; - private final String md5; + private final String cqToken; private final String zoneId; @@ -81,12 +84,15 @@ public class CQScheduleTask implements Runnable { private final long retryWaitTimeInMS; + private final AtomicBoolean cancelled; + private final AtomicReference> scheduledFuture; + private long executionTime; public CQScheduleTask( TCreateCQReq req, long firstExecutionTime, - String md5, + String cqToken, ScheduledExecutorService executor, ConfigManager configManager) { this( @@ -96,7 +102,7 @@ public CQScheduleTask( req.endTimeOffset, TimeoutPolicy.deserialize(req.timeoutPolicy), req.queryBody, - md5, + cqToken, req.zoneId, req.username, executor, @@ -113,7 +119,7 @@ public CQScheduleTask( entry.getEndTimeOffset(), entry.getTimeoutPolicy(), entry.getQueryBody(), - entry.getMd5(), + entry.getCqToken(), entry.getZoneId(), entry.getUsername(), executor, @@ -129,7 +135,7 @@ public CQScheduleTask( long endTimeOffset, TimeoutPolicy timeoutPolicy, String queryBody, - String md5, + String cqToken, String zoneId, String username, ScheduledExecutorService executor, @@ -141,12 +147,14 @@ public CQScheduleTask( this.endTimeOffset = endTimeOffset; this.timeoutPolicy = timeoutPolicy; this.queryBody = queryBody; - this.md5 = md5; + this.cqToken = cqToken; this.zoneId = zoneId; this.username = username; this.executor = executor; this.configManager = configManager; this.retryWaitTimeInMS = Math.min(DEFAULT_RETRY_WAIT_TIME_IN_MS, everyInterval / FACTOR); + this.cancelled = new AtomicBoolean(false); + this.scheduledFuture = new AtomicReference<>(); this.executionTime = executionTime; } @@ -165,6 +173,9 @@ public static long getFirstExecutionTime(long boundaryTime, long everyInterval, @Override public void run() { + if (cancelled.get()) { + return; + } long startTime = executionTime - startTimeOffset; long endTime = executionTime - endTimeOffset; @@ -177,6 +188,9 @@ public void run() { submitSelf(retryWaitTimeInMS, TimeUnit.MILLISECONDS); } } else { + if (cancelled.get()) { + return; + } LOGGER.info( "[StartExecuteCQ] execute CQ {} on DataNode[{}], time range is [{}, {}), current time is {}", cqId, @@ -206,12 +220,32 @@ public void submitSelf() { } private void submitSelf(long delay, TimeUnit unit) { - executor.schedule(this, delay, unit); + if (cancelled.get()) { + return; + } + ScheduledFuture newFuture = executor.schedule(this, delay, unit); + ScheduledFuture previousFuture = scheduledFuture.getAndSet(newFuture); + if (previousFuture != null) { + previousFuture.cancel(false); + } + if (cancelled.get() && scheduledFuture.compareAndSet(newFuture, null)) { + newFuture.cancel(false); + } + } + + public void cancel() { + cancelled.set(true); + ScheduledFuture currentFuture = scheduledFuture.getAndSet(null); + if (currentFuture != null) { + currentFuture.cancel(false); + } } private boolean needSubmit() { // current node is still leader and thread pool is not shut down. - return configManager.getConsensusManager().isLeader() && !executor.isShutdown(); + return !cancelled.get() + && configManager.getConsensusManager().isLeader() + && !executor.isShutdown(); } private class AsyncExecuteCQCallback implements AsyncMethodCallback { @@ -238,6 +272,9 @@ private void updateExecutionTime() { @Override public void onComplete(TSStatus response) { + if (cancelled.get()) { + return; + } if (response.code == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { LOGGER.info( @@ -251,7 +288,7 @@ public void onComplete(TSStatus response) { result = configManager .getConsensusManager() - .write(new UpdateCQLastExecTimePlan(cqId, executionTime, md5)); + .write(new UpdateCQLastExecTimePlan(cqId, executionTime, cqToken)); } catch (ConsensusException e) { result = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); result.setMessage(e.getMessage()); @@ -291,6 +328,9 @@ public void onComplete(TSStatus response) { @Override public void onError(Exception exception) { + if (cancelled.get()) { + return; + } LOGGER.warn("Execute CQ {} failed", cqId, exception); if (needSubmit()) { submitSelf(retryWaitTimeInMS, TimeUnit.MILLISECONDS); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java index 4c1225ef48a05..b24f1ca9aa528 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/cq/CQInfo.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.cq.CQState; import org.apache.iotdb.commons.cq.TimeoutPolicy; import org.apache.iotdb.commons.snapshot.SnapshotProcessor; +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.ActiveCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.DropCQPlan; @@ -44,7 +45,9 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -61,7 +64,7 @@ public class CQInfo implements SnapshotProcessor { private static final String CQ_NOT_EXIST_FORMAT = "CQ %s doesn't exist."; - private static final String MD5_NOT_MATCH_FORMAT = "MD5 of CQ %s doesn't match"; + private static final String CQ_TOKEN_NOT_MATCH_FORMAT = "Token of CQ %s doesn't match"; private final Map cqMap; @@ -91,7 +94,7 @@ public TSStatus addCQ(AddCQPlan plan) { CQEntry cqEntry = new CQEntry( plan.getReq(), - plan.getMd5(), + plan.getCqToken(), plan.getFirstExecutionTime() - plan.getReq().everyInterval); cqMap.put(cqId, cqEntry); res.code = TSStatusCode.SUCCESS_STATUS.getStatusCode(); @@ -105,13 +108,13 @@ public TSStatus addCQ(AddCQPlan plan) { /** * Drop the CQ whose ID is same as cqId in plan. * - * @return SUCCESS_STATUS if there is CQ whose ID and md5 is same as cqId in plan, + * @return SUCCESS_STATUS if there is CQ whose ID and token is same as cqId in plan, * otherwise NO_SUCH_CQ. */ public TSStatus dropCQ(DropCQPlan plan) { TSStatus res = new TSStatus(); String cqId = plan.getCqId(); - Optional md5 = plan.getMd5(); + Optional cqToken = plan.getCqToken(); lock.writeLock().lock(); try { CQEntry cqEntry = cqMap.get(cqId); @@ -119,10 +122,10 @@ public TSStatus dropCQ(DropCQPlan plan) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); res.message = String.format(CQ_NOT_EXIST_FORMAT, cqId); LOGGER.warn("Drop CQ {} failed, because it doesn't exist.", cqId); - } else if ((md5.isPresent() && !md5.get().equals(cqEntry.md5))) { + } else if ((cqToken.isPresent() && !cqToken.get().equals(cqEntry.cqToken))) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); - res.message = String.format(MD5_NOT_MATCH_FORMAT, cqId); - LOGGER.warn("Drop CQ {} failed, because its MD5 doesn't match.", cqId); + res.message = String.format(CQ_TOKEN_NOT_MATCH_FORMAT, cqId); + LOGGER.warn("Drop CQ {} failed, because its token doesn't match.", cqId); } else { cqMap.remove(cqId); res.code = TSStatusCode.SUCCESS_STATUS.getStatusCode(); @@ -135,11 +138,24 @@ public TSStatus dropCQ(DropCQPlan plan) { } public ShowCQResp showCQ() { + return showCQ(new ShowCQPlan()); + } + + public ShowCQResp showCQ(ShowCQPlan plan) { lock.readLock().lock(); try { - return new ShowCQResp( - new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), - cqMap.values().stream().map(CQEntry::new).collect(Collectors.toList())); + Optional cqId = plan.getCqId(); + List cqList; + if (cqId.isPresent()) { + CQEntry cqEntry = cqMap.get(cqId.get()); + cqList = + cqEntry == null + ? Collections.emptyList() + : Collections.singletonList(new CQEntry(cqEntry)); + } else { + cqList = cqMap.values().stream().map(CQEntry::new).collect(Collectors.toList()); + } + return new ShowCQResp(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), cqList); } finally { lock.readLock().unlock(); } @@ -153,16 +169,16 @@ public ShowCQResp showCQ() { public TSStatus activeCQ(ActiveCQPlan plan) { TSStatus res = new TSStatus(); String cqId = plan.getCqId(); - String md5 = plan.getMd5(); + String cqToken = plan.getCqToken(); lock.writeLock().lock(); try { CQEntry cqEntry = cqMap.get(cqId); if (cqEntry == null) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); res.message = String.format(CQ_NOT_EXIST_FORMAT, cqId); - } else if (!md5.equals(cqEntry.md5)) { + } else if (!cqToken.equals(cqEntry.cqToken)) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); - res.message = String.format(MD5_NOT_MATCH_FORMAT, cqId); + res.message = String.format(CQ_TOKEN_NOT_MATCH_FORMAT, cqId); } else if (cqEntry.state == CQState.ACTIVE) { res.code = TSStatusCode.CQ_ALREADY_ACTIVE.getStatusCode(); res.message = String.format("CQ %s has already been active", cqId); @@ -180,22 +196,22 @@ public TSStatus activeCQ(ActiveCQPlan plan) { * Update the last execution time of the corresponding CQ. * * @return SUCCESS_STATUS if successfully updated, or NO_SUCH_CQ if 1. the CQ doesn't exist; or 2. - * md5 is different. or CQ_UPDATE_LAST_EXEC_TIME_FAILED 3. original lastExecutionTime >= + * token is different. or CQ_UPDATE_LAST_EXEC_TIME_FAILED 3. original lastExecutionTime >= * current lastExecutionTime; */ public TSStatus updateCQLastExecutionTime(UpdateCQLastExecTimePlan plan) { TSStatus res = new TSStatus(); String cqId = plan.getCqId(); - String md5 = plan.getMd5(); + String cqToken = plan.getCqToken(); lock.writeLock().lock(); try { CQEntry cqEntry = cqMap.get(cqId); if (cqEntry == null) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); res.message = String.format(CQ_NOT_EXIST_FORMAT, cqId); - } else if (!md5.equals(cqEntry.md5)) { + } else if (!cqToken.equals(cqEntry.cqToken)) { res.code = TSStatusCode.NO_SUCH_CQ.getStatusCode(); - res.message = String.format(MD5_NOT_MATCH_FORMAT, cqId); + res.message = String.format(CQ_TOKEN_NOT_MATCH_FORMAT, cqId); } else if (cqEntry.lastExecutionTime >= plan.getExecutionTime()) { res.code = TSStatusCode.CQ_UPDATE_LAST_EXEC_TIME_ERROR.getStatusCode(); res.message = @@ -299,7 +315,7 @@ public static class CQEntry { private final TimeoutPolicy timeoutPolicy; private final String queryBody; private final String sql; - private final String md5; + private final String cqToken; private final String zoneId; @@ -308,7 +324,7 @@ public static class CQEntry { private CQState state; private long lastExecutionTime; - private CQEntry(TCreateCQReq req, String md5, long lastExecutionTime) { + private CQEntry(TCreateCQReq req, String cqToken, long lastExecutionTime) { this( req.cqId, req.everyInterval, @@ -318,7 +334,7 @@ private CQEntry(TCreateCQReq req, String md5, long lastExecutionTime) { TimeoutPolicy.deserialize(req.timeoutPolicy), req.queryBody, req.sql, - md5, + cqToken, req.zoneId, req.username, CQState.INACTIVE, @@ -335,7 +351,7 @@ private CQEntry(CQEntry other) { other.timeoutPolicy, other.queryBody, other.sql, - other.md5, + other.cqToken, other.zoneId, other.username, other.state, @@ -352,7 +368,7 @@ private CQEntry( TimeoutPolicy timeoutPolicy, String queryBody, String sql, - String md5, + String cqToken, String zoneId, String username, CQState state, @@ -365,7 +381,7 @@ private CQEntry( this.timeoutPolicy = timeoutPolicy; this.queryBody = queryBody; this.sql = sql; - this.md5 = md5; + this.cqToken = cqToken; this.zoneId = zoneId; this.username = username; this.state = state; @@ -381,7 +397,7 @@ private void serialize(OutputStream stream) throws IOException { ReadWriteIOUtils.write(timeoutPolicy.getType(), stream); ReadWriteIOUtils.write(queryBody, stream); ReadWriteIOUtils.write(sql, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); ReadWriteIOUtils.write(zoneId, stream); ReadWriteIOUtils.write(username, stream); ReadWriteIOUtils.write(state.getType(), stream); @@ -397,7 +413,7 @@ private static CQEntry deserialize(InputStream stream) throws IOException { TimeoutPolicy timeoutPolicy = TimeoutPolicy.deserialize(ReadWriteIOUtils.readByte(stream)); String queryBody = ReadWriteIOUtils.readString(stream); String sql = ReadWriteIOUtils.readString(stream); - String md5 = ReadWriteIOUtils.readString(stream); + String cqToken = ReadWriteIOUtils.readString(stream); String zoneId = ReadWriteIOUtils.readString(stream); String username = ReadWriteIOUtils.readString(stream); CQState state = CQState.deserialize(ReadWriteIOUtils.readByte(stream)); @@ -411,7 +427,7 @@ private static CQEntry deserialize(InputStream stream) throws IOException { timeoutPolicy, queryBody, sql, - md5, + cqToken, zoneId, username, state, @@ -450,8 +466,8 @@ public String getSql() { return sql; } - public String getMd5() { - return md5; + public String getCqToken() { + return cqToken; } public CQState getState() { @@ -488,7 +504,7 @@ public boolean equals(Object o) { && timeoutPolicy == cqEntry.timeoutPolicy && Objects.equals(queryBody, cqEntry.queryBody) && Objects.equals(sql, cqEntry.sql) - && Objects.equals(md5, cqEntry.md5) + && Objects.equals(cqToken, cqEntry.cqToken) && Objects.equals(zoneId, cqEntry.zoneId) && Objects.equals(username, cqEntry.username) && state == cqEntry.state; @@ -505,7 +521,7 @@ public int hashCode() { timeoutPolicy, queryBody, sql, - md5, + cqToken, zoneId, username, state, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index 09017a841c73c..60ec748fb5478 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -30,6 +30,7 @@ import org.apache.iotdb.confignode.consensus.request.read.ConfigPhysicalReadPlan; import org.apache.iotdb.confignode.consensus.request.read.ainode.GetAINodeConfigurationPlan; import org.apache.iotdb.confignode.consensus.request.read.auth.AuthorReadPlan; +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.read.database.CountDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.database.GetDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.datanode.GetDataNodeConfigurationPlan; @@ -322,7 +323,7 @@ public DataSet executeQueryPlan(final ConfigPhysicalReadPlan req) case GetSeriesSlotList: return partitionInfo.getSeriesSlotList((GetSeriesSlotListPlan) req); case SHOW_CQ: - return cqInfo.showCQ(); + return cqInfo.showCQ((ShowCQPlan) req); case GetFunctionTable: return udfInfo.getUDFTable(); case GetFunctionJar: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java index 3da60c5ad847b..af7f968e8a57c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java @@ -22,10 +22,14 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils; +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.ActiveCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.DropCQPlan; +import org.apache.iotdb.confignode.consensus.response.cq.ShowCQResp; +import org.apache.iotdb.confignode.manager.cq.CQManager; import org.apache.iotdb.confignode.manager.cq.CQScheduleTask; +import org.apache.iotdb.confignode.persistence.cq.CQInfo; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.impl.node.AbstractNodeProcedure; @@ -35,7 +39,6 @@ import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.commons.codec.digest.DigestUtils; import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,6 +47,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Objects; +import java.util.Optional; +import java.util.UUID; import java.util.concurrent.ScheduledExecutorService; import static org.apache.iotdb.confignode.procedure.state.cq.CreateCQState.INACTIVE; @@ -59,7 +64,7 @@ public class CreateCQProcedure extends AbstractNodeProcedure { private TCreateCQReq req; - private String md5; + private String cqToken; private long firstExecutionTime; @@ -74,7 +79,7 @@ public CreateCQProcedure(ScheduledExecutorService executor) { public CreateCQProcedure(TCreateCQReq req, ScheduledExecutorService executor) { super(); this.req = req; - this.md5 = DigestUtils.md2Hex(req.cqId); + this.cqToken = generateCQToken(); this.executor = executor; this.firstExecutionTime = CQScheduleTask.getFirstExecutionTime(req.boundaryTime, req.everyInterval); @@ -90,12 +95,16 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, CreateCQState state) addCQ(env); return Flow.HAS_MORE_STATE; case INACTIVE: - CQScheduleTask cqScheduleTask = - new CQScheduleTask(req, firstExecutionTime, md5, executor, env.getConfigManager()); - cqScheduleTask.submitSelf(); + submitScheduleTask( + env, + new CQScheduleTask( + req, firstExecutionTime, cqToken, executor, env.getConfigManager())); setNextState(SCHEDULED); break; case SCHEDULED: + if (isStateDeserialized()) { + recoverScheduledTask(env); + } activeCQ(env); return Flow.NO_MORE_STATE; default: @@ -125,7 +134,7 @@ private void addCQ(ConfigNodeProcedureEnv env) { res = env.getConfigManager() .getConsensusManager() - .write(new AddCQPlan(req, md5, firstExecutionTime)); + .write(new AddCQPlan(req, cqToken, firstExecutionTime)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -146,7 +155,7 @@ private void addCQ(ConfigNodeProcedureEnv env) { private void activeCQ(ConfigNodeProcedureEnv env) { TSStatus res; try { - res = env.getConfigManager().getConsensusManager().write(new ActiveCQPlan(req.cqId, md5)); + res = env.getConfigManager().getConsensusManager().write(new ActiveCQPlan(req.cqId, cqToken)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -164,6 +173,42 @@ private void activeCQ(ConfigNodeProcedureEnv env) { } } + void recoverScheduledTask(ConfigNodeProcedureEnv env) throws ConsensusException { + Optional cqEntry = getCurrentCQEntry(env); + if (!cqEntry.isPresent()) { + LOGGER.info( + "Skip recovering the schedule task of CQ {} because its metadata is unavailable.", + req.cqId); + return; + } + submitScheduleTask(env, new CQScheduleTask(cqEntry.get(), executor, env.getConfigManager())); + } + + Optional getCurrentCQEntry(ConfigNodeProcedureEnv env) throws ConsensusException { + ShowCQResp response = + (ShowCQResp) env.getConfigManager().getConsensusManager().read(new ShowCQPlan(req.cqId)); + return response.getCqList().stream() + .filter(entry -> cqToken.equals(entry.getCqToken())) + .findFirst(); + } + + private static String generateCQToken() { + return UUID.randomUUID().toString(); + } + + private void submitScheduleTask(ConfigNodeProcedureEnv env, CQScheduleTask cqScheduleTask) { + CQManager cqManager = env.getConfigManager().getCQManager(); + if (!cqManager.markCQLocallyScheduled(req.cqId, cqToken, cqScheduleTask)) { + return; + } + try { + cqScheduleTask.submitSelf(); + } catch (RuntimeException e) { + cqManager.unmarkCQLocallyScheduled(req.cqId, cqToken); + throw e; + } + } + @Override protected void rollbackState(ConfigNodeProcedureEnv env, CreateCQState state) throws IOException, InterruptedException, ProcedureException { @@ -176,7 +221,8 @@ protected void rollbackState(ConfigNodeProcedureEnv env, CreateCQState state) LOGGER.info("Start [INACTIVE] rollback of CQ {}", req.cqId); TSStatus res; try { - res = env.getConfigManager().getConsensusManager().write(new DropCQPlan(req.cqId, md5)); + res = + env.getConfigManager().getConsensusManager().write(new DropCQPlan(req.cqId, cqToken)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -227,7 +273,7 @@ public void serialize(DataOutputStream stream) throws IOException { stream.writeShort(ProcedureType.CREATE_CQ_PROCEDURE.getTypeCode()); super.serialize(stream); ThriftCommonsSerDeUtils.serializeTCreateCQReq(req, stream); - ReadWriteIOUtils.write(md5, stream); + ReadWriteIOUtils.write(cqToken, stream); ReadWriteIOUtils.write(firstExecutionTime, stream); } @@ -235,7 +281,7 @@ public void serialize(DataOutputStream stream) throws IOException { public void deserialize(ByteBuffer byteBuffer) { super.deserialize(byteBuffer); this.req = ThriftCommonsSerDeUtils.deserializeTCreateCQReq(byteBuffer); - this.md5 = ReadWriteIOUtils.readString(byteBuffer); + this.cqToken = ReadWriteIOUtils.readString(byteBuffer); this.firstExecutionTime = ReadWriteIOUtils.readLong(byteBuffer); } @@ -254,7 +300,7 @@ && getCycles() == that.getCycles() && isGeneratedByPipe == that.isGeneratedByPipe && firstExecutionTime == that.firstExecutionTime && Objects.equals(req, that.req) - && Objects.equals(md5, that.md5); + && Objects.equals(cqToken, that.cqToken); } @Override @@ -265,7 +311,15 @@ public int hashCode() { getCycles(), isGeneratedByPipe, req, - md5, + cqToken, firstExecutionTime); } + + public String getCqId() { + return req == null ? null : req.getCqId(); + } + + public String getCqToken() { + return cqToken; + } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java index 3b0554baa11fa..1c73fe323cec3 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java @@ -1213,7 +1213,7 @@ public void UpdateTriggerStateInTablePlanTest() throws IOException { @Test public void ActiveCQPlanTest() throws IOException { - ActiveCQPlan activeCQPlan0 = new ActiveCQPlan("testCq", "testCq_md5"); + ActiveCQPlan activeCQPlan0 = new ActiveCQPlan("testCq", "testCqToken"); ActiveCQPlan activeCQPlan1 = (ActiveCQPlan) ConfigPhysicalPlan.Factory.create(activeCQPlan0.serializeToByteBuffer()); @@ -1236,7 +1236,7 @@ public void AddCQPlanTest() throws IOException { "create cq testCq1 BEGIN select s1 into root.backup.d1.s1 from root.sg.d1 END", "Asia", "root"), - "testCq1_md5", + "testCq1Token", executionTime); AddCQPlan addCQPlan1 = (AddCQPlan) ConfigPhysicalPlan.Factory.create(addCQPlan0.serializeToByteBuffer()); @@ -1251,7 +1251,7 @@ public void DropCQPlanTest() throws IOException { (DropCQPlan) ConfigPhysicalPlan.Factory.create(dropCQPlan0.serializeToByteBuffer()); Assert.assertEquals(dropCQPlan0, dropCQPlan1); - dropCQPlan0 = new DropCQPlan("testCq1", "testCq1_md5"); + dropCQPlan0 = new DropCQPlan("testCq1", "testCq1Token"); dropCQPlan1 = (DropCQPlan) ConfigPhysicalPlan.Factory.create(dropCQPlan0.serializeToByteBuffer()); Assert.assertEquals(dropCQPlan0, dropCQPlan1); @@ -1260,7 +1260,7 @@ public void DropCQPlanTest() throws IOException { @Test public void UpdateCQLastExecTimePlanTest() throws IOException { UpdateCQLastExecTimePlan updateCQLastExecTimePlan0 = - new UpdateCQLastExecTimePlan("testCq", System.currentTimeMillis(), "testCq_md5"); + new UpdateCQLastExecTimePlan("testCq", System.currentTimeMillis(), "testCqToken"); UpdateCQLastExecTimePlan updateCQLastExecTimePlan1 = (UpdateCQLastExecTimePlan) ConfigPhysicalPlan.Factory.create(updateCQLastExecTimePlan0.serializeToByteBuffer()); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java new file mode 100644 index 0000000000000..a0bc5a523ba70 --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/cq/CQManagerTest.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iotdb.confignode.cq; + +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.cq.TimeoutPolicy; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; +import org.apache.iotdb.confignode.manager.cq.CQManager; +import org.apache.iotdb.confignode.manager.cq.CQScheduleTask; +import org.apache.iotdb.confignode.rpc.thrift.TDropCQReq; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertTrue; + +public class CQManagerTest { + + @SuppressWarnings("unchecked") + @Test + public void dropCQShouldCancelLocallyScheduledTask() throws Exception { + ConfigManager configManager = Mockito.mock(ConfigManager.class); + ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); + Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); + Mockito.when(consensusManager.write(Mockito.any())) + .thenReturn(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + CQManager cqManager = new CQManager(configManager); + ScheduledFuture future = Mockito.mock(ScheduledFuture.class); + CQScheduleTask task = newScheduledTask(configManager, future, "token"); + + try { + assertTrue(cqManager.markCQLocallyScheduled("testCq", "token", task)); + task.submitSelf(); + cqManager.dropCQ(new TDropCQReq("testCq")); + + Mockito.verify(future).cancel(false); + } finally { + cqManager.stopCQScheduler(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void newTokenShouldCancelPreviousLocallyScheduledTask() { + ConfigManager configManager = Mockito.mock(ConfigManager.class); + CQManager cqManager = new CQManager(configManager); + ScheduledFuture previousFuture = Mockito.mock(ScheduledFuture.class); + CQScheduleTask previousTask = newScheduledTask(configManager, previousFuture, "previousToken"); + ScheduledFuture currentFuture = Mockito.mock(ScheduledFuture.class); + CQScheduleTask currentTask = newScheduledTask(configManager, currentFuture, "currentToken"); + + try { + assertTrue(cqManager.markCQLocallyScheduled("testCq", "previousToken", previousTask)); + previousTask.submitSelf(); + assertTrue(cqManager.markCQLocallyScheduled("testCq", "currentToken", currentTask)); + + Mockito.verify(previousFuture).cancel(false); + } finally { + cqManager.stopCQScheduler(); + } + } + + @SuppressWarnings("unchecked") + private CQScheduleTask newScheduledTask( + ConfigManager configManager, ScheduledFuture scheduledFuture, String cqToken) { + ScheduledExecutorService executor = Mockito.mock(ScheduledExecutorService.class); + Mockito.when( + executor.schedule( + Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class))) + .thenReturn((ScheduledFuture) scheduledFuture); + return new CQScheduleTask( + "testCq", + 1000, + 0, + 1000, + TimeoutPolicy.BLOCKED, + "select s1 into root.backup.d1.s1 from root.sg.d1", + cqToken, + "Asia", + "root", + executor, + configManager, + System.currentTimeMillis() + 10_000); + } +} diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java index bfadd3d05e8df..8bf2d12cd5941 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CQInfoTest.java @@ -18,9 +18,14 @@ */ package org.apache.iotdb.confignode.persistence; +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; +import org.apache.iotdb.confignode.consensus.request.write.cq.DropCQPlan; +import org.apache.iotdb.confignode.consensus.request.write.cq.UpdateCQLastExecTimePlan; +import org.apache.iotdb.confignode.consensus.response.cq.ShowCQResp; import org.apache.iotdb.confignode.persistence.cq.CQInfo; import org.apache.iotdb.confignode.rpc.thrift.TCreateCQReq; +import org.apache.iotdb.rpc.TSStatusCode; import org.apache.commons.io.FileUtils; import org.apache.thrift.TException; @@ -70,7 +75,7 @@ public void testSnapshot() throws TException, IOException { "create cq testCq1 BEGIN select s1 into root.backup.d1.s1 from root.sg.d1 END", "Asia", "root"), - "testCq1_md5", + "testCq1Token", executionTime); cqInfo.addCQ(addCQPlan); @@ -89,7 +94,7 @@ public void testSnapshot() throws TException, IOException { "create cq testCq2 BEGIN select s1 into root.backup.d2.s1 from root.sg.d2 END", "Asia", "root"), - "testCq2_md5", + "testCq2Token", executionTime); cqInfo.addCQ(addCQPlan); @@ -99,4 +104,59 @@ public void testSnapshot() throws TException, IOException { Assert.assertEquals(cqInfo, actualCQInfo); } + + @Test + public void testOldCallbackCannotTouchRecreatedCQ() throws Exception { + long executionTime = System.currentTimeMillis(); + TCreateCQReq req = + new TCreateCQReq( + "testCq3", + 1000, + 0, + 1000, + 0, + (byte) 0, + "select s1 into root.backup.d3.s1 from root.sg.d3", + "create cq testCq3 BEGIN select s1 into root.backup.d3.s1 from root.sg.d3 END", + "Asia", + "root"); + + cqInfo.addCQ(new AddCQPlan(req, "oldToken", executionTime)); + cqInfo.dropCQ(new DropCQPlan("testCq3")); + cqInfo.addCQ(new AddCQPlan(req, "newToken", executionTime)); + + Assert.assertEquals( + TSStatusCode.NO_SUCH_CQ.getStatusCode(), + cqInfo.updateCQLastExecutionTime( + new UpdateCQLastExecTimePlan("testCq3", executionTime + 1000, "oldToken")) + .code); + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), + cqInfo.updateCQLastExecutionTime( + new UpdateCQLastExecTimePlan("testCq3", executionTime + 1000, "newToken")) + .code); + } + + @Test + public void testShowCQCanFilterByCQId() throws Exception { + long executionTime = System.currentTimeMillis(); + TCreateCQReq req = + new TCreateCQReq( + "testCq4", + 1000, + 0, + 1000, + 0, + (byte) 0, + "select s1 into root.backup.d4.s1 from root.sg.d4", + "create cq testCq4 BEGIN select s1 into root.backup.d4.s1 from root.sg.d4 END", + "Asia", + "root"); + cqInfo.addCQ(new AddCQPlan(req, "testCq4Token", executionTime)); + + ShowCQResp showCQResp = cqInfo.showCQ(new ShowCQPlan("testCq4")); + + Assert.assertEquals(1, showCQResp.getCqList().size()); + Assert.assertEquals("testCq4", showCQResp.getCqList().get(0).getCqId()); + } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java index d0e92b3281666..3e7fd2052ad53 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateCQProcedureTest.java @@ -36,10 +36,36 @@ import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.fail; public class CreateCQProcedureTest { + @Test + public void tokenShouldBeUniqueForSameCQId() { + ScheduledExecutorService executor = Executors.newScheduledThreadPool(1); + try { + TCreateCQReq req = + new TCreateCQReq( + "testCq1", + 1000, + 0, + 1000, + 0, + (byte) 0, + "select s1 into root.backup.d1(s1) from root.sg.d1", + "create cq testCq1 BEGIN select s1 into root.backup.d1(s1) from root.sg.d1 END", + "Asia", + "root"); + CreateCQProcedure createCQProcedure1 = new CreateCQProcedure(req, executor); + CreateCQProcedure createCQProcedure2 = new CreateCQProcedure(req, executor); + + assertNotEquals(createCQProcedure1.getCqToken(), createCQProcedure2.getCqToken()); + } finally { + executor.shutdown(); + } + } + @Test public void serializeDeserializeTest() { diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java new file mode 100644 index 0000000000000..a90e282494f0e --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedureRecoveryTest.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.impl.cq; + +import org.apache.iotdb.confignode.consensus.request.read.cq.ShowCQPlan; +import org.apache.iotdb.confignode.consensus.request.write.cq.AddCQPlan; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; +import org.apache.iotdb.confignode.manager.cq.CQManager; +import org.apache.iotdb.confignode.manager.cq.CQScheduleTask; +import org.apache.iotdb.confignode.persistence.cq.CQInfo; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; +import org.apache.iotdb.confignode.rpc.thrift.TCreateCQReq; + +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +public class CreateCQProcedureRecoveryTest { + + private TCreateCQReq newCreateCQReq() { + return new TCreateCQReq( + "testCq1", + 1000, + 0, + 1000, + 0, + (byte) 0, + "select s1 into root.backup.d1.s1 from root.sg.d1", + "create cq testCq1 BEGIN select s1 into root.backup.d1.s1 from root.sg.d1 END", + "Asia", + "root"); + } + + @SuppressWarnings("unchecked") + @Test + public void recoverScheduledTaskShouldResubmitFromLatestMetadata() throws Exception { + ScheduledExecutorService executor = Mockito.mock(ScheduledExecutorService.class); + Mockito.when( + executor.schedule( + Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class))) + .thenReturn(Mockito.mock(ScheduledFuture.class)); + + ConfigManager configManager = Mockito.mock(ConfigManager.class); + ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); + CQManager cqManager = Mockito.mock(CQManager.class); + ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); + Mockito.when(configManager.getCQManager()).thenReturn(cqManager); + Mockito.when( + cqManager.markCQLocallyScheduled( + Mockito.anyString(), Mockito.anyString(), Mockito.any(CQScheduleTask.class))) + .thenReturn(true); + + TCreateCQReq req = newCreateCQReq(); + CreateCQProcedure procedure = new CreateCQProcedure(req, executor); + + CQInfo cqInfo = new CQInfo(); + cqInfo.addCQ(new AddCQPlan(req, procedure.getCqToken(), System.currentTimeMillis() + 10_000)); + Mockito.when(consensusManager.read(Mockito.any(ShowCQPlan.class))).thenReturn(cqInfo.showCQ()); + + procedure.recoverScheduledTask(env); + + Mockito.verify(executor) + .schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class)); + } + + @SuppressWarnings("unchecked") + @Test + public void recoverScheduledTaskShouldSkipDuplicatedLocalSchedule() throws Exception { + ScheduledExecutorService executor = Mockito.mock(ScheduledExecutorService.class); + ConfigManager configManager = Mockito.mock(ConfigManager.class); + ConsensusManager consensusManager = Mockito.mock(ConsensusManager.class); + CQManager cqManager = Mockito.mock(CQManager.class); + ConfigNodeProcedureEnv env = Mockito.mock(ConfigNodeProcedureEnv.class); + Mockito.when(env.getConfigManager()).thenReturn(configManager); + Mockito.when(configManager.getConsensusManager()).thenReturn(consensusManager); + Mockito.when(configManager.getCQManager()).thenReturn(cqManager); + Mockito.when( + cqManager.markCQLocallyScheduled( + Mockito.anyString(), Mockito.anyString(), Mockito.any(CQScheduleTask.class))) + .thenReturn(false); + + TCreateCQReq req = newCreateCQReq(); + CreateCQProcedure procedure = new CreateCQProcedure(req, executor); + + CQInfo cqInfo = new CQInfo(); + cqInfo.addCQ(new AddCQPlan(req, procedure.getCqToken(), System.currentTimeMillis() + 10_000)); + Mockito.when(consensusManager.read(Mockito.any(ShowCQPlan.class))).thenReturn(cqInfo.showCQ()); + + procedure.recoverScheduledTask(env); + + Mockito.verify(executor, Mockito.never()) + .schedule(Mockito.any(Runnable.class), Mockito.anyLong(), Mockito.any(TimeUnit.class)); + } +} From 7d0b74b943787cd4c1831ce124f15046d7ce384b Mon Sep 17 00:00:00 2001 From: ZhangHongYin <46039728+SpriCoder@users.noreply.github.com> Date: Wed, 3 Jun 2026 15:00:47 +0800 Subject: [PATCH 078/102] [to dev/1.3] C++ client: backport thread-safe SessionPool (#17800) (#17817) * C++ client: add thread-safe SessionPool, enable RPC compression, and harden buffers (#17800) * Wire RPC compression flag through Session to its connections The enableRPCCompression option set via Session::open(bool) or the session builder was never propagated to SessionConnection, whose flag was hardcoded to false, so the compact Thrift protocol never took effect. Thread the flag from the builder/open() into both the data SessionConnection and the node-discovery NodesSupplier client so compression actually applies. * Use snprintf for Tablet bounds-check error messages Tablet::addValue and the OBJECT-value overload formatted out-of-range diagnostics with sprintf into a fixed 100-byte stack buffer, risking an overflow. Switch to snprintf bounded by sizeof(buffer) and cast the size_t arguments to long to match the %ld format. * Append big-endian bytes in MyStringBuffer instead of overwriting On big-endian hosts MyStringBuffer::putOrderedByte used str.assign, which replaced the whole buffer with each numeric write and corrupted previously serialized content. Use str.append so bytes accumulate, matching the little-endian path. * Add thread-safe SessionPool to the C++ client Introduce SessionPool and SessionPoolBuilder so multiple threads can share a bounded set of connections without external locking. A single Session is not safe to use concurrently, so the pool lends each Session to one borrower at a time via an RAII PooledSession handle and reclaims it on scope exit. Sessions are created outside the lock to avoid blocking other borrowers during the handshake, and getSession() blocks up to a configurable timeout when the pool is exhausted. Query results are returned as a PooledSessionDataSet that keeps the Session leased until the result set is fully read, since SessionDataSet lazily fetches further blocks over the same connection. Connections that raise IoTDBConnectionException are evicted rather than recycled. Add integration tests covering basic borrow/insert/query, concurrent writers, and exhaustion-timeout behavior. * Reject zero maxSize in SessionPool instead of clamping to 1 Address review feedback: maxSize is size_t, so a non-positive check reduces to == 0 (and "<= 0" would be a tautological-comparison warning under -Wall). Rather than silently clamping an invalid 0 to 1, fail fast by throwing IoTDBException so the misuse surfaces at construction time. * Tolerate missing timeseries in SessionPool test cleanup The pre-test cleanup deleted root.test.pool.* timeseries unconditionally, which threw 508 (does not exist) on a fresh database and failed the new [sessionPool] cases. Ignore that error since the cleanup is best-effort. * Revert "Wire RPC compression flag through Session to its connections" This reverts commit 2f35cc51f7. Honoring the compression flag makes the client negotiate the compact Thrift protocol, which the binary-only IoTDB server used by the C++ integration tests cannot speak, breaking the pre-existing ts_session_open_with_compression smoke test (it had only passed because the flag was a no-op). Compression needs a compact-protocol-enabled test server, so it will be reintroduced in a dedicated PR with the matching server-side test support. SessionPool keeps its compression option for forward compatibility; it is currently a no-op, as the rest of the client has always been. * fix format * Discard SessionPool session if pool closed during construction Address review feedback: acquire() releases the lock while building a new connection, so a concurrent close() could set closed_ after the slot was reserved, and the freshly opened session would still be handed out from a closed pool. Re-check closed_ under the lock after construction; if the pool was closed meanwhile, release the slot, tear the session down outside the lock, and throw instead of returning it. * C++ client: adapt SessionPool to dev/1.3 Session/builder APIs The cherry-picked SessionPool targeted master's expanded AbstractSessionBuilder (DEFAULT_* constants plus nodeUrls/connectTimeoutMs/useSSL/trustCertFilePath fields), none of which exist on dev/1.3, so the C++ client failed to compile. Adapt only the newly added SessionPool.h/.cpp (no existing dev/1.3 interface or implementation is changed): - Replace AbstractSessionBuilder::DEFAULT_* with dev/1.3's literal defaults. - Keep connectTimeoutMs (still applied via Session::open()). - Drop the multi-node (nodeUrls) constructor and SSL options, since dev/1.3's Session(AbstractSessionBuilder*) wires neither and would silently ignore them. --- iotdb-client/client-cpp/src/main/Common.cpp | 2 +- iotdb-client/client-cpp/src/main/Session.h | 11 +- .../client-cpp/src/main/SessionPool.cpp | 258 +++++++++++++ .../client-cpp/src/main/SessionPool.h | 356 ++++++++++++++++++ .../client-cpp/src/test/cpp/sessionIT.cpp | 136 +++++++ 5 files changed, 758 insertions(+), 5 deletions(-) create mode 100644 iotdb-client/client-cpp/src/main/SessionPool.cpp create mode 100644 iotdb-client/client-cpp/src/main/SessionPool.h diff --git a/iotdb-client/client-cpp/src/main/Common.cpp b/iotdb-client/client-cpp/src/main/Common.cpp index f58b6cc21ddc1..91cd806d08b55 100644 --- a/iotdb-client/client-cpp/src/main/Common.cpp +++ b/iotdb-client/client-cpp/src/main/Common.cpp @@ -384,7 +384,7 @@ const char* MyStringBuffer::getOrderedByte(size_t len) { void MyStringBuffer::putOrderedByte(char* buf, int len) { if (isBigEndian) { - str.assign(buf, len); + str.append(buf, len); } else { for (int i = len - 1; i > -1; i--) { diff --git a/iotdb-client/client-cpp/src/main/Session.h b/iotdb-client/client-cpp/src/main/Session.h index a1176e42bbe2b..ecdc20e6e1648 100644 --- a/iotdb-client/client-cpp/src/main/Session.h +++ b/iotdb-client/client-cpp/src/main/Session.h @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -272,15 +273,17 @@ class Tablet { void addValue(size_t schemaId, size_t rowIndex, const T& value) { if (schemaId >= schemas.size()) { char tmpStr[100]; - sprintf(tmpStr, "Tablet::addValue(), schemaId >= schemas.size(). schemaId=%ld, schemas.size()=%ld.", - schemaId, schemas.size()); + snprintf(tmpStr, sizeof(tmpStr), + "Tablet::addValue(), schemaId >= schemas.size(). schemaId=%ld, schemas.size()=%ld.", + (long)schemaId, (long)schemas.size()); throw std::out_of_range(tmpStr); } if (rowIndex >= rowSize) { char tmpStr[100]; - sprintf(tmpStr, "Tablet::addValue(), rowIndex >= rowSize. rowIndex=%ld, rowSize.size()=%ld.", rowIndex, - rowSize); + snprintf(tmpStr, sizeof(tmpStr), + "Tablet::addValue(), rowIndex >= rowSize. rowIndex=%ld, rowSize.size()=%ld.", + (long)rowIndex, (long)rowSize); throw std::out_of_range(tmpStr); } diff --git a/iotdb-client/client-cpp/src/main/SessionPool.cpp b/iotdb-client/client-cpp/src/main/SessionPool.cpp new file mode 100644 index 0000000000000..7f673a0171a8d --- /dev/null +++ b/iotdb-client/client-cpp/src/main/SessionPool.cpp @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "SessionPool.h" + +void PooledSession::reset() { + if (session_ && pool_ != nullptr) { + pool_->putBack(session_, broken_); + } + pool_ = nullptr; + session_ = nullptr; + broken_ = false; +} + +SessionPool::SessionPool(std::string host, int rpcPort, std::string username, std::string password, + size_t maxSize) + : host_(std::move(host)), rpcPort_(rpcPort), username_(std::move(username)), + password_(std::move(password)), maxSize_(maxSize) { + if (maxSize_ == 0) { + throw IoTDBException("SessionPool maxSize must be greater than 0."); + } +} + +SessionPool::~SessionPool() { + try { + close(); + } catch (const std::exception& e) { + log_debug(std::string("SessionPool::~SessionPool(), ") + e.what()); + } +} + +SessionPool& SessionPool::setFetchSize(int fetchSize) { + fetchSize_ = fetchSize; + return *this; +} + +SessionPool& SessionPool::setZoneId(std::string zoneId) { + zoneId_ = std::move(zoneId); + return *this; +} + +SessionPool& SessionPool::setSqlDialect(std::string sqlDialect) { + sqlDialect_ = std::move(sqlDialect); + return *this; +} + +SessionPool& SessionPool::setDatabase(std::string database) { + database_ = std::move(database); + return *this; +} + +SessionPool& SessionPool::setEnableRedirection(bool enable) { + enableRedirection_ = enable; + return *this; +} + +SessionPool& SessionPool::setEnableAutoFetch(bool enable) { + enableAutoFetch_ = enable; + return *this; +} + +SessionPool& SessionPool::setEnableRPCCompression(bool enable) { + enableRPCCompression_ = enable; + return *this; +} + +SessionPool& SessionPool::setConnectTimeoutMs(int connectTimeoutMs) { + connectTimeoutMs_ = connectTimeoutMs; + return *this; +} + +SessionPool& SessionPool::setWaitToGetSessionTimeoutMs(int64_t timeoutMs) { + waitTimeoutMs_ = timeoutMs; + return *this; +} + +std::shared_ptr SessionPool::constructNewSession() { + AbstractSessionBuilder builder; + builder.host = host_; + builder.rpcPort = rpcPort_; + builder.username = username_; + builder.password = password_; + builder.zoneId = zoneId_; + builder.fetchSize = fetchSize_; + builder.sqlDialect = sqlDialect_; + builder.database = database_; + builder.enableAutoFetch = enableAutoFetch_; + builder.enableRedirections = enableRedirection_; + builder.enableRPCCompression = enableRPCCompression_; + + auto session = std::make_shared(&builder); + session->open(enableRPCCompression_, connectTimeoutMs_); + return session; +} + +std::shared_ptr SessionPool::acquire(int64_t timeoutMs) { + const int64_t effectiveTimeout = timeoutMs > 0 ? timeoutMs : waitTimeoutMs_; + std::unique_lock lock(mutex_); + const auto deadline = + std::chrono::steady_clock::now() + std::chrono::milliseconds(effectiveTimeout); + + while (true) { + if (closed_) { + throw IoTDBException("SessionPool is closed."); + } + if (!idleQueue_.empty()) { + auto session = idleQueue_.front(); + idleQueue_.pop_front(); + return session; + } + if (size_ < maxSize_) { + // Reserve a slot, then build the connection outside the lock so other + // borrowers are not blocked by network/handshake latency. + ++size_; + lock.unlock(); + std::shared_ptr session; + try { + session = constructNewSession(); + } catch (...) { + lock.lock(); + --size_; + cv_.notify_one(); + throw; + } + lock.lock(); + if (closed_) { + // The pool was closed while this session was being built; do not hand it + // out. Release its slot and let it be torn down outside the lock. + --size_; + lock.unlock(); + throw IoTDBException("SessionPool is closed."); + } + return session; + } + + // Pool exhausted: wait for a Session to be returned. + if (effectiveTimeout <= 0) { + cv_.wait(lock); + } else { + if (cv_.wait_until(lock, deadline) == std::cv_status::timeout && idleQueue_.empty() && + size_ >= maxSize_ && !closed_) { + throw IoTDBException( + "Wait to get session timeout in SessionPool, maxSize=" + std::to_string(maxSize_) + + ", waitTimeoutMs=" + std::to_string(effectiveTimeout) + "."); + } + } + } +} + +void SessionPool::putBack(const std::shared_ptr& session, bool broken) { + std::lock_guard lock(mutex_); + if (broken || closed_) { + // Drop the Session and free its slot so a healthy replacement can be created + // on demand. The caller (PooledSession::reset) still holds the last reference + // and tears the connection down after we return, i.e. outside this lock. + --size_; + } else { + idleQueue_.push_back(session); + } + cv_.notify_one(); +} + +PooledSession SessionPool::getSession() { + return getSession(waitTimeoutMs_); +} + +PooledSession SessionPool::getSession(int64_t timeoutMs) { + return PooledSession(this, acquire(timeoutMs)); +} + +void SessionPool::insertTablet(Tablet& tablet, bool sorted) { + execute([&](Session& s) { s.insertTablet(tablet, sorted); }); +} + +void SessionPool::insertAlignedTablet(Tablet& tablet, bool sorted) { + execute([&](Session& s) { s.insertAlignedTablet(tablet, sorted); }); +} + +void SessionPool::insertTablets(std::unordered_map& tablets, bool sorted) { + execute([&](Session& s) { s.insertTablets(tablets, sorted); }); +} + +void SessionPool::insertRecord(const std::string& deviceId, int64_t time, + const std::vector& measurements, + const std::vector& values) { + execute([&](Session& s) { s.insertRecord(deviceId, time, measurements, values); }); +} + +void SessionPool::insertRecords(const std::vector& deviceIds, + const std::vector& times, + const std::vector>& measurementsList, + const std::vector>& valuesList) { + execute([&](Session& s) { s.insertRecords(deviceIds, times, measurementsList, valuesList); }); +} + +void SessionPool::executeNonQueryStatement(const std::string& sql) { + execute([&](Session& s) { s.executeNonQueryStatement(sql); }); +} + +PooledSessionDataSet SessionPool::executeQueryStatement(const std::string& sql) { + PooledSession lease = getSession(); + try { + auto dataSet = lease->executeQueryStatement(sql); + return PooledSessionDataSet(std::move(lease), std::move(dataSet)); + } catch (const IoTDBConnectionException&) { + lease.markBroken(); + throw; + } +} + +PooledSessionDataSet SessionPool::executeQueryStatement(const std::string& sql, + int64_t timeoutInMs) { + PooledSession lease = getSession(); + try { + auto dataSet = lease->executeQueryStatement(sql, timeoutInMs); + return PooledSessionDataSet(std::move(lease), std::move(dataSet)); + } catch (const IoTDBConnectionException&) { + lease.markBroken(); + throw; + } +} + +void SessionPool::close() { + std::deque> toClose; + { + std::lock_guard lock(mutex_); + if (closed_) { + return; + } + closed_ = true; + toClose.swap(idleQueue_); + size_ -= toClose.size(); + } + cv_.notify_all(); + // Sessions destructed here (outside the lock) close their connections. + toClose.clear(); +} + +size_t SessionPool::activeCount() { + std::lock_guard lock(mutex_); + return size_ - idleQueue_.size(); +} diff --git a/iotdb-client/client-cpp/src/main/SessionPool.h b/iotdb-client/client-cpp/src/main/SessionPool.h new file mode 100644 index 0000000000000..8838b298ec905 --- /dev/null +++ b/iotdb-client/client-cpp/src/main/SessionPool.h @@ -0,0 +1,356 @@ +/* + * 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. + */ + +#ifndef IOTDB_SESSIONPOOL_H +#define IOTDB_SESSIONPOOL_H + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "Session.h" + +/* + * A thread-safe pool of opened Session objects. + * + * A Session is NOT safe to use from multiple threads concurrently. SessionPool + * solves this by lending each Session to exactly one borrower at a time and + * reclaiming it afterwards, so many application threads can share a bounded set + * of physical connections without external locking. + * + * Two usage styles are supported: + * + * 1. RAII lease (recommended for arbitrary calls): + * { + * PooledSession s = pool.getSession(); // blocks up to the timeout + * s->insertTablet(tablet); // call any Session method + * } // automatically returned here + * + * 2. Convenience wrappers / generic execute() (recommended for hot paths): + * pool.insertTablet(tablet); + * pool.execute([&](Session& s) { s.insertRecord(...); }); + * + * Both styles evict a Session from the pool (instead of recycling it) when the + * operation throws IoTDBConnectionException, so a dead connection is never + * handed to the next borrower; a fresh one is created lazily on demand. + * + * Lifetime: a PooledSession returns its Session to the owning SessionPool when + * destroyed, so every PooledSession must not outlive the pool it came from. + */ +class SessionPool; + +class PooledSession { +public: + PooledSession() noexcept : pool_(nullptr), session_(nullptr), broken_(false) {} + + PooledSession(SessionPool* pool, std::shared_ptr session) + : pool_(pool), session_(std::move(session)), broken_(false) {} + + // Non-copyable: a leased Session is owned by exactly one borrower. + PooledSession(const PooledSession&) = delete; + PooledSession& operator=(const PooledSession&) = delete; + + PooledSession(PooledSession&& other) noexcept + : pool_(other.pool_), session_(std::move(other.session_)), broken_(other.broken_) { + other.pool_ = nullptr; + other.session_ = nullptr; + other.broken_ = false; + } + + PooledSession& operator=(PooledSession&& other) noexcept { + if (this != &other) { + reset(); + pool_ = other.pool_; + session_ = std::move(other.session_); + broken_ = other.broken_; + other.pool_ = nullptr; + other.session_ = nullptr; + other.broken_ = false; + } + return *this; + } + + ~PooledSession() { + reset(); + } + + Session* operator->() const { + return session_.get(); + } + + Session& operator*() const { + return *session_; + } + + explicit operator bool() const { + return static_cast(session_); + } + + // Mark the underlying connection as unusable so it is discarded (not recycled) + // when this lease is returned. Call this if you caught a connection error. + void markBroken() { + broken_ = true; + } + + // Eagerly return the Session to the pool before scope exit. + void release() { + reset(); + } + +private: + void reset(); + + SessionPool* pool_; + std::shared_ptr session_; + bool broken_; +}; + +/* + * Couples a query result set with the pooled Session that produced it. + * + * A SessionDataSet lazily fetches further result blocks over its Session's + * connection, so that Session must stay exclusively leased until iteration is + * finished. This wrapper holds the lease for exactly that long and returns the + * Session to the pool when destroyed. + */ +class PooledSessionDataSet { +public: + PooledSessionDataSet(PooledSession session, std::unique_ptr dataSet) + : session_(std::move(session)), dataSet_(std::move(dataSet)) {} + + PooledSessionDataSet(const PooledSessionDataSet&) = delete; + PooledSessionDataSet& operator=(const PooledSessionDataSet&) = delete; + PooledSessionDataSet(PooledSessionDataSet&&) noexcept = default; + PooledSessionDataSet& operator=(PooledSessionDataSet&&) noexcept = default; + + SessionDataSet* operator->() const { + return dataSet_.get(); + } + SessionDataSet& operator*() const { + return *dataSet_; + } + +private: + PooledSession session_; + std::unique_ptr dataSet_; +}; + +class SessionPool { +public: + static constexpr size_t DEFAULT_MAX_SIZE = 5; + static constexpr int64_t DEFAULT_WAIT_TIMEOUT_MS = 60 * 1000; + + // Single-host constructor. + SessionPool(std::string host, int rpcPort, std::string username, std::string password, + size_t maxSize = DEFAULT_MAX_SIZE); + + ~SessionPool(); + + // Non-copyable, non-movable: the pool owns mutex/condition state. + SessionPool(const SessionPool&) = delete; + SessionPool& operator=(const SessionPool&) = delete; + + // ---- configuration (apply before the first getSession()) ---- + SessionPool& setFetchSize(int fetchSize); + SessionPool& setZoneId(std::string zoneId); + SessionPool& setSqlDialect(std::string sqlDialect); + SessionPool& setDatabase(std::string database); + SessionPool& setEnableRedirection(bool enable); + SessionPool& setEnableAutoFetch(bool enable); + SessionPool& setEnableRPCCompression(bool enable); + SessionPool& setConnectTimeoutMs(int connectTimeoutMs); + SessionPool& setWaitToGetSessionTimeoutMs(int64_t timeoutMs); + + // Borrow a Session. Blocks until one is free or a new one can be created, + // up to timeoutMs (<= 0 means use the pool default). Throws IoTDBException on + // timeout or when the pool is closed. + PooledSession getSession(); + PooledSession getSession(int64_t timeoutMs); + + // Generic helper: borrow a Session, run func(Session&), return/evict it, and + // forward the result. Evicts the Session on IoTDBConnectionException. + template auto execute(Func&& func) -> decltype(func(std::declval())); + + // ---- convenience wrappers for common operations (with eviction on failure) ---- + void insertTablet(Tablet& tablet, bool sorted = false); + void insertAlignedTablet(Tablet& tablet, bool sorted = false); + void insertTablets(std::unordered_map& tablets, bool sorted = false); + void insertRecord(const std::string& deviceId, int64_t time, + const std::vector& measurements, + const std::vector& values); + void insertRecords(const std::vector& deviceIds, const std::vector& times, + const std::vector>& measurementsList, + const std::vector>& valuesList); + void executeNonQueryStatement(const std::string& sql); + // The returned wrapper keeps the underlying Session leased until it is + // destroyed, so it is safe to iterate the result set across multiple fetches. + PooledSessionDataSet executeQueryStatement(const std::string& sql); + PooledSessionDataSet executeQueryStatement(const std::string& sql, int64_t timeoutInMs); + + // Close the pool: idle Sessions are closed immediately, in-use Sessions are + // closed when they are returned. Idempotent. + void close(); + + // ---- observability ---- + size_t getMaxSize() const { + return maxSize_; + } + // Number of Sessions currently borrowed. + size_t activeCount(); + +private: + friend class PooledSession; + + std::shared_ptr constructNewSession(); + std::shared_ptr acquire(int64_t timeoutMs); + void putBack(const std::shared_ptr& session, bool broken); + + // connection parameters + std::string host_; + int rpcPort_; + std::string username_; + std::string password_; + std::string zoneId_; + int fetchSize_ = 10000; + std::string sqlDialect_ = "tree"; + std::string database_; + bool enableRedirection_ = true; + bool enableAutoFetch_ = true; + bool enableRPCCompression_ = false; + int connectTimeoutMs_ = 3000; + + // pool sizing / waiting policy + size_t maxSize_; + int64_t waitTimeoutMs_ = DEFAULT_WAIT_TIMEOUT_MS; + + // pool state, guarded by mutex_ + std::mutex mutex_; + std::condition_variable cv_; + std::deque> idleQueue_; + size_t size_ = 0; // total live Sessions (idle + borrowed) + bool closed_ = false; +}; + +template +auto SessionPool::execute(Func&& func) -> decltype(func(std::declval())) { + PooledSession lease = getSession(); + try { + return func(*lease); + } catch (const IoTDBConnectionException&) { + lease.markBroken(); + throw; + } +} + +/* + * Fluent builder for SessionPool, mirroring SessionBuilder / TableSessionBuilder. + * + * auto pool = SessionPoolBuilder() + * .host("127.0.0.1")->rpcPort(6667) + * ->username("root")->password("root") + * ->maxSize(10)->build(); + */ +class SessionPoolBuilder : public AbstractSessionBuilder { +public: + SessionPoolBuilder* host(const std::string& v) { + AbstractSessionBuilder::host = v; + return this; + } + SessionPoolBuilder* rpcPort(int v) { + AbstractSessionBuilder::rpcPort = v; + return this; + } + SessionPoolBuilder* username(const std::string& v) { + AbstractSessionBuilder::username = v; + return this; + } + SessionPoolBuilder* password(const std::string& v) { + AbstractSessionBuilder::password = v; + return this; + } + SessionPoolBuilder* zoneId(const std::string& v) { + AbstractSessionBuilder::zoneId = v; + return this; + } + SessionPoolBuilder* fetchSize(int v) { + AbstractSessionBuilder::fetchSize = v; + return this; + } + SessionPoolBuilder* database(const std::string& v) { + AbstractSessionBuilder::database = v; + return this; + } + SessionPoolBuilder* enableAutoFetch(bool v) { + AbstractSessionBuilder::enableAutoFetch = v; + return this; + } + SessionPoolBuilder* enableRedirections(bool v) { + AbstractSessionBuilder::enableRedirections = v; + return this; + } + SessionPoolBuilder* enableRPCCompression(bool v) { + AbstractSessionBuilder::enableRPCCompression = v; + return this; + } + SessionPoolBuilder* connectTimeoutMs(int v) { + connectTimeoutMs_ = v; + return this; + } + SessionPoolBuilder* maxSize(size_t v) { + maxSize_ = v; + return this; + } + SessionPoolBuilder* waitToGetSessionTimeoutMs(int64_t v) { + waitTimeoutMs_ = v; + return this; + } + SessionPoolBuilder* sqlDialect(const std::string& v) { + AbstractSessionBuilder::sqlDialect = v; + return this; + } + + std::shared_ptr build() { + auto pool = std::make_shared( + AbstractSessionBuilder::host, AbstractSessionBuilder::rpcPort, + AbstractSessionBuilder::username, AbstractSessionBuilder::password, maxSize_); + pool->setFetchSize(AbstractSessionBuilder::fetchSize) + .setZoneId(AbstractSessionBuilder::zoneId) + .setSqlDialect(AbstractSessionBuilder::sqlDialect) + .setDatabase(AbstractSessionBuilder::database) + .setEnableRedirection(AbstractSessionBuilder::enableRedirections) + .setEnableAutoFetch(AbstractSessionBuilder::enableAutoFetch) + .setEnableRPCCompression(AbstractSessionBuilder::enableRPCCompression) + .setConnectTimeoutMs(connectTimeoutMs_) + .setWaitToGetSessionTimeoutMs(waitTimeoutMs_); + return pool; + } + +private: + size_t maxSize_ = SessionPool::DEFAULT_MAX_SIZE; + int64_t waitTimeoutMs_ = SessionPool::DEFAULT_WAIT_TIMEOUT_MS; + int connectTimeoutMs_ = 3000; +}; + +#endif // IOTDB_SESSIONPOOL_H diff --git a/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp b/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp index b2db95cc4eca0..a527975589435 100644 --- a/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp +++ b/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp @@ -19,9 +19,13 @@ #include "catch.hpp" #include "Session.h" +#include "SessionPool.h" #include "TsBlock.h" #include +#include +#include + using namespace std; extern std::shared_ptr session; @@ -795,3 +799,135 @@ TEST_CASE("TsBlock deserialize rejects truncated malicious payload", "[TsBlockDe data[3] = '\x10'; REQUIRE_THROWS_AS(TsBlock::deserialize(data), IoTDBException); } + +TEST_CASE("SessionPool basic borrow/insert/query via RAII lease", "[sessionPool]") { + CaseReporter cr("SessionPool basic"); + auto pool = SessionPoolBuilder() + .host("127.0.0.1") + ->rpcPort(6667) + ->username("root") + ->password("root") + ->maxSize(3) + ->build(); + + { + PooledSession s = pool->getSession(); + try { + s->executeNonQueryStatement("delete timeseries root.test.pool.d1.*"); + } catch (const std::exception&) { + // Ignore: the timeseries may not exist yet on a fresh database. + } + } + + const int rows = 50; + for (int i = 0; i < rows; i++) { + PooledSession s = pool->getSession(); + s->insertRecord("root.test.pool.d1", i, {"s1"}, {to_string(i)}); + } + + int count = 0; + { + PooledSessionDataSet ds = pool->executeQueryStatement("select s1 from root.test.pool.d1"); + while (ds->hasNext()) { + ds->next(); + count++; + } + } + REQUIRE(count == rows); + + { + PooledSession s = pool->getSession(); + s->executeNonQueryStatement("delete timeseries root.test.pool.d1.*"); + } + pool->close(); +} + +TEST_CASE("SessionPool is safe under concurrent writers", "[sessionPool]") { + CaseReporter cr("SessionPool concurrency"); + auto pool = SessionPoolBuilder() + .host("127.0.0.1") + ->rpcPort(6667) + ->username("root") + ->password("root") + ->maxSize(4) + ->build(); + + { + PooledSession s = pool->getSession(); + try { + s->executeNonQueryStatement("delete timeseries root.test.pool.d2.*"); + } catch (const std::exception&) { + // Ignore: the timeseries may not exist yet on a fresh database. + } + } + + const int threadCount = 8; + const int rowsPerThread = 100; + std::atomic failures(0); + std::vector threads; + for (int t = 0; t < threadCount; t++) { + threads.emplace_back([&pool, t, rowsPerThread, &failures]() { + try { + for (int i = 0; i < rowsPerThread; i++) { + int64_t ts = static_cast(t) * rowsPerThread + i; + // Mix RAII and convenience APIs to exercise both borrow paths. + if (i % 2 == 0) { + pool->insertRecord("root.test.pool.d2", ts, {"s1"}, {to_string(ts)}); + } else { + PooledSession s = pool->getSession(); + s->insertRecord("root.test.pool.d2", ts, {"s1"}, {to_string(ts)}); + } + } + } catch (const std::exception& e) { + std::cerr << "writer thread failed: " << e.what() << std::endl; + failures++; + } + }); + } + for (auto& th : threads) { + th.join(); + } + REQUIRE(failures.load() == 0); + REQUIRE(pool->getMaxSize() == 4); + + int count = 0; + { + PooledSessionDataSet ds = pool->executeQueryStatement("select s1 from root.test.pool.d2"); + while (ds->hasNext()) { + ds->next(); + count++; + } + } + REQUIRE(count == threadCount * rowsPerThread); + + { + PooledSession s = pool->getSession(); + s->executeNonQueryStatement("delete timeseries root.test.pool.d2.*"); + } + pool->close(); +} + +TEST_CASE("SessionPool getSession times out when exhausted", "[sessionPool]") { + CaseReporter cr("SessionPool exhaustion timeout"); + auto pool = SessionPoolBuilder() + .host("127.0.0.1") + ->rpcPort(6667) + ->username("root") + ->password("root") + ->maxSize(1) + ->waitToGetSessionTimeoutMs(200) + ->build(); + + PooledSession held = pool->getSession(); + REQUIRE(static_cast(held)); + REQUIRE(pool->activeCount() == 1); + // The only Session is checked out, so a second borrow must time out. + REQUIRE_THROWS_AS(pool->getSession(), IoTDBException); + + held.release(); + // After returning it, a borrow succeeds again. + PooledSession reused = pool->getSession(); + REQUIRE(static_cast(reused)); + reused.release(); + pool->close(); +} From 44cb7c2f6fb67095d9a9c1e1d551faa13548f664 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 3 Jun 2026 15:37:55 +0800 Subject: [PATCH 079/102] Fix flaky space quota IT (#17812) (#17824) (cherry picked from commit 99f0af16c890529320fa5248c15f2da41953ebcb) --- .../iotdb/db/it/quotas/IoTDBSpaceQuotaIT.java | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/quotas/IoTDBSpaceQuotaIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/quotas/IoTDBSpaceQuotaIT.java index 7ed80f7ab4ad6..8930985717f5d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/quotas/IoTDBSpaceQuotaIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/quotas/IoTDBSpaceQuotaIT.java @@ -26,6 +26,7 @@ import org.apache.iotdb.itbase.category.ClusterIT; import org.apache.iotdb.itbase.category.LocalStandaloneIT; +import org.awaitility.Awaitility; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -40,6 +41,7 @@ import java.sql.Statement; import java.util.Arrays; import java.util.List; +import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -297,8 +299,6 @@ public void setSpaceQuotaTest5() { "create timeseries root.sg0.wf01.wt02.status0 with datatype=BOOLEAN,encoding=PLAIN;"); adminStmt.execute( "create timeseries root.sg0.wf01.wt02.status1 with datatype=BOOLEAN,encoding=PLAIN;"); - Thread.sleep(2000); - ResultSet resultSet1 = adminStmt.executeQuery("show space quota root.sg0;"); String ans1 = "root.sg0,diskSize,0.09765625G,0.0G" + ",\n" @@ -306,8 +306,8 @@ public void setSpaceQuotaTest5() { + ",\n" + "root.sg0,timeSeriesNum,5,3" + ",\n"; - validateResultSet(resultSet1, ans1); - } catch (InterruptedException | SQLException e) { + validateResultSetEventually(adminStmt, "show space quota root.sg0;", ans1); + } catch (SQLException e) { Assert.fail(e.getMessage()); } } @@ -509,6 +509,22 @@ public void showSpaceQuotaTest1() throws SQLException { } } + private void validateResultSetEventually(Statement statement, String sql, String ans) { + Awaitility.await() + .pollInSameThread() + .pollDelay(0, TimeUnit.MILLISECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .atMost(30, TimeUnit.SECONDS) + .untilAsserted( + () -> { + try { + validateResultSet(statement.executeQuery(sql), ans); + } catch (SQLException e) { + Assert.fail(e.getMessage()); + } + }); + } + private void validateResultSet(ResultSet set, String ans) throws SQLException { try { StringBuilder builder = new StringBuilder(); @@ -521,12 +537,12 @@ private void validateResultSet(ResultSet set, String ans) throws SQLException { builder.append("\n"); } String result = builder.toString(); - assertEquals(ans.length(), result.length()); + assertEquals(result, ans.length(), result.length()); List ansLines = Arrays.asList(ans.split("\n")); List resultLines = Arrays.asList(result.split("\n")); - assertEquals(ansLines.size(), resultLines.size()); + assertEquals(result, ansLines.size(), resultLines.size()); for (String resultLine : resultLines) { - assertTrue(ansLines.contains(resultLine)); + assertTrue(result, ansLines.contains(resultLine)); } } finally { set.close(); From 23d1367f6d167916a7529a316a7fcadc1dd960e7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 3 Jun 2026 17:28:26 +0800 Subject: [PATCH 080/102] Pipe IT: Stabilize tree model leader stop test (#17825) --- .../it/autocreate/IoTDBPipeClusterIT.java | 28 ++++++++++++++----- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java index 96ec056cd6cf5..16e21fdb2956d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java @@ -37,6 +37,7 @@ import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.MultiClusterIT2AutoCreateSchema; +import org.apache.iotdb.itbase.env.BaseEnv; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.thrift.TException; @@ -315,6 +316,8 @@ public void testPipeAfterDataRegionLeaderStop() throws Exception { TestUtils.executeNonQueries( senderEnv, Arrays.asList("insert into root.db.d1(time,s1) values (1,1)", "flush"), null); + flushTreeDataRegionReplicasAfterReplicationComplete( + senderEnv, Collections.singletonList("root.db")); final int leaderIndex = restartTreeDataRegionLeader(client, "root.db"); if (leaderIndex == -1) { // ensure the leader is stopped @@ -333,7 +336,10 @@ public void testPipeAfterDataRegionLeaderStop() throws Exception { "select count(*) from root.db.d1", "count(root.db.d1.s1),", Collections.singleton("2,")); - waitForTreeDataRegionReplicationComplete(Collections.singletonList("root.db")); + flushTreeDataRegionReplicasAfterReplicationComplete( + senderEnv, Collections.singletonList("root.db")); + flushTreeDataRegionReplicasAfterReplicationComplete( + receiverEnv, Collections.singletonList("root.db")); } try { @@ -427,14 +433,22 @@ private int restartTreeDataRegionLeader( return -1; } - private void waitForTreeDataRegionReplicationComplete(final List databases) { + private void flushTreeDataRegionReplicasAfterReplicationComplete( + final BaseEnv env, final List databases) { + waitForTreeDataRegionReplicationComplete(env, databases); + TestUtils.executeNonQueryWithRetry(env, "flush"); + waitForTreeDataRegionReplicationComplete(env, databases); + } + + private void waitForTreeDataRegionReplicationComplete( + final BaseEnv env, final List databases) { await() .pollInterval(500, TimeUnit.MILLISECONDS) .atMost(2, TimeUnit.MINUTES) .untilAsserted( () -> { try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + (SyncConfigNodeIServiceClient) env.getLeaderConfigNodeConnection()) { final List leaderRegionInfoList = showTreeDataRegionLeaders(databases, client); Assert.assertFalse( @@ -443,14 +457,14 @@ private void waitForTreeDataRegionReplicationComplete(final List databas for (final TRegionInfo regionInfo : leaderRegionInfoList) { final DataNodeWrapper leaderNode = - findDataNodeWrapperByPort(regionInfo.getClientRpcPort()); + findDataNodeWrapperByPort(env, regionInfo.getClientRpcPort()); final String metricsUrl = "http://" + leaderNode.getIp() + ":" + leaderNode.getMetricPort() + "/metrics"; - final String metricsContent = senderEnv.getUrlContent(metricsUrl, null); + final String metricsContent = env.getUrlContent(metricsUrl, null); Assert.assertNotNull( "Failed to fetch metrics from leader DataNode at " + metricsUrl, metricsContent); @@ -478,8 +492,8 @@ private List showTreeDataRegionLeaders( return result; } - private DataNodeWrapper findDataNodeWrapperByPort(final int port) { - for (final DataNodeWrapper dataNodeWrapper : senderEnv.getDataNodeWrapperList()) { + private DataNodeWrapper findDataNodeWrapperByPort(final BaseEnv env, final int port) { + for (final DataNodeWrapper dataNodeWrapper : env.getDataNodeWrapperList()) { if (dataNodeWrapper.getPort() == port) { return dataNodeWrapper; } From cc6ec3121182091c6719d927eb8217f986355d46 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 3 Jun 2026 17:30:00 +0800 Subject: [PATCH 081/102] Ratis CI: Fixed the ratis CI port clash & Optimized the logs to avoid printing JVM crash (#17675) (#17830) --- .../consensus/ratis/RecoverReadTest.java | 9 ++- .../iotdb/consensus/ratis/TestUtils.java | 70 ++++++++++++++++++- 2 files changed, 76 insertions(+), 3 deletions(-) diff --git a/iotdb-core/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RecoverReadTest.java b/iotdb-core/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RecoverReadTest.java index ac8115ced8555..82c81a29a6707 100644 --- a/iotdb-core/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RecoverReadTest.java +++ b/iotdb-core/consensus/src/test/java/org/apache/iotdb/consensus/ratis/RecoverReadTest.java @@ -82,6 +82,7 @@ public boolean takeSnapshot(File snapshotDir) { @Before public void setUp() throws Exception { logger.info("[RECOVER TEST] start setting up the test env"); + TestUtils.prepareJvmForRatisTest(); final TestUtils.MiniClusterFactory factory = new TestUtils.MiniClusterFactory(); miniCluster = factory @@ -117,8 +118,12 @@ public void setUp() throws Exception { @After public void tearUp() throws Exception { logger.info("[RECOVER TEST] start tearing down the test env"); - miniCluster.cleanUp(); - logger.info("[RECOVER TEST] end tearing down the test env"); + try { + miniCluster.cleanUp(); + } finally { + TestUtils.assertNoUnexpectedRatisExit(); + logger.info("[RECOVER TEST] end tearing down the test env"); + } } /* mimics the situation before this patch */ diff --git a/iotdb-core/consensus/src/test/java/org/apache/iotdb/consensus/ratis/TestUtils.java b/iotdb-core/consensus/src/test/java/org/apache/iotdb/consensus/ratis/TestUtils.java index f5997c261fdff..a8731c9b37d99 100644 --- a/iotdb-core/consensus/src/test/java/org/apache/iotdb/consensus/ratis/TestUtils.java +++ b/iotdb-core/consensus/src/test/java/org/apache/iotdb/consensus/ratis/TestUtils.java @@ -40,6 +40,7 @@ import org.apache.iotdb.consensus.ratis.utils.Utils; import org.apache.ratis.thirdparty.com.google.common.base.Preconditions; +import org.apache.ratis.util.ExitUtils; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; @@ -55,9 +56,11 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Objects; import java.util.Scanner; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -70,6 +73,9 @@ public class TestUtils { private static final Logger logger = LoggerFactory.getLogger(TestUtils.class); + private static final TimeDuration PORT_RELEASE_WAIT = TimeDuration.valueOf(10, TimeUnit.SECONDS); + private static final TimeDuration PORT_RELEASE_POLL = + TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); public static class TestDataSet implements DataSet { @@ -230,6 +236,7 @@ static class MiniCluster { private final RatisConfig config; private final List servers; private final ConsensusGroup group; + private final List peerPorts; private Supplier smProvider; private final AtomicBoolean isStopped = new AtomicBoolean(false); @@ -250,9 +257,10 @@ private MiniCluster( this.peerStorage = new ArrayList<>(); this.stateMachines = new ArrayList<>(); this.servers = new ArrayList<>(); + this.peerPorts = randomDistinctPorts(replicas); for (int i = 0; i < replicas; i++) { - peers.add(new Peer(gid, i, new TEndPoint("127.0.0.1", randomFreePort()))); + peers.add(new Peer(gid, i, new TEndPoint("127.0.0.1", peerPorts.get(i)))); final File storage = storageProvider.apply(i); FileUtils.deleteFileQuietly(storage); @@ -300,6 +308,7 @@ void stop() throws IOException { for (RatisConsensus server : servers) { server.stop(); } + waitUntilPortsReleased(peerPorts); isStopped.set(true); } @@ -495,6 +504,65 @@ MiniCluster create() { } } + static void prepareJvmForRatisTest() { + ExitUtils.disableSystemExit(); + ExitUtils.clear(); + } + + static void assertNoUnexpectedRatisExit() { + ExitUtils.assertNotTerminated(); + ExitUtils.clear(); + } + + private static List randomDistinctPorts(int count) { + final List ports = new ArrayList<>(count); + final Set uniquePorts = new HashSet<>(); + while (ports.size() < count) { + final int port = randomFreePort(); + if (uniquePorts.add(port)) { + ports.add(port); + } + } + return ports; + } + + private static void waitUntilPortsReleased(List ports) throws IOException { + final Timestamp start = Timestamp.currentTime(); + while (true) { + boolean allReleased = true; + for (int port : ports) { + if (!isLocalPortAvailable(port)) { + allReleased = false; + break; + } + } + + if (allReleased) { + return; + } + + if (start.elapsedTime().compareTo(PORT_RELEASE_WAIT) > 0) { + throw new IOException("Timed out waiting for Ratis test ports to be released: " + ports); + } + + try { + PORT_RELEASE_POLL.sleep(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while waiting for Ratis test ports to be released", e); + } + } + } + + private static boolean isLocalPortAvailable(int port) { + try (ServerSocket socket = new ServerSocket(port)) { + socket.setReuseAddress(true); + return true; + } catch (IOException e) { + return false; + } + } + private static int randomFreePort() { try (ServerSocket socket = new ServerSocket(0)) { return socket.getLocalPort(); From 30b4d49fa6b7dd1626533a2e06aaf657f300f8e7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 3 Jun 2026 18:27:44 +0800 Subject: [PATCH 082/102] [to dev/1.3] Pipe: account page decode memory in scan parser (#17807) (#17833) * Pipe: account page decode memory in scan parser (#17807) * Pipe: account page decode memory in scan parser * Fix pipe scan parser single page row count * Fix pipe scan parser page memory test * Fix tsfile writer API in pipe test --- .../AlignedSinglePageWholeChunkReader.java | 55 ++++- .../scan/MemoryControlledChunkReader.java | 71 ++++++ .../scan/SinglePageWholeChunkReader.java | 193 ++++++++++++++- .../TsFileInsertionScanDataContainer.java | 55 +++-- .../TsFileInsertionDataContainerTest.java | 219 ++++++++++++++++++ 5 files changed, 568 insertions(+), 25 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/MemoryControlledChunkReader.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/AlignedSinglePageWholeChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/AlignedSinglePageWholeChunkReader.java index a9de04cbe6463..9c3b3514c830e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/AlignedSinglePageWholeChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/AlignedSinglePageWholeChunkReader.java @@ -37,6 +37,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import java.util.Objects; /** * The {@link AlignedSinglePageWholeChunkReader} is used to read a whole single page aligned chunk @@ -64,7 +65,7 @@ public AlignedSinglePageWholeChunkReader(Chunk timeChunk, List valueChunk this.timeChunkHeader = timeChunk.getHeader(); this.timeChunkDataBuffer = timeChunk.getData(); this.pageEstimatedMemoryUsageInBytes = - calculatePageEstimatedMemoryUsageInBytes(timeChunk, valueChunkList); + calculateMaxPageEstimatedMemoryUsageInBytesWithBatchData(timeChunk, valueChunkList); valueChunkList.forEach( chunk -> { @@ -206,4 +207,56 @@ public static long calculatePageEstimatedMemoryUsageInBytes( return estimatedMemoryUsageInBytes; } + + public static long calculateMaxPageEstimatedMemoryUsageInBytesWithBatchData( + final Chunk timeChunk, final List valueChunkList) throws IOException { + final List pageEstimatedMemoryUsageInBytesList = + calculatePageEstimatedMemoryUsageInBytesWithBatchDataList(timeChunk, valueChunkList); + return pageEstimatedMemoryUsageInBytesList.isEmpty() + ? 0 + : pageEstimatedMemoryUsageInBytesList.get(0); + } + + public static List calculatePageEstimatedMemoryUsageInBytesWithBatchDataList( + final Chunk timeChunk, final List valueChunkList) throws IOException { + final ByteBuffer timeChunkDataBuffer = timeChunk.getData().duplicate(); + final List valueChunkDataBufferList = new ArrayList<>(valueChunkList.size()); + for (final Chunk valueChunk : valueChunkList) { + valueChunkDataBufferList.add( + Objects.isNull(valueChunk) ? null : valueChunk.getData().duplicate()); + } + + final List pageEstimatedMemoryUsageInBytesList = new ArrayList<>(); + while (timeChunkDataBuffer.remaining() > 0) { + long pageUncompressedSizeInBytes = 0; + final PageHeader timePageHeader = + SinglePageWholeChunkReader.deserializePageHeader( + timeChunkDataBuffer, timeChunk.getHeader()); + pageUncompressedSizeInBytes += timePageHeader.getUncompressedSize(); + SinglePageWholeChunkReader.skipCompressedPageData(timeChunkDataBuffer, timePageHeader); + + final List valueDataTypeList = new ArrayList<>(valueChunkList.size()); + for (int i = 0; i < valueChunkList.size(); ++i) { + final Chunk valueChunk = valueChunkList.get(i); + final ByteBuffer valueChunkDataBuffer = valueChunkDataBufferList.get(i); + if (Objects.isNull(valueChunk) || Objects.isNull(valueChunkDataBuffer)) { + valueDataTypeList.add(null); + continue; + } + + final PageHeader valuePageHeader = + SinglePageWholeChunkReader.deserializePageHeader( + valueChunkDataBuffer, valueChunk.getHeader()); + pageUncompressedSizeInBytes += valuePageHeader.getUncompressedSize(); + valueDataTypeList.add(valueChunk.getHeader().getDataType()); + SinglePageWholeChunkReader.skipCompressedPageData(valueChunkDataBuffer, valuePageHeader); + } + pageEstimatedMemoryUsageInBytesList.add( + SinglePageWholeChunkReader.estimatePageMemoryUsageInBytesWithBatchData( + pageUncompressedSizeInBytes, + SinglePageWholeChunkReader.getPageRowCount(timePageHeader, timeChunk), + valueDataTypeList)); + } + return SinglePageWholeChunkReader.toSuffixMaxList(pageEstimatedMemoryUsageInBytesList); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/MemoryControlledChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/MemoryControlledChunkReader.java new file mode 100644 index 0000000000000..6a74ffc54fcec --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/MemoryControlledChunkReader.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.event.common.tsfile.container.scan; + +import org.apache.tsfile.read.common.BatchData; +import org.apache.tsfile.read.reader.IChunkReader; +import org.apache.tsfile.read.reader.IPageReader; + +import java.io.IOException; +import java.util.List; + +class MemoryControlledChunkReader implements IChunkReader, EstimatedMemoryChunkReader { + + private final IChunkReader delegate; + private final List pageEstimatedMemoryUsageInBytesList; + private int pageIndex; + + MemoryControlledChunkReader( + final IChunkReader delegate, final List pageEstimatedMemoryUsageInBytesList) { + this.delegate = delegate; + this.pageEstimatedMemoryUsageInBytesList = pageEstimatedMemoryUsageInBytesList; + } + + @Override + public long getCurrentPageEstimatedMemoryUsageInBytes() { + return pageIndex < pageEstimatedMemoryUsageInBytesList.size() + ? pageEstimatedMemoryUsageInBytesList.get(pageIndex) + : 0; + } + + @Override + public boolean hasNextSatisfiedPage() throws IOException { + return delegate.hasNextSatisfiedPage(); + } + + @Override + public BatchData nextPageData() throws IOException { + try { + return delegate.nextPageData(); + } finally { + ++pageIndex; + } + } + + @Override + public void close() throws IOException { + delegate.close(); + } + + @Override + public List loadPageReaderList() throws IOException { + return delegate.loadPageReaderList(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/SinglePageWholeChunkReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/SinglePageWholeChunkReader.java index ade5001290348..f41a6861120bd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/SinglePageWholeChunkReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/SinglePageWholeChunkReader.java @@ -21,6 +21,8 @@ import org.apache.tsfile.compress.IUnCompressor; import org.apache.tsfile.encoding.decoder.Decoder; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.MetaMarker; import org.apache.tsfile.file.header.ChunkHeader; import org.apache.tsfile.file.header.PageHeader; import org.apache.tsfile.file.metadata.statistics.Statistics; @@ -28,10 +30,15 @@ import org.apache.tsfile.read.reader.chunk.AbstractChunkReader; import org.apache.tsfile.read.reader.page.LazyLoadPageData; import org.apache.tsfile.read.reader.page.PageReader; +import org.apache.tsfile.utils.RamUsageEstimator; import java.io.IOException; import java.io.Serializable; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; public class SinglePageWholeChunkReader extends AbstractChunkReader implements EstimatedMemoryChunkReader { @@ -44,7 +51,8 @@ public SinglePageWholeChunkReader(Chunk chunk) throws IOException { this.chunkHeader = chunk.getHeader(); this.chunkDataBuffer = chunk.getData(); - this.pageEstimatedMemoryUsageInBytes = calculatePageEstimatedMemoryUsageInBytes(chunk); + this.pageEstimatedMemoryUsageInBytes = + calculateMaxPageEstimatedMemoryUsageInBytesWithBatchData(chunk); initAllPageReaders(); } @@ -81,11 +89,190 @@ public long getCurrentPageEstimatedMemoryUsageInBytes() { public static long calculatePageEstimatedMemoryUsageInBytes(final Chunk chunk) throws IOException { final ByteBuffer chunkDataBuffer = chunk.getData().duplicate(); - final PageHeader pageHeader = - PageHeader.deserializeFrom(chunkDataBuffer, (Statistics) null); + final PageHeader pageHeader = deserializePageHeader(chunkDataBuffer, chunk.getHeader()); return pageHeader.getUncompressedSize(); } + public static long calculateMaxPageEstimatedMemoryUsageInBytes(final Chunk chunk) + throws IOException { + final ByteBuffer chunkDataBuffer = chunk.getData().duplicate(); + long maxPageEstimatedMemoryUsageInBytes = 0; + while (chunkDataBuffer.remaining() > 0) { + final PageHeader pageHeader = deserializePageHeader(chunkDataBuffer, chunk.getHeader()); + maxPageEstimatedMemoryUsageInBytes = + Math.max(maxPageEstimatedMemoryUsageInBytes, pageHeader.getUncompressedSize()); + skipCompressedPageData(chunkDataBuffer, pageHeader); + } + return maxPageEstimatedMemoryUsageInBytes; + } + + public static long calculateMaxPageEstimatedMemoryUsageInBytesWithBatchData(final Chunk chunk) + throws IOException { + final List pageEstimatedMemoryUsageInBytesList = + calculatePageEstimatedMemoryUsageInBytesWithBatchDataList(chunk); + return pageEstimatedMemoryUsageInBytesList.isEmpty() + ? 0 + : pageEstimatedMemoryUsageInBytesList.get(0); + } + + public static List calculatePageEstimatedMemoryUsageInBytesWithBatchDataList( + final Chunk chunk) throws IOException { + final ByteBuffer chunkDataBuffer = chunk.getData().duplicate(); + final List pageEstimatedMemoryUsageInBytesList = new ArrayList<>(); + while (chunkDataBuffer.remaining() > 0) { + final PageHeader pageHeader = deserializePageHeader(chunkDataBuffer, chunk.getHeader()); + pageEstimatedMemoryUsageInBytesList.add( + estimatePageMemoryUsageInBytesWithBatchData( + pageHeader, chunk, Collections.singletonList(chunk.getHeader().getDataType()))); + skipCompressedPageData(chunkDataBuffer, pageHeader); + } + return toSuffixMaxList(pageEstimatedMemoryUsageInBytesList); + } + + static PageHeader deserializePageHeader( + final ByteBuffer chunkDataBuffer, final ChunkHeader chunkHeader) throws IOException { + return isSinglePageChunk(chunkHeader) + ? PageHeader.deserializeFrom(chunkDataBuffer, (Statistics) null) + : PageHeader.deserializeFrom(chunkDataBuffer, chunkHeader.getDataType()); + } + + static boolean isSinglePageChunk(final ChunkHeader chunkHeader) { + return (chunkHeader.getChunkType() & 0x3F) == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER; + } + + static void skipCompressedPageData( + final ByteBuffer chunkDataBuffer, final PageHeader pageHeader) { + chunkDataBuffer.position(chunkDataBuffer.position() + pageHeader.getCompressedSize()); + } + + static List toSuffixMaxList(final List pageEstimatedMemoryUsageInBytesList) { + long suffixMaxPageEstimatedMemoryUsageInBytes = 0; + for (int i = pageEstimatedMemoryUsageInBytesList.size() - 1; i >= 0; --i) { + suffixMaxPageEstimatedMemoryUsageInBytes = + Math.max( + suffixMaxPageEstimatedMemoryUsageInBytes, pageEstimatedMemoryUsageInBytesList.get(i)); + pageEstimatedMemoryUsageInBytesList.set(i, suffixMaxPageEstimatedMemoryUsageInBytes); + } + return pageEstimatedMemoryUsageInBytesList; + } + + static long estimatePageMemoryUsageInBytesWithBatchData( + final PageHeader timePageHeader, + final Chunk timeChunk, + final List valueDataTypeList) { + return estimatePageMemoryUsageInBytesWithBatchData( + timePageHeader.getUncompressedSize(), + getPageRowCount(timePageHeader, timeChunk), + valueDataTypeList); + } + + static int getPageRowCount(final PageHeader pageHeader, final Chunk chunk) { + if (isSinglePageChunk(chunk.getHeader())) { + return Objects.isNull(chunk.getChunkStatistic()) + ? 0 + : saturateToInt(chunk.getChunkStatistic().getCount()); + } + return saturateToInt(pageHeader.getNumOfValues()); + } + + private static int saturateToInt(final long value) { + return (int) Math.min(Integer.MAX_VALUE, value); + } + + static long estimatePageMemoryUsageInBytesWithBatchData( + final long pageUncompressedSizeInBytes, + final int rowCount, + final List valueDataTypeList) { + return pageUncompressedSizeInBytes + + estimateBatchDataMemoryUsageInBytes(rowCount, valueDataTypeList); + } + + private static long estimateBatchDataMemoryUsageInBytes( + final int rowCount, final List valueDataTypeList) { + final int valueCount = valueDataTypeList.size(); + final long segmentCount = Math.max(1, (rowCount + 15L) / 16); + long estimatedMemoryUsageInBytes = RamUsageEstimator.sizeOfLongArray(16) * segmentCount; + + if (valueCount == 1) { + estimatedMemoryUsageInBytes += + estimateSingleValueArrayMemoryUsageInBytes(rowCount, valueDataTypeList.get(0)); + } else if (valueCount > 1) { + estimatedMemoryUsageInBytes += RamUsageEstimator.sizeOfObjectArray(16) * segmentCount; + estimatedMemoryUsageInBytes += + (long) rowCount + * (RamUsageEstimator.sizeOfObjectArray(valueCount) + + estimateVectorValueMemoryUsageInBytes(valueDataTypeList)); + } + + return estimatedMemoryUsageInBytes; + } + + private static long estimateSingleValueArrayMemoryUsageInBytes( + final int rowCount, final TSDataType dataType) { + final long segmentCount = Math.max(1, (rowCount + 15L) / 16); + if (Objects.isNull(dataType)) { + return 0; + } + + switch (dataType) { + case BOOLEAN: + return RamUsageEstimator.sizeOfBooleanArray(16) * segmentCount; + case INT32: + case DATE: + return RamUsageEstimator.sizeOfIntArray(16) * segmentCount; + case INT64: + case TIMESTAMP: + return RamUsageEstimator.sizeOfLongArray(16) * segmentCount; + case FLOAT: + return RamUsageEstimator.sizeOfFloatArray(16) * segmentCount; + case DOUBLE: + return RamUsageEstimator.sizeOfDoubleArray(16) * segmentCount; + case TEXT: + case BLOB: + case STRING: + return RamUsageEstimator.sizeOfObjectArray(16) * segmentCount; + default: + return 0; + } + } + + private static long estimateVectorValueMemoryUsageInBytes( + final List valueDataTypeList) { + long estimatedMemoryUsageInBytes = 0; + for (final TSDataType dataType : valueDataTypeList) { + if (Objects.isNull(dataType)) { + continue; + } + + estimatedMemoryUsageInBytes += + RamUsageEstimator.alignObjectSize( + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + + estimateTsPrimitiveTypeValueMemoryUsageInBytes(dataType)); + } + return estimatedMemoryUsageInBytes; + } + + private static long estimateTsPrimitiveTypeValueMemoryUsageInBytes(final TSDataType dataType) { + switch (dataType) { + case BOOLEAN: + return 1; + case INT32: + case DATE: + case FLOAT: + return Integer.BYTES; + case INT64: + case TIMESTAMP: + case DOUBLE: + return Long.BYTES; + case TEXT: + case BLOB: + case STRING: + return RamUsageEstimator.NUM_BYTES_OBJECT_REF; + default: + return 0; + } + } + ///////////////////////////////////////////////////////////////////////////////////////////////// // util methods ///////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java index e903c7340e4f5..ac1fdb94db77e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/scan/TsFileInsertionScanDataContainer.java @@ -307,7 +307,7 @@ private Tablet getNextTablet() { data.next(); while (!data.hasCurrent() && chunkReader.hasNextSatisfiedPage()) { - data = chunkReader.nextPageData(); + data = nextPageData(); } if (tablet != null && tablet.rowSize == tablet.getMaxRowNumber()) { @@ -343,16 +343,18 @@ private void prepareData() throws IOException { } do { - resizePageDataMemoryForCurrentPageIfNeeded(); - data = chunkReader.nextPageData(); - long size = PipeMemoryWeightUtil.calculateBatchDataRamBytesUsed(data); - if (allocatedMemoryBlockForBatchData.getMemoryUsageInBytes() < size) { - PipeDataNodeResourceManager.memory().forceResize(allocatedMemoryBlockForBatchData, size); - } + data = nextPageData(); } while (!data.hasCurrent() && chunkReader.hasNextSatisfiedPage()); } while (!data.hasCurrent()); } + private BatchData nextPageData() throws IOException { + resizePageDataMemoryForCurrentPageIfNeeded(); + final BatchData nextData = chunkReader.nextPageData(); + resizePageDataMemoryIfNeeded(PipeMemoryWeightUtil.calculateBatchDataRamBytesUsed(nextData)); + return nextData; + } + private void resizePageDataMemoryForCurrentPageIfNeeded() { if (!(chunkReader instanceof EstimatedMemoryChunkReader)) { return; @@ -523,10 +525,8 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { timeChunkList.add(timeChunk); isMultiPageList.add(isMultiPage); timeChunkPageMemorySizeList.add( - isMultiPage - ? 0 - : SinglePageWholeChunkReader.calculatePageEstimatedMemoryUsageInBytes( - timeChunk)); + SinglePageWholeChunkReader.calculateMaxPageEstimatedMemoryUsageInBytes( + timeChunk)); break; } @@ -572,10 +572,14 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { Chunk chunk = new Chunk( chunkHeader, tsFileSequenceReader.readChunk(-1, chunkHeader.getDataSize())); + final List pageEstimatedMemoryUsageInBytesList = + SinglePageWholeChunkReader + .calculatePageEstimatedMemoryUsageInBytesWithBatchDataList(chunk); chunkReader = currentIsMultiPage - ? new ChunkReader(chunk, filter) + ? new MemoryControlledChunkReader( + new ChunkReader(chunk, filter), pageEstimatedMemoryUsageInBytesList) : new SinglePageWholeChunkReader(chunk); currentIsAligned = false; final String measurementID = @@ -649,8 +653,7 @@ private void moveToNextChunkReader() throws IOException, IllegalStateException { chunk = new Chunk( chunkHeader, tsFileSequenceReader.readChunk(-1, chunkHeader.getDataSize())); - currentValueChunkPageMemorySize = - calculatePageMemorySizeIfSinglePageValueChunk(chunk); + currentValueChunkPageMemorySize = calculateMaxPageMemorySize(chunk); boolean needReturn = false; final long timeChunkSize = lastIndex >= 0 @@ -687,8 +690,7 @@ > getPageDataMemoryLimitInBytes()) { chunk = firstChunk4NextSequentialValueChunks; chunkHeader = chunk.getHeader(); firstChunk4NextSequentialValueChunks = null; - currentValueChunkPageMemorySize = - calculatePageMemorySizeIfSinglePageValueChunk(chunk); + currentValueChunkPageMemorySize = calculateMaxPageMemorySize(chunk); resizeChunkMemoryBlockIfFirstValueChunkExceedsLimit(valueChunkList, chunkHeader); resizePageDataMemoryBlockIfFirstValueChunkExceedsLimit( valueChunkList, currentValueChunkPageMemorySize); @@ -759,9 +761,22 @@ private boolean recordAlignedChunk(final List valueChunkList, final byte AlignedSinglePageWholeChunkReader.calculatePageEstimatedMemoryUsageInBytes( timeChunk, valueChunkList)); } + final List pageEstimatedMemoryUsageInBytesList = + currentIsMultiPage + ? AlignedSinglePageWholeChunkReader + .calculatePageEstimatedMemoryUsageInBytesWithBatchDataList( + timeChunk, valueChunkList) + : Collections.emptyList(); + final long maxPageEstimatedMemoryUsageInBytes = + pageEstimatedMemoryUsageInBytesList.isEmpty() + ? 0 + : pageEstimatedMemoryUsageInBytesList.get(0); + resizePageDataMemoryIfNeeded(maxPageEstimatedMemoryUsageInBytes); chunkReader = currentIsMultiPage - ? new AlignedChunkReader(timeChunk, valueChunkList, filter) + ? new MemoryControlledChunkReader( + new AlignedChunkReader(timeChunk, valueChunkList, filter), + pageEstimatedMemoryUsageInBytesList) : new AlignedSinglePageWholeChunkReader(timeChunk, valueChunkList); currentIsAligned = true; lastMarker = marker; @@ -802,10 +817,8 @@ private void resizePageDataMemoryBlockIfFirstValueChunkExceedsLimit( } } - private long calculatePageMemorySizeIfSinglePageValueChunk(final Chunk chunk) throws IOException { - return isSinglePageValueChunk(chunk.getHeader()) - ? SinglePageWholeChunkReader.calculatePageEstimatedMemoryUsageInBytes(chunk) - : 0; + private long calculateMaxPageMemorySize(final Chunk chunk) throws IOException { + return SinglePageWholeChunkReader.calculateMaxPageEstimatedMemoryUsageInBytes(chunk); } private boolean isSinglePageValueChunk(final ChunkHeader chunkHeader) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java index 7fe514b277e63..36f56e0e6068c 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java @@ -27,6 +27,7 @@ import org.apache.iotdb.db.pipe.event.common.tsfile.container.TsFileInsertionDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.container.query.TsFileInsertionQueryDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.container.scan.AlignedSinglePageWholeChunkReader; +import org.apache.iotdb.db.pipe.event.common.tsfile.container.scan.SinglePageWholeChunkReader; import org.apache.iotdb.db.pipe.event.common.tsfile.container.scan.TsFileInsertionScanDataContainer; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; @@ -130,6 +131,85 @@ public void testScanContainer() throws Exception { System.out.println(System.currentTimeMillis() - startTime); } + @Test + public void testScanParserSplitNonAlignedSinglePageChunkByEstimatedPageMemory() throws Exception { + final long originalPipeMaxReaderChunkSize = + CommonDescriptor.getInstance().getConfig().getPipeMaxReaderChunkSize(); + final int originalPageSizeInByte = + TSFileDescriptor.getInstance().getConfig().getPageSizeInByte(); + final int originalMaxNumberOfPointsInPage = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); + + try { + TSFileDescriptor.getInstance().getConfig().setPageSizeInByte(64 * 1024); + TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(10000); + + final int measurementCount = 16; + final int rowCount = 64; + final List schemaList = new ArrayList<>(); + for (int i = 0; i < measurementCount; ++i) { + schemaList.add( + new MeasurementSchema( + "s" + i, TSDataType.STRING, TSEncoding.PLAIN, CompressionType.LZ4)); + } + + nonalignedTsFile = new File("nonaligned-single-page-high-compression.tsfile"); + final Tablet tablet = new Tablet("root.sg.d", schemaList, rowCount); + final Binary value = + new Binary(new String(new char[512]).replace('\0', 'a'), TSFileConfig.STRING_CHARSET); + for (int row = 0; row < rowCount; ++row) { + tablet.addTimestamp(row, row); + for (int measurementIndex = 0; measurementIndex < measurementCount; ++measurementIndex) { + tablet.addValue("s" + measurementIndex, row, value); + } + } + tablet.rowSize = rowCount; + + try (final TsFileWriter writer = new TsFileWriter(nonalignedTsFile)) { + writer.registerTimeseries(new Path("root.sg.d"), schemaList); + writer.write(tablet); + } + + CommonDescriptor.getInstance() + .getConfig() + .setPipeMaxReaderChunkSize( + calculatePipeMaxReaderChunkSizeForSinglePageNonAlignedChunk(nonalignedTsFile)); + + int tabletCount = 0; + int maxMeasurementCount = 0; + int pointCount = 0; + try (final TsFileInsertionScanDataContainer parser = + new TsFileInsertionScanDataContainer( + nonalignedTsFile, + new PrefixPipePattern("root"), + Long.MIN_VALUE, + Long.MAX_VALUE, + null, + null, + false)) { + for (final Pair tabletWithIsAligned : parser.toTabletWithIsAligneds()) { + Assert.assertFalse(tabletWithIsAligned.getRight()); + final Tablet parsedTablet = tabletWithIsAligned.getLeft(); + tabletCount++; + maxMeasurementCount = Math.max(maxMeasurementCount, parsedTablet.getSchemas().size()); + pointCount += getNonNullSize(parsedTablet); + } + } + + Assert.assertTrue(tabletCount > 1); + Assert.assertTrue(maxMeasurementCount < measurementCount); + Assert.assertEquals(measurementCount * rowCount, pointCount); + } finally { + CommonDescriptor.getInstance() + .getConfig() + .setPipeMaxReaderChunkSize(originalPipeMaxReaderChunkSize); + TSFileDescriptor.getInstance().getConfig().setPageSizeInByte(originalPageSizeInByte); + TSFileDescriptor.getInstance() + .getConfig() + .setMaxNumberOfPointsInPage(originalMaxNumberOfPointsInPage); + } + } + @Test public void testScanParserSplitAlignedSinglePageChunkByEstimatedPageMemory() throws Exception { final long originalPipeMaxReaderChunkSize = @@ -209,6 +289,85 @@ public void testScanParserSplitAlignedSinglePageChunkByEstimatedPageMemory() thr } } + @Test + public void testScanParserSplitAlignedMultiPageChunkByEstimatedPageMemory() throws Exception { + final long originalPipeMaxReaderChunkSize = + CommonDescriptor.getInstance().getConfig().getPipeMaxReaderChunkSize(); + final int originalPageSizeInByte = + TSFileDescriptor.getInstance().getConfig().getPageSizeInByte(); + final int originalMaxNumberOfPointsInPage = + TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage(); + + try { + TSFileDescriptor.getInstance().getConfig().setPageSizeInByte(64 * 1024); + TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(32); + + final int measurementCount = 16; + final int rowCount = 64; + final List schemaList = new ArrayList<>(); + for (int i = 0; i < measurementCount; ++i) { + schemaList.add( + new MeasurementSchema( + "s" + i, TSDataType.STRING, TSEncoding.PLAIN, CompressionType.LZ4)); + } + + alignedTsFile = new File("aligned-multi-page-high-compression.tsfile"); + final Tablet tablet = new Tablet("root.sg.d", schemaList, rowCount); + final Binary value = + new Binary(new String(new char[512]).replace('\0', 'a'), TSFileConfig.STRING_CHARSET); + for (int row = 0; row < rowCount; ++row) { + tablet.addTimestamp(row, row); + for (int measurementIndex = 0; measurementIndex < measurementCount; ++measurementIndex) { + tablet.addValue("s" + measurementIndex, row, value); + } + } + tablet.rowSize = rowCount; + + try (final TsFileWriter writer = new TsFileWriter(alignedTsFile)) { + writer.registerAlignedTimeseries(new Path("root.sg.d"), schemaList); + writer.writeAligned(tablet); + } + + CommonDescriptor.getInstance() + .getConfig() + .setPipeMaxReaderChunkSize( + calculatePipeMaxReaderChunkSizeForMultiPageAlignedChunk(alignedTsFile)); + + int tabletCount = 0; + int maxMeasurementCount = 0; + int pointCount = 0; + try (final TsFileInsertionScanDataContainer parser = + new TsFileInsertionScanDataContainer( + alignedTsFile, + new PrefixPipePattern("root"), + Long.MIN_VALUE, + Long.MAX_VALUE, + null, + null, + false)) { + for (final Pair tabletWithIsAligned : parser.toTabletWithIsAligneds()) { + Assert.assertTrue(tabletWithIsAligned.getRight()); + final Tablet parsedTablet = tabletWithIsAligned.getLeft(); + tabletCount++; + maxMeasurementCount = Math.max(maxMeasurementCount, parsedTablet.getSchemas().size()); + pointCount += getNonNullSize(parsedTablet); + } + } + + Assert.assertTrue(tabletCount > 1); + Assert.assertTrue(maxMeasurementCount < measurementCount); + Assert.assertEquals(measurementCount * rowCount, pointCount); + } finally { + CommonDescriptor.getInstance() + .getConfig() + .setPipeMaxReaderChunkSize(originalPipeMaxReaderChunkSize); + TSFileDescriptor.getInstance().getConfig().setPageSizeInByte(originalPageSizeInByte); + TSFileDescriptor.getInstance() + .getConfig() + .setMaxNumberOfPointsInPage(originalMaxNumberOfPointsInPage); + } + } + @Test public void testScanParserResizesChunkMemoryForFirstAlignedValueChunk() throws Exception { final long originalPipeMaxReaderChunkSize = @@ -840,4 +999,64 @@ private long calculatePipeMaxReaderChunkSizeForSinglePageAlignedChunk(final File return chunkSizeLimit; } } + + private long calculatePipeMaxReaderChunkSizeForSinglePageNonAlignedChunk(final File tsFile) + throws Exception { + try (final TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + final IDeviceID deviceID = reader.getDeviceMeasurementsMap().keySet().iterator().next(); + final List measurements = reader.getDeviceMeasurementsMap().get(deviceID); + Assert.assertFalse(measurements.isEmpty()); + + long chunkSizeLimit = 0; + long estimatedPageMemorySize = 0; + for (final String measurement : measurements) { + final List chunkMetadataList = + reader.getChunkMetadataList(new Path(deviceID, measurement, false)); + Assert.assertEquals(1, chunkMetadataList.size()); + + final Chunk chunk = reader.readMemChunk(chunkMetadataList.get(0)); + Assert.assertEquals( + MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER, chunk.getHeader().getChunkType() & 0x3F); + chunkSizeLimit += chunk.getHeader().getDataSize(); + estimatedPageMemorySize += + SinglePageWholeChunkReader.calculateMaxPageEstimatedMemoryUsageInBytesWithBatchData( + chunk); + } + + Assert.assertTrue(estimatedPageMemorySize > chunkSizeLimit); + return chunkSizeLimit; + } + } + + private long calculatePipeMaxReaderChunkSizeForMultiPageAlignedChunk(final File tsFile) + throws Exception { + try (final TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { + final List deviceIDList = reader.getAllDevices(); + Assert.assertEquals(1, deviceIDList.size()); + final IDeviceID deviceID = deviceIDList.get(0); + final List alignedChunkMetadataList = + reader.getAlignedChunkMetadata(deviceID); + Assert.assertEquals(1, alignedChunkMetadataList.size()); + + final AlignedChunkMetadata alignedChunkMetadata = alignedChunkMetadataList.get(0); + final Chunk timeChunk = + reader.readMemChunk((ChunkMetadata) alignedChunkMetadata.getTimeChunkMetadata()); + Assert.assertEquals(MetaMarker.CHUNK_HEADER, timeChunk.getHeader().getChunkType() & 0x3F); + + long chunkSizeLimit = PipeMemoryWeightUtil.calculateChunkRamBytesUsed(timeChunk); + long estimatedMaxPageMemorySize = + SinglePageWholeChunkReader.calculateMaxPageEstimatedMemoryUsageInBytes(timeChunk); + for (final IChunkMetadata valueChunkMetadata : + alignedChunkMetadata.getValueChunkMetadataList()) { + final Chunk valueChunk = reader.readMemChunk((ChunkMetadata) valueChunkMetadata); + Assert.assertEquals(MetaMarker.CHUNK_HEADER, valueChunk.getHeader().getChunkType() & 0x3F); + chunkSizeLimit += valueChunk.getHeader().getDataSize(); + estimatedMaxPageMemorySize += + SinglePageWholeChunkReader.calculateMaxPageEstimatedMemoryUsageInBytes(valueChunk); + } + + Assert.assertTrue(estimatedMaxPageMemorySize > chunkSizeLimit); + return chunkSizeLimit; + } + } } From 4052320c6c031c634f924ebe2d8c32f8cf1d3629 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 4 Jun 2026 10:32:02 +0800 Subject: [PATCH 083/102] Fixed UDF jar metadata handling in `UDFInfo` when multiple UDFs share the same jar (#17732) (#17835) * UDF Fix * sp * fix * Filter invalid jar reference counts in snapshots --- .../iotdb/confignode/persistence/UDFInfo.java | 54 ++++--- .../confignode/persistence/UDFInfoTest.java | 71 +++++++-- .../ReferenceCountedJarMetaKeeper.java | 136 ++++++++++++++++++ .../meta/ConfigNodePipePluginMetaKeeper.java | 50 ++----- .../ReferenceCountedJarMetaKeeperTest.java | 116 +++++++++++++++ 5 files changed, 352 insertions(+), 75 deletions(-) create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/executable/ReferenceCountedJarMetaKeeper.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/executable/ReferenceCountedJarMetaKeeperTest.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/UDFInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/UDFInfo.java index 1c7682dd35b7c..4b80a2625f0ef 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/UDFInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/UDFInfo.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.executable.ExecutableManager; +import org.apache.iotdb.commons.executable.ReferenceCountedJarMetaKeeper; import org.apache.iotdb.commons.snapshot.SnapshotProcessor; import org.apache.iotdb.commons.udf.UDFInformation; import org.apache.iotdb.commons.udf.UDFTable; @@ -37,7 +38,6 @@ import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.udf.api.exception.UDFManagementException; -import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,7 +50,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.locks.ReentrantLock; @@ -63,7 +62,7 @@ public class UDFInfo implements SnapshotProcessor { ConfigNodeDescriptor.getInstance().getConf(); private final UDFTable udfTable; - private final Map existedJarToMD5; + private final ReferenceCountedJarMetaKeeper jarMetaKeeper; private final UDFExecutableManager udfExecutableManager; @@ -73,7 +72,7 @@ public class UDFInfo implements SnapshotProcessor { public UDFInfo() throws IOException { udfTable = new UDFTable(); - existedJarToMD5 = new HashMap<>(); + jarMetaKeeper = new ReferenceCountedJarMetaKeeper(); udfExecutableManager = UDFExecutableManager.setupAndGetInstance( CONFIG_NODE_CONF.getUdfTemporaryLibDir(), CONFIG_NODE_CONF.getUdfDir()); @@ -97,7 +96,8 @@ public void validate(String udfName, String jarName, String jarMD5) String.format("Failed to create UDF [%s], the same name UDF has been created", udfName)); } - if (existedJarToMD5.containsKey(jarName) && !existedJarToMD5.get(jarName).equals(jarMD5)) { + if (jarMetaKeeper.containsJar(jarName) + && !jarMetaKeeper.jarNameExistsAndMatchesMd5(jarName, jarMD5)) { throw new UDFManagementException( String.format( "Failed to create UDF [%s], the same name Jar [%s] but different MD5 [%s] has existed", @@ -115,7 +115,7 @@ public void validate(String udfName) throws UDFManagementException { } public boolean needToSaveJar(String jarName) { - return !existedJarToMD5.containsKey(jarName); + return jarMetaKeeper.needToSaveJar(jarName); } public TSStatus addUDFInTable(CreateFunctionPlan physicalPlan) { @@ -123,7 +123,7 @@ public TSStatus addUDFInTable(CreateFunctionPlan physicalPlan) { final UDFInformation udfInformation = physicalPlan.getUdfInformation(); udfTable.addUDFInformation(udfInformation.getFunctionName(), udfInformation); if (udfInformation.isUsingURI()) { - existedJarToMD5.put(udfInformation.getJarName(), udfInformation.getJarMD5()); + addJarReference(udfInformation.getJarName(), udfInformation.getJarMD5()); if (physicalPlan.getJarFile() != null) { udfExecutableManager.saveToInstallDir( ByteBuffer.wrap(physicalPlan.getJarFile().getValues()), udfInformation.getJarName()); @@ -168,7 +168,10 @@ public JarResp getUDFJar(GetUDFJarPlan physicalPlan) { public TSStatus dropFunction(DropFunctionPlan req) { String udfName = req.getFunctionName(); if (udfTable.containsUDF(udfName)) { - existedJarToMD5.remove(udfTable.getUDFInformation(udfName).getJarName()); + final UDFInformation udfInformation = udfTable.getUDFInformation(udfName); + if (udfInformation.isUsingURI()) { + removeJarReference(udfInformation.getJarName()); + } udfTable.removeUDFInformation(udfName); } return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); @@ -181,7 +184,7 @@ public Map getRawUDFTable() { @TestOnly public Map getRawExistedJarToMD5() { - return existedJarToMD5; + return jarMetaKeeper.getJarNameToMd5Map(); } @Override @@ -225,30 +228,39 @@ public void processLoadSnapshot(File snapshotDir) throws IOException { deserializeExistedJarToMD5(fileInputStream); udfTable.deserializeUDFTable(fileInputStream); + rebuildJarMetadataFromUDFTable(); } finally { releaseUDFTableLock(); } } public void serializeExistedJarToMD5(OutputStream outputStream) throws IOException { - ReadWriteIOUtils.write(existedJarToMD5.size(), outputStream); - for (Map.Entry entry : existedJarToMD5.entrySet()) { - ReadWriteIOUtils.write(entry.getKey(), outputStream); - ReadWriteIOUtils.write(entry.getValue(), outputStream); - } + jarMetaKeeper.serializeJarNameToMd5(outputStream); } public void deserializeExistedJarToMD5(InputStream inputStream) throws IOException { - int size = ReadWriteIOUtils.readInt(inputStream); - while (size > 0) { - existedJarToMD5.put( - ReadWriteIOUtils.readString(inputStream), ReadWriteIOUtils.readString(inputStream)); - size--; - } + jarMetaKeeper.deserializeJarNameToMd5(inputStream); } public void clear() { - existedJarToMD5.clear(); + jarMetaKeeper.clear(); udfTable.clear(); } + + private void addJarReference(String jarName, String jarMD5) { + jarMetaKeeper.addReference(jarName, jarMD5); + } + + private void removeJarReference(String jarName) { + jarMetaKeeper.removeReference(jarName); + } + + private void rebuildJarMetadataFromUDFTable() { + jarMetaKeeper.clear(); + for (UDFInformation udfInformation : udfTable.getAllNonBuiltInUDFInformation()) { + if (udfInformation.isUsingURI()) { + addJarReference(udfInformation.getJarName(), udfInformation.getJarMD5()); + } + } + } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/UDFInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/UDFInfoTest.java index 708ea779fad72..c15cd0642288e 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/UDFInfoTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/UDFInfoTest.java @@ -18,12 +18,12 @@ */ package org.apache.iotdb.confignode.persistence; -import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.udf.UDFInformation; import org.apache.iotdb.confignode.consensus.request.write.function.CreateFunctionPlan; +import org.apache.iotdb.confignode.consensus.request.write.function.DropFunctionPlan; +import org.apache.iotdb.udf.api.exception.UDFManagementException; import org.apache.commons.io.FileUtils; -import org.apache.thrift.TException; import org.apache.tsfile.utils.Binary; import org.junit.AfterClass; import org.junit.Assert; @@ -37,6 +37,10 @@ public class UDFInfoTest { + private static final String SHARED_JAR_NAME = "shared.jar"; + private static final String SHARED_JAR_MD5 = "12345"; + private static final String DIFFERENT_JAR_MD5 = "54321"; + private static UDFInfo udfInfo; private static UDFInfo udfInfoSaveBefore; private static final File snapshotDir = new File(BASE_OUTPUT_PATH, "snapshot"); @@ -59,18 +63,42 @@ public static void cleanup() throws IOException { } @Test - public void testSnapshot() throws TException, IOException, IllegalPathException { - UDFInformation udfInformation = - new UDFInformation("test1", "test1", false, true, "test1.jar", "12345"); - CreateFunctionPlan createFunctionPlan = - new CreateFunctionPlan(udfInformation, new Binary(new byte[] {1, 2, 3})); - udfInfo.addUDFInTable(createFunctionPlan); - udfInfoSaveBefore.addUDFInTable(createFunctionPlan); + public void testDropOneSharedJarReferenceKeepsJarMetadata() + throws IOException, UDFManagementException { + clearUdfInfos(); + + udfInfo.addUDFInTable(createFunctionPlan("test1", SHARED_JAR_NAME, SHARED_JAR_MD5, true)); + udfInfo.addUDFInTable(createFunctionPlan("test2", SHARED_JAR_NAME, SHARED_JAR_MD5, false)); + + udfInfo.dropFunction(new DropFunctionPlan("TEST1")); + + Assert.assertFalse(udfInfo.needToSaveJar(SHARED_JAR_NAME)); + Assert.assertEquals(1, udfInfo.getRawExistedJarToMD5().size()); + Assert.assertEquals(SHARED_JAR_MD5, udfInfo.getRawExistedJarToMD5().get(SHARED_JAR_NAME)); + + udfInfo.validate("TEST3", SHARED_JAR_NAME, SHARED_JAR_MD5); + try { + udfInfo.validate("TEST3", SHARED_JAR_NAME, DIFFERENT_JAR_MD5); + Assert.fail("Expected shared jar conflict after dropping only one referenced UDF."); + } catch (UDFManagementException e) { + Assert.assertTrue(e.getMessage().contains("different MD5")); + } + } + + @Test + public void testSnapshotRebuildsSharedJarReferences() throws IOException { + clearUdfInfos(); + FileUtils.cleanDirectory(snapshotDir); + + CreateFunctionPlan createFunctionPlan1 = + createFunctionPlan("test1", SHARED_JAR_NAME, SHARED_JAR_MD5, true); + CreateFunctionPlan createFunctionPlan2 = + createFunctionPlan("test2", SHARED_JAR_NAME, SHARED_JAR_MD5, false); - udfInformation = new UDFInformation("test2", "test2", false, true, "test2.jar", "123456"); - createFunctionPlan = new CreateFunctionPlan(udfInformation, new Binary(new byte[] {1, 2, 3})); - udfInfo.addUDFInTable(createFunctionPlan); - udfInfoSaveBefore.addUDFInTable(createFunctionPlan); + udfInfo.addUDFInTable(createFunctionPlan1); + udfInfo.addUDFInTable(createFunctionPlan2); + udfInfoSaveBefore.addUDFInTable(createFunctionPlan1); + udfInfoSaveBefore.addUDFInTable(createFunctionPlan2); udfInfo.processTakeSnapshot(snapshotDir); udfInfo.clear(); @@ -78,5 +106,22 @@ public void testSnapshot() throws TException, IOException, IllegalPathException Assert.assertEquals(udfInfoSaveBefore.getRawExistedJarToMD5(), udfInfo.getRawExistedJarToMD5()); Assert.assertEquals(udfInfoSaveBefore.getRawUDFTable(), udfInfo.getRawUDFTable()); + + udfInfo.dropFunction(new DropFunctionPlan("TEST1")); + Assert.assertFalse(udfInfo.needToSaveJar(SHARED_JAR_NAME)); + Assert.assertEquals(SHARED_JAR_MD5, udfInfo.getRawExistedJarToMD5().get(SHARED_JAR_NAME)); + } + + private static void clearUdfInfos() { + udfInfo.clear(); + udfInfoSaveBefore.clear(); + } + + private static CreateFunctionPlan createFunctionPlan( + String functionName, String jarName, String jarMD5, boolean includeJarFile) { + UDFInformation udfInformation = + new UDFInformation(functionName, functionName, false, true, jarName, jarMD5); + return new CreateFunctionPlan( + udfInformation, includeJarFile ? new Binary(new byte[] {1, 2, 3}) : null); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/executable/ReferenceCountedJarMetaKeeper.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/executable/ReferenceCountedJarMetaKeeper.java new file mode 100644 index 0000000000000..52102aa96f2b3 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/executable/ReferenceCountedJarMetaKeeper.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.executable; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +public class ReferenceCountedJarMetaKeeper { + + private final Map jarNameToMd5Map = new HashMap<>(); + private final Map jarNameToReferenceCountMap = new HashMap<>(); + + public synchronized boolean containsJar(final String jarName) { + return jarNameToMd5Map.containsKey(jarName); + } + + public synchronized boolean needToSaveJar(final String jarName) { + return !containsJar(jarName); + } + + public synchronized boolean jarNameExistsAndMatchesMd5(final String jarName, final String md5) { + return containsJar(jarName) && Objects.equals(jarNameToMd5Map.get(jarName), md5); + } + + public synchronized void addReference(final String jarName, final String md5) { + if (jarNameToReferenceCountMap.containsKey(jarName)) { + jarNameToReferenceCountMap.put(jarName, jarNameToReferenceCountMap.get(jarName) + 1); + return; + } + + jarNameToReferenceCountMap.put(jarName, 1); + jarNameToMd5Map.put(jarName, md5); + } + + public synchronized void removeReference(final String jarName) { + final Integer referenceCount = jarNameToReferenceCountMap.get(jarName); + if (referenceCount == null || referenceCount <= 1) { + jarNameToReferenceCountMap.remove(jarName); + jarNameToMd5Map.remove(jarName); + return; + } + + jarNameToReferenceCountMap.put(jarName, referenceCount - 1); + } + + public synchronized void clear() { + jarNameToMd5Map.clear(); + jarNameToReferenceCountMap.clear(); + } + + public synchronized Map getJarNameToMd5Map() { + return new HashMap<>(jarNameToMd5Map); + } + + public synchronized void serializeJarNameToMd5(final OutputStream outputStream) + throws IOException { + ReadWriteIOUtils.write(jarNameToMd5Map.size(), outputStream); + for (final Map.Entry entry : jarNameToMd5Map.entrySet()) { + ReadWriteIOUtils.write(entry.getKey(), outputStream); + ReadWriteIOUtils.write(entry.getValue(), outputStream); + } + } + + public synchronized void deserializeJarNameToMd5(final InputStream inputStream) + throws IOException { + clear(); + + int size = ReadWriteIOUtils.readInt(inputStream); + while (size > 0) { + addReference( + ReadWriteIOUtils.readString(inputStream), ReadWriteIOUtils.readString(inputStream)); + size--; + } + } + + public synchronized void serializeJarNameToMd5AndReferenceCount(final OutputStream outputStream) + throws IOException { + int size = 0; + for (final Map.Entry entry : jarNameToReferenceCountMap.entrySet()) { + if (entry.getValue() > 0 && jarNameToMd5Map.containsKey(entry.getKey())) { + size++; + } + } + + ReadWriteIOUtils.write(size, outputStream); + for (final Map.Entry entry : jarNameToReferenceCountMap.entrySet()) { + final String jarName = entry.getKey(); + final int referenceCount = entry.getValue(); + if (referenceCount <= 0 || !jarNameToMd5Map.containsKey(jarName)) { + continue; + } + ReadWriteIOUtils.write(jarName, outputStream); + ReadWriteIOUtils.write(jarNameToMd5Map.get(jarName), outputStream); + ReadWriteIOUtils.write(referenceCount, outputStream); + } + } + + public synchronized void deserializeJarNameToMd5AndReferenceCount(final InputStream inputStream) + throws IOException { + clear(); + + final int jarSize = ReadWriteIOUtils.readInt(inputStream); + for (int i = 0; i < jarSize; i++) { + final String jarName = ReadWriteIOUtils.readString(inputStream); + final String md5 = ReadWriteIOUtils.readString(inputStream); + final int referenceCount = ReadWriteIOUtils.readInt(inputStream); + if (referenceCount > 0) { + jarNameToMd5Map.put(jarName, md5); + jarNameToReferenceCountMap.put(jarName, referenceCount); + } + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/meta/ConfigNodePipePluginMetaKeeper.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/meta/ConfigNodePipePluginMetaKeeper.java index da4eb9cf714da..2db7cf4613585 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/meta/ConfigNodePipePluginMetaKeeper.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/meta/ConfigNodePipePluginMetaKeeper.java @@ -19,80 +19,48 @@ package org.apache.iotdb.commons.pipe.agent.plugin.meta; -import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.apache.iotdb.commons.executable.ReferenceCountedJarMetaKeeper; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.util.HashMap; -import java.util.Map; public class ConfigNodePipePluginMetaKeeper extends PipePluginMetaKeeper { - protected final Map jarNameToMd5Map; - protected final Map jarNameToReferenceCountMap; + protected final ReferenceCountedJarMetaKeeper jarMetaKeeper; public ConfigNodePipePluginMetaKeeper() { super(); - jarNameToMd5Map = new HashMap<>(); - jarNameToReferenceCountMap = new HashMap<>(); + jarMetaKeeper = new ReferenceCountedJarMetaKeeper(); } public synchronized boolean containsJar(String jarName) { - return jarNameToMd5Map.containsKey(jarName); + return jarMetaKeeper.containsJar(jarName); } public synchronized boolean jarNameExistsAndMatchesMd5(String jarName, String md5) { - return jarNameToMd5Map.containsKey(jarName) && jarNameToMd5Map.get(jarName).equals(md5); + return jarMetaKeeper.jarNameExistsAndMatchesMd5(jarName, md5); } public synchronized void addJarNameAndMd5(String jarName, String md5) { - if (jarNameToReferenceCountMap.containsKey(jarName)) { - jarNameToReferenceCountMap.put(jarName, jarNameToReferenceCountMap.get(jarName) + 1); - } else { - jarNameToReferenceCountMap.put(jarName, 1); - jarNameToMd5Map.put(jarName, md5); - } + jarMetaKeeper.addReference(jarName, md5); } public synchronized void removeJarNameAndMd5IfPossible(String jarName) { - if (jarNameToReferenceCountMap.containsKey(jarName)) { - int count = jarNameToReferenceCountMap.get(jarName); - if (count == 1) { - jarNameToReferenceCountMap.remove(jarName); - jarNameToMd5Map.remove(jarName); - } else { - jarNameToReferenceCountMap.put(jarName, count - 1); - } - } + jarMetaKeeper.removeReference(jarName); } @Override public void processTakeSnapshot(OutputStream outputStream) throws IOException { - ReadWriteIOUtils.write(jarNameToMd5Map.size(), outputStream); - for (Map.Entry entry : jarNameToMd5Map.entrySet()) { - ReadWriteIOUtils.write(entry.getKey(), outputStream); - ReadWriteIOUtils.write(entry.getValue(), outputStream); - ReadWriteIOUtils.write(jarNameToReferenceCountMap.get(entry.getKey()), outputStream); - } + jarMetaKeeper.serializeJarNameToMd5AndReferenceCount(outputStream); super.processTakeSnapshot(outputStream); } @Override public void processLoadSnapshot(InputStream inputStream) throws IOException { - jarNameToMd5Map.clear(); - jarNameToReferenceCountMap.clear(); - - final int jarSize = ReadWriteIOUtils.readInt(inputStream); - for (int i = 0; i < jarSize; i++) { - final String jarName = ReadWriteIOUtils.readString(inputStream); - final String md5 = ReadWriteIOUtils.readString(inputStream); - final int count = ReadWriteIOUtils.readInt(inputStream); - jarNameToMd5Map.put(jarName, md5); - jarNameToReferenceCountMap.put(jarName, count); - } + jarMetaKeeper.deserializeJarNameToMd5AndReferenceCount(inputStream); super.processLoadSnapshot(inputStream); } diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/executable/ReferenceCountedJarMetaKeeperTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/executable/ReferenceCountedJarMetaKeeperTest.java new file mode 100644 index 0000000000000..6a48701f5747a --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/executable/ReferenceCountedJarMetaKeeperTest.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.executable; + +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +public class ReferenceCountedJarMetaKeeperTest { + + private static final String JAR_NAME = "shared.jar"; + private static final String MD5 = "12345"; + + @Test + public void testReferenceCounting() { + final ReferenceCountedJarMetaKeeper keeper = new ReferenceCountedJarMetaKeeper(); + + keeper.addReference(JAR_NAME, MD5); + keeper.addReference(JAR_NAME, MD5); + + Assert.assertTrue(keeper.containsJar(JAR_NAME)); + Assert.assertFalse(keeper.needToSaveJar(JAR_NAME)); + Assert.assertTrue(keeper.jarNameExistsAndMatchesMd5(JAR_NAME, MD5)); + Assert.assertFalse(keeper.jarNameExistsAndMatchesMd5(JAR_NAME, "54321")); + + keeper.removeReference(JAR_NAME); + Assert.assertTrue(keeper.containsJar(JAR_NAME)); + + keeper.removeReference(JAR_NAME); + Assert.assertFalse(keeper.containsJar(JAR_NAME)); + } + + @Test + public void testJarNameToMd5Snapshot() throws IOException { + final ReferenceCountedJarMetaKeeper keeper = new ReferenceCountedJarMetaKeeper(); + keeper.addReference(JAR_NAME, MD5); + keeper.addReference(JAR_NAME, MD5); + + final ReferenceCountedJarMetaKeeper loaded = new ReferenceCountedJarMetaKeeper(); + loaded.deserializeJarNameToMd5( + new ByteArrayInputStream(serializeJarNameToMd5(keeper).toByteArray())); + + Assert.assertTrue(loaded.jarNameExistsAndMatchesMd5(JAR_NAME, MD5)); + loaded.removeReference(JAR_NAME); + Assert.assertFalse(loaded.containsJar(JAR_NAME)); + } + + @Test + public void testJarNameToMd5AndReferenceCountSnapshot() throws IOException { + final ReferenceCountedJarMetaKeeper keeper = new ReferenceCountedJarMetaKeeper(); + keeper.addReference(JAR_NAME, MD5); + keeper.addReference(JAR_NAME, MD5); + + final ReferenceCountedJarMetaKeeper loaded = new ReferenceCountedJarMetaKeeper(); + loaded.deserializeJarNameToMd5AndReferenceCount( + new ByteArrayInputStream(serializeJarNameToMd5AndReferenceCount(keeper).toByteArray())); + + Assert.assertTrue(loaded.jarNameExistsAndMatchesMd5(JAR_NAME, MD5)); + loaded.removeReference(JAR_NAME); + Assert.assertTrue(loaded.containsJar(JAR_NAME)); + + loaded.removeReference(JAR_NAME); + Assert.assertFalse(loaded.containsJar(JAR_NAME)); + } + + @Test + public void testDeserializeJarNameToMd5AndReferenceCountSkipsZeroReferenceCount() + throws IOException { + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + ReadWriteIOUtils.write(1, outputStream); + ReadWriteIOUtils.write(JAR_NAME, outputStream); + ReadWriteIOUtils.write(MD5, outputStream); + ReadWriteIOUtils.write(0, outputStream); + + final ReferenceCountedJarMetaKeeper loaded = new ReferenceCountedJarMetaKeeper(); + loaded.deserializeJarNameToMd5AndReferenceCount( + new ByteArrayInputStream(outputStream.toByteArray())); + + Assert.assertFalse(loaded.containsJar(JAR_NAME)); + } + + private ByteArrayOutputStream serializeJarNameToMd5(final ReferenceCountedJarMetaKeeper keeper) + throws IOException { + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + keeper.serializeJarNameToMd5(outputStream); + return outputStream; + } + + private ByteArrayOutputStream serializeJarNameToMd5AndReferenceCount( + final ReferenceCountedJarMetaKeeper keeper) throws IOException { + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + keeper.serializeJarNameToMd5AndReferenceCount(outputStream); + return outputStream; + } +} From 251125ae7f4e7618b82f6808e9fe634e939bce02 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 4 Jun 2026 10:32:53 +0800 Subject: [PATCH 084/102] Fix pipe tsfile receiver database handling (#17815) (#17836) --- .../tsfile/PipeTsFileInsertionEvent.java | 4 + .../thrift/IoTDBDataNodeReceiver.java | 88 ++++++++++---- .../PipeStatementExceptionVisitor.java | 8 ++ .../PipeTransferTsFileSealWithModReq.java | 68 ++++++++++- .../airgap/IoTDBDataRegionAirGapSink.java | 11 +- .../async/IoTDBDataRegionAsyncSink.java | 6 +- .../handler/PipeTransferTsFileHandler.java | 17 ++- .../thrift/sync/IoTDBDataRegionSyncSink.java | 20 +++- .../statement/crud/LoadTsFileStatement.java | 39 +++++++ .../load/active/ActiveLoadPathHelper.java | 30 +++++ .../load/config/LoadTsFileConfigurator.java | 7 ++ .../PipeStatementTsStatusVisitorTest.java | 14 +++ .../thrift/IoTDBDataNodeReceiverTest.java | 110 ++++++++++++++++++ 13 files changed, 385 insertions(+), 37 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiverTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 8ffbc9f2f9bde..c9cd2d44af04b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -265,6 +265,10 @@ public File getTsFile() { return tsFile; } + public String getDatabaseName() { + return Objects.isNull(resource) ? null : resource.getDatabaseName(); + } + public File getModFile() { return modFile; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index 58d4c29eddc57..1ec4d6d53b7ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -52,6 +52,7 @@ import org.apache.iotdb.db.pipe.receiver.visitor.PipeStatementTSStatusVisitor; import org.apache.iotdb.db.pipe.receiver.visitor.PipeStatementToBatchVisitor; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.pipe.resource.log.PipePeriodicalLogReducer; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; import org.apache.iotdb.db.pipe.sink.payload.evolvable.request.PipeTransferDataNodeHandshakeV1Req; import org.apache.iotdb.db.pipe.sink.payload.evolvable.request.PipeTransferDataNodeHandshakeV2Req; @@ -459,29 +460,31 @@ protected String getSenderPort() { protected TSStatus loadFileV1(final PipeTransferFileSealReqV1 req, final String fileAbsolutePath) throws IOException { return isUsingAsyncLoadTsFileStrategy.get() - ? loadTsFileAsync(Collections.singletonList(fileAbsolutePath)) - : loadTsFileSync(fileAbsolutePath); + ? loadTsFileAsync(null, Collections.singletonList(fileAbsolutePath)) + : loadTsFileSync(null, fileAbsolutePath); } @Override protected TSStatus loadFileV2( final PipeTransferFileSealReqV2 req, final List fileAbsolutePaths) throws IOException, IllegalPathException { - return req instanceof PipeTransferTsFileSealWithModReq - // TsFile's absolute path will be the second element - ? (isUsingAsyncLoadTsFileStrategy.get() - ? loadTsFileAsync(fileAbsolutePaths) - : loadTsFileSync(fileAbsolutePaths.get(1))) - : loadSchemaSnapShot(req.getParameters(), fileAbsolutePaths); + if (req instanceof PipeTransferTsFileSealWithModReq) { + final String dataBaseName = + ((PipeTransferTsFileSealWithModReq) req).getDatabaseNameByTsFileName(); + return isUsingAsyncLoadTsFileStrategy.get() + ? loadTsFileAsync(dataBaseName, fileAbsolutePaths) + : loadTsFileSync(dataBaseName, fileAbsolutePaths.get(req.getFileNames().size() - 1)); + } + return loadSchemaSnapShot(req.getParameters(), fileAbsolutePaths); } - private TSStatus loadTsFileAsync(final List absolutePaths) throws IOException { + private TSStatus loadTsFileAsync(final String dataBaseName, final List absolutePaths) + throws IOException { final Map loadAttributes = - ActiveLoadPathHelper.buildAttributes( - null, + buildLoadTsFileAttributesForAsync( + dataBaseName, shouldConvertDataTypeOnTypeMismatch, validateTsFile.get(), - null, shouldMarkAsPipeRequest.get()); if (!ActiveLoadUtil.loadFilesToActiveDir(loadAttributes, absolutePaths, true)) { throw new PipeException("Load active listening pipe dir is not set."); @@ -489,15 +492,38 @@ private TSStatus loadTsFileAsync(final List absolutePaths) throws IOExce return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } - private TSStatus loadTsFileSync(final String fileAbsolutePath) throws FileNotFoundException { + static Map buildLoadTsFileAttributesForAsync( + final String dataBaseName, + final boolean shouldConvertDataTypeOnTypeMismatch, + final boolean validateTsFile, + final boolean shouldMarkAsPipeRequest) { + return ActiveLoadPathHelper.buildAttributes( + dataBaseName, + LoadTsFileStatement.getDatabaseLevelByTreeDatabase(dataBaseName), + shouldConvertDataTypeOnTypeMismatch, + validateTsFile, + null, + shouldMarkAsPipeRequest); + } + + private TSStatus loadTsFileSync(final String dataBaseName, final String fileAbsolutePath) + throws FileNotFoundException { + return executeStatementAndClassifyExceptions( + buildLoadTsFileStatementForSync(dataBaseName, fileAbsolutePath, validateTsFile.get())); + } + + static LoadTsFileStatement buildLoadTsFileStatementForSync( + final String dataBaseName, final String fileAbsolutePath, final boolean validateTsFile) + throws FileNotFoundException { final LoadTsFileStatement statement = LoadTsFileStatement.createUnchecked(fileAbsolutePath); statement.setDeleteAfterLoad(true); statement.setConvertOnTypeMismatch(true); - statement.setVerifySchema(validateTsFile.get()); + statement.setVerifySchema(validateTsFile); statement.setAutoCreateDatabase( IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled()); - - return executeStatementAndClassifyExceptions(statement); + statement.setDatabase(dataBaseName); + statement.updateDatabaseLevelByTreeDatabase(); + return statement; } private TSStatus loadSchemaSnapShot( @@ -704,12 +730,7 @@ private TSStatus executeStatementAndClassifyExceptions( return STATEMENT_STATUS_VISITOR.process(statement, result); } } catch (final Exception e) { - PipeLogger.log( - LOGGER::warn, - e, - "Receiver id = %s: Exception encountered while executing statement %s: ", - receiverId.get(), - statement.getPipeLoggingString()); + logStatementExceptionIfNecessary(statement, e); return STATEMENT_EXCEPTION_VISITOR.process(statement, e); } finally { if (Objects.nonNull(allocatedMemoryBlock)) { @@ -719,6 +740,29 @@ private TSStatus executeStatementAndClassifyExceptions( } } + private void logStatementExceptionIfNecessary(final Statement statement, final Exception e) { + if (shouldLogStatementException(receiverId.get(), statement, e)) { + PipeLogger.log( + LOGGER::warn, + e, + "Receiver id = %s: Exception encountered while executing statement %s: ", + receiverId.get(), + Objects.isNull(statement) ? null : statement.getPipeLoggingString()); + } + } + + static boolean shouldLogStatementException( + final long receiverId, final Statement statement, final Exception e) { + // Use the reducer cache as a gate. The actual stack trace is logged only when it passes. + return PipePeriodicalLogReducer.log( + message -> {}, + "Receiver id = %s, statement = %s, exception = %s, message = %s", + receiverId, + Objects.isNull(statement) ? null : statement.getPipeLoggingString(), + e.getClass().getName(), + e.getMessage()); + } + private TSStatus executeStatementWithRetryOnDataTypeMismatch(final Statement statement) { if (statement == null) { return RpcUtils.getStatus( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementExceptionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementExceptionVisitor.java index 098c983977a65..7b8246e3dab7a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementExceptionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/visitor/PipeStatementExceptionVisitor.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.exception.IoTDBException; +import org.apache.iotdb.commons.exception.IoTDBRuntimeException; import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.db.exception.load.LoadRuntimeOutOfMemoryException; import org.apache.iotdb.db.exception.sql.SemanticException; @@ -48,6 +49,13 @@ public class PipeStatementExceptionVisitor extends StatementVisitor { @Override public TSStatus visitNode(final StatementNode node, final Exception context) { + if (context instanceof IoTDBRuntimeException + && ((IoTDBRuntimeException) context).getErrorCode() + == TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()) { + return new TSStatus( + TSStatusCode.PIPE_RECEIVER_PARALLEL_OR_USER_CONFLICT_EXCEPTION.getStatusCode()) + .setMessage(context.getMessage()); + } return new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()) .setMessage(context.getMessage()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTsFileSealWithModReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTsFileSealWithModReq.java index 28959a1a0903c..7d0aa99cb13ea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTsFileSealWithModReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/request/PipeTransferTsFileSealWithModReq.java @@ -25,7 +25,9 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; +import java.util.Map; public class PipeTransferTsFileSealWithModReq extends PipeTransferFileSealReqV2 { @@ -38,17 +40,59 @@ protected PipeRequestType getPlanType() { return PipeRequestType.TRANSFER_TS_FILE_SEAL_WITH_MOD; } + private static final String DATABASE_NAME_KEY_PREFIX = "DATABASE_NAME_"; + + public String getDatabaseNameByTsFileName() { + return getParameters() == null + ? null + : getParameters() + .get( + generateDatabaseNameWithFileNameKey(getFileNames().get(getFileNames().size() - 1))); + } + + private static String generateDatabaseNameWithFileNameKey(final String fileName) { + return DATABASE_NAME_KEY_PREFIX + fileName; + } + + private static Map generateDatabaseNameParameter( + final String tsFileName, final String dataBaseName) { + return dataBaseName == null + ? new HashMap<>() + : Collections.singletonMap(generateDatabaseNameWithFileNameKey(tsFileName), dataBaseName); + } + /////////////////////////////// Thrift /////////////////////////////// public static PipeTransferTsFileSealWithModReq toTPipeTransferReq( String modFileName, long modFileLength, String tsFileName, long tsFileLength) throws IOException { + return toTPipeTransferReq(modFileName, modFileLength, tsFileName, tsFileLength, null); + } + + public static PipeTransferTsFileSealWithModReq toTPipeTransferReq( + final String modFileName, + final long modFileLength, + final String tsFileName, + final long tsFileLength, + final String dataBaseName) + throws IOException { return (PipeTransferTsFileSealWithModReq) new PipeTransferTsFileSealWithModReq() .convertToTPipeTransferReq( Arrays.asList(modFileName, tsFileName), Arrays.asList(modFileLength, tsFileLength), - new HashMap<>()); + generateDatabaseNameParameter(tsFileName, dataBaseName)); + } + + public static PipeTransferTsFileSealWithModReq toTPipeTransferReq( + final String tsFileName, final long tsFileLength, final String dataBaseName) + throws IOException { + return (PipeTransferTsFileSealWithModReq) + new PipeTransferTsFileSealWithModReq() + .convertToTPipeTransferReq( + Collections.singletonList(tsFileName), + Collections.singletonList(tsFileLength), + generateDatabaseNameParameter(tsFileName, dataBaseName)); } public static PipeTransferTsFileSealWithModReq fromTPipeTransferReq(TPipeTransferReq req) { @@ -61,11 +105,31 @@ public static PipeTransferTsFileSealWithModReq fromTPipeTransferReq(TPipeTransfe public static byte[] toTPipeTransferBytes( String modFileName, long modFileLength, String tsFileName, long tsFileLength) throws IOException { + return toTPipeTransferBytes(modFileName, modFileLength, tsFileName, tsFileLength, null); + } + + public static byte[] toTPipeTransferBytes( + final String modFileName, + final long modFileLength, + final String tsFileName, + final long tsFileLength, + final String dataBaseName) + throws IOException { return new PipeTransferTsFileSealWithModReq() .convertToTPipeTransferSnapshotSealBytes( Arrays.asList(modFileName, tsFileName), Arrays.asList(modFileLength, tsFileLength), - new HashMap<>()); + generateDatabaseNameParameter(tsFileName, dataBaseName)); + } + + public static byte[] toTPipeTransferBytes( + final String tsFileName, final long tsFileLength, final String dataBaseName) + throws IOException { + return new PipeTransferTsFileSealWithModReq() + .convertToTPipeTransferSnapshotSealBytes( + Collections.singletonList(tsFileName), + Collections.singletonList(tsFileLength), + generateDatabaseNameParameter(tsFileName, dataBaseName)); } /////////////////////////////// Object /////////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java index 81e745dc6a7ff..b2af96a37c5f3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/airgap/IoTDBDataRegionAirGapSink.java @@ -255,7 +255,7 @@ private void doTransfer( final Map, Double> pipe2WeightMap = batchToTransfer.deepCopyPipe2WeightMap(); for (final File tsFile : sealedFiles) { - doTransfer(pipe2WeightMap, socket, tsFile, null, tsFile.getName()); + doTransfer(pipe2WeightMap, socket, tsFile, null, null, tsFile.getName()); try { RetryUtils.retryOnException( () -> { @@ -379,6 +379,7 @@ private void doTransfer( pipeTsFileInsertionEvent.isWithMod() && supportModsIfIsDataNodeReceiver ? pipeTsFileInsertionEvent.getModFile() : null, + pipeTsFileInsertionEvent.getDatabaseName(), pipeTsFileInsertionEvent.toString()); } @@ -387,6 +388,7 @@ private void doTransfer( final AirGapSocket socket, final File tsFile, final File modFile, + final String dataBaseName, final String receiverStatusContext) throws PipeException, IOException { final String errorMessage = String.format("Seal file %s error. Socket %s.", tsFile, socket); @@ -397,7 +399,7 @@ private void doTransfer( if (!sendWeighted( socket, PipeTransferTsFileSealWithModReq.toTPipeTransferBytes( - modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length()), + modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length(), dataBaseName), pipe2WeightMap)) { receiverStatusHandler.handle( new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) @@ -411,7 +413,10 @@ private void doTransfer( transferFilePieces(pipe2WeightMap, tsFile, socket, false); if (!sendWeighted( socket, - PipeTransferTsFileSealReq.toTPipeTransferBytes(tsFile.getName(), tsFile.length()), + dataBaseName == null + ? PipeTransferTsFileSealReq.toTPipeTransferBytes(tsFile.getName(), tsFile.length()) + : PipeTransferTsFileSealWithModReq.toTPipeTransferBytes( + tsFile.getName(), tsFile.length(), dataBaseName), pipe2WeightMap)) { receiverStatusHandler.handle( new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java index 9d0c1563c7883..b61cb4543c74e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java @@ -255,7 +255,8 @@ private void transferInBatchWithoutCheck( eventsHadBeenAddedToRetryQueue, sealedFile, null, - false)); + false, + null)); } } catch (final Exception e) { PipeLogger.log(LOGGER::warn, e, "Failed to transfer tsfile batch (%s).", sealedFiles); @@ -400,7 +401,8 @@ private boolean transferWithoutCheck(final TsFileInsertionEvent tsFileInsertionE pipeTsFileInsertionEvent.getTsFile(), pipeTsFileInsertionEvent.getModFile(), pipeTsFileInsertionEvent.isWithMod() - && clientManager.supportModsIfIsDataNodeReceiver()); + && clientManager.supportModsIfIsDataNodeReceiver(), + pipeTsFileInsertionEvent.getDatabaseName()); transfer(pipeTransferTsFileHandler); return true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java index 8d9648f52921d..d7515141daeb8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/handler/PipeTransferTsFileHandler.java @@ -78,6 +78,7 @@ public class PipeTransferTsFileHandler extends PipeTransferTrackableHandler { private File currentFile; private final boolean transferMod; + private final String dataBaseName; private final int readFileBufferSize; private PipeTsFileMemoryBlock memoryBlock; @@ -98,7 +99,8 @@ public PipeTransferTsFileHandler( final AtomicBoolean eventsHadBeenAddedToRetryQueue, final File tsFile, final File modFile, - final boolean transferMod) + final boolean transferMod, + final String dataBaseName) throws InterruptedException { super(connector); @@ -111,6 +113,7 @@ public PipeTransferTsFileHandler( this.tsFile = tsFile; this.modFile = modFile; this.transferMod = transferMod; + this.dataBaseName = dataBaseName; currentFile = transferMod ? modFile : tsFile; // NOTE: Waiting for resource enough for slicing here may cause deadlock! @@ -191,8 +194,16 @@ public void transfer( final TPipeTransferReq uncompressedReq = transferMod ? PipeTransferTsFileSealWithModReq.toTPipeTransferReq( - modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length()) - : PipeTransferTsFileSealReq.toTPipeTransferReq(tsFile.getName(), tsFile.length()); + modFile.getName(), + modFile.length(), + tsFile.getName(), + tsFile.length(), + dataBaseName) + : dataBaseName == null + ? PipeTransferTsFileSealReq.toTPipeTransferReq( + tsFile.getName(), tsFile.length()) + : PipeTransferTsFileSealWithModReq.toTPipeTransferReq( + tsFile.getName(), tsFile.length(), dataBaseName); final TPipeTransferReq req = sink.compressIfNeeded(uncompressedReq); pipeName2WeightMap.forEach( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java index ef3d59f0d2a54..1bb0c383ff87b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/sync/IoTDBDataRegionSyncSink.java @@ -275,7 +275,7 @@ private void doTransfer(final PipeTabletEventTsFileBatch batchToTransfer) final Map, Double> pipe2WeightMap = batchToTransfer.deepCopyPipe2WeightMap(); for (final File tsFile : sealedFiles) { - doTransfer(pipe2WeightMap, tsFile, null); + doTransfer(pipe2WeightMap, tsFile, null, null); try { RetryUtils.retryOnException( () -> { @@ -428,7 +428,8 @@ private void doTransferWrapper(final PipeTsFileInsertionEvent pipeTsFileInsertio pipeTsFileInsertionEvent.getCreationTime()), 1.0), pipeTsFileInsertionEvent.getTsFile(), - pipeTsFileInsertionEvent.isWithMod() ? pipeTsFileInsertionEvent.getModFile() : null); + pipeTsFileInsertionEvent.isWithMod() ? pipeTsFileInsertionEvent.getModFile() : null, + pipeTsFileInsertionEvent.getDatabaseName()); } finally { pipeTsFileInsertionEvent.decreaseReferenceCount( IoTDBDataRegionSyncSink.class.getName(), false); @@ -438,7 +439,8 @@ private void doTransferWrapper(final PipeTsFileInsertionEvent pipeTsFileInsertio private void doTransfer( final Map, Double> pipeName2WeightMap, final File tsFile, - final File modFile) + final File modFile, + final String dataBaseName) throws PipeException, IOException { final Pair clientAndStatus = clientManager.getClient(); @@ -454,7 +456,11 @@ private void doTransfer( final TPipeTransferReq req = compressIfNeeded( PipeTransferTsFileSealWithModReq.toTPipeTransferReq( - modFile.getName(), modFile.length(), tsFile.getName(), tsFile.length())); + modFile.getName(), + modFile.length(), + tsFile.getName(), + tsFile.length(), + dataBaseName)); pipeName2WeightMap.forEach( (pipePair, weight) -> @@ -479,7 +485,11 @@ private void doTransfer( try { final TPipeTransferReq req = compressIfNeeded( - PipeTransferTsFileSealReq.toTPipeTransferReq(tsFile.getName(), tsFile.length())); + dataBaseName == null + ? PipeTransferTsFileSealReq.toTPipeTransferReq( + tsFile.getName(), tsFile.length()) + : PipeTransferTsFileSealWithModReq.toTPipeTransferReq( + tsFile.getName(), tsFile.length(), dataBaseName)); pipeName2WeightMap.forEach( (pipePair, weight) -> diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java index 6e74ceed206ec..404b957c7865c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java @@ -20,7 +20,9 @@ package org.apache.iotdb.db.queryengine.plan.statement.crud; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.statement.Statement; @@ -42,10 +44,13 @@ import java.util.List; import java.util.Map; +import static org.apache.iotdb.commons.conf.IoTDBConstant.PATH_ROOT; + public class LoadTsFileStatement extends Statement { private final File file; private int databaseLevel; + private String database; private boolean verifySchema = true; private boolean deleteAfterLoad = false; private boolean convertOnTypeMismatch = true; @@ -201,6 +206,14 @@ public int getDatabaseLevel() { return databaseLevel; } + public void setDatabase(String database) { + this.database = database; + } + + public String getDatabase() { + return database; + } + public void setVerifySchema(boolean verifySchema) { this.verifySchema = verifySchema; } @@ -281,6 +294,7 @@ public boolean isAsyncLoad() { private void initAttributes(final Map loadAttributes) { this.databaseLevel = LoadTsFileConfigurator.parseOrGetDefaultDatabaseLevel(loadAttributes); + this.database = LoadTsFileConfigurator.parseDatabaseName(loadAttributes); this.deleteAfterLoad = LoadTsFileConfigurator.parseOrGetDefaultOnSuccess(loadAttributes); this.convertOnTypeMismatch = LoadTsFileConfigurator.parseOrGetDefaultConvertOnTypeMismatch(loadAttributes); @@ -293,6 +307,28 @@ private void initAttributes(final Map loadAttributes) { } } + public void updateDatabaseLevelByTreeDatabase() { + final Integer databaseLevel = getDatabaseLevelByTreeDatabase(database); + if (databaseLevel != null) { + this.databaseLevel = databaseLevel; + } + } + + public static Integer getDatabaseLevelByTreeDatabase(final String database) { + if (database == null) { + return null; + } + try { + final String[] nodes = PathUtils.splitPathToDetachedNodes(database); + if (nodes.length > 1 && PATH_ROOT.equals(nodes[0])) { + return nodes.length - 1; + } + } catch (final IllegalPathException ignored) { + // Keep the configured database level when database is not a legal tree path. + } + return null; + } + public boolean reconstructStatementIfMiniFileConverted(final List isMiniTsFile) { int lastNonMiniTsFileIndex = -1; @@ -352,6 +388,7 @@ public List getSubStatements() { final LoadTsFileStatement statement = new LoadTsFileStatement(); statement.databaseLevel = this.databaseLevel; + statement.database = this.database; statement.verifySchema = this.verifySchema; statement.deleteAfterLoad = this.deleteAfterLoad; statement.convertOnTypeMismatch = this.convertOnTypeMismatch; @@ -395,6 +432,8 @@ public String toString() { + deleteAfterLoad + ", database-level=" + databaseLevel + + ", database=" + + database + ", verify-schema=" + verifySchema + ", convert-on-type-mismatch=" diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPathHelper.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPathHelper.java index 965f2941dc659..2503b822b9eac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPathHelper.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/active/ActiveLoadPathHelper.java @@ -46,6 +46,7 @@ public final class ActiveLoadPathHelper { private static final List KEY_ORDER = Collections.unmodifiableList( Arrays.asList( + LoadTsFileConfigurator.DATABASE_NAME_KEY, LoadTsFileConfigurator.DATABASE_LEVEL_KEY, LoadTsFileConfigurator.CONVERT_ON_TYPE_MISMATCH_KEY, LoadTsFileConfigurator.TABLET_CONVERSION_THRESHOLD_KEY, @@ -62,8 +63,28 @@ public static Map buildAttributes( final Boolean verify, final Long tabletConversionThresholdBytes, final Boolean pipeGenerated) { + return buildAttributes( + null, + databaseLevel, + convertOnTypeMismatch, + verify, + tabletConversionThresholdBytes, + pipeGenerated); + } + + public static Map buildAttributes( + final String databaseName, + final Integer databaseLevel, + final Boolean convertOnTypeMismatch, + final Boolean verify, + final Long tabletConversionThresholdBytes, + final Boolean pipeGenerated) { final Map attributes = new LinkedHashMap<>(); + if (Objects.nonNull(databaseName) && !databaseName.isEmpty()) { + attributes.put(LoadTsFileConfigurator.DATABASE_NAME_KEY, databaseName); + } + if (Objects.nonNull(databaseLevel)) { attributes.put(LoadTsFileConfigurator.DATABASE_LEVEL_KEY, databaseLevel.toString()); } @@ -149,6 +170,10 @@ public static void applyAttributesToStatement( final LoadTsFileStatement statement, final boolean defaultVerify) { + Optional.ofNullable(attributes.get(LoadTsFileConfigurator.DATABASE_NAME_KEY)) + .filter(name -> !name.isEmpty()) + .ifPresent(statement::setDatabase); + Optional.ofNullable(attributes.get(LoadTsFileConfigurator.DATABASE_LEVEL_KEY)) .ifPresent( level -> { @@ -216,6 +241,11 @@ private static Optional extractAndValidateAttributeValue( private static void validateAttributeValue(final String key, final String value) { switch (key) { + case LoadTsFileConfigurator.DATABASE_NAME_KEY: + if (value == null || value.isEmpty()) { + throw new SemanticException("Database name must not be empty."); + } + break; case LoadTsFileConfigurator.DATABASE_LEVEL_KEY: LoadTsFileConfigurator.validateDatabaseLevelParam(value); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/config/LoadTsFileConfigurator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/config/LoadTsFileConfigurator.java index 510d47b0b23df..8b689c6fb22ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/config/LoadTsFileConfigurator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/config/LoadTsFileConfigurator.java @@ -40,6 +40,7 @@ public static void validateParameters(final String key, final String value) { case ON_SUCCESS_KEY: validateOnSuccessParam(value); break; + case DATABASE_NAME_KEY: case TABLET_CONVERSION_THRESHOLD_KEY: break; case CONVERT_ON_TYPE_MISMATCH_KEY: @@ -87,6 +88,12 @@ public static int parseOrGetDefaultDatabaseLevel(final Map loadA DATABASE_LEVEL_KEY, String.valueOf(DATABASE_LEVEL_DEFAULT_VALUE))); } + public static final String DATABASE_NAME_KEY = "database-name"; + + public static String parseDatabaseName(final Map loadAttributes) { + return loadAttributes.get(DATABASE_NAME_KEY); + } + public static final String ON_SUCCESS_KEY = "on-success"; public static final String ON_SUCCESS_DELETE_VALUE = "delete"; public static final String ON_SUCCESS_NONE_VALUE = "none"; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/PipeStatementTsStatusVisitorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/PipeStatementTsStatusVisitorTest.java index 2b20f1d91efb8..756d11818251f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/PipeStatementTsStatusVisitorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/PipeStatementTsStatusVisitorTest.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.receiver; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.exception.IoTDBRuntimeException; import org.apache.iotdb.commons.utils.StatusUtils; import org.apache.iotdb.db.pipe.receiver.protocol.thrift.IoTDBDataNodeReceiver; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowsStatement; @@ -62,4 +63,17 @@ public void testTTLIdempotency() { StatusUtils.OK, new TSStatus(TSStatusCode.OUT_OF_TTL.getStatusCode())))) .getCode()); } + + @Test + public void testDatabaseNotExistRuntimeExceptionClassification() { + Assert.assertEquals( + TSStatusCode.PIPE_RECEIVER_PARALLEL_OR_USER_CONFLICT_EXCEPTION.getStatusCode(), + IoTDBDataNodeReceiver.STATEMENT_EXCEPTION_VISITOR + .process( + new InsertRowsStatement(), + new IoTDBRuntimeException( + "Create DataPartition failed because the database: root.test.sg_0 is not exists", + TSStatusCode.DATABASE_NOT_EXIST.getStatusCode())) + .getCode()); + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiverTest.java new file mode 100644 index 0000000000000..f41c44763f997 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiverTest.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.receiver.protocol.thrift; + +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.queryengine.plan.statement.crud.LoadTsFileStatement; +import org.apache.iotdb.db.storageengine.load.active.ActiveLoadPathHelper; +import org.apache.iotdb.db.storageengine.load.config.LoadTsFileConfigurator; + +import org.junit.Assert; +import org.junit.Test; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Map; + +public class IoTDBDataNodeReceiverTest { + + @Test + public void testLoadTsFileSyncStatementUsesTreeDatabaseLevelFromDatabaseName() throws Exception { + final Path tsFile = Files.createTempFile("pipe-load-tree-database-level", ".tsfile"); + try { + final LoadTsFileStatement statement = + IoTDBDataNodeReceiver.buildLoadTsFileStatementForSync( + "root.test.sg_0", tsFile.toString(), true); + + Assert.assertEquals("root.test.sg_0", statement.getDatabase()); + Assert.assertEquals(2, statement.getDatabaseLevel()); + } finally { + Files.deleteIfExists(tsFile); + } + } + + @Test + public void testLoadTsFileAsyncAttributesUseTreeDatabaseLevelFromDatabaseName() throws Exception { + final Path tsFile = Files.createTempFile("pipe-async-load-tree-database-level", ".tsfile"); + try { + final Map attributes = + IoTDBDataNodeReceiver.buildLoadTsFileAttributesForAsync( + "root.test.sg_0", true, true, true); + + Assert.assertEquals( + "root.test.sg_0", attributes.get(LoadTsFileConfigurator.DATABASE_NAME_KEY)); + Assert.assertEquals("2", attributes.get(LoadTsFileConfigurator.DATABASE_LEVEL_KEY)); + + final LoadTsFileStatement statement = LoadTsFileStatement.createUnchecked(tsFile.toString()); + ActiveLoadPathHelper.applyAttributesToStatement(attributes, statement, true); + Assert.assertEquals("root.test.sg_0", statement.getDatabase()); + Assert.assertEquals(2, statement.getDatabaseLevel()); + } finally { + Files.deleteIfExists(tsFile); + } + } + + @Test + public void testLoadTsFileSyncStatementKeepsDefaultDatabaseLevelWhenDatabaseNameIsNull() + throws Exception { + final Path tsFile = Files.createTempFile("pipe-load-default-database-level", ".tsfile"); + try { + final LoadTsFileStatement statement = + IoTDBDataNodeReceiver.buildLoadTsFileStatementForSync(null, tsFile.toString(), true); + + Assert.assertNull(statement.getDatabase()); + Assert.assertEquals( + IoTDBDescriptor.getInstance().getConfig().getDefaultDatabaseLevel(), + statement.getDatabaseLevel()); + } finally { + Files.deleteIfExists(tsFile); + } + } + + @Test + public void testRepeatedStatementExceptionLogIsReduced() throws Exception { + final Path tsFile = Files.createTempFile("pipe-load-log-reducer", ".tsfile"); + try { + final LoadTsFileStatement statement = + IoTDBDataNodeReceiver.buildLoadTsFileStatementForSync( + "root.test.sg_0", tsFile.toString(), true); + final long receiverId = System.nanoTime(); + final Exception exception = new RuntimeException("repeated receiver exception " + receiverId); + + Assert.assertTrue( + IoTDBDataNodeReceiver.shouldLogStatementException(receiverId, statement, exception)); + Assert.assertFalse( + IoTDBDataNodeReceiver.shouldLogStatementException(receiverId, statement, exception)); + Assert.assertTrue( + IoTDBDataNodeReceiver.shouldLogStatementException( + receiverId, statement, new RuntimeException("another receiver exception"))); + } finally { + Files.deleteIfExists(tsFile); + } + } +} From 0288aab9d2bdb863bbcda9f1bd228d53fe418e8a Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 4 Jun 2026 14:16:59 +0800 Subject: [PATCH 085/102] Optimize InsertTabletStatement tablet conversion (#17805) (#17842) (cherry picked from commit dacbf9268db160666a3b3a3a8e31c53eb88c8f27) --- .../statement/crud/InsertTabletStatement.java | 163 ++++++++++++++++++ 1 file changed, 163 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java index 525bb48f0dde7..b5672cbbb4fc5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java @@ -492,6 +492,169 @@ protected long calculateBytesUsed() { + InsertNodeMemoryEstimator.sizeOfColumns(columns, measurementSchemas); } + /** + * Convert this InsertTabletStatement to Tablet. This method constructs a Tablet object from this + * statement, converting all necessary fields. All arrays are copied to rowSize length to ensure + * immutability. + * + * @return Tablet object + * @throws MetadataException if conversion fails + */ + public Tablet convertToTablet() throws MetadataException { + try { + // Get deviceId/tableName from devicePath + final String deviceIdOrTableName = + this.getDevicePath() != null ? this.getDevicePath().getFullPath() : ""; + + // Get schemas from measurementSchemas + final MeasurementSchema[] measurementSchemas = this.getMeasurementSchemas(); + final String[] measurements = this.getMeasurements(); + final TSDataType[] dataTypes = this.getDataTypes(); + // If measurements and dataTypes are not null, use measurements.length as the standard length + final int originalSchemaSize = measurements != null ? measurements.length : 0; + + // Build schemas and track valid column indices (skip null columns) + // measurements and dataTypes being null is standard - skip those columns + final List schemas = new ArrayList<>(originalSchemaSize); + final int[] validColumnIndices = new int[originalSchemaSize]; + int validColumnCount = 0; + if (dataTypes != null) { + final int dataTypeSize = Math.min(originalSchemaSize, dataTypes.length); + for (int i = 0; i < dataTypeSize; i++) { + if (measurements[i] != null && dataTypes[i] != null) { + final MeasurementSchema measurementSchema = + measurementSchemas != null && i < measurementSchemas.length + ? measurementSchemas[i] + : null; + schemas.add( + measurementSchema != null + && Objects.equals(measurementSchema.getMeasurementId(), measurements[i]) + && measurementSchema.getType() == dataTypes[i] + ? measurementSchema + : new MeasurementSchema(measurements[i], dataTypes[i])); + validColumnIndices[validColumnCount++] = i; + } + } + } + + final int schemaSize = validColumnCount; + + // Get timestamps - always copy to ensure immutability + final long[] times = this.getTimes(); + final int rowSize = this.getRowCount(); + final long[] timestamps; + if (rowSize == 0) { + timestamps = new long[0]; + } else if (times != null && times.length >= rowSize) { + timestamps = Arrays.copyOf(times, rowSize); + } else { + LOGGER.warn( + "Times array is null or too small. times.length={}, rowSize={}, deviceId={}", + times != null ? times.length : 0, + rowSize, + deviceIdOrTableName); + timestamps = new long[0]; + } + + // Get values - convert Statement columns to Tablet format, only for valid columns + // All arrays are copied to rowSize length + final Object[] statementColumns = this.getColumns(); + final Object[] tabletValues = new Object[schemaSize]; + if (statementColumns != null && statementColumns.length > 0) { + for (int i = 0; i < schemaSize; i++) { + final int originalIndex = validColumnIndices[i]; + if (originalIndex < statementColumns.length + && statementColumns[originalIndex] != null + && dataTypes[originalIndex] != null) { + tabletValues[i] = + convertColumnToTablet( + statementColumns[originalIndex], dataTypes[originalIndex], rowSize); + } else { + tabletValues[i] = null; + } + } + } + + // Get bitMaps - copy and truncate to rowSize, only for valid columns + final BitMap[] originalBitMaps = this.getBitMaps(); + BitMap[] bitMaps = null; + if (originalBitMaps != null && originalBitMaps.length > 0) { + final BitMap[] copiedBitMaps = new BitMap[schemaSize]; + for (int i = 0; i < schemaSize; i++) { + final int originalIndex = validColumnIndices[i]; + if (originalIndex < originalBitMaps.length && originalBitMaps[originalIndex] != null) { + final BitMap originalBitMap = originalBitMaps[originalIndex]; + if (!originalBitMap.isAllUnmarked()) { + copiedBitMaps[i] = + originalBitMap.getRegion(0, Math.min(rowSize, originalBitMap.getSize())); + } + } else { + copiedBitMaps[i] = null; + } + } + bitMaps = BitMapUtils.compactBitMaps(copiedBitMaps, rowSize); + } + + return new Tablet(deviceIdOrTableName, schemas, timestamps, tabletValues, bitMaps, rowSize); + } catch (final Exception e) { + throw new MetadataException("Failed to convert InsertTabletStatement to Tablet: ", e); + } + } + + /** + * Convert a single column value from Statement format to Tablet format. Statement uses primitive + * arrays (e.g., int[], long[], float[]), while Tablet may need different format. All arrays are + * copied to rowSize length to ensure immutability - even if the original array is modified, the + * converted array remains unchanged. + * + * @param columnValue column value from Statement (primitive array) + * @param dataType data type of the column + * @param rowSize number of rows to copy + * @return column value in Tablet format (copied to rowSize) + */ + private Object convertColumnToTablet( + final Object columnValue, final TSDataType dataType, final int rowSize) { + + if (columnValue == null) { + return null; + } + + if (TSDataType.DATE.equals(dataType)) { + final int[] values = (int[]) columnValue; + final LocalDate[] localDateValue = new LocalDate[rowSize]; + final int size = Math.min(values.length, rowSize); + for (int i = 0; i < size; i++) { + localDateValue[i] = DateUtils.parseIntToLocalDate(values[i]); + } + return localDateValue; + } + + // For primitive arrays, copy to rowSize + if (columnValue instanceof boolean[]) { + final boolean[] original = (boolean[]) columnValue; + return Arrays.copyOf(original, rowSize); + } else if (columnValue instanceof int[]) { + final int[] original = (int[]) columnValue; + return Arrays.copyOf(original, rowSize); + } else if (columnValue instanceof long[]) { + final long[] original = (long[]) columnValue; + return Arrays.copyOf(original, rowSize); + } else if (columnValue instanceof float[]) { + final float[] original = (float[]) columnValue; + return Arrays.copyOf(original, rowSize); + } else if (columnValue instanceof double[]) { + final double[] original = (double[]) columnValue; + return Arrays.copyOf(original, rowSize); + } else if (columnValue instanceof Binary[]) { + // For Binary arrays, create a new array and copy references to rowSize + final Binary[] original = (Binary[]) columnValue; + return Arrays.copyOf(original, rowSize); + } + + // For other types, return as-is (should not happen for standard types) + return columnValue; + } + @Override public String toString() { final int size = CommonDescriptor.getInstance().getConfig().getPathLogMaxSize(); From b8852699a495458169e7078a5554d92275e06545 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 5 Jun 2026 11:44:40 +0800 Subject: [PATCH 086/102] Fix operator tests for transient null TsBlocks (#17838) (#17847) --- ...gnedSeriesAggregationScanOperatorTest.java | 33 ++++++------ .../execution/operator/FillOperatorTest.java | 7 +-- .../HorizontallyConcatOperatorTest.java | 3 +- .../operator/LinearFillOperatorTest.java | 5 +- .../operator/MergeSortOperatorTest.java | 3 +- .../operator/OffsetOperatorTest.java | 19 +++++-- .../execution/operator/OperatorTestUtils.java | 53 +++++++++++++++++++ .../SeriesAggregationScanOperatorTest.java | 33 ++++++------ .../operator/SeriesScanOperatorTest.java | 3 +- .../operator/UpdateLastCacheOperatorTest.java | 7 +-- 10 files changed, 118 insertions(+), 48 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorTestUtils.java diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AlignedSeriesAggregationScanOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AlignedSeriesAggregationScanOperatorTest.java index 2a5c5c13d0b37..bc81047b17a36 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AlignedSeriesAggregationScanOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/AlignedSeriesAggregationScanOperatorTest.java @@ -69,6 +69,7 @@ import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; import static org.apache.iotdb.db.queryengine.execution.operator.AggregationOperatorTest.TEST_TIME_SLICE; import static org.apache.iotdb.db.queryengine.execution.operator.AggregationUtil.initTimeRangeIterator; +import static org.apache.iotdb.db.queryengine.execution.operator.OperatorTestUtils.nextNonNullOrEmpty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -122,7 +123,7 @@ public void testAggregationWithoutTimeFilter() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, true, null); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); for (int i = 0; i < measurementSchemas.size(); i++) { assertEquals(500, resultTsBlock.getColumn(i).getLong(0)); } @@ -153,7 +154,7 @@ public void testAggregationWithoutTimeFilterOrderByTimeDesc() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, false, null); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); for (int i = 0; i < measurementSchemas.size(); i++) { assertEquals(500, resultTsBlock.getColumn(i).getLong(0)); } @@ -189,7 +190,7 @@ public void testMultiAggregationFuncWithoutTimeFilter1() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, true, null); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(500, resultTsBlock.getColumn(0).getLong(0)); assertEquals(6524750.0, resultTsBlock.getColumn(1).getDouble(0), 0.0001); count++; @@ -227,7 +228,7 @@ public void testMultiAggregationFuncWithoutTimeFilter2() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, true, null); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertTrue(resultTsBlock.getColumn(0).getBoolean(0)); assertEquals(10499, resultTsBlock.getColumn(1).getInt(0)); assertEquals(20199, resultTsBlock.getColumn(2).getLong(0)); @@ -269,7 +270,7 @@ public void testMultiAggregationFuncWithoutTimeFilterOrderByTimeDesc() throws Ex initAlignedSeriesAggregationScanOperator(aggregators, null, false, null); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertTrue(resultTsBlock.getColumn(0).getBoolean(0)); assertEquals(10499, resultTsBlock.getColumn(1).getInt(0)); assertEquals(20199, resultTsBlock.getColumn(2).getLong(0)); @@ -304,7 +305,7 @@ public void testAggregationWithTimeFilter1() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, timeFilter, true, null); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); for (int i = 0; i < measurementSchemas.size(); i++) { assertEquals(resultTsBlock.getColumn(i).getLong(0), 380); } @@ -337,7 +338,7 @@ public void testAggregationWithTimeFilter2() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, timeFilter, true, null); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); for (int i = 0; i < measurementSchemas.size(); i++) { assertEquals(resultTsBlock.getColumn(i).getLong(0), 380); } @@ -370,7 +371,7 @@ public void testAggregationWithTimeFilter3() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, timeFilter, true, null); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); for (int i = 0; i < measurementSchemas.size(); i++) { assertEquals(resultTsBlock.getColumn(i).getLong(0), 300); } @@ -410,7 +411,7 @@ public void testMultiAggregationWithTimeFilter() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, timeFilter, true, null); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertTrue(resultTsBlock.getColumn(0).getBoolean(0)); assertEquals(399, resultTsBlock.getColumn(1).getInt(0)); assertEquals(20199, resultTsBlock.getColumn(2).getLong(0)); @@ -448,7 +449,7 @@ public void testGroupByWithoutGlobalTimeFilter() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, true, groupByTimeParameter); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(100 * count, resultTsBlock.getTimeColumn().getLong(pos)); @@ -489,7 +490,7 @@ public void testGroupByWithGlobalTimeFilter() throws Exception { aggregators, timeFilter, true, groupByTimeParameter); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(100 * count, resultTsBlock.getTimeColumn().getLong(pos)); @@ -533,7 +534,7 @@ public void testGroupByWithMultiFunction() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, true, groupByTimeParameter); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(100 * count, resultTsBlock.getTimeColumn().getLong(pos)); @@ -578,7 +579,7 @@ public void testGroupByWithMultiFunctionOrderByTimeDesc() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, false, groupByTimeParameter); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(100 * (3 - count), resultTsBlock.getTimeColumn().getLong(pos)); @@ -613,7 +614,7 @@ public void testGroupBySlidingTimeWindow() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, true, groupByTimeParameter); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(50 * count, resultTsBlock.getTimeColumn().getLong(pos)); @@ -645,7 +646,7 @@ public void testGroupBySlidingTimeWindow2() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, true, groupByTimeParameter); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(timeColumn[count], resultTsBlock.getTimeColumn().getLong(pos)); @@ -688,7 +689,7 @@ public void testGroupBySlidingWindowWithMultiFunction() throws Exception { initAlignedSeriesAggregationScanOperator(aggregators, null, true, groupByTimeParameter); int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(timeColumn[count], resultTsBlock.getTimeColumn().getLong(pos)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/FillOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/FillOperatorTest.java index e2bb5a655ec13..ac3c83a8364de 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/FillOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/FillOperatorTest.java @@ -42,6 +42,7 @@ import java.util.concurrent.ExecutorService; import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; +import static org.apache.iotdb.db.queryengine.execution.operator.OperatorTestUtils.nextNonNullOrEmpty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -209,7 +210,7 @@ public long ramBytesUsed() { } }; while (fillOperator.hasNext()) { - TsBlock block = fillOperator.next(); + TsBlock block = nextNonNullOrEmpty(fillOperator); for (int i = 0; i < block.getPositionCount(); i++) { long expectedTime = i + 1 + count * 10000L; assertEquals(expectedTime, block.getTimeByIndex(i)); @@ -386,7 +387,7 @@ public long ramBytesUsed() { } }; while (fillOperator.hasNext()) { - TsBlock block = fillOperator.next(); + TsBlock block = nextNonNullOrEmpty(fillOperator); for (int i = 0; i < block.getPositionCount(); i++) { long expectedTime = i + 1 + count * 10000L; assertEquals(expectedTime, block.getTimeByIndex(i)); @@ -563,7 +564,7 @@ public long ramBytesUsed() { } }; while (fillOperator.hasNext()) { - TsBlock block = fillOperator.next(); + TsBlock block = nextNonNullOrEmpty(fillOperator); for (int i = 0; i < block.getPositionCount(); i++) { long expectedTime = i + 1 + count * 10000L; assertEquals(expectedTime, block.getTimeByIndex(i)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/HorizontallyConcatOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/HorizontallyConcatOperatorTest.java index bad50a7e1c9cf..ec258a36c0e94 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/HorizontallyConcatOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/HorizontallyConcatOperatorTest.java @@ -66,6 +66,7 @@ import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; import static org.apache.iotdb.db.queryengine.execution.operator.AggregationUtil.initTimeRangeIterator; +import static org.apache.iotdb.db.queryengine.execution.operator.OperatorTestUtils.nextNonNullOrEmpty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -197,7 +198,7 @@ public void batchTest1() throws Exception { int count = 0; while (horizontallyConcatOperator.isBlocked().isDone() && horizontallyConcatOperator.hasNext()) { - TsBlock tsBlock = horizontallyConcatOperator.next(); + TsBlock tsBlock = nextNonNullOrEmpty(horizontallyConcatOperator); assertEquals(6, tsBlock.getValueColumnCount()); for (int i = 0; i < tsBlock.getPositionCount(); i++, count++) { assertEquals(count, tsBlock.getTimeByIndex(i)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/LinearFillOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/LinearFillOperatorTest.java index 08859dc127be6..8ea63f132e9ad 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/LinearFillOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/LinearFillOperatorTest.java @@ -41,8 +41,8 @@ import java.util.concurrent.ExecutorService; import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; +import static org.apache.iotdb.db.queryengine.execution.operator.OperatorTestUtils.nextNonNullOrEmpty; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; public class LinearFillOperatorTest { @@ -1301,8 +1301,7 @@ public long ramBytesUsed() { }; while (fillOperator.hasNext()) { - TsBlock block = fillOperator.next(); - assertNotNull(block); + TsBlock block = nextNonNullOrEmpty(fillOperator); for (int i = 0; i < block.getPositionCount(); i++) { long expectedTime = i + count; assertEquals(expectedTime, block.getTimeByIndex(i)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/MergeSortOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/MergeSortOperatorTest.java index 76ec370ad98be..9afc9c7c5409e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/MergeSortOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/MergeSortOperatorTest.java @@ -88,6 +88,7 @@ import java.util.concurrent.atomic.AtomicInteger; import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; +import static org.apache.iotdb.db.queryengine.execution.operator.OperatorTestUtils.nextNonNullOrEmpty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -1765,7 +1766,7 @@ public void mergeSortTest() throws Exception { int index = 0; while (mergeSortOperator.isBlocked().isDone() && mergeSortOperator.hasNext()) { - TsBlock result = mergeSortOperator.next(); + TsBlock result = nextNonNullOrEmpty(mergeSortOperator); for (int i = 0; i < result.getPositionCount(); i++) { long time = result.getTimeByIndex(i); assertEquals(time, ans[index++]); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OffsetOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OffsetOperatorTest.java index a1188becf1096..6602c65ff981a 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OffsetOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OffsetOperatorTest.java @@ -62,6 +62,7 @@ import java.util.concurrent.TimeUnit; import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; +import static org.apache.iotdb.db.queryengine.execution.operator.OperatorTestUtils.nextNonNullOrEmpty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -161,7 +162,7 @@ public void batchTest1() throws Exception { new LimitOperator(driverContext.getOperatorContexts().get(4), 250, offsetOperator); int count = 100; while (limitOperator.isBlocked().isDone() && limitOperator.hasNext()) { - TsBlock tsBlock = limitOperator.next(); + TsBlock tsBlock = nextNonNullOrEmpty(limitOperator); assertEquals(2, tsBlock.getValueColumnCount()); assertTrue(tsBlock.getColumn(0) instanceof IntColumn); assertTrue(tsBlock.getColumn(1) instanceof IntColumn); @@ -265,7 +266,7 @@ public void batchTest2() throws Exception { int count = 0; while (offsetOperator.isBlocked().isDone() && offsetOperator.hasNext()) { - TsBlock tsBlock = offsetOperator.next(); + TsBlock tsBlock = nextNonNullOrEmpty(offsetOperator); assertEquals(2, tsBlock.getValueColumnCount()); assertTrue(tsBlock.getColumn(0) instanceof IntColumn); assertTrue(tsBlock.getColumn(1) instanceof IntColumn); @@ -367,7 +368,7 @@ public void batchTest3() throws Exception { new OffsetOperator(driverContext.getOperatorContexts().get(3), 500, timeJoinOperator); while (offsetOperator.isBlocked().isDone() && offsetOperator.hasNext()) { - TsBlock tsBlock = offsetOperator.next(); + TsBlock tsBlock = nextNonNull(offsetOperator); assertEquals(2, tsBlock.getValueColumnCount()); assertTrue(tsBlock.getColumn(0) instanceof IntColumn); assertTrue(tsBlock.getColumn(1) instanceof IntColumn); @@ -455,7 +456,7 @@ public void batchTest4() throws Exception { driverContext.getOperatorContexts().get(3), 98_784_247_808L, timeJoinOperator); while (offsetOperator.isBlocked().isDone() && offsetOperator.hasNext()) { - TsBlock tsBlock = offsetOperator.next(); + TsBlock tsBlock = nextNonNull(offsetOperator); assertEquals(2, tsBlock.getValueColumnCount()); assertTrue(tsBlock.getColumn(0) instanceof IntColumn); assertTrue(tsBlock.getColumn(1) instanceof IntColumn); @@ -468,4 +469,14 @@ public void batchTest4() throws Exception { instanceNotificationExecutor.shutdown(); } } + + private static TsBlock nextNonNull(Operator operator) throws Exception { + while (operator.hasNext()) { + TsBlock result = operator.next(); + if (result != null) { + return result; + } + } + throw new AssertionError("Expected a non-null TsBlock from operator"); + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorTestUtils.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorTestUtils.java new file mode 100644 index 0000000000000..37b979e87bec1 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/OperatorTestUtils.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iotdb.db.queryengine.execution.operator; + +import org.apache.tsfile.read.common.block.TsBlock; + +public final class OperatorTestUtils { + + private OperatorTestUtils() { + // Utility class. + } + + public static TsBlock nextNonNullOrEmpty(Operator operator) throws Exception { + while (operator.hasNext()) { + TsBlock result = operator.next(); + if (!isNullOrEmpty(result)) { + return result; + } + } + throw new AssertionError("Expected a non-null and non-empty TsBlock from operator"); + } + + public static TsBlock lastNonNullOrEmpty(Operator operator) throws Exception { + TsBlock result = null; + while (operator.isBlocked().isDone() && operator.hasNext()) { + TsBlock nextResult = operator.next(); + if (!isNullOrEmpty(nextResult)) { + result = nextResult; + } + } + return result; + } + + private static boolean isNullOrEmpty(TsBlock tsBlock) { + return tsBlock == null || tsBlock.getPositionCount() == 0; + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesAggregationScanOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesAggregationScanOperatorTest.java index bbf96e89fb4fb..afa08223d6463 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesAggregationScanOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesAggregationScanOperatorTest.java @@ -66,6 +66,7 @@ import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; import static org.apache.iotdb.db.queryengine.execution.operator.AggregationOperatorTest.TEST_TIME_SLICE; import static org.apache.iotdb.db.queryengine.execution.operator.AggregationUtil.initTimeRangeIterator; +import static org.apache.iotdb.db.queryengine.execution.operator.OperatorTestUtils.nextNonNullOrEmpty; import static org.junit.Assert.assertEquals; public class SeriesAggregationScanOperatorTest { @@ -111,7 +112,7 @@ public void testAggregationWithoutTimeFilter() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(500, resultTsBlock.getColumn(0).getLong(0)); count++; } @@ -135,7 +136,7 @@ public void testAggregationWithoutTimeFilterOrderByTimeDesc() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(500, resultTsBlock.getColumn(0).getLong(0)); count++; } @@ -161,7 +162,7 @@ public void testMultiAggregationFuncWithoutTimeFilter1() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(500, resultTsBlock.getColumn(0).getLong(0)); assertEquals(6524750.0, resultTsBlock.getColumn(1).getDouble(0), 0.0001); count++; @@ -192,7 +193,7 @@ public void testMultiAggregationFuncWithoutTimeFilter2() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(20000, resultTsBlock.getColumn(0).getInt(0)); assertEquals(10499, resultTsBlock.getColumn(1).getInt(0)); assertEquals(0, resultTsBlock.getColumn(2).getLong(0)); @@ -228,7 +229,7 @@ public void testMultiAggregationFuncWithoutTimeFilterOrderByTimeDesc() throws Ex int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(20000, resultTsBlock.getColumn(0).getInt(0)); assertEquals(10499, resultTsBlock.getColumn(1).getInt(0)); assertEquals(0, resultTsBlock.getColumn(2).getLong(0)); @@ -259,7 +260,7 @@ public void testAggregationWithTimeFilter1() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(resultTsBlock.getColumn(0).getLong(0), 380); count++; } @@ -285,7 +286,7 @@ public void testAggregationWithTimeFilter2() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(resultTsBlock.getColumn(0).getLong(0), 380); count++; } @@ -311,7 +312,7 @@ public void testAggregationWithTimeFilter3() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(resultTsBlock.getColumn(0).getLong(0), 300); count++; } @@ -343,7 +344,7 @@ public void testMultiAggregationWithTimeFilter() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); assertEquals(20100, resultTsBlock.getColumn(0).getInt(0)); assertEquals(399, resultTsBlock.getColumn(1).getInt(0)); assertEquals(100, resultTsBlock.getColumn(2).getLong(0)); @@ -375,7 +376,7 @@ public void testGroupByWithoutGlobalTimeFilter() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(100 * count, resultTsBlock.getTimeColumn().getLong(pos)); @@ -408,7 +409,7 @@ public void testGroupByWithGlobalTimeFilter() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(100 * count, resultTsBlock.getTimeColumn().getLong(pos)); @@ -450,7 +451,7 @@ public void testGroupByWithMultiFunction() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(100 * count, resultTsBlock.getTimeColumn().getLong(pos)); @@ -495,7 +496,7 @@ public void testGroupByWithMultiFunctionOrderByTimeDesc() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(100 * (3 - count), resultTsBlock.getTimeColumn().getLong(pos)); @@ -530,7 +531,7 @@ public void testGroupBySlidingTimeWindow() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(50 * count, resultTsBlock.getTimeColumn().getLong(pos)); @@ -563,7 +564,7 @@ public void testGroupBySlidingTimeWindow2() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(timeColumn[count], resultTsBlock.getTimeColumn().getLong(pos)); @@ -606,7 +607,7 @@ public void testGroupBySlidingWindowWithMultiFunction() throws Exception { int count = 0; while (seriesAggregationScanOperator.hasNext()) { - TsBlock resultTsBlock = seriesAggregationScanOperator.next(); + TsBlock resultTsBlock = nextNonNullOrEmpty(seriesAggregationScanOperator); int positionCount = resultTsBlock.getPositionCount(); for (int pos = 0; pos < positionCount; pos++) { assertEquals(timeColumn[count], resultTsBlock.getTimeColumn().getLong(pos)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesScanOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesScanOperatorTest.java index 11daae6f1c547..39455ab420895 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesScanOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/SeriesScanOperatorTest.java @@ -55,6 +55,7 @@ import java.util.concurrent.TimeUnit; import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; +import static org.apache.iotdb.db.queryengine.execution.operator.OperatorTestUtils.nextNonNullOrEmpty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -114,7 +115,7 @@ public void batchTest() throws Exception { int count = 0; while (seriesScanOperator.hasNext()) { - TsBlock tsBlock = seriesScanOperator.next(); + TsBlock tsBlock = nextNonNullOrEmpty(seriesScanOperator); assertEquals(1, tsBlock.getValueColumnCount()); assertTrue(tsBlock.getColumn(0) instanceof IntColumn); for (int i = 0; i < tsBlock.getPositionCount(); i++, count++) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/UpdateLastCacheOperatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/UpdateLastCacheOperatorTest.java index 0c9c47815aebf..30604831e2680 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/UpdateLastCacheOperatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/UpdateLastCacheOperatorTest.java @@ -62,6 +62,7 @@ import static org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext.createFragmentInstanceContext; import static org.apache.iotdb.db.queryengine.execution.operator.AggregationOperatorTest.TEST_TIME_SLICE; import static org.apache.iotdb.db.queryengine.execution.operator.AggregationUtil.initTimeRangeIterator; +import static org.apache.iotdb.db.queryengine.execution.operator.OperatorTestUtils.nextNonNullOrEmpty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -103,7 +104,7 @@ public void testUpdateLastCacheOperatorTestWithoutTimeFilter() { assertTrue(updateLastCacheOperator.isBlocked().isDone()); assertTrue(updateLastCacheOperator.hasNext()); - TsBlock result = updateLastCacheOperator.next(); + TsBlock result = nextNonNullOrEmpty(updateLastCacheOperator); assertEquals(1, result.getPositionCount()); assertEquals(3, result.getValueColumnCount()); @@ -133,7 +134,7 @@ public void testUpdateLastCacheOperatorTestWithTimeFilter1() { assertTrue(updateLastCacheOperator.isBlocked().isDone()); assertTrue(updateLastCacheOperator.hasNext()); - TsBlock result = updateLastCacheOperator.next(); + TsBlock result = nextNonNullOrEmpty(updateLastCacheOperator); assertEquals(1, result.getPositionCount()); assertEquals(3, result.getValueColumnCount()); @@ -163,7 +164,7 @@ public void testUpdateLastCacheOperatorTestWithTimeFilter2() { assertTrue(updateLastCacheOperator.isBlocked().isDone()); assertTrue(updateLastCacheOperator.hasNext()); - TsBlock result = updateLastCacheOperator.next(); + TsBlock result = nextNonNullOrEmpty(updateLastCacheOperator); assertEquals(1, result.getPositionCount()); assertEquals(3, result.getValueColumnCount()); From 1cb4bf26a75fcfeeda638592e03b3b2a7311bb57 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 5 Jun 2026 14:50:39 +0800 Subject: [PATCH 087/102] Fix flaky stamped lock test (#17832) (#17839) (cherry picked from commit 2fd70b9d1b17849862f24b719d0d2c16a80c448f) --- .../lock/StampedWriterPreferredLockTest.java | 42 ++++++++++++++----- 1 file changed, 31 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/mtree/lock/StampedWriterPreferredLockTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/mtree/lock/StampedWriterPreferredLockTest.java index 2c9fdbdfd3d54..b559aaf81529a 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/mtree/lock/StampedWriterPreferredLockTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/mtree/lock/StampedWriterPreferredLockTest.java @@ -25,6 +25,7 @@ import org.junit.Assert; import org.junit.Test; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -143,22 +144,41 @@ public void testThreadReadLock() { } @Test - public void testAcquireReadLockWhileWriting() { + public void testAcquireReadLockWhileWriting() throws InterruptedException { StampedWriterPreferredLock lock = new StampedWriterPreferredLock(); lock.writeLock(); AtomicInteger counter = new AtomicInteger(); - new Thread( + CountDownLatch readerFinished = new CountDownLatch(1); + Thread readerThread = + new Thread( () -> { lock.threadReadLock(); - counter.incrementAndGet(); - lock.threadReadUnlock(); - }) - .start(); - // block reader util writer release write lock - Assert.assertEquals(0, counter.get()); - lock.writeUnlock(); - Awaitility.await().atMost(2, TimeUnit.SECONDS).until(() -> counter.get() == 1); - Assert.assertEquals(1, counter.get()); + try { + counter.incrementAndGet(); + } finally { + lock.threadReadUnlock(); + readerFinished.countDown(); + } + }); + readerThread.setDaemon(true); + readerThread.start(); + + boolean writeLocked = true; + try { + // block reader until writer release write lock + Awaitility.await() + .atMost(10, TimeUnit.SECONDS) + .until(() -> readerThread.getState() == Thread.State.WAITING); + Assert.assertEquals(0, counter.get()); + lock.writeUnlock(); + writeLocked = false; + Assert.assertTrue(readerFinished.await(10, TimeUnit.SECONDS)); + Assert.assertEquals(1, counter.get()); + } finally { + if (writeLocked) { + lock.writeUnlock(); + } + } } @Test From 9d0d750da7442cc146ee1541c014201fc78ef475 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 5 Jun 2026 14:51:19 +0800 Subject: [PATCH 088/102] Fix pipe permission retry and table parser progress (#17844) (#17848) (cherry picked from commit eaa5bcb012e48525424dcba0140b293747e7f564) --- .../sink/protocol/writeback/WriteBackSink.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/writeback/WriteBackSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/writeback/WriteBackSink.java index 21976f63ab602..269d0059c6695 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/writeback/WriteBackSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/writeback/WriteBackSink.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.sink.protocol.writeback; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.exception.pipe.PipeRuntimeSinkNonReportTimeConfigurableException; import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -131,7 +132,8 @@ private void doTransfer( status = statement.isEmpty() ? RpcUtils.SUCCESS_STATUS : executeStatement(statement); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - throw new PipeException( + throwWriteBackExceptionIfNecessary( + status, String.format( "Transfer PipeInsertNodeTabletInsertionEvent %s error, result status %s", pipeInsertNodeTabletInsertionEvent, status)); @@ -162,13 +164,23 @@ private void doTransfer(final PipeRawTabletInsertionEvent pipeRawTabletInsertion statement.isEmpty() ? RpcUtils.SUCCESS_STATUS : executeStatement(statement); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - throw new PipeException( + throwWriteBackExceptionIfNecessary( + status, String.format( "Transfer PipeRawTabletInsertionEvent %s error, result status %s", pipeRawTabletInsertionEvent, status)); } } + private static void throwWriteBackExceptionIfNecessary( + final TSStatus status, final String exceptionMessage) { + if (status.getCode() == TSStatusCode.NO_PERMISSION.getStatusCode()) { + throw new PipeRuntimeSinkNonReportTimeConfigurableException(exceptionMessage, Long.MAX_VALUE); + } + + throw new PipeException(exceptionMessage); + } + private TSStatus executeStatement(final InsertBaseStatement statement) { return Coordinator.getInstance() .executeForTreeModel( From 3af79aa3b47c888336898da22b74c19d62008c90 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 5 Jun 2026 15:08:50 +0800 Subject: [PATCH 089/102] [to dev/1.3] Fix count and show timeseries follow-up (#17837) * Fix count and show timeseries follow-up (#17804) * Fix count and show timeseries follow-up * Refine internal timeseries count permissions * fix * Fix timeseries count source permission tests --- .../regionscan/IoTDBActiveSchemaQueryIT.java | 13 + .../impl/DataNodeInternalRPCServiceImpl.java | 2 +- .../queryengine/common/TimeseriesContext.java | 234 ++++++++++++++++-- .../schema/source/SchemaSourceFactory.java | 15 +- .../schema/source/TimeSeriesSchemaSource.java | 20 +- .../ActiveTimeSeriesRegionScanOperator.java | 77 ++++-- .../plan/analyze/AnalyzeVisitor.java | 48 +++- .../plan/planner/LogicalPlanBuilder.java | 12 +- .../plan/planner/LogicalPlanVisitor.java | 6 +- .../plan/planner/OperatorTreeGenerator.java | 6 +- .../planner/distribution/SourceRewriter.java | 10 +- .../read/LevelTimeSeriesCountNode.java | 34 ++- .../metedata/read/TimeSeriesCountNode.java | 28 ++- .../node/source/TimeseriesRegionScanNode.java | 7 + .../statement/metadata/CountStatement.java | 81 ++++++ .../metadata/CountTimeSeriesStatement.java | 33 +-- .../iotdb/db/auth/AuthorityCheckerTest.java | 74 ++++++ .../source/TimeSeriesSchemaSourceTest.java | 26 +- .../distribution/RegionScanPlanningTest.java | 81 ++++++ .../logical/RegionScanLogicalPlannerTest.java | 47 ++++ .../read/SchemaCountNodeSerdeTest.java | 6 +- 21 files changed, 742 insertions(+), 118 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/regionscan/IoTDBActiveSchemaQueryIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/regionscan/IoTDBActiveSchemaQueryIT.java index 98713b94767bb..f062994fa3d46 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/schema/regionscan/IoTDBActiveSchemaQueryIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/schema/regionscan/IoTDBActiveSchemaQueryIT.java @@ -259,10 +259,23 @@ public void testCountTimeSeriesWithTimeConditionIncludesView() { statement, "count timeseries root.view_count.** where time>0", new HashSet<>(Collections.singletonList("2,"))); + checkResultSet( + statement, + "show timeseries root.view_count.** where time>0", + new HashSet<>( + Arrays.asList( + "root.view_count.src.s1,null,root.view_count,INT32,PLAIN,LZ4,null,null,null,null,BASE,", + "root.view_count.dst.v1,null,root.view_count,INT32,null,null,null,null,null,null,VIEW,"))); checkResultSet( statement, "count timeseries root.view_count.dst.** where time>0", new HashSet<>(Collections.singletonList("1,"))); + checkResultSet( + statement, + "show timeseries root.view_count.dst.** where time>0", + new HashSet<>( + Collections.singletonList( + "root.view_count.dst.v1,null,root.view_count,INT32,null,null,null,null,null,null,VIEW,"))); } catch (Exception e) { e.printStackTrace(); Assert.fail(e.getMessage()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index e265994a370ee..f5c84a2375feb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -926,7 +926,7 @@ public TCheckTimeSeriesExistenceResp checkTimeSeriesExistence(TCheckTimeSeriesEx for (PartialPath pattern : filteredPatternTree.getAllPathPatterns()) { ISchemaSource schemaSource = SchemaSourceFactory.getTimeSeriesSchemaCountSource( - pattern, false, null, null, SchemaConstant.ALL_MATCH_SCOPE); + pattern, false, null, null, SchemaConstant.ALL_MATCH_SCOPE, true); try (ISchemaReader schemaReader = schemaSource.getSchemaReader(schemaRegion)) { if (schemaReader.hasNext()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/TimeseriesContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/TimeseriesContext.java index aaf8b51787db5..ecad26bac004c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/TimeseriesContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/TimeseriesContext.java @@ -29,7 +29,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; -import java.util.HashSet; +import java.util.HashMap; +import java.util.Map; import java.util.Objects; import java.util.Set; @@ -45,20 +46,70 @@ public class TimeseriesContext { private final String deadband; private final String deadbandParameters; + private final String database; private final int activeCountMultiplier; - private final Set activeLogicalViewCountSet; + private final boolean logicalView; + private final Map activeLogicalViewContextMap; public TimeseriesContext(IMeasurementSchemaInfo schemaInfo) { - this(schemaInfo, 1, Collections.emptySet()); + this(schemaInfo, 1, Collections.emptyMap()); } public TimeseriesContext( IMeasurementSchemaInfo schemaInfo, int activeCountMultiplier, Set activeLogicalViewCountSet) { + this(schemaInfo, activeCountMultiplier, createLogicalViewContextMap(activeLogicalViewCountSet)); + } + + public TimeseriesContext( + IMeasurementSchemaInfo schemaInfo, + int activeCountMultiplier, + Map activeLogicalViewContextMap) { this.dataType = schemaInfo.getSchema().getType().toString(); - this.encoding = schemaInfo.getSchema().getEncodingType().toString(); - this.compression = schemaInfo.getSchema().getCompressor().toString(); + this.logicalView = schemaInfo.isLogicalView(); + if (logicalView) { + this.encoding = null; + this.compression = null; + } else { + this.encoding = schemaInfo.getSchema().getEncodingType().toString(); + this.compression = schemaInfo.getSchema().getCompressor().toString(); + } + this.alias = schemaInfo.getAlias(); + this.tags = mapToString(schemaInfo.getTagMap()); + this.attributes = mapToString(schemaInfo.getAttributeMap()); + Pair deadbandInfo = + MetaUtils.parseDeadbandInfo(schemaInfo.getSchema().getProps()); + this.deadband = deadbandInfo.left; + this.deadbandParameters = deadbandInfo.right; + this.database = null; + this.activeCountMultiplier = activeCountMultiplier; + this.activeLogicalViewContextMap = new HashMap<>(activeLogicalViewContextMap); + } + + public TimeseriesContext( + IMeasurementSchemaInfo schemaInfo, + String dataType, + int activeCountMultiplier, + Map activeLogicalViewContextMap) { + this(schemaInfo, dataType, null, activeCountMultiplier, activeLogicalViewContextMap); + } + + public TimeseriesContext( + IMeasurementSchemaInfo schemaInfo, + String dataType, + String database, + int activeCountMultiplier, + Map activeLogicalViewContextMap) { + this.dataType = dataType; + this.logicalView = schemaInfo.isLogicalView(); + if (logicalView) { + this.encoding = null; + this.compression = null; + } else { + this.encoding = schemaInfo.getSchema().getEncodingType().toString(); + this.compression = schemaInfo.getSchema().getCompressor().toString(); + } this.alias = schemaInfo.getAlias(); this.tags = mapToString(schemaInfo.getTagMap()); this.attributes = mapToString(schemaInfo.getAttributeMap()); @@ -66,8 +117,9 @@ public TimeseriesContext( MetaUtils.parseDeadbandInfo(schemaInfo.getSchema().getProps()); this.deadband = deadbandInfo.left; this.deadbandParameters = deadbandInfo.right; + this.database = database; this.activeCountMultiplier = activeCountMultiplier; - this.activeLogicalViewCountSet = new HashSet<>(activeLogicalViewCountSet); + this.activeLogicalViewContextMap = new HashMap<>(activeLogicalViewContextMap); } public String getDataType() { @@ -102,12 +154,24 @@ public String getDeadband() { return deadband; } + public String getDatabase() { + return database; + } + public int getActiveCountMultiplier() { return activeCountMultiplier; } public Set getActiveLogicalViewCountSet() { - return activeLogicalViewCountSet; + return activeLogicalViewContextMap.keySet(); + } + + public Map getActiveLogicalViewContextMap() { + return activeLogicalViewContextMap; + } + + public boolean isLogicalView() { + return logicalView; } public TimeseriesContext( @@ -129,7 +193,9 @@ public TimeseriesContext( deadband, deadbandParameters, 1, - Collections.emptySet()); + false, + null, + Collections.emptyMap()); } public TimeseriesContext( @@ -143,6 +209,87 @@ public TimeseriesContext( String deadbandParameters, int activeCountMultiplier, Set activeLogicalViewCountSet) { + this( + dataType, + alias, + encoding, + compression, + tags, + attributes, + deadband, + deadbandParameters, + activeCountMultiplier, + false, + null, + createLogicalViewContextMap(activeLogicalViewCountSet)); + } + + public TimeseriesContext( + String dataType, + String alias, + String encoding, + String compression, + String tags, + String attributes, + String deadband, + String deadbandParameters, + int activeCountMultiplier, + Map activeLogicalViewContextMap) { + this( + dataType, + alias, + encoding, + compression, + tags, + attributes, + deadband, + deadbandParameters, + activeCountMultiplier, + false, + null, + activeLogicalViewContextMap); + } + + public TimeseriesContext( + String dataType, + String alias, + String encoding, + String compression, + String tags, + String attributes, + String deadband, + String deadbandParameters, + int activeCountMultiplier, + boolean logicalView, + Map activeLogicalViewContextMap) { + this( + dataType, + alias, + encoding, + compression, + tags, + attributes, + deadband, + deadbandParameters, + activeCountMultiplier, + logicalView, + null, + activeLogicalViewContextMap); + } + + public TimeseriesContext( + String dataType, + String alias, + String encoding, + String compression, + String tags, + String attributes, + String deadband, + String deadbandParameters, + int activeCountMultiplier, + boolean logicalView, + String database, + Map activeLogicalViewContextMap) { this.dataType = dataType; this.alias = alias; this.encoding = encoding; @@ -151,13 +298,31 @@ public TimeseriesContext( this.attributes = attributes; this.deadband = deadband; this.deadbandParameters = deadbandParameters; + this.database = database; this.activeCountMultiplier = activeCountMultiplier; - this.activeLogicalViewCountSet = new HashSet<>(activeLogicalViewCountSet); + this.logicalView = logicalView; + this.activeLogicalViewContextMap = new HashMap<>(activeLogicalViewContextMap); + } + + private static Map createLogicalViewContextMap( + Set activeLogicalViewCountSet) { + if (activeLogicalViewCountSet.isEmpty()) { + return Collections.emptyMap(); + } + Map activeLogicalViewContextMap = new HashMap<>(); + for (String logicalView : activeLogicalViewCountSet) { + activeLogicalViewContextMap.put( + logicalView, + new TimeseriesContext( + null, null, null, null, null, null, null, null, 1, true, Collections.emptyMap())); + } + return activeLogicalViewContextMap; } public TimeseriesContext mergeActiveCount(TimeseriesContext that) { - Set mergedActiveLogicalViewCountSet = new HashSet<>(activeLogicalViewCountSet); - mergedActiveLogicalViewCountSet.addAll(that.activeLogicalViewCountSet); + Map mergedActiveLogicalViewContextMap = + new HashMap<>(activeLogicalViewContextMap); + mergedActiveLogicalViewContextMap.putAll(that.activeLogicalViewContextMap); return new TimeseriesContext( dataType, alias, @@ -168,7 +333,9 @@ public TimeseriesContext mergeActiveCount(TimeseriesContext that) { deadband, deadbandParameters, activeCountMultiplier + that.activeCountMultiplier, - mergedActiveLogicalViewCountSet); + logicalView, + database, + mergedActiveLogicalViewContextMap); } public void serializeAttributes(ByteBuffer byteBuffer) { @@ -180,10 +347,13 @@ public void serializeAttributes(ByteBuffer byteBuffer) { ReadWriteIOUtils.write(attributes, byteBuffer); ReadWriteIOUtils.write(deadband, byteBuffer); ReadWriteIOUtils.write(deadbandParameters, byteBuffer); + ReadWriteIOUtils.write(database, byteBuffer); ReadWriteIOUtils.write(activeCountMultiplier, byteBuffer); - ReadWriteIOUtils.write(activeLogicalViewCountSet.size(), byteBuffer); - for (String logicalView : activeLogicalViewCountSet) { - ReadWriteIOUtils.write(logicalView, byteBuffer); + ReadWriteIOUtils.write(logicalView, byteBuffer); + ReadWriteIOUtils.write(activeLogicalViewContextMap.size(), byteBuffer); + for (Map.Entry entry : activeLogicalViewContextMap.entrySet()) { + ReadWriteIOUtils.write(entry.getKey(), byteBuffer); + entry.getValue().serializeAttributes(byteBuffer); } } @@ -196,10 +366,13 @@ public void serializeAttributes(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(attributes, stream); ReadWriteIOUtils.write(deadband, stream); ReadWriteIOUtils.write(deadbandParameters, stream); + ReadWriteIOUtils.write(database, stream); ReadWriteIOUtils.write(activeCountMultiplier, stream); - ReadWriteIOUtils.write(activeLogicalViewCountSet.size(), stream); - for (String logicalView : activeLogicalViewCountSet) { - ReadWriteIOUtils.write(logicalView, stream); + ReadWriteIOUtils.write(logicalView, stream); + ReadWriteIOUtils.write(activeLogicalViewContextMap.size(), stream); + for (Map.Entry entry : activeLogicalViewContextMap.entrySet()) { + ReadWriteIOUtils.write(entry.getKey(), stream); + entry.getValue().serializeAttributes(stream); } } @@ -212,11 +385,14 @@ public static TimeseriesContext deserialize(ByteBuffer buffer) { String attributes = ReadWriteIOUtils.readString(buffer); String deadband = ReadWriteIOUtils.readString(buffer); String deadbandParameters = ReadWriteIOUtils.readString(buffer); + String database = ReadWriteIOUtils.readString(buffer); int activeCountMultiplier = ReadWriteIOUtils.readInt(buffer); - int activeLogicalViewCountSetSize = ReadWriteIOUtils.readInt(buffer); - Set activeLogicalViewCountSet = new HashSet<>(); - for (int i = 0; i < activeLogicalViewCountSetSize; i++) { - activeLogicalViewCountSet.add(ReadWriteIOUtils.readString(buffer)); + boolean logicalView = ReadWriteIOUtils.readBool(buffer); + int activeLogicalViewContextMapSize = ReadWriteIOUtils.readInt(buffer); + Map activeLogicalViewContextMap = new HashMap<>(); + for (int i = 0; i < activeLogicalViewContextMapSize; i++) { + activeLogicalViewContextMap.put( + ReadWriteIOUtils.readString(buffer), TimeseriesContext.deserialize(buffer)); } return new TimeseriesContext( dataType, @@ -228,7 +404,9 @@ public static TimeseriesContext deserialize(ByteBuffer buffer) { deadband, deadbandParameters, activeCountMultiplier, - activeLogicalViewCountSet); + logicalView, + database, + activeLogicalViewContextMap); } @Override @@ -243,14 +421,16 @@ public boolean equals(Object obj) { boolean res = Objects.equals(dataType, that.dataType) && Objects.equals(alias, that.alias) - && encoding.equals(that.encoding) + && Objects.equals(encoding, that.encoding) && Objects.equals(compression, that.compression) && Objects.equals(tags, that.tags) && Objects.equals(attributes, that.attributes) && Objects.equals(deadband, that.deadband) && Objects.equals(deadbandParameters, that.deadbandParameters) + && Objects.equals(database, that.database) && activeCountMultiplier == that.activeCountMultiplier - && Objects.equals(activeLogicalViewCountSet, that.activeLogicalViewCountSet); + && logicalView == that.logicalView + && Objects.equals(activeLogicalViewContextMap, that.activeLogicalViewContextMap); return res; } @@ -265,7 +445,9 @@ public int hashCode() { attributes, deadband, deadbandParameters, + database, activeCountMultiplier, - activeLogicalViewCountSet); + logicalView, + activeLogicalViewContextMap); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java index 8a0e4abf3f7fe..ee8ea51baafb0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/SchemaSourceFactory.java @@ -42,9 +42,18 @@ public static ISchemaSource getTimeSeriesSchemaCountSourc boolean isPrefixMatch, SchemaFilter schemaFilter, Map templateMap, - PathPatternTree scope) { + PathPatternTree scope, + boolean includeSystemDatabase) { return new TimeSeriesSchemaSource( - pathPattern, isPrefixMatch, 0, 0, schemaFilter, templateMap, false, true, scope); + pathPattern, + isPrefixMatch, + 0, + 0, + schemaFilter, + templateMap, + false, + includeSystemDatabase, + scope); } // show time series @@ -57,7 +66,7 @@ public static ISchemaSource getTimeSeriesSchemaScanSource Map templateMap, PathPatternTree scope) { return new TimeSeriesSchemaSource( - pathPattern, isPrefixMatch, limit, offset, schemaFilter, templateMap, true, false, scope); + pathPattern, isPrefixMatch, limit, offset, schemaFilter, templateMap, true, true, scope); } // count device diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java index cc47a9361b6fd..6ecd9dc282d5e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSource.java @@ -54,7 +54,7 @@ public class TimeSeriesSchemaSource implements ISchemaSource templateMap; private final boolean needViewDetail; - private final boolean excludeInternalDatabase; + private final boolean includeSystemDatabase; TimeSeriesSchemaSource( PartialPath pathPattern, @@ -64,7 +64,7 @@ public class TimeSeriesSchemaSource implements ISchemaSource templateMap, boolean needViewDetail, - boolean excludeInternalDatabase, + boolean includeSystemDatabase, PathPatternTree scope) { this.pathPattern = pathPattern; this.isPrefixMatch = isPrefixMatch; @@ -73,7 +73,7 @@ public class TimeSeriesSchemaSource implements ISchemaSource map) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ActiveTimeSeriesRegionScanOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ActiveTimeSeriesRegionScanOperator.java index 264bea0dd0e0b..a42386dfd4ef3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ActiveTimeSeriesRegionScanOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/ActiveTimeSeriesRegionScanOperator.java @@ -46,8 +46,12 @@ public class ActiveTimeSeriesRegionScanOperator extends AbstractRegionScanDataSo // Timeseries which need to be checked. private final Map> timeSeriesToSchemasInfo; private final Set countedLogicalViews; - private static final Binary VIEW_TYPE = new Binary("BASE".getBytes()); + private static final Binary BASE_VIEW_TYPE = + new Binary("BASE".getBytes(TSFileConfig.STRING_CHARSET)); + private static final Binary LOGICAL_VIEW_TYPE = + new Binary("VIEW".getBytes(TSFileConfig.STRING_CHARSET)); private final Binary dataBaseName; + private final String dataBaseNameString; private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(ActiveTimeSeriesRegionScanOperator.class) + RamUsageEstimator.shallowSizeOfInstance(Map.class) @@ -66,14 +70,13 @@ public ActiveTimeSeriesRegionScanOperator( this.timeSeriesToSchemasInfo = timeSeriesToSchemasInfo; this.countedLogicalViews = new HashSet<>(); this.regionScanUtil = new RegionScanForActiveTimeSeriesUtil(timeFilter, ttlCache); - this.dataBaseName = - new Binary( - operatorContext - .getDriverContext() - .getFragmentInstanceContext() - .getDataRegion() - .getDatabaseName() - .getBytes(TSFileConfig.STRING_CHARSET)); + this.dataBaseNameString = + operatorContext + .getDriverContext() + .getFragmentInstanceContext() + .getDataRegion() + .getDatabaseName(); + this.dataBaseName = new Binary(this.dataBaseNameString.getBytes(TSFileConfig.STRING_CHARSET)); } @Override @@ -97,9 +100,6 @@ private void checkAndAppend(String info, ColumnBuilder columnBuilders) { @Override protected void updateActiveData() { - TimeColumnBuilder timeColumnBuilder = resultTsBlockBuilder.getTimeColumnBuilder(); - ColumnBuilder[] columnBuilders = resultTsBlockBuilder.getValueColumnBuilders(); - Map> activeTimeSeries = ((RegionScanForActiveTimeSeriesUtil) regionScanUtil).getActiveTimeSeries(); @@ -128,26 +128,49 @@ protected void updateActiveData() { Map timeSeriesInfo = timeSeriesToSchemasInfo.get(deviceID); for (String timeSeries : timeSeriesList) { TimeseriesContext schemaInfo = timeSeriesInfo.get(timeSeries); - timeColumnBuilder.writeLong(-1); - columnBuilders[0].writeBinary( - new Binary(contactDeviceAndMeasurement(deviceStr, timeSeries))); - - checkAndAppend(schemaInfo.getAlias(), columnBuilders[1]); // Measurement - columnBuilders[2].writeBinary(dataBaseName); // Database - checkAndAppend(schemaInfo.getDataType(), columnBuilders[3]); // DataType - checkAndAppend(schemaInfo.getEncoding(), columnBuilders[4]); // Encoding - checkAndAppend(schemaInfo.getCompression(), columnBuilders[5]); // Compression - checkAndAppend(schemaInfo.getTags(), columnBuilders[6]); // Tags - checkAndAppend(schemaInfo.getAttributes(), columnBuilders[7]); // Attributes - checkAndAppend(schemaInfo.getDeadband(), columnBuilders[8]); // Description - checkAndAppend(schemaInfo.getDeadbandParameters(), columnBuilders[9]); // DeadbandParameters - columnBuilders[10].writeBinary(VIEW_TYPE); // ViewType - resultTsBlockBuilder.declarePosition(); + if (schemaInfo.getActiveCountMultiplier() > 0) { + appendTimeseries( + contactDeviceAndMeasurement(deviceStr, timeSeries), schemaInfo, BASE_VIEW_TYPE); + } + for (Map.Entry logicalViewEntry : + schemaInfo.getActiveLogicalViewContextMap().entrySet()) { + if (countedLogicalViews.add(logicalViewEntry.getKey())) { + appendTimeseries( + logicalViewEntry.getKey().getBytes(TSFileConfig.STRING_CHARSET), + logicalViewEntry.getValue(), + LOGICAL_VIEW_TYPE); + } + } } removeTimeseriesListFromDevice(deviceID, timeSeriesList); } } + private void appendTimeseries( + byte[] timeseriesPath, TimeseriesContext schemaInfo, Binary viewType) { + TimeColumnBuilder timeColumnBuilder = resultTsBlockBuilder.getTimeColumnBuilder(); + ColumnBuilder[] columnBuilders = resultTsBlockBuilder.getValueColumnBuilders(); + + timeColumnBuilder.writeLong(-1); + columnBuilders[0].writeBinary(new Binary(timeseriesPath)); + + checkAndAppend(schemaInfo.getAlias(), columnBuilders[1]); // Measurement + if (schemaInfo.getDatabase() == null || dataBaseNameString.equals(schemaInfo.getDatabase())) { + columnBuilders[2].writeBinary(dataBaseName); // Database + } else { + checkAndAppend(schemaInfo.getDatabase(), columnBuilders[2]); // Database + } + checkAndAppend(schemaInfo.getDataType(), columnBuilders[3]); // DataType + checkAndAppend(schemaInfo.getEncoding(), columnBuilders[4]); // Encoding + checkAndAppend(schemaInfo.getCompression(), columnBuilders[5]); // Compression + checkAndAppend(schemaInfo.getTags(), columnBuilders[6]); // Tags + checkAndAppend(schemaInfo.getAttributes(), columnBuilders[7]); // Attributes + checkAndAppend(schemaInfo.getDeadband(), columnBuilders[8]); // Description + checkAndAppend(schemaInfo.getDeadbandParameters(), columnBuilders[9]); // DeadbandParameters + columnBuilders[10].writeBinary(viewType); // ViewType + resultTsBlockBuilder.declarePosition(); + } + private void removeTimeseriesListFromDevice(IDeviceID deviceID, List timeSeriesList) { Map timeSeriesInfo = timeSeriesToSchemasInfo.get(deviceID); for (String timeSeries : timeSeriesList) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index b4d056c1ff7d1..a7f955346700d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -36,6 +36,7 @@ import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.schema.SchemaConstant; import org.apache.iotdb.commons.schema.view.LogicalViewSchema; import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; import org.apache.iotdb.commons.service.metric.PerformanceOverviewMetrics; @@ -53,6 +54,7 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.queryengine.common.DeviceContext; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; +import org.apache.iotdb.db.queryengine.common.NodeRef; import org.apache.iotdb.db.queryengine.common.TimeseriesContext; import org.apache.iotdb.db.queryengine.common.header.ColumnHeader; import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant; @@ -83,6 +85,7 @@ import org.apache.iotdb.db.queryengine.plan.expression.leaf.ConstantOperand; import org.apache.iotdb.db.queryengine.plan.expression.leaf.TimeSeriesOperand; import org.apache.iotdb.db.queryengine.plan.expression.multi.FunctionExpression; +import org.apache.iotdb.db.queryengine.plan.expression.visitor.CompleteMeasurementSchemaVisitor; import org.apache.iotdb.db.queryengine.plan.expression.visitor.ExistUnknownTypeInExpression; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metedata.write.MeasurementGroup; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.DeviceViewIntoPathDescriptor; @@ -157,6 +160,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.sys.ExplainStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowQueriesStatement; import org.apache.iotdb.db.queryengine.plan.statement.sys.ShowVersionStatement; +import org.apache.iotdb.db.schemaengine.schemaregion.view.visitor.TransformToExpressionVisitor; import org.apache.iotdb.db.schemaengine.template.Template; import org.apache.iotdb.db.utils.constant.SqlConstant; import org.apache.iotdb.rpc.RpcUtils; @@ -3113,6 +3117,7 @@ private boolean analyzeTimeseriesRegionScan( Analysis analysis, MPPQueryContext context, PathPatternTree authorityScope, + boolean canSeeSystemDB, boolean includeLogicalView) throws IllegalPathException { analyzeGlobalTimeConditionInShowMetaData(timeCondition, analysis); @@ -3145,6 +3150,9 @@ private boolean analyzeTimeseriesRegionScan( for (DeviceSchemaInfo deviceSchemaInfo : deviceSchemaInfoList) { boolean isAligned = deviceSchemaInfo.isAligned(); PartialPath devicePath = deviceSchemaInfo.getDevicePath(); + if (shouldSkipInternalDatabaseForActiveCount(devicePath, schemaTree, canSeeSystemDB)) { + continue; + } if (isAligned) { List measurementList = new ArrayList<>(); List schemaList = new ArrayList<>(); @@ -3202,6 +3210,15 @@ private boolean analyzeTimeseriesRegionScan( return true; } + private boolean shouldSkipInternalDatabaseForActiveCount( + PartialPath devicePath, ISchemaTree schemaTree, boolean canSeeSystemDB) { + String database = schemaTree.getBelongedDatabase(devicePath); + if (SchemaConstant.SYSTEM_DATABASE.equals(database)) { + return !canSeeSystemDB; + } + return false; + } + private void addLogicalViewSourcesForActiveCount( PartialPath viewDevicePath, IMeasurementSchemaInfo viewSchemaInfo, @@ -3214,6 +3231,8 @@ private void addLogicalViewSourcesForActiveCount( } String viewPath = viewDevicePath.concatNode(viewSchemaInfo.getName()).getFullPath(); + String viewDataType = getLogicalViewDataType(logicalViewSchema, schemaTree); + String viewDatabase = schemaTree.getBelongedDatabase(viewDevicePath); for (PartialPath sourcePath : getSourcePaths(logicalViewSchema.getExpression())) { if (sourcePath.getNodeLength() <= 1) { continue; @@ -3234,16 +3253,39 @@ private void addLogicalViewSourcesForActiveCount( continue; } + Map activeLogicalViewContextMap = + Collections.singletonMap( + viewPath, + new TimeseriesContext( + viewSchemaInfo, viewDataType, viewDatabase, 1, Collections.emptyMap())); addPhysicalTimeseriesForActiveCount( sourceDevicePath, sourceSchemaInfo, sourceDeviceSchemaInfo.isAligned(), - new TimeseriesContext(sourceSchemaInfo, 0, Collections.singleton(viewPath)), + new TimeseriesContext(sourceSchemaInfo, 0, activeLogicalViewContextMap), deviceToTimeseriesContext, deviceSet); } } + private String getLogicalViewDataType( + LogicalViewSchema logicalViewSchema, ISchemaTree schemaTree) { + if (logicalViewSchema.getType() != TSDataType.UNKNOWN) { + return logicalViewSchema.getType().toString(); + } + try { + Expression expression = + new TransformToExpressionVisitor().process(logicalViewSchema.getExpression(), null); + expression = new CompleteMeasurementSchemaVisitor().process(expression, schemaTree); + Map, TSDataType> expressionTypes = new HashMap<>(); + analyzeExpression(expressionTypes, expression); + TSDataType dataType = expressionTypes.get(NodeRef.of(expression)); + return dataType == null ? TSDataType.UNKNOWN.toString() : dataType.toString(); + } catch (Exception e) { + return TSDataType.UNKNOWN.toString(); + } + } + private void addPhysicalTimeseriesForActiveCount( PartialPath devicePath, IMeasurementSchemaInfo measurementSchemaInfo, @@ -3290,7 +3332,8 @@ public Analysis visitShowTimeSeries( analysis, context, showTimeSeriesStatement.getAuthorityScope(), - false); + true, + true); if (!hasSchema) { analysis.setRespDatasetHeader(DatasetHeaderFactory.getShowTimeSeriesHeader()); return analysis; @@ -3535,6 +3578,7 @@ public Analysis visitCountTimeSeries( analysis, context, countTimeSeriesStatement.getAuthorityScope(), + countTimeSeriesStatement.isCanSeeSystemDB(), true); if (!hasSchema) { analysis.setRespDatasetHeader(DatasetHeaderFactory.getCountTimeSeriesHeader()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanBuilder.java index 401e5393aac1e..32d7b31225775 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanBuilder.java @@ -1164,7 +1164,8 @@ public LogicalPlanBuilder planTimeSeriesCountSource( boolean prefixPath, SchemaFilter schemaFilter, Map templateMap, - PathPatternTree scope) { + PathPatternTree scope, + boolean includeSystemDatabase) { this.root = new TimeSeriesCountNode( context.getQueryId().genPlanNodeId(), @@ -1172,7 +1173,8 @@ public LogicalPlanBuilder planTimeSeriesCountSource( prefixPath, schemaFilter, templateMap, - scope); + scope, + includeSystemDatabase); return this; } @@ -1182,7 +1184,8 @@ public LogicalPlanBuilder planLevelTimeSeriesCountSource( int level, SchemaFilter schemaFilter, Map templateMap, - PathPatternTree scope) { + PathPatternTree scope, + boolean includeSystemDatabase) { this.root = new LevelTimeSeriesCountNode( context.getQueryId().genPlanNodeId(), @@ -1191,7 +1194,8 @@ public LogicalPlanBuilder planLevelTimeSeriesCountSource( level, schemaFilter, templateMap, - scope); + scope, + includeSystemDatabase); return this; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java index 69fd1208535d3..5a004d7903869 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java @@ -660,7 +660,8 @@ public PlanNode visitCountTimeSeries( countTimeSeriesStatement.isPrefixPath(), countTimeSeriesStatement.getSchemaFilter(), analysis.getRelatedTemplateInfo(), - countTimeSeriesStatement.getAuthorityScope()) + countTimeSeriesStatement.getAuthorityScope(), + countTimeSeriesStatement.isCanSeeSystemDB()) .planCountMerge() .getRoot(); } @@ -676,7 +677,8 @@ public PlanNode visitCountLevelTimeSeries( countLevelTimeSeriesStatement.getLevel(), countLevelTimeSeriesStatement.getSchemaFilter(), analysis.getRelatedTemplateInfo(), - countLevelTimeSeriesStatement.getAuthorityScope()) + countLevelTimeSeriesStatement.getAuthorityScope(), + countLevelTimeSeriesStatement.isCanSeeSystemDB()) .planCountMerge() .getRoot(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java index c93912c27ebb4..2ec962c82744c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/OperatorTreeGenerator.java @@ -993,7 +993,8 @@ public Operator visitTimeSeriesCount( node.isPrefixPath(), node.getSchemaFilter(), node.getTemplateMap(), - node.getScope())); + node.getScope(), + node.isIncludeSystemDatabase())); } @Override @@ -1015,7 +1016,8 @@ public Operator visitLevelTimeSeriesCount( node.isPrefixPath(), node.getSchemaFilter(), node.getTemplateMap(), - node.getScope())); + node.getScope(), + node.isIncludeSystemDatabase())); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java index 380a351f65ed3..60bb2807668bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java @@ -823,7 +823,10 @@ private List processRegionScan(RegionScanNode node, DistributionPlanCo return planNodeList; } - boolean outputCountInScanNode = node.isOutputCount() && !context.isOneSeriesInMultiRegion(); + boolean outputCountInScanNode = + node.isOutputCount() + && !context.isOneSeriesInMultiRegion() + && !hasActiveLogicalViewContext(node); ActiveRegionScanMergeNode regionMergeNode = new ActiveRegionScanMergeNode( context.queryContext.getQueryId().genPlanNodeId(), @@ -837,6 +840,11 @@ private List processRegionScan(RegionScanNode node, DistributionPlanCo return Collections.singletonList(regionMergeNode); } + private boolean hasActiveLogicalViewContext(RegionScanNode node) { + return node instanceof TimeseriesRegionScanNode + && ((TimeseriesRegionScanNode) node).hasActiveLogicalViewContext(); + } + @Override public List visitDeviceRegionScan( DeviceRegionScanNode node, DistributionPlanContext context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LevelTimeSeriesCountNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LevelTimeSeriesCountNode.java index 2178e06c9e759..c4c0589330088 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LevelTimeSeriesCountNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/LevelTimeSeriesCountNode.java @@ -47,6 +47,7 @@ public class LevelTimeSeriesCountNode extends SchemaQueryScanNode { private final int level; private final SchemaFilter schemaFilter; private final Map templateMap; + private final boolean includeSystemDatabase; public LevelTimeSeriesCountNode( PlanNodeId id, @@ -55,11 +56,13 @@ public LevelTimeSeriesCountNode( int level, SchemaFilter schemaFilter, @NotNull Map templateMap, - @NotNull PathPatternTree scope) { + @NotNull PathPatternTree scope, + boolean includeSystemDatabase) { super(id, partialPath, isPrefixPath, scope); this.level = level; this.schemaFilter = schemaFilter; this.templateMap = templateMap; + this.includeSystemDatabase = includeSystemDatabase; } public SchemaFilter getSchemaFilter() { @@ -74,6 +77,10 @@ public Map getTemplateMap() { return templateMap; } + public boolean isIncludeSystemDatabase() { + return includeSystemDatabase; + } + @Override public PlanNodeType getType() { return PlanNodeType.LEVEL_TIME_SERIES_COUNT; @@ -82,7 +89,14 @@ public PlanNodeType getType() { @Override public PlanNode clone() { return new LevelTimeSeriesCountNode( - getPlanNodeId(), path, isPrefixPath, level, schemaFilter, templateMap, scope); + getPlanNodeId(), + path, + isPrefixPath, + level, + schemaFilter, + templateMap, + scope, + includeSystemDatabase); } @Override @@ -100,6 +114,7 @@ protected void serializeAttributes(ByteBuffer byteBuffer) { ReadWriteIOUtils.write(isPrefixPath, byteBuffer); ReadWriteIOUtils.write(level, byteBuffer); SchemaFilter.serialize(schemaFilter, byteBuffer); + ReadWriteIOUtils.write(includeSystemDatabase, byteBuffer); ReadWriteIOUtils.write(templateMap.size(), byteBuffer); for (Template template : templateMap.values()) { template.serialize(byteBuffer); @@ -114,6 +129,7 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(isPrefixPath, stream); ReadWriteIOUtils.write(level, stream); SchemaFilter.serialize(schemaFilter, stream); + ReadWriteIOUtils.write(includeSystemDatabase, stream); ReadWriteIOUtils.write(templateMap.size(), stream); for (Template template : templateMap.values()) { template.serialize(stream); @@ -132,6 +148,7 @@ public static PlanNode deserialize(ByteBuffer buffer) { boolean isPrefixPath = ReadWriteIOUtils.readBool(buffer); int level = ReadWriteIOUtils.readInt(buffer); SchemaFilter schemaFilter = SchemaFilter.deserialize(buffer); + boolean includeSystemDatabase = ReadWriteIOUtils.readBool(buffer); int templateNum = ReadWriteIOUtils.readInt(buffer); Map templateMap = new HashMap<>(); Template template; @@ -142,7 +159,14 @@ public static PlanNode deserialize(ByteBuffer buffer) { } PlanNodeId planNodeId = PlanNodeId.deserialize(buffer); return new LevelTimeSeriesCountNode( - planNodeId, path, isPrefixPath, level, schemaFilter, templateMap, scope); + planNodeId, + path, + isPrefixPath, + level, + schemaFilter, + templateMap, + scope, + includeSystemDatabase); } @Override @@ -157,12 +181,12 @@ public boolean equals(Object o) { return false; } LevelTimeSeriesCountNode that = (LevelTimeSeriesCountNode) o; - return level == that.level; + return level == that.level && includeSystemDatabase == that.includeSystemDatabase; } @Override public int hashCode() { - return Objects.hash(super.hashCode(), level); + return Objects.hash(super.hashCode(), level, includeSystemDatabase); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesCountNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesCountNode.java index 473d59b8d1532..6f7483132a31a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesCountNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metedata/read/TimeSeriesCountNode.java @@ -49,16 +49,20 @@ public class TimeSeriesCountNode extends SchemaQueryScanNode { private final Map templateMap; + private final boolean includeSystemDatabase; + public TimeSeriesCountNode( PlanNodeId id, PartialPath partialPath, boolean isPrefixPath, SchemaFilter schemaFilter, @NotNull Map templateMap, - @NotNull PathPatternTree scope) { + @NotNull PathPatternTree scope, + boolean includeSystemDatabase) { super(id, partialPath, isPrefixPath, scope); this.schemaFilter = schemaFilter; this.templateMap = templateMap; + this.includeSystemDatabase = includeSystemDatabase; } public SchemaFilter getSchemaFilter() { @@ -69,6 +73,10 @@ public Map getTemplateMap() { return templateMap; } + public boolean isIncludeSystemDatabase() { + return includeSystemDatabase; + } + @Override public PlanNodeType getType() { return PlanNodeType.TIME_SERIES_COUNT; @@ -77,7 +85,13 @@ public PlanNodeType getType() { @Override public PlanNode clone() { return new TimeSeriesCountNode( - getPlanNodeId(), path, isPrefixPath, schemaFilter, templateMap, scope); + getPlanNodeId(), + path, + isPrefixPath, + schemaFilter, + templateMap, + scope, + includeSystemDatabase); } @Override @@ -94,6 +108,7 @@ protected void serializeAttributes(ByteBuffer byteBuffer) { scope.serialize(byteBuffer); ReadWriteIOUtils.write(isPrefixPath, byteBuffer); SchemaFilter.serialize(schemaFilter, byteBuffer); + ReadWriteIOUtils.write(includeSystemDatabase, byteBuffer); ReadWriteIOUtils.write(templateMap.size(), byteBuffer); for (Template template : templateMap.values()) { template.serialize(byteBuffer); @@ -107,6 +122,7 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { scope.serialize(stream); ReadWriteIOUtils.write(isPrefixPath, stream); SchemaFilter.serialize(schemaFilter, stream); + ReadWriteIOUtils.write(includeSystemDatabase, stream); ReadWriteIOUtils.write(templateMap.size(), stream); for (Template template : templateMap.values()) { template.serialize(stream); @@ -124,6 +140,7 @@ public static PlanNode deserialize(ByteBuffer buffer) { PathPatternTree scope = PathPatternTree.deserialize(buffer); boolean isPrefixPath = ReadWriteIOUtils.readBool(buffer); SchemaFilter schemaFilter = SchemaFilter.deserialize(buffer); + boolean includeSystemDatabase = ReadWriteIOUtils.readBool(buffer); int templateNum = ReadWriteIOUtils.readInt(buffer); Map templateMap = new HashMap<>(); @@ -136,7 +153,7 @@ public static PlanNode deserialize(ByteBuffer buffer) { PlanNodeId planNodeId = PlanNodeId.deserialize(buffer); return new TimeSeriesCountNode( - planNodeId, path, isPrefixPath, schemaFilter, templateMap, scope); + planNodeId, path, isPrefixPath, schemaFilter, templateMap, scope, includeSystemDatabase); } @Override @@ -153,11 +170,12 @@ public boolean equals(Object o) { if (!super.equals(o)) return false; TimeSeriesCountNode that = (TimeSeriesCountNode) o; return Objects.equals(schemaFilter, that.schemaFilter) - && Objects.equals(templateMap, that.templateMap); + && Objects.equals(templateMap, that.templateMap) + && includeSystemDatabase == that.includeSystemDatabase; } @Override public int hashCode() { - return Objects.hash(super.hashCode(), schemaFilter, templateMap); + return Objects.hash(super.hashCode(), schemaFilter, templateMap, includeSystemDatabase); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/source/TimeseriesRegionScanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/source/TimeseriesRegionScanNode.java index 5a5abed1aa020..d905c8eca2c4e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/source/TimeseriesRegionScanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/source/TimeseriesRegionScanNode.java @@ -85,6 +85,13 @@ public void setDeviceToTimeseriesSchemaInfo( return deviceToTimeseriesSchemaInfo; } + public boolean hasActiveLogicalViewContext() { + return deviceToTimeseriesSchemaInfo.values().stream() + .flatMap(timeseriesContextMap -> timeseriesContextMap.values().stream()) + .flatMap(List::stream) + .anyMatch(context -> !context.getActiveLogicalViewContextMap().isEmpty()); + } + @Override public List getChildren() { return ImmutableList.of(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountStatement.java index 529a8660dfbfe..6ea44f6c11397 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountStatement.java @@ -19,9 +19,17 @@ package org.apache.iotdb.db.queryengine.plan.statement.metadata; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.auth.entity.PrivilegeType; +import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.schema.SchemaConstant; +import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.queryengine.plan.statement.StatementType; +import org.apache.iotdb.rpc.TSStatusCode; +import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -34,6 +42,7 @@ */ public class CountStatement extends ShowStatement { protected PartialPath pathPattern; + private boolean canSeeSystemDB = true; public CountStatement(PartialPath pathPattern) { this.pathPattern = pathPattern; @@ -52,4 +61,76 @@ public void setPathPattern(PartialPath pathPattern) { public List getPaths() { return Collections.singletonList(pathPattern); } + + @Override + public TSStatus checkPermissionBeforeProcess(String userName) { + if (AuthorityChecker.SUPER_USER.equals(userName)) { + setCanSeeSystemDB(true); + return AuthorityChecker.SUCCEED; + } + TSStatus explicitInternalDatabaseStatus = checkExplicitInternalDatabase(userName); + if (explicitInternalDatabaseStatus != null) { + return explicitInternalDatabaseStatus; + } + setCanSeeSystemDB(userName); + TSStatus status = super.checkPermissionBeforeProcess(userName); + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + appendInternalDatabaseAuthorityScope(); + } + return status; + } + + public boolean isCanSeeSystemDB() { + return canSeeSystemDB; + } + + public void setCanSeeSystemDB(boolean canSeeSystemDB) { + this.canSeeSystemDB = canSeeSystemDB; + } + + protected void setCanSeeSystemDB(String userName) { + setCanSeeSystemDB( + AuthorityChecker.checkSystemPermission(userName, PrivilegeType.MAINTAIN.ordinal())); + } + + protected TSStatus checkExplicitInternalDatabase(String userName) { + if (!isExplicitSystemDatabasePath()) { + return null; + } + if (!AuthorityChecker.checkSystemPermission(userName, PrivilegeType.MAINTAIN.ordinal())) { + return AuthorityChecker.getTSStatus(false, PrivilegeType.MAINTAIN); + } + setCanSeeSystemDB(true); + authorityScope = createAuthorityScope(pathPattern); + return AuthorityChecker.SUCCEED; + } + + protected void appendInternalDatabaseAuthorityScope() { + if (!canSeeSystemDB || SchemaConstant.ALL_MATCH_SCOPE.equals(authorityScope)) { + return; + } + authorityScope.appendPathPattern(createInternalDatabasePathPattern()); + authorityScope.constructTree(); + } + + private boolean isExplicitSystemDatabasePath() { + String[] nodes = pathPattern.getNodes(); + return nodes.length >= 2 + && SchemaConstant.ROOT.equals(nodes[0]) + && SchemaConstant.SYSTEM_DATABASE.equals(SchemaConstant.ROOT + "." + nodes[1]); + } + + private PartialPath createInternalDatabasePathPattern() { + String[] databaseNodes = SchemaConstant.SYSTEM_DATABASE.split("\\."); + String[] pathPatternNodes = Arrays.copyOf(databaseNodes, databaseNodes.length + 1); + pathPatternNodes[databaseNodes.length] = IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD; + return new PartialPath(pathPatternNodes); + } + + private PathPatternTree createAuthorityScope(PartialPath pathPattern) { + PathPatternTree authorityScope = new PathPatternTree(); + authorityScope.appendPathPattern(pathPattern); + authorityScope.constructTree(); + return authorityScope; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountTimeSeriesStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountTimeSeriesStatement.java index c0910185421bd..870c0cf8cb8c9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountTimeSeriesStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/CountTimeSeriesStatement.java @@ -62,22 +62,27 @@ public WhereCondition getTimeCondition() { @Override public TSStatus checkPermissionBeforeProcess(String userName) { - if (hasTimeCondition()) { - try { - if (!AuthorityChecker.SUPER_USER.equals(userName)) { - this.authorityScope = - PathPatternTreeUtils.intersectWithFullPathPrefixTree( - AuthorityChecker.getAuthorizedPathTree( - userName, PrivilegeType.READ_SCHEMA.ordinal()), - AuthorityChecker.getAuthorizedPathTree( - userName, PrivilegeType.READ_DATA.ordinal())); - } - } catch (AuthException e) { - return new TSStatus(e.getCode().getStatusCode()); + if (!hasTimeCondition()) { + return super.checkPermissionBeforeProcess(userName); + } + try { + if (AuthorityChecker.SUPER_USER.equals(userName)) { + setCanSeeSystemDB(true); + return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } + TSStatus explicitInternalDatabaseStatus = checkExplicitInternalDatabase(userName); + if (explicitInternalDatabaseStatus != null) { + return explicitInternalDatabaseStatus; + } + setCanSeeSystemDB(userName); + this.authorityScope = + PathPatternTreeUtils.intersectWithFullPathPrefixTree( + AuthorityChecker.getAuthorizedPathTree(userName, PrivilegeType.READ_SCHEMA.ordinal()), + AuthorityChecker.getAuthorizedPathTree(userName, PrivilegeType.READ_DATA.ordinal())); + appendInternalDatabaseAuthorityScope(); return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); - } else { - return super.checkPermissionBeforeProcess(userName); + } catch (AuthException e) { + return new TSStatus(e.getCode().getStatusCode()); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/auth/AuthorityCheckerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/auth/AuthorityCheckerTest.java index ca730c377b763..2861059808a5f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/auth/AuthorityCheckerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/auth/AuthorityCheckerTest.java @@ -20,18 +20,36 @@ package org.apache.iotdb.db.auth; import org.apache.iotdb.commons.auth.entity.PrivilegeType; +import org.apache.iotdb.commons.auth.entity.User; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountLevelTimeSeriesStatement; +import org.apache.iotdb.db.queryengine.plan.statement.metadata.CountTimeSeriesStatement; +import org.apache.iotdb.rpc.TSStatusCode; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.util.Arrays; +import java.util.Collections; public class AuthorityCheckerTest { + @Before + public void setup() { + AuthorityChecker.getAuthorityFetcher().getAuthorCache().invalidAllCache(); + } + + @After + public void teardown() { + AuthorityChecker.getAuthorityFetcher().getAuthorCache().invalidAllCache(); + } + @Test public void testLogReduce() throws IllegalPathException { final CommonConfig config = CommonDescriptor.getInstance().getConfig(); @@ -48,4 +66,60 @@ public void testLogReduce() throws IllegalPathException { .getMessage()); config.setPathLogMaxSize(oldSize); } + + @Test + public void testCountTimeSeriesExplicitSystemDatabasePermission() throws Exception { + User user = new User("user1", "password"); + AuthorityChecker.getAuthorityFetcher().getAuthorCache().putUserCache(user.getName(), user); + + CountTimeSeriesStatement systemStatement = + new CountTimeSeriesStatement(new PartialPath("root.__system.**")); + Assert.assertEquals( + TSStatusCode.NO_PERMISSION.getStatusCode(), + systemStatement.checkPermissionBeforeProcess(user.getName()).getCode()); + + user.addSysPrivilege(PrivilegeType.MAINTAIN.ordinal()); + systemStatement = new CountTimeSeriesStatement(new PartialPath("root.__system.**")); + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), + systemStatement.checkPermissionBeforeProcess(user.getName()).getCode()); + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.__system.**")), + systemStatement.getAuthorityScope().getAllPathPatterns()); + Assert.assertTrue(systemStatement.isCanSeeSystemDB()); + + CountLevelTimeSeriesStatement systemLevelStatement = + new CountLevelTimeSeriesStatement(new PartialPath("root.__system.**"), 1); + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), + systemLevelStatement.checkPermissionBeforeProcess(user.getName()).getCode()); + Assert.assertEquals( + Collections.singletonList(new PartialPath("root.__system.**")), + systemLevelStatement.getAuthorityScope().getAllPathPatterns()); + Assert.assertTrue(systemLevelStatement.isCanSeeSystemDB()); + } + + @Test + public void testCountTimeSeriesImplicitSystemDatabasePermission() throws Exception { + User user = new User("user2", "password"); + AuthorityChecker.getAuthorityFetcher().getAuthorCache().putUserCache(user.getName(), user); + + CountTimeSeriesStatement statement = new CountTimeSeriesStatement(new PartialPath("root.**")); + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), + statement.checkPermissionBeforeProcess(user.getName()).getCode()); + Assert.assertFalse(statement.isCanSeeSystemDB()); + + user.addSysPrivilege(PrivilegeType.MAINTAIN.ordinal()); + statement = new CountTimeSeriesStatement(new PartialPath("root.**")); + Assert.assertEquals( + TSStatusCode.SUCCESS_STATUS.getStatusCode(), + statement.checkPermissionBeforeProcess(user.getName()).getCode()); + Assert.assertTrue(statement.isCanSeeSystemDB()); + Assert.assertTrue( + statement + .getAuthorityScope() + .getAllPathPatterns() + .contains(new PartialPath("root.__system.**"))); + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSourceTest.java index b7becf649c970..84acab189f43b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/operator/schema/source/TimeSeriesSchemaSourceTest.java @@ -37,14 +37,15 @@ public class TimeSeriesSchemaSourceTest { @Test - public void testCountSourceSkipsImplicitInternalDatabases() throws Exception { + public void testCountSourceSkipsUnauthorizedInternalDatabases() throws Exception { final ISchemaSource countSource = SchemaSourceFactory.getTimeSeriesSchemaCountSource( new PartialPath("root.**"), false, null, Collections.emptyMap(), - SchemaConstant.ALL_MATCH_SCOPE); + SchemaConstant.ALL_MATCH_SCOPE, + false); assertTrue( countSource.shouldSkipSchemaRegion(mockSchemaRegion(SchemaConstant.SYSTEM_DATABASE))); @@ -52,27 +53,30 @@ public void testCountSourceSkipsImplicitInternalDatabases() throws Exception { } @Test - public void testCountSourceKeepsExplicitInternalDatabaseQueries() throws Exception { + public void testCountSourceKeepsAuthorizedInternalDatabases() throws Exception { final ISchemaSource systemCountSource = SchemaSourceFactory.getTimeSeriesSchemaCountSource( - new PartialPath("root.__system.**"), + new PartialPath("root.**"), false, null, Collections.emptyMap(), - SchemaConstant.ALL_MATCH_SCOPE); + SchemaConstant.ALL_MATCH_SCOPE, + true); assertFalse( systemCountSource.shouldSkipSchemaRegion(mockSchemaRegion(SchemaConstant.SYSTEM_DATABASE))); } @Test - public void testCountSourceSkipsWildcardSecondNodeForInternalDatabases() throws Exception { + public void testCountSourceSkipsUnauthorizedInternalDatabasesWithWildcardSecondNode() + throws Exception { final ISchemaSource countSource = SchemaSourceFactory.getTimeSeriesSchemaCountSource( new PartialPath("root.*.**"), false, null, Collections.emptyMap(), - SchemaConstant.ALL_MATCH_SCOPE); + SchemaConstant.ALL_MATCH_SCOPE, + false); assertTrue( countSource.shouldSkipSchemaRegion(mockSchemaRegion(SchemaConstant.SYSTEM_DATABASE))); @@ -80,14 +84,15 @@ public void testCountSourceSkipsWildcardSecondNodeForInternalDatabases() throws } @Test - public void testCountSourceKeepsExactInternalDatabaseQueries() throws Exception { + public void testCountSourceKeepsExactInternalDatabaseQueriesWithPrivilege() throws Exception { final ISchemaSource systemCountSource = SchemaSourceFactory.getTimeSeriesSchemaCountSource( new PartialPath("root.__system"), false, null, Collections.emptyMap(), - SchemaConstant.ALL_MATCH_SCOPE); + SchemaConstant.ALL_MATCH_SCOPE, + true); assertFalse( systemCountSource.shouldSkipSchemaRegion(mockSchemaRegion(SchemaConstant.SYSTEM_DATABASE))); } @@ -116,7 +121,8 @@ public void testCountStatisticIncludesView() throws Exception { false, null, Collections.emptyMap(), - SchemaConstant.ALL_MATCH_SCOPE); + SchemaConstant.ALL_MATCH_SCOPE, + false); final ISchemaRegion schemaRegion = mockSchemaRegion("root.sg"); final ISchemaRegionStatistics schemaRegionStatistics = Mockito.mock(ISchemaRegionStatistics.class); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/RegionScanPlanningTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/RegionScanPlanningTest.java index 7a5eccd87a599..fba26ceea8057 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/RegionScanPlanningTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/RegionScanPlanningTest.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.common.TimeseriesContext; import org.apache.iotdb.db.queryengine.plan.analyze.Analysis; import org.apache.iotdb.db.queryengine.plan.planner.plan.DistributedQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; @@ -37,10 +38,15 @@ import org.apache.tsfile.file.metadata.PlainDeviceID; import org.junit.Test; +import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Set; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class RegionScanPlanningTest { @@ -146,4 +152,79 @@ public void testShowTimeseriesWithTimeCondition() throws IllegalPathException { assertEquals(devicePaths, targetDevicePaths); assertEquals(path, targetMeasurementPaths); } + + @Test + public void testCountTimeseriesWithLogicalViewUsesMergeBeforeCount() throws IllegalPathException { + QueryId queryId = new QueryId("test"); + MPPQueryContext context = + new MPPQueryContext("", queryId, null, new TEndPoint(), new TEndPoint()); + + Map logicalViewContextMap = + Collections.singletonMap( + "root.sg.view.v1", + new TimeseriesContext( + "INT32", + null, + null, + null, + null, + null, + null, + null, + 1, + true, + "root.sg", + Collections.emptyMap())); + Map>> deviceToTimeseriesSchemaInfo = + new HashMap<>(); + deviceToTimeseriesSchemaInfo.put( + new PartialPath("root.sg.d22"), + Collections.singletonMap( + new MeasurementPath("root.sg.d22.s1"), + Collections.singletonList( + new TimeseriesContext( + "INT32", + null, + "PLAIN", + "LZ4", + null, + null, + null, + null, + 0, + logicalViewContextMap)))); + deviceToTimeseriesSchemaInfo.put( + new PartialPath("root.sg.d55555"), + Collections.singletonMap( + new MeasurementPath("root.sg.d55555.s1"), + Collections.singletonList( + new TimeseriesContext( + "INT32", + null, + "PLAIN", + "LZ4", + null, + null, + null, + null, + 0, + logicalViewContextMap)))); + + TimeseriesRegionScanNode regionScanNode = + new TimeseriesRegionScanNode( + queryId.genPlanNodeId(), deviceToTimeseriesSchemaInfo, true, null); + PlanNode rewrittenRoot = + new DistributionPlanner(Util.ANALYSIS, new LogicalQueryPlan(context, regionScanNode)) + .rewriteSource(); + + assertTrue(rewrittenRoot instanceof ActiveRegionScanMergeNode); + ActiveRegionScanMergeNode mergeNode = (ActiveRegionScanMergeNode) rewrittenRoot; + assertTrue(mergeNode.isOutputCount()); + assertTrue(mergeNode.isNeedMerge()); + assertEquals(2, mergeNode.getChildren().size()); + for (PlanNode child : mergeNode.getChildren()) { + assertTrue(child instanceof TimeseriesRegionScanNode); + assertFalse(((TimeseriesRegionScanNode) child).isOutputCount()); + } + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/RegionScanLogicalPlannerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/RegionScanLogicalPlannerTest.java index 2b2d169cafa51..ed9b57a5b9d80 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/RegionScanLogicalPlannerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/logical/RegionScanLogicalPlannerTest.java @@ -287,4 +287,51 @@ public void serializeDeserializeTest() throws IllegalPathException { buffer.flip(); Assert.assertEquals(timeseriesRegionScanNode, PlanNodeType.deserialize(buffer)); } + + @Test + public void serializeDeserializeLogicalViewContextTest() throws IllegalPathException { + Map logicalViewContextMap = + Collections.singletonMap( + "root.sg.view.v1", + new TimeseriesContext( + "INT32", + null, + null, + null, + null, + null, + null, + null, + 1, + true, + "root.sg", + new HashMap<>())); + Map> timeseriesSchemaInfoMap = new HashMap<>(); + timeseriesSchemaInfoMap.put( + new MeasurementPath("root.sg.d3.s1", TSDataType.INT32), + Collections.singletonList( + new TimeseriesContext( + "INT32", + null, + TSFileDescriptor.getInstance().getConfig().getValueEncoder(TSDataType.INT32), + "LZ4", + null, + null, + null, + null, + 0, + logicalViewContextMap))); + Map>> deviceToTimeseriesSchemaInfo = + new HashMap<>(); + deviceToTimeseriesSchemaInfo.put(new PartialPath("root.sg.d3"), timeseriesSchemaInfoMap); + + TimeseriesRegionScanNode timeseriesRegionScanNode = + new TimeseriesRegionScanNode( + new PlanNodeId("timeseries_test_id"), deviceToTimeseriesSchemaInfo, false, null); + + ByteBuffer buffer = ByteBuffer.allocate(10240); + timeseriesRegionScanNode.serialize(buffer); + buffer.flip(); + Assert.assertEquals(timeseriesRegionScanNode, PlanNodeType.deserialize(buffer)); + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/metadata/read/SchemaCountNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/metadata/read/SchemaCountNodeSerdeTest.java index 0139371e26bd8..fb78ea46a2bb5 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/metadata/read/SchemaCountNodeSerdeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/metadata/read/SchemaCountNodeSerdeTest.java @@ -87,7 +87,8 @@ public void testLevelTimeSeriesCountSerializeAndDeserialize() throws IllegalPath 10, null, Collections.emptyMap(), - SchemaConstant.ALL_MATCH_SCOPE); + SchemaConstant.ALL_MATCH_SCOPE, + true); IdentitySinkNode sinkNode = new IdentitySinkNode( new PlanNodeId("sink"), @@ -122,7 +123,8 @@ public void testTimeSeriesCountSerializeAndDeserialize() throws IllegalPathExcep true, null, Collections.emptyMap(), - SchemaConstant.ALL_MATCH_SCOPE); + SchemaConstant.ALL_MATCH_SCOPE, + true); IdentitySinkNode sinkNode = new IdentitySinkNode( new PlanNodeId("sink"), From 78a4790efb1b21d5e694662420fdd3aa65fc7eb2 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 8 Jun 2026 12:22:54 +0800 Subject: [PATCH 090/102] Fix PBTree schema cache release after pre-delete (#17834) (#17852) --- .../pbtree/memory/ReleaseFlushMonitor.java | 17 ++++++ .../mtree/traverser/Traverser.java | 1 + .../schemaRegion/SchemaStatisticsTest.java | 61 ++++--------------- 3 files changed, 29 insertions(+), 50 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/memory/ReleaseFlushMonitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/memory/ReleaseFlushMonitor.java index 4c3ec8d2c9de5..0aaa72899a46b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/memory/ReleaseFlushMonitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/impl/pbtree/memory/ReleaseFlushMonitor.java @@ -230,6 +230,7 @@ public List getRegionsToFlush(long windowsEndTime) { public void forceFlushAndRelease() { boolean needFlush; while (true) { + waitUntilWorkerTasksDone(); needFlush = false; for (CachedMTreeStore store : regionToStoreMap.values()) { if (store.getMemoryManager().getBufferNodeNum() > 0) { @@ -239,13 +240,29 @@ public void forceFlushAndRelease() { } if (needFlush) { scheduler.scheduleFlushAll().join(); + waitUntilWorkerTasksDone(); scheduler.scheduleRelease(true); } else { + // No volatile nodes left, but clean unpinned cache may still remain after previous flushes. + scheduler.scheduleRelease(true); + waitUntilWorkerTasksDone(); break; } } } + @TestOnly + private void waitUntilWorkerTasksDone() { + while (scheduler.getActiveWorkerNum() > 0 || !flushingRegionSet.isEmpty()) { + try { + Thread.sleep(10); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return; + } + } + } + public void clear() { if (releaseMonitor != null) { releaseMonitor.shutdownNow(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java index 6c9e23e9597ed..00fdb1b81b4bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/schemaregion/mtree/traverser/Traverser.java @@ -185,6 +185,7 @@ private N getChild(N parent, String childName, boolean skipTemplateChildren) && skipPreDeletedSchema && child.isMeasurement() && child.getAsMeasurementMNode().isPreDeleted()) { + releaseNode(child); child = null; } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaStatisticsTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaStatisticsTest.java index 04643a88c41e5..40aade6a63b95 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaStatisticsTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/SchemaStatisticsTest.java @@ -75,24 +75,12 @@ public void testPBTreeMemoryStatistics() throws Exception { || testParams.getTestModeName().equals("PBTree-NonMemory")) { final IMNodeFactory nodeFactory = MNodeFactoryLoader.getInstance().getCachedMNodeIMNodeFactory(); - // wait release and flush task - Thread.sleep(6000); + ReleaseFlushMonitor.getInstance().forceFlushAndRelease(); // schemaRegion1 final IMNode sg1 = nodeFactory.createDatabaseMNode(null, "sg1"); sg1.setFullPath("root.sg1"); final long size1 = sg1.estimateSize(); - if (size1 != schemaRegion1.getSchemaRegionStatistics().getRegionMemoryUsage()) { - // There are two possibilities here in PartialMemory mode: - // 1. only the "sg1" node remains - // 2. the "sg1" node and the "n" node remain - Assert.assertEquals("PBTree-PartialMemory", testParams.getTestModeName()); - Assert.assertEquals( - size1 + nodeFactory.createDeviceMNode(sg1.getAsMNode(), "n").estimateSize(), - schemaRegion1.getSchemaRegionStatistics().getRegionMemoryUsage()); - ReleaseFlushMonitor.getInstance().forceFlushAndRelease(); - Assert.assertEquals( - size1, schemaRegion1.getSchemaRegionStatistics().getRegionMemoryUsage()); - } + Assert.assertEquals(size1, schemaRegion1.getSchemaRegionStatistics().getRegionMemoryUsage()); } Assert.assertEquals(0, schemaRegion1.getSchemaRegionStatistics().getSchemaRegionId()); checkPBTreeStatistics(engineStatistics); @@ -122,8 +110,7 @@ public void testMemoryStatistics2() throws Exception { final IMNodeFactory nodeFactory = MNodeFactoryLoader.getInstance().getCachedMNodeIMNodeFactory(); - // wait release and flush task - Thread.sleep(1000); + ReleaseFlushMonitor.getInstance().forceFlushAndRelease(); // schemaRegion1 final IMNode sg1 = nodeFactory.createDatabaseMNode(null, "sg1"); sg1.setFullPath("root.sg1"); @@ -231,36 +218,12 @@ public void testMemoryStatistics() throws Exception { || testParams.getTestModeName().equals("PBTree-NonMemory")) { final IMNodeFactory nodeFactory = MNodeFactoryLoader.getInstance().getCachedMNodeIMNodeFactory(); - // wait release and flush task - Thread.sleep(1000); + ReleaseFlushMonitor.getInstance().forceFlushAndRelease(); // schemaRegion1 final IMNode sg1 = nodeFactory.createDatabaseDeviceMNode(null, "sg1"); sg1.setFullPath("root.sg1"); final long size1 = sg1.estimateSize(); - if (sg1.estimateSize() != schemaRegion1.getSchemaRegionStatistics().getRegionMemoryUsage()) { - // "d0" or "d1" node may remain in PartialMemory mode - Assert.assertEquals("PBTree-PartialMemory", testParams.getTestModeName()); - final long d0ExistSize = - size1 - + nodeFactory - .createMeasurementMNode( - sg1.getAsDeviceMNode(), - "d0", - new MeasurementSchema( - "d0", TSDataType.INT64, TSEncoding.PLAIN, CompressionType.SNAPPY), - null) - .estimateSize(); - final long d1ExistSize = - size1 + nodeFactory.createInternalMNode(sg1.getAsMNode(), "d1").estimateSize(); - Assert.assertTrue( - d0ExistSize == schemaRegion1.getSchemaRegionStatistics().getRegionMemoryUsage() - || d1ExistSize == schemaRegion1.getSchemaRegionStatistics().getRegionMemoryUsage()); - ReleaseFlushMonitor.getInstance().forceFlushAndRelease(); - // wait release and flush task - Thread.sleep(1000); - Assert.assertEquals( - size1, schemaRegion1.getSchemaRegionStatistics().getRegionMemoryUsage()); - } + Assert.assertEquals(size1, schemaRegion1.getSchemaRegionStatistics().getRegionMemoryUsage()); // schemaRegion2 final IMNode sg2 = nodeFactory.createDatabaseMNode(null, "sg2"); sg2.setFullPath("root.sg2"); @@ -420,7 +383,11 @@ public void testPBTreeNodeStatistics() throws Exception { schemaRegion1.deleteTimeseriesInBlackList(patternTree); schemaRegion2.deleteTimeseriesInBlackList(patternTree); - Thread.sleep(1000); + if (testParams.getCachedMNodeSize() <= 3) { + ReleaseFlushMonitor.getInstance().forceFlushAndRelease(); + } else { + Thread.sleep(1000); + } final CachedSchemaRegionStatistics cachedRegionStatistics1 = schemaRegion1.getSchemaRegionStatistics().getAsCachedSchemaRegionStatistics(); final CachedSchemaRegionStatistics cachedRegionStatistics2 = @@ -433,13 +400,7 @@ public void testPBTreeNodeStatistics() throws Exception { Assert.assertEquals(4, cachedRegionStatistics2.getUnpinnedMNodeNum()); } else { Assert.assertEquals(1, cachedRegionStatistics1.getPinnedMNodeNum()); - if (0 != cachedRegionStatistics1.getUnpinnedMNodeNum()) { - // "d0" may remain in PartialMemory mode - Assert.assertEquals("PBTree-PartialMemory", testParams.getTestModeName()); - ReleaseFlushMonitor.getInstance().forceFlushAndRelease(); - Thread.sleep(1000); - Assert.assertEquals(0, cachedRegionStatistics1.getUnpinnedMNodeNum()); - } + Assert.assertEquals(0, cachedRegionStatistics1.getUnpinnedMNodeNum()); Assert.assertEquals(1, cachedRegionStatistics2.getPinnedMNodeNum()); Assert.assertEquals(0, cachedRegionStatistics2.getUnpinnedMNodeNum()); } From 6348c2db2a04973d203659380b849efabbffb0f2 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 9 Jun 2026 15:14:42 +0800 Subject: [PATCH 091/102] [to dev/1.3] Pipe: backport auto split fixes (#17866) * Pipe: Fixed the bug that separated historical pipe may not include mod on deletion & The pipe without data.insert may be wrongly separated by pipe and transfer data (#17346) * fix * fix * f (cherry picked from commit 355a872cfaeb77e1e7fe68d78a90544b3054d3a5) * Pipe: Made the historical pipe split auto dropped after completion (#17295) * snapshot * may-comp * auto (cherry picked from commit 9bfe0b0a78eec2f55b0d3f5388669f488b0e4a4b) --- .../it/autocreate/IoTDBPipeAutoDropIT.java | 92 ++++++++++--------- .../it/autocreate/IoTDBPipeAutoSplitIT.java | 55 ++++++++++- .../agent/task/PipeDataNodeTaskAgent.java | 7 +- .../executor/ClusterConfigTaskExecutor.java | 62 +++++++------ 4 files changed, 138 insertions(+), 78 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoDropIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoDropIT.java index dc1d1fc93aec3..09657e2deb2db 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoDropIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoDropIT.java @@ -21,8 +21,10 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStaticMeta; import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq; import org.apache.iotdb.db.it.utils.TestUtils; +import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant; import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.it.framework.IoTDBTestRunner; import org.apache.iotdb.itbase.category.MultiClusterIT2AutoCreateSchema; @@ -33,59 +35,67 @@ import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.awaitility.Awaitility.await; @RunWith(IoTDBTestRunner.class) @Category({MultiClusterIT2AutoCreateSchema.class}) public class IoTDBPipeAutoDropIT extends AbstractPipeDualAutoIT { + @Override + protected void setupConfig() { + super.setupConfig(); + senderEnv.getConfig().getCommonConfig().setPipeAutoSplitFullEnabled(true); + receiverEnv.getConfig().getCommonConfig().setPipeAutoSplitFullEnabled(true); + senderEnv.getConfig().getConfigNodeConfig().setLeaderDistributionPolicy("HASH"); + } + @Test public void testAutoDropInHistoricalTransfer() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); - final String receiverIp = receiverDataNode.getIp(); - final int receiverPort = receiverDataNode.getPort(); - - try (final SyncConfigNodeIServiceClient client = - (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { - - TestUtils.executeNonQuery(senderEnv, "insert into root.db.d1(time,s1) values (1,1)", null); - - final Map extractorAttributes = new HashMap<>(); - final Map processorAttributes = new HashMap<>(); - final Map connectorAttributes = new HashMap<>(); - - extractorAttributes.put("extractor.mode", "query"); - - connectorAttributes.put("connector", "iotdb-thrift-connector"); - connectorAttributes.put("connector.batch.enable", "false"); - connectorAttributes.put("connector.ip", receiverIp); - connectorAttributes.put("connector.port", Integer.toString(receiverPort)); - - final TSStatus status = - client.createPipe( - new TCreatePipeReq("p1", connectorAttributes) - .setExtractorAttributes(extractorAttributes) - .setProcessorAttributes(processorAttributes)); - - Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode()); - Assert.assertEquals( - TSStatusCode.SUCCESS_STATUS.getStatusCode(), client.startPipe("p1").getCode()); - - TestUtils.assertDataEventuallyOnEnv( - receiverEnv, - "select count(*) from root.**", - "count(root.db.d1.s1),", - Collections.singleton("1,")); - - TestUtils.assertDataEventuallyOnEnv( - senderEnv, - "show pipes", - "ID,CreationTime,State,PipeSource,PipeProcessor,PipeSink,ExceptionMessage,RemainingEventCount,EstimatedRemainingSeconds,", - Collections.emptySet()); - } + TestUtils.executeNonQuery( + senderEnv, + String.format( + "create pipe a2b with sink ('node-urls'='%s')", receiverDataNode.getIpAndPortString()), + null); + + TestUtils.executeNonQuery(senderEnv, "insert into root.db.d1(time,s1) values (1,1)", null); + + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select count(*) from root.**", + "count(root.db.d1.s1),", + Collections.singleton("1,")); + + await() + .pollInSameThread() + .pollDelay(1L, TimeUnit.SECONDS) + .pollInterval(1L, TimeUnit.SECONDS) + .atMost(600, TimeUnit.SECONDS) + .untilAsserted( + () -> { + try (final Connection connection = senderEnv.getConnection(); + final Statement statement = connection.createStatement(); + final ResultSet result = statement.executeQuery("show pipes")) { + int pipeNum = 0; + while (result.next()) { + final String pipeName = result.getString(ColumnHeaderConstant.ID); + if (!pipeName.contains(PipeStaticMeta.CONSENSUS_PIPE_PREFIX) + && pipeName.endsWith("_history")) { + pipeNum++; + } + } + Assert.assertEquals(0, pipeNum); + } + }); } @Test diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoSplitIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoSplitIT.java index 5f732b6d390c5..ced1e9ee13ce5 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoSplitIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoSplitIT.java @@ -23,6 +23,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowPipeInfo; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; import org.apache.iotdb.consensus.ConsensusFactory; +import org.apache.iotdb.db.it.utils.TestUtils; import org.apache.iotdb.it.env.MultiEnvFactory; import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.it.framework.IoTDBTestRunner; @@ -37,6 +38,8 @@ import java.sql.Connection; import java.sql.SQLException; import java.sql.Statement; +import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Objects; @@ -78,13 +81,12 @@ public void setUp() { public void testSingleEnv() throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); - final String sql = - String.format( - "create pipe a2b with source ('source'='iotdb-source') with processor ('processor'='do-nothing-processor') with sink ('node-urls'='%s')", - receiverDataNode.getIpAndPortString()); try (final Connection connection = senderEnv.getConnection(); final Statement statement = connection.createStatement()) { - statement.execute(sql); + statement.execute( + String.format( + "create pipe a2b with sink ('node-urls'='%s')", + receiverDataNode.getIpAndPortString())); } catch (final SQLException e) { fail(e.getMessage()); } @@ -100,5 +102,48 @@ public void testSingleEnv() throws Exception { || (Objects.equals(showPipeResult.get(1).id, "a2b_history") && Objects.equals(showPipeResult.get(0).id, "a2b_realtime"))); } + + // Do not split for pipes without insertion or non-full + TestUtils.executeNonQueries( + senderEnv, + Arrays.asList( + "drop pipe a2b_history", + "drop pipe a2b_realtime", + String.format( + "create pipe a2b1 with source ('inclusion'='schema') with sink ('node-urls'='%s')", + receiverDataNode.getIpAndPortString()), + String.format( + "create pipe a2b2 with source ('realtime.enable'='false') with sink ('node-urls'='%s')", + receiverDataNode.getIpAndPortString()), + String.format( + "create pipe a2b3 with source ('history.enable'='false') with sink ('node-urls'='%s')", + receiverDataNode.getIpAndPortString())), + null); + + try (final SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) senderEnv.getLeaderConfigNodeConnection()) { + final List showPipeResult = client.showPipe(new TShowPipeReq()).pipeInfoList; + showPipeResult.removeIf(i -> i.getId().startsWith("__consensus")); + Assert.assertEquals(3, showPipeResult.size()); + } + + TestUtils.executeNonQueries( + senderEnv, + Arrays.asList( + "drop pipe a2b1", + "drop pipe a2b2", + "drop pipe a2b3", + "insert into root.test.device(time, field) values(0,1),(1,2)", + "delete from root.test.device.* where time == 0", + String.format( + "create pipe a2b with source ('inclusion'='all') with sink ('node-urls'='%s')", + receiverDataNode.getIpAndPortString())), + null); + + TestUtils.assertDataEventuallyOnEnv( + receiverEnv, + "select * from root.test.device", + "Time,root.test.device.field,", + Collections.singleton("1,2.0,")); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 9038e7c3a7191..67b9460c15a12 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -586,7 +586,7 @@ public boolean hasPipeReleaseRegionRelatedResource(final int consensusGroupId) { } } - public boolean isFullSync(final PipeParameters parameters) { + public boolean isFullSync(final PipeParameters parameters) throws IllegalPathException { if (isSnapshotMode(parameters)) { return false; } @@ -600,7 +600,10 @@ public boolean isFullSync(final PipeParameters parameters) { Arrays.asList(EXTRACTOR_REALTIME_ENABLE_KEY, SOURCE_REALTIME_ENABLE_KEY), EXTRACTOR_REALTIME_ENABLE_DEFAULT_VALUE); - return isHistoryEnable && isRealtimeEnable; + return isHistoryEnable + && isRealtimeEnable + && DataRegionListeningFilter.parseInsertionDeletionListeningOptionPair(parameters) + .getLeft(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index d582832864a8f..99eee84964e97 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -140,6 +140,7 @@ import org.apache.iotdb.db.exception.metadata.SchemaQuotaExceededException; import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.source.dataregion.DataRegionListeningFilter; import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; @@ -1826,10 +1827,10 @@ public SettableFuture createPipe( new PipeParameters(createPipeStatement.getSourceAttributes()); final PipeParameters sinkPipeParameters = new PipeParameters(createPipeStatement.getSinkAttributes()); - if (PipeConfig.getInstance().getPipeAutoSplitFullEnabled() - && PipeDataNodeAgent.task().isFullSync(sourcePipeParameters)) { - try (final ConfigNodeClient configNodeClient = - CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + try (final ConfigNodeClient configNodeClient = + CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + if (PipeConfig.getInstance().getPipeAutoSplitFullEnabled() + && PipeDataNodeAgent.task().isFullSync(sourcePipeParameters)) { // 1. Send request to create the real-time data synchronization pipeline final TCreatePipeReq realtimeReq = new TCreatePipeReq() @@ -1874,11 +1875,19 @@ public SettableFuture createPipe( Boolean.toString(false), PipeSourceConstant.EXTRACTOR_HISTORY_ENABLE_KEY, Boolean.toString(true), - // We force the historical pipe to transfer data only + PipeSourceConstant.EXTRACTOR_MODE_KEY, + PipeSourceConstant.EXTRACTOR_MODE_SNAPSHOT_VALUE, + // We force the historical pipe to transfer data (and maybe + // deletion) only // Thus we can transfer schema only once // And may drop the historical pipe on successfully transferred PipeSourceConstant.SOURCE_INCLUSION_KEY, - PipeSourceConstant.EXTRACTOR_INCLUSION_DEFAULT_VALUE, + DataRegionListeningFilter + .parseInsertionDeletionListeningOptionPair( + sourcePipeParameters) + .getRight() + ? "data" + : PipeSourceConstant.EXTRACTOR_INCLUSION_DEFAULT_VALUE, PipeSourceConstant.SOURCE_EXCLUSION_KEY, PipeSourceConstant.EXTRACTOR_EXCLUSION_DEFAULT_VALUE))) .getAttribute()) @@ -1901,31 +1910,24 @@ public SettableFuture createPipe( // 3. Set success status only if both pipelines are created successfully future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); - } catch (final Exception e) { - // Catch any other exceptions (e.g., network issues) - future.setException(e); - } - return future; - } - - try (final ConfigNodeClient configNodeClient = - CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { - TCreatePipeReq req = - new TCreatePipeReq() - .setPipeName(pipeName) - .setIfNotExistsCondition(createPipeStatement.hasIfNotExistsCondition()) - .setExtractorAttributes(createPipeStatement.getSourceAttributes()) - .setProcessorAttributes(createPipeStatement.getProcessorAttributes()) - .setConnectorAttributes(createPipeStatement.getSinkAttributes()); - TSStatus tsStatus = configNodeClient.createPipe(req); - if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) { - LOGGER.warn( - "Failed to create pipe {} in config node, status is {}.", - createPipeStatement.getPipeName(), - tsStatus); - future.setException(new IoTDBException(tsStatus)); } else { - future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); + final TCreatePipeReq req = + new TCreatePipeReq() + .setPipeName(pipeName) + .setIfNotExistsCondition(createPipeStatement.hasIfNotExistsCondition()) + .setExtractorAttributes(createPipeStatement.getSourceAttributes()) + .setProcessorAttributes(createPipeStatement.getProcessorAttributes()) + .setConnectorAttributes(createPipeStatement.getSinkAttributes()); + final TSStatus tsStatus = configNodeClient.createPipe(req); + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) { + LOGGER.warn( + "Failed to create pipe {} in config node, status is {}.", + createPipeStatement.getPipeName(), + tsStatus); + future.setException(new IoTDBException(tsStatus)); + } else { + future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); + } } } catch (final Exception e) { future.setException(e); From abcc51ca99272eefc3a492ce5c85f75061691ac4 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 9 Jun 2026 16:49:32 +0800 Subject: [PATCH 092/102] Fix pipe receiver type conversion load path (#17849) (#17873) Backport receiver-side tree-model changes to dev/1.3. The 2.x table-model load analyzer changes and table/dual ITs are not applicable to dev/1.3. (cherry picked from commit 90055d55b6e1166580824e5cbed77a7c253ef514) --- .../thrift/IoTDBDataNodeReceiver.java | 33 +++++++++++--- .../thrift/IoTDBDataNodeReceiverTest.java | 43 ++++++++++++++++--- 2 files changed, 65 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index 1ec4d6d53b7ac..8a5c4ad060fa6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -501,7 +501,7 @@ static Map buildLoadTsFileAttributesForAsync( dataBaseName, LoadTsFileStatement.getDatabaseLevelByTreeDatabase(dataBaseName), shouldConvertDataTypeOnTypeMismatch, - validateTsFile, + validateTsFile || shouldConvertDataTypeOnTypeMismatch, null, shouldMarkAsPipeRequest); } @@ -509,16 +509,23 @@ static Map buildLoadTsFileAttributesForAsync( private TSStatus loadTsFileSync(final String dataBaseName, final String fileAbsolutePath) throws FileNotFoundException { return executeStatementAndClassifyExceptions( - buildLoadTsFileStatementForSync(dataBaseName, fileAbsolutePath, validateTsFile.get())); + buildLoadTsFileStatementForSync( + dataBaseName, + fileAbsolutePath, + validateTsFile.get(), + shouldConvertDataTypeOnTypeMismatch)); } static LoadTsFileStatement buildLoadTsFileStatementForSync( - final String dataBaseName, final String fileAbsolutePath, final boolean validateTsFile) + final String dataBaseName, + final String fileAbsolutePath, + final boolean validateTsFile, + final boolean shouldConvertDataTypeOnTypeMismatch) throws FileNotFoundException { final LoadTsFileStatement statement = LoadTsFileStatement.createUnchecked(fileAbsolutePath); statement.setDeleteAfterLoad(true); - statement.setConvertOnTypeMismatch(true); - statement.setVerifySchema(validateTsFile); + statement.setConvertOnTypeMismatch(shouldConvertDataTypeOnTypeMismatch); + statement.setVerifySchema(validateTsFile || shouldConvertDataTypeOnTypeMismatch); statement.setAutoCreateDatabase( IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled()); statement.setDatabase(dataBaseName); @@ -769,9 +776,23 @@ private TSStatus executeStatementWithRetryOnDataTypeMismatch(final Statement sta TSStatusCode.PIPE_TRANSFER_EXECUTE_STATEMENT_ERROR, "Execute null statement."); } + // Execute insert statements through the conversion wrapper first to avoid writing a partial + // row/tablet before the type mismatch is converted. + if (shouldConvertDataTypeOnTypeMismatch && statement instanceof InsertBaseStatement) { + final Optional convertedStatus = + statement.accept( + statementDataTypeConvertExecutionVisitor, + new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + if (convertedStatus.isPresent()) { + return convertedStatus.get(); + } + } + final TSStatus status = executeStatement(statement); - // Try to convert the data type and the status code is not success + // Try to convert data type if the status code is not success. Insert statements normally return + // above after the first converted execution. The retry path is kept for load and fallback + // cases. return shouldConvertDataTypeOnTypeMismatch && ((statement instanceof InsertBaseStatement && ((InsertBaseStatement) statement).hasFailedMeasurements()) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiverTest.java index f41c44763f997..1e279a18febfb 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiverTest.java @@ -39,7 +39,7 @@ public void testLoadTsFileSyncStatementUsesTreeDatabaseLevelFromDatabaseName() t try { final LoadTsFileStatement statement = IoTDBDataNodeReceiver.buildLoadTsFileStatementForSync( - "root.test.sg_0", tsFile.toString(), true); + "root.test.sg_0", tsFile.toString(), true, true); Assert.assertEquals("root.test.sg_0", statement.getDatabase()); Assert.assertEquals(2, statement.getDatabaseLevel()); @@ -54,16 +54,17 @@ public void testLoadTsFileAsyncAttributesUseTreeDatabaseLevelFromDatabaseName() try { final Map attributes = IoTDBDataNodeReceiver.buildLoadTsFileAttributesForAsync( - "root.test.sg_0", true, true, true); + "root.test.sg_0", true, false, true); Assert.assertEquals( "root.test.sg_0", attributes.get(LoadTsFileConfigurator.DATABASE_NAME_KEY)); Assert.assertEquals("2", attributes.get(LoadTsFileConfigurator.DATABASE_LEVEL_KEY)); final LoadTsFileStatement statement = LoadTsFileStatement.createUnchecked(tsFile.toString()); - ActiveLoadPathHelper.applyAttributesToStatement(attributes, statement, true); + ActiveLoadPathHelper.applyAttributesToStatement(attributes, statement, false); Assert.assertEquals("root.test.sg_0", statement.getDatabase()); Assert.assertEquals(2, statement.getDatabaseLevel()); + Assert.assertTrue(statement.isVerifySchema()); } finally { Files.deleteIfExists(tsFile); } @@ -75,7 +76,8 @@ public void testLoadTsFileSyncStatementKeepsDefaultDatabaseLevelWhenDatabaseName final Path tsFile = Files.createTempFile("pipe-load-default-database-level", ".tsfile"); try { final LoadTsFileStatement statement = - IoTDBDataNodeReceiver.buildLoadTsFileStatementForSync(null, tsFile.toString(), true); + IoTDBDataNodeReceiver.buildLoadTsFileStatementForSync( + null, tsFile.toString(), true, true); Assert.assertNull(statement.getDatabase()); Assert.assertEquals( @@ -92,7 +94,7 @@ public void testRepeatedStatementExceptionLogIsReduced() throws Exception { try { final LoadTsFileStatement statement = IoTDBDataNodeReceiver.buildLoadTsFileStatementForSync( - "root.test.sg_0", tsFile.toString(), true); + "root.test.sg_0", tsFile.toString(), true, true); final long receiverId = System.nanoTime(); final Exception exception = new RuntimeException("repeated receiver exception " + receiverId); @@ -107,4 +109,35 @@ public void testRepeatedStatementExceptionLogIsReduced() throws Exception { Files.deleteIfExists(tsFile); } } + + @Test + public void testLoadTsFileSyncStatementVerifiesSchemaWhenConvertingType() throws Exception { + final Path tsFile = Files.createTempFile("pipe-load-convert-verify-schema", ".tsfile"); + try { + final LoadTsFileStatement statement = + IoTDBDataNodeReceiver.buildLoadTsFileStatementForSync( + "root.test.sg_0", tsFile.toString(), false, true); + + Assert.assertTrue(statement.isConvertOnTypeMismatch()); + Assert.assertTrue(statement.isVerifySchema()); + } finally { + Files.deleteIfExists(tsFile); + } + } + + @Test + public void testLoadTsFileSyncStatementCanSkipVerifySchemaWhenNotConvertingType() + throws Exception { + final Path tsFile = Files.createTempFile("pipe-load-no-convert-no-verify-schema", ".tsfile"); + try { + final LoadTsFileStatement statement = + IoTDBDataNodeReceiver.buildLoadTsFileStatementForSync( + "root.test.sg_0", tsFile.toString(), false, false); + + Assert.assertFalse(statement.isConvertOnTypeMismatch()); + Assert.assertFalse(statement.isVerifySchema()); + } finally { + Files.deleteIfExists(tsFile); + } + } } From 6a59cf43a416189c796fcdcdcc7e5d6a11ab4fa2 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 9 Jun 2026 18:17:35 +0800 Subject: [PATCH 093/102] Optimize local load TsFile piece dispatch (#17851) (#17876) --- .../load/LoadTsFileDispatcherImpl.java | 8 +- .../load/splitter/AlignedChunkData.java | 8 +- .../load/splitter/NonAlignedChunkData.java | 14 +++ .../load/LoadTsFileDispatcherImplTest.java | 98 +++++++++++++++++++ .../splitter/ChunkDataDirectWriteTest.java | 89 +++++++++++++++++ 5 files changed, 209 insertions(+), 8 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImplTest.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/load/splitter/ChunkDataDirectWriteTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java index c0703113cdf45..37f2fba832d4c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java @@ -38,7 +38,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; import org.apache.iotdb.db.queryengine.plan.planner.plan.SubPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.load.LoadSingleTsFileNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.load.LoadTsFilePieceNode; import org.apache.iotdb.db.queryengine.plan.scheduler.FragInstanceDispatchResult; @@ -156,12 +155,7 @@ public void dispatchLocally(FragmentInstance instance) throws FragmentInstanceDi PlanNode planNode = instance.getFragment().getPlanNodeTree(); if (planNode instanceof LoadTsFilePieceNode) { // split - LoadTsFilePieceNode pieceNode = - (LoadTsFilePieceNode) PlanNodeType.deserialize(planNode.serializeToByteBuffer()); - if (pieceNode == null) { - throw new FragmentInstanceDispatchException( - new TSStatus(TSStatusCode.DESERIALIZE_PIECE_OF_TSFILE_ERROR.getStatusCode())); - } + LoadTsFilePieceNode pieceNode = (LoadTsFilePieceNode) planNode; TSStatus resultStatus = StorageEngine.getInstance().writeLoadTsFileNode((DataRegionId) groupId, pieceNode, uuid); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/AlignedChunkData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/AlignedChunkData.java index 86bd02c9947d1..8567d30660def 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/AlignedChunkData.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/AlignedChunkData.java @@ -287,7 +287,7 @@ public void writeDecodeValuePage( } protected void deserializeTsFileData(TsFileIOWriter writer) throws IOException, PageException { - final InputStream stream = new ByteArrayInputStream(chunkData); + final InputStream stream = createTsFileDataInputStream(); if (needDecodeChunk) { buildChunkWriter(stream, writer); } else { @@ -295,6 +295,12 @@ protected void deserializeTsFileData(TsFileIOWriter writer) throws IOException, } } + private InputStream createTsFileDataInputStream() { + return chunkData == null + ? new ByteArrayInputStream(byteStream.getBuf(), 0, byteStream.size()) + : new ByteArrayInputStream(chunkData); + } + protected void deserializeTsFileDataByte(final InputStream stream) throws IOException { final int size = ReadWriteIOUtils.readInt(stream); this.chunkData = new byte[size]; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/NonAlignedChunkData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/NonAlignedChunkData.java index 2d8a75052ffb9..6c5504e7a99d5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/NonAlignedChunkData.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/NonAlignedChunkData.java @@ -38,6 +38,7 @@ import org.apache.tsfile.write.schema.MeasurementSchema; import org.apache.tsfile.write.writer.TsFileIOWriter; +import java.io.ByteArrayInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; @@ -113,6 +114,7 @@ public boolean isAligned() { @Override public void writeToFileWriter(final TsFileIOWriter writer) throws IOException { + ensureDataReadyForWriting(); if (chunk != null) { writer.writeChunk(chunk); } else { @@ -120,6 +122,18 @@ public void writeToFileWriter(final TsFileIOWriter writer) throws IOException { } } + private void ensureDataReadyForWriting() throws IOException { + if (chunk != null || chunkWriter != null) { + return; + } + + try { + deserializeTsFileData(new ByteArrayInputStream(byteStream.getBuf(), 0, byteStream.size())); + } catch (final PageException e) { + throw new IOException(e); + } + } + @Override public void serialize(final DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(isModification(), stream); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImplTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImplTest.java new file mode 100644 index 0000000000000..3d496470597ca --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImplTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.scheduler.load; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.partition.StorageExecutor; +import org.apache.iotdb.db.queryengine.common.PlanFragmentId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; +import org.apache.iotdb.db.queryengine.plan.planner.plan.PlanFragment; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.load.LoadTsFilePieceNode; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.rpc.RpcUtils; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.io.File; +import java.util.Collections; + +@PowerMockIgnore({"com.sun.org.apache.xerces.*", "javax.xml.*", "org.xml.*", "javax.management.*"}) +@RunWith(PowerMockRunner.class) +@PrepareForTest(StorageEngine.class) +public class LoadTsFileDispatcherImplTest { + + @Test + public void testDispatchLocallyPieceNodeSkipsSerdeRoundTrip() throws Exception { + final StorageEngine storageEngine = Mockito.mock(StorageEngine.class); + PowerMockito.mockStatic(StorageEngine.class); + PowerMockito.when(StorageEngine.getInstance()).thenReturn(storageEngine); + + final LoadTsFileDispatcherImpl dispatcher = new LoadTsFileDispatcherImpl(null, false); + dispatcher.setUuid("test-uuid"); + + final LoadTsFilePieceNode pieceNode = + new LoadTsFilePieceNode(new PlanNodeId("piece"), new File("test.tsfile")); + final FragmentInstance instance = createFragmentInstance(pieceNode); + + Mockito.when( + storageEngine.writeLoadTsFileNode( + Mockito.eq(new DataRegionId(1)), Mockito.same(pieceNode), Mockito.eq("test-uuid"))) + .thenReturn(RpcUtils.SUCCESS_STATUS); + + dispatcher.dispatchLocally(instance); + + Mockito.verify(storageEngine) + .writeLoadTsFileNode( + Mockito.eq(new DataRegionId(1)), Mockito.same(pieceNode), Mockito.eq("test-uuid")); + } + + private static FragmentInstance createFragmentInstance(final LoadTsFilePieceNode pieceNode) { + final PlanFragmentId fragmentId = new PlanFragmentId("test", 0); + final FragmentInstance instance = + new FragmentInstance( + new PlanFragment(fragmentId, pieceNode), + fragmentId.genFragmentInstanceId(), + null, + null, + 0, + null, + false, + false); + final TConsensusGroupId consensusGroupId = new DataRegionId(1).convertToTConsensusGroupId(); + instance.setExecutorAndHost( + new StorageExecutor( + new TRegionReplicaSet( + consensusGroupId, + Collections.singletonList( + new TDataNodeLocation().setInternalEndPoint(new TEndPoint("127.0.0.1", 1)))))); + return instance; + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/load/splitter/ChunkDataDirectWriteTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/load/splitter/ChunkDataDirectWriteTest.java new file mode 100644 index 0000000000000..e98541a9125b3 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/load/splitter/ChunkDataDirectWriteTest.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.load.splitter; + +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.header.ChunkHeader; +import org.apache.tsfile.file.metadata.IChunkMetadata; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.file.metadata.statistics.Statistics; +import org.apache.tsfile.read.common.Chunk; +import org.apache.tsfile.write.writer.TsFileIOWriter; +import org.junit.Test; +import org.mockito.Mockito; + +import java.nio.ByteBuffer; + +public class ChunkDataDirectWriteTest { + + @Test + public void testNonAlignedChunkDataCanWriteWithoutSerdeRoundTrip() throws Exception { + final NonAlignedChunkData chunkData = createNonAlignedChunkData(); + chunkData.setNotDecode(); + final IChunkMetadata chunkMetadata = Mockito.mock(IChunkMetadata.class); + Mockito.doReturn(createInt32Statistics()).when(chunkMetadata).getStatistics(); + chunkData.writeEntireChunk(ByteBuffer.allocate(0), chunkMetadata); + + final TsFileIOWriter writer = Mockito.mock(TsFileIOWriter.class); + chunkData.writeToFileWriter(writer); + + Mockito.verify(writer).writeChunk(Mockito.any(Chunk.class)); + } + + @Test + public void testAlignedChunkDataCanWriteWithoutSerdeRoundTrip() throws Exception { + final AlignedChunkData chunkData = createAlignedChunkData(); + chunkData.setNotDecode(); + final IChunkMetadata chunkMetadata = Mockito.mock(IChunkMetadata.class); + Mockito.doReturn(createInt32Statistics()).when(chunkMetadata).getStatistics(); + chunkData.writeEntireChunk(ByteBuffer.allocate(0), chunkMetadata); + + final TsFileIOWriter writer = Mockito.mock(TsFileIOWriter.class); + chunkData.writeToFileWriter(writer); + + Mockito.verify(writer).writeChunk(Mockito.any(Chunk.class)); + } + + private static Statistics createInt32Statistics() { + final Statistics statistics = Statistics.getStatsByType(TSDataType.INT32); + statistics.update(1L, 1); + return statistics; + } + + private static NonAlignedChunkData createNonAlignedChunkData() { + return (NonAlignedChunkData) + ChunkData.createChunkData( + false, "root.sg.d1", createChunkHeader(), new TTimePartitionSlot(0L)); + } + + private static AlignedChunkData createAlignedChunkData() { + return (AlignedChunkData) + ChunkData.createChunkData( + true, "root.sg.d1", createChunkHeader(), new TTimePartitionSlot(0L)); + } + + private static ChunkHeader createChunkHeader() { + return new ChunkHeader( + "temperature", 0, TSDataType.INT32, CompressionType.UNCOMPRESSED, TSEncoding.PLAIN, 0); + } +} From d39d22fa63afe7885e77cd93ea74ddbf8576f9cf Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 9 Jun 2026 18:17:49 +0800 Subject: [PATCH 094/102] Support sink config key for pipe request slicing (#17858) (#17883) * Support sink config key for pipe request slicing * Support processor output series aliases --- .../env/cluster/config/MppCommonConfig.java | 2 +- .../plugin/TwoStageCountProcessor.java | 22 +++++-- .../plugin/TwoStageCountProcessorTest.java | 57 ++++++++++++++++ .../commons/pipe/config/PipeDescriptor.java | 8 ++- .../pipe/config/PipeDescriptorTest.java | 65 +++++++++++++++++++ 5 files changed, 146 insertions(+), 8 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessorTest.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/config/PipeDescriptorTest.java diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java index d9017df36e767..8109bf02c7ea4 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java @@ -508,7 +508,7 @@ public CommonConfig setPipeMetaSyncerSyncIntervalMinutes(long pipeMetaSyncerSync public CommonConfig setPipeConnectorRequestSliceThresholdBytes( int pipeConnectorRequestSliceThresholdBytes) { setProperty( - "pipe_connector_request_slice_threshold_bytes", + "pipe_sink_request_slice_threshold_bytes", String.valueOf(pipeConnectorRequestSliceThresholdBytes)); return this; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java index 16a20a5f509aa..340ac085a010c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java @@ -74,6 +74,7 @@ public class TwoStageCountProcessor implements PipeProcessor { private static final Logger LOGGER = LoggerFactory.getLogger(TwoStageCountProcessor.class); + private static final String LEGACY_PROCESSOR_OUTPUT_SERIES_KEY = "processor.output.series"; private String pipeName; private long creationTime; @@ -98,10 +99,17 @@ public class TwoStageCountProcessor implements PipeProcessor { @Override public void validate(PipeParameterValidator validator) throws Exception { - validator.validateRequiredAttribute(PipeProcessorConstant.PROCESSOR_OUTPUT_SERIES_KEY); + validator.validateSynonymAttributes( + Collections.singletonList(PipeProcessorConstant.PROCESSOR_OUTPUT_SERIES_KEY), + Collections.singletonList(LEGACY_PROCESSOR_OUTPUT_SERIES_KEY), + true); final String rawOutputSeries = - validator.getParameters().getString(PipeProcessorConstant.PROCESSOR_OUTPUT_SERIES_KEY); + validator + .getParameters() + .getStringByKeys( + PipeProcessorConstant.PROCESSOR_OUTPUT_SERIES_KEY, + LEGACY_PROCESSOR_OUTPUT_SERIES_KEY); try { PathUtils.isLegalPath(rawOutputSeries); } catch (IllegalPathException e) { @@ -119,8 +127,7 @@ public void customize(PipeParameters parameters, PipeProcessorRuntimeConfigurati regionId = runtimeEnvironment.getRegionId(); pipeTaskMeta = runtimeEnvironment.getPipeTaskMeta(); - outputSeries = - new PartialPath(parameters.getString(PipeProcessorConstant.PROCESSOR_OUTPUT_SERIES_KEY)); + outputSeries = parseOutputSeries(parameters); if (Objects.nonNull(pipeTaskMeta) && Objects.nonNull(pipeTaskMeta.getProgressIndex())) { if (pipeTaskMeta.getProgressIndex() instanceof MinimumProgressIndex) { @@ -152,6 +159,13 @@ public void customize(PipeParameters parameters, PipeProcessorRuntimeConfigurati twoStageAggregateSender = new TwoStageAggregateSender(pipeName, creationTime); } + static PartialPath parseOutputSeries(final PipeParameters parameters) + throws IllegalPathException { + return new PartialPath( + parameters.getStringByKeys( + PipeProcessorConstant.PROCESSOR_OUTPUT_SERIES_KEY, LEGACY_PROCESSOR_OUTPUT_SERIES_KEY)); + } + @Override public void process(TabletInsertionEvent tabletInsertionEvent, EventCollector eventCollector) throws Exception { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessorTest.java new file mode 100644 index 0000000000000..2957ffd4e3e55 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessorTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.processor.twostage.plugin; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; + +public class TwoStageCountProcessorTest { + + @Test + public void testOutputSeriesSupportsNewAndLegacyKeys() throws Exception { + Assert.assertEquals( + "root.db.d.s1", parseOutputSeries("processor.output.series", "root.db.d.s1").getFullPath()); + Assert.assertEquals( + "root.db.d.s2", parseOutputSeries("processor.output-series", "root.db.d.s2").getFullPath()); + } + + @Test + public void testValidateOutputSeriesSupportsNewAndLegacyKeys() throws Exception { + validateOutputSeries("processor.output.series", "root.db.d.s1"); + validateOutputSeries("processor.output-series", "root.db.d.s2"); + } + + private PartialPath parseOutputSeries(final String key, final String value) throws Exception { + return TwoStageCountProcessor.parseOutputSeries( + new PipeParameters(Collections.singletonMap(key, value))); + } + + private void validateOutputSeries(final String key, final String value) throws Exception { + new TwoStageCountProcessor() + .validate( + new PipeParameterValidator(new PipeParameters(Collections.singletonMap(key, value)))); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index 83db711e5ec63..ee035ba3eb1f6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -445,9 +445,11 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr config.setPipeSinkRequestSliceThresholdBytes( Integer.parseInt( - properties.getProperty( - "pipe_connector_request_slice_threshold_bytes", - String.valueOf(config.getPipeSinkRequestSliceThresholdBytes())))); + Optional.ofNullable(properties.getProperty("pipe_sink_request_slice_threshold_bytes")) + .orElse( + properties.getProperty( + "pipe_connector_request_slice_threshold_bytes", + String.valueOf(config.getPipeSinkRequestSliceThresholdBytes()))))); config.setPipeReceiverLoginPeriodicVerificationIntervalMs( Long.parseLong( diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/config/PipeDescriptorTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/config/PipeDescriptorTest.java new file mode 100644 index 0000000000000..00d982122719b --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/config/PipeDescriptorTest.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.config; + +import org.apache.iotdb.commons.conf.CommonConfig; +import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.commons.conf.TrimProperties; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class PipeDescriptorTest { + + private final CommonConfig config = CommonDescriptor.getInstance().getConfig(); + + private int originalRequestSliceThresholdBytes; + + @Before + public void setUp() { + originalRequestSliceThresholdBytes = config.getPipeSinkRequestSliceThresholdBytes(); + } + + @After + public void tearDown() { + config.setPipeSinkRequestSliceThresholdBytes(originalRequestSliceThresholdBytes); + } + + @Test + public void testPipeRequestSliceThresholdSupportsSinkAndConnectorKeys() { + final TrimProperties connectorProperties = new TrimProperties(); + connectorProperties.setProperty("pipe_connector_request_slice_threshold_bytes", "123"); + PipeDescriptor.loadPipeInternalConfig(config, connectorProperties); + Assert.assertEquals(123, config.getPipeSinkRequestSliceThresholdBytes()); + + final TrimProperties sinkProperties = new TrimProperties(); + sinkProperties.setProperty("pipe_sink_request_slice_threshold_bytes", "456"); + PipeDescriptor.loadPipeInternalConfig(config, sinkProperties); + Assert.assertEquals(456, config.getPipeSinkRequestSliceThresholdBytes()); + + final TrimProperties bothProperties = new TrimProperties(); + bothProperties.setProperty("pipe_connector_request_slice_threshold_bytes", "123"); + bothProperties.setProperty("pipe_sink_request_slice_threshold_bytes", "456"); + PipeDescriptor.loadPipeInternalConfig(config, bothProperties); + Assert.assertEquals(456, config.getPipeSinkRequestSliceThresholdBytes()); + } +} From 7cffde507d6e81dd0d5af385c645d60093ce97f8 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 9 Jun 2026 18:18:02 +0800 Subject: [PATCH 095/102] Optimize load partition routing (#17863) (#17884) --- .../plan/analyze/ClusterPartitionFetcher.java | 38 +++++----- .../plan/node/load/LoadSingleTsFileNode.java | 40 ++++++---- .../scheduler/load/LoadTsFileScheduler.java | 74 +++++++++++-------- 3 files changed, 88 insertions(+), 64 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java index eb65666035a99..6b28dbda34705 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java @@ -258,26 +258,28 @@ public DataPartition getOrCreateDataPartition( dataPartitionQueryParams, config.isAutoCreateSchemaEnabled(), userName); DataPartition dataPartition = partitionCache.getDataPartition(splitDataPartitionQueryParams); - if (null == dataPartition) { - try (ConfigNodeClient client = - configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { - TDataPartitionReq req = constructDataPartitionReq(splitDataPartitionQueryParams); - TDataPartitionTableResp dataPartitionTableResp = client.getOrCreateDataPartitionTable(req); + if (null != dataPartition) { + return dataPartition; + } - if (dataPartitionTableResp.getStatus().getCode() - == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - dataPartition = parseDataPartitionResp(dataPartitionTableResp); - partitionCache.updateDataPartitionCache(dataPartitionTableResp.getDataPartitionTable()); - } else { - throw new RuntimeException( - new IoTDBException( - dataPartitionTableResp.getStatus().getMessage(), - dataPartitionTableResp.getStatus().getCode())); - } - } catch (ClientManagerException | TException e) { - throw new StatementAnalyzeException( - "An error occurred when executing getOrCreateDataPartition():" + e.getMessage()); + try (ConfigNodeClient client = + configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + TDataPartitionReq req = constructDataPartitionReq(splitDataPartitionQueryParams); + TDataPartitionTableResp dataPartitionTableResp = client.getOrCreateDataPartitionTable(req); + + if (dataPartitionTableResp.getStatus().getCode() + == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + dataPartition = parseDataPartitionResp(dataPartitionTableResp); + partitionCache.updateDataPartitionCache(dataPartitionTableResp.getDataPartitionTable()); + } else { + throw new RuntimeException( + new IoTDBException( + dataPartitionTableResp.getStatus().getMessage(), + dataPartitionTableResp.getStatus().getCode())); } + } catch (ClientManagerException | TException e) { + throw new StatementAnalyzeException( + "An error occurred when executing getOrCreateDataPartition():" + e.getMessage()); } return dataPartition; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java index 64b4655be2863..a168228deb30f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java @@ -79,27 +79,35 @@ public boolean isTsFileEmpty() { public boolean needDecodeTsFile( Function>, List> partitionFetcher) { - List> slotList = new ArrayList<>(); - resource - .getDevices() - .forEach( - o -> { - // iterating the index, must present - slotList.add( - new Pair<>( - o, TimePartitionUtils.getTimePartitionSlot(resource.getStartTime(o).get()))); - slotList.add( - new Pair<>( - o, TimePartitionUtils.getTimePartitionSlot(resource.getEndTime(o).get()))); - }); + if (needDecodeTsFile) { + return true; + } + + List> slotList = + new ArrayList<>(resource.getDevices().size() << 1); + for (final IDeviceID device : resource.getDevices()) { + // iterating the index, must present + final TTimePartitionSlot startSlot = + TimePartitionUtils.getTimePartitionSlot(resource.getStartTime(device).get()); + final TTimePartitionSlot endSlot = + TimePartitionUtils.getTimePartitionSlot(resource.getEndTime(device).get()); + slotList.add(new Pair<>(device, startSlot)); + if (!startSlot.equals(endSlot)) { + slotList.add(new Pair<>(device, endSlot)); + } + } if (slotList.isEmpty()) { throw new IllegalStateException( String.format("Devices in TsFile %s is empty, this should not happen here.", tsFile)); - } else if (slotList.stream() - .anyMatch(slotPair -> !slotPair.getRight().equals(slotList.get(0).right))) { - needDecodeTsFile = true; } else { + final TTimePartitionSlot firstSlot = slotList.get(0).right; + for (int i = 1, size = slotList.size(); i < size; i++) { + if (!slotList.get(i).right.equals(firstSlot)) { + needDecodeTsFile = true; + return true; + } + } needDecodeTsFile = !isDispatchedToLocal(new HashSet<>(partitionFetcher.apply(slotList))); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java index 5cafe3b838461..e61367ed896a0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java @@ -709,18 +709,29 @@ private void routeChunkData() throws LoadFileException { return; } + final List> partitionSlotList = new ArrayList<>(); + final int[] chunkPartitionIndexes = new int[nonDirectionalChunkData.size()]; + final Map> partitionSlotIndexes = new HashMap<>(); + for (int i = 0, size = nonDirectionalChunkData.size(); i < size; i++) { + final ChunkData chunkData = nonDirectionalChunkData.get(i); + final IDeviceID device = new PlainDeviceID(chunkData.getDevice()); + final TTimePartitionSlot timePartitionSlot = chunkData.getTimePartitionSlot(); + final Map slotIndexes = + partitionSlotIndexes.computeIfAbsent(device, key -> new HashMap<>()); + Integer partitionSlotIndex = slotIndexes.get(timePartitionSlot); + if (partitionSlotIndex == null) { + partitionSlotIndex = partitionSlotList.size(); + slotIndexes.put(timePartitionSlot, partitionSlotIndex); + partitionSlotList.add(new Pair<>(device, timePartitionSlot)); + } + chunkPartitionIndexes[i] = partitionSlotIndex; + } + List replicaSets = scheduler.partitionFetcher.queryDataPartition( - nonDirectionalChunkData.stream() - .map( - data -> - new Pair<>( - (IDeviceID) new PlainDeviceID(data.getDevice()), - data.getTimePartitionSlot())) - .collect(Collectors.toList()), - scheduler.queryContext.getSession().getUserName()); - for (int i = 0; i < replicaSets.size(); i++) { - final TRegionReplicaSet replicaSet = replicaSets.get(i); + partitionSlotList, scheduler.queryContext.getSession().getUserName()); + for (int i = 0, size = nonDirectionalChunkData.size(); i < size; i++) { + final TRegionReplicaSet replicaSet = replicaSets.get(chunkPartitionIndexes[i]); final TConsensusGroupId regionId = replicaSet.getRegionId(); if (regionId2ReplicaSetAndNode.containsKey(regionId) && !Objects.equals(regionId2ReplicaSetAndNode.get(regionId).getLeft(), replicaSet)) { @@ -790,7 +801,7 @@ public DataPartitionBatchFetcher(IPartitionFetcher fetcher) { public List queryDataPartition( List> slotList, String userName) { - List replicaSets = new ArrayList<>(); + List replicaSets = new ArrayList<>(slotList.size()); int size = slotList.size(); for (int i = 0; i < size; i += TRANSMIT_LIMIT) { @@ -798,31 +809,34 @@ public List queryDataPartition( slotList.subList(i, Math.min(size, i + TRANSMIT_LIMIT)); DataPartition dataPartition = fetcher.getOrCreateDataPartition(toQueryParam(subSlotList), userName); - replicaSets.addAll( - subSlotList.stream() - .map( - pair -> - dataPartition.getDataRegionReplicaSetForWriting( - ((PlainDeviceID) pair.left).toStringID(), pair.right)) - .collect(Collectors.toList())); + for (final Pair pair : subSlotList) { + replicaSets.add( + dataPartition.getDataRegionReplicaSetForWriting( + ((PlainDeviceID) pair.left).toStringID(), pair.right)); + } } return replicaSets; } private List toQueryParam( List> slots) { - return slots.stream() - .collect( - Collectors.groupingBy( - Pair::getLeft, Collectors.mapping(Pair::getRight, Collectors.toSet()))) - .entrySet() - .stream() - .map( - entry -> - new DataPartitionQueryParam( - ((PlainDeviceID) entry.getKey()).toStringID(), - new ArrayList<>(entry.getValue()))) - .collect(Collectors.toList()); + final Map> device2TimePartitionSlots = new HashMap<>(); + for (final Pair slot : slots) { + device2TimePartitionSlots + .computeIfAbsent(slot.left, key -> new HashSet<>()) + .add(slot.right); + } + + final List queryParams = + new ArrayList<>(device2TimePartitionSlots.size()); + for (final Map.Entry> entry : + device2TimePartitionSlots.entrySet()) { + final DataPartitionQueryParam queryParam = + new DataPartitionQueryParam( + ((PlainDeviceID) entry.getKey()).toStringID(), new ArrayList<>(entry.getValue())); + queryParams.add(queryParam); + } + return queryParams; } } } From de0a48fc6e61874739aecc3e813515e47b935de0 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 10 Jun 2026 10:39:26 +0800 Subject: [PATCH 096/102] Optimize pipe event batching and listener stop (#17864) (#17885) --- .../batch/PipeTransferBatchReqBuilder.java | 43 +++++++++++-------- .../PipeInsertionDataNodeListener.java | 2 +- 2 files changed, 26 insertions(+), 19 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java index 45264138596ff..52827ac000885 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/payload/evolvable/batch/PipeTransferBatchReqBuilder.java @@ -39,10 +39,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.ConcurrentHashMap; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_FORMAT_HYBRID_VALUE; import static org.apache.iotdb.commons.pipe.config.constant.PipeSinkConstant.CONNECTOR_FORMAT_KEY; @@ -84,8 +84,7 @@ public class PipeTransferBatchReqBuilder implements AutoCloseable { // If the leader cache is enabled, the batch will be divided by the leader endpoint, // each endpoint has a batch. // This is only used in plain batch since tsfile does not return redirection info. - private final Map endPointToBatch = - new ConcurrentHashMap<>(); + private final Map endPointToBatch = new HashMap<>(); public PipeTransferBatchReqBuilder(final PipeParameters parameters) { final boolean usingTsFileBatch = @@ -178,22 +177,29 @@ public synchronized void onEvent(final TabletInsertionEvent event) public synchronized List> getAllNonEmptyAndShouldEmitBatches() { final List> nonEmptyAndShouldEmitBatches = - new ArrayList<>(); + new ArrayList<>(endPointToBatch.size() + 1); if (!defaultBatch.isEmpty() && defaultBatch.shouldEmit()) { nonEmptyAndShouldEmitBatches.add(new Pair<>(null, defaultBatch)); } - endPointToBatch.forEach( - (endPoint, batch) -> { - if (!batch.isEmpty() && batch.shouldEmit()) { - nonEmptyAndShouldEmitBatches.add(new Pair<>(endPoint, batch)); - } - }); + for (final Map.Entry entry : endPointToBatch.entrySet()) { + final PipeTabletEventPlainBatch batch = entry.getValue(); + if (!batch.isEmpty() && batch.shouldEmit()) { + nonEmptyAndShouldEmitBatches.add(new Pair<>(entry.getKey(), batch)); + } + } return nonEmptyAndShouldEmitBatches; } - public boolean isEmpty() { - return defaultBatch.isEmpty() - && endPointToBatch.values().stream().allMatch(PipeTabletEventPlainBatch::isEmpty); + public synchronized boolean isEmpty() { + if (!defaultBatch.isEmpty()) { + return false; + } + for (final PipeTabletEventPlainBatch batch : endPointToBatch.values()) { + if (!batch.isEmpty()) { + return false; + } + } + return true; } public synchronized void discardEventsOfPipe( @@ -206,12 +212,13 @@ public synchronized void discardEventsOfPipe(final CommitterKey committerKey) { endPointToBatch.values().forEach(batch -> batch.discardEventsOfPipe(committerKey)); } - public int size() { + public synchronized int size() { try { - return defaultBatch.events.size() - + endPointToBatch.values().stream() - .map(batch -> batch.events.size()) - .reduce(0, Integer::sum); + int size = defaultBatch.events.size(); + for (final PipeTabletEventPlainBatch batch : endPointToBatch.values()) { + size += batch.events.size(); + } + return size; } catch (final Exception e) { LOGGER.warn( "Failed to get the size of PipeTransferBatchReqBuilder, return 0. Exception: {}", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index ad3586df830e9..9345d25746295 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -63,7 +63,7 @@ public synchronized void startListenAndAssign( }); } - public synchronized void stopListenAndAssign( + public void stopListenAndAssign( final String dataRegionId, final PipeRealtimeDataRegionSource source) { PipeDataRegionAssigner assignerToClose = null; From d906c6a496bdeb5bc45b178226126f27d0940d68 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 10 Jun 2026 12:28:18 +0800 Subject: [PATCH 097/102] Fix Windows path segment validation (#17868) (#17888) * Fix Windows reserved device name validation * Reject additional illegal Windows path segments * Reject empty directory path segments (cherry picked from commit 3d98ea5fd7876c4f8f96033457824f10842100c8) --- .../apache/iotdb/commons/utils/FileUtils.java | 4 ++ .../iotdb/commons/utils/WindowsOSUtils.java | 47 ++++++++++++---- .../iotdb/commons/utils/FileUtilsTest.java | 8 +++ .../commons/utils/WindowsOSUtilsTest.java | 53 +++++++++++++++++++ 4 files changed, 101 insertions(+), 11 deletions(-) create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/WindowsOSUtilsTest.java diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java index fe413bf85ef86..557adf43f6274 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java @@ -54,6 +54,7 @@ public class FileUtils { "Renamed file {} to {} because it already exists in the target directory: {}"; private static final String COPY_FILE_MESSAGE = "Copy file {} to {} because it already exists in the target directory: {}"; + private static final String ILLEGAL_EMPTY_PATH_MESSAGE = "The path cannot be empty. "; private static final String ILLEGAL_PATH_MESSAGE = "The path cannot be '.', '..', './' or '.\\'. "; @@ -556,6 +557,9 @@ private static void copyFileRename(final File sourceFile, final File targetFile) } public static String getIllegalError4Directory(final String path) { + if (path == null || path.isEmpty()) { + return ILLEGAL_EMPTY_PATH_MESSAGE; + } if (path.equals(".") || path.equals("..") || path.contains("/") || path.contains("\\")) { return ILLEGAL_PATH_MESSAGE; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/WindowsOSUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/WindowsOSUtils.java index 3019c47cf0e1c..0be89f92747a9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/WindowsOSUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/WindowsOSUtils.java @@ -23,15 +23,27 @@ import java.util.Arrays; import java.util.HashSet; +import java.util.Locale; import java.util.Set; public class WindowsOSUtils { private static final String ILLEGAL_WINDOWS_CHARS = "\\/:*?\"<>|"; private static final Set ILLEGAL_WINDOWS_NAMES = - new HashSet<>(Arrays.asList("CON", "PRN", "AUX", "NUL", "COM1-COM9, LPT1-LPT9")); + new HashSet<>( + Arrays.asList( + "CON", + "PRN", + "AUX", + "NUL", + "COM\u00B9", + "COM\u00B2", + "COM\u00B3", + "LPT\u00B9", + "LPT\u00B2", + "LPT\u00B3")); static { - for (int i = 0; i < 10; ++i) { + for (int i = 1; i < 10; ++i) { ILLEGAL_WINDOWS_NAMES.add("COM" + i); ILLEGAL_WINDOWS_NAMES.add("LPT" + i); } @@ -39,26 +51,39 @@ public class WindowsOSUtils { public static final String OS_SEGMENT_ERROR = String.format( - "In Windows System, the path shall not contains %s, equals one of %s, or ends with '.' or ' '.", + "In Windows System, the path shall not contain %s or ASCII control characters, equals one of %s with or without an extension, or ends with '.' or ' '.", ILLEGAL_WINDOWS_CHARS, ILLEGAL_WINDOWS_NAMES); public static boolean isLegalPathSegment4Windows(final String pathSegment) { if (!SystemUtils.IS_OS_WINDOWS) { return true; } - for (final char illegalChar : ILLEGAL_WINDOWS_CHARS.toCharArray()) { - if (pathSegment.indexOf(illegalChar) != -1) { - return false; - } + if (containsIllegalWindowsChar(pathSegment)) { + return false; } if (pathSegment.endsWith(".") || pathSegment.endsWith(" ")) { return false; } - for (final String illegalName : ILLEGAL_WINDOWS_NAMES) { - if (pathSegment.equalsIgnoreCase(illegalName)) { - return false; - } + if (isIllegalWindowsName(pathSegment)) { + return false; } return true; } + + private static boolean containsIllegalWindowsChar(final String pathSegment) { + for (int i = 0; i < pathSegment.length(); ++i) { + final char ch = pathSegment.charAt(i); + if (ch < ' ' || ILLEGAL_WINDOWS_CHARS.indexOf(ch) != -1) { + return true; + } + } + return false; + } + + private static boolean isIllegalWindowsName(final String pathSegment) { + final int extensionStartIndex = pathSegment.indexOf('.'); + final String nameWithoutExtension = + extensionStartIndex < 0 ? pathSegment : pathSegment.substring(0, extensionStartIndex); + return ILLEGAL_WINDOWS_NAMES.contains(nameWithoutExtension.toUpperCase(Locale.ENGLISH)); + } } diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/FileUtilsTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/FileUtilsTest.java index c5384e07fadae..f4b1ed962bc8e 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/FileUtilsTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/FileUtilsTest.java @@ -26,6 +26,7 @@ import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.MeasurementSchema; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -61,6 +62,13 @@ public void testFileUtils() throws WriteProcessException, IOException { FileUtils.moveFileWithMD5Check(tstFile, targetDir); } + @Test + public void testGetIllegalError4DirectoryRejectsEmptyPath() { + Assert.assertNotNull(FileUtils.getIllegalError4Directory(null)); + Assert.assertNotNull(FileUtils.getIllegalError4Directory("")); + Assert.assertNull(FileUtils.getIllegalError4Directory("valid_dir")); + } + private void generateFile(File tsfile) throws WriteProcessException, IOException { try (TsFileWriter writer = new TsFileWriter(tsfile)) { writer.registerAlignedTimeseries( diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/WindowsOSUtilsTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/WindowsOSUtilsTest.java new file mode 100644 index 0000000000000..f088ba80730f2 --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/WindowsOSUtilsTest.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.utils; + +import org.apache.commons.lang3.SystemUtils; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.iotdb.commons.utils.WindowsOSUtils.isLegalPathSegment4Windows; + +public class WindowsOSUtilsTest { + @Test + public void testIllegalDetection() { + if (!SystemUtils.IS_OS_WINDOWS) { + return; + } + Assert.assertTrue(isLegalPathSegment4Windows("abc")); + Assert.assertTrue(isLegalPathSegment4Windows(".A!")); + + Assert.assertFalse(isLegalPathSegment4Windows("C.")); + Assert.assertFalse(isLegalPathSegment4Windows("a:b<|")); + Assert.assertFalse(isLegalPathSegment4Windows("COM1")); + Assert.assertFalse(isLegalPathSegment4Windows("com1")); + Assert.assertFalse(isLegalPathSegment4Windows("COM1.txt")); + Assert.assertFalse(isLegalPathSegment4Windows("NUL.log")); + Assert.assertFalse(isLegalPathSegment4Windows("LPT9.tmp")); + Assert.assertFalse(isLegalPathSegment4Windows("COM\u00B9")); + Assert.assertFalse(isLegalPathSegment4Windows("LPT\u00B2.log")); + Assert.assertFalse(isLegalPathSegment4Windows("name\tpart")); + Assert.assertFalse(isLegalPathSegment4Windows("name" + Character.toString((char) 0) + "part")); + + Assert.assertTrue(isLegalPathSegment4Windows("COM0")); + Assert.assertTrue(isLegalPathSegment4Windows("LPT0")); + Assert.assertTrue(isLegalPathSegment4Windows("COM\u00B4")); + } +} From 56a0fcce1c93432aac1af8105fd663f37af3068b Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 10 Jun 2026 14:20:31 +0800 Subject: [PATCH 098/102] [to dev/1.3] Fix tree model WAL serialized sizes (#17886) * [to dev/1.3] Fix tree model WAL serialized sizes * Fix WAL charset deserialization * Fix insert node serde charset handling * spotless --- .../plan/node/write/DeleteDataNode.java | 12 +- .../planner/plan/node/write/InsertNode.java | 105 +++++++++++++++++- .../plan/node/write/InsertRowNode.java | 34 +++--- .../node/write/InsertRowsOfOneDeviceNode.java | 7 +- .../plan/node/write/InsertTabletNode.java | 27 +++-- .../dataregion/memtable/AbstractMemTable.java | 5 +- .../memtable/WritableMemChunkGroup.java | 8 +- .../dataregion/modification/Deletion.java | 39 ++++++- .../dataregion/wal/buffer/WALInfoEntry.java | 9 +- .../dataregion/wal/utils/WALReadUtils.java | 101 +++++++++++++++++ .../dataregion/wal/utils/WALWriteUtils.java | 16 ++- .../node/write/DeleteDataNodeSerdeTest.java | 28 +++++ .../node/write/InsertRowNodeSerdeTest.java | 12 +- .../node/write/InsertTabletNodeSerdeTest.java | 12 +- .../dataregion/modification/DeletionTest.java | 52 +++++++++ .../dataregion/wal/io/WALFileTest.java | 12 +- 16 files changed, 406 insertions(+), 73 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALReadUtils.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/modification/DeletionTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java index c4c4159f66dea..dda2d245f0921 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java @@ -36,6 +36,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALReadUtils; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.tsfile.read.filter.factory.TimeFilterApi; @@ -129,7 +130,7 @@ public List getOutputColumnNames() { public int serializedSize() { int size = FIXED_SERIALIZED_SIZE; for (PartialPath path : pathList) { - size += ReadWriteIOUtils.sizeToWrite(path.getFullPath()); + size += WALWriteUtils.sizeToWrite(path.getFullPath()); } return size; } @@ -153,10 +154,9 @@ public static DeleteDataNode deserializeFromWAL(DataInputStream stream) throws I for (int i = 0; i < size; i++) { try { pathList.add( - DataNodeDevicePathCache.getInstance() - .getPartialPath(ReadWriteIOUtils.readString(stream))); + DataNodeDevicePathCache.getInstance().getPartialPath(WALReadUtils.readString(stream))); } catch (IllegalPathException e) { - throw new IllegalArgumentException("Cannot deserialize InsertRowNode", e); + throw new IllegalArgumentException("Cannot deserialize DeleteDataNode", e); } } long deleteStartTime = stream.readLong(); @@ -174,9 +174,9 @@ public static DeleteDataNode deserializeFromWAL(ByteBuffer buffer) { List pathList = new ArrayList<>(size); for (int i = 0; i < size; i++) { try { - pathList.add(new PartialPath(ReadWriteIOUtils.readString(buffer))); + pathList.add(new PartialPath(WALReadUtils.readString(buffer))); } catch (IllegalPathException e) { - throw new IllegalArgumentException("Cannot deserialize InsertRowNode", e); + throw new IllegalArgumentException("Cannot deserialize DeleteDataNode", e); } } long deleteStartTime = buffer.getLong(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java index c664bbb9ddfaa..aee5cc1ec9384 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java @@ -31,11 +31,14 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.storageengine.dataregion.memtable.DeviceIDFactory; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALReadUtils; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; +import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.exception.NotImplementedException; import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.write.schema.MeasurementSchema; import java.io.DataInputStream; @@ -43,7 +46,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; @@ -213,6 +218,102 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { throw new NotImplementedException("serializeAttributes of InsertNode is not implemented"); } + protected static int serializeString(String value, ByteBuffer buffer) { + if (value == null) { + return ReadWriteIOUtils.write(-1, buffer); + } + byte[] bytes = value.getBytes(TSFileConfig.STRING_CHARSET); + int len = ReadWriteIOUtils.write(bytes.length, buffer); + buffer.put(bytes); + return len + bytes.length; + } + + protected static int serializeString(String value, DataOutputStream stream) throws IOException { + if (value == null) { + return ReadWriteIOUtils.write(-1, stream); + } + byte[] bytes = value.getBytes(TSFileConfig.STRING_CHARSET); + int len = ReadWriteIOUtils.write(bytes.length, stream); + stream.write(bytes); + return len + bytes.length; + } + + protected static String deserializeString(ByteBuffer buffer) { + int strLength = ReadWriteIOUtils.readInt(buffer); + if (strLength < 0) { + return null; + } else if (strLength == 0) { + return ""; + } + byte[] bytes = new byte[strLength]; + buffer.get(bytes); + return new String(bytes, TSFileConfig.STRING_CHARSET); + } + + protected static void serializeMeasurementSchema( + MeasurementSchema measurementSchema, ByteBuffer buffer) { + serializeString(measurementSchema.getMeasurementId(), buffer); + ReadWriteIOUtils.write(measurementSchema.getTypeInByte(), buffer); + ReadWriteIOUtils.write(measurementSchema.getEncodingType().serialize(), buffer); + ReadWriteIOUtils.write(measurementSchema.getCompressor().serialize(), buffer); + serializeProps(measurementSchema.getProps(), buffer); + } + + protected static void serializeMeasurementSchema( + MeasurementSchema measurementSchema, DataOutputStream stream) throws IOException { + serializeString(measurementSchema.getMeasurementId(), stream); + ReadWriteIOUtils.write(measurementSchema.getTypeInByte(), stream); + ReadWriteIOUtils.write(measurementSchema.getEncodingType().serialize(), stream); + ReadWriteIOUtils.write(measurementSchema.getCompressor().serialize(), stream); + serializeProps(measurementSchema.getProps(), stream); + } + + protected static MeasurementSchema deserializeMeasurementSchema(ByteBuffer buffer) { + String measurementId = deserializeString(buffer); + byte type = ReadWriteIOUtils.readByte(buffer); + byte encoding = ReadWriteIOUtils.readByte(buffer); + byte compressor = ReadWriteIOUtils.readByte(buffer); + Map props = deserializeProps(buffer); + return new MeasurementSchema(measurementId, type, encoding, compressor, props); + } + + private static void serializeProps(Map props, ByteBuffer buffer) { + if (props == null) { + ReadWriteIOUtils.write(0, buffer); + return; + } + ReadWriteIOUtils.write(props.size(), buffer); + for (Map.Entry entry : props.entrySet()) { + serializeString(entry.getKey(), buffer); + serializeString(entry.getValue(), buffer); + } + } + + private static void serializeProps(Map props, DataOutputStream stream) + throws IOException { + if (props == null) { + ReadWriteIOUtils.write(0, stream); + return; + } + ReadWriteIOUtils.write(props.size(), stream); + for (Map.Entry entry : props.entrySet()) { + serializeString(entry.getKey(), stream); + serializeString(entry.getValue(), stream); + } + } + + private static Map deserializeProps(ByteBuffer buffer) { + int size = ReadWriteIOUtils.readInt(buffer); + if (size <= 0) { + return null; + } + Map props = new HashMap<>(); + for (int i = 0; i < size; i++) { + props.put(deserializeString(buffer), deserializeString(buffer)); + } + return props; + } + // region Serialization methods for WAL /** Serialized size of measurement schemas, ignoring failed time series */ protected int serializeMeasurementSchemasSize() { @@ -244,7 +345,7 @@ protected void serializeMeasurementSchemasToWAL(IWALByteBufferView buffer) { */ protected void deserializeMeasurementSchemas(DataInputStream stream) throws IOException { for (int i = 0; i < measurements.length; i++) { - measurementSchemas[i] = MeasurementSchema.deserializeFrom(stream); + measurementSchemas[i] = WALReadUtils.readMeasurementSchema(stream); measurements[i] = measurementSchemas[i].getMeasurementId(); dataTypes[i] = measurementSchemas[i].getType(); } @@ -252,7 +353,7 @@ protected void deserializeMeasurementSchemas(DataInputStream stream) throws IOEx protected void deserializeMeasurementSchemas(ByteBuffer buffer) { for (int i = 0; i < measurements.length; i++) { - measurementSchemas[i] = MeasurementSchema.deserializeFrom(buffer); + measurementSchemas[i] = WALReadUtils.readMeasurementSchema(buffer); measurements[i] = measurementSchemas[i].getMeasurementId(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java index 6518385c48a86..23c17808d493c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java @@ -34,6 +34,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALReadUtils; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.iotdb.db.utils.TypeInferenceUtils; @@ -238,13 +239,13 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { void subSerialize(ByteBuffer buffer) { ReadWriteIOUtils.write(time, buffer); - ReadWriteIOUtils.write(devicePath.getFullPath(), buffer); + serializeString(devicePath.getFullPath(), buffer); serializeMeasurementsAndValues(buffer); } void subSerialize(DataOutputStream stream) throws IOException { ReadWriteIOUtils.write(time, stream); - ReadWriteIOUtils.write(devicePath.getFullPath(), stream); + serializeString(devicePath.getFullPath(), stream); serializeMeasurementsAndValues(stream); } @@ -281,9 +282,9 @@ private void serializeMeasurementsOrSchemas(ByteBuffer buffer) { } // serialize measurement schemas when exist if (measurementSchemas != null) { - measurementSchemas[i].serializeTo(buffer); + serializeMeasurementSchema(measurementSchemas[i], buffer); } else { - ReadWriteIOUtils.write(measurements[i], buffer); + serializeString(measurements[i], buffer); } } } @@ -303,9 +304,9 @@ private void serializeMeasurementsOrSchemas(DataOutputStream stream) throws IOEx } // serialize measurement schemas when exist if (measurementSchemas != null) { - measurementSchemas[i].serializeTo(stream); + serializeMeasurementSchema(measurementSchemas[i], stream); } else { - ReadWriteIOUtils.write(measurements[i], stream); + serializeString(measurements[i], stream); } } } @@ -331,7 +332,7 @@ private void putDataTypesAndValues(ByteBuffer buffer) { // and is forwarded to other nodes if (isNeedInferType) { ReadWriteIOUtils.write(TYPE_RAW_STRING, buffer); - ReadWriteIOUtils.write(values[i].toString(), buffer); + serializeString(values[i].toString(), buffer); } else { ReadWriteIOUtils.write(dataTypes[i], buffer); switch (dataTypes[i]) { @@ -386,7 +387,7 @@ private void putDataTypesAndValues(DataOutputStream stream) throws IOException { // and is forwarded to other nodes if (isNeedInferType) { ReadWriteIOUtils.write(TYPE_RAW_STRING, stream); - ReadWriteIOUtils.write(values[i].toString(), stream); + serializeString(values[i].toString(), stream); } else { ReadWriteIOUtils.write(dataTypes[i], stream); switch (dataTypes[i]) { @@ -431,8 +432,7 @@ void subDeserialize(ByteBuffer byteBuffer) { time = byteBuffer.getLong(); try { devicePath = - DataNodeDevicePathCache.getInstance() - .getPartialPath(ReadWriteIOUtils.readString(byteBuffer)); + DataNodeDevicePathCache.getInstance().getPartialPath(deserializeString(byteBuffer)); } catch (IllegalPathException e) { throw new IllegalArgumentException(DESERIALIZE_ERROR, e); } @@ -447,12 +447,12 @@ void deserializeMeasurementsAndValues(ByteBuffer buffer) { if (hasSchema) { measurementSchemas = new MeasurementSchema[measurementSize]; for (int i = 0; i < measurementSize; i++) { - measurementSchemas[i] = MeasurementSchema.deserializeFrom(buffer); + measurementSchemas[i] = deserializeMeasurementSchema(buffer); measurements[i] = measurementSchemas[i].getMeasurementId(); } } else { for (int i = 0; i < measurementSize; i++) { - measurements[i] = ReadWriteIOUtils.readString(buffer); + measurements[i] = deserializeString(buffer); } } @@ -476,7 +476,7 @@ private void fillDataTypesAndValues(ByteBuffer buffer) { // and is forwarded to other nodes byte typeNum = (byte) ReadWriteIOUtils.read(buffer); if (typeNum == TYPE_RAW_STRING || typeNum == TYPE_NULL) { - values[i] = typeNum == TYPE_RAW_STRING ? ReadWriteIOUtils.readString(buffer) : null; + values[i] = typeNum == TYPE_RAW_STRING ? deserializeString(buffer) : null; continue; } dataTypes[i] = TSDataType.values()[typeNum]; @@ -524,7 +524,7 @@ public int serializedSize() { protected int subSerializeSize() { int size = 0; size += Long.BYTES; - size += ReadWriteIOUtils.sizeToWrite(devicePath.getFullPath()); + size += WALWriteUtils.sizeToWrite(devicePath.getFullPath()); return size + serializeMeasurementsAndValuesSize(); } @@ -671,8 +671,7 @@ protected static InsertRowNode subDeserializeFromWAL(DataInputStream stream) thr insertNode.setTime(stream.readLong()); try { insertNode.setDevicePath( - DataNodeDevicePathCache.getInstance() - .getPartialPath(ReadWriteIOUtils.readString(stream))); + DataNodeDevicePathCache.getInstance().getPartialPath(WALReadUtils.readString(stream))); } catch (IllegalPathException e) { throw new IllegalArgumentException(DESERIALIZE_ERROR, e); } @@ -757,8 +756,7 @@ protected static InsertRowNode subDeserializeFromWAL(ByteBuffer buffer) { insertNode.setTime(buffer.getLong()); try { insertNode.setDevicePath( - DataNodeDevicePathCache.getInstance() - .getPartialPath(ReadWriteIOUtils.readString(buffer))); + DataNodeDevicePathCache.getInstance().getPartialPath(WALReadUtils.readString(buffer))); } catch (IllegalPathException e) { throw new IllegalArgumentException(DESERIALIZE_ERROR, e); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java index c7b193841d45c..d7a6e8b6f5f59 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java @@ -236,8 +236,7 @@ public static InsertRowsOfOneDeviceNode deserialize(ByteBuffer byteBuffer) { try { devicePath = - DataNodeDevicePathCache.getInstance() - .getPartialPath((ReadWriteIOUtils.readString(byteBuffer))); + DataNodeDevicePathCache.getInstance().getPartialPath((deserializeString(byteBuffer))); } catch (IllegalPathException e) { throw new IllegalArgumentException("Cannot deserialize InsertRowsOfOneDeviceNode", e); } @@ -269,7 +268,7 @@ public static InsertRowsOfOneDeviceNode deserialize(ByteBuffer byteBuffer) { @Override protected void serializeAttributes(ByteBuffer byteBuffer) { PlanNodeType.INSERT_ROWS_OF_ONE_DEVICE.serialize(byteBuffer); - ReadWriteIOUtils.write(devicePath.getFullPath(), byteBuffer); + serializeString(devicePath.getFullPath(), byteBuffer); ReadWriteIOUtils.write(insertRowNodeList.size(), byteBuffer); @@ -285,7 +284,7 @@ protected void serializeAttributes(ByteBuffer byteBuffer) { @Override protected void serializeAttributes(DataOutputStream stream) throws IOException { PlanNodeType.INSERT_ROWS_OF_ONE_DEVICE.serialize(stream); - ReadWriteIOUtils.write(devicePath.getFullPath(), stream); + serializeString(devicePath.getFullPath(), stream); ReadWriteIOUtils.write(insertRowNodeList.size(), stream); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java index 6a1e9bf1bb48c..472c1e79d7e47 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java @@ -36,6 +36,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALReadUtils; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.iotdb.db.utils.BitMapUtils; import org.apache.iotdb.db.utils.QueryDataSetUtils; @@ -413,7 +414,7 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { } void subSerialize(ByteBuffer buffer) { - ReadWriteIOUtils.write(devicePath.getFullPath(), buffer); + serializeString(devicePath.getFullPath(), buffer); writeMeasurementsOrSchemas(buffer); writeDataTypes(buffer); writeTimes(buffer); @@ -423,7 +424,7 @@ void subSerialize(ByteBuffer buffer) { } void subSerialize(DataOutputStream stream) throws IOException { - ReadWriteIOUtils.write(devicePath.getFullPath(), stream); + serializeString(devicePath.getFullPath(), stream); writeMeasurementsOrSchemas(stream); writeDataTypes(stream); writeTimes(stream); @@ -444,9 +445,9 @@ private void writeMeasurementsOrSchemas(ByteBuffer buffer) { } // serialize measurement schemas when exist if (measurementSchemas != null) { - measurementSchemas[i].serializeTo(buffer); + serializeMeasurementSchema(measurementSchemas[i], buffer); } else { - ReadWriteIOUtils.write(measurements[i], buffer); + serializeString(measurements[i], buffer); } } } @@ -463,9 +464,9 @@ private void writeMeasurementsOrSchemas(DataOutputStream stream) throws IOExcept } // serialize measurement schemas when exist if (measurementSchemas != null) { - measurementSchemas[i].serializeTo(stream); + serializeMeasurementSchema(measurementSchemas[i], stream); } else { - ReadWriteIOUtils.write(measurements[i], stream); + serializeString(measurements[i], stream); } } } @@ -682,8 +683,7 @@ public static InsertTabletNode deserialize(ByteBuffer byteBuffer) { public void subDeserialize(ByteBuffer buffer) { try { devicePath = - DataNodeDevicePathCache.getInstance() - .getPartialPath((ReadWriteIOUtils.readString(buffer))); + DataNodeDevicePathCache.getInstance().getPartialPath((deserializeString(buffer))); } catch (IllegalPathException e) { throw new IllegalArgumentException("Cannot deserialize InsertTabletNode", e); } @@ -695,12 +695,12 @@ public void subDeserialize(ByteBuffer buffer) { if (hasSchema) { this.measurementSchemas = new MeasurementSchema[measurementSize]; for (int i = 0; i < measurementSize; i++) { - measurementSchemas[i] = MeasurementSchema.deserializeFrom(buffer); + measurementSchemas[i] = deserializeMeasurementSchema(buffer); measurements[i] = measurementSchemas[i].getMeasurementId(); } } else { for (int i = 0; i < measurementSize; i++) { - measurements[i] = ReadWriteIOUtils.readString(buffer); + measurements[i] = deserializeString(buffer); } } @@ -738,7 +738,7 @@ public int serializedSize(int start, int end) { int subSerializeSize(int start, int end) { int size = 0; size += Long.BYTES; - size += ReadWriteIOUtils.sizeToWrite(devicePath.getFullPath()); + size += WALWriteUtils.sizeToWrite(devicePath.getFullPath()); // measurements size size += Integer.BYTES; size += serializeMeasurementSchemasSize(); @@ -943,7 +943,7 @@ private void subDeserializeFromWAL(DataInputStream stream) throws IOException { searchIndex = stream.readLong(); try { devicePath = - DataNodeDevicePathCache.getInstance().getPartialPath(ReadWriteIOUtils.readString(stream)); + DataNodeDevicePathCache.getInstance().getPartialPath(WALReadUtils.readString(stream)); } catch (IllegalPathException e) { throw new IllegalArgumentException("Cannot deserialize InsertTabletNode", e); } @@ -979,8 +979,7 @@ private void subDeserializeFromWAL(ByteBuffer buffer) { searchIndex = buffer.getLong(); try { devicePath = - DataNodeDevicePathCache.getInstance() - .getPartialPath((ReadWriteIOUtils.readString(buffer))); + DataNodeDevicePathCache.getInstance().getPartialPath((WALReadUtils.readString(buffer))); } catch (IllegalPathException e) { throw new IllegalArgumentException("Cannot deserialize InsertTabletNode", e); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java index eb49d5325b0b3..ab95fc57662aa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java @@ -40,6 +40,7 @@ import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.MemAlignedChunkHandleImpl; import org.apache.iotdb.db.storageengine.dataregion.read.filescan.impl.MemChunkHandleImpl; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALReadUtils; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.iotdb.db.utils.MemUtils; import org.apache.iotdb.db.utils.ModificationUtils; @@ -901,7 +902,7 @@ public int serializedSize() { } int size = FIXED_SERIALIZED_SIZE; for (Map.Entry entry : memTableMap.entrySet()) { - size += ReadWriteIOUtils.sizeToWrite(((PlainDeviceID) entry.getKey()).toStringID()); + size += WALWriteUtils.sizeToWrite(((PlainDeviceID) entry.getKey()).toStringID()); size += Byte.BYTES; size += entry.getValue().serializedSize(); } @@ -948,7 +949,7 @@ protected void deserialize(DataInputStream stream, boolean multiTvListMemChunk) int memTableMapSize = stream.readInt(); for (int i = 0; i < memTableMapSize; ++i) { - IDeviceID deviceID = deviceIDFactory.getDeviceID(ReadWriteIOUtils.readString(stream)); + IDeviceID deviceID = deviceIDFactory.getDeviceID(WALReadUtils.readString(stream)); boolean isAligned = ReadWriteIOUtils.readBool(stream); IWritableMemChunkGroup memChunkGroup; if (multiTvListMemChunk) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java index 226c2fd398098..ff519a2206aa1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/WritableMemChunkGroup.java @@ -22,10 +22,10 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.path.PathPatternUtil; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALReadUtils; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.tsfile.utils.BitMap; -import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.write.schema.IMeasurementSchema; import java.io.DataInputStream; @@ -180,7 +180,7 @@ public int serializedSize() { int size = 0; size += Integer.BYTES; for (Map.Entry entry : memChunkMap.entrySet()) { - size += ReadWriteIOUtils.sizeToWrite(entry.getKey()); + size += WALWriteUtils.sizeToWrite(entry.getKey()); size += entry.getValue().serializedSize(); } return size; @@ -200,7 +200,7 @@ public static WritableMemChunkGroup deserialize(DataInputStream stream) throws I WritableMemChunkGroup memChunkGroup = new WritableMemChunkGroup(); int memChunkMapSize = stream.readInt(); for (int i = 0; i < memChunkMapSize; ++i) { - String measurement = ReadWriteIOUtils.readString(stream); + String measurement = WALReadUtils.readString(stream); IWritableMemChunk memChunk = WritableMemChunk.deserialize(stream); memChunkGroup.memChunkMap.put(measurement, memChunk); } @@ -212,7 +212,7 @@ public static WritableMemChunkGroup deserializeSingleTVListMemChunks(DataInputSt WritableMemChunkGroup memChunkGroup = new WritableMemChunkGroup(); int memChunkMapSize = stream.readInt(); for (int i = 0; i < memChunkMapSize; ++i) { - String measurement = ReadWriteIOUtils.readString(stream); + String measurement = WALReadUtils.readString(stream); IWritableMemChunk memChunk = WritableMemChunk.deserializeSingleTVListMemChunks(stream); memChunkGroup.memChunkMap.put(measurement, memChunk); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/Deletion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/Deletion.java index 07f8aecbfc026..c4c74df1623e2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/Deletion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/modification/Deletion.java @@ -24,10 +24,10 @@ import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.db.queryengine.execution.MemoryEstimationHelper; +import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.utils.RamUsageEstimator; -import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataInputStream; import java.io.DataOutputStream; @@ -113,7 +113,7 @@ public long serializeWithoutFileOffset(DataOutputStream stream) throws IOExcepti serializeSize += Long.BYTES; stream.writeLong(getEndTime()); serializeSize += Long.BYTES; - serializeSize += ReadWriteIOUtils.write(getPathString(), stream); + serializeSize += writeString(getPathString(), stream); return serializeSize; } @@ -121,8 +121,30 @@ public static Deletion deserializeWithoutFileOffset(DataInputStream stream) throws IOException, IllegalPathException { long startTime = stream.readLong(); long endTime = stream.readLong(); - return new Deletion( - getMeasurementPath(ReadWriteIOUtils.readString(stream)), 0, startTime, endTime); + return new Deletion(getMeasurementPath(readString(stream)), 0, startTime, endTime); + } + + private static int writeString(String value, DataOutputStream stream) throws IOException { + if (value == null) { + stream.writeInt(-1); + return Integer.BYTES; + } + byte[] bytes = value.getBytes(TSFileConfig.STRING_CHARSET); + stream.writeInt(bytes.length); + stream.write(bytes); + return Integer.BYTES + bytes.length; + } + + private static String readString(DataInputStream stream) throws IOException { + int strLength = stream.readInt(); + if (strLength < 0) { + return null; + } else if (strLength == 0) { + return ""; + } + byte[] bytes = new byte[strLength]; + stream.readFully(bytes); + return new String(bytes, TSFileConfig.STRING_CHARSET); } private static PartialPath getMeasurementPath(String path) throws IllegalPathException { @@ -137,7 +159,14 @@ private static PartialPath getMeasurementPath(String path) throws IllegalPathExc } public long getSerializedSize() { - return Long.BYTES * 2 + Integer.BYTES + (long) getPathString().length() * Character.BYTES; + return Long.BYTES * 2L + sizeToWriteString(getPathString()); + } + + private static int sizeToWriteString(String value) { + if (value == null) { + return Integer.BYTES; + } + return Integer.BYTES + value.getBytes(TSFileConfig.STRING_CHARSET).length; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALInfoEntry.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALInfoEntry.java index 15c8faa8993d7..a9af8f81c7d62 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALInfoEntry.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALInfoEntry.java @@ -64,7 +64,14 @@ public WALInfoEntry(long memTableId, InsertTabletNode value, int tabletStart, in @Override public int serializedSize() { - return FIXED_SERIALIZED_SIZE + (value == null ? 0 : value.serializedSize()); + if (value == null) { + return FIXED_SERIALIZED_SIZE; + } + if (value instanceof InsertTabletNode && tabletInfo != null) { + return FIXED_SERIALIZED_SIZE + + ((InsertTabletNode) value).serializedSize(tabletInfo.tabletStart, tabletInfo.tabletEnd); + } + return FIXED_SERIALIZED_SIZE + value.serializedSize(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALReadUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALReadUtils.java new file mode 100644 index 0000000000000..6d467f19934a5 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALReadUtils.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.wal.utils; + +import org.apache.tsfile.common.conf.TSFileConfig; +import org.apache.tsfile.write.schema.MeasurementSchema; + +import java.io.DataInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +/** Read methods paired with {@link WALWriteUtils}. */ +public class WALReadUtils { + + private WALReadUtils() {} + + public static String readString(DataInputStream stream) throws IOException { + int strLength = stream.readInt(); + if (strLength < 0) { + return null; + } else if (strLength == 0) { + return ""; + } + byte[] bytes = new byte[strLength]; + stream.readFully(bytes); + return new String(bytes, TSFileConfig.STRING_CHARSET); + } + + public static String readString(ByteBuffer buffer) { + int strLength = buffer.getInt(); + if (strLength < 0) { + return null; + } else if (strLength == 0) { + return ""; + } + byte[] bytes = new byte[strLength]; + buffer.get(bytes); + return new String(bytes, TSFileConfig.STRING_CHARSET); + } + + public static MeasurementSchema readMeasurementSchema(DataInputStream stream) throws IOException { + String measurementId = readString(stream); + byte type = stream.readByte(); + byte encoding = stream.readByte(); + byte compressor = stream.readByte(); + Map props = readProps(stream); + return new MeasurementSchema(measurementId, type, encoding, compressor, props); + } + + public static MeasurementSchema readMeasurementSchema(ByteBuffer buffer) { + String measurementId = readString(buffer); + byte type = buffer.get(); + byte encoding = buffer.get(); + byte compressor = buffer.get(); + Map props = readProps(buffer); + return new MeasurementSchema(measurementId, type, encoding, compressor, props); + } + + private static Map readProps(DataInputStream stream) throws IOException { + int size = stream.readInt(); + if (size <= 0) { + return null; + } + Map props = new HashMap<>(); + for (int i = 0; i < size; i++) { + props.put(readString(stream), readString(stream)); + } + return props; + } + + private static Map readProps(ByteBuffer buffer) { + int size = buffer.getInt(); + if (size <= 0) { + return null; + } + Map props = new HashMap<>(); + for (int i = 0; i < size; i++) { + props.put(readString(buffer), readString(buffer)); + } + return props; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALWriteUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALWriteUtils.java index e6c0eb02722ae..72dd2671fa313 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALWriteUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALWriteUtils.java @@ -21,6 +21,7 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; +import org.apache.tsfile.common.conf.TSFileConfig; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.file.metadata.enums.CompressionType; @@ -127,13 +128,20 @@ public static int write(String s, IWALByteBufferView buffer) { return write(NO_BYTE_TO_READ, buffer); } int len = 0; - byte[] bytes = s.getBytes(); + byte[] bytes = s.getBytes(TSFileConfig.STRING_CHARSET); len += write(bytes.length, buffer); buffer.put(bytes); len += bytes.length; return len; } + public static int sizeToWrite(String s) { + if (s == null) { + return INT_LEN; + } + return INT_LEN + s.getBytes(TSFileConfig.STRING_CHARSET).length; + } + /** * Write IDeviceID to byteBuffer. * @@ -196,15 +204,15 @@ public static int write(MeasurementSchema measurementSchema, IWALByteBufferView public static int sizeToWrite(MeasurementSchema measurementSchema) { int byteLen = 0; - byteLen += ReadWriteIOUtils.sizeToWrite(measurementSchema.getMeasurementId()); + byteLen += sizeToWrite(measurementSchema.getMeasurementId()); byteLen += 3 * Byte.BYTES; Map props = measurementSchema.getProps(); byteLen += Integer.BYTES; if (props != null) { for (Map.Entry entry : props.entrySet()) { - byteLen += ReadWriteIOUtils.sizeToWrite(entry.getKey()); - byteLen += ReadWriteIOUtils.sizeToWrite(entry.getValue()); + byteLen += sizeToWrite(entry.getKey()); + byteLen += sizeToWrite(entry.getValue()); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/DeleteDataNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/DeleteDataNodeSerdeTest.java index cbfba2a687914..3087085538326 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/DeleteDataNodeSerdeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/DeleteDataNodeSerdeTest.java @@ -25,10 +25,14 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; +import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALByteBufferForTest; import org.junit.Assert; import org.junit.Test; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -64,4 +68,28 @@ public void testSerializeAndDeserialize() throws IllegalPathException { Assert.assertEquals(pathList.get(i), deserializedPathList.get(i)); } } + + @Test + public void testSerializeAndDeserializeForWAL() throws IllegalPathException, IOException { + long startTime = 1; + long endTime = 10; + List pathList = new ArrayList<>(); + pathList.add(new PartialPath("root.\u6570\u636e\u5e93.d1.\u6e29\u5ea6")); + pathList.add(new PartialPath("root.\u6570\u636e\u5e93.d2.*")); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("DeleteDataNode"), pathList, startTime, endTime); + + ByteBuffer byteBuffer = ByteBuffer.allocate(deleteDataNode.serializedSize()); + deleteDataNode.serializeToWAL(new WALByteBufferForTest(byteBuffer)); + Assert.assertEquals(deleteDataNode.serializedSize(), byteBuffer.position()); + + DataInputStream dataInputStream = + new DataInputStream(new ByteArrayInputStream(byteBuffer.array())); + Assert.assertEquals(PlanNodeType.DELETE_DATA.getNodeType(), dataInputStream.readShort()); + + DeleteDataNode deserializedNode = DeleteDataNode.deserializeFromWAL(dataInputStream); + Assert.assertEquals(startTime, deserializedNode.getDeleteStartTime()); + Assert.assertEquals(endTime, deserializedNode.getDeleteEndTime()); + Assert.assertEquals(pathList, deserializedNode.getPathList()); + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/InsertRowNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/InsertRowNodeSerdeTest.java index 523b061428241..a4e19d9717629 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/InsertRowNodeSerdeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/InsertRowNodeSerdeTest.java @@ -89,8 +89,8 @@ public void TestSerializeAndDeserializeForWAL() throws IllegalPathException, IOE tmpNode.setPlanNodeId(insertRowNode.getPlanNodeId()); tmpNode.setMeasurementSchemas( new MeasurementSchema[] { - new MeasurementSchema("s1", TSDataType.DOUBLE), - new MeasurementSchema("s2", TSDataType.FLOAT), + new MeasurementSchema("\u6e29\u5ea6", TSDataType.DOUBLE), + new MeasurementSchema("\u6e7f\u5ea6", TSDataType.FLOAT), new MeasurementSchema("s3", TSDataType.INT64), new MeasurementSchema("s4", TSDataType.INT32), new MeasurementSchema("s5", TSDataType.BOOLEAN) @@ -148,9 +148,9 @@ private InsertRowNode getInsertRowNodeWithMeasurementSchemas() throws IllegalPat InsertRowNode insertRowNode = new InsertRowNode( new PlanNodeId("plannode 2"), - new PartialPath("root.isp.d2"), + new PartialPath("root.\u6570\u636e\u5e93.d2"), false, - new String[] {"s1", "s2", "s3", "s4", "s5"}, + new String[] {"\u6e29\u5ea6", "\u6e7f\u5ea6", "s3", "s4", "s5"}, dataTypes, time, columns, @@ -158,8 +158,8 @@ private InsertRowNode getInsertRowNodeWithMeasurementSchemas() throws IllegalPat insertRowNode.setMeasurementSchemas( new MeasurementSchema[] { - new MeasurementSchema("s1", TSDataType.DOUBLE), - new MeasurementSchema("s2", TSDataType.FLOAT), + new MeasurementSchema("\u6e29\u5ea6", TSDataType.DOUBLE), + new MeasurementSchema("\u6e7f\u5ea6", TSDataType.FLOAT), new MeasurementSchema("s3", TSDataType.INT64), new MeasurementSchema("s4", TSDataType.INT32), new MeasurementSchema("s5", TSDataType.BOOLEAN) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/InsertTabletNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/InsertTabletNodeSerdeTest.java index 754e7a6e1386d..a883b7d24a87e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/InsertTabletNodeSerdeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/InsertTabletNodeSerdeTest.java @@ -84,8 +84,8 @@ public void testSerializeAndDeserializeForWAL() throws IllegalPathException, IOE tmpNode.setMeasurementSchemas( new MeasurementSchema[] { - new MeasurementSchema("s1", TSDataType.DOUBLE), - new MeasurementSchema("s2", TSDataType.FLOAT), + new MeasurementSchema("\u6e29\u5ea6", TSDataType.DOUBLE), + new MeasurementSchema("\u6e7f\u5ea6", TSDataType.FLOAT), new MeasurementSchema("s3", TSDataType.INT64), new MeasurementSchema("s4", TSDataType.INT32), new MeasurementSchema("s5", TSDataType.BOOLEAN) @@ -191,9 +191,9 @@ private InsertTabletNode getInsertTabletNodeWithSchema() throws IllegalPathExcep InsertTabletNode insertTabletNode = new InsertTabletNode( new PlanNodeId("plannode 1"), - new PartialPath("root.isp.d1"), + new PartialPath("root.\u6570\u636e\u5e93.d1"), false, - new String[] {"s1", "s2", "s3", "s4", "s5"}, + new String[] {"\u6e29\u5ea6", "\u6e7f\u5ea6", "s3", "s4", "s5"}, dataTypes, times, null, @@ -201,8 +201,8 @@ private InsertTabletNode getInsertTabletNodeWithSchema() throws IllegalPathExcep times.length); insertTabletNode.setMeasurementSchemas( new MeasurementSchema[] { - new MeasurementSchema("s1", TSDataType.DOUBLE), - new MeasurementSchema("s2", TSDataType.FLOAT), + new MeasurementSchema("\u6e29\u5ea6", TSDataType.DOUBLE), + new MeasurementSchema("\u6e7f\u5ea6", TSDataType.FLOAT), new MeasurementSchema("s3", TSDataType.INT64), new MeasurementSchema("s4", TSDataType.INT32), new MeasurementSchema("s5", TSDataType.BOOLEAN) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/modification/DeletionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/modification/DeletionTest.java new file mode 100644 index 0000000000000..c8c1a7fd0ca8d --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/modification/DeletionTest.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.modification; + +import org.apache.iotdb.commons.path.PartialPath; + +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; + +import static org.junit.Assert.assertEquals; + +public class DeletionTest { + + @Test + public void testSerializedSize() throws Exception { + Deletion deletion = + new Deletion(new PartialPath("root.\u6570\u636e\u5e93.d1.\u6e29\u5ea6"), 0, 1, 5); + + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + long serializedSize = + deletion.serializeWithoutFileOffset(new DataOutputStream(byteArrayOutputStream)); + byte[] bytes = byteArrayOutputStream.toByteArray(); + + assertEquals(deletion.getSerializedSize(), serializedSize); + assertEquals(deletion.getSerializedSize(), bytes.length); + assertEquals( + deletion, + Deletion.deserializeWithoutFileOffset( + new DataInputStream(new ByteArrayInputStream(bytes)))); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALFileTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALFileTest.java index f483bcfd49265..8e2f45ce5564c 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALFileTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALFileTest.java @@ -61,7 +61,7 @@ public class WALFileTest { new File( TestConstant.BASE_OUTPUT_PATH.concat( WALFileUtils.getLogFileName(0, 0, WALFileStatus.CONTAINS_SEARCH_INDEX))); - private final String devicePath = "root.test_sg.test_d"; + private final String devicePath = "root.\u6570\u636e\u5e93.test_d"; @Before public void setUp() throws Exception { @@ -171,6 +171,16 @@ public void testReadMetadataFromBrokenFile() throws IOException { assertTrue(walMetaData.getMemTablesId().isEmpty()); } + @Test + public void testInsertTabletEntrySerializedSizeWithRange() + throws IOException, IllegalPathException { + WALEntry walEntry = new WALInfoEntry(1, getInsertTabletNode(devicePath), 1, 3); + ByteBuffer byteBuffer = ByteBuffer.allocate(walEntry.serializedSize()); + WALByteBufferForTest buffer = new WALByteBufferForTest(byteBuffer); + walEntry.serialize(buffer); + assertEquals(walEntry.serializedSize(), byteBuffer.position()); + } + public static InsertRowNode getInsertRowNode(String devicePath) throws IllegalPathException { long time = 110L; TSDataType[] dataTypes = From 85b7038f4d7853f92543fe1485015332fb65ab6c Mon Sep 17 00:00:00 2001 From: Hongzhi Gao <761417898@qq.com> Date: Wed, 10 Jun 2026 14:31:54 +0800 Subject: [PATCH 099/102] [To dev/1.3] refactor client-cpp (#17803) * refactor(client-cpp): port SDK refactor to dev/1.3 without table model Reorganize the C++ client into include/session/rpc with CMake-driven build, embedded Thrift, Windows DLL packaging, SessionC API, and CI release workflows. Scope excludes 2.x table-model APIs and Thrift fields. * fix(client-cpp): align cmake plugin and clang-format for CI Replace pluginManagement cmake-maven-plugin with io.github 4.2.3-b1 so with-cpp Maven builds resolve the plugin. Pin clang.format.version to 17.0.6 and apply Spotless formatting to C++ sources. * fix(ci): pin clang-format 17 in multi-language-client cpp job ubuntu-latest ships clang-format 18.x which breaks Spotless when clang.format.version is 17.0.6. Install LLVM 17 clang-format on Linux, macOS, and Windows like refactor/cpp-client-sdk. * fix(build): bump Maven wrapper to 3.9.12 for cmake-maven-plugin cmake-maven-plugin 4.2.3-b1 requires Maven 3.9.9+. The wrapper was on 3.9.6, causing with-cpp CI to fail even when using ./mvnw. * fix(client-cpp): include vector in AbstractSessionBuilder.h GCC on Linux CI fails when Session.h includes AbstractSessionBuilder.h before vector; add missing #include for nodeUrls member. * fix(client-cpp): fix query result parsing and IT on IoTDB 1.3 Apply server columnNameIndexMap for TsBlock reads, align Time column handling with the Java client, and read text types by physical column type. Defer C API session open, disable auto-fetch in tests, and fall back when SHOW AVAILABLE URLS is unsupported. * fix(client-cpp): address PR review items for public API and Catch2 - Revert unrelated .gitignore changes (.run, .claude, relational-grammar) - Remove using namespace std from public headers Session.h and Common.h - Download Catch2 at build time via Maven/CMake instead of vendoring catch.hpp * style(client-cpp): apply clang-format to Session.h * fix(client-cpp): download Catch2 before CMake compile phase Run wget in generate-resources (not generate-test-resources, which runs after compile). CMake also downloads catch.hpp when the header is still missing. * fix(client-cpp): qualify std types for MSVC after removing using namespace std - SessionConnection.h/cpp: std::string, std::vector, std::exception, std::shared_ptr - IoTDBRpcDataSet.cpp: std::exception in catch handlers - Session.h: (std::max) to avoid Windows max macro conflict * fix(client-cpp): use sbin start/stop scripts on Windows Align Windows profile paths with the 1.3 distribution layout (scripts under sbin/, not windows/). * refactor(client-cpp): remove table-model docs and dead code from dev/1.3 port Align example READMEs with the tree-only scope, drop unused table-model API surface, and remove IDeviceID redirect helpers that have no callers. * fix(ci): auto-detect Visual Studio generator for cpp on Windows windows-latest now ships VS 2026 while pom.xml defaults to VS 2022. Use vswhere in the multi-language-client workflow and pass -Dcmake.generator so CMake matches the installed toolchain. * fix(ci): align cpp Windows matrix with refactor/cpp-client-sdk workflow Replace windows-latest and broken vswhere detection with explicit windows-2022 and windows-2025-vs2026 runners; pass Visual Studio 18 2026 generator only on the VS 2026 image. Also match ubuntu/mac matrix and add Spotless check from the refactor branch. * Sync C++ client packaging updates to 1.3 * Fix VS2017 C++ package Boost setup * Fix Windows C++ client CI dependencies * Fix C++ client workflow verification --- .../package-client-cpp-manylinux228.sh | 101 + .github/workflows/client-cpp-package.yml | 454 ++++ .github/workflows/multi-language-client.yml | 52 +- .gitignore | 10 +- .mvn/wrapper/maven-wrapper.properties | 2 +- distribution/pom.xml | 66 +- example/client-cpp-example/README.md | 248 +- example/client-cpp-example/README_zh.md | 252 ++ example/client-cpp-example/pom.xml | 136 +- .../src/AlignedTimeseriesSessionExample.cpp | 639 ++--- example/client-cpp-example/src/CMakeLists.txt | 137 +- .../src/MultiSvrNodeClient.cpp | 116 + .../client-cpp-example/src/SessionExample.cpp | 689 +++--- example/client-cpp-example/src/tree_example.c | 118 + iotdb-client/client-cpp/CMakeLists.txt | 271 ++ iotdb-client/client-cpp/README.md | 396 ++- .../client-cpp/cmake/FetchBoost.cmake | 130 + .../client-cpp/cmake/FetchBuildTools.cmake | 266 ++ .../client-cpp/cmake/FetchOpenSSL.cmake | 121 + .../client-cpp/cmake/FetchThrift.cmake | 277 +++ .../cmake/GenerateThriftSources.cmake | 126 + .../cmake/iotdb-session-config.cmake.in | 56 + .../client-cpp/cmake/iotdb-session.pc.in | 26 + iotdb-client/client-cpp/pom.xml | 427 ++-- .../client-cpp/src/assembly/client-cpp.xml | 59 +- .../third_party/DEPENDENCIES.md | 34 + .../src/include/AbstractSessionBuilder.h | 65 + iotdb-client/client-cpp/src/include/Column.h | 364 +++ .../client-cpp/src/include/ColumnDecoder.h | 83 + iotdb-client/client-cpp/src/include/Common.h | 428 ++++ iotdb-client/client-cpp/src/include/Date.h | 65 + .../Endpoint.h} | 40 +- iotdb-client/client-cpp/src/include/Export.h | 30 + .../client-cpp/src/include/Optional.h | 57 + iotdb-client/client-cpp/src/include/Session.h | 828 +++++++ .../client-cpp/src/include/SessionBuilder.h | 101 + .../client-cpp/src/include/SessionC.h | 399 +++ .../client-cpp/src/include/SessionConfig.h | 34 + .../client-cpp/src/include/SessionDataSet.h | 124 + .../src/{main => include}/SessionPool.h | 176 +- .../src/{test/main.cpp => include/Status.h} | 30 +- iotdb-client/client-cpp/src/include/TsBlock.h | 54 + .../client-cpp/src/main/CMakeLists.txt | 53 - iotdb-client/client-cpp/src/main/Column.cpp | 359 --- iotdb-client/client-cpp/src/main/Column.h | 353 --- .../client-cpp/src/main/ColumnDecoder.cpp | 181 -- .../client-cpp/src/main/ColumnDecoder.h | 75 - iotdb-client/client-cpp/src/main/Common.cpp | 498 ---- iotdb-client/client-cpp/src/main/Common.h | 492 ---- iotdb-client/client-cpp/src/main/DeviceID.h | 161 -- .../client-cpp/src/main/IoTDBRpcDataSet.cpp | 558 ----- .../client-cpp/src/main/IoTDBRpcDataSet.h | 150 -- .../client-cpp/src/main/NodesSupplier.cpp | 222 -- .../client-cpp/src/main/NodesSupplier.h | 137 -- iotdb-client/client-cpp/src/main/Session.cpp | 2048 ---------------- iotdb-client/client-cpp/src/main/Session.h | 1017 -------- .../client-cpp/src/main/SessionConnection.cpp | 673 ----- .../client-cpp/src/main/SessionConnection.h | 364 --- .../client-cpp/src/main/SessionDataSet.cpp | 254 -- .../client-cpp/src/main/SessionDataSet.h | 142 -- .../client-cpp/src/main/ThriftConnection.cpp | 170 -- .../client-cpp/src/main/ThriftConnection.h | 71 - iotdb-client/client-cpp/src/main/TsBlock.cpp | 121 - iotdb-client/client-cpp/src/main/TsBlock.h | 55 - .../client-cpp/src/rpc/IoTDBRpcDataSet.cpp | 734 ++++++ .../client-cpp/src/rpc/IoTDBRpcDataSet.h | 164 ++ .../client-cpp/src/rpc/NodesSupplier.cpp | 247 ++ .../client-cpp/src/rpc/NodesSupplier.h | 144 ++ iotdb-client/client-cpp/src/rpc/RpcCommon.cpp | 216 ++ iotdb-client/client-cpp/src/rpc/RpcCommon.h | 86 + .../client-cpp/src/rpc/SessionConnection.cpp | 680 ++++++ .../client-cpp/src/rpc/SessionConnection.h | 364 +++ .../src/rpc/SessionDataSetFactory.h | 41 + iotdb-client/client-cpp/src/rpc/SessionImpl.h | 230 ++ .../client-cpp/src/rpc/ThriftConnection.cpp | 187 ++ .../client-cpp/src/rpc/ThriftConnection.h | 79 + .../client-cpp/src/rpc/ThriftConvert.cpp | 62 + .../client-cpp/src/rpc/ThriftConvert.h | 41 + .../client-cpp/src/session/Column.cpp | 459 ++++ .../client-cpp/src/session/ColumnDecoder.cpp | 207 ++ .../client-cpp/src/session/Common.cpp | 315 +++ iotdb-client/client-cpp/src/session/Date.cpp | 65 + .../client-cpp/src/session/Session.cpp | 2174 +++++++++++++++++ .../client-cpp/src/session/SessionC.cpp | 1410 +++++++++++ .../client-cpp/src/session/SessionDataSet.cpp | 341 +++ .../src/{main => session}/SessionPool.cpp | 102 +- .../client-cpp/src/session/TsBlock.cpp | 116 + .../client-cpp/src/test/CMakeLists.txt | 64 - .../client-cpp/src/test/cpp/sessionIT.cpp | 933 ------- iotdb-client/client-cpp/test/CMakeLists.txt | 94 + .../client-cpp/test/catch2/.gitignore | 4 + .../client-cpp/test/cpp/sessionCIT.cpp | 748 ++++++ .../client-cpp/test/cpp/sessionIT.cpp | 1042 ++++++++ iotdb-client/client-cpp/test/main.cpp | 59 + iotdb-client/client-cpp/test/main_c.cpp | 52 + .../client-cpp/third-party/.gitignore | 7 + iotdb-client/client-cpp/third-party/README.md | 75 + .../client-cpp/third-party/linux/.gitignore | 2 + .../client-cpp/third-party/mac/.gitignore | 2 + .../client-cpp/third-party/windows/.gitignore | 2 + pom.xml | 5 +- 101 files changed, 17409 insertions(+), 10385 deletions(-) create mode 100755 .github/scripts/package-client-cpp-manylinux228.sh create mode 100644 .github/workflows/client-cpp-package.yml create mode 100644 example/client-cpp-example/README_zh.md create mode 100644 example/client-cpp-example/src/MultiSvrNodeClient.cpp create mode 100644 example/client-cpp-example/src/tree_example.c create mode 100644 iotdb-client/client-cpp/CMakeLists.txt create mode 100644 iotdb-client/client-cpp/cmake/FetchBoost.cmake create mode 100644 iotdb-client/client-cpp/cmake/FetchBuildTools.cmake create mode 100644 iotdb-client/client-cpp/cmake/FetchOpenSSL.cmake create mode 100644 iotdb-client/client-cpp/cmake/FetchThrift.cmake create mode 100644 iotdb-client/client-cpp/cmake/GenerateThriftSources.cmake create mode 100644 iotdb-client/client-cpp/cmake/iotdb-session-config.cmake.in create mode 100644 iotdb-client/client-cpp/cmake/iotdb-session.pc.in create mode 100644 iotdb-client/client-cpp/src/assembly/package-metadata/third_party/DEPENDENCIES.md create mode 100644 iotdb-client/client-cpp/src/include/AbstractSessionBuilder.h create mode 100644 iotdb-client/client-cpp/src/include/Column.h create mode 100644 iotdb-client/client-cpp/src/include/ColumnDecoder.h create mode 100644 iotdb-client/client-cpp/src/include/Common.h create mode 100644 iotdb-client/client-cpp/src/include/Date.h rename iotdb-client/client-cpp/src/{main/AbstractSessionBuilder.h => include/Endpoint.h} (61%) create mode 100644 iotdb-client/client-cpp/src/include/Export.h create mode 100644 iotdb-client/client-cpp/src/include/Optional.h create mode 100644 iotdb-client/client-cpp/src/include/Session.h create mode 100644 iotdb-client/client-cpp/src/include/SessionBuilder.h create mode 100644 iotdb-client/client-cpp/src/include/SessionC.h create mode 100644 iotdb-client/client-cpp/src/include/SessionConfig.h create mode 100644 iotdb-client/client-cpp/src/include/SessionDataSet.h rename iotdb-client/client-cpp/src/{main => include}/SessionPool.h (66%) rename iotdb-client/client-cpp/src/{test/main.cpp => include/Status.h} (52%) create mode 100644 iotdb-client/client-cpp/src/include/TsBlock.h delete mode 100644 iotdb-client/client-cpp/src/main/CMakeLists.txt delete mode 100644 iotdb-client/client-cpp/src/main/Column.cpp delete mode 100644 iotdb-client/client-cpp/src/main/Column.h delete mode 100644 iotdb-client/client-cpp/src/main/ColumnDecoder.cpp delete mode 100644 iotdb-client/client-cpp/src/main/ColumnDecoder.h delete mode 100644 iotdb-client/client-cpp/src/main/Common.cpp delete mode 100644 iotdb-client/client-cpp/src/main/Common.h delete mode 100644 iotdb-client/client-cpp/src/main/DeviceID.h delete mode 100644 iotdb-client/client-cpp/src/main/IoTDBRpcDataSet.cpp delete mode 100644 iotdb-client/client-cpp/src/main/IoTDBRpcDataSet.h delete mode 100644 iotdb-client/client-cpp/src/main/NodesSupplier.cpp delete mode 100644 iotdb-client/client-cpp/src/main/NodesSupplier.h delete mode 100644 iotdb-client/client-cpp/src/main/Session.cpp delete mode 100644 iotdb-client/client-cpp/src/main/Session.h delete mode 100644 iotdb-client/client-cpp/src/main/SessionConnection.cpp delete mode 100644 iotdb-client/client-cpp/src/main/SessionConnection.h delete mode 100644 iotdb-client/client-cpp/src/main/SessionDataSet.cpp delete mode 100644 iotdb-client/client-cpp/src/main/SessionDataSet.h delete mode 100644 iotdb-client/client-cpp/src/main/ThriftConnection.cpp delete mode 100644 iotdb-client/client-cpp/src/main/ThriftConnection.h delete mode 100644 iotdb-client/client-cpp/src/main/TsBlock.cpp delete mode 100644 iotdb-client/client-cpp/src/main/TsBlock.h create mode 100644 iotdb-client/client-cpp/src/rpc/IoTDBRpcDataSet.cpp create mode 100644 iotdb-client/client-cpp/src/rpc/IoTDBRpcDataSet.h create mode 100644 iotdb-client/client-cpp/src/rpc/NodesSupplier.cpp create mode 100644 iotdb-client/client-cpp/src/rpc/NodesSupplier.h create mode 100644 iotdb-client/client-cpp/src/rpc/RpcCommon.cpp create mode 100644 iotdb-client/client-cpp/src/rpc/RpcCommon.h create mode 100644 iotdb-client/client-cpp/src/rpc/SessionConnection.cpp create mode 100644 iotdb-client/client-cpp/src/rpc/SessionConnection.h create mode 100644 iotdb-client/client-cpp/src/rpc/SessionDataSetFactory.h create mode 100644 iotdb-client/client-cpp/src/rpc/SessionImpl.h create mode 100644 iotdb-client/client-cpp/src/rpc/ThriftConnection.cpp create mode 100644 iotdb-client/client-cpp/src/rpc/ThriftConnection.h create mode 100644 iotdb-client/client-cpp/src/rpc/ThriftConvert.cpp create mode 100644 iotdb-client/client-cpp/src/rpc/ThriftConvert.h create mode 100644 iotdb-client/client-cpp/src/session/Column.cpp create mode 100644 iotdb-client/client-cpp/src/session/ColumnDecoder.cpp create mode 100644 iotdb-client/client-cpp/src/session/Common.cpp create mode 100644 iotdb-client/client-cpp/src/session/Date.cpp create mode 100644 iotdb-client/client-cpp/src/session/Session.cpp create mode 100644 iotdb-client/client-cpp/src/session/SessionC.cpp create mode 100644 iotdb-client/client-cpp/src/session/SessionDataSet.cpp rename iotdb-client/client-cpp/src/{main => session}/SessionPool.cpp (66%) create mode 100644 iotdb-client/client-cpp/src/session/TsBlock.cpp delete mode 100644 iotdb-client/client-cpp/src/test/CMakeLists.txt delete mode 100644 iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp create mode 100644 iotdb-client/client-cpp/test/CMakeLists.txt create mode 100644 iotdb-client/client-cpp/test/catch2/.gitignore create mode 100644 iotdb-client/client-cpp/test/cpp/sessionCIT.cpp create mode 100644 iotdb-client/client-cpp/test/cpp/sessionIT.cpp create mode 100644 iotdb-client/client-cpp/test/main.cpp create mode 100644 iotdb-client/client-cpp/test/main_c.cpp create mode 100644 iotdb-client/client-cpp/third-party/.gitignore create mode 100644 iotdb-client/client-cpp/third-party/README.md create mode 100644 iotdb-client/client-cpp/third-party/linux/.gitignore create mode 100644 iotdb-client/client-cpp/third-party/mac/.gitignore create mode 100644 iotdb-client/client-cpp/third-party/windows/.gitignore diff --git a/.github/scripts/package-client-cpp-manylinux228.sh b/.github/scripts/package-client-cpp-manylinux228.sh new file mode 100755 index 0000000000000..d8e073c459979 --- /dev/null +++ b/.github/scripts/package-client-cpp-manylinux228.sh @@ -0,0 +1,101 @@ +#!/usr/bin/env bash +# 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. +# +# Build client-cpp in manylinux_2_28 and verify max required GLIBC symbol <= 2.28. +set -euxo pipefail + +MACHINE=$(uname -m) +case "${MACHINE}" in + x86_64) + CMAKE_PKG_ARCH=linux-x86_64 + JDK_API_ARCH=linux/x64 + DEFAULT_CLASSIFIER=linux-x86_64-glibc2.28 + ;; + aarch64) + CMAKE_PKG_ARCH=linux-aarch64 + JDK_API_ARCH=linux/aarch64 + DEFAULT_CLASSIFIER=linux-aarch64-glibc2.28 + ;; + *) + echo "Unsupported architecture: ${MACHINE}" >&2 + exit 1 + ;; +esac + +PACKAGE_CLASSIFIER="${PACKAGE_CLASSIFIER:-${DEFAULT_CLASSIFIER}}" + +CMAKE_VERSION=3.28.4 +CMAKE_DIR="/opt/cmake-${CMAKE_VERSION}" +if [[ ! -x "${CMAKE_DIR}/bin/cmake" ]]; then + curl -fsSL -o /tmp/cmake.tar.gz "https://github.com/Kitware/CMake/releases/download/v${CMAKE_VERSION}/cmake-${CMAKE_VERSION}-${CMAKE_PKG_ARCH}.tar.gz" + rm -rf "${CMAKE_DIR}" + mkdir -p /opt + tar xf /tmp/cmake.tar.gz -C /opt + mv "/opt/cmake-${CMAKE_VERSION}-${CMAKE_PKG_ARCH}" "${CMAKE_DIR}" +fi + +JAVA_HOME=/opt/jdk-17 +if [[ ! -x "${JAVA_HOME}/bin/java" ]]; then + curl -fsSL -o /tmp/jdk17.tar.gz "https://api.adoptium.net/v3/binary/latest/17/ga/${JDK_API_ARCH}/jdk/hotspot/normal/eclipse?project=jdk" + rm -rf /opt/jdk-17* + mkdir -p /opt + tar xf /tmp/jdk17.tar.gz -C /opt + JAVA_HOME=$(find /opt -maxdepth 1 -type d -name 'jdk-17*' -print -quit) + ln -sfn "${JAVA_HOME}" /opt/jdk-17 + JAVA_HOME=/opt/jdk-17 +fi + +export PATH="${CMAKE_DIR}/bin:${JAVA_HOME}/bin:${PATH}" +export JAVA_HOME + +gcc --version +c++ --version +gcc_major=$(gcc -dumpversion | cut -d. -f1) +if (( gcc_major < 14 )); then + echo "ERROR: GCC >= 14 is required; got $(gcc -dumpversion)" + exit 1 +fi +cmake --version +java -version + +cd "${GITHUB_WORKSPACE:?GITHUB_WORKSPACE is not set}" +./mvnw clean package -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests \ + -Dspotless.skip=true \ + -Dclient.cpp.package.classifier="${PACKAGE_CLASSIFIER}" + +SO="iotdb-client/client-cpp/target/install/lib/libiotdb_session.so" +test -f "${SO}" + +echo "=== Build host glibc ===" +ldd --version 2>&1 | sed -n '1p' + +echo "=== Highest GLIBC_* symbols in libiotdb_session.so ===" +objdump -T "${SO}" | grep GLIBC_ | sed "s/.*GLIBC_/GLIBC_/" | sort -Vu | tail -10 + +max_glibc=$(objdump -T "${SO}" | grep -oE "GLIBC_[0-9.]+" | sed "s/GLIBC_//" | sort -t. -k1,1n -k2,2n -k3,3n | tail -1) +echo "max_glibc=${max_glibc}" + +if [[ -z "${max_glibc}" ]]; then + echo "ERROR: could not determine max GLIBC version from ${SO}" + exit 1 +fi + +if awk -v max="${max_glibc}" "BEGIN { exit !(max > 2.28) }"; then + echo "ERROR: libiotdb_session.so requires glibc > 2.28 (max=${max_glibc})" + exit 1 +fi + +echo "glibc compatibility check passed (max=${max_glibc} <= 2.28)" diff --git a/.github/workflows/client-cpp-package.yml b/.github/workflows/client-cpp-package.yml new file mode 100644 index 0000000000000..db2d754be8eaa --- /dev/null +++ b/.github/workflows/client-cpp-package.yml @@ -0,0 +1,454 @@ +# Publish-oriented packaging workflow: +# - manual runs, release:published, v* tags +# - rc/** pushes only when C++-related paths change (job should-package) +# release events always use the workflow file from the default branch. +name: C++ Client package + +on: + workflow_dispatch: + inputs: + variants: + description: "Which packages to build" + required: false + default: all + type: choice + options: + - all + - linux + - macos + - windows + - windows-vs2017 + - windows-vs2019 + - windows-vs2022 + - windows-vs2026 + release: + types: [published] + push: + branches: + - "rc/**" + tags: + - "v*" + +concurrency: + group: client-cpp-package-${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: true + +env: + MAVEN_OPTS: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 + MAVEN_ARGS: --batch-mode --no-transfer-progress + +jobs: + should-package: + # Keep rc branch cost low: skip full matrix when unrelated files change. + runs-on: ubuntu-latest + outputs: + run: ${{ steps.result.outputs.run }} + steps: + - uses: actions/checkout@v5 + if: github.event_name == 'push' && startsWith(github.ref, 'refs/heads/rc/') + - uses: dorny/paths-filter@v3 + id: filter + if: github.event_name == 'push' && startsWith(github.ref, 'refs/heads/rc/') + with: + filters: | + cpp: + - 'iotdb-client/client-cpp/**' + - 'iotdb-client/pom.xml' + - 'iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift' + - 'iotdb-protocol/thrift-commons/src/main/thrift/common.thrift' + - '.github/workflows/client-cpp-package.yml' + - '.github/scripts/package-client-cpp-*.sh' + - id: result + shell: bash + run: | + set -euo pipefail + E="${{ github.event_name }}" + R="${{ github.ref }}" + if [[ "$E" == "workflow_dispatch" ]] || [[ "$E" == "release" ]]; then + echo "run=true" >> "$GITHUB_OUTPUT" + exit 0 + fi + if [[ "$E" == "push" ]] && [[ "$R" == refs/tags/v* ]]; then + echo "run=true" >> "$GITHUB_OUTPUT" + exit 0 + fi + if [[ "$E" == "push" ]] && [[ "$R" =~ ^refs/heads/rc/ ]]; then + if [[ "${{ steps.filter.outputs.cpp }}" == "true" ]]; then + echo "run=true" >> "$GITHUB_OUTPUT" + else + echo "run=false" >> "$GITHUB_OUTPUT" + fi + exit 0 + fi + echo "run=false" >> "$GITHUB_OUTPUT" + + resolve-matrix: + name: Resolve package matrix + needs: should-package + if: needs.should-package.outputs.run == 'true' + runs-on: ubuntu-latest + outputs: + run_linux: ${{ steps.filter.outputs.run_linux }} + run_macos: ${{ steps.filter.outputs.run_macos }} + run_windows: ${{ steps.filter.outputs.run_windows }} + windows_matrix: ${{ steps.filter.outputs.windows_matrix }} + steps: + - id: filter + # Resolve workflow_dispatch variants to a compact matrix payload for fromJSON(). + shell: bash + run: | + set -euo pipefail + VARIANT="${{ github.event.inputs.variants || 'all' }}" + + run_linux=false + run_macos=false + run_windows=false + case "$VARIANT" in + all) + run_linux=true + run_macos=true + run_windows=true + ;; + linux) run_linux=true ;; + macos) run_macos=true ;; + windows) run_windows=true ;; + windows-vs2017|windows-vs2019|windows-vs2022|windows-vs2026) + run_windows=true + ;; + *) + echo "Unknown variant: $VARIANT" >&2 + exit 1 + ;; + esac + + echo "run_linux=${run_linux}" >> "$GITHUB_OUTPUT" + echo "run_macos=${run_macos}" >> "$GITHUB_OUTPUT" + echo "run_windows=${run_windows}" >> "$GITHUB_OUTPUT" + + # Compact JSON (no leading whitespace); required for GITHUB_OUTPUT + fromJSON(). + WINDOWS_MATRIX='[{"name":"windows-vs2026","runs-on":"windows-2025-vs2026","boost_choco":"","boost_choco_version":"","cmake_generator":"Visual Studio 18 2026","package_classifier":"windows-x86_64-msvc14.4","vs_choco":"","vs_choco_params":""},{"name":"windows-vs2022","runs-on":"windows-2022","boost_choco":"","boost_choco_version":"","cmake_generator":"","package_classifier":"windows-x86_64-msvc14.3","vs_choco":"","vs_choco_params":""},{"name":"windows-vs2019","runs-on":"windows-2022","boost_choco":"","boost_choco_version":"","cmake_generator":"Visual Studio 16 2019","package_classifier":"windows-x86_64-msvc14.2","vs_choco":"visualstudio2019buildtools","vs_choco_params":"--add Microsoft.VisualStudio.Workload.VCTools --includeRecommended"},{"name":"windows-vs2017","runs-on":"windows-2022","boost_choco":"","boost_choco_version":"","cmake_generator":"Visual Studio 15 2017","package_classifier":"windows-x86_64-msvc14.1","vs_choco":"visualstudio2017buildtools","vs_choco_params":"--add Microsoft.VisualStudio.Workload.VCTools --includeRecommended"}]' + + write_windows_matrix_output() { + local matrix_json="$1" + { + echo 'windows_matrix<<__MATRIX_EOF__' + echo "${matrix_json}" + echo '__MATRIX_EOF__' + } >> "$GITHUB_OUTPUT" + } + + if [[ "$run_windows" == true ]]; then + if [[ "$VARIANT" == all || "$VARIANT" == windows ]]; then + FILTERED="$WINDOWS_MATRIX" + elif [[ "$VARIANT" =~ ^windows-vs ]]; then + FILTERED=$(echo "$WINDOWS_MATRIX" | jq -c --arg v "$VARIANT" '[.[] | select(.name == $v)]') + else + FILTERED='[]' + fi + if [[ $(echo "$FILTERED" | jq 'length') -eq 0 ]]; then + echo "No Windows matrix rows for variant=${VARIANT}" >&2 + exit 1 + fi + FILTERED=$(echo "$FILTERED" | jq -c '.') + write_windows_matrix_output "${FILTERED}" + else + write_windows_matrix_output '[]' + fi + + package-linux-glibc228: + name: Package (linux-x86_64-glibc2.28) + needs: [should-package, resolve-matrix] + if: needs.should-package.outputs.run == 'true' && needs.resolve-matrix.outputs.run_linux == 'true' + # Checkout/cache on host (Node actions need modern glibc); build in manylinux_2_28 via docker run. + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v5 + - name: Cache Maven packages + uses: actions/cache@v5 + with: + path: ~/.m2 + key: linux-glibc228-m2-${{ hashFiles('**/pom.xml') }} + restore-keys: | + linux-glibc228-m2- + - name: Package client-cpp (glibc 2.28 baseline) + shell: bash + run: | + set -euxo pipefail + chmod +x .github/scripts/package-client-cpp-manylinux228.sh + docker run --rm \ + -v "${{ github.workspace }}:/workspace" \ + -v "${HOME}/.m2:/root/.m2" \ + -w /workspace \ + -e GITHUB_WORKSPACE=/workspace \ + quay.io/pypa/manylinux_2_28_x86_64 \ + bash .github/scripts/package-client-cpp-manylinux228.sh + - name: Restore workspace ownership after container build + if: always() + run: sudo chown -R "$(id -u):$(id -g)" "${{ github.workspace }}" + - name: Resolve package zip + id: pkg + shell: bash + run: | + set -euo pipefail + shopt -s nullglob + packages=(iotdb-client/client-cpp/target/iotdb-session-cpp-*-linux-x86_64-glibc2.28.zip) + if [ "${#packages[@]}" -ne 1 ]; then + echo "Expected exactly one package zip, got: ${packages[*]:-(none)}" + exit 1 + fi + echo "path=${packages[0]}" >> "$GITHUB_OUTPUT" + echo "name=$(basename "${packages[0]}" .zip)" >> "$GITHUB_OUTPUT" + sha512="${packages[0]}.sha512" + if [ ! -f "${sha512}" ]; then + echo "Expected checksum file: ${sha512}" + exit 1 + fi + echo "sha512_path=${sha512}" >> "$GITHUB_OUTPUT" + - name: Upload zip artifact + uses: actions/upload-artifact@v6 + with: + name: ${{ steps.pkg.outputs.name }} + path: | + ${{ steps.pkg.outputs.path }} + ${{ steps.pkg.outputs.sha512_path }} + if-no-files-found: error + + package-linux-aarch64-glibc228: + name: Package (linux-aarch64-glibc2.28) + needs: [should-package, resolve-matrix] + if: needs.should-package.outputs.run == 'true' && needs.resolve-matrix.outputs.run_linux == 'true' + # Checkout/cache on host; build in manylinux_2_28 aarch64 via docker run (glibc 2.28 baseline). + runs-on: ubuntu-22.04-arm + steps: + - uses: actions/checkout@v5 + - name: Cache Maven packages + uses: actions/cache@v5 + with: + path: ~/.m2 + key: linux-aarch64-glibc228-m2-${{ hashFiles('**/pom.xml') }} + restore-keys: | + linux-aarch64-glibc228-m2- + - name: Package client-cpp (glibc 2.28 baseline) + shell: bash + run: | + set -euxo pipefail + chmod +x .github/scripts/package-client-cpp-manylinux228.sh + docker run --rm \ + -v "${{ github.workspace }}:/workspace" \ + -v "${HOME}/.m2:/root/.m2" \ + -w /workspace \ + -e GITHUB_WORKSPACE=/workspace \ + quay.io/pypa/manylinux_2_28_aarch64 \ + bash .github/scripts/package-client-cpp-manylinux228.sh + - name: Restore workspace ownership after container build + if: always() + run: sudo chown -R "$(id -u):$(id -g)" "${{ github.workspace }}" + - name: Resolve package zip + id: pkg + shell: bash + run: | + set -euo pipefail + shopt -s nullglob + packages=(iotdb-client/client-cpp/target/iotdb-session-cpp-*-linux-aarch64-glibc2.28.zip) + if [ "${#packages[@]}" -ne 1 ]; then + echo "Expected exactly one package zip, got: ${packages[*]:-(none)}" + exit 1 + fi + echo "path=${packages[0]}" >> "$GITHUB_OUTPUT" + echo "name=$(basename "${packages[0]}" .zip)" >> "$GITHUB_OUTPUT" + sha512="${packages[0]}.sha512" + if [ ! -f "${sha512}" ]; then + echo "Expected checksum file: ${sha512}" + exit 1 + fi + echo "sha512_path=${sha512}" >> "$GITHUB_OUTPUT" + - name: Upload zip artifact + uses: actions/upload-artifact@v6 + with: + name: ${{ steps.pkg.outputs.name }} + path: | + ${{ steps.pkg.outputs.path }} + ${{ steps.pkg.outputs.sha512_path }} + if-no-files-found: error + + package-macos: + name: Package (${{ matrix.name }}) + needs: [should-package, resolve-matrix] + if: needs.should-package.outputs.run == 'true' && needs.resolve-matrix.outputs.run_macos == 'true' + strategy: + fail-fast: false + matrix: + include: + - name: macos-x86_64 + runs-on: macos-15-intel + - name: macos-arm64 + runs-on: macos-latest + runs-on: ${{ matrix.runs-on }} + steps: + - uses: actions/checkout@v5 + - name: Set up JDK 17 + uses: actions/setup-java@v5 + with: + distribution: temurin + java-version: "17" + - name: Install C++ dependencies (macOS) + shell: bash + run: | + set -euxo pipefail + brew install boost openssl llvm@17 bison + ln -sf "$(brew --prefix llvm@17)/bin/clang-format" "$(brew --prefix)/bin/clang-format" + echo "$(brew --prefix bison)/bin" >> "$GITHUB_PATH" + echo "$(brew --prefix llvm@17)/bin" >> "$GITHUB_PATH" + clang-format --version + bison --version + - name: Cache Maven packages + uses: actions/cache@v5 + with: + path: ~/.m2 + key: macos-${{ matrix.name }}-m2-${{ hashFiles('**/pom.xml') }} + restore-keys: | + macos-${{ matrix.name }}-m2- + - name: Package client-cpp + shell: bash + env: + MACOSX_DEPLOYMENT_TARGET: "12.0" + run: | + set -euxo pipefail + ./mvnw clean package -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests \ + -Dspotless.skip=true + - name: Resolve package zip + id: pkg + shell: bash + run: | + set -euo pipefail + shopt -s nullglob + packages=(iotdb-client/client-cpp/target/iotdb-session-cpp-*.zip) + if [ "${#packages[@]}" -ne 1 ]; then + echo "Expected exactly one package zip, got: ${packages[*]:-(none)}" + exit 1 + fi + echo "path=${packages[0]}" >> "$GITHUB_OUTPUT" + echo "name=$(basename "${packages[0]}" .zip)" >> "$GITHUB_OUTPUT" + sha512="${packages[0]}.sha512" + if [ ! -f "${sha512}" ]; then + echo "Expected checksum file: ${sha512}" + exit 1 + fi + echo "sha512_path=${sha512}" >> "$GITHUB_OUTPUT" + - name: Upload zip artifact + uses: actions/upload-artifact@v6 + with: + name: ${{ steps.pkg.outputs.name }} + path: | + ${{ steps.pkg.outputs.path }} + ${{ steps.pkg.outputs.sha512_path }} + if-no-files-found: error + + package-windows: + name: Package (${{ matrix.name }}) + needs: [should-package, resolve-matrix] + if: needs.should-package.outputs.run == 'true' && needs.resolve-matrix.outputs.run_windows == 'true' + strategy: + fail-fast: false + matrix: + include: ${{ fromJSON(needs.resolve-matrix.outputs.windows_matrix) }} + runs-on: ${{ matrix.runs-on }} + steps: + - uses: actions/checkout@v5 + - name: Set up JDK 17 + uses: actions/setup-java@v5 + with: + distribution: temurin + java-version: "17" + - name: Install Visual Studio Build Tools (${{ matrix.name }}) + if: matrix.vs_choco != '' + shell: pwsh + run: | + $params = '${{ matrix.vs_choco_params }}' + if ($params) { + choco install ${{ matrix.vs_choco }} -y --package-parameters="$params" --no-progress + } else { + choco install ${{ matrix.vs_choco }} -y --no-progress + } + - name: Install C++ dependencies (Windows) + shell: pwsh + run: | + choco install winflexbison3 -y --no-progress + if ('${{ matrix.boost_choco }}' -ne '') { + $boostArgs = @('install', '${{ matrix.boost_choco }}', '-y', '--no-progress') + if ('${{ matrix.boost_choco_version }}' -ne '') { + $boostArgs += @("--version=${{ matrix.boost_choco_version }}") + } + & choco @boostArgs + $boostDir = Get-ChildItem -Path 'C:\local\' -Filter 'boost_*' -ErrorAction SilentlyContinue | Select-Object -First 1 + if (-not $boostDir) { + throw "Boost not found under C:\local after installing ${{ matrix.boost_choco }}" + } + echo "BOOST_INCLUDEDIR=$($boostDir.FullName)" >> $env:GITHUB_ENV + } + - name: Cache Maven packages + uses: actions/cache@v5 + with: + path: ~/.m2 + key: windows-${{ matrix.name }}-m2-${{ hashFiles('**/pom.xml') }} + restore-keys: | + windows-${{ matrix.name }}-m2- + - name: Package client-cpp + shell: bash + env: + CMAKE_GENERATOR: ${{ matrix.cmake_generator }} + PACKAGE_CLASSIFIER: ${{ matrix.package_classifier }} + run: | + set -euxo pipefail + MVN_ARGS=(./mvnw clean package -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests \ + -Dspotless.skip=true \ + "-Dclient.cpp.package.classifier=${PACKAGE_CLASSIFIER}") + if [ -n "${BOOST_INCLUDEDIR:-}" ]; then + MVN_ARGS+=("-Dboost.include.dir=${BOOST_INCLUDEDIR}") + fi + if [ -n "${CMAKE_GENERATOR:-}" ]; then + MVN_ARGS+=("-Dcmake.generator=${CMAKE_GENERATOR}") + fi + "${MVN_ARGS[@]}" + - name: Verify Windows SDK is x64 + # Guard against accidental Win32 generator defaults on legacy VS matrices. + shell: pwsh + run: | + $dll = "iotdb-client/client-cpp/target/install/lib/iotdb_session.dll" + if (-not (Test-Path $dll)) { + throw "Missing $dll" + } + $bytes = [System.IO.File]::ReadAllBytes($dll) + $peOffset = [BitConverter]::ToInt32($bytes, 0x3C) + $machine = [BitConverter]::ToUInt16($bytes, $peOffset + 4) + if ($machine -ne 0x8664) { + throw "Expected PE32+ x64 (machine=0x8664), got 0x$($machine.ToString('X4'))" + } + Write-Host "Verified x64 DLL: $dll" + - name: Resolve package zip + id: pkg + shell: bash + run: | + set -euo pipefail + shopt -s nullglob + packages=(iotdb-client/client-cpp/target/iotdb-session-cpp-*-${{ matrix.package_classifier }}.zip) + if [ "${#packages[@]}" -ne 1 ]; then + echo "Expected exactly one package zip, got: ${packages[*]:-(none)}" + exit 1 + fi + echo "path=${packages[0]}" >> "$GITHUB_OUTPUT" + echo "name=$(basename "${packages[0]}" .zip)" >> "$GITHUB_OUTPUT" + sha512="${packages[0]}.sha512" + if [ ! -f "${sha512}" ]; then + echo "Expected checksum file: ${sha512}" + exit 1 + fi + echo "sha512_path=${sha512}" >> "$GITHUB_OUTPUT" + - name: Upload zip artifact + uses: actions/upload-artifact@v6 + with: + name: ${{ steps.pkg.outputs.name }} + path: | + ${{ steps.pkg.outputs.path }} + ${{ steps.pkg.outputs.sha512_path }} + if-no-files-found: error diff --git a/.github/workflows/multi-language-client.yml b/.github/workflows/multi-language-client.yml index a1fc44af61c95..65629ce12d050 100644 --- a/.github/workflows/multi-language-client.yml +++ b/.github/workflows/multi-language-client.yml @@ -46,41 +46,64 @@ jobs: fail-fast: false max-parallel: 15 matrix: - os: [ubuntu-latest, windows-latest, macos-latest] + os: [ubuntu-22.04, ubuntu-24.04, windows-2022, windows-2025-vs2026, macos-latest] runs-on: ${{ matrix.os}} steps: - uses: actions/checkout@v4 - name: Install CPP Dependencies (Ubuntu) - if: matrix.os == 'ubuntu-latest' + if: runner.os == 'Linux' shell: bash run: | + set -euxo pipefail sudo apt-get update - sudo apt-get install libboost-all-dev + sudo apt-get install -y libboost-all-dev openssl libssl-dev wget + # jammy (22.04): no clang-format-17 in default repos — use apt.llvm.org + . /etc/os-release + if [[ "${VERSION_CODENAME}" == "jammy" ]]; then + wget -qO /tmp/llvm.sh https://apt.llvm.org/llvm.sh + chmod +x /tmp/llvm.sh + sudo DEBIAN_FRONTEND=noninteractive /tmp/llvm.sh 17 + sudo apt-get install -y clang-format-17 + else + sudo apt-get install -y clang-format-17 + fi + sudo update-alternatives --install /usr/bin/clang-format clang-format /usr/bin/clang-format-17 100 + sudo update-alternatives --set clang-format /usr/bin/clang-format-17 + clang-format --version - name: Install CPP Dependencies (Mac) # remove some xcode to release disk space - if: matrix.os == 'macos-latest' + if: runner.os == 'macOS' shell: bash run: | - brew install boost + brew install boost openssl llvm@17 bison + ln -sf "$(brew --prefix llvm@17)/bin/clang-format" "$(brew --prefix)/bin/clang-format" + echo "$(brew --prefix bison)/bin" >> "$GITHUB_PATH" + echo "$(brew --prefix llvm@17)/bin" >> "$GITHUB_PATH" + clang-format --version + bison --version sudo rm -rf /Applications/Xcode_14.3.1.app sudo rm -rf /Applications/Xcode_15.0.1.app sudo rm -rf /Applications/Xcode_15.1.app sudo rm -rf /Applications/Xcode_15.2.app sudo rm -rf /Applications/Xcode_15.3.app - name: Install CPP Dependencies (Windows) - if: matrix.os == 'windows-latest' + if: runner.os == 'Windows' run: | - choco install winflexbison3 - choco install boost-msvc-14.3 - $boost_path = (Get-ChildItem -Path 'C:\local\' -Filter 'boost_*').FullName - echo $boost_path >> $env:GITHUB_PATH + choco install winflexbison3 -y + choco install llvm --version=17.0.6 --force -y + clang-format --version - name: Cache Maven packages uses: actions/cache@v4 with: path: ~/.m2 key: ${{ runner.os }}-m2-${{ hashFiles('**/pom.xml') }} restore-keys: ${{ runner.os }}-m2- + - name: Check C++ format (Spotless) + shell: bash + run: | + ./mvnw -P with-cpp -pl iotdb-client/client-cpp spotless:check + ./mvnw -P with-cpp -pl example/client-cpp-example spotless:check - name: Build IoTDB server shell: bash run: ./mvnw clean install -pl distribution -am -DskipTests @@ -89,7 +112,14 @@ jobs: # Explicitly using mvnw here as the build requires maven 3.9 and the default installation is older # Explicitly using "install" instead of package in order to be sure we're using libs built on this machine # (was causing problems on windows, but could cause problem on linux, when updating the thrift module) - run: ./mvnw clean verify -P with-cpp -pl iotdb-client/client-cpp,example/client-cpp-example -am + run: | + if [[ "${{ matrix.os }}" == "windows-2025-vs2026" ]]; then + ./mvnw verify -P with-cpp -pl iotdb-client/client-cpp,example/client-cpp-example -am -Dcmake.generator="Visual Studio 18 2026" -Dbuild.tests=OFF -Dmaven.test.skip=true + elif [[ "${{ runner.os }}" == "Windows" ]]; then + ./mvnw verify -P with-cpp -pl iotdb-client/client-cpp,example/client-cpp-example -am -Dbuild.tests=OFF -Dmaven.test.skip=true + else + ./mvnw verify -P with-cpp -pl iotdb-client/client-cpp,example/client-cpp-example -am + fi - name: Upload Artifact if: failure() uses: actions/upload-artifact@v4 diff --git a/.gitignore b/.gitignore index 71e0fc85edb50..21e32c3fcb61e 100644 --- a/.gitignore +++ b/.gitignore @@ -40,8 +40,8 @@ tsfile-jdbc/src/main/resources/output/queryRes.csv *.txt !iotdb-client/client-py/requirements.txt !iotdb-client/client-py/requirements_dev.txt -!iotdb-client/client-cpp/src/main/CMakeLists.txt -!iotdb-client/client-cpp/src/test/CMakeLists.txt +!iotdb-client/client-cpp/CMakeLists.txt +!iotdb-client/client-cpp/test/CMakeLists.txt !example/rest-client-c-example/CMakeLists.txt !example/client-cpp-example/src/CMakeLists.txt @@ -88,6 +88,11 @@ tsfile-jdbc/src/main/resources/output/queryRes.csv *.tsfile tsfile/src/test/resources/*.ts +### clion project +**/cmake-build-debug/ +**/cmake-build-release/ +iotdb-client/client-cpp/build-*/ + ### Apache release ### local-snapshots-dir/ venv/ @@ -107,6 +112,7 @@ classes/ ### Cmake files ### *.cmake +!iotdb-client/client-cpp/cmake/*.cmake Makefile **/CMakeFiles/ diff --git a/.mvn/wrapper/maven-wrapper.properties b/.mvn/wrapper/maven-wrapper.properties index 38b4fe1781a88..ac504fa7b3a22 100644 --- a/.mvn/wrapper/maven-wrapper.properties +++ b/.mvn/wrapper/maven-wrapper.properties @@ -15,5 +15,5 @@ # specific language governing permissions and limitations # under the License. -distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.6/apache-maven-3.9.6-bin.zip +distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.12/apache-maven-3.9.12-bin.zip wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.3.2/maven-wrapper-3.3.2.jar diff --git a/distribution/pom.xml b/distribution/pom.xml index 138cea976408c..1aa47d8122077 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -29,6 +29,9 @@ iotdb-distribution pom IoTDB: Distribution + + ${os.classifier} + org.apache.iotdb @@ -117,6 +120,67 @@ + + .os-unix + + + Linux + unix + !aarch64 + + + + linux-x86_64-glibc2.28 + + + + .os-unix-arm + + + Linux + unix + aarch64 + + + + linux-aarch64-glibc2.28 + + + + .os-mac + + + mac + !aarch64 + + + + macos-x86_64 + + + + .os-mac-arm + + + mac + aarch64 + + + + macos-aarch64 + + + + .os-windows + + + windows + + + + windows-x86_64-msvc14.3 + + with-cpp @@ -157,7 +221,7 @@ - apache-iotdb-${project.version}-client-cpp-${os.classifier}-bin.zip + apache-iotdb-${project.version}-client-cpp-${client.cpp.package.classifier}-bin.zip diff --git a/example/client-cpp-example/README.md b/example/client-cpp-example/README.md index f36a084f888e5..5ff964595fda7 100644 --- a/example/client-cpp-example/README.md +++ b/example/client-cpp-example/README.md @@ -19,30 +19,240 @@ --> -# How to get a complete CPP client demo project +# IoTDB C++ client examples -## Get a project +[中文说明](README_zh.md) -Using maven to build this example project: +Sample programs that link against the pre-built **IoTDB C++ Session SDK** +(`iotdb_session`). Thrift and Boost are **not** required at application compile +time; they are embedded inside the SDK shared library. -* cd the root path of the whole project -* run `mvn clean package -DskipTests -P with-cpp -pl example/client-cpp-example -am` -* cd example/client-cpp-example/target +All examples connect to a running IoTDB instance (default `127.0.0.1:6667`, +user `root` / `root`). + +| Example | Description | +|---------|-------------| +| `SessionExample` | Tree model: DDL, insert, query, delete | +| `AlignedTimeseriesSessionExample` | Aligned time series and templates | +| `MultiSvrNodeClient` | Multi-node insert/query loop | +| `tree_example` | C Session API (tree model) | + +## Which SDK zip to use + +Release CI ([client-cpp-package.yml](../../.github/workflows/client-cpp-package.yml)) +publishes one zip per platform/toolchain: +`client-cpp--.zip` (zip root contains `include/` and `lib/`). + +| Deployment target | Classifier suffix | +|-------------------|-------------------| +| Linux x86_64, glibc ≥ 2.24, CXX11 ABI (**recommended**) | `linux-x86_64-glibc224` | +| Linux aarch64, glibc ≥ 2.24, CXX11 ABI (**recommended**) | `linux-aarch64-glibc224` | +| Linux x86_64, glibc ≥ 2.17, legacy ABI | `linux-x86_64-glibc217` | +| Linux aarch64, glibc ≥ 2.17, legacy ABI | `linux-aarch64-glibc217` | +| macOS x86_64 | `mac-x86_64` | +| macOS arm64 | `mac-aarch64` | +| Windows (match your Visual Studio version) | `windows-x86_64-vs2017` … `vs2026` | + +The current build compiles Thrift 0.21 from source at CMake configure time. +Legacy `-Diotdb-tools-thrift.version=...` flags applied to the **old** +pre-built Thrift workflow only. On Linux, prefer **`glibc224`** when your host has +glibc ≥ 2.24 and you use the default `g++`. Use **`glibc217`** only for glibc 2.17 +systems or legacy ABI; on Ubuntu 22/24 you may need `-D_GLIBCXX_USE_CXX11_ABI=0` +when linking against `glibc217`. See [client-cpp README](../../iotdb-client/client-cpp/README.md). + +## SDK layout (after unpack) + +The SDK zip produced by `client-cpp` contains **public headers only** and one +shared library: + +``` +client/ +├── include/ +│ ├── Session.h +│ ├── Export.h +│ └── ... (17 public headers; no thrift/ or boost/) +└── lib/ + ├── iotdb_session.dll + iotdb_session.lib (Windows) + ├── libiotdb_session.so (Linux) + └── libiotdb_session.dylib (macOS) +``` + +## Build the examples + +### Option A – Maven (recommended in this repo) + +From the repository root: + +```bash +mvn clean package -DskipTests -P with-cpp -pl example/client-cpp-example -am +``` + +Maven unpacks the SDK zip into `example/client-cpp-example/target/client/` and +runs CMake in `target/`. Binaries are under `target/` (exact path depends on +the generator; on Windows with Visual Studio: `target/Release/`). + +### Option B – CMake only (manual SDK) + +1. Build or download the SDK and unpack it so `client/include` and + `client/lib` exist (see layout above). +2. Copy `src/*.{cpp,c}` and `src/CMakeLists.txt` into one directory (or use + `src/` as the source tree and place `client/` beside it). +3. Configure and build: + +```bash +cmake -S . -B build -DCMAKE_BUILD_TYPE=Release +cmake --build build +``` + +Windows (Visual Studio generator): + +```powershell +cmake -S . -B build -A x64 +cmake --build build --config Release +``` + +Each executable is built with the IoTDB runtime library copied **next to the +`.exe` / binary** (POST_BUILD step). Linux/macOS binaries use `$ORIGIN` rpath +so they resolve the `.so` / `.dylib` in the same directory. + +Optional staging folder for deployment: + +```bash +cmake --build build --target example-dist +# -> build/dist/ contains all example binaries + libiotdb_session.{so,dll,dylib} +``` + +## Run on a clean machine (no compiler, no IoTDB SDK headers) + +You only need: + +1. A running IoTDB server reachable from the machine. +2. The **example executable(s)** and the **IoTDB runtime library** in the + **same directory** (or on the system library path). + +Copy either from `build/.../Release/` (Windows) / `build/` (Ninja/Make) or from +`build/dist/` after `example-dist`. + +### Windows + +**Files to copy** + +``` +SessionExample.exe +iotdb_session.dll +``` + +(Repeat for the other example names if needed.) + +**Prerequisites on the target PC** + +- **64-bit Windows** (examples are built x64). +- **[Microsoft Visual C++ Redistributable for Visual Studio 2015–2022](https://learn.microsoft.com/en-us/cpp/windows/latest-supported-vc-redist)** + (x64). The SDK and examples are built with **`/MD`**; the redistributable + supplies `vcruntime140.dll`, `msvcp140.dll`, etc. + Installing this package is enough—you do **not** need Visual Studio or the + IoTDB SDK on the target machine. + +**Run** + +```powershell +.\SessionExample.exe +``` + +If you see “The code execution cannot proceed because VCRUNRuntime140.dll was +missing”, install the VC++ redistributable above. + +You do **not** need a separate Thrift or Boost runtime; they are inside +`iotdb_session.dll`. + +### Linux + +**Files to copy** -You can find some files to form a complete project: ``` -+-- client -| +-- include -| +-- Session.h -| +-- IClientRPCService.h -| +-- rpc_types.h -| +-- rpc_constants.h -| +-- thrift -| +-- thrift_headers... -| +-- lib -| +-- libiotdb_session.dylib -+-- CMakeLists.txt -+-- SessionExample.cpp +SessionExample +libiotdb_session.so +chmod +x SessionExample ``` +**Prerequisites on the target machine** + +- **glibc** on the target must be **≥ the glibc version on the machine that + built the SDK** (backward compatible only in that direction). + +Check **build machine** (record in release notes): + +```bash +ldd --version | head -1 +# e.g. ldd (Ubuntu GLIBC 2.35-0ubuntu3) 2.35 +``` + +Check **target machine**: + +```bash +ldd --version | head -1 +# must be >= build glibc (same major.minor or newer) +``` + +See which `GLIBC_` symbols the binary needs: + +```bash +objdump -T SessionExample | grep GLIBC_ | sed 's/.*GLIBC_/GLIBC_/' | sort -Vu | tail -5 +objdump -T libiotdb_session.so | grep GLIBC_ | sed 's/.*GLIBC_/GLIBC_/' | sort -Vu | tail -5 +``` + +If the target glibc is too old, you'll get errors like +`version 'GLIBC_2.34' not found` at runtime. Rebuild the SDK on an older distro +(or in an older container) to widen compatibility. + +**Run** (with `.so` beside the binary): + +```bash +./SessionExample +``` + +If the shared library is not found: + +```bash +export LD_LIBRARY_PATH=. +./SessionExample +``` + +No separate Thrift install is required. + +### macOS + +Copy the example binary and `libiotdb_session.dylib` together. The target macOS +version should be **≥ the deployment target used to build the SDK**. Check with: + +```bash +otool -L SessionExample +``` + +## Development notes + +- **Windows**: Application and SDK both use **`/MD`** (dynamic CRT). This + matches a default Visual Studio project; link `iotdb_session.lib`, ship + `iotdb_session.dll`. +- **Linux**: SDK is `libiotdb_session.so`; link it directly. Prefer shipping + the `.so` next to your binary or setting `RPATH` to `$ORIGIN`. +- Examples assume IoTDB is listening on `127.0.0.1:6667`; change host/port in + the source if needed. + +## Project layout in this module + +``` +client-cpp-example/ +├── pom.xml # Maven: unpack SDK + invoke CMake +├── README.md +├── README_zh.md +└── src/ + ├── CMakeLists.txt + ├── SessionExample.cpp + ├── AlignedTimeseriesSessionExample.cpp + ├── MultiSvrNodeClient.cpp + └── tree_example.c +``` +After `mvn package`, the runnable tree is under `target/` (sources, `client/`, +and CMake build output). diff --git a/example/client-cpp-example/README_zh.md b/example/client-cpp-example/README_zh.md new file mode 100644 index 0000000000000..84df19792a118 --- /dev/null +++ b/example/client-cpp-example/README_zh.md @@ -0,0 +1,252 @@ + + +# IoTDB C++ 客户端示例 + +[English](README.md) + +本目录提供链接 **IoTDB C++ Session SDK**(`iotdb_session`)的示例程序。 +应用侧编译 **不需要** 单独安装 Thrift 或 Boost 头文件/库,它们已封装在 SDK +共享库内部。 + +所有示例默认连接本机 IoTDB(`127.0.0.1:6667`,用户 `root` / 密码 `root`)。 + +| 示例 | 说明 | +|------|------| +| `SessionExample` | 树模型:建库建序列、写入、查询、删除 | +| `AlignedTimeseriesSessionExample` | 对齐时间序列与模板 | +| `MultiSvrNodeClient` | 多节点写入/查询循环 | +| `tree_example` | C Session API(树模型) | + +## 选择哪个 SDK 压缩包 + +CI 发版([client-cpp-package.yml](../../.github/workflows/client-cpp-package.yml)) +会按平台/工具链打出多份 zip,文件名形如 +`client-cpp--.zip`(解压后根目录即为 `include/` 与 `lib/`)。请按目标环境选择: + +| 目标环境 | classifier 后缀 | +|----------|-----------------| +| Linux x86_64,glibc ≥ 2.24,CXX11 ABI(**推荐**) | `linux-x86_64-glibc224` | +| Linux aarch64,glibc ≥ 2.24,CXX11 ABI(**推荐**) | `linux-aarch64-glibc224` | +| Linux x86_64,glibc ≥ 2.17,旧 libstdc++ ABI | `linux-x86_64-glibc217` | +| Linux aarch64,glibc ≥ 2.17,旧 libstdc++ ABI | `linux-aarch64-glibc217` | +| macOS x86_64 | `mac-x86_64` | +| macOS arm64 | `mac-aarch64` | +| Windows + 与工程相同的 VS 版本 | `windows-x86_64-vs2017` … `vs2026` | + +当前 CMake 构建在配置阶段从源码编译 Thrift 0.21,**不再**通过 +`-Diotdb-tools-thrift.version=0.14.1.1-gcc4-SNAPSHOT` 等旧参数控制 glibc; +Linux 上若部署机 glibc ≥ 2.24 且使用系统默认 `g++`,请优先选用 **`glibc224`** +包。仅当目标机停留在 glibc 2.17(如 CentOS 7)或必须与旧 libstdc++ ABI 一致时, +选用 **`glibc217`** 包;在 Ubuntu 22/24 上链 `glibc217` 时常需 +`-D_GLIBCXX_USE_CXX11_ABI=0`。详见 [client-cpp README](../../iotdb-client/client-cpp/README.md)。 + +## SDK 目录结构(解压后) + +`client-cpp` 打出的 SDK 压缩包只包含 **公开头文件** 和 **一个共享库**: + +``` +client/ +├── include/ +│ ├── Session.h +│ ├── Export.h +│ └── ... (17 个公开头;无 thrift/、boost/) +└── lib/ + ├── iotdb_session.dll + iotdb_session.lib (Windows) + ├── libiotdb_session.so (Linux) + └── libiotdb_session.dylib (macOS) +``` + +## 编译示例 + +### 方式 A:Maven(本仓库推荐) + +在仓库根目录执行: + +```bash +mvn clean package -DskipTests -P with-cpp -pl example/client-cpp-example -am +``` + +Maven 会将 SDK 解压到 `example/client-cpp-example/target/client/`,并在 +`target/` 下调用 CMake。可执行文件位于 `target/`(具体路径取决于生成器; +Windows + Visual Studio 一般为 `target/Release/`)。 + +### 方式 B:仅 CMake(手动准备 SDK) + +1. 自行编译或下载 SDK,解压后保证存在 `client/include` 与 `client/lib`(见 + 上文目录结构)。 +2. 将 `src/*.{cpp,c}` 与 `src/CMakeLists.txt` 放在同一目录(或保留 `src/` + 结构,并在同级放置 `client/`)。 +3. 配置并编译: + +```bash +cmake -S . -B build -DCMAKE_BUILD_TYPE=Release +cmake --build build +``` + +Windows(Visual Studio 生成器): + +```powershell +cmake -S . -B build -A x64 +cmake --build build --config Release +``` + +编译完成后,IoTDB 运行时库会通过 POST_BUILD 自动复制到 **与可执行文件相同 +的目录**。Linux/macOS 可执行文件还设置了 `$ORIGIN` rpath,可在同目录加载 +`.so` / `.dylib`。 + +可选:打部署包目录 + +```bash +cmake --build build --target example-dist +# 生成 build/dist/,内含全部示例二进制 + libiotdb_session.{so,dll,dylib} +``` + +## 在「干净机器」上运行(无需编译器、无需 SDK 头文件) + +目标机器只需: + +1. 可访问的 **IoTDB 服务**(已启动)。 +2. **示例可执行文件** 与 **IoTDB 运行时库** 放在 **同一目录**(或位于系统 + 库搜索路径中)。 + +可从 `build/.../Release/`(Windows)或 `build/`(Ninja/Make)复制,也可在 +执行 `example-dist` 后直接使用 `build/dist/`。 + +### Windows + +**需要拷贝的文件** + +``` +SessionExample.exe +iotdb_session.dll +``` + +(其他示例同理,可执行文件与 `iotdb_session.dll` 成对拷贝。) + +**目标机器前置条件** + +- **64 位 Windows**(示例为 x64 构建)。 +- 安装 **[Microsoft Visual C++ 2015–2022 可再发行组件包(x64)](https://learn.microsoft.com/zh-cn/cpp/windows/latest-supported-vc-redist)**。 + SDK 与示例均使用 **`/MD`**(动态 CRT),该安装包提供 `vcruntime140.dll`、 + `msvcp140.dll` 等运行时。 + **仅安装此 Redistributable 即可**,目标机 **不需要** Visual Studio,也 + **不需要** IoTDB SDK 头文件或 Thrift/Boost。 + +**运行** + +```powershell +.\SessionExample.exe +``` + +若提示缺少 `VCRUNTIME140.dll`,请安装上述 VC++ 可再发行包。 + +Thrift、Boost 已包含在 `iotdb_session.dll` 内,无需单独部署。 + +### Linux + +**需要拷贝的文件** + +``` +SessionExample +libiotdb_session.so +chmod +x SessionExample +``` + +**目标机器前置条件** + +- 目标机的 **glibc 版本必须 ≥ 编译 SDK 时的 glibc 版本**(仅向后兼容: + 新系统可跑旧库要求,旧系统不能跑需要更高 glibc 的二进制)。 + +在 **编译机** 上记录版本(建议写入发布说明): + +```bash +ldd --version | head -1 +# 例如:ldd (Ubuntu GLIBC 2.35-0ubuntu3) 2.35 +``` + +在 **目标机** 上检查: + +```bash +ldd --version | head -1 +# 版本号应 >= 编译机(同主版本次版本或更新) +``` + +查看二进制依赖的最高 `GLIBC_` 符号: + +```bash +objdump -T SessionExample | grep GLIBC_ | sed 's/.*GLIBC_/GLIBC_/' | sort -Vu | tail -5 +objdump -T libiotdb_session.so | grep GLIBC_ | sed 's/.*GLIBC_/GLIBC_/' | sort -Vu | tail -5 +``` + +若目标 glibc 过旧,运行时会报错,例如 +`version 'GLIBC_2.34' not found`。可在更旧的发行版(或旧版容器)上重新编译 +SDK,以扩大兼容范围。 + +**运行**(`.so` 与可执行文件同目录): + +```bash +./SessionExample +``` + +若找不到共享库: + +```bash +export LD_LIBRARY_PATH=. +./SessionExample +``` + +无需单独安装 Thrift。 + +### macOS + +将示例可执行文件与 `libiotdb_session.dylib` 放在同一目录。目标 macOS 版本 +应 **≥ 编译 SDK 时设置的 deployment target**。可用以下命令检查依赖: + +```bash +otool -L SessionExample +``` + +## 开发说明 + +- **Windows**:应用与 SDK 均使用 **`/MD`**,与 Visual Studio 默认工程一致; + 链接 `iotdb_session.lib`,部署时携带 `iotdb_session.dll`。 +- **Linux**:直接链接 `libiotdb_session.so`;建议与可执行文件同目录发布,或 + 设置 `RPATH=$ORIGIN`。 +- 示例默认连接 `127.0.0.1:6667`;如需修改地址/端口,请编辑对应源码。 + +## 本模块目录结构 + +``` +client-cpp-example/ +├── pom.xml # Maven:解压 SDK + 调用 CMake +├── README.md # 英文说明 +├── README_zh.md # 中文说明(本文件) +└── src/ + ├── CMakeLists.txt + ├── SessionExample.cpp + ├── AlignedTimeseriesSessionExample.cpp + ├── MultiSvrNodeClient.cpp + └── tree_example.c +``` + +执行 `mvn package` 后,可在 `target/` 下找到源码、`client/` SDK 与 CMake +构建产物。 diff --git a/example/client-cpp-example/pom.xml b/example/client-cpp-example/pom.xml index cf6f65775df73..c6ac16dbdf09f 100644 --- a/example/client-cpp-example/pom.xml +++ b/example/client-cpp-example/pom.xml @@ -28,6 +28,9 @@ client-cpp-example IoTDB: Example: CPP Client + + ${os.classifier} + org.apache.iotdb @@ -60,6 +63,14 @@ ${project.basedir}/src/AlignedTimeseriesSessionExample.cpp ${project.build.directory}/AlignedTimeseriesSessionExample.cpp + + ${project.basedir}/src/MultiSvrNodeClient.cpp + ${project.build.directory}/MultiSvrNodeClient.cpp + + + ${project.basedir}/src/tree_example.c + ${project.build.directory}/tree_example.c + ${project.basedir}/src/CMakeLists.txt ${project.build.directory}/CMakeLists.txt @@ -90,37 +101,17 @@ client-cpp ${project.version} zip - cpp-${os.classifier} + ${client.cpp.package.classifier} true ${project.build.directory}/client - - unpack-thrift - - unpack - - generate-sources - - - - org.apache.iotdb.tools - iotdb-tools-thrift - ${iotdb-tools-thrift.version} - ${os.classifier} - zip - true - ${project.build.directory}/thrift - - - - - com.googlecode.cmake-maven-project + io.github.cmake-maven-plugin cmake-maven-plugin @@ -132,10 +123,7 @@ ${cmake.generator} ${project.build.directory} - ${project.build.directory} - - - + ${project.build.directory} @@ -154,4 +142,100 @@ + + + .os-unix + + + Linux + unix + !aarch64 + + + + linux-x86_64-glibc2.28 + + + + .os-unix-arm + + + Linux + unix + aarch64 + + + + linux-aarch64-glibc2.28 + + + + .os-mac + + + mac + !aarch64 + + + + macos-x86_64 + + + + .os-mac-arm + + + mac + aarch64 + + + + macos-aarch64 + + + + .os-windows + + + windows + + + + windows-x86_64-msvc14.3 + + + + spotless-cpp + + [11,) + + + + + com.diffplug.spotless + spotless-maven-plugin + 2.44.5 + + + + src/**/*.h + src/**/*.hpp + src/**/*.c + src/**/*.cc + src/**/*.cpp + + + src/test/catch2/** + + + ${clang.format.version} + + + + + + + + + diff --git a/example/client-cpp-example/src/AlignedTimeseriesSessionExample.cpp b/example/client-cpp-example/src/AlignedTimeseriesSessionExample.cpp index 8f175cb7650c6..c8e00efe68e8b 100644 --- a/example/client-cpp-example/src/AlignedTimeseriesSessionExample.cpp +++ b/example/client-cpp-example/src/AlignedTimeseriesSessionExample.cpp @@ -26,388 +26,403 @@ Session *session; #define DEFAULT_ROW_NUMBER 1000000 void createAlignedTimeseries() { - string alignedDeviceId = "root.sg1.d1"; - vector measurements = {"s1", "s2", "s3"}; - vector alignedTimeseries = {"root.sg1.d1.s1", "root.sg1.d1.s2", "root.sg1.d1.s3"}; - vector dataTypes = {TSDataType::INT32, TSDataType::DOUBLE, TSDataType::BOOLEAN}; - vector encodings = {TSEncoding::PLAIN, TSEncoding::GORILLA, TSEncoding::RLE}; - vector compressors = { - CompressionType::SNAPPY, CompressionType::UNCOMPRESSED, CompressionType::SNAPPY}; - for (const string ×eries: alignedTimeseries) { - if (session->checkTimeseriesExists(timeseries)) { - session->deleteTimeseries(timeseries); - } + string alignedDeviceId = "root.sg1.d1"; + vector measurements = {"s1", "s2", "s3"}; + vector alignedTimeseries = {"root.sg1.d1.s1", "root.sg1.d1.s2", + "root.sg1.d1.s3"}; + vector dataTypes = { + TSDataType::INT32, TSDataType::DOUBLE, TSDataType::BOOLEAN}; + vector encodings = { + TSEncoding::PLAIN, TSEncoding::GORILLA, TSEncoding::RLE}; + vector compressors = { + CompressionType::SNAPPY, CompressionType::UNCOMPRESSED, + CompressionType::SNAPPY}; + for (const string ×eries : alignedTimeseries) { + if (session->checkTimeseriesExists(timeseries)) { + session->deleteTimeseries(timeseries); } - session->createAlignedTimeseries(alignedDeviceId, measurements, dataTypes, encodings, compressors); + } + session->createAlignedTimeseries(alignedDeviceId, measurements, dataTypes, + encodings, compressors); } void createSchemaTemplate() { - if (!session->checkTemplateExists("template1")) { - Template temp("template1", false); - - InternalNode iNodeD99("d99", true); - - MeasurementNode mNodeS1("s1", TSDataType::INT32, TSEncoding::RLE, CompressionType::SNAPPY); - MeasurementNode mNodeS2("s2", TSDataType::INT64, TSEncoding::RLE, CompressionType::SNAPPY); - MeasurementNode mNodeD99S1("s1", TSDataType::DOUBLE, TSEncoding::RLE, CompressionType::SNAPPY); - MeasurementNode mNodeD99S2("s2", TSDataType::BOOLEAN, TSEncoding::RLE, CompressionType::SNAPPY); - - iNodeD99.addChild(mNodeD99S1); - iNodeD99.addChild(mNodeD99S2); - - temp.addToTemplate(iNodeD99); - temp.addToTemplate(mNodeS1); - temp.addToTemplate(mNodeS2); - - session->createSchemaTemplate(temp); - session->setSchemaTemplate("template1", "root.sg3.d1"); - } + if (!session->checkTemplateExists("template1")) { + Template temp("template1", false); + + InternalNode iNodeD99("d99", true); + + MeasurementNode mNodeS1("s1", TSDataType::INT32, TSEncoding::RLE, + CompressionType::SNAPPY); + MeasurementNode mNodeS2("s2", TSDataType::INT64, TSEncoding::RLE, + CompressionType::SNAPPY); + MeasurementNode mNodeD99S1("s1", TSDataType::DOUBLE, TSEncoding::RLE, + CompressionType::SNAPPY); + MeasurementNode mNodeD99S2("s2", TSDataType::BOOLEAN, TSEncoding::RLE, + CompressionType::SNAPPY); + + iNodeD99.addChild(mNodeD99S1); + iNodeD99.addChild(mNodeD99S2); + + temp.addToTemplate(iNodeD99); + temp.addToTemplate(mNodeS1); + temp.addToTemplate(mNodeS2); + + session->createSchemaTemplate(temp); + session->setSchemaTemplate("template1", "root.sg3.d1"); + } } void ActivateTemplate() { - session->executeNonQueryStatement("insert into root.sg3.d1(timestamp,s1, s2) values(200, 1, 1);"); + session->executeNonQueryStatement( + "insert into root.sg3.d1(timestamp,s1, s2) values(200, 1, 1);"); } void showDevices() { - unique_ptr dataSet = session->executeQueryStatement("show devices with database"); - for (const string &name: dataSet->getColumnNames()) { - cout << name << " "; - } - cout << endl; - - dataSet->setFetchSize(1024); - while (dataSet->hasNext()) { - cout << dataSet->next()->toString(); - } - cout << endl; - - dataSet->closeOperationHandle(); + unique_ptr dataSet = + session->executeQueryStatement("show devices with database"); + for (const string &name : dataSet->getColumnNames()) { + cout << name << " "; + } + cout << endl; + + dataSet->setFetchSize(1024); + while (dataSet->hasNext()) { + cout << dataSet->next()->toString(); + } + cout << endl; + + dataSet->closeOperationHandle(); } void showTimeseries() { - unique_ptr dataSet = session->executeQueryStatement("show timeseries"); - for (const string &name: dataSet->getColumnNames()) { - cout << name << " "; - } - cout << endl; - - dataSet->setFetchSize(1024); - while (dataSet->hasNext()) { - cout << dataSet->next()->toString(); - } - cout << endl; - - dataSet->closeOperationHandle(); + unique_ptr dataSet = + session->executeQueryStatement("show timeseries"); + for (const string &name : dataSet->getColumnNames()) { + cout << name << " "; + } + cout << endl; + + dataSet->setFetchSize(1024); + while (dataSet->hasNext()) { + cout << dataSet->next()->toString(); + } + cout << endl; + + dataSet->closeOperationHandle(); } void insertAlignedRecord() { - string deviceId = "root.sg1.d1"; - vector measurements; - measurements.emplace_back("s1"); - measurements.emplace_back("s2"); - measurements.emplace_back("s3"); - - for (int64_t time = 0; time < 10; time++) { - vector values; - values.emplace_back("1"); - values.emplace_back("1.0"); - values.emplace_back("true"); - session->insertAlignedRecord(deviceId, time, measurements, values); - } + string deviceId = "root.sg1.d1"; + vector measurements; + measurements.emplace_back("s1"); + measurements.emplace_back("s2"); + measurements.emplace_back("s3"); + + for (int64_t time = 0; time < 10; time++) { + vector values; + values.emplace_back("1"); + values.emplace_back("1.0"); + values.emplace_back("true"); + session->insertAlignedRecord(deviceId, time, measurements, values); + } } void insertAlignedRecords() { - string deviceId = "root.sg1.d1"; - vector measurements; - measurements.emplace_back("s1"); - measurements.emplace_back("s2"); - measurements.emplace_back("s3"); - - vector deviceIds; - vector> measurementsList; - vector> valuesList; - vector timestamps; - - for (int64_t time = 10; time < 20; time++) { - vector values; - values.emplace_back("1"); - values.emplace_back("1.0"); - values.emplace_back("true"); - - deviceIds.push_back(deviceId); - measurementsList.push_back(measurements); - valuesList.push_back(values); - timestamps.push_back(time); - if (time != 10 && time % 10 == 0) { - session->insertAlignedRecords(deviceIds, timestamps, measurementsList, valuesList); - deviceIds.clear(); - measurementsList.clear(); - valuesList.clear(); - timestamps.clear(); - } + string deviceId = "root.sg1.d1"; + vector measurements; + measurements.emplace_back("s1"); + measurements.emplace_back("s2"); + measurements.emplace_back("s3"); + + vector deviceIds; + vector> measurementsList; + vector> valuesList; + vector timestamps; + + for (int64_t time = 10; time < 20; time++) { + vector values; + values.emplace_back("1"); + values.emplace_back("1.0"); + values.emplace_back("true"); + + deviceIds.push_back(deviceId); + measurementsList.push_back(measurements); + valuesList.push_back(values); + timestamps.push_back(time); + if (time != 10 && time % 10 == 0) { + session->insertAlignedRecords(deviceIds, timestamps, measurementsList, + valuesList); + deviceIds.clear(); + measurementsList.clear(); + valuesList.clear(); + timestamps.clear(); } + } - session->insertAlignedRecords(deviceIds, timestamps, measurementsList, valuesList); + session->insertAlignedRecords(deviceIds, timestamps, measurementsList, + valuesList); } void insertAlignedTablet() { - pair pairA("s1", TSDataType::INT32); - pair pairB("s2", TSDataType::DOUBLE); - pair pairC("s3", TSDataType::DOUBLE); - vector> schemas; - schemas.push_back(pairA); - schemas.push_back(pairB); - schemas.push_back(pairC); - - Tablet tablet("root.sg2.d2", schemas, 100000); - tablet.setAligned(true); - - for (int64_t time = 0; time < DEFAULT_ROW_NUMBER; time++) { - size_t row = tablet.rowSize++; - tablet.timestamps[row] = time; - int randVal1 = 123456; - double randVal2 = 123456.1234; - double randVal3 = 123456.1234; - tablet.addValue(0, row, &randVal1); - tablet.addValue(1, row, &randVal2); - tablet.addValue(2, row, &randVal3); - if (tablet.rowSize == tablet.maxRowNumber) { - session->insertTablet(tablet, true); - tablet.reset(); - } + pair pairA("s1", TSDataType::INT32); + pair pairB("s2", TSDataType::DOUBLE); + pair pairC("s3", TSDataType::DOUBLE); + vector> schemas; + schemas.push_back(pairA); + schemas.push_back(pairB); + schemas.push_back(pairC); + + Tablet tablet("root.sg2.d2", schemas, 100000); + tablet.setAligned(true); + + for (int64_t time = 0; time < DEFAULT_ROW_NUMBER; time++) { + size_t row = tablet.rowSize++; + tablet.timestamps[row] = time; + int randVal1 = 123456; + double randVal2 = 123456.1234; + double randVal3 = 123456.1234; + tablet.addValue(0, row, randVal1); + tablet.addValue(1, row, randVal2); + tablet.addValue(2, row, randVal3); + if (tablet.rowSize == tablet.maxRowNumber) { + session->insertTablet(tablet, true); + tablet.reset(); } + } - if (tablet.rowSize != 0) { - session->insertTablet(tablet); - tablet.reset(); - } + if (tablet.rowSize != 0) { + session->insertTablet(tablet); + tablet.reset(); + } } void insertAlignedTablets() { - pair pairA("s1", TSDataType::INT32); - pair pairB("s2", TSDataType::DOUBLE); - pair pairC("s3", TSDataType::BOOLEAN); - vector> schemas; - schemas.push_back(pairA); - schemas.push_back(pairB); - schemas.push_back(pairC); - - Tablet tablet1("root.sg1.d1", schemas, 100); - Tablet tablet2("root.sg1.d2", schemas, 100); - Tablet tablet3("root.sg1.d3", schemas, 100); - - unordered_map tabletMap; - tabletMap["root.sg1.d1"] = &tablet1; - tabletMap["root.sg1.d2"] = &tablet2; - tabletMap["root.sg1.d3"] = &tablet3; - - for (int64_t time = 0; time < 20; time++) { - size_t row1 = tablet1.rowSize++; - size_t row2 = tablet2.rowSize++; - size_t row3 = tablet3.rowSize++; - tablet1.timestamps[row1] = time; - tablet2.timestamps[row2] = time; - tablet3.timestamps[row3] = time; - - int randVal11 = rand(); - int randVal12 = rand(); - int randVal13 = rand(); - tablet1.addValue(0, row1, &randVal11); - tablet2.addValue(0, row2, &randVal12); - tablet3.addValue(0, row3, &randVal13); - - double randVal21 = rand() / 99.9; - double randVal22 = rand() / 99.9; - double randVal23 = rand() / 99.9; - tablet1.addValue(1, row1, &randVal21); - tablet2.addValue(1, row2, &randVal22); - tablet3.addValue(1, row3, &randVal23); - - bool randVal31 = (bool)(rand() % 2); - bool randVal32 = (bool)(rand() % 2); - bool randVal33 = (bool)(rand() % 2); - tablet1.addValue(2, row1, &randVal31); - tablet2.addValue(2, row2, &randVal32); - tablet3.addValue(2, row3, &randVal33); - - if (tablet1.rowSize == tablet1.maxRowNumber) { - session->insertAlignedTablets(tabletMap, true); - - tablet1.reset(); - tablet2.reset(); - tablet3.reset(); - } - } - - if (tablet1.rowSize != 0) { - session->insertAlignedTablets(tabletMap, true); - tablet1.reset(); - tablet2.reset(); - tablet3.reset(); + pair pairA("s1", TSDataType::INT32); + pair pairB("s2", TSDataType::DOUBLE); + pair pairC("s3", TSDataType::BOOLEAN); + vector> schemas; + schemas.push_back(pairA); + schemas.push_back(pairB); + schemas.push_back(pairC); + + Tablet tablet1("root.sg1.d1", schemas, 100); + Tablet tablet2("root.sg1.d2", schemas, 100); + Tablet tablet3("root.sg1.d3", schemas, 100); + + unordered_map tabletMap; + tabletMap["root.sg1.d1"] = &tablet1; + tabletMap["root.sg1.d2"] = &tablet2; + tabletMap["root.sg1.d3"] = &tablet3; + + for (int64_t time = 0; time < 20; time++) { + size_t row1 = tablet1.rowSize++; + size_t row2 = tablet2.rowSize++; + size_t row3 = tablet3.rowSize++; + tablet1.timestamps[row1] = time; + tablet2.timestamps[row2] = time; + tablet3.timestamps[row3] = time; + + int randVal11 = rand(); + int randVal12 = rand(); + int randVal13 = rand(); + tablet1.addValue(0, row1, randVal11); + tablet2.addValue(0, row2, randVal12); + tablet3.addValue(0, row3, randVal13); + + double randVal21 = rand() / 99.9; + double randVal22 = rand() / 99.9; + double randVal23 = rand() / 99.9; + tablet1.addValue(1, row1, randVal21); + tablet2.addValue(1, row2, randVal22); + tablet3.addValue(1, row3, randVal23); + + bool randVal31 = (bool)(rand() % 2); + bool randVal32 = (bool)(rand() % 2); + bool randVal33 = (bool)(rand() % 2); + tablet1.addValue(2, row1, randVal31); + tablet2.addValue(2, row2, randVal32); + tablet3.addValue(2, row3, randVal33); + + if (tablet1.rowSize == tablet1.maxRowNumber) { + session->insertAlignedTablets(tabletMap, true); + + tablet1.reset(); + tablet2.reset(); + tablet3.reset(); } + } + + if (tablet1.rowSize != 0) { + session->insertAlignedTablets(tabletMap, true); + tablet1.reset(); + tablet2.reset(); + tablet3.reset(); + } } void insertNullableTabletWithAlignedTimeseries() { - pair pairA("s1", TSDataType::INT32); - pair pairB("s2", TSDataType::INT64); - pair pairC("s3", TSDataType::BOOLEAN); - vector> schemas; - schemas.push_back(pairA); - schemas.push_back(pairB); - schemas.push_back(pairC); - - Tablet tablet("root.sg1.d4", schemas, 20); - tablet.setAligned(true); - - for (int64_t time = 0; time < 20; time++) { - size_t row = tablet.rowSize++; - tablet.timestamps[row] = time; - for (int i = 0; i < 3; i++) { - int randVal1 = rand(); - int64_t randVal2 = rand(); - bool randVal3 = (bool)(rand() % 2); - if (i == 0) { - tablet.addValue(i, row, &randVal1); - } else if (i == 1) { - tablet.addValue(i, row, &randVal2); - } else { - tablet.addValue(i, row, &randVal3); - } - // mark null value - if ((row % 3) == (unsigned int) i) { - tablet.bitMaps[i].mark(row); - } - } - if (tablet.rowSize == tablet.maxRowNumber) { - session->insertTablet(tablet, true); - tablet.reset(); - } + pair pairA("s1", TSDataType::INT32); + pair pairB("s2", TSDataType::INT64); + pair pairC("s3", TSDataType::BOOLEAN); + vector> schemas; + schemas.push_back(pairA); + schemas.push_back(pairB); + schemas.push_back(pairC); + + Tablet tablet("root.sg1.d4", schemas, 20); + tablet.setAligned(true); + + for (int64_t time = 0; time < 20; time++) { + size_t row = tablet.rowSize++; + tablet.timestamps[row] = time; + for (int i = 0; i < 3; i++) { + int randVal1 = rand(); + int64_t randVal2 = rand(); + bool randVal3 = (bool)(rand() % 2); + if (i == 0) { + tablet.addValue(i, row, randVal1); + } else if (i == 1) { + tablet.addValue(i, row, randVal2); + } else { + tablet.addValue(i, row, randVal3); + } + // mark null value + if ((row % 3) == (unsigned int)i) { + tablet.bitMaps[i].mark(row); + } } - - if (tablet.rowSize != 0) { - session->insertTablet(tablet); - tablet.reset(); + if (tablet.rowSize == tablet.maxRowNumber) { + session->insertTablet(tablet, true); + tablet.reset(); } + } + + if (tablet.rowSize != 0) { + session->insertTablet(tablet); + tablet.reset(); + } } void query() { - unique_ptr dataSet = session->executeQueryStatement("select * from root.sg1.**"); - cout << "timestamp" << " "; - for (const string &name: dataSet->getColumnNames()) { - cout << name << " "; - } - cout << endl; - - dataSet->setFetchSize(1024); - while (dataSet->hasNext()) { - cout << dataSet->next()->toString(); - } - cout << endl; - - dataSet->closeOperationHandle(); + unique_ptr dataSet = + session->executeQueryStatement("select * from root.sg1.**"); + cout << "timestamp" + << " "; + for (const string &name : dataSet->getColumnNames()) { + cout << name << " "; + } + cout << endl; + + dataSet->setFetchSize(1024); + while (dataSet->hasNext()) { + cout << dataSet->next()->toString(); + } + cout << endl; + + dataSet->closeOperationHandle(); } void deleteData() { - string path = "root.**"; - int64_t deleteTime = 49; - session->deleteData(path, deleteTime); + string path = "root.**"; + int64_t deleteTime = 49; + session->deleteData(path, deleteTime); } void deleteTimeseries() { - vector paths; - vector alignedTimeseries = {"root.sg1.d1.s1", "root.sg1.d1.s2", "root.sg1.d1.s3", "root.sg1.d1.s4", - "root.sg1.d2.s1", "root.sg1.d2.s2", "root.sg1.d2.s3", - "root.sg1.d3.s1", "root.sg1.d3.s2", "root.sg1.d3.s3", - "root.sg1.d4.s1", "root.sg1.d4.s2", "root.sg1.d4.s3", - "root.sg2.d2.s1", "root.sg2.d2.s2", "root.sg2.d2.s3", }; - for (const string ×eries: alignedTimeseries) { - if (session->checkTimeseriesExists(timeseries)) { - paths.push_back(timeseries); - } + vector paths; + vector alignedTimeseries = { + "root.sg1.d1.s1", "root.sg1.d1.s2", "root.sg1.d1.s3", "root.sg1.d1.s4", + "root.sg1.d2.s1", "root.sg1.d2.s2", "root.sg1.d2.s3", "root.sg1.d3.s1", + "root.sg1.d3.s2", "root.sg1.d3.s3", "root.sg1.d4.s1", "root.sg1.d4.s2", + "root.sg1.d4.s3", "root.sg2.d2.s1", "root.sg2.d2.s2", "root.sg2.d2.s3", + }; + for (const string ×eries : alignedTimeseries) { + if (session->checkTimeseriesExists(timeseries)) { + paths.push_back(timeseries); } - session->deleteTimeseries(paths); + } + session->deleteTimeseries(paths); } void deleteStorageGroups() { - vector storageGroups; - storageGroups.emplace_back("root.sg1"); - storageGroups.emplace_back("root.sg2"); - session->deleteStorageGroups(storageGroups); + vector storageGroups; + storageGroups.emplace_back("root.sg1"); + storageGroups.emplace_back("root.sg2"); + session->deleteStorageGroups(storageGroups); } - int main() { - LOG_LEVEL = LEVEL_DEBUG; + LOG_LEVEL = LEVEL_DEBUG; - session = new Session("127.0.0.1", 6667, "root", "root"); + session = new Session("127.0.0.1", 6667, "root", "root"); - cout << "session open\n" << endl; - session->open(false); + cout << "session open\n" << endl; + session->open(false); - cout << "setStorageGroup\n" << endl; - try { - session->setStorageGroup("root.sg1"); - } - catch (IoTDBException &e) { - string errorMessage(e.what()); - if (errorMessage.find("StorageGroupAlreadySetException") == string::npos) { - cout << errorMessage << endl; - //throw e; - } + cout << "setStorageGroup\n" << endl; + try { + session->setStorageGroup("root.sg1"); + } catch (IoTDBException &e) { + string errorMessage(e.what()); + if (errorMessage.find("StorageGroupAlreadySetException") == string::npos) { + cout << errorMessage << endl; + // throw e; } + } - cout << "createAlignedTimeseries\n" << endl; - createAlignedTimeseries(); + cout << "createAlignedTimeseries\n" << endl; + createAlignedTimeseries(); - cout << "createSchemaTemplate\n" << endl; - createSchemaTemplate(); + cout << "createSchemaTemplate\n" << endl; + createSchemaTemplate(); - cout << "ActivateTemplate\n" << endl; - ActivateTemplate(); + cout << "ActivateTemplate\n" << endl; + ActivateTemplate(); - cout << "showDevices\n" << endl; - showDevices(); + cout << "showDevices\n" << endl; + showDevices(); - cout << "showTimeseries\n" << endl; - showTimeseries(); + cout << "showTimeseries\n" << endl; + showTimeseries(); - cout << "insertAlignedRecord\n" << endl; - insertAlignedRecord(); + cout << "insertAlignedRecord\n" << endl; + insertAlignedRecord(); - cout << "insertAlignedRecords\n" << endl; - insertAlignedRecords(); + cout << "insertAlignedRecords\n" << endl; + insertAlignedRecords(); - cout << "insertAlignedTablet" << endl; - cout << "Insert " << DEFAULT_ROW_NUMBER << " records." << endl; - time_t now1 = time(0); - insertAlignedTablet(); - time_t now2 = time(0); - time_t useTime = now2 - now1; - cout << "Use time: " << useTime << "s.\n" << endl; + cout << "insertAlignedTablet" << endl; + cout << "Insert " << DEFAULT_ROW_NUMBER << " records." << endl; + time_t now1 = time(0); + insertAlignedTablet(); + time_t now2 = time(0); + time_t useTime = now2 - now1; + cout << "Use time: " << useTime << "s.\n" << endl; - cout << "insertAlignedTablets\n" << endl; - insertAlignedTablets(); + cout << "insertAlignedTablets\n" << endl; + insertAlignedTablets(); - cout << "insertNullableTabletWithAlignedTimeseries\n" << endl; - insertNullableTabletWithAlignedTimeseries(); + cout << "insertNullableTabletWithAlignedTimeseries\n" << endl; + insertNullableTabletWithAlignedTimeseries(); - cout << "query\n" << endl; - query(); + cout << "query\n" << endl; + query(); - cout << "deleteData\n" << endl; - deleteData(); + cout << "deleteData\n" << endl; + deleteData(); - cout << "deleteTimeseries\n" << endl; - deleteTimeseries(); + cout << "deleteTimeseries\n" << endl; + deleteTimeseries(); - cout << "deleteStorageGroups\n" << endl; - deleteStorageGroups(); + cout << "deleteStorageGroups\n" << endl; + deleteStorageGroups(); - cout << "session close\n" << endl; - session->close(); + cout << "session close\n" << endl; + session->close(); - delete session; + delete session; - cout << "finished\n" << endl; - return 0; + cout << "finished\n" << endl; + return 0; } diff --git a/example/client-cpp-example/src/CMakeLists.txt b/example/client-cpp-example/src/CMakeLists.txt index 91d93193e38b9..e4bb9eac354c4 100644 --- a/example/client-cpp-example/src/CMakeLists.txt +++ b/example/client-cpp-example/src/CMakeLists.txt @@ -16,33 +16,134 @@ # under the License. # -PROJECT(SessionExample) -CMAKE_MINIMUM_REQUIRED(VERSION 3.7) +CMAKE_MINIMUM_REQUIRED(VERSION 3.15) +CMAKE_POLICY(SET CMP0091 NEW) + +PROJECT(iotdb_cpp_client_examples) SET(CMAKE_CXX_STANDARD 11) SET(CMAKE_CXX_STANDARD_REQUIRED ON) SET(CMAKE_POSITION_INDEPENDENT_CODE ON) -# Add Thrift include directory -INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/thrift/include) -# Add cpp-client include directory -INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/client/include) +IF(MSVC) + # Match the IoTDB C++ SDK (/MD); same as a default Visual Studio application. + SET(CMAKE_MSVC_RUNTIME_LIBRARY "MultiThreaded$<$:Debug>DLL") +ENDIF() + +set(IOTDB_SDK_ROOT "${CMAKE_SOURCE_DIR}/client" + CACHE PATH "Unpacked IoTDB C++ SDK directory (contains include/ and lib/)") -FIND_PACKAGE(Boost REQUIRED) -IF (DEFINED BOOST_INCLUDEDIR) - INCLUDE_DIRECTORIES(${Boost_INCLUDE_DIR}) +if(NOT EXISTS "${IOTDB_SDK_ROOT}/include/Session.h" + OR NOT EXISTS "${IOTDB_SDK_ROOT}/include/SessionC.h") + file(GLOB _iotdb_sdk_dirs LIST_DIRECTORIES true "${IOTDB_SDK_ROOT}/iotdb-session-cpp-*") + foreach(_iotdb_sdk_dir IN LISTS _iotdb_sdk_dirs) + if(EXISTS "${_iotdb_sdk_dir}/include/Session.h" + AND EXISTS "${_iotdb_sdk_dir}/include/SessionC.h") + set(IOTDB_SDK_ROOT "${_iotdb_sdk_dir}" + CACHE PATH "Unpacked IoTDB C++ SDK directory (contains include/ and lib/)" FORCE) + break() + endif() + endforeach() +endif() + +INCLUDE_DIRECTORIES("${IOTDB_SDK_ROOT}/include") + +option(WITH_SSL "Build with SSL support" OFF) + +IF(WITH_SSL) + FIND_PACKAGE(OpenSSL REQUIRED) + IF(OpenSSL_FOUND) + MESSAGE(STATUS "OpenSSL found: ${OPENSSL_VERSION}") + INCLUDE_DIRECTORIES(${OPENSSL_INCLUDE_DIR}) + ADD_DEFINITIONS(-DWITH_SSL=1) + ELSE() + MESSAGE(FATAL_ERROR "OpenSSL not found, but WITH_SSL is enabled") + ENDIF() +ELSE() + MESSAGE(STATUS "Building without SSL support") + ADD_DEFINITIONS(-DWITH_SSL=0) ENDIF() -# Add the libs for the cpp-client -LINK_DIRECTORIES(${CMAKE_SOURCE_DIR}/client/lib) +if(WIN32) + set(_iotdb_link_lib "${IOTDB_SDK_ROOT}/lib/iotdb_session.lib") + set(_iotdb_runtime "${IOTDB_SDK_ROOT}/lib/iotdb_session.dll") + if(NOT EXISTS "${_iotdb_link_lib}" AND CMAKE_BUILD_TYPE) + set(_iotdb_link_lib "${IOTDB_SDK_ROOT}/lib/${CMAKE_BUILD_TYPE}/iotdb_session.lib") + set(_iotdb_runtime "${IOTDB_SDK_ROOT}/lib/${CMAKE_BUILD_TYPE}/iotdb_session.dll") + endif() + if(NOT EXISTS "${_iotdb_link_lib}") + set(_iotdb_link_lib "${IOTDB_SDK_ROOT}/lib/Release/iotdb_session.lib") + set(_iotdb_runtime "${IOTDB_SDK_ROOT}/lib/Release/iotdb_session.dll") + endif() + if(NOT EXISTS "${_iotdb_link_lib}") + set(_iotdb_link_lib "${IOTDB_SDK_ROOT}/lib/Debug/iotdb_session.lib") + set(_iotdb_runtime "${IOTDB_SDK_ROOT}/lib/Debug/iotdb_session.dll") + endif() +elseif(APPLE) + set(_iotdb_link_lib "${IOTDB_SDK_ROOT}/lib/libiotdb_session.dylib") + set(_iotdb_runtime "${_iotdb_link_lib}") +else() + set(_iotdb_link_lib "${IOTDB_SDK_ROOT}/lib/libiotdb_session.so") + set(_iotdb_runtime "${_iotdb_link_lib}") +endif() + +if(NOT EXISTS "${_iotdb_link_lib}") + message(FATAL_ERROR + "IoTDB SDK not found at ${IOTDB_SDK_ROOT}. " + "Unpack client-cpp--.zip so that ${_iotdb_link_lib} exists.") +endif() ADD_EXECUTABLE(SessionExample SessionExample.cpp) ADD_EXECUTABLE(AlignedTimeseriesSessionExample AlignedTimeseriesSessionExample.cpp) +ADD_EXECUTABLE(MultiSvrNodeClient MultiSvrNodeClient.cpp) +ADD_EXECUTABLE(tree_example tree_example.c) -IF(MSVC) - TARGET_LINK_LIBRARIES(SessionExample iotdb_session "${CMAKE_SOURCE_DIR}/thrift/lib/Release/thriftmd.lib") - TARGET_LINK_LIBRARIES(AlignedTimeseriesSessionExample iotdb_session "${CMAKE_SOURCE_DIR}/thrift/lib/Release/thriftmd.lib") -ELSE() - TARGET_LINK_LIBRARIES(SessionExample iotdb_session pthread) - TARGET_LINK_LIBRARIES(AlignedTimeseriesSessionExample iotdb_session pthread) -ENDIF() +set(_example_targets + SessionExample + AlignedTimeseriesSessionExample + MultiSvrNodeClient + tree_example) + +foreach(_t IN LISTS _example_targets) + IF(WITH_SSL) + TARGET_LINK_LIBRARIES(${_t} PRIVATE "${_iotdb_link_lib}" OpenSSL::SSL OpenSSL::Crypto) + ELSE() + TARGET_LINK_LIBRARIES(${_t} PRIVATE "${_iotdb_link_lib}") + ENDIF() + IF(UNIX) + TARGET_LINK_LIBRARIES(${_t} PRIVATE pthread) + ENDIF() + + # Run from the build output directory without setting LD_LIBRARY_PATH / PATH. + if(UNIX) + set_target_properties(${_t} PROPERTIES + BUILD_RPATH "\$ORIGIN" + INSTALL_RPATH "\$ORIGIN") + endif() + + if(EXISTS "${_iotdb_runtime}") + add_custom_command(TARGET ${_t} POST_BUILD + COMMAND ${CMAKE_COMMAND} -E copy_if_different + "${_iotdb_runtime}" $ + COMMENT "Copy IoTDB runtime library next to ${_t}") + elseif(WIN32) + message(WARNING "Missing ${_iotdb_runtime}; copy iotdb_session.dll manually before running ${_t}.") + endif() +endforeach() + +# Optional: stage a self-contained folder for copying to another machine (see README). +set(_example_dist_dir "${CMAKE_BINARY_DIR}/dist") +add_custom_target(example-dist DEPENDS ${_example_targets} + COMMENT "Collect example binaries and IoTDB runtime into ${_example_dist_dir}") +foreach(_t IN LISTS _example_targets) + add_custom_command(TARGET example-dist POST_BUILD + COMMAND ${CMAKE_COMMAND} -E make_directory "${_example_dist_dir}" + COMMAND ${CMAKE_COMMAND} -E copy_if_different + $ "${_example_dist_dir}/" + COMMENT "Stage ${_t}") +endforeach() +if(EXISTS "${_iotdb_runtime}") + add_custom_command(TARGET example-dist POST_BUILD + COMMAND ${CMAKE_COMMAND} -E copy_if_different + "${_iotdb_runtime}" "${_example_dist_dir}/") +endif() diff --git a/example/client-cpp-example/src/MultiSvrNodeClient.cpp b/example/client-cpp-example/src/MultiSvrNodeClient.cpp new file mode 100644 index 0000000000000..c4b1ca46a8366 --- /dev/null +++ b/example/client-cpp-example/src/MultiSvrNodeClient.cpp @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include +#include +#include +#include +#include +#include + +#include "Session.h" +#include "SessionBuilder.h" +#include "SessionDataSet.h" + +namespace { + +void RunTreeExample() { + try { + std::vector node_urls = {"127.0.0.1:6667", "127.0.0.1:6668", + "127.0.0.1:6669"}; + + auto builder = std::make_shared(); + auto session = std::shared_ptr(builder->username("root") + ->password("root") + ->nodeUrls(node_urls) + ->build()); + + session->open(); + if (!session->checkTimeseriesExists("root.test.d1.s1")) { + session->createTimeseries("root.test.d1.s1", TSDataType::INT64, + TSEncoding::RLE, CompressionType::SNAPPY); + } + session->deleteTimeseries("root.test.d1.s1"); + session->close(); + } catch (const std::exception &e) { + std::cout << "Caught exception: " << e.what() << std::endl; + } +} + +// Example: continuously write/query data so you can manually stop a node +// to test client failover behavior. +void RunResilienceExample() { + try { + std::vector node_urls = {"127.0.0.1:6667", "127.0.0.1:6668", + "127.0.0.1:6669"}; + + auto builder = std::make_shared(); + auto session = std::shared_ptr(builder->username("root") + ->password("root") + ->nodeUrls(node_urls) + ->build()); + + session->open(); + + if (!session->checkTimeseriesExists("root.resilience.d1.s1")) { + session->createTimeseries("root.resilience.d1.s1", TSDataType::INT64, + TSEncoding::RLE, CompressionType::SNAPPY); + } + + std::cout << "Starting resilience test. " + "Stop one node manually to see failover handling..." + << std::endl; + + for (int i = 0; i < 60; ++i) { // run ~60 seconds + int64_t timestamp = std::chrono::system_clock::now().time_since_epoch() / + std::chrono::milliseconds(1); + std::string value = std::to_string(i); + + try { + session->insertRecord("root.resilience.d1", timestamp, {"s1"}, {value}); + std::cout << "[Insert] ts=" << timestamp << ", value=" << value + << std::endl; + + auto dataset = session->executeQueryStatement( + "SELECT s1 FROM root.resilience.d1 LIMIT 1"); + std::cout << "[Query] Got dataset: " << (dataset ? "Success" : "Null") + << std::endl; + + } catch (const std::exception &e) { + std::cout << "Caught exception during resilience loop: " << e.what() + << std::endl; + } + + std::this_thread::sleep_for(std::chrono::seconds(1)); + } + + session->close(); + } catch (const std::exception &e) { + std::cout << "Caught exception in RunResilienceExample: " << e.what() + << std::endl; + } +} + +} // namespace + +int main() { + // RunTreeExample(); + RunResilienceExample(); + return 0; +} diff --git a/example/client-cpp-example/src/SessionExample.cpp b/example/client-cpp-example/src/SessionExample.cpp index c3f5602cb08a8..bcc243c44f22f 100644 --- a/example/client-cpp-example/src/SessionExample.cpp +++ b/example/client-cpp-example/src/SessionExample.cpp @@ -24,438 +24,449 @@ using namespace std; Session *session; void createTimeseries() { - if (!session->checkTimeseriesExists("root.sg1.d1.s1")) { - session->createTimeseries("root.sg1.d1.s1", TSDataType::BOOLEAN, TSEncoding::RLE, - CompressionType::SNAPPY); - } - if (!session->checkTimeseriesExists("root.sg1.d1.s2")) { - session->createTimeseries("root.sg1.d1.s2", TSDataType::INT32, TSEncoding::RLE, - CompressionType::SNAPPY); - } - if (!session->checkTimeseriesExists("root.sg1.d1.s3")) { - session->createTimeseries("root.sg1.d1.s3", TSDataType::FLOAT, TSEncoding::RLE, - CompressionType::SNAPPY); - } - - // create timeseries with tags and attributes - if (!session->checkTimeseriesExists("root.sg1.d1.s4")) { - map tags; - tags["tag1"] = "v1"; - map attributes; - attributes["description"] = "v1"; - session->createTimeseries("root.sg1.d1.s4", TSDataType::INT64, TSEncoding::RLE, - CompressionType::SNAPPY, nullptr, &tags, &attributes, "temperature"); - } + if (!session->checkTimeseriesExists("root.sg1.d1.s1")) { + session->createTimeseries("root.sg1.d1.s1", TSDataType::BOOLEAN, + TSEncoding::RLE, CompressionType::SNAPPY); + } + if (!session->checkTimeseriesExists("root.sg1.d1.s2")) { + session->createTimeseries("root.sg1.d1.s2", TSDataType::INT32, + TSEncoding::RLE, CompressionType::SNAPPY); + } + if (!session->checkTimeseriesExists("root.sg1.d1.s3")) { + session->createTimeseries("root.sg1.d1.s3", TSDataType::FLOAT, + TSEncoding::RLE, CompressionType::SNAPPY); + } + + // create timeseries with tags and attributes + if (!session->checkTimeseriesExists("root.sg1.d1.s4")) { + map tags; + tags["tag1"] = "v1"; + map attributes; + attributes["description"] = "v1"; + session->createTimeseries("root.sg1.d1.s4", TSDataType::INT64, + TSEncoding::RLE, CompressionType::SNAPPY, nullptr, + &tags, &attributes, "temperature"); + } } void createMultiTimeseries() { - if (!session->checkTimeseriesExists("root.sg1.d2.s1") && !session->checkTimeseriesExists("root.sg1.d2.s2")) { - vector paths; - paths.emplace_back("root.sg1.d2.s1"); - paths.emplace_back("root.sg1.d2.s2"); - vector tsDataTypes; - tsDataTypes.push_back(TSDataType::INT64); - tsDataTypes.push_back(TSDataType::DOUBLE); - vector tsEncodings; - tsEncodings.push_back(TSEncoding::RLE); - tsEncodings.push_back(TSEncoding::RLE); - vector compressionTypes; - compressionTypes.push_back(CompressionType::SNAPPY); - compressionTypes.push_back(CompressionType::SNAPPY); - - vector> tagsList; - map tags; - tags["unit"] = "kg"; - tagsList.push_back(tags); - tagsList.push_back(tags); - - vector> attributesList; - map attributes; - attributes["minValue"] = "1"; - attributes["maxValue"] = "100"; - attributesList.push_back(attributes); - attributesList.push_back(attributes); - - vector alias; - alias.emplace_back("weight1"); - alias.emplace_back("weight2"); - - session->createMultiTimeseries(paths, tsDataTypes, tsEncodings, compressionTypes, nullptr, &tagsList, - &attributesList, &alias); - } + if (!session->checkTimeseriesExists("root.sg1.d2.s1") && + !session->checkTimeseriesExists("root.sg1.d2.s2")) { + vector paths; + paths.emplace_back("root.sg1.d2.s1"); + paths.emplace_back("root.sg1.d2.s2"); + vector tsDataTypes; + tsDataTypes.push_back(TSDataType::INT64); + tsDataTypes.push_back(TSDataType::DOUBLE); + vector tsEncodings; + tsEncodings.push_back(TSEncoding::RLE); + tsEncodings.push_back(TSEncoding::RLE); + vector compressionTypes; + compressionTypes.push_back(CompressionType::SNAPPY); + compressionTypes.push_back(CompressionType::SNAPPY); + + vector> tagsList; + map tags; + tags["unit"] = "kg"; + tagsList.push_back(tags); + tagsList.push_back(tags); + + vector> attributesList; + map attributes; + attributes["minValue"] = "1"; + attributes["maxValue"] = "100"; + attributesList.push_back(attributes); + attributesList.push_back(attributes); + + vector alias; + alias.emplace_back("weight1"); + alias.emplace_back("weight2"); + + session->createMultiTimeseries(paths, tsDataTypes, tsEncodings, + compressionTypes, nullptr, &tagsList, + &attributesList, &alias); + } } void createSchemaTemplate() { - if (!session->checkTemplateExists("template1")) { - Template temp("template1", false); - - InternalNode iNodeD99("d99", false); - - MeasurementNode mNodeS1("s1", TSDataType::INT32, TSEncoding::RLE, CompressionType::SNAPPY); - MeasurementNode mNodeS2("s2", TSDataType::INT64, TSEncoding::RLE, CompressionType::SNAPPY); - MeasurementNode mNodeD99S1("s1", TSDataType::DOUBLE, TSEncoding::RLE, CompressionType::SNAPPY); - MeasurementNode mNodeD99S2("s2", TSDataType::BOOLEAN, TSEncoding::RLE, CompressionType::SNAPPY); - - iNodeD99.addChild(mNodeD99S1); - iNodeD99.addChild(mNodeD99S2); - - temp.addToTemplate(iNodeD99); - temp.addToTemplate(mNodeS1); - temp.addToTemplate(mNodeS2); - - session->createSchemaTemplate(temp); - session->setSchemaTemplate("template1", "root.sg3.d1"); - } + if (!session->checkTemplateExists("template1")) { + Template temp("template1", false); + + InternalNode iNodeD99("d99", false); + + MeasurementNode mNodeS1("s1", TSDataType::INT32, TSEncoding::RLE, + CompressionType::SNAPPY); + MeasurementNode mNodeS2("s2", TSDataType::INT64, TSEncoding::RLE, + CompressionType::SNAPPY); + MeasurementNode mNodeD99S1("s1", TSDataType::DOUBLE, TSEncoding::RLE, + CompressionType::SNAPPY); + MeasurementNode mNodeD99S2("s2", TSDataType::BOOLEAN, TSEncoding::RLE, + CompressionType::SNAPPY); + + iNodeD99.addChild(mNodeD99S1); + iNodeD99.addChild(mNodeD99S2); + + temp.addToTemplate(iNodeD99); + temp.addToTemplate(mNodeS1); + temp.addToTemplate(mNodeS2); + + session->createSchemaTemplate(temp); + session->setSchemaTemplate("template1", "root.sg3.d1"); + } } void ActivateTemplate() { - session->executeNonQueryStatement("insert into root.sg3.d1(timestamp,s1, s2) values(200, 1, 1);"); + session->executeNonQueryStatement( + "insert into root.sg3.d1(timestamp,s1, s2) values(200, 1, 1);"); } void showTimeseries() { - unique_ptr dataSet = session->executeQueryStatement("show timeseries"); - for (const string &name: dataSet->getColumnNames()) { - cout << name << " "; - } - cout << endl; - - dataSet->setFetchSize(1024); - while (dataSet->hasNext()) { - cout << dataSet->next()->toString(); - } - cout << endl; - - dataSet->closeOperationHandle(); + unique_ptr dataSet = + session->executeQueryStatement("show timeseries"); + for (const string &name : dataSet->getColumnNames()) { + cout << name << " "; + } + cout << endl; + + dataSet->setFetchSize(1024); + while (dataSet->hasNext()) { + cout << dataSet->next()->toString(); + } + cout << endl; + + dataSet->closeOperationHandle(); } void insertRecord() { - string deviceId = "root.sg2.d1"; - vector measurements; - measurements.emplace_back("s1"); - measurements.emplace_back("s2"); - measurements.emplace_back("s3"); - for (int64_t time = 0; time < 10; time++) { - vector values; - values.emplace_back("1"); - values.emplace_back("2"); - values.emplace_back("3"); - session->insertRecord(deviceId, time, measurements, values); - } + string deviceId = "root.sg2.d1"; + vector measurements; + measurements.emplace_back("s1"); + measurements.emplace_back("s2"); + measurements.emplace_back("s3"); + for (int64_t time = 0; time < 10; time++) { + vector values; + values.emplace_back("1"); + values.emplace_back("2"); + values.emplace_back("3"); + session->insertRecord(deviceId, time, measurements, values); + } } void insertTablet() { - pair pairA("s1", TSDataType::BOOLEAN); - pair pairB("s2", TSDataType::INT32); - pair pairC("s3", TSDataType::FLOAT); - vector> schemas; - schemas.push_back(pairA); - schemas.push_back(pairB); - schemas.push_back(pairC); + pair pairA("s1", TSDataType::BOOLEAN); + pair pairB("s2", TSDataType::INT32); + pair pairC("s3", TSDataType::FLOAT); + vector> schemas; + schemas.push_back(pairA); + schemas.push_back(pairB); + schemas.push_back(pairC); - Tablet tablet("root.sg1.d1", schemas, 100); + Tablet tablet("root.sg1.d1", schemas, 100); - for (int64_t time = 0; time < 30; time++) { - size_t row = tablet.rowSize++; - tablet.timestamps[row] = time; + for (int64_t time = 0; time < 30; time++) { + size_t row = tablet.rowSize++; + tablet.timestamps[row] = time; - bool randVal1 = rand() % 2; - tablet.addValue(0, row, &randVal1); + bool randVal1 = rand() % 2; + tablet.addValue(0, row, randVal1); - int randVal2 = rand(); - tablet.addValue(1, row, &randVal2); + int randVal2 = rand(); + tablet.addValue(1, row, randVal2); - float randVal3 = (float)(rand() / 99.9); - tablet.addValue(2, row, &randVal3); + float randVal3 = (float)(rand() / 99.9); + tablet.addValue(2, row, randVal3); - if (tablet.rowSize == tablet.maxRowNumber) { - session->insertTablet(tablet, true); - tablet.reset(); - } + if (tablet.rowSize == tablet.maxRowNumber) { + session->insertTablet(tablet, true); + tablet.reset(); } + } - if (tablet.rowSize != 0) { - session->insertTablet(tablet); - tablet.reset(); - } + if (tablet.rowSize != 0) { + session->insertTablet(tablet); + tablet.reset(); + } } void insertRecords() { - string deviceId = "root.sg2.d1"; - vector measurements; - measurements.emplace_back("s1"); - measurements.emplace_back("s2"); - measurements.emplace_back("s3"); - - vector deviceIds; - vector> measurementsList; - vector> valuesList; - vector timestamps; - - for (int64_t time = 10; time < 30; time++) { - vector values; - values.emplace_back("1"); - values.emplace_back("2"); - values.emplace_back("3"); - - deviceIds.push_back(deviceId); - measurementsList.push_back(measurements); - valuesList.push_back(values); - timestamps.push_back(time); - if (time != 20 && time % 10 == 0) { - session->insertRecords(deviceIds, timestamps, measurementsList, valuesList); - deviceIds.clear(); - measurementsList.clear(); - valuesList.clear(); - timestamps.clear(); - } + string deviceId = "root.sg2.d1"; + vector measurements; + measurements.emplace_back("s1"); + measurements.emplace_back("s2"); + measurements.emplace_back("s3"); + + vector deviceIds; + vector> measurementsList; + vector> valuesList; + vector timestamps; + + for (int64_t time = 10; time < 30; time++) { + vector values; + values.emplace_back("1"); + values.emplace_back("2"); + values.emplace_back("3"); + + deviceIds.push_back(deviceId); + measurementsList.push_back(measurements); + valuesList.push_back(values); + timestamps.push_back(time); + if (time != 20 && time % 10 == 0) { + session->insertRecords(deviceIds, timestamps, measurementsList, + valuesList); + deviceIds.clear(); + measurementsList.clear(); + valuesList.clear(); + timestamps.clear(); } + } - session->insertRecords(deviceIds, timestamps, measurementsList, valuesList); + session->insertRecords(deviceIds, timestamps, measurementsList, valuesList); } void insertTablets() { - pair pairA("s1", TSDataType::INT64); - pair pairB("s2", TSDataType::DOUBLE); - pair pairC("s3", TSDataType::TEXT); - vector> schemas; - schemas.push_back(pairA); - schemas.push_back(pairB); - schemas.push_back(pairC); + pair pairA("s1", TSDataType::INT64); + pair pairB("s2", TSDataType::DOUBLE); + pair pairC("s3", TSDataType::TEXT); + vector> schemas; + schemas.push_back(pairA); + schemas.push_back(pairB); + schemas.push_back(pairC); - Tablet tablet1("root.sg1.d2", schemas, 100); - Tablet tablet2("root.sg1.d3", schemas, 100); + Tablet tablet1("root.sg1.d2", schemas, 100); + Tablet tablet2("root.sg1.d3", schemas, 100); - unordered_map tabletMap; - tabletMap["root.sg1.d2"] = &tablet1; - tabletMap["root.sg1.d3"] = &tablet2; + unordered_map tabletMap; + tabletMap["root.sg1.d2"] = &tablet1; + tabletMap["root.sg1.d3"] = &tablet2; - for (int64_t time = 0; time < 30; time++) { - size_t row1 = tablet1.rowSize++; - size_t row2 = tablet2.rowSize++; - tablet1.timestamps[row1] = time; - tablet2.timestamps[row2] = time; + for (int64_t time = 0; time < 30; time++) { + size_t row1 = tablet1.rowSize++; + size_t row2 = tablet2.rowSize++; + tablet1.timestamps[row1] = time; + tablet2.timestamps[row2] = time; - int64_t randVal11 = rand(); - tablet1.addValue(0, row1, &randVal11); + int64_t randVal11 = rand(); + tablet1.addValue(0, row1, randVal11); - double randVal12 = rand() / 99.9; - tablet1.addValue(1, row1, &randVal12); + double randVal12 = rand() / 99.9; + tablet1.addValue(1, row1, randVal12); - string randVal13 = "string" + to_string(rand()); - tablet1.addValue(2, row1, &randVal13); + string randVal13 = "string" + to_string(rand()); + tablet1.addValue(2, row1, randVal13); - int64_t randVal21 = rand(); - tablet2.addValue(0, row2, &randVal21); + int64_t randVal21 = rand(); + tablet2.addValue(0, row2, randVal21); - double randVal22 = rand() / 99.9; - tablet2.addValue(1, row2, &randVal22); + double randVal22 = rand() / 99.9; + tablet2.addValue(1, row2, randVal22); - string randVal23 = "string" + to_string(rand()); - tablet2.addValue(2, row2, &randVal23); + string randVal23 = "string" + to_string(rand()); + tablet2.addValue(2, row2, randVal23); - if (tablet1.rowSize == tablet1.maxRowNumber) { - session->insertTablets(tabletMap, true); + if (tablet1.rowSize == tablet1.maxRowNumber) { + session->insertTablets(tabletMap, true); - tablet1.reset(); - tablet2.reset(); - } + tablet1.reset(); + tablet2.reset(); } + } - if (tablet1.rowSize != 0) { - session->insertTablets(tabletMap, true); - tablet1.reset(); - tablet2.reset(); - } + if (tablet1.rowSize != 0) { + session->insertTablets(tabletMap, true); + tablet1.reset(); + tablet2.reset(); + } } void insertTabletWithNullValues() { - /* - * A Tablet example: - * device1 - * time s1, s2, s3 - * 1, null, 1, 1 - * 2, 2, null, 2 - * 3, 3, 3, null - */ - pair pairA("s1", TSDataType::INT64); - pair pairB("s2", TSDataType::INT64); - pair pairC("s3", TSDataType::INT64); - vector> schemas; - schemas.push_back(pairA); - schemas.push_back(pairB); - schemas.push_back(pairC); - - Tablet tablet("root.sg1.d4", schemas, 30); - - for (int64_t time = 0; time < 30; time++) { - size_t row = tablet.rowSize++; - tablet.timestamps[row] = time; - for (int i = 0; i < 3; i++) { - int64_t randVal = rand(); - tablet.addValue(i, row, &randVal); - // mark null value - if (row % 3 == (unsigned int) i) { - tablet.bitMaps[i].mark(row); - } - } - if (tablet.rowSize == tablet.maxRowNumber) { - session->insertTablet(tablet, true); - tablet.reset(); - } + /* + * A Tablet example: + * device1 + * time s1, s2, s3 + * 1, null, 1, 1 + * 2, 2, null, 2 + * 3, 3, 3, null + */ + pair pairA("s1", TSDataType::INT64); + pair pairB("s2", TSDataType::INT64); + pair pairC("s3", TSDataType::INT64); + vector> schemas; + schemas.push_back(pairA); + schemas.push_back(pairB); + schemas.push_back(pairC); + + Tablet tablet("root.sg1.d4", schemas, 30); + + for (int64_t time = 0; time < 30; time++) { + size_t row = tablet.rowSize++; + tablet.timestamps[row] = time; + for (int i = 0; i < 3; i++) { + int64_t randVal = rand(); + tablet.addValue(i, row, randVal); + // mark null value + if (row % 3 == (unsigned int)i) { + tablet.bitMaps[i].mark(row); + } } - - if (tablet.rowSize != 0) { - session->insertTablet(tablet); - tablet.reset(); + if (tablet.rowSize == tablet.maxRowNumber) { + session->insertTablet(tablet, true); + tablet.reset(); } + } + + if (tablet.rowSize != 0) { + session->insertTablet(tablet); + tablet.reset(); + } } void nonQuery() { - session->executeNonQueryStatement("insert into root.sg1.d1(timestamp,s1) values(100, 1);"); + session->executeNonQueryStatement( + "insert into root.sg1.d1(timestamp,s1) values(100, 1);"); } void query() { - unique_ptr dataSet = session->executeQueryStatement("select s1, s2, s3 from root.**"); - for (const string &name: dataSet->getColumnNames()) { - cout << name << " "; - } - cout << endl; - - dataSet->setFetchSize(1024); - while (dataSet->hasNext()) { - cout << dataSet->next()->toString(); - } - cout << endl; - - dataSet->closeOperationHandle(); + unique_ptr dataSet = + session->executeQueryStatement("select s1, s2, s3 from root.**"); + for (const string &name : dataSet->getColumnNames()) { + cout << name << " "; + } + cout << endl; + + dataSet->setFetchSize(1024); + while (dataSet->hasNext()) { + cout << dataSet->next()->toString(); + } + cout << endl; + + dataSet->closeOperationHandle(); } void deleteData() { - string path = "root.sg1.d1.s1"; - int64_t deleteTime = 99; - session->deleteData(path, deleteTime); + string path = "root.sg1.d1.s1"; + int64_t deleteTime = 99; + session->deleteData(path, deleteTime); } void deleteTimeseries() { - vector paths; - vector timeseriesGrp = { "root.sg1.d1.s1", "root.sg1.d1.s2", "root.sg1.d1.s3", - "root.sg1.d2.s1", "root.sg1.d2.s2", "root.sg1.d2.s3", - "root.sg1.d3.s1", "root.sg1.d3.s2", "root.sg1.d3.s3", - "root.sg1.d4.s1", "root.sg1.d4.s2", "root.sg1.d4.s3", - "root.sg2.d1.s1", "root.sg2.d1.s2", "root.sg2.d1.s3" }; - for (const string& timeseries : timeseriesGrp) { - if (session->checkTimeseriesExists(timeseries)) { - paths.push_back(timeseries); - } + vector paths; + vector timeseriesGrp = { + "root.sg1.d1.s1", "root.sg1.d1.s2", "root.sg1.d1.s3", "root.sg1.d2.s1", + "root.sg1.d2.s2", "root.sg1.d2.s3", "root.sg1.d3.s1", "root.sg1.d3.s2", + "root.sg1.d3.s3", "root.sg1.d4.s1", "root.sg1.d4.s2", "root.sg1.d4.s3", + "root.sg2.d1.s1", "root.sg2.d1.s2", "root.sg2.d1.s3"}; + for (const string ×eries : timeseriesGrp) { + if (session->checkTimeseriesExists(timeseries)) { + paths.push_back(timeseries); } - session->deleteTimeseries(paths); + } + session->deleteTimeseries(paths); } void deleteStorageGroups() { - vector storageGroups; - storageGroups.emplace_back("root.sg1"); - storageGroups.emplace_back("root.sg2"); - session->deleteStorageGroups(storageGroups); + vector storageGroups; + storageGroups.emplace_back("root.sg1"); + storageGroups.emplace_back("root.sg2"); + session->deleteStorageGroups(storageGroups); } void queryLast() { - unique_ptr dataSet = session->executeQueryStatement("select last s1,s2,s3 from root.**"); - for (const string &name: dataSet->getColumnNames()) { - cout << name << " "; - } - cout << endl; - - while (dataSet->hasNext()) { - cout << dataSet->next()->toString(); - } - cout << endl; - - dataSet->closeOperationHandle(); + unique_ptr dataSet = + session->executeQueryStatement("select last s1,s2,s3 from root.**"); + for (const string &name : dataSet->getColumnNames()) { + cout << name << " "; + } + cout << endl; + + while (dataSet->hasNext()) { + cout << dataSet->next()->toString(); + } + cout << endl; + + dataSet->closeOperationHandle(); } int main() { - LOG_LEVEL = LEVEL_DEBUG; - - session = new Session("127.0.0.1", 6667, "root", "root"); - session->open(false); - - cout << "setStorageGroup: root.sg1\n" << endl; - try { - session->setStorageGroup("root.sg1"); - } - catch (IoTDBException &e) { - string errorMessage(e.what()); - if (errorMessage.find("StorageGroupAlreadySetException") == string::npos) { - cout << errorMessage << endl; - } - //throw e; - } - - cout << "setStorageGroup: root.sg2\n" << endl; - try { - session->setStorageGroup("root.sg2"); + LOG_LEVEL = LEVEL_DEBUG; + + session = new Session("127.0.0.1", 6667, "root", "root"); + session->open(false); + + cout << "setStorageGroup: root.sg1\n" << endl; + try { + session->setStorageGroup("root.sg1"); + } catch (IoTDBException &e) { + string errorMessage(e.what()); + if (errorMessage.find("StorageGroupAlreadySetException") == string::npos) { + cout << errorMessage << endl; } - catch (IoTDBException &e) { - string errorMessage(e.what()); - if (errorMessage.find("StorageGroupAlreadySetException") == string::npos) { - cout << errorMessage << endl; - } - //throw e; + // throw e; + } + + cout << "setStorageGroup: root.sg2\n" << endl; + try { + session->setStorageGroup("root.sg2"); + } catch (IoTDBException &e) { + string errorMessage(e.what()); + if (errorMessage.find("StorageGroupAlreadySetException") == string::npos) { + cout << errorMessage << endl; } + // throw e; + } - cout << "createTimeseries\n" << endl; - createTimeseries(); + cout << "createTimeseries\n" << endl; + createTimeseries(); - cout << "createMultiTimeseries\n" << endl; - createMultiTimeseries(); + cout << "createMultiTimeseries\n" << endl; + createMultiTimeseries(); - cout << "createSchemaTemplate\n" << endl; - createSchemaTemplate(); + cout << "createSchemaTemplate\n" << endl; + createSchemaTemplate(); - cout << "ActivateTemplate\n" << endl; - ActivateTemplate(); + cout << "ActivateTemplate\n" << endl; + ActivateTemplate(); - cout << "showTimeseries\n" << endl; - showTimeseries(); + cout << "showTimeseries\n" << endl; + showTimeseries(); - cout << "insertRecord\n" << endl; - insertRecord(); + cout << "insertRecord\n" << endl; + insertRecord(); - cout << "insertTablet\n" << endl; - insertTablet(); + cout << "insertTablet\n" << endl; + insertTablet(); - cout << "insertRecords\n" << endl; - insertRecords(); + cout << "insertRecords\n" << endl; + insertRecords(); - cout << "insertTablets\n" << endl; - insertTablets(); + cout << "insertTablets\n" << endl; + insertTablets(); - cout << "insertTabletWithNullValues\n" << endl; - insertTabletWithNullValues(); + cout << "insertTabletWithNullValues\n" << endl; + insertTabletWithNullValues(); - cout << "nonQuery\n" << endl; - nonQuery(); + cout << "nonQuery\n" << endl; + nonQuery(); - cout << "queryLast\n" << endl; - queryLast(); + cout << "queryLast\n" << endl; + queryLast(); - cout << "query\n" << endl; - query(); + cout << "query\n" << endl; + query(); - cout << "deleteData\n" << endl; - deleteData(); + cout << "deleteData\n" << endl; + deleteData(); - cout << "deleteTimeseries\n" << endl; - deleteTimeseries(); + cout << "deleteTimeseries\n" << endl; + deleteTimeseries(); - cout << "deleteStorageGroups\n" << endl; - deleteStorageGroups(); + cout << "deleteStorageGroups\n" << endl; + deleteStorageGroups(); - cout << "session close\n" << endl; - session->close(); + cout << "session close\n" << endl; + session->close(); - delete session; + delete session; - cout << "finished!\n" << endl; - return 0; + cout << "finished!\n" << endl; + return 0; } diff --git a/example/client-cpp-example/src/tree_example.c b/example/client-cpp-example/src/tree_example.c new file mode 100644 index 0000000000000..1d1045a8c4eee --- /dev/null +++ b/example/client-cpp-example/src/tree_example.c @@ -0,0 +1,118 @@ +/** + * 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. + */ + +/* + * Tree model: create one timeseries, insert one row via string values, SELECT, + * cleanup. Edit HOST / PORT / credentials below to match your IoTDB. + */ + +#include +#include +#include +#include + +#include "SessionC.h" + +#define HOST "127.0.0.1" +#define PORT 6667 +#define USER "root" +#define PASS "root" + +#define TS_PATH "root.cdemo.d0.s0" +#define DEVICE "root.cdemo.d0" + +static void fail(const char *ctx, CSession *s) { + fprintf(stderr, "[tree_example] %s failed: %s\n", ctx, ts_get_last_error()); + if (s) { + ts_session_close(s); + ts_session_destroy(s); + } + exit(1); +} + +int main(void) { + const char *path = TS_PATH; + CSession *session = ts_session_new(HOST, PORT, USER, PASS); + if (!session) { + fprintf(stderr, "[tree_example] ts_session_new returned NULL: %s\n", + ts_get_last_error()); + return 1; + } + if (ts_session_open(session) != TS_OK) { + fail("ts_session_open", session); + } + + bool exists = false; + if (ts_session_check_timeseries_exists(session, path, &exists) != TS_OK) { + fail("ts_session_check_timeseries_exists", session); + } + if (exists) { + if (ts_session_delete_timeseries(session, path) != TS_OK) { + fail("ts_session_delete_timeseries (cleanup old)", session); + } + } + if (ts_session_create_timeseries(session, path, TS_TYPE_INT64, + TS_ENCODING_RLE, + TS_COMPRESSION_SNAPPY) != TS_OK) { + fail("ts_session_create_timeseries", session); + } + + const char *measurements[] = {"s0"}; + const char *values[] = {"100"}; + if (ts_session_insert_record_str(session, DEVICE, 1LL, 1, measurements, + values) != TS_OK) { + fail("ts_session_insert_record_str", session); + } + + CSessionDataSet *dataSet = NULL; + if (ts_session_execute_query(session, "select s0 from root.cdemo.d0", + &dataSet) != TS_OK) { + fail("ts_session_execute_query", session); + } + if (!dataSet) { + fprintf(stderr, "[tree_example] dataSet is NULL\n"); + ts_session_close(session); + ts_session_destroy(session); + return 1; + } + ts_dataset_set_fetch_size(dataSet, 1024); + + int rows = 0; + while (ts_dataset_has_next(dataSet)) { + CRowRecord *record = ts_dataset_next(dataSet); + if (!record) { + break; + } + int64_t v = ts_row_record_get_int64(record, 0); + printf("[tree_example] row %d: s0 = %lld\n", rows, (long long)v); + ts_row_record_destroy(record); + rows++; + } + ts_dataset_destroy(dataSet); + + printf("[tree_example] done, read %d row(s).\n", rows); + + if (ts_session_delete_timeseries(session, path) != TS_OK) { + fail("ts_session_delete_timeseries", session); + } + + ts_session_close(session); + ts_session_destroy(session); + return 0; +} diff --git a/iotdb-client/client-cpp/CMakeLists.txt b/iotdb-client/client-cpp/CMakeLists.txt new file mode 100644 index 0000000000000..8ee20d3d241b2 --- /dev/null +++ b/iotdb-client/client-cpp/CMakeLists.txt @@ -0,0 +1,271 @@ +# 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. +# +# ============================================================================= +# Apache IoTDB - C++ Session Client (top-level CMake build) +# ============================================================================= + +cmake_minimum_required(VERSION 3.15) +project(iotdb_session CXX C) + +set(CMAKE_CXX_STANDARD 11) +set(CMAKE_CXX_STANDARD_REQUIRED ON) +set(CMAKE_CXX_EXTENSIONS OFF) +set(CMAKE_POSITION_INDEPENDENT_CODE ON) +set(CMAKE_POLICY_DEFAULT_CMP0091 NEW) + +if(NOT CMAKE_BUILD_TYPE AND NOT CMAKE_CONFIGURATION_TYPES) + set(CMAKE_BUILD_TYPE Release CACHE STRING "Build type" FORCE) +endif() + +set(IOTDB_EXTRA_CXX_FLAGS "" + CACHE STRING "Extra flags appended to CMAKE_CXX_FLAGS") +option(IOTDB_LIBSTDCXX_CXX11_ABI + "Use libstdc++ dual-ABI new layout (_GLIBCXX_USE_CXX11_ABI=1); release glibc224 zips" + OFF) + +if(NOT MSVC) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wall -g -O2") + if(IOTDB_LIBSTDCXX_CXX11_ABI) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -D_GLIBCXX_USE_CXX11_ABI=1") + endif() + if(IOTDB_EXTRA_CXX_FLAGS) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${IOTDB_EXTRA_CXX_FLAGS}") + endif() + # CI/release scripts read this tag; nm-based checks are unreliable on some toolchains. + if(IOTDB_LIBSTDCXX_CXX11_ABI) + file(WRITE "${CMAKE_BINARY_DIR}/iotdb_libstdcxx_abi_tag" "cxx11\n") + else() + file(WRITE "${CMAKE_BINARY_DIR}/iotdb_libstdcxx_abi_tag" "legacy\n") + endif() +endif() + +if(MSVC) + # /MD: matches default Visual Studio projects; CRT lives in the VC redistributable. + set(CMAKE_MSVC_RUNTIME_LIBRARY "MultiThreaded$<$:Debug>DLL") +endif() + +option(WITH_SSL "Build with OpenSSL support" OFF) +option(BUILD_TESTING "Build IT test executables" OFF) +option(IOTDB_OFFLINE "Disable all network access during configure" OFF) +set(IOTDB_SESSION_VERSION "0.0.0" + CACHE STRING "IoTDB C++ session package version") +string(REGEX MATCH "^[0-9]+" IOTDB_SESSION_SOVERSION "${IOTDB_SESSION_VERSION}") +if(NOT IOTDB_SESSION_SOVERSION) + set(IOTDB_SESSION_SOVERSION "0") +endif() + +set(IOTDB_DEPS_DIR "${CMAKE_CURRENT_SOURCE_DIR}/third-party" + CACHE PATH "Local tarball cache for third-party dependencies (lives under client-cpp/)") +if(WIN32) + set(_iotdb_default_boost_version "1.74.0") +elseif(APPLE) + set(_iotdb_default_boost_version "1.84.0") +else() + set(_iotdb_default_boost_version "1.60.0") +endif() +set(BOOST_VERSION "${_iotdb_default_boost_version}" + CACHE STRING "Boost version used when downloading / unpacking (Thrift build only)") +set(THRIFT_VERSION "0.21.0" + CACHE STRING "Apache Thrift version used when downloading / building") + +if(WIN32) + set(IOTDB_OS_DEPS_DIR "${IOTDB_DEPS_DIR}/windows") +elseif(APPLE) + set(IOTDB_OS_DEPS_DIR "${IOTDB_DEPS_DIR}/mac") +else() + set(IOTDB_OS_DEPS_DIR "${IOTDB_DEPS_DIR}/linux") +endif() +file(MAKE_DIRECTORY "${IOTDB_OS_DEPS_DIR}") + +if(CMAKE_INSTALL_PREFIX_INITIALIZED_TO_DEFAULT) + set(CMAKE_INSTALL_PREFIX "${CMAKE_BINARY_DIR}/install" + CACHE PATH "Install prefix" FORCE) +endif() + +list(APPEND CMAKE_MODULE_PATH "${CMAKE_CURRENT_SOURCE_DIR}/cmake") + +include(FetchBoost) # -> BOOST_INCLUDE_DIR (Thrift build only) +include(FetchBuildTools) +if(WITH_SSL) + include(FetchOpenSSL) +endif() +include(FetchThrift) +include(GenerateThriftSources) + +file(GLOB SESSION_PUBLIC_SRCS CONFIGURE_DEPENDS + "${CMAKE_CURRENT_SOURCE_DIR}/src/session/*.cpp") +file(GLOB SESSION_RPC_SRCS CONFIGURE_DEPENDS + "${CMAKE_CURRENT_SOURCE_DIR}/src/rpc/*.cpp") + +add_library(iotdb_session SHARED + ${SESSION_PUBLIC_SRCS} + ${SESSION_RPC_SRCS} + ${THRIFT_GENERATED_SRCS}) + +if(WIN32) + set_target_properties(iotdb_session PROPERTIES WINDOWS_EXPORT_ALL_SYMBOLS ON) +endif() + +if(UNIX AND NOT APPLE) + set_target_properties(iotdb_session PROPERTIES + VERSION "${IOTDB_SESSION_VERSION}" + SOVERSION "${IOTDB_SESSION_SOVERSION}") +endif() + +add_dependencies(iotdb_session iotdb_thrift_external iotdb_thrift_codegen) + +target_compile_definitions(iotdb_session PRIVATE THRIFT_STATIC_DEFINE IOTDB_BUILDING_SHARED) + +target_include_directories(iotdb_session + PUBLIC + $ + $ + PRIVATE + ${CMAKE_CURRENT_SOURCE_DIR}/src/rpc + ${THRIFT_GEN_CPP_DIR} + ${THRIFT_INCLUDE_DIR} + $<$:${BOOST_INCLUDE_DIR}>) + +if(APPLE) + target_link_libraries(iotdb_session PRIVATE "-Wl,-force_load,${THRIFT_STATIC_LIB_PATH}") +elseif(UNIX AND NOT MSVC) + # whole-archive pulls all Thrift objects into libiotdb_session.so; allow-multiple-definition + # avoids libgcc __morestack_* duplicate symbol errors on some GCC/toolchain combos. + target_link_libraries(iotdb_session PRIVATE + -Wl,--whole-archive + ${THRIFT_STATIC_LIB_PATH} + -Wl,--no-whole-archive + -Wl,--allow-multiple-definition) +else() + target_link_libraries(iotdb_session PRIVATE iotdb_thrift_static) +endif() + +if(WITH_SSL) + target_link_libraries(iotdb_session PUBLIC OpenSSL::SSL OpenSSL::Crypto) + target_compile_definitions(iotdb_session PUBLIC WITH_SSL=1) +else() + target_compile_definitions(iotdb_session PUBLIC WITH_SSL=0) +endif() + +if(UNIX) + target_link_libraries(iotdb_session PUBLIC pthread) +endif() + +include(GNUInstallDirs) + +set(IOTDB_PUBLIC_HEADERS + Export.h + SessionConfig.h + Session.h + Common.h + Optional.h + Date.h + Status.h + Endpoint.h + SessionBuilder.h + AbstractSessionBuilder.h + SessionPool.h + SessionC.h + SessionDataSet.h + Column.h + ColumnDecoder.h + TsBlock.h) + +# Windows: RUNTIME = iotdb_session.dll, ARCHIVE = import .lib (both under lib/ in the zip). +install(TARGETS iotdb_session + RUNTIME DESTINATION lib + LIBRARY DESTINATION lib + ARCHIVE DESTINATION lib) + +foreach(_hdr IN LISTS IOTDB_PUBLIC_HEADERS) + install(FILES "${CMAKE_CURRENT_SOURCE_DIR}/src/include/${_hdr}" + DESTINATION include) +endforeach() + +set(IOTDB_SESSION_PC_LIBS "-liotdb_session") +set(IOTDB_SESSION_PC_CFLAGS "") +set(IOTDB_SESSION_CXX11_ABI_COMPILE_DEFINITION "") +if(UNIX) + set(IOTDB_SESSION_PC_LIBS "${IOTDB_SESSION_PC_LIBS} -pthread") +endif() +if(IOTDB_LIBSTDCXX_CXX11_ABI AND NOT MSVC) + set(IOTDB_SESSION_PC_CFLAGS "-D_GLIBCXX_USE_CXX11_ABI=1") + set(IOTDB_SESSION_CXX11_ABI_COMPILE_DEFINITION "_GLIBCXX_USE_CXX11_ABI=1") +endif() +configure_file( + "${CMAKE_CURRENT_SOURCE_DIR}/cmake/iotdb-session-config.cmake.in" + "${CMAKE_BINARY_DIR}/package-metadata/cmake/iotdb-session-config.cmake" + @ONLY) +configure_file( + "${CMAKE_CURRENT_SOURCE_DIR}/cmake/iotdb-session.pc.in" + "${CMAKE_BINARY_DIR}/package-metadata/pkgconfig/iotdb-session.pc" + @ONLY) +execute_process( + COMMAND git rev-parse HEAD + WORKING_DIRECTORY "${CMAKE_CURRENT_SOURCE_DIR}" + OUTPUT_VARIABLE IOTDB_SESSION_GIT_COMMIT + OUTPUT_STRIP_TRAILING_WHITESPACE + ERROR_QUIET) +if(NOT IOTDB_SESSION_GIT_COMMIT) + set(IOTDB_SESSION_GIT_COMMIT "unknown") +endif() +string(TIMESTAMP IOTDB_SESSION_BUILD_TIME "%Y-%m-%dT%H:%M:%SZ" UTC) +set(IOTDB_SESSION_CI_BUILD_ID "$ENV{GITHUB_RUN_ID}") +if(NOT IOTDB_SESSION_CI_BUILD_ID) + set(IOTDB_SESSION_CI_BUILD_ID "local") +endif() +file(WRITE "${CMAKE_BINARY_DIR}/package-metadata/VERSION" + "${IOTDB_SESSION_VERSION}\n") +file(WRITE "${CMAKE_BINARY_DIR}/package-metadata/BUILD-INFO.txt" + "Apache IoTDB C++ session client build information\n\n" + "version=${IOTDB_SESSION_VERSION}\n" + "commit=${IOTDB_SESSION_GIT_COMMIT}\n" + "build.time=${IOTDB_SESSION_BUILD_TIME}\n" + "ci.build.id=${IOTDB_SESSION_CI_BUILD_ID}\n" + "compiler=${CMAKE_CXX_COMPILER_ID} ${CMAKE_CXX_COMPILER_VERSION}\n" + "cmake.generator=${CMAKE_GENERATOR}\n" + "cmake.build.type=${CMAKE_BUILD_TYPE}\n" + "with.ssl=${WITH_SSL}\n" + "iotdb.offline=${IOTDB_OFFLINE}\n" + "iotdb.libstdcxx.cxx11.abi=${IOTDB_LIBSTDCXX_CXX11_ABI}\n" + "iotdb.extra.cxx.flags=${IOTDB_EXTRA_CXX_FLAGS}\n") +install(FILES "${CMAKE_BINARY_DIR}/package-metadata/cmake/iotdb-session-config.cmake" + DESTINATION cmake) +install(FILES "${CMAKE_BINARY_DIR}/package-metadata/pkgconfig/iotdb-session.pc" + DESTINATION pkgconfig) +install(FILES + "${CMAKE_BINARY_DIR}/package-metadata/VERSION" + "${CMAKE_BINARY_DIR}/package-metadata/BUILD-INFO.txt" + DESTINATION .) + +if(BUILD_TESTING) + enable_testing() + add_subdirectory(test) +endif() + +message(STATUS "iotdb_session configuration summary:") +message(STATUS " WITH_SSL = ${WITH_SSL}") +message(STATUS " BUILD_TESTING = ${BUILD_TESTING}") +message(STATUS " IOTDB_OFFLINE = ${IOTDB_OFFLINE}") +message(STATUS " IOTDB_DEPS_DIR = ${IOTDB_DEPS_DIR}") +message(STATUS " BOOST_INCLUDE_DIR = ${BOOST_INCLUDE_DIR} (Thrift build only)") +message(STATUS " THRIFT_INCLUDE_DIR = ${THRIFT_INCLUDE_DIR}") +message(STATUS " THRIFT_STATIC_LIB = ${THRIFT_STATIC_LIB_PATH}") +message(STATUS " THRIFT_EXECUTABLE = ${THRIFT_EXECUTABLE}") +message(STATUS " CMAKE_INSTALL_PREFIX = ${CMAKE_INSTALL_PREFIX}") +message(STATUS " IOTDB_LIBSTDCXX_CXX11_ABI = ${IOTDB_LIBSTDCXX_CXX11_ABI}") diff --git a/iotdb-client/client-cpp/README.md b/iotdb-client/client-cpp/README.md index 81983e06e2e4d..1ce08dc560a1a 100644 --- a/iotdb-client/client-cpp/README.md +++ b/iotdb-client/client-cpp/README.md @@ -18,133 +18,351 @@ under the License. --> -# Building C++ Client +# Apache IoTDB C++ Client -## Compile and Test: +The C++ client is built by a single top-level `CMakeLists.txt` in this +directory. The outer Maven POM is a thin wrapper that invokes CMake; you can +also build the client standalone with just `cmake` if you don't have Maven +available. -### Compile +## Build layout at a glance -#### Unix -To compile the cpp client, use the following command: -`mvn clean package -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests` - -#### Windows -To compile on Windows, please install Boost first and add following Maven -settings: ``` --Dboost.include.dir=${your boost header folder} -Dboost.library.dir=${your boost lib (stage) folder}` +iotdb-client/client-cpp/ +├── CMakeLists.txt # single entry point - manages everything +├── cmake/ # helpers (FetchBoost / FetchThrift / ...) +├── third-party/ # local tarball cache (one sub-dir per OS) +│ ├── linux/ mac/ windows/ +├── src/include/ # public API headers (installed to include/) +├── src/session/ # Session, Tablet, and C API implementation (.cpp) +├── src/rpc/ # Thrift RPC layer (private, not installed) +├── test/ # Catch2-based integration tests +└── pom.xml # Maven wrapper (cmake-maven-plugin) ``` -The thrift dependency that the cpp client uses is incompatible with MinGW, please use Visual -Studio. It is highly recommended to use Visual Studio 2022 or later. - -##### Visual Studio 2022 -If you are using Visual Studio 2022, you can compile the cpp client with the following command: - -``` -mvn clean package -P with-cpp -pl iotdb-client/client-cpp -am -DskipTest --D"boost.include.dir"="D:\boost_1_75_0" -D"boost.library.dir"="D:\boost_1_75_0\stage\lib" +During configure CMake will, in order: + +1. Resolve Boost headers (`find_package` → local `third-party//` tarball → + download from `archives.boost.io` when not in offline mode). +2. On Linux/macOS, ensure `m4` / `flex` / `bison` are available; if not, + build them from local tarballs into `build/tools/bin` (no `sudo` + required). +3. Build a static Apache Thrift from source (tarball cache → download fallback). +4. Run the produced `thrift` compiler on + `iotdb-protocol/thrift-{commons,datanode}/src/main/thrift/*.thrift`. +5. Compile `iotdb_session` (the C/C++ session library) and, optionally, + the Catch2 integration test binaries. +6. `cmake --install` lays out the SDK under `target/install/{include,lib}`, + which Maven's assembly step packages into a zip. + +## Build matrix + +| Goal | Command | +|-------------------------------|--------------------------------------------------------------------------------------------------------| +| Library only (Linux/macOS) | `mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests package` | +| Library only (Windows / MSVC) | `mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests "-Dboost.include.dir=C:\boost_1_88_0" package` | +| Library + ITs (Linux/macOS) | `mvn clean install -P with-cpp -pl distribution,iotdb-client/client-cpp -am` then `mvn -P with-cpp -pl iotdb-client/client-cpp -am verify` | +| Direct CMake (no Maven) | `cmake -S iotdb-client/client-cpp -B build && cmake --build build --target install` | + +The Maven build sets `cmake.install.prefix` to `target/install/`. Output zips +land at `iotdb-client/client-cpp/target/iotdb-session-cpp--.zip` +(with a package root directory and a `.sha512` checksum generated alongside), +where `` defaults to the OS name (for example `linux-x86_64`) and +can be overridden with `-Dclient.cpp.package.classifier=...` when building +multiple toolchains on the same platform. + +### Release packages (CI) + +The [C++ Client package](../../.github/workflows/client-cpp-package.yml) workflow +builds one zip per platform/toolchain. Pick the artifact that matches your +deployment environment: + +| Target environment | Zip classifier (suffix) | +|--------------------|-------------------------| +| Linux x86_64, glibc >= 2.28 | `linux-x86_64-glibc2.28` | +| Linux aarch64, glibc >= 2.28 | `linux-aarch64-glibc2.28` | +| macOS x86_64 | `macos-x86_64` | +| macOS arm64 | `macos-aarch64` | +| Windows + Visual Studio 2017 | `windows-x86_64-msvc14.1` | +| Windows + Visual Studio 2019 | `windows-x86_64-msvc14.2` | +| Windows + Visual Studio 2022 | `windows-x86_64-msvc14.3` | +| Windows + Visual Studio 2026 | `windows-x86_64-msvc14.4` | + +Example file name: +`iotdb-session-cpp-1.3.7-SNAPSHOT-linux-x86_64-glibc2.28.zip`. + +**Linux package choice:** The CI Linux packages are built in manylinux_2_28 +containers, so use them on hosts with glibc 2.28 or newer. Build locally with +the same Maven/CMake options if you need a different baseline. + +Thrift **0.21.0** is compiled from source during the CMake configure step (see +`cmake/FetchThrift.cmake`). Older releases that used pre-built +`iotdb-tools-thrift` Maven artifacts and `-Diotdb-tools-thrift.version=...` +for glibc/MSVC compatibility apply only to the **legacy** client-cpp build; +with the current CMake build, compatibility is determined by the **compiler +and OS used to build** the SDK, not by that Maven property. + +### Local build for a specific classifier + +Linux x86_64 (glibc 2.28 baseline): + +```bash +mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests \ + -Dclient.cpp.package.classifier=linux-x86_64-glibc2.28 package ``` -#### Visual Studio 2019 or older -If you are using Visual Studio 2019 or older, the pre-built Thrift library is incompatible. You -will have to compile the thrift library manually: +Windows (match the Visual Studio version you use to build your application): + +```powershell +# Visual Studio 2022 (default on recent Windows) +mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests package -1. Install the dependencies of Thrift: -* flex http://gnuwin32.sourceforge.net/packages/flex.htm -* bison http://gnuwin32.sourceforge.net/packages/bison.htm -* openssl https://slproweb.com/products/Win32OpenSSL.html +# Visual Studio 2019 +mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests ` + "-Dcmake.generator=Visual Studio 16 2019" ` + -Dclient.cpp.package.classifier=windows-x86_64-msvc14.2 package -2. Clone the repository: https://github.com/apache/iotdb-bin-resources. +# Visual Studio 2017 (CMake uses -A x64 on Windows automatically) +mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests ` + "-Dcmake.generator=Visual Studio 15 2017" ` + -Dclient.cpp.package.classifier=windows-x86_64-msvc14.1 package -3. Enter the "iotdb-tools-thrift" folder in the cloned repository; use the following command to - compile the thrift library: +# Debug build +mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests ` + "-Dcmake.build.type=Debug" package +``` -`mvn install` +On Windows, the build passes `-DCMAKE_GENERATOR_PLATFORM=x64` so Visual Studio +generators target **x64** (VS2017 otherwise defaults to Win32). + +## CMake options + +The table below lists CMake cache variables. When building through **Maven**, +pass them as Maven properties (the POM maps them to `-D` options for CMake): + +| CMake variable | Maven property (`-D...`) | +|----------------|--------------------------| +| `WITH_SSL` | `with.ssl` (e.g. `-Dwith.ssl=ON`) | +| `IOTDB_OFFLINE` | `iotdb.offline` | +| `BUILD_TESTING` | `build.tests` | +| `IOTDB_DEPS_DIR` | `iotdb.deps.dir` | +| `BOOST_INCLUDEDIR` | `boost.include.dir` (legacy alias) | + +For a **standalone** `cmake` configure, pass `-DWITH_SSL=ON`, `-DIOTDB_OFFLINE=ON`, +etc. directly. + +| Option | Default | Purpose | +|-----------------------|----------------------------------|----------------------------------------------------------------------------------------------------------| +| `WITH_SSL` | `OFF` | Link against OpenSSL. See *SSL* below. | +| `BUILD_TESTING` | `OFF` (Maven sets `ON` for verify) | Build Catch2 IT executables (Catch2 v2.13.7 header downloaded at configure time). | +| `CATCH2_INCLUDE_DIR` | (unset) | Pre-downloaded Catch2 include dir (Maven sets this under `target/test/catch2`). | +| `IOTDB_OFFLINE` | `OFF` | Disallow any network access during configure. | +| `IOTDB_DEPS_DIR` | `/third-party` | Override the local tarball cache directory. | +| `BOOST_VERSION` | `1.60.0` (`1.84.0` on macOS) | Boost version that CMake will look for / download. | +| `THRIFT_VERSION` | `0.21.0` | Apache Thrift version to build from source. | +| `BOOST_ROOT` | (unset) | Existing Boost install to reuse, equivalent to `-Dboost.include.dir=...` from the legacy build. | +| `OPENSSL_ROOT_DIR` | (unset) | Existing OpenSSL install when `WITH_SSL=ON`. | +| `CMAKE_INSTALL_PREFIX`| `/install` | Install location. | + +## Online build (default) + +CMake will download any missing tarball at configure time. The first run is +slow (≈100 MB download + a Thrift build); subsequent runs reuse the +extracted artifacts under `build/_deps/`. + +```bash +# Linux / macOS +mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests package + +# Windows (Developer Command Prompt for VS, PowerShell, or cmd) +mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests "-Dboost.include.dir=C:\boost_1_88_0" package +``` -4. If you encounter a problem like "cannot find 'unistd.h'", please open the file -"iotdb-bin-resources\iotdb-tools-thrift\target\build\compiler\cpp\thrift\thriftl.cc" and replace -"#include " with "#include " and "#include "; then, rerun the command - in the third step; +## Offline build -5. Return to the cpp client repository and compile it with: +1. Pre-populate the platform-specific sub-directory under `third-party/`: -``` -mvn clean package -P with-cpp -pl iotdb-client/client-cpp -am -DskipTest --D"boost.include.dir"="D:\boost_1_75_0" -D"boost.library.dir"="D:\boost_1_75_0\stage\lib" + | Platform | Required files | + |------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------| + | `linux/` | `thrift-0.21.0.tar.gz`, `boost_1_60_0.tar.gz`, `m4-1.4.19.tar.gz`, `flex-2.6.4.tar.gz`, `bison-3.8.tar.gz` (and `openssl-3.5.0.tar.gz` when `WITH_SSL=ON`) | + | `mac/` | `thrift-0.21.0.tar.gz`, `boost_1_84_0.tar.gz` (newer Boost for Xcode/Clang; Apple ships m4/flex/bison; `openssl-3.5.0.tar.gz` optional) | + | `windows/` | `thrift-0.21.0.tar.gz`, `boost_1_60_0.tar.gz` (Boost headers only - no `b2` build required for `iotdb_session`) | + + Reference URLs (the configure step uses the same): + - Apache Thrift 0.21.0: + - Boost 1.60.0: + - GNU m4 1.4.19: + - GNU flex 2.6.4: + - GNU bison 3.8: + - OpenSSL 3.5.0: + +2. Run the build with offline mode enabled: + + ```bash + mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests \ + -Diotdb.offline=ON package + ``` + + or, going straight through CMake: + + ```bash + cmake -S iotdb-client/client-cpp -B build -DIOTDB_OFFLINE=ON + cmake --build build --config Release --target install + ``` + +CI environments can share a single cache by setting +`-DIOTDB_DEPS_DIR=/path/to/cache` instead of copying tarballs around. + +## Platform-specific notes + +### Linux + +- Tested with GCC 7+ and Clang 9+. Anything that can compile Apache Thrift + 0.21.0 works. +- Build deps that must already exist on the host (only required when + CMake auto-builds m4/flex/bison from tarball): `make`, `autoconf`, + `gcc`, plus the standard C/C++ toolchain. `sudo` is **not** required; + the helper tools install under `build/tools/`. +- If you would rather use distro-provided tools (`apt install m4 flex + bison`), CMake will pick them up first. + +### macOS + +- Xcode Command Line Tools provide `m4`, `flex`, `bison`, and `make`, + so the auto-build path normally skips them. +- Homebrew users can `brew install boost` to short-circuit `FetchBoost`. + +### Windows + +Visual Studio **2017, 2019, 2022, or 2026** is supported for building the SDK. +Link your application against the zip built with the **same VS generation** you +use for your project. + +Prerequisites: + +1. **Boost.** Download and extract + + (any 1.60+ release will work). `iotdb_session` only needs Boost + headers, so running `bootstrap.bat` / `b2` is optional. Pass the + location with either `-Dboost.include.dir="C:\boost_1_88_0"` (Maven) + or `-DBOOST_ROOT="C:\boost_1_88_0"` (raw CMake). +2. **flex / bison.** Install + and rename `win_flex.exe`→`flex.exe`, `win_bison.exe`→`bison.exe` on + `PATH`. +3. **OpenSSL** *(only when `WITH_SSL=ON`)*: run the Win64 OpenSSL + installer from , then + pass `-DOPENSSL_ROOT_DIR=...` to CMake. + +On Windows the SDK ships as **`iotdb_session.dll`** plus an import library +**`iotdb_session.lib`**, built with **`/MD`** (dynamic CRT, same as a +default Visual Studio application). Thrift is linked into the DLL; users +do not install separate Thrift headers or libraries. Place +`iotdb_session.dll` next to your `.exe` or on `PATH`. + +Auto-building m4/flex/bison from tarball is **not** supported on Windows; +the GNU autotools tarballs assume a POSIX shell environment. + +## SSL + +Both Thrift and `iotdb_session` build without OpenSSL by default. Enable +SSL with `-Dwith.ssl=ON` (Maven) or `-DWITH_SSL=ON` (standalone CMake). +CMake first calls `find_package(OpenSSL)`; +if nothing is found, it falls back to: + +- **Linux / macOS** – use a local `openssl-.tar.gz` (or download it + when not in offline mode), configure with `no-shared`, install into + `build/_deps/openssl/install`, and link statically. +- **Windows** – fail with a friendly message that points at the Win64 + OpenSSL installer. Building OpenSSL from source via MSVC is out of scope. + +## Tests + +Maven binds `cmake-maven-plugin`'s `test` goal to the `integration-test` +phase and runs `ctest`. `pre-integration-test` spawns a local IoTDB server +from `distribution/target/.../sbin/start-standalone.{sh,bat}`, so make sure +the distribution module is built first: + +```bash +mvn clean install -P with-cpp -pl distribution,iotdb-client/client-cpp -am -DskipTests +mvn -P with-cpp -pl iotdb-client/client-cpp -am verify ``` +Running ctest directly (after a `mvn ... package` build) is also supported: -### Test -First build IoTDB server together with the cpp client. +```bash +cd iotdb-client/client-cpp/target/build/test +ctest --output-on-failure +``` -Explicitly using "install" instead of package in order to be sure we're using libs built on this -machine. +## Code formatting -`mvn clean install -P with-cpp -pl distribution,iotdb-client/client-cpp -am -DskipTests` +We use `clang-format` (pinned by the root POM as `clang.format.version`) +through Maven Spotless. **clang-format 17.0.6** is the version CI runs. -After run verify +```bash +mvn -P with-cpp -pl iotdb-client/client-cpp spotless:check +mvn -P with-cpp -pl iotdb-client/client-cpp spotless:apply +``` -`mvn clean verify -P with-cpp -pl iotdb-client/client-cpp -am` +On JDK 8 the C++ Spotless profile is skipped automatically (Spotless's +clang-format integration requires Spotless 2.44+, which itself requires +JDK 11+). -## Package Hierarchy +## Package layout -If the compilation finishes successfully, the packaged zip file will be placed under -"client-cpp/target/client-cpp-${project.version}-cpp-${os}.zip". +A successful `mvn ... package` produces +`target/iotdb-session-cpp--.zip` with this layout: -On macOS, the hierarchy of the package should look like this: ``` -. -+-- client -| +-- include -| +-- Session.h -| +-- IClientRPCService.h -| +-- client_types.h -| +-- common_types.h -| +-- thrift -| +-- thrift_headers... -| +-- lib -| +-- Release -| +-- libiotdb_session.dylib -| +-- parser.dylib -| +-- thriftmd.dylib -| +-- tutorialgencpp.dylib +iotdb-session-cpp--/ +|-- BUILD-INFO.txt +|-- VERSION +|-- cmake/ +|-- pkgconfig/ +|-- include/ +| |-- Session.h +| |-- SessionC.h +| `-- ... (public API headers only; no Thrift/Boost) +`-- lib/ + |-- libiotdb_session.{so,dylib} (Linux / macOS) + |-- iotdb_session.dll (Windows runtime) + `-- iotdb_session.lib (Windows import library for linking) ``` -## Using C++ Client: -``` -1. Put the zip file "client-cpp-${project.version}-cpp-${os}.zip" wherever you want; - -2. Unzip the archive using the following command, and then you can get the two directories -mentioned above, the header file and the dynamic library: - unzip client-cpp-${project.version}-cpp-${os}.zip +Thrift is embedded inside `iotdb_session` on all platforms; it is not shipped +as a separate install artifact. -3. Write C++ code to call the operation interface of the cpp client to operate IoTDB, - for detail interface information, please refer to the link: https://iotdb.apache.org/zh/UserGuide/Master/API/Programming-Cpp-Native-API.html +## Using the C++ client - E.g: - #include "include/Session.h" +```cpp +#include "Session.h" #include #include int main() { - std::cout << "open session" << std::endl; - std::shared_ptr session(new Session("127.0.0.1", 6667, "root", "root")); + auto session = std::make_shared("127.0.0.1", 6667, "root", "root"); session->open(false); - - std::cout << "setStorageGroup: root.test01" << std::endl; session->setStorageGroup("root.test01"); - if (!session->checkTimeseriesExists("root.test01.d0.s0")) { - session->createTimeseries("root.test01.d0.s0", TSDataType::INT64, TSEncoding::RLE, CompressionType::SNAPPY); - std::cout << "create Timeseries: root.test01.d0.s0" << std::endl; - } - - std::cout << "session close" << std::endl; + session->createTimeseries( + "root.test01.d0.s0", + TSDataType::INT64, + TSEncoding::RLE, + CompressionType::SNAPPY); + } session->close(); } +``` -4. Compile and execute - clang++ -O2 user-cpp-code.cpp -liotdb_session -L/user-unzip-absolute-path/lib -Wl,-rpath /user-unzip-absolute-path/lib -std=c++11 - ./a.out +Compile against the produced SDK: + +```bash +clang++ -O2 user-cpp-code.cpp \ + -I/path/to/sdk/include \ + -L/path/to/sdk/lib \ + -liotdb_session -lpthread \ + -Wl,-rpath,/path/to/sdk/lib \ + -std=c++11 ``` + +For full API documentation see the [C++ Native API guide](https://iotdb.apache.org/UserGuide/latest/API/Programming-Cpp-Native-API.html). diff --git a/iotdb-client/client-cpp/cmake/FetchBoost.cmake b/iotdb-client/client-cpp/cmake/FetchBoost.cmake new file mode 100644 index 0000000000000..d1ab9c18ae1b9 --- /dev/null +++ b/iotdb-client/client-cpp/cmake/FetchBoost.cmake @@ -0,0 +1,130 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +# ============================================================================= +# FetchBoost.cmake +# +# Resolves the location of Boost headers needed at thrift / iotdb_session +# compile time, in three stages: +# +# 1. Use system / user-provided Boost (find_package, BOOST_ROOT, etc.). +# 2. Fall back to a local tarball under ${IOTDB_OS_DEPS_DIR}/. +# 3. Otherwise download the tarball from archives.boost.io (unless +# IOTDB_OFFLINE is ON). +# +# Output (cache) variables: +# BOOST_INCLUDE_DIR - directory containing headers. +# ============================================================================= + +if(DEFINED BOOST_INCLUDE_DIR AND EXISTS "${BOOST_INCLUDE_DIR}/boost/version.hpp") + message(STATUS "[Boost] reusing cached BOOST_INCLUDE_DIR=${BOOST_INCLUDE_DIR}") + return() +endif() + +# --------------------------------------------------------------------------- +# Stage 1: find_package(Boost) - respects BOOST_ROOT / Boost_INCLUDE_DIR +# --------------------------------------------------------------------------- +find_package(Boost QUIET) +if(Boost_FOUND AND Boost_INCLUDE_DIRS) + set(BOOST_INCLUDE_DIR "${Boost_INCLUDE_DIRS}" CACHE PATH "Boost include directory" FORCE) + message(STATUS "[Boost] using system Boost at ${BOOST_INCLUDE_DIR}") + return() +endif() + +# Allow plain -DBOOST_INCLUDEDIR= / -DBoost_INCLUDE_DIR= as a fast path. +foreach(_hint Boost_INCLUDE_DIR BOOST_INCLUDEDIR BOOST_ROOT) + if(DEFINED ${_hint}) + set(_candidate "${${_hint}}") + if(EXISTS "${_candidate}/boost/version.hpp") + set(BOOST_INCLUDE_DIR "${_candidate}" CACHE PATH "Boost include directory" FORCE) + message(STATUS "[Boost] using hinted path ${BOOST_INCLUDE_DIR}") + return() + elseif(EXISTS "${_candidate}/include/boost/version.hpp") + set(BOOST_INCLUDE_DIR "${_candidate}/include" CACHE PATH "Boost include directory" FORCE) + message(STATUS "[Boost] using hinted path ${BOOST_INCLUDE_DIR}") + return() + endif() + endif() +endforeach() + +# --------------------------------------------------------------------------- +# Stage 2: local tarball cache +# --------------------------------------------------------------------------- +string(REPLACE "." "_" _boost_us "${BOOST_VERSION}") +set(_boost_dirname "boost_${_boost_us}") +set(_boost_tarname_gz "${_boost_dirname}.tar.gz") +set(_boost_tarname_zip "${_boost_dirname}.zip") + +set(_boost_tarball "") +foreach(_name IN ITEMS ${_boost_tarname_gz} ${_boost_tarname_zip}) + if(EXISTS "${IOTDB_OS_DEPS_DIR}/${_name}") + set(_boost_tarball "${IOTDB_OS_DEPS_DIR}/${_name}") + break() + endif() +endforeach() + +# --------------------------------------------------------------------------- +# Stage 3: download +# --------------------------------------------------------------------------- +if(NOT _boost_tarball) + if(IOTDB_OFFLINE) + message(FATAL_ERROR + "[Boost] IOTDB_OFFLINE=ON but no Boost tarball found in " + "${IOTDB_OS_DEPS_DIR}. Expected one of: ${_boost_tarname_gz}, ${_boost_tarname_zip}.") + endif() + + set(_boost_url "https://archives.boost.io/release/${BOOST_VERSION}/source/${_boost_tarname_gz}") + set(_boost_tarball "${IOTDB_OS_DEPS_DIR}/${_boost_tarname_gz}") + message(STATUS "[Boost] downloading ${_boost_url}") + file(DOWNLOAD "${_boost_url}" "${_boost_tarball}" + SHOW_PROGRESS + STATUS _boost_dl_status + TLS_VERIFY ON) + list(GET _boost_dl_status 0 _boost_dl_code) + if(NOT _boost_dl_code EQUAL 0) + list(GET _boost_dl_status 1 _boost_dl_msg) + file(REMOVE "${_boost_tarball}") + message(FATAL_ERROR "[Boost] download failed: ${_boost_dl_msg}") + endif() +endif() + +# --------------------------------------------------------------------------- +# Extract headers-only into ${CMAKE_BINARY_DIR}/_deps/boost +# --------------------------------------------------------------------------- +set(_boost_extract_dir "${CMAKE_BINARY_DIR}/_deps/boost") +set(_boost_marker "${_boost_extract_dir}/.extracted-${BOOST_VERSION}") + +if(NOT EXISTS "${_boost_marker}") + file(REMOVE_RECURSE "${_boost_extract_dir}") + file(MAKE_DIRECTORY "${_boost_extract_dir}") + message(STATUS "[Boost] extracting ${_boost_tarball}") + file(ARCHIVE_EXTRACT + INPUT "${_boost_tarball}" + DESTINATION "${_boost_extract_dir}") + file(TOUCH "${_boost_marker}") +endif() + +set(BOOST_INCLUDE_DIR "${_boost_extract_dir}/${_boost_dirname}" + CACHE PATH "Boost include directory" FORCE) + +if(NOT EXISTS "${BOOST_INCLUDE_DIR}/boost/version.hpp") + message(FATAL_ERROR + "[Boost] Could not locate boost/version.hpp after extraction. " + "Looked in: ${BOOST_INCLUDE_DIR}") +endif() + +message(STATUS "[Boost] BOOST_INCLUDE_DIR = ${BOOST_INCLUDE_DIR}") diff --git a/iotdb-client/client-cpp/cmake/FetchBuildTools.cmake b/iotdb-client/client-cpp/cmake/FetchBuildTools.cmake new file mode 100644 index 0000000000000..c9d7482e906ea --- /dev/null +++ b/iotdb-client/client-cpp/cmake/FetchBuildTools.cmake @@ -0,0 +1,266 @@ +# 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. +# +# ============================================================================= +# FetchBuildTools.cmake +# +# Apache Thrift's source build needs a working flex / bison toolchain (m4 too +# on Unix). When the host already provides them on PATH we use them as-is. +# Otherwise we provision them locally: +# +# * Linux / macOS - configure-make-install each tool from a tarball into +# ${CMAKE_BINARY_DIR}/tools (no sudo required). +# * Windows - extract the winflexbison zip and copy +# win_flex.exe -> flex.exe, win_bison.exe -> bison.exe. +# +# Tarballs / zips are resolved with the standard three-stage pattern: +# 1. ${IOTDB_OS_DEPS_DIR}/ (or any match for an optional GLOB) +# 2. file(DOWNLOAD) when IOTDB_OFFLINE is OFF +# 3. FATAL_ERROR otherwise +# ============================================================================= + +set(_tools_prefix "${CMAKE_BINARY_DIR}/tools") +set(_tools_bin "${_tools_prefix}/bin") +file(MAKE_DIRECTORY "${_tools_bin}") + +# Make sure any tool we install locally takes precedence over the system PATH +# for the remainder of the configure step (and child ExternalProject calls). +if(WIN32) + set(ENV{PATH} "${_tools_bin};$ENV{PATH}") +else() + set(ENV{PATH} "${_tools_bin}:$ENV{PATH}") +endif() + +set(M4_VERSION "1.4.19" CACHE STRING "GNU m4 version to build when missing") +set(FLEX_VERSION "2.6.4" CACHE STRING "GNU flex version to build when missing") +set(BISON_VERSION "3.8" CACHE STRING "GNU bison version to build when missing") +set(WINFLEXBISON_VERSION "2.5.25" + CACHE STRING "winflexbison version to download when no local zip is present") + +set(_m4_url "https://ftp.gnu.org/gnu/m4/m4-${M4_VERSION}.tar.gz") +set(_flex_url "https://github.com/westes/flex/releases/download/v${FLEX_VERSION}/flex-${FLEX_VERSION}.tar.gz") +set(_bison_url "https://ftp.gnu.org/gnu/bison/bison-${BISON_VERSION}.tar.gz") +set(_winflexbison_url + "https://github.com/lexxmark/winflexbison/releases/download/v${WINFLEXBISON_VERSION}/win_flex_bison-${WINFLEXBISON_VERSION}.zip") +set(_winflexbison_filename "win_flex_bison-${WINFLEXBISON_VERSION}.zip") + +include(ProcessorCount) +ProcessorCount(_jobs) +if(_jobs LESS 1) + set(_jobs 1) +endif() + +# Resolve tarball: prefer the exact filename in ${IOTDB_OS_DEPS_DIR}/, then +# any path matching GLOB_PATTERN (caller-supplied wildcard for relaxed naming, +# e.g. win_flex_bison*.zip), and finally fall back to a download. +function(_iotdb_resolve_tarball OUT_VAR FILENAME URL) + cmake_parse_arguments(ARG "" "GLOB_PATTERN" "" ${ARGN}) + + set(_local "${IOTDB_OS_DEPS_DIR}/${FILENAME}") + if(EXISTS "${_local}") + set(${OUT_VAR} "${_local}" PARENT_SCOPE) + return() + endif() + + if(ARG_GLOB_PATTERN) + file(GLOB _matches "${IOTDB_OS_DEPS_DIR}/${ARG_GLOB_PATTERN}") + if(_matches) + list(GET _matches 0 _hit) + message(STATUS "[BuildTools] reusing ${_hit}") + set(${OUT_VAR} "${_hit}" PARENT_SCOPE) + return() + endif() + endif() + + if(IOTDB_OFFLINE) + set(_hint "${FILENAME}") + if(ARG_GLOB_PATTERN) + set(_hint "${FILENAME} (or any ${ARG_GLOB_PATTERN})") + endif() + message(FATAL_ERROR + "[BuildTools] IOTDB_OFFLINE=ON but ${_hint} is missing in " + "${IOTDB_OS_DEPS_DIR}.") + endif() + + message(STATUS "[BuildTools] downloading ${URL}") + file(DOWNLOAD "${URL}" "${_local}" SHOW_PROGRESS STATUS _st TLS_VERIFY ON) + list(GET _st 0 _code) + if(NOT _code EQUAL 0) + list(GET _st 1 _msg) + file(REMOVE "${_local}") + message(FATAL_ERROR "[BuildTools] download failed for ${FILENAME}: ${_msg}") + endif() + set(${OUT_VAR} "${_local}" PARENT_SCOPE) +endfunction() + +# Configure-make-install from into ${_tools_prefix}. +function(_iotdb_build_autotools NAME TARBALL EXTRACTED_DIRNAME) + set(_src_root "${CMAKE_BINARY_DIR}/_deps/${NAME}") + set(_marker "${_tools_prefix}/.${NAME}-installed") + if(EXISTS "${_marker}") + return() + endif() + file(REMOVE_RECURSE "${_src_root}") + file(MAKE_DIRECTORY "${_src_root}") + message(STATUS "[BuildTools] extracting ${TARBALL}") + file(ARCHIVE_EXTRACT INPUT "${TARBALL}" DESTINATION "${_src_root}") + set(_src "${_src_root}/${EXTRACTED_DIRNAME}") + if(NOT EXISTS "${_src}/configure") + message(FATAL_ERROR + "[BuildTools] expected configure script at ${_src}/configure") + endif() + message(STATUS "[BuildTools] building ${NAME} -> ${_tools_prefix}") + # flex 2.6.4: reallocarray() needs _GNU_SOURCE on glibc 2.26+ (westes/flex#241). + set(_env_prefix "") + if(NAME STREQUAL "flex" AND CMAKE_SYSTEM_NAME STREQUAL "Linux") + set(_env_prefix env CFLAGS=-D_GNU_SOURCE CXXFLAGS=-D_GNU_SOURCE) + endif() + if(_env_prefix) + execute_process( + COMMAND ${_env_prefix} ./configure --prefix=${_tools_prefix} + WORKING_DIRECTORY "${_src}" + RESULT_VARIABLE _rc) + else() + execute_process( + COMMAND ./configure --prefix=${_tools_prefix} + WORKING_DIRECTORY "${_src}" + RESULT_VARIABLE _rc) + endif() + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[BuildTools] configure failed for ${NAME}") + endif() + if(_env_prefix) + execute_process( + COMMAND ${_env_prefix} make -j${_jobs} + WORKING_DIRECTORY "${_src}" + RESULT_VARIABLE _rc) + else() + execute_process( + COMMAND make -j${_jobs} + WORKING_DIRECTORY "${_src}" + RESULT_VARIABLE _rc) + endif() + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[BuildTools] make failed for ${NAME}") + endif() + if(_env_prefix) + execute_process( + COMMAND ${_env_prefix} make install + WORKING_DIRECTORY "${_src}" + RESULT_VARIABLE _rc) + else() + execute_process( + COMMAND make install + WORKING_DIRECTORY "${_src}" + RESULT_VARIABLE _rc) + endif() + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[BuildTools] make install failed for ${NAME}") + endif() + file(TOUCH "${_marker}") +endfunction() + +# ============================================================================= +# Windows branch - winflexbison +# ============================================================================= +if(WIN32) + # Stage 1: pick up an existing flex/bison if the host already provides it. + find_program(FLEX_EXECUTABLE NAMES flex win_flex) + find_program(BISON_EXECUTABLE NAMES bison win_bison) + + if(FLEX_EXECUTABLE AND BISON_EXECUTABLE) + message(STATUS "[BuildTools] using system flex = ${FLEX_EXECUTABLE}") + message(STATUS "[BuildTools] using system bison = ${BISON_EXECUTABLE}") + set(IOTDB_LOCAL_TOOLS_BIN "${_tools_bin}" CACHE INTERNAL "") + return() + endif() + + # Stage 2/3: resolve and extract the winflexbison zip into _tools_bin. + _iotdb_resolve_tarball(_wfb_zip + "${_winflexbison_filename}" + "${_winflexbison_url}" + GLOB_PATTERN "win_flex_bison*.zip") + + set(_wfb_marker "${_tools_bin}/.winflexbison-installed") + if(NOT EXISTS "${_wfb_marker}") + message(STATUS "[BuildTools] extracting ${_wfb_zip}") + file(ARCHIVE_EXTRACT INPUT "${_wfb_zip}" DESTINATION "${_tools_bin}") + + if(NOT EXISTS "${_tools_bin}/win_flex.exe" OR NOT EXISTS "${_tools_bin}/win_bison.exe") + message(FATAL_ERROR + "[BuildTools] win_flex.exe / win_bison.exe not found after " + "extracting ${_wfb_zip} into ${_tools_bin}.") + endif() + + # Copy with renamed targets so thrift's CMakeLists sees flex/bison. + execute_process(COMMAND ${CMAKE_COMMAND} -E copy + "${_tools_bin}/win_flex.exe" "${_tools_bin}/flex.exe" + RESULT_VARIABLE _rc) + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[BuildTools] failed to copy win_flex.exe -> flex.exe") + endif() + execute_process(COMMAND ${CMAKE_COMMAND} -E copy + "${_tools_bin}/win_bison.exe" "${_tools_bin}/bison.exe" + RESULT_VARIABLE _rc) + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[BuildTools] failed to copy win_bison.exe -> bison.exe") + endif() + file(TOUCH "${_wfb_marker}") + endif() + + find_program(FLEX_EXECUTABLE flex PATHS "${_tools_bin}" NO_DEFAULT_PATH REQUIRED) + find_program(BISON_EXECUTABLE bison PATHS "${_tools_bin}" NO_DEFAULT_PATH REQUIRED) + + message(STATUS "[BuildTools] flex = ${FLEX_EXECUTABLE}") + message(STATUS "[BuildTools] bison = ${BISON_EXECUTABLE}") + set(IOTDB_LOCAL_TOOLS_BIN "${_tools_bin}" CACHE INTERNAL "") + return() +endif() + +# ============================================================================= +# Linux / macOS branch - m4 / flex / bison from autotools tarballs +# ============================================================================= + +# m4 (flex/bison both depend on this) +find_program(M4_EXECUTABLE m4) +if(NOT M4_EXECUTABLE) + _iotdb_resolve_tarball(_m4_tarball "m4-${M4_VERSION}.tar.gz" "${_m4_url}") + _iotdb_build_autotools(m4 "${_m4_tarball}" "m4-${M4_VERSION}") + find_program(M4_EXECUTABLE m4 PATHS "${_tools_bin}" NO_DEFAULT_PATH REQUIRED) +endif() +message(STATUS "[BuildTools] m4 = ${M4_EXECUTABLE}") + +# flex +find_program(FLEX_EXECUTABLE flex) +if(NOT FLEX_EXECUTABLE) + _iotdb_resolve_tarball(_flex_tarball "flex-${FLEX_VERSION}.tar.gz" "${_flex_url}") + _iotdb_build_autotools(flex "${_flex_tarball}" "flex-${FLEX_VERSION}") + find_program(FLEX_EXECUTABLE flex PATHS "${_tools_bin}" NO_DEFAULT_PATH REQUIRED) +endif() +message(STATUS "[BuildTools] flex = ${FLEX_EXECUTABLE}") + +# bison +find_program(BISON_EXECUTABLE bison) +if(NOT BISON_EXECUTABLE) + _iotdb_resolve_tarball(_bison_tarball "bison-${BISON_VERSION}.tar.gz" "${_bison_url}") + _iotdb_build_autotools(bison "${_bison_tarball}" "bison-${BISON_VERSION}") + find_program(BISON_EXECUTABLE bison PATHS "${_tools_bin}" NO_DEFAULT_PATH REQUIRED) +endif() +message(STATUS "[BuildTools] bison = ${BISON_EXECUTABLE}") + +# Expose the bin dir for downstream ExternalProject_Add calls. +set(IOTDB_LOCAL_TOOLS_BIN "${_tools_bin}" CACHE INTERNAL "") diff --git a/iotdb-client/client-cpp/cmake/FetchOpenSSL.cmake b/iotdb-client/client-cpp/cmake/FetchOpenSSL.cmake new file mode 100644 index 0000000000000..575e2803f2bd4 --- /dev/null +++ b/iotdb-client/client-cpp/cmake/FetchOpenSSL.cmake @@ -0,0 +1,121 @@ +# 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. +# +# ============================================================================= +# FetchOpenSSL.cmake (only included when WITH_SSL=ON) +# +# Resolution order: +# 1. find_package(OpenSSL) - any system / vendor install is taken as-is. +# 2. On Linux/macOS: +# use tarball ${IOTDB_OS_DEPS_DIR}/openssl-${OPENSSL_VERSION}.tar.gz +# or download from openssl.org when not in offline mode, then +# ./Configure && make && make install_sw into ${CMAKE_BINARY_DIR}/_deps/openssl. +# 3. On Windows: emit a FATAL_ERROR with instructions to run the bundled +# Win64OpenSSL installer (or any other prebuilt OpenSSL); building +# OpenSSL from source on MSVC is out of scope. +# +# Side effects: +# Defines imported targets OpenSSL::SSL / OpenSSL::Crypto via find_package +# so callers can just link against them. +# ============================================================================= + +set(OPENSSL_VERSION "3.5.0" CACHE STRING "OpenSSL version to fetch when missing") + +find_package(OpenSSL QUIET) +if(OpenSSL_FOUND) + message(STATUS "[OpenSSL] using system OpenSSL ${OPENSSL_VERSION_MAJOR}.${OPENSSL_VERSION_MINOR}") + return() +endif() + +if(WIN32) + message(FATAL_ERROR + "[OpenSSL] WITH_SSL=ON but no OpenSSL was found on Windows. " + "Please run third-party/windows/Win64OpenSSL-3_5_0.exe (or any " + "OpenSSL installer), then re-run the configure step with " + "-DOPENSSL_ROOT_DIR=.") +endif() + +# --- Linux / macOS fallback: build from source --------------------------- +set(_ossl_tarname "openssl-${OPENSSL_VERSION}.tar.gz") +set(_ossl_tarball "${IOTDB_OS_DEPS_DIR}/${_ossl_tarname}") + +if(NOT EXISTS "${_ossl_tarball}") + if(IOTDB_OFFLINE) + message(FATAL_ERROR + "[OpenSSL] IOTDB_OFFLINE=ON but ${_ossl_tarname} is missing in ${IOTDB_OS_DEPS_DIR}.") + endif() + set(_ossl_url "https://www.openssl.org/source/${_ossl_tarname}") + message(STATUS "[OpenSSL] downloading ${_ossl_url}") + file(DOWNLOAD "${_ossl_url}" "${_ossl_tarball}" + SHOW_PROGRESS TLS_VERIFY ON STATUS _st) + list(GET _st 0 _code) + if(NOT _code EQUAL 0) + list(GET _st 1 _msg) + file(REMOVE "${_ossl_tarball}") + message(FATAL_ERROR "[OpenSSL] download failed: ${_msg}") + endif() +endif() + +set(_ossl_root "${CMAKE_BINARY_DIR}/_deps/openssl") +set(_ossl_src "${_ossl_root}/src/openssl-${OPENSSL_VERSION}") +set(_ossl_inst "${_ossl_root}/install") +set(_ossl_stamp "${_ossl_root}/.built-${OPENSSL_VERSION}") + +if(NOT EXISTS "${_ossl_stamp}") + file(REMOVE_RECURSE "${_ossl_root}/src") + file(MAKE_DIRECTORY "${_ossl_root}/src") + message(STATUS "[OpenSSL] extracting ${_ossl_tarball}") + file(ARCHIVE_EXTRACT INPUT "${_ossl_tarball}" DESTINATION "${_ossl_root}/src") + + include(ProcessorCount) + ProcessorCount(_jobs) + if(_jobs LESS 1) + set(_jobs 1) + endif() + + message(STATUS "[OpenSSL] configuring -> ${_ossl_inst}") + execute_process( + COMMAND ./Configure --prefix=${_ossl_inst} --openssldir=${_ossl_inst}/ssl no-shared + WORKING_DIRECTORY "${_ossl_src}" + RESULT_VARIABLE _rc) + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[OpenSSL] Configure failed (rc=${_rc})") + endif() + + message(STATUS "[OpenSSL] building (-j${_jobs})") + execute_process( + COMMAND make -j${_jobs} + WORKING_DIRECTORY "${_ossl_src}" + RESULT_VARIABLE _rc) + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[OpenSSL] make failed (rc=${_rc})") + endif() + + execute_process( + COMMAND make install_sw + WORKING_DIRECTORY "${_ossl_src}" + RESULT_VARIABLE _rc) + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[OpenSSL] make install_sw failed (rc=${_rc})") + endif() + file(TOUCH "${_ossl_stamp}") +endif() + +set(OPENSSL_ROOT_DIR "${_ossl_inst}" CACHE PATH "OpenSSL root" FORCE) +set(OPENSSL_USE_STATIC_LIBS ON) +find_package(OpenSSL REQUIRED) +message(STATUS "[OpenSSL] built locally at ${OPENSSL_ROOT_DIR}") diff --git a/iotdb-client/client-cpp/cmake/FetchThrift.cmake b/iotdb-client/client-cpp/cmake/FetchThrift.cmake new file mode 100644 index 0000000000000..d17e3075d2199 --- /dev/null +++ b/iotdb-client/client-cpp/cmake/FetchThrift.cmake @@ -0,0 +1,277 @@ +# 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. +# +# ============================================================================= +# FetchThrift.cmake +# +# Downloads (or uses a local copy of) the Apache Thrift source tarball and +# builds it from source as a static-only, runtime-and-compiler artifact. The +# build runs at configure time so the thrift compiler is available for the +# code generation step that follows. +# +# Exported variables: +# THRIFT_EXECUTABLE - path to the thrift binary that just got built +# THRIFT_INCLUDE_DIR - include directory containing +# THRIFT_STATIC_LIB_PATH- path to libthrift.a / thriftmd.lib +# THRIFT_RUNTIME_LIB_DIR- directory containing the static thrift library +# (and any other build artifacts you may want to +# bundle with the SDK) +# +# Imported targets: +# iotdb_thrift_static - INTERFACE-compatible IMPORTED target that +# propagates include dirs and the static lib. +# iotdb_thrift_external - phony custom target representing the build +# (use add_dependencies( iotdb_thrift_external) +# to ensure the thrift build runs first). +# ============================================================================= + +include(ExternalProject) + +set(_thrift_dirname "thrift-${THRIFT_VERSION}") +set(_thrift_tarname "${_thrift_dirname}.tar.gz") + +# --------------------------------------------------------------------------- +# Resolve tarball (local cache -> download) +# --------------------------------------------------------------------------- +set(_thrift_tarball "${IOTDB_OS_DEPS_DIR}/${_thrift_tarname}") +if(NOT EXISTS "${_thrift_tarball}") + if(IOTDB_OFFLINE) + message(FATAL_ERROR + "[Thrift] IOTDB_OFFLINE=ON but ${_thrift_tarname} is missing in " + "${IOTDB_OS_DEPS_DIR}.") + endif() + set(_thrift_url "https://archive.apache.org/dist/thrift/${THRIFT_VERSION}/${_thrift_tarname}") + message(STATUS "[Thrift] downloading ${_thrift_url}") + file(DOWNLOAD "${_thrift_url}" "${_thrift_tarball}" + SHOW_PROGRESS TLS_VERIFY ON STATUS _thrift_dl) + list(GET _thrift_dl 0 _code) + if(NOT _code EQUAL 0) + list(GET _thrift_dl 1 _msg) + file(REMOVE "${_thrift_tarball}") + message(FATAL_ERROR "[Thrift] download failed: ${_msg}") + endif() +endif() + +# --------------------------------------------------------------------------- +# Extract once into ${CMAKE_BINARY_DIR}/_deps/thrift/src +# --------------------------------------------------------------------------- +set(_thrift_root "${CMAKE_BINARY_DIR}/_deps/thrift") +set(_thrift_src "${_thrift_root}/src/${_thrift_dirname}") +set(_thrift_build "${_thrift_root}/build") +set(_thrift_install "${_thrift_root}/install") +set(_thrift_marker "${_thrift_root}/.extracted-${THRIFT_VERSION}") + +set(_thrift_build_config "Release") +if(MSVC AND CMAKE_BUILD_TYPE) + set(_thrift_build_config "${CMAKE_BUILD_TYPE}") +endif() + +if(NOT EXISTS "${_thrift_marker}") + file(REMOVE_RECURSE "${_thrift_root}/src") + file(MAKE_DIRECTORY "${_thrift_root}/src") + message(STATUS "[Thrift] extracting ${_thrift_tarball}") + file(ARCHIVE_EXTRACT INPUT "${_thrift_tarball}" + DESTINATION "${_thrift_root}/src") + file(TOUCH "${_thrift_marker}") +endif() + +if(NOT EXISTS "${_thrift_src}/CMakeLists.txt") + message(FATAL_ERROR + "[Thrift] could not find ${_thrift_src}/CMakeLists.txt after " + "extracting ${_thrift_tarball}.") +endif() + +# --------------------------------------------------------------------------- +# ExternalProject_Add: build thrift at *configure* time so the produced +# binary / library can immediately drive code generation and linking. +# --------------------------------------------------------------------------- +set(_thrift_cmake_args + # CMake 4.x rejects Thrift 0.21's cmake_minimum_required(3.0); set policy first. + "-DCMAKE_POLICY_VERSION_MINIMUM=3.5" + "-DCMAKE_INSTALL_PREFIX=${_thrift_install}" + "-DCMAKE_BUILD_TYPE=${_thrift_build_config}" + "-DBUILD_JAVA=OFF" + "-DBUILD_NODEJS=OFF" + "-DBUILD_JAVASCRIPT=OFF" + "-DBUILD_PYTHON=OFF" + "-DBUILD_TESTING=OFF" + "-DBUILD_TUTORIALS=OFF" + "-DBUILD_SHARED_LIBS=OFF" + "-DWITH_SHARED_LIB=OFF" + "-DWITH_STATIC_LIB=ON" + "-DCMAKE_POSITION_INDEPENDENT_CODE=ON" + "-DCMAKE_POLICY_DEFAULT_CMP0091=NEW" + "-DCMAKE_CXX_STANDARD=11") + +if(BOOST_INCLUDE_DIR) + list(APPEND _thrift_cmake_args + "-DBoost_INCLUDE_DIR=${BOOST_INCLUDE_DIR}" + "-DBOOST_INCLUDEDIR=${BOOST_INCLUDE_DIR}") +endif() + +if(MSVC) + list(APPEND _thrift_cmake_args + "-DCMAKE_MSVC_RUNTIME_LIBRARY=MultiThreaded$<$:Debug>DLL") +else() + set(_thrift_cxxflags "-fPIC") + if(IOTDB_LIBSTDCXX_CXX11_ABI) + set(_thrift_cxxflags "-D_GLIBCXX_USE_CXX11_ABI=1 -fPIC") + elseif(IOTDB_EXTRA_CXX_FLAGS MATCHES "_GLIBCXX_USE_CXX11_ABI=0") + set(_thrift_cxxflags "-D_GLIBCXX_USE_CXX11_ABI=0 -fPIC") + elseif(IOTDB_EXTRA_CXX_FLAGS MATCHES "_GLIBCXX_USE_CXX11_ABI=1") + set(_thrift_cxxflags "-D_GLIBCXX_USE_CXX11_ABI=1 -fPIC") + endif() + list(APPEND _thrift_cmake_args + "-DCMAKE_C_FLAGS=-fPIC" + "-DCMAKE_CXX_FLAGS=${_thrift_cxxflags}") +endif() + +if(WITH_SSL) + list(APPEND _thrift_cmake_args "-DWITH_OPENSSL=ON") +else() + list(APPEND _thrift_cmake_args "-DWITH_OPENSSL=OFF") +endif() + +# Run the ExternalProject build immediately so the thrift compiler is +# available for the subsequent code-generation step. We do this by +# invoking cmake twice via execute_process and only register a phony +# ExternalProject for dependency ordering. + +set(_thrift_stamp "${_thrift_build}/.built-${THRIFT_VERSION}-${_thrift_build_config}-mdll") +if(NOT EXISTS "${_thrift_stamp}") + file(MAKE_DIRECTORY "${_thrift_build}") + message(STATUS "[Thrift] configuring ${_thrift_dirname}") + + # When we built m4/flex/bison locally, make sure CMake passes the + # updated PATH down to the nested cmake invocation. + if(IOTDB_LOCAL_TOOLS_BIN) + set(_thrift_env "PATH=${IOTDB_LOCAL_TOOLS_BIN}:$ENV{PATH}") + endif() + + if(CMAKE_GENERATOR_PLATFORM) + set(_gen_platform_arg "-A" "${CMAKE_GENERATOR_PLATFORM}") + else() + set(_gen_platform_arg "") + endif() + + execute_process( + COMMAND ${CMAKE_COMMAND} + -G "${CMAKE_GENERATOR}" + ${_gen_platform_arg} + ${_thrift_cmake_args} + "${_thrift_src}" + WORKING_DIRECTORY "${_thrift_build}" + RESULT_VARIABLE _rc) + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[Thrift] configure step failed (rc=${_rc})") + endif() + + message(STATUS "[Thrift] building (${_thrift_build_config})") + execute_process( + COMMAND ${CMAKE_COMMAND} --build . --config ${_thrift_build_config} --target install + WORKING_DIRECTORY "${_thrift_build}" + RESULT_VARIABLE _rc) + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[Thrift] build/install step failed (rc=${_rc})") + endif() + file(TOUCH "${_thrift_stamp}") +endif() + +# --------------------------------------------------------------------------- +# Locate produced artifacts +# --------------------------------------------------------------------------- +set(THRIFT_INCLUDE_DIR "${_thrift_install}/include" CACHE PATH "" FORCE) + +# Thrift binary +if(WIN32) + find_program(THRIFT_EXECUTABLE + NAMES thrift + HINTS "${_thrift_install}/bin" "${_thrift_build}/bin/Release" + "${_thrift_build}/compiler/cpp/Release" + "${_thrift_build}/compiler/cpp/bin/Release" + NO_DEFAULT_PATH) +else() + find_program(THRIFT_EXECUTABLE + NAMES thrift + HINTS "${_thrift_install}/bin" + "${_thrift_build}/bin" + "${_thrift_build}/compiler/cpp/bin" + "${_thrift_build}/compiler/cpp" + NO_DEFAULT_PATH) +endif() + +if(NOT THRIFT_EXECUTABLE) + message(FATAL_ERROR + "[Thrift] could not find the thrift binary under ${_thrift_install} or ${_thrift_build}") +endif() +message(STATUS "[Thrift] THRIFT_EXECUTABLE = ${THRIFT_EXECUTABLE}") + +# Thrift static library (search a few standard install/lib locations). +set(_thrift_libname_candidates) +if(MSVC) + if(_thrift_build_config STREQUAL "Debug") + list(APPEND _thrift_libname_candidates thriftmdd.lib thriftmd.lib thriftmtd.lib thriftmt.lib thrift.lib) + else() + list(APPEND _thrift_libname_candidates thriftmd.lib thriftmt.lib thrift.lib) + endif() +else() + list(APPEND _thrift_libname_candidates libthrift.a) +endif() + +set(THRIFT_STATIC_LIB_PATH "") +foreach(_dir + "${_thrift_install}/lib" + "${_thrift_install}/lib64" + "${_thrift_build}/lib" + "${_thrift_build}/lib/Release" + "${_thrift_build}/lib/release") + if(THRIFT_STATIC_LIB_PATH) + break() + endif() + foreach(_n ${_thrift_libname_candidates}) + if(EXISTS "${_dir}/${_n}") + set(THRIFT_STATIC_LIB_PATH "${_dir}/${_n}") + set(THRIFT_RUNTIME_LIB_DIR "${_dir}") + break() + endif() + endforeach() +endforeach() + +if(NOT THRIFT_STATIC_LIB_PATH) + message(FATAL_ERROR + "[Thrift] could not locate the thrift static library under ${_thrift_install}/lib") +endif() +message(STATUS "[Thrift] THRIFT_STATIC_LIB_PATH = ${THRIFT_STATIC_LIB_PATH}") + +# Cache as well so subsequent reconfigures keep the same values. +set(THRIFT_STATIC_LIB_PATH "${THRIFT_STATIC_LIB_PATH}" CACHE FILEPATH "" FORCE) +set(THRIFT_RUNTIME_LIB_DIR "${THRIFT_RUNTIME_LIB_DIR}" CACHE PATH "" FORCE) + +# --------------------------------------------------------------------------- +# Imported target wrapping the static library. +# --------------------------------------------------------------------------- +if(NOT TARGET iotdb_thrift_static) + add_library(iotdb_thrift_static STATIC IMPORTED GLOBAL) + set_target_properties(iotdb_thrift_static PROPERTIES + IMPORTED_LOCATION "${THRIFT_STATIC_LIB_PATH}" + INTERFACE_INCLUDE_DIRECTORIES "${THRIFT_INCLUDE_DIR}") +endif() + +# Phony target so downstream code can express ordering deps. +if(NOT TARGET iotdb_thrift_external) + add_custom_target(iotdb_thrift_external ALL) +endif() diff --git a/iotdb-client/client-cpp/cmake/GenerateThriftSources.cmake b/iotdb-client/client-cpp/cmake/GenerateThriftSources.cmake new file mode 100644 index 0000000000000..6a3685af45847 --- /dev/null +++ b/iotdb-client/client-cpp/cmake/GenerateThriftSources.cmake @@ -0,0 +1,126 @@ +# 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. +# +# ============================================================================= +# GenerateThriftSources.cmake +# +# Generates C++ stubs from the iotdb-protocol Thrift files using the thrift +# compiler that FetchThrift just built. +# +# Inputs (resolved relative to the workspace root): +# iotdb-protocol/thrift-commons/src/main/thrift/common.thrift +# iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift +# +# Outputs: +# ${THRIFT_GEN_CPP_DIR}/*.{h,cpp} - generated C++ sources +# THRIFT_GENERATED_SRCS - list of *.cpp files for linking +# THRIFT_GENERATED_HDRS - list of *.h files (informational) +# ============================================================================= + +if(NOT THRIFT_EXECUTABLE) + message(FATAL_ERROR "[GenThrift] THRIFT_EXECUTABLE not set - did FetchThrift run first?") +endif() + +# Anchor the source-of-truth .thrift files via the workspace root. The CMake +# project lives at /iotdb-client/client-cpp/, so the workspace root is +# two levels up. +get_filename_component(_workspace_root "${CMAKE_CURRENT_SOURCE_DIR}/../.." ABSOLUTE) + +set(_common_thrift "${_workspace_root}/iotdb-protocol/thrift-commons/src/main/thrift/common.thrift") +set(_client_thrift "${_workspace_root}/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift") + +foreach(_f IN ITEMS "${_common_thrift}" "${_client_thrift}") + if(NOT EXISTS "${_f}") + message(FATAL_ERROR "[GenThrift] missing thrift input: ${_f}") + endif() +endforeach() + +# Mirror the bash reference script: stage both .thrift files into a flat +# working directory so `include "common.thrift"` resolves without -I args. +set(_thrift_proto_dir "${CMAKE_BINARY_DIR}/thrift-protocols") +file(MAKE_DIRECTORY "${_thrift_proto_dir}") +configure_file("${_common_thrift}" "${_thrift_proto_dir}/common.thrift" COPYONLY) +configure_file("${_client_thrift}" "${_thrift_proto_dir}/client.thrift" COPYONLY) + +set(THRIFT_GEN_CPP_DIR "${CMAKE_BINARY_DIR}/gen-cpp" CACHE PATH "" FORCE) +file(MAKE_DIRECTORY "${THRIFT_GEN_CPP_DIR}") + +# A stamp file lets us re-run thrift only when one of the inputs changes +# (handled by add_custom_command at build time) while also making sure we +# run once at configure time so the initial file(GLOB) finds something. +set(_thrift_stamp "${THRIFT_GEN_CPP_DIR}/.generated-stamp") + +set(_thrift_inputs + "${_thrift_proto_dir}/common.thrift" + "${_thrift_proto_dir}/client.thrift") + +set(_run_thrift FALSE) +if(NOT EXISTS "${_thrift_stamp}") + set(_run_thrift TRUE) +else() + foreach(_in IN LISTS _thrift_inputs) + if("${_in}" IS_NEWER_THAN "${_thrift_stamp}") + set(_run_thrift TRUE) + break() + endif() + endforeach() +endif() + +if(_run_thrift) + message(STATUS "[GenThrift] running ${THRIFT_EXECUTABLE} on common.thrift / client.thrift") + foreach(_in IN LISTS _thrift_inputs) + execute_process( + COMMAND "${THRIFT_EXECUTABLE}" -r --gen cpp:no_skeleton -out "${THRIFT_GEN_CPP_DIR}" "${_in}" + WORKING_DIRECTORY "${_thrift_proto_dir}" + RESULT_VARIABLE _rc) + if(NOT _rc EQUAL 0) + message(FATAL_ERROR "[GenThrift] thrift compile failed for ${_in} (rc=${_rc})") + endif() + endforeach() + # Defensive: remove any accidentally produced server skeleton. + file(GLOB _skeletons "${THRIFT_GEN_CPP_DIR}/*_server.skeleton.cpp") + if(_skeletons) + file(REMOVE ${_skeletons}) + endif() + file(TOUCH "${_thrift_stamp}") +endif() + +# Build-time regeneration: whenever the workspace .thrift files change, rerun +# the thrift compiler. The OUTPUT is the stamp; downstream targets that +# depend on iotdb_thrift_codegen will get the rebuild for free. +add_custom_command( + OUTPUT "${_thrift_stamp}" + COMMAND ${CMAKE_COMMAND} -E copy_if_different "${_common_thrift}" "${_thrift_proto_dir}/common.thrift" + COMMAND ${CMAKE_COMMAND} -E copy_if_different "${_client_thrift}" "${_thrift_proto_dir}/client.thrift" + COMMAND "${THRIFT_EXECUTABLE}" -r --gen cpp:no_skeleton -out "${THRIFT_GEN_CPP_DIR}" "${_thrift_proto_dir}/common.thrift" + COMMAND "${THRIFT_EXECUTABLE}" -r --gen cpp:no_skeleton -out "${THRIFT_GEN_CPP_DIR}" "${_thrift_proto_dir}/client.thrift" + COMMAND ${CMAKE_COMMAND} -E touch "${_thrift_stamp}" + DEPENDS "${_common_thrift}" "${_client_thrift}" iotdb_thrift_external + WORKING_DIRECTORY "${_thrift_proto_dir}" + COMMENT "Regenerating thrift C++ stubs" + VERBATIM) + +add_custom_target(iotdb_thrift_codegen DEPENDS "${_thrift_stamp}") + +# Glob results with CONFIGURE_DEPENDS so re-cmake picks up newly produced files. +file(GLOB THRIFT_GENERATED_SRCS CONFIGURE_DEPENDS "${THRIFT_GEN_CPP_DIR}/*.cpp") +file(GLOB THRIFT_GENERATED_HDRS CONFIGURE_DEPENDS "${THRIFT_GEN_CPP_DIR}/*.h") + +list(FILTER THRIFT_GENERATED_SRCS EXCLUDE REGEX ".*_server\\.skeleton\\.cpp$") + +message(STATUS "[GenThrift] generated ${THRIFT_GEN_CPP_DIR} with " + "$ cpp files") diff --git a/iotdb-client/client-cpp/cmake/iotdb-session-config.cmake.in b/iotdb-client/client-cpp/cmake/iotdb-session-config.cmake.in new file mode 100644 index 0000000000000..57207aa2fa71a --- /dev/null +++ b/iotdb-client/client-cpp/cmake/iotdb-session-config.cmake.in @@ -0,0 +1,56 @@ +# 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. + +get_filename_component(_IOTDB_SESSION_PACKAGE_ROOT "${CMAKE_CURRENT_LIST_DIR}/.." ABSOLUTE) + +if(UNIX) + include(CMakeFindDependencyMacro) + find_dependency(Threads) +endif() + +if(WIN32) + set(_IOTDB_SESSION_LIBRARY "${_IOTDB_SESSION_PACKAGE_ROOT}/lib/iotdb_session.lib") +elseif(APPLE) + set(_IOTDB_SESSION_LIBRARY "${_IOTDB_SESSION_PACKAGE_ROOT}/lib/libiotdb_session.dylib") +else() + set(_IOTDB_SESSION_LIBRARY "${_IOTDB_SESSION_PACKAGE_ROOT}/lib/libiotdb_session.so") +endif() + +if(NOT TARGET IoTDB::iotdb_session) + add_library(IoTDB::iotdb_session SHARED IMPORTED) + set_target_properties(IoTDB::iotdb_session PROPERTIES + INTERFACE_INCLUDE_DIRECTORIES "${_IOTDB_SESSION_PACKAGE_ROOT}/include") + set(_IOTDB_SESSION_CXX11_ABI_COMPILE_DEFINITION "@IOTDB_SESSION_CXX11_ABI_COMPILE_DEFINITION@") + if(_IOTDB_SESSION_CXX11_ABI_COMPILE_DEFINITION) + set_target_properties(IoTDB::iotdb_session PROPERTIES + INTERFACE_COMPILE_DEFINITIONS "${_IOTDB_SESSION_CXX11_ABI_COMPILE_DEFINITION}") + endif() + if(UNIX) + set_target_properties(IoTDB::iotdb_session PROPERTIES + INTERFACE_LINK_LIBRARIES Threads::Threads) + endif() + if(WIN32) + set_target_properties(IoTDB::iotdb_session PROPERTIES + IMPORTED_IMPLIB "${_IOTDB_SESSION_LIBRARY}" + IMPORTED_LOCATION "${_IOTDB_SESSION_PACKAGE_ROOT}/lib/iotdb_session.dll") + else() + set_target_properties(IoTDB::iotdb_session PROPERTIES + IMPORTED_LOCATION "${_IOTDB_SESSION_LIBRARY}") + endif() +endif() + +set(iotdb-session_VERSION "@IOTDB_SESSION_VERSION@") diff --git a/iotdb-client/client-cpp/cmake/iotdb-session.pc.in b/iotdb-client/client-cpp/cmake/iotdb-session.pc.in new file mode 100644 index 0000000000000..0cc6833e1ce46 --- /dev/null +++ b/iotdb-client/client-cpp/cmake/iotdb-session.pc.in @@ -0,0 +1,26 @@ +# 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. + +prefix=${pcfiledir}/.. +libdir=${prefix}/lib +includedir=${prefix}/include + +Name: iotdb-session-cpp +Description: Apache IoTDB C++ session client +Version: @IOTDB_SESSION_VERSION@ +Libs: -L${libdir} @IOTDB_SESSION_PC_LIBS@ +Cflags: -I${includedir} @IOTDB_SESSION_PC_CFLAGS@ diff --git a/iotdb-client/client-cpp/pom.xml b/iotdb-client/client-cpp/pom.xml index af57b78817191..418ef22fdf635 100644 --- a/iotdb-client/client-cpp/pom.xml +++ b/iotdb-client/client-cpp/pom.xml @@ -30,14 +30,38 @@ pom IoTDB: Client: Client for CPP C++ client - + https://github.com/catchorg/Catch2/releases/download/v2.13.7/catch.hpp Release - ${project.build.directory}/dependency/cmake/ - ${project.build.directory}/thrift/bin/${thrift.executable} + ${project.basedir} + ${project.build.directory}/build + ${project.build.directory}/install + ${project.basedir}/third-party + OFF + OFF + + ON ${ctest.skip.tests} + + + + OFF + + + ${os.classifier} + iotdb-session-cpp-${project.version}-${client.cpp.package.classifier} @@ -49,168 +73,30 @@ - - - org.apache.maven.plugins - maven-resources-plugin - - - copy-test-resources - - copy-resources - - validate - - ${project.build.directory}/build/test - - - ${project.basedir}/src/test - true - - - - - - - - org.apache.maven.plugins - maven-resources-plugin - - - copy-cpp-files - - copy-resources - - validate - - ${project.build.directory}/build/main/generated-sources-cpp - - - ${project.basedir}/src/main - - **/*.h - **/*.cpp - **/*.cc - - - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - get-thrift - - unpack - - generate-sources - - - - org.apache.iotdb.tools - iotdb-tools-thrift - ${iotdb-tools-thrift.version} - ${os.classifier} - zip - true - ${project.build.directory}/thrift - - - - - - com.googlecode.maven-download-plugin download-maven-plugin - get-catch2 wget - generate-test-resources + generate-resources ${catch2.url} false - ${project.build.directory}/build/test/catch2 - ${ctest.skip.tests} + ${project.build.directory}/test/catch2 + ${maven.test.skip} - com.coderplus.maven.plugins - copy-rename-maven-plugin - - - - copy-protocol-thrift-source - - copy - - - - - ../../iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift - ${project.build.directory}/protocols/client.thrift - - - ../../iotdb-protocol/thrift-commons/src/main/thrift/common.thrift - ${project.build.directory}/protocols/common.thrift - - - - - - - - copy-cmakelist-file - - copy - - compile - - - - ${project.basedir}/src/main/CMakeLists.txt - ${project.build.directory}/build/main/CMakeLists.txt - - - - - - - - org.apache.thrift - thrift-maven-plugin - - - generate-thrift-sources-cpp - - compile - - - generate-resources - - cpp:no_skeleton - ${thrift.exec.absolute.path} - ${project.build.directory}/protocols - ${project.build.directory}/build/main/generated-sources-cpp - - - - - - com.googlecode.cmake-maven-project + io.github.cmake-maven-plugin cmake-maven-plugin - + cmake-generate @@ -219,30 +105,25 @@ compile ${cmake.generator} - ${project.build.directory}/build/main - ${project.build.directory}/build/main - - - - - - - - cmake-generate-test - - generate - - test-compile - - ${cmake.generator} - ${project.build.directory}/build/test - ${project.build.directory}/build/test + ${cmake.source.dir} + ${cmake.project.dir} + + + + + + + + + + + - + cmake-compile @@ -251,24 +132,11 @@ compile ${cmake.build.type} - - ${project.build.directory}/build/main - - - - - cmake-compile-test - - compile - - test-compile - - ${cmake.build.type} - ${project.build.directory}/build/test + ${cmake.project.dir} + install - - + cmake-run-test @@ -276,37 +144,18 @@ integration-test - ${project.build.directory}/build/test - ${ctest.skip.tests} - - - - - - - org.apache.maven.plugins - maven-assembly-plugin - - - package-client-cpp - - single - - package - - ${project.artifactId}-${project.version} - - src/assembly/client-cpp.xml - + ${cmake.build.type} + ${cmake.project.dir}/test + ${maven.test.skip} + com.bazaarvoice.maven.plugins process-exec-maven-plugin - start-iotdb @@ -314,20 +163,18 @@ pre-integration-test - ${ctest.skip.tests} + ${maven.test.skip} iotdb-server false - - 15 - - ${project.build.directory}/build/test/test.log + 45 + ${cmake.project.dir}/test.log - + ${iotdb.shell.command} + ${iotdb.shell.option} ${project.basedir}/../../distribution/target/apache-iotdb-${project.version}-all-bin/apache-iotdb-${project.version}-all-bin/sbin/${iotdb.start.script} - stop-iotdb @@ -338,17 +185,85 @@ ${ctest.skip.tests} iotdb-server false - 5 - - ${project.build.directory}/build/test/test1.log + ${cmake.project.dir}/stop.log + ${iotdb.shell.command} + ${iotdb.shell.option} ${project.basedir}/../../distribution/target/apache-iotdb-${project.version}-all-bin/apache-iotdb-${project.version}-all-bin/sbin/${iotdb.stop.script} + + maven-resources-plugin + + + prepare-client-cpp-package-metadata + + copy-resources + + prepare-package + + ${project.build.directory}/package-metadata + + + ${project.basedir}/src/assembly/package-metadata + true + + + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + package-client-cpp + + single + + package + + iotdb-session-cpp-${project.version} + + src/assembly/client-cpp.xml + + + + + + + + net.nicoulaj.maven.plugins + checksum-maven-plugin + + + checksum-client-cpp-package + + files + + package + + + SHA-512 + + + + ${project.build.directory} + + ${client.cpp.package.name}.zip + + + + + + + @@ -362,9 +277,12 @@ + bash + -c start-standalone.sh stop-standalone.sh linux + linux-x86_64-glibc2.28 @@ -377,9 +295,12 @@ + bash + -c start-standalone.sh stop-standalone.sh linux + linux-aarch64-glibc2.28 @@ -391,9 +312,12 @@ + bash + -c start-standalone.sh stop-standalone.sh mac + macos-x86_64 @@ -405,9 +329,12 @@ + bash + -c start-standalone.sh stop-standalone.sh mac + macos-aarch64 @@ -418,33 +345,89 @@ + cmd + /c start-standalone.bat stop-standalone.bat win + windows-x86_64-msvc14.3 + - .skipTests + client-cpp-vs-x64 - - skipTests - + + windows + - com.googlecode.cmake-maven-project + io.github.cmake-maven-plugin cmake-maven-plugin - cmake-compile-test - none - + cmake-generate + + + + + + + .skipTests + + + skipTests + + + + OFF + + + + + spotless-cpp + + [11,) + + + + + com.diffplug.spotless + spotless-maven-plugin + 2.44.5 + + + + src/**/*.h + src/**/*.hpp + src/**/*.c + src/**/*.cc + src/**/*.cpp + + + test/catch2/** + + + ${clang.format.version} + + + + + + + + diff --git a/iotdb-client/client-cpp/src/assembly/client-cpp.xml b/iotdb-client/client-cpp/src/assembly/client-cpp.xml index 729fdadde235a..4764523cc3799 100644 --- a/iotdb-client/client-cpp/src/assembly/client-cpp.xml +++ b/iotdb-client/client-cpp/src/assembly/client-cpp.xml @@ -17,51 +17,78 @@ specific language governing permissions and limitations under the License. --> + - cpp-${os.classifier} + ${client.cpp.package.classifier} zip dir - false + true + ${client.cpp.package.name} - ${project.build.directory}/build/main/generated-sources-cpp + ${project.basedir}/../.. - **/*.h + LICENSE + NOTICE - include + ${file.separator} + + + ${project.basedir} + + README.md + + ${file.separator} + + + ${project.build.directory}/package-metadata + + third_party/DEPENDENCIES.md + + ${file.separator} + + + ${project.build.directory}/install + + VERSION + BUILD-INFO.txt + + ${file.separator} - ${project.build.directory}/thrift/include + ${project.build.directory}/install/include ** include - ${project.build.directory}/thrift/lib + ${project.build.directory}/install/lib ** lib - ${project.build.directory}/build/main + ${project.build.directory}/install/cmake - *.so - *.dylib - **/*.dll - **/*.lib + ** - lib + cmake - ${project.basedir}/../compile-tools/thrift/target/build/lib/Release + ${project.build.directory}/install/pkgconfig - **/*.lib + ** - lib + pkgconfig diff --git a/iotdb-client/client-cpp/src/assembly/package-metadata/third_party/DEPENDENCIES.md b/iotdb-client/client-cpp/src/assembly/package-metadata/third_party/DEPENDENCIES.md new file mode 100644 index 0000000000000..e921c7eb94833 --- /dev/null +++ b/iotdb-client/client-cpp/src/assembly/package-metadata/third_party/DEPENDENCIES.md @@ -0,0 +1,34 @@ + +# Third-party Dependencies + +The release library is built with the following third-party components. Some +components are linked into the produced IoTDB C++ session library; this file is +included for provenance. + +| Component | Version | License | +| --- | --- | --- | +| Apache Thrift | 0.21.0 | Apache License 2.0 | +| Boost | 1.60.0 on Linux/Windows, 1.84.0 on macOS by default | Boost Software License 1.0 | +| OpenSSL | 3.5.0 when `WITH_SSL=ON` | Apache License 2.0 | +| GNU m4 | 1.4.19 on Linux build bootstrap | GPL-3.0-or-later | +| GNU flex | 2.6.4 on Linux build bootstrap | BSD-style flex license | +| GNU bison | 3.8 on Linux build bootstrap | GPL-3.0-or-later | diff --git a/iotdb-client/client-cpp/src/include/AbstractSessionBuilder.h b/iotdb-client/client-cpp/src/include/AbstractSessionBuilder.h new file mode 100644 index 0000000000000..8d5ec5729c514 --- /dev/null +++ b/iotdb-client/client-cpp/src/include/AbstractSessionBuilder.h @@ -0,0 +1,65 @@ +/* + * 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. + */ + +#ifndef IOTDB_ABSTRACTSESSIONBUILDER_H +#define IOTDB_ABSTRACTSESSIONBUILDER_H + +#include +#include + +#include "SessionConfig.h" + +class AbstractSessionBuilder { +public: + static constexpr const char *DEFAULT_HOST = "localhost"; + static constexpr int DEFAULT_RPC_PORT = 6667; + static constexpr const char *DEFAULT_USERNAME = "root"; + static constexpr const char *DEFAULT_PASSWORD = "root"; + static constexpr int DEFAULT_FETCH_SIZE = iotdb::session::DEFAULT_FETCH_SIZE; + static constexpr int DEFAULT_CONNECT_TIMEOUT_MS = + iotdb::session::DEFAULT_CONNECT_TIMEOUT_MS; + static constexpr int DEFAULT_MAX_RETRIES = + iotdb::session::DEFAULT_MAX_RETRIES; + static constexpr int DEFAULT_RETRY_DELAY_MS = + iotdb::session::DEFAULT_RETRY_DELAY_MS; + static constexpr const char *DEFAULT_SQL_DIALECT = "tree"; + static constexpr bool DEFAULT_ENABLE_AUTO_FETCH = true; + static constexpr bool DEFAULT_ENABLE_REDIRECTIONS = true; + static constexpr bool DEFAULT_ENABLE_RPC_COMPRESSION = false; + + std::string host = DEFAULT_HOST; + int rpcPort = DEFAULT_RPC_PORT; + std::string username = DEFAULT_USERNAME; + std::string password = DEFAULT_PASSWORD; + std::string zoneId = ""; + int fetchSize = DEFAULT_FETCH_SIZE; + int connectTimeoutMs = DEFAULT_CONNECT_TIMEOUT_MS; + int maxRetries = DEFAULT_MAX_RETRIES; + int retryDelayMs = DEFAULT_RETRY_DELAY_MS; + std::string sqlDialect = DEFAULT_SQL_DIALECT; + std::string database = ""; + bool enableAutoFetch = DEFAULT_ENABLE_AUTO_FETCH; + bool enableRedirections = DEFAULT_ENABLE_REDIRECTIONS; + bool enableRPCCompression = DEFAULT_ENABLE_RPC_COMPRESSION; + std::vector nodeUrls; + bool useSSL = false; + std::string trustCertFilePath; +}; + +#endif // IOTDB_ABSTRACTSESSIONBUILDER_H \ No newline at end of file diff --git a/iotdb-client/client-cpp/src/include/Column.h b/iotdb-client/client-cpp/src/include/Column.h new file mode 100644 index 0000000000000..548e577c01a01 --- /dev/null +++ b/iotdb-client/client-cpp/src/include/Column.h @@ -0,0 +1,364 @@ +/** + * 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. + */ +#ifndef IOTDB_COLUMN_H +#define IOTDB_COLUMN_H + +#include +#include +#include +#include + +#include "ColumnDecoder.h" +#include "Common.h" + +enum class ColumnEncoding : uint8_t { + ByteArray, + Int32Array, + Int64Array, + BinaryArray, + Rle +}; + +class Binary { +public: + explicit Binary(std::vector data) : data_(std::move(data)) {} + + const std::vector &getData() const { return data_; } + + std::string getStringValue() const { return {data_.begin(), data_.end()}; } + +private: + std::vector data_; +}; + +const std::map> + kEncodingToDecoder = { + {ColumnEncoding::Int32Array, + std::make_shared()}, + {ColumnEncoding::Int64Array, + std::make_shared()}, + {ColumnEncoding::ByteArray, std::make_shared()}, + {ColumnEncoding::BinaryArray, + std::make_shared()}, + {ColumnEncoding::Rle, std::make_shared()}}; + +const std::map kByteToEncoding = { + {0, ColumnEncoding::ByteArray}, + {1, ColumnEncoding::Int32Array}, + {2, ColumnEncoding::Int64Array}, + {3, ColumnEncoding::BinaryArray}, + {4, ColumnEncoding::Rle}}; + +inline std::shared_ptr +getColumnDecoder(ColumnEncoding encoding) { + auto it = kEncodingToDecoder.find(encoding); + if (it == kEncodingToDecoder.end()) { + throw IoTDBException("Unsupported column encoding"); + } + return it->second; +} + +inline ColumnEncoding getColumnEncodingByByte(uint8_t b) { + auto it = kByteToEncoding.find(b); + if (it == kByteToEncoding.end()) { + throw IoTDBException("Invalid encoding value: " + std::to_string(b)); + } + return it->second; +} + +class Column { +public: + virtual ~Column() = default; + + virtual TSDataType::TSDataType getDataType() const = 0; + virtual ColumnEncoding getEncoding() const = 0; + + virtual bool getBoolean(int32_t position) const { + throw IoTDBException("Unsupported operation: getBoolean"); + } + + virtual int32_t getInt(int32_t position) const { + throw IoTDBException("Unsupported operation: getInt"); + } + + virtual int64_t getLong(int32_t position) const { + throw IoTDBException("Unsupported operation: getLong"); + } + + virtual float getFloat(int32_t position) const { + throw IoTDBException("Unsupported operation: getFloat"); + } + + virtual double getDouble(int32_t position) const { + throw IoTDBException("Unsupported operation: getDouble"); + } + + virtual std::shared_ptr getBinary(int32_t position) const { + throw IoTDBException("Unsupported operation: getBinary"); + } + + virtual std::vector getBooleans() const { + throw IoTDBException("Unsupported operation: getBooleans"); + } + + virtual std::vector getInts() const { + throw IoTDBException("Unsupported operation: getInts"); + } + + virtual std::vector getLongs() const { + throw IoTDBException("Unsupported operation: getLongs"); + } + + virtual std::vector getFloats() const { + throw IoTDBException("Unsupported operation: getFloats"); + } + + virtual std::vector getDoubles() const { + throw IoTDBException("Unsupported operation: getDoubles"); + } + + virtual std::vector> getBinaries() const { + throw IoTDBException("Unsupported operation: getBinaries"); + } + + virtual bool mayHaveNull() const = 0; + virtual bool isNull(int32_t position) const = 0; + virtual std::vector isNulls() const = 0; + + virtual int32_t getPositionCount() const = 0; +}; + +class TimeColumn : public Column { +public: + TimeColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &values); + + TSDataType::TSDataType getDataType() const override; + ColumnEncoding getEncoding() const override; + + int64_t getLong(int32_t position) const override; + + bool mayHaveNull() const override; + bool isNull(int32_t position) const override; + std::vector isNulls() const override; + + int32_t getPositionCount() const override; + + int64_t getStartTime() const; + int64_t getEndTime() const; + + const std::vector &getTimes() const; + std::vector getLongs() const override; + +private: + int32_t arrayOffset_; + int32_t positionCount_; + std::vector values_; +}; + +class BinaryColumn : public Column { +public: + BinaryColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector> &values); + + TSDataType::TSDataType getDataType() const override; + ColumnEncoding getEncoding() const override; + + std::shared_ptr getBinary(int32_t position) const override; + std::vector> getBinaries() const override; + + bool mayHaveNull() const override; + bool isNull(int32_t position) const override; + std::vector isNulls() const override; + + int32_t getPositionCount() const override; + +private: + int32_t arrayOffset_; + int32_t positionCount_; + std::vector valueIsNull_; + std::vector> values_; +}; + +class IntColumn : public Column { +public: + IntColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values); + + TSDataType::TSDataType getDataType() const override; + ColumnEncoding getEncoding() const override; + + int32_t getInt(int32_t position) const override; + int64_t getLong(int32_t position) const override; + float getFloat(int32_t position) const override; + double getDouble(int32_t position) const override; + std::vector getInts() const override; + + bool mayHaveNull() const override; + bool isNull(int32_t position) const override; + std::vector isNulls() const override; + + int32_t getPositionCount() const override; + +private: + int32_t arrayOffset_; + int32_t positionCount_; + std::vector valueNull_; + std::vector values_; +}; + +class FloatColumn : public Column { +public: + FloatColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values); + + TSDataType::TSDataType getDataType() const override; + ColumnEncoding getEncoding() const override; + + float getFloat(int32_t position) const override; + double getDouble(int32_t position) const override; + std::vector getFloats() const override; + + bool mayHaveNull() const override; + bool isNull(int32_t position) const override; + std::vector isNulls() const override; + + int32_t getPositionCount() const override; + +private: + int32_t arrayOffset_; + int32_t positionCount_; + std::vector valueIsNull_; + std::vector values_; +}; + +class LongColumn : public Column { +public: + LongColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values); + + TSDataType::TSDataType getDataType() const override; + ColumnEncoding getEncoding() const override; + + int64_t getLong(int32_t position) const override; + double getDouble(int32_t position) const override; + std::vector getLongs() const override; + + bool mayHaveNull() const override; + bool isNull(int32_t position) const override; + std::vector isNulls() const override; + + int32_t getPositionCount() const override; + +private: + int32_t arrayOffset_; + int32_t positionCount_; + std::vector valueIsNull_; + std::vector values_; +}; + +class DoubleColumn : public Column { +public: + DoubleColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values); + + TSDataType::TSDataType getDataType() const override; + ColumnEncoding getEncoding() const override; + + double getDouble(int32_t position) const override; + std::vector getDoubles() const override; + + bool mayHaveNull() const override; + bool isNull(int32_t position) const override; + std::vector isNulls() const override; + + int32_t getPositionCount() const override; + +private: + int32_t arrayOffset_; + int32_t positionCount_; + std::vector valueIsNull_; + std::vector values_; +}; + +class BooleanColumn : public Column { +public: + BooleanColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values); + + TSDataType::TSDataType getDataType() const override; + ColumnEncoding getEncoding() const override; + + bool getBoolean(int32_t position) const override; + std::vector getBooleans() const override; + + bool mayHaveNull() const override; + bool isNull(int32_t position) const override; + std::vector isNulls() const override; + + int32_t getPositionCount() const override; + +private: + int32_t arrayOffset_; + int32_t positionCount_; + std::vector valueIsNull_; + std::vector values_; +}; + +class RunLengthEncodedColumn : public Column { +public: + RunLengthEncodedColumn(std::shared_ptr value, int32_t positionCount); + + std::shared_ptr getValue() const; + + TSDataType::TSDataType getDataType() const override; + ColumnEncoding getEncoding() const override; + + bool getBoolean(int32_t position) const override; + int32_t getInt(int32_t position) const override; + int64_t getLong(int32_t position) const override; + float getFloat(int32_t position) const override; + double getDouble(int32_t position) const override; + std::shared_ptr getBinary(int32_t position) const override; + + std::vector getBooleans() const override; + std::vector getInts() const override; + std::vector getLongs() const override; + std::vector getFloats() const override; + std::vector getDoubles() const override; + std::vector> getBinaries() const override; + + bool mayHaveNull() const override; + bool isNull(int32_t position) const override; + std::vector isNulls() const override; + + int32_t getPositionCount() const override; + +private: + std::shared_ptr value_; + int32_t positionCount_; +}; + +#endif diff --git a/iotdb-client/client-cpp/src/include/ColumnDecoder.h b/iotdb-client/client-cpp/src/include/ColumnDecoder.h new file mode 100644 index 0000000000000..6affa72060ef1 --- /dev/null +++ b/iotdb-client/client-cpp/src/include/ColumnDecoder.h @@ -0,0 +1,83 @@ +/** + * 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. + */ +#ifndef IOTDB_COLUMN_DECODER_H +#define IOTDB_COLUMN_DECODER_H + +#include +#include + +#include "Common.h" + +class Column; + +class ColumnDecoder { +public: + virtual ~ColumnDecoder() = default; + virtual std::unique_ptr readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) = 0; +}; + +std::vector deserializeNullIndicators(MyStringBuffer &buffer, + int32_t positionCount); +std::vector deserializeBooleanArray(MyStringBuffer &buffer, int32_t size); + +class BaseColumnDecoder : public ColumnDecoder { +public: + std::unique_ptr readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) override; +}; + +class Int32ArrayColumnDecoder : public BaseColumnDecoder { +public: + std::unique_ptr readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) override; +}; + +class Int64ArrayColumnDecoder : public BaseColumnDecoder { +public: + std::unique_ptr readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) override; +}; + +class ByteArrayColumnDecoder : public BaseColumnDecoder { +public: + std::unique_ptr readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) override; +}; + +class BinaryArrayColumnDecoder : public BaseColumnDecoder { +public: + std::unique_ptr readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) override; +}; + +class RunLengthColumnDecoder : public BaseColumnDecoder { +public: + std::unique_ptr readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) override; +}; + +#endif diff --git a/iotdb-client/client-cpp/src/include/Common.h b/iotdb-client/client-cpp/src/include/Common.h new file mode 100644 index 0000000000000..c203edf2de69f --- /dev/null +++ b/iotdb-client/client-cpp/src/include/Common.h @@ -0,0 +1,428 @@ +/** + * 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. + */ +#ifndef IOTDB_COMMON_H +#define IOTDB_COMMON_H + +#include +#include +#include +#include +#include +#include + +#include "Date.h" +#include "Endpoint.h" +#include "Export.h" +#include "Optional.h" +#include "Status.h" + +std::string getTimePrecision(int32_t timeFactor); + +std::string formatDatetime(const std::string &format, + const std::string &precision, int64_t timestamp, + const std::string &zoneId); + +std::tm convertToTimestamp(int64_t value, int32_t timeFactor); +std::tm int32ToDate(int32_t value); + +namespace Version { +enum Version { V_0_12, V_0_13, V_1_0 }; +} + +namespace CompressionType { +enum CompressionType { + UNCOMPRESSED = (char)0, + SNAPPY = (char)1, + GZIP = (char)2, + LZO = (char)3, + SDT = (char)4, + PAA = (char)5, + PLA = (char)6, + LZ4 = (char)7, + ZSTD = (char)8, + LZMA2 = (char)9, +}; +} + +namespace TSDataType { +enum TSDataType { + BOOLEAN = (char)0, + INT32 = (char)1, + INT64 = (char)2, + FLOAT = (char)3, + DOUBLE = (char)4, + TEXT = (char)5, + VECTOR = (char)6, + UNKNOWN = (char)7, + TIMESTAMP = (char)8, + DATE = (char)9, + BLOB = (char)10, + STRING = (char)11, + OBJECT = (char)12 +}; +} + +TSDataType::TSDataType getDataTypeByStr(const std::string &typeStr); + +namespace TSEncoding { +enum TSEncoding { + PLAIN = (char)0, + DICTIONARY = (char)1, + RLE = (char)2, + DIFF = (char)3, + TS_2DIFF = (char)4, + BITMAP = (char)5, + GORILLA_V1 = (char)6, + REGULAR = (char)7, + GORILLA = (char)8, + ZIGZAG = (char)9, + FREQ = (char)10, + INVALID_ENCODING = (char)255 +}; +} + +namespace TSStatusCode { +enum TSStatusCode { + SUCCESS_STATUS = 200, + + INCOMPATIBLE_VERSION = 201, + CONFIGURATION_ERROR = 202, + START_UP_ERROR = 203, + SHUT_DOWN_ERROR = 204, + + UNSUPPORTED_OPERATION = 300, + EXECUTE_STATEMENT_ERROR = 301, + MULTIPLE_ERROR = 302, + ILLEGAL_PARAMETER = 303, + OVERLAP_WITH_EXISTING_TASK = 304, + INTERNAL_SERVER_ERROR = 305, + + REDIRECTION_RECOMMEND = 400, + + DATABASE_NOT_EXIST = 500, + DATABASE_ALREADY_EXISTS = 501, + SERIES_OVERFLOW = 502, + TIMESERIES_ALREADY_EXIST = 503, + TIMESERIES_IN_BLACK_LIST = 504, + ALIAS_ALREADY_EXIST = 505, + PATH_ALREADY_EXIST = 506, + METADATA_ERROR = 507, + PATH_NOT_EXIST = 508, + ILLEGAL_PATH = 509, + CREATE_TEMPLATE_ERROR = 510, + DUPLICATED_TEMPLATE = 511, + UNDEFINED_TEMPLATE = 512, + TEMPLATE_NOT_SET = 513, + DIFFERENT_TEMPLATE = 514, + TEMPLATE_IS_IN_USE = 515, + TEMPLATE_INCOMPATIBLE = 516, + SEGMENT_NOT_FOUND = 517, + PAGE_OUT_OF_SPACE = 518, + RECORD_DUPLICATED = 519, + SEGMENT_OUT_OF_SPACE = 520, + PBTREE_FILE_NOT_EXISTS = 521, + OVERSIZE_RECORD = 522, + PBTREE_FILE_REDO_LOG_BROKEN = 523, + TEMPLATE_NOT_ACTIVATED = 524, + + SYSTEM_READ_ONLY = 600, + STORAGE_ENGINE_ERROR = 601, + STORAGE_ENGINE_NOT_READY = 602, + + PLAN_FAILED_NETWORK_PARTITION = 721 +}; +} + +class Field { +public: + TSDataType::TSDataType dataType = TSDataType::UNKNOWN; + Optional boolV; + Optional intV; + Optional dateV; + Optional longV; + Optional floatV; + Optional doubleV; + Optional stringV; + + explicit Field(TSDataType::TSDataType a) { dataType = a; } + + Field() = default; + + bool isNull() const { + switch (dataType) { + case TSDataType::BOOLEAN: + return !boolV.is_initialized(); + case TSDataType::INT32: + return !intV.is_initialized(); + case TSDataType::INT64: + case TSDataType::TIMESTAMP: + return !longV.is_initialized(); + case TSDataType::FLOAT: + return !floatV.is_initialized(); + case TSDataType::DOUBLE: + return !doubleV.is_initialized(); + case TSDataType::TEXT: + case TSDataType::STRING: + case TSDataType::BLOB: + return !stringV.is_initialized(); + case TSDataType::DATE: + return !dateV.is_initialized(); + default: + return true; + } + } +}; + +enum class ColumnCategory { TAG, FIELD, ATTRIBUTE }; + +class MyStringBuffer { +public: + MyStringBuffer(); + explicit MyStringBuffer(const std::string &str); + + void reserve(size_t n); + void clear(); + bool hasRemaining(); + int getInt(); + IoTDBDate getDate(); + int64_t getInt64(); + float getFloat(); + double getDouble(); + char getChar(); + bool getBool(); + std::string getString(); + + void putInt(int ins); + void putDate(IoTDBDate date); + void putInt64(int64_t ins); + void putFloat(float ins); + void putDouble(double ins); + void putChar(char ins); + void putBool(bool ins); + void putString(const std::string &ins); + void concat(const std::string &ins); + +public: + std::string str; + size_t pos; + +private: + void checkBigEndian(); + const char *getOrderedByte(size_t len); + void putOrderedByte(char *buf, int len); + +private: + bool isBigEndian{}; + char numericBuf[8]{}; +}; + +class BitMap { +public: + explicit BitMap(size_t size = 0); + void resize(size_t size); + bool mark(size_t position); + bool unmark(size_t position); + void markAll(); + void reset(); + bool isMarked(size_t position) const; + bool isAllUnmarked() const; + bool isAllMarked() const; + const std::vector &getByteArray() const; + size_t getSize() const; + +private: + size_t size; + std::vector bits; +}; + +class IoTDBException : public std::exception { +public: + IoTDBException() = default; + + explicit IoTDBException(const std::string &m) : message(m) {} + + explicit IoTDBException(const char *m) : message(m) {} + + virtual const char *what() const noexcept override { return message.c_str(); } + +private: + std::string message; +}; + +std::string extractExceptionMessage(const std::exception &exception); +std::string extractExceptionMessage(const std::exception_ptr &exceptionPtr); + +class DateTimeParseException : public IoTDBException { +private: + std::string parsedString; + int errorIndex; + +public: + explicit DateTimeParseException(const std::string &message, + std::string parsedData, int errorIndex) + : IoTDBException(message), parsedString(std::move(parsedData)), + errorIndex(errorIndex) {} + + explicit DateTimeParseException(const std::string &message, + std::string parsedData, int errorIndex, + const std::exception &cause) + : IoTDBException(message + " [Caused by: " + cause.what() + "]"), + parsedString(std::move(parsedData)), errorIndex(errorIndex) {} + + const std::string &getParsedString() const noexcept { return parsedString; } + + int getErrorIndex() const noexcept { return errorIndex; } + + const char *what() const noexcept override { + static std::string fullMsg; + fullMsg = std::string(IoTDBException::what()) + + "\nParsed data: " + parsedString + + "\nError index: " + std::to_string(errorIndex); + return fullMsg.c_str(); + } +}; + +class IoTDBConnectionException : public IoTDBException { +public: + IoTDBConnectionException() {} + + explicit IoTDBConnectionException(const char *m) : IoTDBException(m) {} + + explicit IoTDBConnectionException(const std::string &m) : IoTDBException(m) {} +}; + +class ExecutionException : public IoTDBException { +public: + ExecutionException() {} + + explicit ExecutionException(const char *m) : IoTDBException(m) {} + + explicit ExecutionException(const std::string &m) : IoTDBException(m) {} + + explicit ExecutionException(const std::string &m, const Status &tsStatus) + : IoTDBException(m), status(tsStatus) {} + + Status status; +}; + +class BatchExecutionException : public IoTDBException { +public: + BatchExecutionException() {} + + explicit BatchExecutionException(const char *m) : IoTDBException(m) {} + + explicit BatchExecutionException(const std::string &m) : IoTDBException(m) {} + + explicit BatchExecutionException(const std::vector &statusList) + : statusList(statusList) {} + + BatchExecutionException(const std::string &m, + const std::vector &statusList) + : IoTDBException(m), statusList(statusList) {} + + std::vector statusList; +}; + +class RedirectException : public IoTDBException { +public: + RedirectException() {} + + explicit RedirectException(const char *m) : IoTDBException(m) {} + + explicit RedirectException(const std::string &m) : IoTDBException(m) {} + + RedirectException(const std::string &m, const Endpoint &endPoint) + : IoTDBException(m), endPoint(endPoint) {} + + RedirectException(const std::string &m, + const std::map &deviceEndPointMap) + : IoTDBException(m), deviceEndPointMap(deviceEndPointMap) {} + + RedirectException(const std::string &m, + const std::vector &endPointList) + : IoTDBException(m), endPointList(endPointList) {} + + Endpoint endPoint; + std::map deviceEndPointMap; + std::vector endPointList; +}; + +class UnSupportedDataTypeException : public IoTDBException { +public: + UnSupportedDataTypeException() {} + + explicit UnSupportedDataTypeException(const char *m) : IoTDBException(m) {} + + explicit UnSupportedDataTypeException(const std::string &m) + : IoTDBException("UnSupported dataType: " + m) {} +}; + +class SchemaNotFoundException : public IoTDBException { +public: + SchemaNotFoundException() {} + + explicit SchemaNotFoundException(const char *m) : IoTDBException(m) {} + + explicit SchemaNotFoundException(const std::string &m) : IoTDBException(m) {} +}; + +class StatementExecutionException : public IoTDBException { +public: + StatementExecutionException() {} + + explicit StatementExecutionException(const char *m) : IoTDBException(m) {} + + explicit StatementExecutionException(const std::string &m) + : IoTDBException(m) {} +}; + +enum LogLevelType { LEVEL_DEBUG = 0, LEVEL_INFO, LEVEL_WARN, LEVEL_ERROR }; + +extern IOTDB_SESSION_API LogLevelType LOG_LEVEL; + +#define log_debug(fmt, ...) \ + do { \ + if (LOG_LEVEL <= LEVEL_DEBUG) { \ + std::string s = std::string("[DEBUG] %s:%d (%s) - ") + fmt + "\n"; \ + printf(s.c_str(), __FILE__, __LINE__, __FUNCTION__, ##__VA_ARGS__); \ + } \ + } while (0) +#define log_info(fmt, ...) \ + do { \ + if (LOG_LEVEL <= LEVEL_INFO) { \ + std::string s = std::string("[INFO] %s:%d (%s) - ") + fmt + "\n"; \ + printf(s.c_str(), __FILE__, __LINE__, __FUNCTION__, ##__VA_ARGS__); \ + } \ + } while (0) +#define log_warn(fmt, ...) \ + do { \ + if (LOG_LEVEL <= LEVEL_WARN) { \ + std::string s = std::string("[WARN] %s:%d (%s) - ") + fmt + "\n"; \ + printf(s.c_str(), __FILE__, __LINE__, __FUNCTION__, ##__VA_ARGS__); \ + } \ + } while (0) +#define log_error(fmt, ...) \ + do { \ + if (LOG_LEVEL <= LEVEL_ERROR) { \ + std::string s = std::string("[ERROR] %s:%d (%s) - ") + fmt + "\n"; \ + printf(s.c_str(), __FILE__, __LINE__, __FUNCTION__, ##__VA_ARGS__); \ + } \ + } while (0) + +#endif diff --git a/iotdb-client/client-cpp/src/include/Date.h b/iotdb-client/client-cpp/src/include/Date.h new file mode 100644 index 0000000000000..91fe286f0c4d0 --- /dev/null +++ b/iotdb-client/client-cpp/src/include/Date.h @@ -0,0 +1,65 @@ +/** + * 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. + */ +#ifndef IOTDB_DATE_H +#define IOTDB_DATE_H + +#include +#include + +constexpr int32_t EMPTY_DATE_INT = 10000101; + +class IoTDBDate { +public: + IoTDBDate() : valid_(false), year_(0), month_(0), day_(0) {} + + IoTDBDate(int year, int month, int day) + : valid_(true), year_(year), month_(month), day_(day) {} + + static IoTDBDate notADate() { return IoTDBDate(); } + + bool is_not_a_date() const { return !valid_; } + + int year() const { return year_; } + + int month() const { return month_; } + + int day() const { return day_; } + + std::string toIsoExtendedString() const; + + friend bool operator==(const IoTDBDate &lhs, const IoTDBDate &rhs) { + return lhs.valid_ == rhs.valid_ && lhs.year_ == rhs.year_ && + lhs.month_ == rhs.month_ && lhs.day_ == rhs.day_; + } + + friend bool operator!=(const IoTDBDate &lhs, const IoTDBDate &rhs) { + return !(lhs == rhs); + } + +private: + bool valid_; + int year_; + int month_; + int day_; +}; + +int32_t parseDateExpressionToInt(const IoTDBDate &date); +IoTDBDate parseIntToDate(int32_t dateInt); + +#endif diff --git a/iotdb-client/client-cpp/src/main/AbstractSessionBuilder.h b/iotdb-client/client-cpp/src/include/Endpoint.h similarity index 61% rename from iotdb-client/client-cpp/src/main/AbstractSessionBuilder.h rename to iotdb-client/client-cpp/src/include/Endpoint.h index 441e3a41f4fac..9c43a6f48c1b8 100644 --- a/iotdb-client/client-cpp/src/main/AbstractSessionBuilder.h +++ b/iotdb-client/client-cpp/src/include/Endpoint.h @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -16,25 +16,29 @@ * specific language governing permissions and limitations * under the License. */ +#ifndef IOTDB_ENDPOINT_H +#define IOTDB_ENDPOINT_H -#ifndef IOTDB_ABSTRACTSESSIONBUILDER_H -#define IOTDB_ABSTRACTSESSIONBUILDER_H - +#include +#include #include -class AbstractSessionBuilder { -public: - std::string host = "localhost"; - int rpcPort = 6667; - std::string username = "root"; - std::string password = "root"; - std::string zoneId = ""; - int fetchSize = 10000; - std::string sqlDialect = "tree"; - std::string database = ""; - bool enableAutoFetch = true; - bool enableRedirections = true; - bool enableRPCCompression = false; +struct Endpoint { + std::string host; + int32_t port = 0; +}; + +struct EndpointHash { + size_t operator()(const Endpoint &endpoint) const { + return std::hash()(endpoint.host) ^ + std::hash()(endpoint.port); + } +}; + +struct EndpointEqual { + bool operator()(const Endpoint &lhs, const Endpoint &rhs) const { + return lhs.host == rhs.host && lhs.port == rhs.port; + } }; -#endif // IOTDB_ABSTRACTSESSIONBUILDER_H \ No newline at end of file +#endif diff --git a/iotdb-client/client-cpp/src/include/Export.h b/iotdb-client/client-cpp/src/include/Export.h new file mode 100644 index 0000000000000..71262d0cffb77 --- /dev/null +++ b/iotdb-client/client-cpp/src/include/Export.h @@ -0,0 +1,30 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +#ifndef IOTDB_EXPORT_H +#define IOTDB_EXPORT_H + +#if defined(_WIN32) && defined(IOTDB_BUILDING_SHARED) +#define IOTDB_SESSION_API __declspec(dllexport) +#elif defined(_WIN32) +#define IOTDB_SESSION_API __declspec(dllimport) +#else +#define IOTDB_SESSION_API +#endif + +#endif // IOTDB_EXPORT_H diff --git a/iotdb-client/client-cpp/src/include/Optional.h b/iotdb-client/client-cpp/src/include/Optional.h new file mode 100644 index 0000000000000..f8c1d63af8757 --- /dev/null +++ b/iotdb-client/client-cpp/src/include/Optional.h @@ -0,0 +1,57 @@ +/** + * 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. + */ +#ifndef IOTDB_OPTIONAL_H +#define IOTDB_OPTIONAL_H + +#include + +template class Optional { +public: + Optional() : hasValue_(false) {} + + Optional(const T &value) : hasValue_(true), value_(value) {} + + Optional(T &&value) : hasValue_(true), value_(std::move(value)) {} + + static Optional of(const T &value) { return Optional(value); } + + static Optional of(T &&value) { return Optional(std::move(value)); } + + static Optional none() { return Optional(); } + + bool is_initialized() const { return hasValue_; } + + bool has_value() const { return hasValue_; } + + const T &value() const { return value_; } + + T &value() { return value_; } + + const T &get() const { return value_; } + + T &get() { return value_; } + + explicit operator bool() const { return hasValue_; } + +private: + bool hasValue_; + T value_{}; +}; + +#endif diff --git a/iotdb-client/client-cpp/src/include/Session.h b/iotdb-client/client-cpp/src/include/Session.h new file mode 100644 index 0000000000000..af33e343bb7af --- /dev/null +++ b/iotdb-client/client-cpp/src/include/Session.h @@ -0,0 +1,828 @@ +/** + * 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. + */ +#ifndef IOTDB_SESSION_H +#define IOTDB_SESSION_H + +#include "AbstractSessionBuilder.h" +#include "Common.h" +#include "Date.h" +#include "SessionDataSet.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +//== For compatible with Windows OS == +#ifndef LONG_LONG_MIN +#define LONG_LONG_MIN 0x8000000000000000 +#endif + +template +void safe_cast(const T &value, Target &target) { + /* + Target Allowed Source Types + BOOLEAN BOOLEAN + INT32 INT32 + INT64 INT32 INT64 + FLOAT INT32 FLOAT + DOUBLE INT32 INT64 FLOAT DOUBLE + TEXT TEXT + */ + if (std::is_same::value) { + target = *(Target *)&value; + } else if (std::is_same::value && + std::is_array::value && + std::is_same::type>::value) { + std::string tmp((const char *)&value); + target = *(Target *)&tmp; + } else if (std::is_same::value && + std::is_same::value) { + int64_t tmp = *(int32_t *)&value; + target = *(Target *)&tmp; + } else if (std::is_same::value && + std::is_same::value) { + float tmp = *(int32_t *)&value; + target = *(Target *)&tmp; + } else if (std::is_same::value && + std::is_same::value) { + double tmp = *(int32_t *)&value; + target = *(Target *)&tmp; + } else if (std::is_same::value && + std::is_same::value) { + double tmp = *(int64_t *)&value; + target = *(Target *)&tmp; + } else if (std::is_same::value && + std::is_same::value) { + double tmp = *(float *)&value; + target = *(Target *)&tmp; + } else { + throw UnSupportedDataTypeException("Error: Parameter type " + + std::string(typeid(T).name()) + + " cannot be converted to DataType" + + std::string(typeid(Target).name())); + } +} + +/* + * A tablet data of one device, the tablet contains multiple measurements of + * this device that share the same time column. + * + * for example: device root.sg1.d1 + * + * time, m1, m2, m3 + * 1, 1, 2, 3 + * 2, 1, 2, 3 + * 3, 1, 2, 3 + * + * Notice: The tablet should not have empty cell + * + */ +class Tablet { +private: + static const int DEFAULT_ROW_SIZE = 1024; + + void createColumns(); + void deleteColumns(); + +public: + std::string deviceId; // deviceId of this tablet + std::vector> schemas; + // the list of measurement schemas for creating the tablet + std::map + schemaNameIndex; // the map of schema name to index + std::vector + columnTypes; // column category (TAG/FIELD/ATTRIBUTE); tree clients use + // FIELD only + std::vector timestamps; // timestamps in this tablet + std::vector values; // each object is a primitive type array, which + // represents values of one measurement + std::vector bitMaps; // each bitmap represents the existence of each + // value in the current column + size_t rowSize; // the number of rows to include in this tablet + size_t maxRowNumber; // the maximum number of rows for this tablet + bool isAligned; // whether this tablet store data of aligned timeseries or not + + Tablet() = default; + + /** + * Return a tablet with default specified row number. This is the standard + * constructor (all Tablet should be the same size). + * + * @param deviceId the name of the device specified to be written in + * @param timeseries the list of measurement schemas for creating the tablet + */ + Tablet(const std::string &deviceId, + const std::vector> + ×eries) + : Tablet(deviceId, timeseries, DEFAULT_ROW_SIZE) {} + + /** + * Return a tablet with the specified number of rows (maxBatchSize). Only + * call this constructor directly for testing purposes. Tablet should normally + * always be default size. + * + * @param deviceId the name of the device specified to be written in + * @param schemas the list of measurement schemas for creating the row + * batch + * @param columnTypes column category per schema (tree clients use FIELD only) + * @param maxRowNumber the maximum number of rows for this tablet + */ + Tablet(const std::string &deviceId, + const std::vector> + &schemas, + int maxRowNumber) + : Tablet( + deviceId, schemas, + std::vector(schemas.size(), ColumnCategory::FIELD), + maxRowNumber) {} + + Tablet(const std::string &deviceId, + const std::vector> + &schemas, + const std::vector columnTypes, size_t maxRowNumber, + bool _isAligned = false) + : deviceId(deviceId), schemas(schemas), columnTypes(columnTypes), + maxRowNumber(maxRowNumber), isAligned(_isAligned) { + // create timestamp column + timestamps.resize(maxRowNumber); + // create value columns + values.resize(schemas.size()); + createColumns(); + // create bitMaps + bitMaps.resize(schemas.size()); + for (size_t i = 0; i < schemas.size(); i++) { + bitMaps[i].resize(maxRowNumber); + } + // create schemaNameIndex + for (size_t i = 0; i < schemas.size(); i++) { + schemaNameIndex[schemas[i].first] = i; + } + this->rowSize = 0; + } + + Tablet(const Tablet &other) + : deviceId(other.deviceId), schemas(other.schemas), + schemaNameIndex(other.schemaNameIndex), columnTypes(other.columnTypes), + timestamps(other.timestamps), maxRowNumber(other.maxRowNumber), + bitMaps(other.bitMaps), rowSize(other.rowSize), + isAligned(other.isAligned) { + values.resize(other.values.size()); + for (size_t i = 0; i < other.values.size(); ++i) { + if (!other.values[i]) + continue; + TSDataType::TSDataType type = schemas[i].second; + deepCopyTabletColValue(&(other.values[i]), &values[i], type, + maxRowNumber); + } + } + + Tablet &operator=(const Tablet &other) { + if (this != &other) { + deleteColumns(); + deviceId = other.deviceId; + schemas = other.schemas; + schemaNameIndex = other.schemaNameIndex; + columnTypes = other.columnTypes; + timestamps = other.timestamps; + maxRowNumber = other.maxRowNumber; + rowSize = other.rowSize; + isAligned = other.isAligned; + bitMaps = other.bitMaps; + values.resize(other.values.size()); + for (size_t i = 0; i < other.values.size(); ++i) { + if (!other.values[i]) + continue; + TSDataType::TSDataType type = schemas[i].second; + deepCopyTabletColValue(&(other.values[i]), &values[i], type, + maxRowNumber); + } + } + return *this; + } + + ~Tablet() { + try { + deleteColumns(); + } catch (std::exception &e) { + log_debug(std::string("Tablet::~Tablet(), ") + e.what()); + } + } + + void addTimestamp(size_t rowIndex, int64_t timestamp) { + timestamps[rowIndex] = timestamp; + rowSize = (std::max)(rowSize, rowIndex + 1); + } + + static void deepCopyTabletColValue(void *const *srcPtr, void **destPtr, + TSDataType::TSDataType type, + int maxRowNumber); + + template + void addValue(size_t schemaId, size_t rowIndex, const T &value) { + if (schemaId >= schemas.size()) { + char tmpStr[100]; + snprintf(tmpStr, sizeof(tmpStr), + "Tablet::addValue(), schemaId >= schemas.size(). schemaId=%ld, " + "schemas.size()=%ld.", + (long)schemaId, (long)schemas.size()); + throw std::out_of_range(tmpStr); + } + + if (rowIndex >= rowSize) { + char tmpStr[100]; + snprintf(tmpStr, sizeof(tmpStr), + "Tablet::addValue(), rowIndex >= rowSize. rowIndex=%ld, " + "rowSize.size()=%ld.", + (long)rowIndex, (long)rowSize); + throw std::out_of_range(tmpStr); + } + + TSDataType::TSDataType dataType = schemas[schemaId].second; + switch (dataType) { + case TSDataType::BOOLEAN: { + safe_cast(value, ((bool *)values[schemaId])[rowIndex]); + break; + } + case TSDataType::INT32: { + safe_cast(value, ((int *)values[schemaId])[rowIndex]); + break; + } + case TSDataType::DATE: { + safe_cast(value, ((IoTDBDate *)values[schemaId])[rowIndex]); + break; + } + case TSDataType::TIMESTAMP: + case TSDataType::INT64: { + safe_cast(value, ((int64_t *)values[schemaId])[rowIndex]); + break; + } + case TSDataType::FLOAT: { + safe_cast(value, ((float *)values[schemaId])[rowIndex]); + break; + } + case TSDataType::DOUBLE: { + safe_cast(value, ((double *)values[schemaId])[rowIndex]); + break; + } + case TSDataType::BLOB: + case TSDataType::STRING: + case TSDataType::TEXT: { + safe_cast(value, + ((std::string *)values[schemaId])[rowIndex]); + break; + } + default: + throw UnSupportedDataTypeException(std::string("Data type ") + + std::to_string(dataType) + + " is not supported."); + } + } + + // Add a Binary value with extra metadata: [isEOF (1 byte)] + [offset (8 + // bytes)] + [actual content] + void addValue(size_t schemaId, size_t rowIndex, bool isEOF, int64_t offset, + const std::vector &content) { + // Check schemaId bounds + if (schemaId >= schemas.size()) { + char tmpStr[100]; + sprintf(tmpStr, + "Tablet::addBinaryValueWithMeta(), schemaId >= schemas.size(). " + "schemaId=%ld, " + "schemas.size()=%ld.", + schemaId, schemas.size()); + throw std::out_of_range(tmpStr); + } + + // Check rowIndex bounds + if (rowIndex >= rowSize) { + char tmpStr[100]; + sprintf(tmpStr, + "Tablet::addBinaryValueWithMeta(), rowIndex >= rowSize. " + "rowIndex=%ld, rowSize=%ld.", + rowIndex, rowSize); + throw std::out_of_range(tmpStr); + } + + TSDataType::TSDataType dataType = schemas[schemaId].second; + if (dataType != TSDataType::OBJECT) { + throw std::invalid_argument("The data type of schemaId " + + std::to_string(schemaId) + " is not OBJECT."); + } + + // Create a byte array of size [1 (isEOF) + 8 (offset) + content size] + std::vector val(content.size() + 9); + + // Write the isEOF flag (1 byte) + val[0] = isEOF ? 1 : 0; + + // Write the 8-byte offset in big-endian order + for (int i = 0; i < 8; ++i) { + val[1 + i] = static_cast((offset >> (56 - i * 8)) & 0xFF); + } + + // Append the content bytes + std::copy(content.begin(), content.end(), val.begin() + 9); + + // Cast the value array and assign the Binary data (stored as string) + std::string valEncoded = + std::string(reinterpret_cast(val.data()), val.size()); + safe_cast( + valEncoded, ((std::string *)values[schemaId])[rowIndex]); + } + + void addValue(const std::string &schemaName, size_t rowIndex, bool isEOF, + int64_t offset, const std::vector &content) { + if (schemaNameIndex.find(schemaName) == schemaNameIndex.end()) { + throw SchemaNotFoundException(std::string("Schema ") + schemaName + + " not found."); + } + size_t schemaId = schemaNameIndex[schemaName]; + addValue(schemaId, rowIndex, isEOF, offset, content); + } + + template + void addValue(const std::string &schemaName, size_t rowIndex, + const T &value) { + if (schemaNameIndex.find(schemaName) == schemaNameIndex.end()) { + throw SchemaNotFoundException(std::string("Schema ") + schemaName + + " not found."); + } + size_t schemaId = schemaNameIndex[schemaName]; + addValue(schemaId, rowIndex, value); + } + + void *getValue(size_t schemaId, size_t rowIndex, + TSDataType::TSDataType dataType) { + if (schemaId >= schemas.size()) { + throw std::out_of_range("Tablet::getValue schemaId out of range: " + + std::to_string(schemaId)); + } + if (rowIndex >= rowSize) { + throw std::out_of_range("Tablet::getValue rowIndex out of range: " + + std::to_string(rowIndex)); + } + + switch (dataType) { + case TSDataType::BOOLEAN: + return &(reinterpret_cast(values[schemaId])[rowIndex]); + case TSDataType::INT32: + return &(reinterpret_cast(values[schemaId])[rowIndex]); + case TSDataType::DATE: + return &(reinterpret_cast(values[schemaId])[rowIndex]); + case TSDataType::TIMESTAMP: + case TSDataType::INT64: + return &(reinterpret_cast(values[schemaId])[rowIndex]); + case TSDataType::FLOAT: + return &(reinterpret_cast(values[schemaId])[rowIndex]); + case TSDataType::DOUBLE: + return &(reinterpret_cast(values[schemaId])[rowIndex]); + case TSDataType::BLOB: + case TSDataType::STRING: + case TSDataType::OBJECT: + case TSDataType::TEXT: + return &(reinterpret_cast(values[schemaId])[rowIndex]); + default: + throw UnSupportedDataTypeException("Unsupported data type: " + + std::to_string(dataType)); + } + } + + std::vector> + getSchemas() const { + return schemas; + } + + void reset(); // Reset Tablet to the default state - set the rowSize to 0 + + size_t getTimeBytesSize(); + + size_t getValueByteSize(); // total byte size that values occupies + + void setAligned(bool isAligned); +}; + +class SessionUtils { +public: + static std::string getTime(const Tablet &tablet); + + static std::string getValue(const Tablet &tablet); +}; + +class TemplateNode { +public: + explicit TemplateNode(const std::string &name) : name_(name) {} + + const std::string &getName() const { return name_; } + + virtual const std::unordered_map> & + getChildren() const { + throw BatchExecutionException("Should call exact sub class!"); + } + + virtual bool isMeasurement() = 0; + + virtual bool isAligned() { + throw BatchExecutionException("Should call exact sub class!"); + } + + virtual std::string serialize() const { + throw BatchExecutionException("Should call exact sub class!"); + } + +private: + std::string name_; +}; + +class MeasurementNode : public TemplateNode { +public: + MeasurementNode(const std::string &name_, TSDataType::TSDataType data_type_, + TSEncoding::TSEncoding encoding_, + CompressionType::CompressionType compression_type_) + : TemplateNode(name_) { + this->data_type_ = data_type_; + this->encoding_ = encoding_; + this->compression_type_ = compression_type_; + } + + TSDataType::TSDataType getDataType() const { return data_type_; } + + TSEncoding::TSEncoding getEncoding() const { return encoding_; } + + CompressionType::CompressionType getCompressionType() const { + return compression_type_; + } + + bool isMeasurement() override { return true; } + + std::string serialize() const override; + +private: + TSDataType::TSDataType data_type_; + TSEncoding::TSEncoding encoding_; + CompressionType::CompressionType compression_type_; +}; + +class InternalNode : public TemplateNode { +public: + InternalNode(const std::string &name, bool is_aligned) + : TemplateNode(name), is_aligned_(is_aligned) {} + + void addChild(const InternalNode &node) { + if (this->children_.count(node.getName())) { + throw BatchExecutionException("Duplicated child of node in template."); + } + this->children_[node.getName()] = std::make_shared(node); + } + + void addChild(const MeasurementNode &node) { + if (this->children_.count(node.getName())) { + throw BatchExecutionException("Duplicated child of node in template."); + } + this->children_[node.getName()] = std::make_shared(node); + } + + void deleteChild(const TemplateNode &node) { + this->children_.erase(node.getName()); + } + + const std::unordered_map> & + getChildren() const override { + return children_; + } + + bool isMeasurement() override { return false; } + + bool isAligned() override { return is_aligned_; } + +private: + std::unordered_map> children_; + bool is_aligned_; +}; + +namespace TemplateQueryType { +enum TemplateQueryType { + COUNT_MEASUREMENTS, + IS_MEASUREMENT, + PATH_EXIST, + SHOW_MEASUREMENTS +}; +} + +class Template { +public: + Template(const std::string &name, bool is_aligned) + : name_(name), is_aligned_(is_aligned) {} + + const std::string &getName() const { return name_; } + + bool isAligned() const { return is_aligned_; } + + void addToTemplate(const InternalNode &child) { + if (this->children_.count(child.getName())) { + throw BatchExecutionException("Duplicated child of node in template."); + } + this->children_[child.getName()] = std::make_shared(child); + } + + void addToTemplate(const MeasurementNode &child) { + if (this->children_.count(child.getName())) { + throw BatchExecutionException("Duplicated child of node in template."); + } + this->children_[child.getName()] = std::make_shared(child); + } + + std::string serialize() const; + +private: + std::string name_; + std::unordered_map> children_; + bool is_aligned_; +}; + +class SessionConnection; + +class Session { + struct Impl; + std::unique_ptr impl_; + friend class SessionConnection; + +public: + Session(const std::string &host, int rpcPort); + Session(const std::vector &nodeUrls, const std::string &username, + const std::string &password); + Session(const std::string &host, int rpcPort, const std::string &username, + const std::string &password); + Session(const std::string &host, int rpcPort, const std::string &username, + const std::string &password, const std::string &zoneId, + int fetchSize = AbstractSessionBuilder::DEFAULT_FETCH_SIZE); + Session(const std::string &host, const std::string &rpcPort, + const std::string &username = "user", + const std::string &password = "password", + const std::string &zoneId = "", + int fetchSize = AbstractSessionBuilder::DEFAULT_FETCH_SIZE); + Session(AbstractSessionBuilder *builder); + ~Session(); + + void open(); + + void open(bool enableRPCCompression); + + void open(bool enableRPCCompression, int connectionTimeoutInMs); + + void close(); + + void setTimeZone(const std::string &zoneId); + + std::string getTimeZone(); + + void insertRecord(const std::string &deviceId, int64_t time, + const std::vector &measurements, + const std::vector &values); + + void insertRecord(const std::string &deviceId, int64_t time, + const std::vector &measurements, + const std::vector &types, + const std::vector &values); + + void insertAlignedRecord(const std::string &deviceId, int64_t time, + const std::vector &measurements, + const std::vector &values); + + void insertAlignedRecord(const std::string &deviceId, int64_t time, + const std::vector &measurements, + const std::vector &types, + const std::vector &values); + + void + insertRecords(const std::vector &deviceIds, + const std::vector ×, + const std::vector> &measurementsList, + const std::vector> &valuesList); + + void insertRecords( + const std::vector &deviceIds, + const std::vector ×, + const std::vector> &measurementsList, + const std::vector> &typesList, + const std::vector> &valuesList); + + void insertAlignedRecords( + const std::vector &deviceIds, + const std::vector ×, + const std::vector> &measurementsList, + const std::vector> &valuesList); + + void insertAlignedRecords( + const std::vector &deviceIds, + const std::vector ×, + const std::vector> &measurementsList, + const std::vector> &typesList, + const std::vector> &valuesList); + + void insertRecordsOfOneDevice( + const std::string &deviceId, std::vector ×, + std::vector> &measurementsList, + std::vector> &typesList, + std::vector> &valuesList); + + void insertRecordsOfOneDevice( + const std::string &deviceId, std::vector ×, + std::vector> &measurementsList, + std::vector> &typesList, + std::vector> &valuesList, bool sorted); + + void insertAlignedRecordsOfOneDevice( + const std::string &deviceId, std::vector ×, + std::vector> &measurementsList, + std::vector> &typesList, + std::vector> &valuesList); + + void insertAlignedRecordsOfOneDevice( + const std::string &deviceId, std::vector ×, + std::vector> &measurementsList, + std::vector> &typesList, + std::vector> &valuesList, bool sorted); + + void insertTablet(Tablet &tablet); + + void insertTablet(Tablet &tablet, bool sorted); + + void insertAlignedTablet(Tablet &tablet); + + void insertAlignedTablet(Tablet &tablet, bool sorted); + + void insertTablets(std::unordered_map &tablets); + + void insertTablets(std::unordered_map &tablets, + bool sorted); + + void insertAlignedTablets(std::unordered_map &tablets, + bool sorted = false); + + void testInsertRecord(const std::string &deviceId, int64_t time, + const std::vector &measurements, + const std::vector &values); + + void testInsertTablet(const Tablet &tablet); + + void testInsertRecords( + const std::vector &deviceIds, + const std::vector ×, + const std::vector> &measurementsList, + const std::vector> &valuesList); + + void deleteTimeseries(const std::string &path); + + void deleteTimeseries(const std::vector &paths); + + void deleteData(const std::string &path, int64_t endTime); + + void deleteData(const std::vector &paths, int64_t endTime); + + void deleteData(const std::vector &paths, int64_t startTime, + int64_t endTime); + + void setStorageGroup(const std::string &storageGroupId); + + void deleteStorageGroup(const std::string &storageGroup); + + void deleteStorageGroups(const std::vector &storageGroups); + + void createDatabase(const std::string &database); + + void deleteDatabase(const std::string &database); + + void deleteDatabases(const std::vector &databases); + + void createTimeseries(const std::string &path, + TSDataType::TSDataType dataType, + TSEncoding::TSEncoding encoding, + CompressionType::CompressionType compressor); + + void createTimeseries(const std::string &path, + TSDataType::TSDataType dataType, + TSEncoding::TSEncoding encoding, + CompressionType::CompressionType compressor, + std::map *props, + std::map *tags, + std::map *attributes, + const std::string &measurementAlias); + + void createMultiTimeseries( + const std::vector &paths, + const std::vector &dataTypes, + const std::vector &encodings, + const std::vector &compressors, + std::vector> *propsList, + std::vector> *tagsList, + std::vector> *attributesList, + std::vector *measurementAliasList); + + void createAlignedTimeseries( + const std::string &deviceId, const std::vector &measurements, + const std::vector &dataTypes, + const std::vector &encodings, + const std::vector &compressors); + + bool checkTimeseriesExists(const std::string &path); + + std::unique_ptr executeQueryStatement(const std::string &sql); + + std::unique_ptr executeQueryStatement(const std::string &sql, + int64_t timeoutInMs); + + std::unique_ptr + executeQueryStatementMayRedirect(const std::string &sql, int64_t timeoutInMs); + + void executeNonQueryStatement(const std::string &sql); + + std::unique_ptr + executeRawDataQuery(const std::vector &paths, int64_t startTime, + int64_t endTime); + + std::unique_ptr + executeLastDataQuery(const std::vector &paths); + + std::unique_ptr + executeLastDataQuery(const std::vector &paths, int64_t lastTime); + + void createSchemaTemplate(const Template &templ); + + void setSchemaTemplate(const std::string &template_name, + const std::string &prefix_path); + + void unsetSchemaTemplate(const std::string &prefix_path, + const std::string &template_name); + + void addAlignedMeasurementsInTemplate( + const std::string &template_name, + const std::vector &measurements, + const std::vector &dataTypes, + const std::vector &encodings, + const std::vector &compressors); + + void addAlignedMeasurementsInTemplate( + const std::string &template_name, const std::string &measurement, + TSDataType::TSDataType dataType, TSEncoding::TSEncoding encoding, + CompressionType::CompressionType compressor); + + void addUnalignedMeasurementsInTemplate( + const std::string &template_name, + const std::vector &measurements, + const std::vector &dataTypes, + const std::vector &encodings, + const std::vector &compressors); + + void addUnalignedMeasurementsInTemplate( + const std::string &template_name, const std::string &measurement, + TSDataType::TSDataType dataType, TSEncoding::TSEncoding encoding, + CompressionType::CompressionType compressor); + + void deleteNodeInTemplate(const std::string &template_name, + const std::string &path); + + int countMeasurementsInTemplate(const std::string &template_name); + + bool isMeasurementInTemplate(const std::string &template_name, + const std::string &path); + + bool isPathExistInTemplate(const std::string &template_name, + const std::string &path); + + std::vector + showMeasurementsInTemplate(const std::string &template_name); + + std::vector + showMeasurementsInTemplate(const std::string &template_name, + const std::string &pattern); + + bool checkTemplateExists(const std::string &template_name); +}; + +#endif // IOTDB_SESSION_H diff --git a/iotdb-client/client-cpp/src/include/SessionBuilder.h b/iotdb-client/client-cpp/src/include/SessionBuilder.h new file mode 100644 index 0000000000000..1598699da451c --- /dev/null +++ b/iotdb-client/client-cpp/src/include/SessionBuilder.h @@ -0,0 +1,101 @@ +/* + * 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. + */ + +#ifndef IOTDB_SESSION_BUILDER_H +#define IOTDB_SESSION_BUILDER_H + +#include "AbstractSessionBuilder.h" + +class SessionBuilder : public AbstractSessionBuilder { +public: + SessionBuilder *host(const std::string &host) { + AbstractSessionBuilder::host = host; + return this; + } + + SessionBuilder *rpcPort(int rpcPort) { + AbstractSessionBuilder::rpcPort = rpcPort; + return this; + } + + SessionBuilder *useSSL(bool useSSL) { + AbstractSessionBuilder::useSSL = useSSL; + return this; + } + + SessionBuilder *trustCertFilePath(const std::string &trustCertFilePath) { + AbstractSessionBuilder::trustCertFilePath = trustCertFilePath; + return this; + } + + SessionBuilder *username(const std::string &username) { + AbstractSessionBuilder::username = username; + return this; + } + + SessionBuilder *password(const std::string &password) { + AbstractSessionBuilder::password = password; + return this; + } + + SessionBuilder *zoneId(const std::string &zoneId) { + AbstractSessionBuilder::zoneId = zoneId; + return this; + } + + SessionBuilder *fetchSize(int fetchSize) { + AbstractSessionBuilder::fetchSize = fetchSize; + return this; + } + + SessionBuilder *nodeUrls(const std::vector &nodeUrls) { + AbstractSessionBuilder::nodeUrls = nodeUrls; + return this; + } + + SessionBuilder *enableAutoFetch(bool enableAutoFetch) { + AbstractSessionBuilder::enableAutoFetch = enableAutoFetch; + return this; + } + + SessionBuilder *enableRedirections(bool enableRedirections) { + AbstractSessionBuilder::enableRedirections = enableRedirections; + return this; + } + + SessionBuilder *enableRPCCompression(bool enableRPCCompression) { + AbstractSessionBuilder::enableRPCCompression = enableRPCCompression; + return this; + } + + std::shared_ptr build() { + if (!AbstractSessionBuilder::nodeUrls.empty() && + (AbstractSessionBuilder::host != DEFAULT_HOST || + AbstractSessionBuilder::rpcPort != DEFAULT_RPC_PORT)) { + throw IoTDBException("Session builder does not support setting node urls " + "and host/rpcPort at the same time."); + } + sqlDialect = "tree"; + auto newSession = std::make_shared(this); + newSession->open(false); + return newSession; + } +}; + +#endif // IOTDB_SESSION_BUILDER_H \ No newline at end of file diff --git a/iotdb-client/client-cpp/src/include/SessionC.h b/iotdb-client/client-cpp/src/include/SessionC.h new file mode 100644 index 0000000000000..510fbb01a5719 --- /dev/null +++ b/iotdb-client/client-cpp/src/include/SessionC.h @@ -0,0 +1,399 @@ +/** + * 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. + */ + +#ifndef IOTDB_SESSION_C_H +#define IOTDB_SESSION_C_H + +#include +#include +#include + +#ifdef __cplusplus +extern "C" { +#endif + +/* ============================================================ + * Error Handling + * ============================================================ */ + +typedef int64_t TsStatus; + +#define TS_OK 0 +#define TS_ERR_CONNECTION -1 +#define TS_ERR_EXECUTION -2 +#define TS_ERR_INVALID_PARAM -3 +#define TS_ERR_NULL_PTR -4 +#define TS_ERR_UNKNOWN -99 + +/** + * Returns the error message from the last failed C API call on the current + * thread. The returned pointer is valid until the next C API call on the same + * thread. + */ +const char *ts_get_last_error(void); + +/* ============================================================ + * Opaque Handle Types + * ============================================================ */ + +typedef struct CSession_ CSession; +typedef struct CTablet_ CTablet; +typedef struct CSessionDataSet_ CSessionDataSet; +typedef struct CRowRecord_ CRowRecord; + +/* ============================================================ + * Enums (values match C++ TSDataType / TSEncoding / CompressionType) + * ============================================================ */ + +typedef enum { + TS_TYPE_BOOLEAN = 0, + TS_TYPE_INT32 = 1, + TS_TYPE_INT64 = 2, + TS_TYPE_FLOAT = 3, + TS_TYPE_DOUBLE = 4, + TS_TYPE_TEXT = 5, + TS_TYPE_TIMESTAMP = 8, + TS_TYPE_DATE = 9, + TS_TYPE_BLOB = 10, + TS_TYPE_STRING = 11, + /** Not a server type; used for invalid arguments / error paths in the C API. + */ + TS_TYPE_INVALID = 255 +} TSDataType_C; + +typedef enum { + TS_ENCODING_PLAIN = 0, + TS_ENCODING_DICTIONARY = 1, + TS_ENCODING_RLE = 2, + TS_ENCODING_DIFF = 3, + TS_ENCODING_TS_2DIFF = 4, + TS_ENCODING_BITMAP = 5, + TS_ENCODING_GORILLA_V1 = 6, + TS_ENCODING_REGULAR = 7, + TS_ENCODING_GORILLA = 8, + TS_ENCODING_ZIGZAG = 9, + TS_ENCODING_FREQ = 10 +} TSEncoding_C; + +typedef enum { + TS_COMPRESSION_UNCOMPRESSED = 0, + TS_COMPRESSION_SNAPPY = 1, + TS_COMPRESSION_GZIP = 2, + TS_COMPRESSION_LZO = 3, + TS_COMPRESSION_SDT = 4, + TS_COMPRESSION_PAA = 5, + TS_COMPRESSION_PLA = 6, + TS_COMPRESSION_LZ4 = 7, + TS_COMPRESSION_ZSTD = 8, + TS_COMPRESSION_LZMA2 = 9 +} TSCompressionType_C; + +/* ============================================================ + * Session Lifecycle — Tree Model + * ============================================================ */ + +CSession *ts_session_new(const char *host, int rpcPort, const char *username, + const char *password); + +CSession *ts_session_new_with_zone(const char *host, int rpcPort, + const char *username, const char *password, + const char *zoneId, int fetchSize); + +CSession *ts_session_new_multi_node(const char *const *nodeUrls, int urlCount, + const char *username, const char *password); + +void ts_session_destroy(CSession *session); + +TsStatus ts_session_open(CSession *session); + +TsStatus ts_session_open_with_compression(CSession *session, + bool enableRPCCompression); + +TsStatus ts_session_close(CSession *session); + +/* ============================================================ + * Timezone + * ============================================================ */ + +TsStatus ts_session_set_timezone(CSession *session, const char *zoneId); + +TsStatus ts_session_get_timezone(CSession *session, char *buf, int bufLen); + +/* ============================================================ + * Database Management (Tree Model) + * ============================================================ */ + +TsStatus ts_session_create_database(CSession *session, const char *database); + +TsStatus ts_session_delete_database(CSession *session, const char *database); + +TsStatus ts_session_delete_databases(CSession *session, + const char *const *databases, int count); + +/* ============================================================ + * Timeseries Management (Tree Model) + * ============================================================ */ + +TsStatus ts_session_create_timeseries(CSession *session, const char *path, + TSDataType_C dataType, + TSEncoding_C encoding, + TSCompressionType_C compressor); + +TsStatus ts_session_create_timeseries_ex( + CSession *session, const char *path, TSDataType_C dataType, + TSEncoding_C encoding, TSCompressionType_C compressor, int propsCount, + const char *const *propKeys, const char *const *propValues, int tagsCount, + const char *const *tagKeys, const char *const *tagValues, int attrsCount, + const char *const *attrKeys, const char *const *attrValues, + const char *measurementAlias); + +TsStatus ts_session_create_multi_timeseries( + CSession *session, int count, const char *const *paths, + const TSDataType_C *dataTypes, const TSEncoding_C *encodings, + const TSCompressionType_C *compressors); + +TsStatus ts_session_create_aligned_timeseries( + CSession *session, const char *deviceId, int count, + const char *const *measurements, const TSDataType_C *dataTypes, + const TSEncoding_C *encodings, const TSCompressionType_C *compressors); + +TsStatus ts_session_check_timeseries_exists(CSession *session, const char *path, + bool *exists); + +TsStatus ts_session_delete_timeseries(CSession *session, const char *path); + +TsStatus ts_session_delete_timeseries_batch(CSession *session, + const char *const *paths, + int count); + +/* ============================================================ + * Tablet Operations + * ============================================================ */ + +CTablet *ts_tablet_new(const char *deviceId, int columnCount, + const char *const *columnNames, + const TSDataType_C *dataTypes, int maxRowNumber); + +void ts_tablet_destroy(CTablet *tablet); + +void ts_tablet_reset(CTablet *tablet); + +int ts_tablet_get_row_count(CTablet *tablet); + +TsStatus ts_tablet_set_row_count(CTablet *tablet, int rowCount); + +TsStatus ts_tablet_add_timestamp(CTablet *tablet, int rowIndex, + int64_t timestamp); + +TsStatus ts_tablet_add_value_bool(CTablet *tablet, int colIndex, int rowIndex, + bool value); + +TsStatus ts_tablet_add_value_int32(CTablet *tablet, int colIndex, int rowIndex, + int32_t value); + +TsStatus ts_tablet_add_value_int64(CTablet *tablet, int colIndex, int rowIndex, + int64_t value); + +TsStatus ts_tablet_add_value_float(CTablet *tablet, int colIndex, int rowIndex, + float value); + +TsStatus ts_tablet_add_value_double(CTablet *tablet, int colIndex, int rowIndex, + double value); + +TsStatus ts_tablet_add_value_string(CTablet *tablet, int colIndex, int rowIndex, + const char *value); + +/* ============================================================ + * Data Insertion — Tree Model (Record) + * ============================================================ */ + +TsStatus ts_session_insert_record_str(CSession *session, const char *deviceId, + int64_t time, int count, + const char *const *measurements, + const char *const *values); + +TsStatus ts_session_insert_record(CSession *session, const char *deviceId, + int64_t time, int count, + const char *const *measurements, + const TSDataType_C *types, + const void *const *values); + +TsStatus ts_session_insert_aligned_record_str(CSession *session, + const char *deviceId, + int64_t time, int count, + const char *const *measurements, + const char *const *values); + +TsStatus ts_session_insert_aligned_record(CSession *session, + const char *deviceId, int64_t time, + int count, + const char *const *measurements, + const TSDataType_C *types, + const void *const *values); + +/* Batch insert — multiple devices (string values) */ +TsStatus ts_session_insert_records_str( + CSession *session, int deviceCount, const char *const *deviceIds, + const int64_t *times, const int *measurementCounts, + const char *const *const *measurementsList, + const char *const *const *valuesList); + +TsStatus ts_session_insert_aligned_records_str( + CSession *session, int deviceCount, const char *const *deviceIds, + const int64_t *times, const int *measurementCounts, + const char *const *const *measurementsList, + const char *const *const *valuesList); + +/* Batch insert — multiple devices (typed values) */ +TsStatus ts_session_insert_records(CSession *session, int deviceCount, + const char *const *deviceIds, + const int64_t *times, + const int *measurementCounts, + const char *const *const *measurementsList, + const TSDataType_C *const *typesList, + const void *const *const *valuesList); + +TsStatus ts_session_insert_aligned_records( + CSession *session, int deviceCount, const char *const *deviceIds, + const int64_t *times, const int *measurementCounts, + const char *const *const *measurementsList, + const TSDataType_C *const *typesList, const void *const *const *valuesList); + +/* Batch insert — single device (typed values) */ +TsStatus ts_session_insert_records_of_one_device( + CSession *session, const char *deviceId, int rowCount, const int64_t *times, + const int *measurementCounts, const char *const *const *measurementsList, + const TSDataType_C *const *typesList, const void *const *const *valuesList, + bool sorted); + +TsStatus ts_session_insert_aligned_records_of_one_device( + CSession *session, const char *deviceId, int rowCount, const int64_t *times, + const int *measurementCounts, const char *const *const *measurementsList, + const TSDataType_C *const *typesList, const void *const *const *valuesList, + bool sorted); + +/* ============================================================ + * Data Insertion — Tree Model (Tablet) + * ============================================================ */ + +TsStatus ts_session_insert_tablet(CSession *session, CTablet *tablet, + bool sorted); + +TsStatus ts_session_insert_aligned_tablet(CSession *session, CTablet *tablet, + bool sorted); + +TsStatus ts_session_insert_tablets(CSession *session, int tabletCount, + const char *const *deviceIds, + CTablet **tablets, bool sorted); + +TsStatus ts_session_insert_aligned_tablets(CSession *session, int tabletCount, + const char *const *deviceIds, + CTablet **tablets, bool sorted); + +/* ============================================================ + * Query — Tree Model + * ============================================================ */ + +TsStatus ts_session_execute_query(CSession *session, const char *sql, + CSessionDataSet **dataSet); + +TsStatus ts_session_execute_query_with_timeout(CSession *session, + const char *sql, + int64_t timeoutInMs, + CSessionDataSet **dataSet); + +TsStatus ts_session_execute_non_query(CSession *session, const char *sql); + +TsStatus ts_session_execute_raw_data_query(CSession *session, int pathCount, + const char *const *paths, + int64_t startTime, int64_t endTime, + CSessionDataSet **dataSet); + +TsStatus ts_session_execute_last_data_query(CSession *session, int pathCount, + const char *const *paths, + CSessionDataSet **dataSet); + +TsStatus ts_session_execute_last_data_query_with_time( + CSession *session, int pathCount, const char *const *paths, + int64_t lastTime, CSessionDataSet **dataSet); + +/* ============================================================ + * SessionDataSet & RowRecord — Result Iteration + * ============================================================ */ + +void ts_dataset_destroy(CSessionDataSet *dataSet); + +/** On failure (null handle, exception), see ts_get_last_error(). */ +bool ts_dataset_has_next(CSessionDataSet *dataSet); + +/** On failure (null handle, exception), see ts_get_last_error(); nullptr may + * also mean end of rows. */ +CRowRecord *ts_dataset_next(CSessionDataSet *dataSet); + +int ts_dataset_get_column_count(CSessionDataSet *dataSet); + +const char *ts_dataset_get_column_name(CSessionDataSet *dataSet, int index); + +const char *ts_dataset_get_column_type(CSessionDataSet *dataSet, int index); + +void ts_dataset_set_fetch_size(CSessionDataSet *dataSet, int fetchSize); + +void ts_row_record_destroy(CRowRecord *record); + +int64_t ts_row_record_get_timestamp(CRowRecord *record); + +int ts_row_record_get_field_count(CRowRecord *record); + +bool ts_row_record_is_null(CRowRecord *record, int index); + +bool ts_row_record_get_bool(CRowRecord *record, int index); + +int32_t ts_row_record_get_int32(CRowRecord *record, int index); + +int64_t ts_row_record_get_int64(CRowRecord *record, int index); + +float ts_row_record_get_float(CRowRecord *record, int index); + +double ts_row_record_get_double(CRowRecord *record, int index); + +const char *ts_row_record_get_string(CRowRecord *record, int index); + +/** Returns TS_TYPE_INVALID if record is null or index is out of range. */ +TSDataType_C ts_row_record_get_data_type(CRowRecord *record, int index); + +/* ============================================================ + * Data Deletion (Tree Model) + * ============================================================ */ + +TsStatus ts_session_delete_data(CSession *session, const char *path, + int64_t endTime); + +TsStatus ts_session_delete_data_batch(CSession *session, int pathCount, + const char *const *paths, + int64_t endTime); + +TsStatus ts_session_delete_data_range(CSession *session, int pathCount, + const char *const *paths, + int64_t startTime, int64_t endTime); + +#ifdef __cplusplus +} /* extern "C" */ +#endif + +#endif /* IOTDB_SESSION_C_H */ diff --git a/iotdb-client/client-cpp/src/include/SessionConfig.h b/iotdb-client/client-cpp/src/include/SessionConfig.h new file mode 100644 index 0000000000000..2106a1b6c7b5a --- /dev/null +++ b/iotdb-client/client-cpp/src/include/SessionConfig.h @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +#ifndef IOTDB_SESSION_CONFIG_H +#define IOTDB_SESSION_CONFIG_H + +namespace iotdb { +namespace session { + +// Defaults aligned with org.apache.iotdb.isession.SessionConfig (Java client). +constexpr int DEFAULT_FETCH_SIZE = 5000; +constexpr int DEFAULT_CONNECT_TIMEOUT_MS = 3 * 1000; +constexpr int DEFAULT_MAX_RETRIES = 3; +constexpr int DEFAULT_RETRY_DELAY_MS = 500; + +} // namespace session +} // namespace iotdb + +#endif // IOTDB_SESSION_CONFIG_H diff --git a/iotdb-client/client-cpp/src/include/SessionDataSet.h b/iotdb-client/client-cpp/src/include/SessionDataSet.h new file mode 100644 index 0000000000000..c7eac0f81cb1d --- /dev/null +++ b/iotdb-client/client-cpp/src/include/SessionDataSet.h @@ -0,0 +1,124 @@ +/** + * 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. + */ + +#ifndef IOTDB_SESSION_DATA_SET_H +#define IOTDB_SESSION_DATA_SET_H + +#include +#include +#include +#include +#include + +#include "Column.h" +#include "Date.h" +#include "Optional.h" + +class RowRecord { +public: + int64_t timestamp; + std::vector fields; + + explicit RowRecord(int64_t timestamp); + RowRecord(int64_t timestamp, const std::vector &fields); + explicit RowRecord(const std::vector &fields); + RowRecord(); + + void addField(const Field &f); + std::string toString(); +}; + +class Session; + +class SessionDataSet { + struct Impl; + std::unique_ptr impl_; + SessionDataSet() = default; + friend class Session; + friend std::unique_ptr createSessionDataSet( + const std::string &sql, const std::vector &columnNameList, + const std::vector &columnTypeList, + const std::map &columnNameIndex, int64_t queryId, + int64_t statementId, + std::shared_ptr client, int64_t sessionId, + const std::vector &queryResult, bool ignoreTimestamp, + int64_t timeout, bool moreData, int32_t fetchSize, + const std::string &zoneId); + +private: + std::shared_ptr constructRowRecordFromValueArray(); + +public: + ~SessionDataSet(); + + bool hasNext(); + std::shared_ptr next(); + + int getFetchSize(); + void setFetchSize(int fetchSize); + + const std::vector &getColumnNames() const; + const std::vector &getColumnTypeList() const; + void closeOperationHandle(bool forceClose = false); + + class DataIterator { + std::shared_ptr impl_; + + public: + explicit DataIterator(std::shared_ptr impl); + + bool next(); + + bool isNull(const std::string &columnName); + bool isNullByIndex(int32_t columnIndex); + + Optional getBooleanByIndex(int32_t columnIndex); + Optional getBoolean(const std::string &columnName); + + Optional getDoubleByIndex(int32_t columnIndex); + Optional getDouble(const std::string &columnName); + + Optional getFloatByIndex(int32_t columnIndex); + Optional getFloat(const std::string &columnName); + + Optional getIntByIndex(int32_t columnIndex); + Optional getInt(const std::string &columnName); + + Optional getLongByIndex(int32_t columnIndex); + Optional getLong(const std::string &columnName); + + Optional getStringByIndex(int32_t columnIndex); + Optional getString(const std::string &columnName); + + Optional getTimestampByIndex(int32_t columnIndex); + Optional getTimestamp(const std::string &columnName); + + Optional getDateByIndex(int32_t columnIndex); + Optional getDate(const std::string &columnName); + + int32_t findColumn(const std::string &columnName); + const std::vector &getColumnNames() const; + const std::vector &getColumnTypeList() const; + }; + + // The returned iterator must not outlive this SessionDataSet. + DataIterator getIterator(); +}; + +#endif diff --git a/iotdb-client/client-cpp/src/main/SessionPool.h b/iotdb-client/client-cpp/src/include/SessionPool.h similarity index 66% rename from iotdb-client/client-cpp/src/main/SessionPool.h rename to iotdb-client/client-cpp/src/include/SessionPool.h index 8838b298ec905..7c83fcaa87233 100644 --- a/iotdb-client/client-cpp/src/main/SessionPool.h +++ b/iotdb-client/client-cpp/src/include/SessionPool.h @@ -46,7 +46,8 @@ * { * PooledSession s = pool.getSession(); // blocks up to the timeout * s->insertTablet(tablet); // call any Session method - * } // automatically returned here + * } // automatically returned + * here * * 2. Convenience wrappers / generic execute() (recommended for hot paths): * pool.insertTablet(tablet); @@ -63,23 +64,25 @@ class SessionPool; class PooledSession { public: - PooledSession() noexcept : pool_(nullptr), session_(nullptr), broken_(false) {} + PooledSession() noexcept + : pool_(nullptr), session_(nullptr), broken_(false) {} - PooledSession(SessionPool* pool, std::shared_ptr session) + PooledSession(SessionPool *pool, std::shared_ptr session) : pool_(pool), session_(std::move(session)), broken_(false) {} // Non-copyable: a leased Session is owned by exactly one borrower. - PooledSession(const PooledSession&) = delete; - PooledSession& operator=(const PooledSession&) = delete; + PooledSession(const PooledSession &) = delete; + PooledSession &operator=(const PooledSession &) = delete; - PooledSession(PooledSession&& other) noexcept - : pool_(other.pool_), session_(std::move(other.session_)), broken_(other.broken_) { + PooledSession(PooledSession &&other) noexcept + : pool_(other.pool_), session_(std::move(other.session_)), + broken_(other.broken_) { other.pool_ = nullptr; other.session_ = nullptr; other.broken_ = false; } - PooledSession& operator=(PooledSession&& other) noexcept { + PooledSession &operator=(PooledSession &&other) noexcept { if (this != &other) { reset(); pool_ = other.pool_; @@ -92,37 +95,26 @@ class PooledSession { return *this; } - ~PooledSession() { - reset(); - } + ~PooledSession() { reset(); } - Session* operator->() const { - return session_.get(); - } + Session *operator->() const { return session_.get(); } - Session& operator*() const { - return *session_; - } + Session &operator*() const { return *session_; } - explicit operator bool() const { - return static_cast(session_); - } + explicit operator bool() const { return static_cast(session_); } - // Mark the underlying connection as unusable so it is discarded (not recycled) - // when this lease is returned. Call this if you caught a connection error. - void markBroken() { - broken_ = true; - } + // Mark the underlying connection as unusable so it is discarded (not + // recycled) when this lease is returned. Call this if you caught a connection + // error. + void markBroken() { broken_ = true; } // Eagerly return the Session to the pool before scope exit. - void release() { - reset(); - } + void release() { reset(); } private: void reset(); - SessionPool* pool_; + SessionPool *pool_; std::shared_ptr session_; bool broken_; }; @@ -137,20 +129,17 @@ class PooledSession { */ class PooledSessionDataSet { public: - PooledSessionDataSet(PooledSession session, std::unique_ptr dataSet) + PooledSessionDataSet(PooledSession session, + std::unique_ptr dataSet) : session_(std::move(session)), dataSet_(std::move(dataSet)) {} - PooledSessionDataSet(const PooledSessionDataSet&) = delete; - PooledSessionDataSet& operator=(const PooledSessionDataSet&) = delete; - PooledSessionDataSet(PooledSessionDataSet&&) noexcept = default; - PooledSessionDataSet& operator=(PooledSessionDataSet&&) noexcept = default; + PooledSessionDataSet(const PooledSessionDataSet &) = delete; + PooledSessionDataSet &operator=(const PooledSessionDataSet &) = delete; + PooledSessionDataSet(PooledSessionDataSet &&) noexcept = default; + PooledSessionDataSet &operator=(PooledSessionDataSet &&) noexcept = default; - SessionDataSet* operator->() const { - return dataSet_.get(); - } - SessionDataSet& operator*() const { - return *dataSet_; - } + SessionDataSet *operator->() const { return dataSet_.get(); } + SessionDataSet &operator*() const { return *dataSet_; } private: PooledSession session_; @@ -163,25 +152,25 @@ class SessionPool { static constexpr int64_t DEFAULT_WAIT_TIMEOUT_MS = 60 * 1000; // Single-host constructor. - SessionPool(std::string host, int rpcPort, std::string username, std::string password, - size_t maxSize = DEFAULT_MAX_SIZE); + SessionPool(std::string host, int rpcPort, std::string username, + std::string password, size_t maxSize = DEFAULT_MAX_SIZE); ~SessionPool(); // Non-copyable, non-movable: the pool owns mutex/condition state. - SessionPool(const SessionPool&) = delete; - SessionPool& operator=(const SessionPool&) = delete; + SessionPool(const SessionPool &) = delete; + SessionPool &operator=(const SessionPool &) = delete; // ---- configuration (apply before the first getSession()) ---- - SessionPool& setFetchSize(int fetchSize); - SessionPool& setZoneId(std::string zoneId); - SessionPool& setSqlDialect(std::string sqlDialect); - SessionPool& setDatabase(std::string database); - SessionPool& setEnableRedirection(bool enable); - SessionPool& setEnableAutoFetch(bool enable); - SessionPool& setEnableRPCCompression(bool enable); - SessionPool& setConnectTimeoutMs(int connectTimeoutMs); - SessionPool& setWaitToGetSessionTimeoutMs(int64_t timeoutMs); + SessionPool &setFetchSize(int fetchSize); + SessionPool &setZoneId(std::string zoneId); + SessionPool &setSqlDialect(std::string sqlDialect); + SessionPool &setDatabase(std::string database); + SessionPool &setEnableRedirection(bool enable); + SessionPool &setEnableAutoFetch(bool enable); + SessionPool &setEnableRPCCompression(bool enable); + SessionPool &setConnectTimeoutMs(int connectTimeoutMs); + SessionPool &setWaitToGetSessionTimeoutMs(int64_t timeoutMs); // Borrow a Session. Blocks until one is free or a new one can be created, // up to timeoutMs (<= 0 means use the pool default). Throws IoTDBException on @@ -191,32 +180,36 @@ class SessionPool { // Generic helper: borrow a Session, run func(Session&), return/evict it, and // forward the result. Evicts the Session on IoTDBConnectionException. - template auto execute(Func&& func) -> decltype(func(std::declval())); - - // ---- convenience wrappers for common operations (with eviction on failure) ---- - void insertTablet(Tablet& tablet, bool sorted = false); - void insertAlignedTablet(Tablet& tablet, bool sorted = false); - void insertTablets(std::unordered_map& tablets, bool sorted = false); - void insertRecord(const std::string& deviceId, int64_t time, - const std::vector& measurements, - const std::vector& values); - void insertRecords(const std::vector& deviceIds, const std::vector& times, - const std::vector>& measurementsList, - const std::vector>& valuesList); - void executeNonQueryStatement(const std::string& sql); + template + auto execute(Func &&func) -> decltype(func(std::declval())); + + // ---- convenience wrappers for common operations (with eviction on failure) + // ---- + void insertTablet(Tablet &tablet, bool sorted = false); + void insertAlignedTablet(Tablet &tablet, bool sorted = false); + void insertTablets(std::unordered_map &tablets, + bool sorted = false); + void insertRecord(const std::string &deviceId, int64_t time, + const std::vector &measurements, + const std::vector &values); + void + insertRecords(const std::vector &deviceIds, + const std::vector ×, + const std::vector> &measurementsList, + const std::vector> &valuesList); + void executeNonQueryStatement(const std::string &sql); // The returned wrapper keeps the underlying Session leased until it is // destroyed, so it is safe to iterate the result set across multiple fetches. - PooledSessionDataSet executeQueryStatement(const std::string& sql); - PooledSessionDataSet executeQueryStatement(const std::string& sql, int64_t timeoutInMs); + PooledSessionDataSet executeQueryStatement(const std::string &sql); + PooledSessionDataSet executeQueryStatement(const std::string &sql, + int64_t timeoutInMs); // Close the pool: idle Sessions are closed immediately, in-use Sessions are // closed when they are returned. Idempotent. void close(); // ---- observability ---- - size_t getMaxSize() const { - return maxSize_; - } + size_t getMaxSize() const { return maxSize_; } // Number of Sessions currently borrowed. size_t activeCount(); @@ -225,7 +218,7 @@ class SessionPool { std::shared_ptr constructNewSession(); std::shared_ptr acquire(int64_t timeoutMs); - void putBack(const std::shared_ptr& session, bool broken); + void putBack(const std::shared_ptr &session, bool broken); // connection parameters std::string host_; @@ -254,18 +247,20 @@ class SessionPool { }; template -auto SessionPool::execute(Func&& func) -> decltype(func(std::declval())) { +auto SessionPool::execute(Func &&func) + -> decltype(func(std::declval())) { PooledSession lease = getSession(); try { return func(*lease); - } catch (const IoTDBConnectionException&) { + } catch (const IoTDBConnectionException &) { lease.markBroken(); throw; } } /* - * Fluent builder for SessionPool, mirroring SessionBuilder / TableSessionBuilder. + * Fluent builder for SessionPool, mirroring SessionBuilder / + * TableSessionBuilder. * * auto pool = SessionPoolBuilder() * .host("127.0.0.1")->rpcPort(6667) @@ -274,59 +269,59 @@ auto SessionPool::execute(Func&& func) -> decltype(func(std::declval() */ class SessionPoolBuilder : public AbstractSessionBuilder { public: - SessionPoolBuilder* host(const std::string& v) { + SessionPoolBuilder *host(const std::string &v) { AbstractSessionBuilder::host = v; return this; } - SessionPoolBuilder* rpcPort(int v) { + SessionPoolBuilder *rpcPort(int v) { AbstractSessionBuilder::rpcPort = v; return this; } - SessionPoolBuilder* username(const std::string& v) { + SessionPoolBuilder *username(const std::string &v) { AbstractSessionBuilder::username = v; return this; } - SessionPoolBuilder* password(const std::string& v) { + SessionPoolBuilder *password(const std::string &v) { AbstractSessionBuilder::password = v; return this; } - SessionPoolBuilder* zoneId(const std::string& v) { + SessionPoolBuilder *zoneId(const std::string &v) { AbstractSessionBuilder::zoneId = v; return this; } - SessionPoolBuilder* fetchSize(int v) { + SessionPoolBuilder *fetchSize(int v) { AbstractSessionBuilder::fetchSize = v; return this; } - SessionPoolBuilder* database(const std::string& v) { + SessionPoolBuilder *database(const std::string &v) { AbstractSessionBuilder::database = v; return this; } - SessionPoolBuilder* enableAutoFetch(bool v) { + SessionPoolBuilder *enableAutoFetch(bool v) { AbstractSessionBuilder::enableAutoFetch = v; return this; } - SessionPoolBuilder* enableRedirections(bool v) { + SessionPoolBuilder *enableRedirections(bool v) { AbstractSessionBuilder::enableRedirections = v; return this; } - SessionPoolBuilder* enableRPCCompression(bool v) { + SessionPoolBuilder *enableRPCCompression(bool v) { AbstractSessionBuilder::enableRPCCompression = v; return this; } - SessionPoolBuilder* connectTimeoutMs(int v) { + SessionPoolBuilder *connectTimeoutMs(int v) { connectTimeoutMs_ = v; return this; } - SessionPoolBuilder* maxSize(size_t v) { + SessionPoolBuilder *maxSize(size_t v) { maxSize_ = v; return this; } - SessionPoolBuilder* waitToGetSessionTimeoutMs(int64_t v) { + SessionPoolBuilder *waitToGetSessionTimeoutMs(int64_t v) { waitTimeoutMs_ = v; return this; } - SessionPoolBuilder* sqlDialect(const std::string& v) { + SessionPoolBuilder *sqlDialect(const std::string &v) { AbstractSessionBuilder::sqlDialect = v; return this; } @@ -334,7 +329,8 @@ class SessionPoolBuilder : public AbstractSessionBuilder { std::shared_ptr build() { auto pool = std::make_shared( AbstractSessionBuilder::host, AbstractSessionBuilder::rpcPort, - AbstractSessionBuilder::username, AbstractSessionBuilder::password, maxSize_); + AbstractSessionBuilder::username, AbstractSessionBuilder::password, + maxSize_); pool->setFetchSize(AbstractSessionBuilder::fetchSize) .setZoneId(AbstractSessionBuilder::zoneId) .setSqlDialect(AbstractSessionBuilder::sqlDialect) diff --git a/iotdb-client/client-cpp/src/test/main.cpp b/iotdb-client/client-cpp/src/include/Status.h similarity index 52% rename from iotdb-client/client-cpp/src/test/main.cpp rename to iotdb-client/client-cpp/src/include/Status.h index 82b3e60ddb593..147c20cffd21c 100644 --- a/iotdb-client/client-cpp/src/test/main.cpp +++ b/iotdb-client/client-cpp/src/include/Status.h @@ -7,7 +7,7 @@ * "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 + * 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 @@ -16,27 +16,15 @@ * specific language governing permissions and limitations * under the License. */ +#ifndef IOTDB_STATUS_H +#define IOTDB_STATUS_H -#define CATCH_CONFIG_MAIN +#include +#include -#include -#include "Session.h" - -std::shared_ptr session = std::make_shared("127.0.0.1", 6667, "root", "root"); - -struct SessionListener : Catch::TestEventListenerBase { - - using TestEventListenerBase::TestEventListenerBase; - - void testCaseStarting(Catch::TestCaseInfo const &testInfo) override { - // Perform some setup before a test case is run - session->open(false); - } - - void testCaseEnded(Catch::TestCaseStats const &testCaseStats) override { - // Tear-down after a test case is run - session->close(); - } +struct Status { + int32_t code = 0; + std::string message; }; -CATCH_REGISTER_LISTENER( SessionListener ) \ No newline at end of file +#endif diff --git a/iotdb-client/client-cpp/src/include/TsBlock.h b/iotdb-client/client-cpp/src/include/TsBlock.h new file mode 100644 index 0000000000000..482aeacc3f3b2 --- /dev/null +++ b/iotdb-client/client-cpp/src/include/TsBlock.h @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#ifndef IOTDB_TS_BLOCK_H +#define IOTDB_TS_BLOCK_H + +#include "Column.h" +#include +#include + +class TsBlock { +public: + static std::shared_ptr + create(int32_t positionCount, std::shared_ptr timeColumn, + std::vector> valueColumns); + + static std::shared_ptr deserialize(const std::string &data); + + int32_t getPositionCount() const; + int64_t getStartTime() const; + int64_t getEndTime() const; + bool isEmpty() const; + int64_t getTimeByIndex(int32_t index) const; + int32_t getValueColumnCount() const; + const std::shared_ptr getTimeColumn() const; + const std::vector> &getValueColumns() const; + const std::shared_ptr getColumn(int32_t columnIndex) const; + +private: + TsBlock(int32_t positionCount, std::shared_ptr timeColumn, + std::vector> valueColumns); + + std::shared_ptr timeColumn_; + std::vector> valueColumns_; + int32_t positionCount_; +}; + +#endif diff --git a/iotdb-client/client-cpp/src/main/CMakeLists.txt b/iotdb-client/client-cpp/src/main/CMakeLists.txt deleted file mode 100644 index c6924060ba9b2..0000000000000 --- a/iotdb-client/client-cpp/src/main/CMakeLists.txt +++ /dev/null @@ -1,53 +0,0 @@ -# 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.7) -PROJECT(iotdb_session CXX) -SET(CMAKE_CXX_STANDARD 11) -SET(CMAKE_CXX_STANDARD_REQUIRED ON) -SET(CMAKE_POSITION_INDEPENDENT_CODE ON) -SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11 -Wall -g -O2 ") - -# Add Thrift include directory -INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/../../thrift/include) - -# Add Boost include path for MacOS -INCLUDE_DIRECTORIES(/usr/local/include) - -# Add Boost library headers for MaxOS -FIND_PACKAGE(Boost REQUIRED) -IF (DEFINED BOOST_INCLUDEDIR) - INCLUDE_DIRECTORIES(${Boost_INCLUDE_DIR}) -ENDIF() - -IF(MSVC) - SET(THRIFT_STATIC_LIB "${CMAKE_SOURCE_DIR}/../../thrift/lib/Release/thriftmd.lib") -ELSE() - SET(THRIFT_STATIC_LIB "${CMAKE_SOURCE_DIR}/../../thrift/lib/libthrift.a") -ENDIF() - -# Add the generated source files to the sources for the library. -AUX_SOURCE_DIRECTORY(./generated-sources-cpp SESSION_SRCS) -IF(MSVC) - ADD_LIBRARY(iotdb_session ${SESSION_SRCS}) -ELSE() - ADD_LIBRARY(iotdb_session SHARED ${SESSION_SRCS}) -ENDIF() - -# Link with Thrift static library -TARGET_LINK_LIBRARIES(iotdb_session ${THRIFT_STATIC_LIB}) diff --git a/iotdb-client/client-cpp/src/main/Column.cpp b/iotdb-client/client-cpp/src/main/Column.cpp deleted file mode 100644 index 4c533676d075a..0000000000000 --- a/iotdb-client/client-cpp/src/main/Column.cpp +++ /dev/null @@ -1,359 +0,0 @@ -/** -* Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -#include "Column.h" -#include "ColumnDecoder.h" - -TimeColumn::TimeColumn(int32_t arrayOffset, int32_t positionCount, const std::vector& values) - : arrayOffset_(arrayOffset), positionCount_(positionCount), values_(values) { - if (arrayOffset < 0) throw IoTDBException("arrayOffset is negative"); - if (positionCount < 0) throw IoTDBException("positionCount is negative"); - if (static_cast(values.size()) - arrayOffset < positionCount) { - throw IoTDBException("values length is less than positionCount"); - } -} - -TSDataType::TSDataType TimeColumn::getDataType() const { return TSDataType::INT64; } -ColumnEncoding TimeColumn::getEncoding() const { return ColumnEncoding::Int64Array; } - -int64_t TimeColumn::getLong(int32_t position) const { - return values_[position + arrayOffset_]; -} - -bool TimeColumn::mayHaveNull() const { return false; } -bool TimeColumn::isNull(int32_t position) const { return false; } -std::vector TimeColumn::isNulls() const { return {}; } - -int32_t TimeColumn::getPositionCount() const { return positionCount_; } - -int64_t TimeColumn::getStartTime() const { return values_[arrayOffset_]; } -int64_t TimeColumn::getEndTime() const { return values_[positionCount_ + arrayOffset_ - 1]; } - -const std::vector& TimeColumn::getTimes() const { return values_; } -std::vector TimeColumn::getLongs() const { return getTimes(); } - -BinaryColumn::BinaryColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector>& values) - : arrayOffset_(arrayOffset), positionCount_(positionCount), - valueIsNull_(valueIsNull), values_(values) { - if (arrayOffset < 0) throw IoTDBException("arrayOffset is negative"); - if (positionCount < 0) throw IoTDBException("positionCount is negative"); - if (static_cast(values.size()) - arrayOffset < positionCount) { - throw IoTDBException("values length is less than positionCount"); - } - if (!valueIsNull.empty() && static_cast(valueIsNull.size()) - arrayOffset < positionCount) { - throw IoTDBException("isNull length is less than positionCount"); - } -} - -TSDataType::TSDataType BinaryColumn::getDataType() const { return TSDataType::TSDataType::TEXT; } -ColumnEncoding BinaryColumn::getEncoding() const { return ColumnEncoding::BinaryArray; } - -std::shared_ptr BinaryColumn::getBinary(int32_t position) const { - return values_[position + arrayOffset_]; -} - -std::vector> BinaryColumn::getBinaries() const { return values_; } - - -bool BinaryColumn::mayHaveNull() const { return !valueIsNull_.empty(); } - -bool BinaryColumn::isNull(int32_t position) const { - return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; -} - -std::vector BinaryColumn::isNulls() const { - if (!valueIsNull_.empty()) return valueIsNull_; - - std::vector result(positionCount_, false); - return result; -} - -int32_t BinaryColumn::getPositionCount() const { return positionCount_; } - -IntColumn::IntColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values) - : arrayOffset_(arrayOffset), positionCount_(positionCount), - valueNull_(valueIsNull), values_(values) { - if (arrayOffset < 0) throw IoTDBException("arrayOffset is negative"); - if (positionCount < 0) throw IoTDBException("positionCount is negative"); - if (static_cast(values.size()) - arrayOffset < positionCount) { - throw IoTDBException("values length is less than positionCount"); - } - if (!valueIsNull.empty() && static_cast(valueIsNull.size()) - arrayOffset < positionCount) { - throw IoTDBException("isNull length is less than positionCount"); - } -} - -TSDataType::TSDataType IntColumn::getDataType() const { return TSDataType::INT32; } -ColumnEncoding IntColumn::getEncoding() const { return ColumnEncoding::Int32Array; } - -int32_t IntColumn::getInt(int32_t position) const { - return values_[position + arrayOffset_]; -} - -std::vector IntColumn::getInts() const { return values_; } - -bool IntColumn::mayHaveNull() const { return !valueNull_.empty(); } - -bool IntColumn::isNull(int32_t position) const { - return !valueNull_.empty() && valueNull_[position + arrayOffset_]; -} - -std::vector IntColumn::isNulls() const { - if (!valueNull_.empty()) return valueNull_; - - std::vector result(positionCount_, false); - return result; -} - -int32_t IntColumn::getPositionCount() const { return positionCount_; } - -FloatColumn::FloatColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values) - : arrayOffset_(arrayOffset), positionCount_(positionCount), - valueIsNull_(valueIsNull), values_(values) { - if (arrayOffset < 0) throw IoTDBException("arrayOffset is negative"); - if (positionCount < 0) throw IoTDBException("positionCount is negative"); - if (static_cast(values.size()) - arrayOffset < positionCount) { - throw IoTDBException("values length is less than positionCount"); - } - if (!valueIsNull.empty() && static_cast(valueIsNull.size()) - arrayOffset < positionCount) { - throw IoTDBException("isNull length is less than positionCount"); - } -} - -TSDataType::TSDataType FloatColumn::getDataType() const { return TSDataType::TSDataType::FLOAT; } -ColumnEncoding FloatColumn::getEncoding() const { return ColumnEncoding::Int32Array; } - -float FloatColumn::getFloat(int32_t position) const { - return values_[position + arrayOffset_]; -} - -std::vector FloatColumn::getFloats() const { return values_; } - -bool FloatColumn::mayHaveNull() const { return !valueIsNull_.empty(); } - -bool FloatColumn::isNull(int32_t position) const { - return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; -} - -std::vector FloatColumn::isNulls() const { - if (!valueIsNull_.empty()) return valueIsNull_; - - std::vector result(positionCount_, false); - return result; -} - -int32_t FloatColumn::getPositionCount() const { return positionCount_; } - -LongColumn::LongColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values) - : arrayOffset_(arrayOffset), positionCount_(positionCount), - valueIsNull_(valueIsNull), values_(values) { - if (arrayOffset < 0) throw IoTDBException("arrayOffset is negative"); - if (positionCount < 0) throw IoTDBException("positionCount is negative"); - if (static_cast(values.size()) - arrayOffset < positionCount) { - throw IoTDBException("values length is less than positionCount"); - } - if (!valueIsNull.empty() && static_cast(valueIsNull.size()) - arrayOffset < positionCount) { - throw IoTDBException("isNull length is less than positionCount"); - } -} - -TSDataType::TSDataType LongColumn::getDataType() const { return TSDataType::TSDataType::INT64; } -ColumnEncoding LongColumn::getEncoding() const { return ColumnEncoding::Int64Array; } - -int64_t LongColumn::getLong(int32_t position) const { - return values_[position + arrayOffset_]; -} - -std::vector LongColumn::getLongs() const { return values_; } - -bool LongColumn::mayHaveNull() const { return !valueIsNull_.empty(); } - -bool LongColumn::isNull(int32_t position) const { - return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; -} - -std::vector LongColumn::isNulls() const { - if (!valueIsNull_.empty()) return valueIsNull_; - - std::vector result(positionCount_, false); - return result; -} - -int32_t LongColumn::getPositionCount() const { return positionCount_; } - -DoubleColumn::DoubleColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values) - : arrayOffset_(arrayOffset), positionCount_(positionCount), - valueIsNull_(valueIsNull), values_(values) { - if (arrayOffset < 0) throw IoTDBException("arrayOffset is negative"); - if (positionCount < 0) throw IoTDBException("positionCount is negative"); - if (static_cast(values.size()) - arrayOffset < positionCount) { - throw IoTDBException("values length is less than positionCount"); - } - if (!valueIsNull.empty() && static_cast(valueIsNull.size()) - arrayOffset < positionCount) { - throw IoTDBException("isNull length is less than positionCount"); - } -} - -TSDataType::TSDataType DoubleColumn::getDataType() const { return TSDataType::TSDataType::DOUBLE; } -ColumnEncoding DoubleColumn::getEncoding() const { return ColumnEncoding::Int64Array; } - -double DoubleColumn::getDouble(int32_t position) const { - return values_[position + arrayOffset_]; -} - -std::vector DoubleColumn::getDoubles() const { return values_; } - -bool DoubleColumn::mayHaveNull() const { return !valueIsNull_.empty(); } - -bool DoubleColumn::isNull(int32_t position) const { - return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; -} - -std::vector DoubleColumn::isNulls() const { - if (!valueIsNull_.empty()) return valueIsNull_; - - std::vector result(positionCount_, false); - return result; -} - -int32_t DoubleColumn::getPositionCount() const { return positionCount_; } - -BooleanColumn::BooleanColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values) - : arrayOffset_(arrayOffset), positionCount_(positionCount), - valueIsNull_(valueIsNull), values_(values) { - if (arrayOffset < 0) throw IoTDBException("arrayOffset is negative"); - if (positionCount < 0) throw IoTDBException("positionCount is negative"); - if (static_cast(values.size()) - arrayOffset < positionCount) { - throw IoTDBException("values length is less than positionCount"); - } - if (!valueIsNull.empty() && static_cast(valueIsNull.size()) - arrayOffset < positionCount) { - throw IoTDBException("isNull length is less than positionCount"); - } -} - -TSDataType::TSDataType BooleanColumn::getDataType() const { return TSDataType::TSDataType::BOOLEAN; } -ColumnEncoding BooleanColumn::getEncoding() const { return ColumnEncoding::ByteArray; } - -bool BooleanColumn::getBoolean(int32_t position) const { - return values_[position + arrayOffset_]; -} - -std::vector BooleanColumn::getBooleans() const { return values_; } - -bool BooleanColumn::mayHaveNull() const { return !valueIsNull_.empty(); } - -bool BooleanColumn::isNull(int32_t position) const { - return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; -} - -std::vector BooleanColumn::isNulls() const { - if (!valueIsNull_.empty()) return valueIsNull_; - - std::vector result(positionCount_, false); - return result; -} - -int32_t BooleanColumn::getPositionCount() const { return positionCount_; } - -RunLengthEncodedColumn::RunLengthEncodedColumn(std::shared_ptr value, int32_t positionCount) - : value_(value), positionCount_(positionCount) { - if (!value) throw IoTDBException("value is null"); - if (value->getPositionCount() != 1) { - throw IoTDBException("Expected value to contain a single position"); - } - if (positionCount < 0) throw IoTDBException("positionCount is negative"); -} - -std::shared_ptr RunLengthEncodedColumn::getValue() const { return value_; } - -TSDataType::TSDataType RunLengthEncodedColumn::getDataType() const { return value_->getDataType(); } -ColumnEncoding RunLengthEncodedColumn::getEncoding() const { return ColumnEncoding::Rle; } - -bool RunLengthEncodedColumn::getBoolean(int32_t position) const { - return value_->getBoolean(0); -} - -int32_t RunLengthEncodedColumn::getInt(int32_t position) const { - return value_->getInt(0); -} - -int64_t RunLengthEncodedColumn::getLong(int32_t position) const { - return value_->getLong(0); -} - -float RunLengthEncodedColumn::getFloat(int32_t position) const { - return value_->getFloat(0); -} - -double RunLengthEncodedColumn::getDouble(int32_t position) const { - return value_->getDouble(0); -} - -std::shared_ptr RunLengthEncodedColumn::getBinary(int32_t position) const { - return value_->getBinary(0); -} - -std::vector RunLengthEncodedColumn::getBooleans() const { - bool v = value_->getBoolean(0); - return std::vector(positionCount_, v); -} - -std::vector RunLengthEncodedColumn::getInts() const { - int32_t v = value_->getInt(0); - return std::vector(positionCount_, v); -} - -std::vector RunLengthEncodedColumn::getLongs() const { - int64_t v = value_->getLong(0); - return std::vector(positionCount_, v); -} - -std::vector RunLengthEncodedColumn::getFloats() const { - float v = value_->getFloat(0); - return std::vector(positionCount_, v); -} - -std::vector RunLengthEncodedColumn::getDoubles() const { - double v = value_->getDouble(0); - return std::vector(positionCount_, v); -} - -std::vector> RunLengthEncodedColumn::getBinaries() const { - auto v = value_->getBinary(0); - return std::vector>(positionCount_, v); -} - -bool RunLengthEncodedColumn::mayHaveNull() const { return value_->mayHaveNull(); } - -bool RunLengthEncodedColumn::isNull(int32_t position) const { - return value_->isNull(0); -} - -std::vector RunLengthEncodedColumn::isNulls() const { - bool v = value_->isNull(0); - return std::vector(positionCount_, v); -} - -int32_t RunLengthEncodedColumn::getPositionCount() const { return positionCount_; } diff --git a/iotdb-client/client-cpp/src/main/Column.h b/iotdb-client/client-cpp/src/main/Column.h deleted file mode 100644 index 04f611f41a874..0000000000000 --- a/iotdb-client/client-cpp/src/main/Column.h +++ /dev/null @@ -1,353 +0,0 @@ -/** -* 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. - */ -#ifndef IOTDB_COLUMN_H -#define IOTDB_COLUMN_H - -#include -#include -#include -#include - -#include "Common.h" -#include "ColumnDecoder.h" - -enum class ColumnEncoding : uint8_t { - ByteArray, - Int32Array, - Int64Array, - BinaryArray, - Rle -}; - -class Binary { -public: - explicit Binary(std::vector data) : data_(std::move(data)) { - } - - const std::vector& getData() const { return data_; } - - std::string getStringValue() const { - return {data_.begin(), data_.end()}; - } - -private: - std::vector data_; -}; - - -const std::map> kEncodingToDecoder = { - {ColumnEncoding::Int32Array, std::make_shared()}, - {ColumnEncoding::Int64Array, std::make_shared()}, - {ColumnEncoding::ByteArray, std::make_shared()}, - {ColumnEncoding::BinaryArray, std::make_shared()}, - {ColumnEncoding::Rle, std::make_shared()} -}; - -const std::map kByteToEncoding = { - {0, ColumnEncoding::ByteArray}, - {1, ColumnEncoding::Int32Array}, - {2, ColumnEncoding::Int64Array}, - {3, ColumnEncoding::BinaryArray}, - {4, ColumnEncoding::Rle} -}; - -inline std::shared_ptr getColumnDecoder(ColumnEncoding encoding) { - auto it = kEncodingToDecoder.find(encoding); - if (it == kEncodingToDecoder.end()) { - throw IoTDBException("Unsupported column encoding"); - } - return it->second; -} - -inline ColumnEncoding getColumnEncodingByByte(uint8_t b) { - auto it = kByteToEncoding.find(b); - if (it == kByteToEncoding.end()) { - throw IoTDBException("Invalid encoding value: " + std::to_string(b)); - } - return it->second; -} - -class Column { -public: - virtual ~Column() = default; - - virtual TSDataType::TSDataType getDataType() const = 0; - virtual ColumnEncoding getEncoding() const = 0; - - virtual bool getBoolean(int32_t position) const { - throw IoTDBException("Unsupported operation: getBoolean"); - } - - virtual int32_t getInt(int32_t position) const { - throw IoTDBException("Unsupported operation: getInt"); - } - - virtual int64_t getLong(int32_t position) const { - throw IoTDBException("Unsupported operation: getLong"); - } - - virtual float getFloat(int32_t position) const { - throw IoTDBException("Unsupported operation: getFloat"); - } - - virtual double getDouble(int32_t position) const { - throw IoTDBException("Unsupported operation: getDouble"); - } - - virtual std::shared_ptr getBinary(int32_t position) const { - throw IoTDBException("Unsupported operation: getBinary"); - } - - virtual std::vector getBooleans() const { - throw IoTDBException("Unsupported operation: getBooleans"); - } - - virtual std::vector getInts() const { - throw IoTDBException("Unsupported operation: getInts"); - } - - virtual std::vector getLongs() const { - throw IoTDBException("Unsupported operation: getLongs"); - } - - virtual std::vector getFloats() const { - throw IoTDBException("Unsupported operation: getFloats"); - } - - virtual std::vector getDoubles() const { - throw IoTDBException("Unsupported operation: getDoubles"); - } - - virtual std::vector> getBinaries() const { - throw IoTDBException("Unsupported operation: getBinaries"); - } - - virtual bool mayHaveNull() const = 0; - virtual bool isNull(int32_t position) const = 0; - virtual std::vector isNulls() const = 0; - - virtual int32_t getPositionCount() const = 0; -}; - -class TimeColumn : public Column { -public: - TimeColumn(int32_t arrayOffset, int32_t positionCount, const std::vector& values); - - TSDataType::TSDataType getDataType() const override; - ColumnEncoding getEncoding() const override; - - int64_t getLong(int32_t position) const override; - - bool mayHaveNull() const override; - bool isNull(int32_t position) const override; - std::vector isNulls() const override; - - int32_t getPositionCount() const override; - - int64_t getStartTime() const; - int64_t getEndTime() const; - - const std::vector& getTimes() const; - std::vector getLongs() const override; - -private: - int32_t arrayOffset_; - int32_t positionCount_; - std::vector values_; -}; - -class BinaryColumn : public Column { -public: - BinaryColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector>& values); - - TSDataType::TSDataType getDataType() const override; - ColumnEncoding getEncoding() const override; - - std::shared_ptr getBinary(int32_t position) const override; - std::vector> getBinaries() const override; - - bool mayHaveNull() const override; - bool isNull(int32_t position) const override; - std::vector isNulls() const override; - - int32_t getPositionCount() const override; - -private: - int32_t arrayOffset_; - int32_t positionCount_; - std::vector valueIsNull_; - std::vector> values_; -}; - -class IntColumn : public Column { -public: - IntColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values); - - TSDataType::TSDataType getDataType() const override; - ColumnEncoding getEncoding() const override; - - int32_t getInt(int32_t position) const override; - std::vector getInts() const override; - - bool mayHaveNull() const override; - bool isNull(int32_t position) const override; - std::vector isNulls() const override; - - int32_t getPositionCount() const override; - -private: - int32_t arrayOffset_; - int32_t positionCount_; - std::vector valueNull_; - std::vector values_; -}; - -class FloatColumn : public Column { -public: - FloatColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values); - - TSDataType::TSDataType getDataType() const override; - ColumnEncoding getEncoding() const override; - - float getFloat(int32_t position) const override; - std::vector getFloats() const override; - - bool mayHaveNull() const override; - bool isNull(int32_t position) const override; - std::vector isNulls() const override; - - int32_t getPositionCount() const override; - -private: - int32_t arrayOffset_; - int32_t positionCount_; - std::vector valueIsNull_; - std::vector values_; -}; - -class LongColumn : public Column { -public: - LongColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values); - - TSDataType::TSDataType getDataType() const override; - ColumnEncoding getEncoding() const override; - - int64_t getLong(int32_t position) const override; - std::vector getLongs() const override; - - bool mayHaveNull() const override; - bool isNull(int32_t position) const override; - std::vector isNulls() const override; - - int32_t getPositionCount() const override; - -private: - int32_t arrayOffset_; - int32_t positionCount_; - std::vector valueIsNull_; - std::vector values_; -}; - -class DoubleColumn : public Column { -public: - DoubleColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values); - - TSDataType::TSDataType getDataType() const override; - ColumnEncoding getEncoding() const override; - - double getDouble(int32_t position) const override; - std::vector getDoubles() const override; - - bool mayHaveNull() const override; - bool isNull(int32_t position) const override; - std::vector isNulls() const override; - - int32_t getPositionCount() const override; - -private: - int32_t arrayOffset_; - int32_t positionCount_; - std::vector valueIsNull_; - std::vector values_; -}; - -class BooleanColumn : public Column { -public: - BooleanColumn(int32_t arrayOffset, int32_t positionCount, - const std::vector& valueIsNull, const std::vector& values); - - TSDataType::TSDataType getDataType() const override; - ColumnEncoding getEncoding() const override; - - bool getBoolean(int32_t position) const override; - std::vector getBooleans() const override; - - bool mayHaveNull() const override; - bool isNull(int32_t position) const override; - std::vector isNulls() const override; - - int32_t getPositionCount() const override; - -private: - int32_t arrayOffset_; - int32_t positionCount_; - std::vector valueIsNull_; - std::vector values_; -}; - -class RunLengthEncodedColumn : public Column { -public: - RunLengthEncodedColumn(std::shared_ptr value, int32_t positionCount); - - std::shared_ptr getValue() const; - - TSDataType::TSDataType getDataType() const override; - ColumnEncoding getEncoding() const override; - - bool getBoolean(int32_t position) const override; - int32_t getInt(int32_t position) const override; - int64_t getLong(int32_t position) const override; - float getFloat(int32_t position) const override; - double getDouble(int32_t position) const override; - std::shared_ptr getBinary(int32_t position) const override; - - std::vector getBooleans() const override; - std::vector getInts() const override; - std::vector getLongs() const override; - std::vector getFloats() const override; - std::vector getDoubles() const override; - std::vector> getBinaries() const override; - - bool mayHaveNull() const override; - bool isNull(int32_t position) const override; - std::vector isNulls() const override; - - int32_t getPositionCount() const override; - -private: - std::shared_ptr value_; - int32_t positionCount_; -}; - -#endif diff --git a/iotdb-client/client-cpp/src/main/ColumnDecoder.cpp b/iotdb-client/client-cpp/src/main/ColumnDecoder.cpp deleted file mode 100644 index 32f29d876f368..0000000000000 --- a/iotdb-client/client-cpp/src/main/ColumnDecoder.cpp +++ /dev/null @@ -1,181 +0,0 @@ -/** -* Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -#include "ColumnDecoder.h" - -#include "Column.h" - -std::vector deserializeNullIndicators(MyStringBuffer& buffer, int32_t positionCount) { - uint8_t mayHaveNullByte = buffer.getChar(); - - bool mayHaveNull = mayHaveNullByte != 0; - if (!mayHaveNull) { - return {}; - } - - return deserializeBooleanArray(buffer, positionCount); -} - -std::vector deserializeBooleanArray(MyStringBuffer& buffer, int32_t size) { - const int32_t packedSize = (size + 7) / 8; - std::vector packedBytes(packedSize); - for (int i = 0; i < packedSize; i++) { - packedBytes[i] = buffer.getChar(); - } - - std::vector output(size); - int currentByte = 0; - const int fullGroups = size & ~0b111; - - for (int pos = 0; pos < fullGroups; pos += 8) { - const uint8_t b = packedBytes[currentByte++]; - output[pos + 0] = (b & 0b10000000) != 0; - output[pos + 1] = (b & 0b01000000) != 0; - output[pos + 2] = (b & 0b00100000) != 0; - output[pos + 3] = (b & 0b00010000) != 0; - output[pos + 4] = (b & 0b00001000) != 0; - output[pos + 5] = (b & 0b00000100) != 0; - output[pos + 6] = (b & 0b00000010) != 0; - output[pos + 7] = (b & 0b00000001) != 0; - } - - if ((size & 0b111) > 0) { - const uint8_t b = packedBytes.back(); - uint8_t mask = 0b10000000; - - for (int pos = fullGroups; pos < size; pos++) { - output[pos] = (b & mask) != 0; - mask >>= 1; - } - } - - return output; -} - -std::unique_ptr BaseColumnDecoder::readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) { - return nullptr; -} - -std::unique_ptr Int32ArrayColumnDecoder::readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) { - auto nullIndicators = deserializeNullIndicators(buffer, positionCount); - - switch (dataType) { - case TSDataType::INT32: - case TSDataType::DATE: { - std::vector intValues(positionCount); - for (int32_t i = 0; i < positionCount; i++) { - if (!nullIndicators.empty() && nullIndicators[i]) continue; - intValues[i] = buffer.getInt(); - } - return std::unique_ptr(new IntColumn(0, positionCount, nullIndicators, intValues)); - } - case TSDataType::FLOAT: { - std::vector floatValues(positionCount); - for (int32_t i = 0; i < positionCount; i++) { - if (!nullIndicators.empty() && nullIndicators[i]) continue; - floatValues[i] = buffer.getFloat(); - } - return std::unique_ptr(new FloatColumn(0, positionCount, nullIndicators, floatValues)); - } - default: - throw IoTDBException("Invalid data type for Int32ArrayColumnDecoder"); - } -} - -std::unique_ptr Int64ArrayColumnDecoder::readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) { - auto nullIndicators = deserializeNullIndicators(buffer, positionCount); - - switch (dataType) { - case TSDataType::INT64: - case TSDataType::TIMESTAMP: { - std::vector values(positionCount); - for (int32_t i = 0; i < positionCount; i++) { - if (!nullIndicators.empty() && nullIndicators[i]) continue; - values[i] = buffer.getInt64(); - } - return std::unique_ptr(new LongColumn(0, positionCount, nullIndicators, values)); - } - case TSDataType::DOUBLE: { - std::vector values(positionCount); - for (int32_t i = 0; i < positionCount; i++) { - if (!nullIndicators.empty() && nullIndicators[i]) continue; - values[i] = buffer.getDouble(); - } - return std::unique_ptr(new DoubleColumn(0, positionCount, nullIndicators, values)); - } - default: - throw IoTDBException("Invalid data type for Int64ArrayColumnDecoder"); - } -} - -std::unique_ptr ByteArrayColumnDecoder::readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) { - if (dataType != TSDataType::BOOLEAN) { - throw IoTDBException("Invalid data type for ByteArrayColumnDecoder"); - } - - auto nullIndicators = deserializeNullIndicators(buffer, positionCount); - auto values = deserializeBooleanArray(buffer, positionCount); - return std::unique_ptr(new BooleanColumn(0, positionCount, nullIndicators, values)); -} - -std::unique_ptr BinaryArrayColumnDecoder::readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) { - if (dataType != TSDataType::TEXT) { - throw IoTDBException("Invalid data type for BinaryArrayColumnDecoder"); - } - - auto nullIndicators = deserializeNullIndicators(buffer, positionCount); - std::vector> values(positionCount); - - for (int32_t i = 0; i < positionCount; i++) { - if (!nullIndicators.empty() && nullIndicators[i]) continue; - - int32_t length = buffer.getInt(); - if (length < 0) { - throw IoTDBException("BinaryArrayColumnDecoder: negative TEXT length"); - } - - std::vector value(length); - for (int32_t j = 0; j < length; j++) { - value[j] = buffer.getChar(); - } - - values[i] = std::make_shared(value); - } - - return std::unique_ptr(new BinaryColumn(0, positionCount, nullIndicators, values)); -} - -std::unique_ptr RunLengthColumnDecoder::readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) { - uint8_t encodingByte = buffer.getChar(); - - auto columnEncoding = static_cast(encodingByte); - auto decoder = getColumnDecoder(columnEncoding); - - auto column = decoder->readColumn(buffer, dataType, 1); - if (!column) { - throw IoTDBException("Failed to read inner column"); - } - return std::unique_ptr(new RunLengthEncodedColumn(move(column), positionCount)); -} diff --git a/iotdb-client/client-cpp/src/main/ColumnDecoder.h b/iotdb-client/client-cpp/src/main/ColumnDecoder.h deleted file mode 100644 index f5340d1e400ac..0000000000000 --- a/iotdb-client/client-cpp/src/main/ColumnDecoder.h +++ /dev/null @@ -1,75 +0,0 @@ -/** -* 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. - */ -#ifndef IOTDB_COLUMN_DECODER_H -#define IOTDB_COLUMN_DECODER_H - -#include -#include - -#include "Common.h" - -class Column; - -class ColumnDecoder { -public: - virtual ~ColumnDecoder() = default; - virtual std::unique_ptr readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) = 0; -}; - -std::vector deserializeNullIndicators(MyStringBuffer& buffer, int32_t positionCount); -std::vector deserializeBooleanArray(MyStringBuffer& buffer, int32_t size); - -class BaseColumnDecoder : public ColumnDecoder { -public: - std::unique_ptr readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) override; -}; - -class Int32ArrayColumnDecoder : public BaseColumnDecoder { -public: - std::unique_ptr readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) override; -}; - -class Int64ArrayColumnDecoder : public BaseColumnDecoder { -public: - std::unique_ptr readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) override; -}; - -class ByteArrayColumnDecoder : public BaseColumnDecoder { -public: - std::unique_ptr readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) override; -}; - -class BinaryArrayColumnDecoder : public BaseColumnDecoder { -public: - std::unique_ptr readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) override; -}; - -class RunLengthColumnDecoder : public BaseColumnDecoder { -public: - std::unique_ptr readColumn( - MyStringBuffer& buffer, TSDataType::TSDataType dataType, int32_t positionCount) override; -}; - -#endif diff --git a/iotdb-client/client-cpp/src/main/Common.cpp b/iotdb-client/client-cpp/src/main/Common.cpp deleted file mode 100644 index 91cd806d08b55..0000000000000 --- a/iotdb-client/client-cpp/src/main/Common.cpp +++ /dev/null @@ -1,498 +0,0 @@ -/** -* Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -#include "Common.h" -#include -#include - -int32_t parseDateExpressionToInt(const boost::gregorian::date& date) { - if (date.is_not_a_date()) { - throw IoTDBException("Date expression is null or empty."); - } - - const int year = date.year(); - if (year < 1000 || year > 9999) { - throw DateTimeParseException( - "Year must be between 1000 and 9999.", - boost::gregorian::to_iso_extended_string(date), - 0 - ); - } - - const int64_t result = static_cast(year) * 10000 + - date.month() * 100 + - date.day(); - if (result > INT32_MAX || result < INT32_MIN) { - throw DateTimeParseException( - "Date value overflow. ", - boost::gregorian::to_iso_extended_string(date), - 0 - ); - } - return static_cast(result); -} - -boost::gregorian::date parseIntToDate(int32_t dateInt) { - if (dateInt == EMPTY_DATE_INT) { - return boost::gregorian::date(boost::date_time::not_a_date_time); - } - int year = dateInt / 10000; - int month = (dateInt % 10000) / 100; - int day = dateInt % 100; - return boost::gregorian::date(year, month, day); -} - -std::string getTimePrecision(int32_t timeFactor) { - if (timeFactor >= 1000000) return "us"; - if (timeFactor >= 1000) return "ms"; - return "s"; -} - -std::string formatDatetime(const std::string& format, const std::string& precision, - int64_t timestamp, const std::string& zoneId) { - // Simplified implementation - in real code you'd use proper timezone handling - std::time_t time = static_cast(timestamp); - std::tm* tm = std::localtime(&time); - char buffer[80]; - strftime(buffer, sizeof(buffer), format.c_str(), tm); - return std::string(buffer); -} - -std::tm convertToTimestamp(int64_t value, int32_t timeFactor) { - std::time_t time = static_cast(value / timeFactor); - return *std::localtime(&time); -} - -TSDataType::TSDataType getDataTypeByStr(const std::string& typeStr) { - if (typeStr == "BOOLEAN") return TSDataType::BOOLEAN; - if (typeStr == "INT32") return TSDataType::INT32; - if (typeStr == "INT64") return TSDataType::INT64; - if (typeStr == "FLOAT") return TSDataType::FLOAT; - if (typeStr == "DOUBLE") return TSDataType::DOUBLE; - if (typeStr == "TEXT") return TSDataType::TEXT; - if (typeStr == "TIMESTAMP") return TSDataType::TIMESTAMP; - if (typeStr == "DATE") return TSDataType::DATE; - if (typeStr == "BLOB") return TSDataType::BLOB; - if (typeStr == "STRING") return TSDataType::STRING; - return TSDataType::UNKNOWN; -} - -std::tm int32ToDate(int32_t value) { - // Convert days since epoch (1970-01-01) to tm struct - std::time_t time = static_cast(value) * 86400; // seconds per day - return *std::localtime(&time); -} - -void RpcUtils::verifySuccess(const TSStatus& status) { - if (status.code == TSStatusCode::MULTIPLE_ERROR) { - verifySuccess(status.subStatus); - return; - } - if (status.code != TSStatusCode::SUCCESS_STATUS - && status.code != TSStatusCode::REDIRECTION_RECOMMEND) { - throw ExecutionException(to_string(status.code) + ": " + status.message, status); - } -} - -void RpcUtils::verifySuccessWithRedirection(const TSStatus& status) { - verifySuccess(status); - if (status.__isset.redirectNode) { - throw RedirectException(to_string(status.code) + ": " + status.message, status.redirectNode); - } - if (status.__isset.subStatus) { - auto statusSubStatus = status.subStatus; - vector endPointList(statusSubStatus.size()); - int count = 0; - for (TSStatus subStatus : statusSubStatus) { - if (subStatus.__isset.redirectNode) { - endPointList[count++] = subStatus.redirectNode; - } - else { - TEndPoint endPoint; - endPointList[count++] = endPoint; - } - } - if (!endPointList.empty()) { - throw RedirectException(to_string(status.code) + ": " + status.message, endPointList); - } - } -} - -void RpcUtils::verifySuccessWithRedirectionForMultiDevices(const TSStatus& status, vector devices) { - verifySuccess(status); - - if (status.code == TSStatusCode::MULTIPLE_ERROR - || status.code == TSStatusCode::REDIRECTION_RECOMMEND) { - map deviceEndPointMap; - vector statusSubStatus; - for (int i = 0; i < statusSubStatus.size(); i++) { - TSStatus subStatus = statusSubStatus[i]; - if (subStatus.__isset.redirectNode) { - deviceEndPointMap.insert(make_pair(devices[i], subStatus.redirectNode)); - } - } - throw RedirectException(to_string(status.code) + ": " + status.message, deviceEndPointMap); - } - - if (status.__isset.redirectNode) { - throw RedirectException(to_string(status.code) + ": " + status.message, status.redirectNode); - } - if (status.__isset.subStatus) { - auto statusSubStatus = status.subStatus; - vector endPointList(statusSubStatus.size()); - int count = 0; - for (TSStatus subStatus : statusSubStatus) { - if (subStatus.__isset.redirectNode) { - endPointList[count++] = subStatus.redirectNode; - } - else { - TEndPoint endPoint; - endPointList[count++] = endPoint; - } - } - if (!endPointList.empty()) { - throw RedirectException(to_string(status.code) + ": " + status.message, endPointList); - } - } -} - -void RpcUtils::verifySuccess(const vector& statuses) { - for (const TSStatus& status : statuses) { - if (status.code != TSStatusCode::SUCCESS_STATUS) { - throw BatchExecutionException(status.message, statuses); - } - } -} - -TSStatus RpcUtils::getStatus(TSStatusCode::TSStatusCode tsStatusCode) { - TSStatus status; - status.__set_code(tsStatusCode); - return status; -} - -TSStatus RpcUtils::getStatus(int code, const string& message) { - TSStatus status; - status.__set_code(code); - status.__set_message(message); - return status; -} - -shared_ptr RpcUtils::getTSExecuteStatementResp(TSStatusCode::TSStatusCode tsStatusCode) { - TSStatus status = getStatus(tsStatusCode); - return getTSExecuteStatementResp(status); -} - -shared_ptr -RpcUtils::getTSExecuteStatementResp(TSStatusCode::TSStatusCode tsStatusCode, const string& message) { - TSStatus status = getStatus(tsStatusCode, message); - return getTSExecuteStatementResp(status); -} - -shared_ptr RpcUtils::getTSExecuteStatementResp(const TSStatus& status) { - shared_ptr resp(new TSExecuteStatementResp()); - TSStatus tsStatus(status); - resp->__set_status(status); - return resp; -} - -shared_ptr RpcUtils::getTSFetchResultsResp(TSStatusCode::TSStatusCode tsStatusCode) { - TSStatus status = getStatus(tsStatusCode); - return getTSFetchResultsResp(status); -} - -shared_ptr -RpcUtils::getTSFetchResultsResp(TSStatusCode::TSStatusCode tsStatusCode, const string& appendMessage) { - TSStatus status = getStatus(tsStatusCode, appendMessage); - return getTSFetchResultsResp(status); -} - -shared_ptr RpcUtils::getTSFetchResultsResp(const TSStatus& status) { - shared_ptr resp(new TSFetchResultsResp()); - TSStatus tsStatus(status); - resp->__set_status(tsStatus); - return resp; -} - -MyStringBuffer::MyStringBuffer() : pos(0) { - checkBigEndian(); -} - -MyStringBuffer::MyStringBuffer(const std::string& str) : str(str), pos(0) { - checkBigEndian(); -} - -void MyStringBuffer::reserve(size_t n) { - str.reserve(n); -} - -void MyStringBuffer::clear() { - str.clear(); - pos = 0; -} - -bool MyStringBuffer::hasRemaining() { - return pos < str.size(); -} - -int MyStringBuffer::getInt() { - return *(int*)getOrderedByte(4); -} - -boost::gregorian::date MyStringBuffer::getDate() { - return parseIntToDate(getInt()); -} - -int64_t MyStringBuffer::getInt64() { -#ifdef ARCH32 - const char *buf_addr = getOrderedByte(8); - if (reinterpret_cast(buf_addr) % 4 == 0) { - return *(int64_t *)buf_addr; - } else { - char tmp_buf[8]; - memcpy(tmp_buf, buf_addr, 8); - return *(int64_t*)tmp_buf; - } -#else - return *(int64_t*)getOrderedByte(8); -#endif -} - -float MyStringBuffer::getFloat() { - return *(float*)getOrderedByte(4); -} - -double MyStringBuffer::getDouble() { -#ifdef ARCH32 - const char *buf_addr = getOrderedByte(8); - if (reinterpret_cast(buf_addr) % 4 == 0) { - return *(double*)buf_addr; - } else { - char tmp_buf[8]; - memcpy(tmp_buf, buf_addr, 8); - return *(double*)tmp_buf; - } -#else - return *(double*)getOrderedByte(8); -#endif -} - -char MyStringBuffer::getChar() { - if (pos >= str.size()) { - throw IoTDBException("MyStringBuffer::getChar: read past end (pos=" + std::to_string(pos) + - ", size=" + std::to_string(str.size()) + ")"); - } - return str[pos++]; -} - -bool MyStringBuffer::getBool() { - return getChar() == 1; -} - -std::string MyStringBuffer::getString() { - const int lenInt = getInt(); - if (lenInt < 0) { - throw IoTDBException("MyStringBuffer::getString: negative length"); - } - const size_t len = static_cast(lenInt); - if (pos > str.size() || len > str.size() - pos) { - throw IoTDBException("MyStringBuffer::getString: length exceeds buffer (pos=" + std::to_string(pos) + - ", len=" + std::to_string(len) + ", size=" + std::to_string(str.size()) + ")"); - } - const size_t tmpPos = pos; - pos += len; - return str.substr(tmpPos, len); -} - -void MyStringBuffer::putInt(int ins) { - putOrderedByte((char*)&ins, 4); -} - -void MyStringBuffer::putDate(boost::gregorian::date date) { - putInt(parseDateExpressionToInt(date)); -} - -void MyStringBuffer::putInt64(int64_t ins) { - putOrderedByte((char*)&ins, 8); -} - -void MyStringBuffer::putFloat(float ins) { - putOrderedByte((char*)&ins, 4); -} - -void MyStringBuffer::putDouble(double ins) { - putOrderedByte((char*)&ins, 8); -} - -void MyStringBuffer::putChar(char ins) { - str += ins; -} - -void MyStringBuffer::putBool(bool ins) { - char tmp = ins ? 1 : 0; - str += tmp; -} - -void MyStringBuffer::putString(const std::string& ins) { - putInt((int)(ins.size())); - str += ins; -} - -void MyStringBuffer::concat(const std::string& ins) { - str.append(ins); -} - -void MyStringBuffer::checkBigEndian() { - static int chk = 0x0201; //used to distinguish CPU's type (BigEndian or LittleEndian) - isBigEndian = (0x01 != *(char*)(&chk)); -} - -const char* MyStringBuffer::getOrderedByte(size_t len) { - if (pos > str.size() || len > str.size() - pos) { - throw IoTDBException("MyStringBuffer::getOrderedByte: read past end (pos=" + std::to_string(pos) + - ", len=" + std::to_string(len) + ", size=" + std::to_string(str.size()) + ")"); - } - const char* p = nullptr; - if (isBigEndian) { - p = str.c_str() + pos; - } - else { - const char* tmp = str.c_str(); - for (size_t i = pos; i < pos + len; i++) { - numericBuf[pos + len - 1 - i] = tmp[i]; - } - p = numericBuf; - } - pos += len; - return p; -} - -void MyStringBuffer::putOrderedByte(char* buf, int len) { - if (isBigEndian) { - str.append(buf, len); - } - else { - for (int i = len - 1; i > -1; i--) { - str += buf[i]; - } - } -} - -BitMap::BitMap(size_t size) { - resize(size); -} - -void BitMap::resize(size_t size) { - this->size = size; - this->bits.resize((size >> 3) + 1); // equal to "size/8 + 1" - reset(); -} - -bool BitMap::mark(size_t position) { - if (position >= size) - return false; - - bits[position >> 3] |= (char)1 << (position % 8); - return true; -} - -bool BitMap::unmark(size_t position) { - if (position >= size) - return false; - - bits[position >> 3] &= ~((char)1 << (position % 8)); - return true; -} - -void BitMap::markAll() { - std::fill(bits.begin(), bits.end(), (char)0XFF); -} - -void BitMap::reset() { - std::fill(bits.begin(), bits.end(), (char)0); -} - -bool BitMap::isMarked(size_t position) const { - if (position >= size) - return false; - - return (bits[position >> 3] & ((char)1 << (position % 8))) != 0; -} - -bool BitMap::isAllUnmarked() const { - size_t j; - for (j = 0; j < size >> 3; j++) { - if (bits[j] != (char)0) { - return false; - } - } - for (j = 0; j < size % 8; j++) { - if ((bits[size >> 3] & ((char)1 << j)) != 0) { - return false; - } - } - return true; -} - -bool BitMap::isAllMarked() const { - size_t j; - for (j = 0; j < size >> 3; j++) { - if (bits[j] != (char)0XFF) { - return false; - } - } - for (j = 0; j < size % 8; j++) { - if ((bits[size >> 3] & ((char)1 << j)) == 0) { - return false; - } - } - return true; -} - -const std::vector& BitMap::getByteArray() const { - return this->bits; -} - -size_t BitMap::getSize() const { - return this->size; -} - -TEndPoint UrlUtils::parseTEndPointIpv4AndIpv6Url(const std::string& endPointUrl) { - TEndPoint endPoint; - const size_t colonPos = endPointUrl.find_last_of(':'); - if (colonPos == std::string::npos) { - endPoint.__set_ip(endPointUrl); - endPoint.__set_port(0); - return endPoint; - } - std::string ip = endPointUrl.substr(0, colonPos); - const std::string portStr = endPointUrl.substr(colonPos + 1); - try { - const int port = std::stoi(portStr); - endPoint.__set_port(port); - } catch (const std::logic_error&) { - endPoint.__set_ip(endPointUrl); - endPoint.__set_port(0); - return endPoint; - } - if (ip.size() >= 2 && ip.front() == '[' && ip.back() == ']') { - ip = ip.substr(1, ip.size() - 2); - } - endPoint.__set_ip(ip); - return endPoint; -} diff --git a/iotdb-client/client-cpp/src/main/Common.h b/iotdb-client/client-cpp/src/main/Common.h deleted file mode 100644 index af9cf46e62e54..0000000000000 --- a/iotdb-client/client-cpp/src/main/Common.h +++ /dev/null @@ -1,492 +0,0 @@ -/** -* 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. - */ -#ifndef IOTDB_COMMON_H -#define IOTDB_COMMON_H - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include - -#include "client_types.h" -#include "common_types.h" - -using namespace std; - -using ::apache::thrift::protocol::TBinaryProtocol; -using ::apache::thrift::protocol::TCompactProtocol; -using ::apache::thrift::transport::TSocket; -using ::apache::thrift::transport::TTransport; -using ::apache::thrift::transport::TTransportException; -using ::apache::thrift::transport::TBufferedTransport; -using ::apache::thrift::transport::TFramedTransport; -using ::apache::thrift::TException; - -using namespace std; - -constexpr int32_t EMPTY_DATE_INT = 10000101; - -int32_t parseDateExpressionToInt(const boost::gregorian::date& date); -boost::gregorian::date parseIntToDate(int32_t dateInt); - -std::string getTimePrecision(int32_t timeFactor); - -std::string formatDatetime(const std::string& format, const std::string& precision, - int64_t timestamp, const std::string& zoneId); - -std::tm convertToTimestamp(int64_t value, int32_t timeFactor); -std::tm int32ToDate(int32_t value); - -namespace Version { -enum Version { - V_0_12, V_0_13, V_1_0 -}; -} - -namespace CompressionType { -enum CompressionType { - UNCOMPRESSED = (char)0, - SNAPPY = (char)1, - GZIP = (char)2, - LZO = (char)3, - SDT = (char)4, - PAA = (char)5, - PLA = (char)6, - LZ4 = (char)7, - ZSTD = (char)8, - LZMA2 = (char)9, -}; -} - -namespace TSDataType { -enum TSDataType { - BOOLEAN = (char)0, - INT32 = (char)1, - INT64 = (char)2, - FLOAT = (char)3, - DOUBLE = (char)4, - TEXT = (char)5, - VECTOR = (char)6, - UNKNOWN = (char)7, - TIMESTAMP = (char)8, - DATE = (char)9, - BLOB = (char)10, - STRING = (char)11 -}; -} - -TSDataType::TSDataType getDataTypeByStr(const std::string& typeStr); - -namespace TSEncoding { -enum TSEncoding { - PLAIN = (char)0, - DICTIONARY = (char)1, - RLE = (char)2, - DIFF = (char)3, - TS_2DIFF = (char)4, - BITMAP = (char)5, - GORILLA_V1 = (char)6, - REGULAR = (char)7, - GORILLA = (char)8, - ZIGZAG = (char)9, - FREQ = (char)10, - INVALID_ENCODING = (char)255 -}; -} - -namespace TSStatusCode { -enum TSStatusCode { - SUCCESS_STATUS = 200, - - // System level - INCOMPATIBLE_VERSION = 201, - CONFIGURATION_ERROR = 202, - START_UP_ERROR = 203, - SHUT_DOWN_ERROR = 204, - - // General Error - UNSUPPORTED_OPERATION = 300, - EXECUTE_STATEMENT_ERROR = 301, - MULTIPLE_ERROR = 302, - ILLEGAL_PARAMETER = 303, - OVERLAP_WITH_EXISTING_TASK = 304, - INTERNAL_SERVER_ERROR = 305, - - // Client, - REDIRECTION_RECOMMEND = 400, - - // Schema Engine - DATABASE_NOT_EXIST = 500, - DATABASE_ALREADY_EXISTS = 501, - SERIES_OVERFLOW = 502, - TIMESERIES_ALREADY_EXIST = 503, - TIMESERIES_IN_BLACK_LIST = 504, - ALIAS_ALREADY_EXIST = 505, - PATH_ALREADY_EXIST = 506, - METADATA_ERROR = 507, - PATH_NOT_EXIST = 508, - ILLEGAL_PATH = 509, - CREATE_TEMPLATE_ERROR = 510, - DUPLICATED_TEMPLATE = 511, - UNDEFINED_TEMPLATE = 512, - TEMPLATE_NOT_SET = 513, - DIFFERENT_TEMPLATE = 514, - TEMPLATE_IS_IN_USE = 515, - TEMPLATE_INCOMPATIBLE = 516, - SEGMENT_NOT_FOUND = 517, - PAGE_OUT_OF_SPACE = 518, - RECORD_DUPLICATED = 519, - SEGMENT_OUT_OF_SPACE = 520, - PBTREE_FILE_NOT_EXISTS = 521, - OVERSIZE_RECORD = 522, - PBTREE_FILE_REDO_LOG_BROKEN = 523, - TEMPLATE_NOT_ACTIVATED = 524, - - // Storage Engine - SYSTEM_READ_ONLY = 600, - STORAGE_ENGINE_ERROR = 601, - STORAGE_ENGINE_NOT_READY = 602, - - // Query Engine - PLAN_FAILED_NETWORK_PARTITION = 721 -}; -} - -class Field { -public: - TSDataType::TSDataType dataType = TSDataType::UNKNOWN; - bool boolV{}; - int intV{}; - boost::gregorian::date dateV; - int64_t longV{}; - float floatV{}; - double doubleV{}; - std::string stringV; - - explicit Field(TSDataType::TSDataType a) { - dataType = a; - } - - Field() = default; -}; - -enum class ColumnCategory { - TAG, - FIELD, - ATTRIBUTE -}; - -class MyStringBuffer { -public: - MyStringBuffer(); - explicit MyStringBuffer(const std::string& str); - - void reserve(size_t n); - void clear(); - bool hasRemaining(); - int getInt(); - boost::gregorian::date getDate(); - int64_t getInt64(); - float getFloat(); - double getDouble(); - char getChar(); - bool getBool(); - std::string getString(); - - void putInt(int ins); - void putDate(boost::gregorian::date date); - void putInt64(int64_t ins); - void putFloat(float ins); - void putDouble(double ins); - void putChar(char ins); - void putBool(bool ins); - void putString(const std::string& ins); - void concat(const std::string& ins); - -public: - std::string str; - size_t pos; - -private: - void checkBigEndian(); - const char* getOrderedByte(size_t len); - void putOrderedByte(char* buf, int len); - -private: - bool isBigEndian{}; - char numericBuf[8]{}; //only be used by int, long, float, double etc. -}; - -class BitMap { -public: - explicit BitMap(size_t size = 0); - void resize(size_t size); - bool mark(size_t position); - bool unmark(size_t position); - void markAll(); - void reset(); - bool isMarked(size_t position) const; - bool isAllUnmarked() const; - bool isAllMarked() const; - const std::vector& getByteArray() const; - size_t getSize() const; - -private: - size_t size; - std::vector bits; -}; - -class IoTDBException : public std::exception { -public: - IoTDBException() = default; - - explicit IoTDBException(const std::string& m) : message(m) { - } - - explicit IoTDBException(const char* m) : message(m) { - } - - virtual const char* what() const noexcept override { - return message.c_str(); - } - -private: - std::string message; -}; - -class DateTimeParseException : public IoTDBException { -private: - std::string parsedString; - int errorIndex; - -public: - explicit DateTimeParseException(const std::string& message, - std::string parsedData, - int errorIndex) - : IoTDBException(message), - parsedString(std::move(parsedData)), - errorIndex(errorIndex) { - } - - explicit DateTimeParseException(const std::string& message, - std::string parsedData, - int errorIndex, - const std::exception& cause) - : IoTDBException(message + " [Caused by: " + cause.what() + "]"), - parsedString(std::move(parsedData)), - errorIndex(errorIndex) { - } - - const std::string& getParsedString() const noexcept { - return parsedString; - } - - int getErrorIndex() const noexcept { - return errorIndex; - } - - const char* what() const noexcept override { - static std::string fullMsg; - fullMsg = std::string(IoTDBException::what()) + - "\nParsed data: " + parsedString + - "\nError index: " + std::to_string(errorIndex); - return fullMsg.c_str(); - } -}; - -class IoTDBConnectionException : public IoTDBException { -public: - IoTDBConnectionException() { - } - - explicit IoTDBConnectionException(const char* m) : IoTDBException(m) { - } - - explicit IoTDBConnectionException(const std::string& m) : IoTDBException(m) { - } -}; - -class ExecutionException : public IoTDBException { -public: - ExecutionException() { - } - - explicit ExecutionException(const char* m) : IoTDBException(m) { - } - - explicit ExecutionException(const std::string& m) : IoTDBException(m) { - } - - explicit ExecutionException(const std::string& m, const TSStatus& tsStatus) : IoTDBException(m), status(tsStatus) { - } - - TSStatus status; -}; - -class BatchExecutionException : public IoTDBException { -public: - BatchExecutionException() { - } - - explicit BatchExecutionException(const char* m) : IoTDBException(m) { - } - - explicit BatchExecutionException(const std::string& m) : IoTDBException(m) { - } - - explicit BatchExecutionException(const std::vector& statusList) : statusList(statusList) { - } - - BatchExecutionException(const std::string& m, const std::vector& statusList) : IoTDBException(m), - statusList(statusList) { - } - - std::vector statusList; -}; - -class RedirectException : public IoTDBException { -public: - RedirectException() { - } - - explicit RedirectException(const char* m) : IoTDBException(m) { - } - - explicit RedirectException(const std::string& m) : IoTDBException(m) { - } - - RedirectException(const std::string& m, const TEndPoint& endPoint) : IoTDBException(m), endPoint(endPoint) { - } - - RedirectException(const std::string& m, const map& deviceEndPointMap) : IoTDBException(m), - deviceEndPointMap(deviceEndPointMap) { - } - - RedirectException(const std::string& m, const vector& endPointList) : IoTDBException(m), - endPointList(endPointList) { - } - - TEndPoint endPoint; - map deviceEndPointMap; - vector endPointList; -}; - -class UnSupportedDataTypeException : public IoTDBException { -public: - UnSupportedDataTypeException() { - } - - explicit UnSupportedDataTypeException(const char* m) : IoTDBException(m) { - } - - explicit UnSupportedDataTypeException(const std::string& m) : IoTDBException("UnSupported dataType: " + m) { - } -}; - -class SchemaNotFoundException : public IoTDBException { -public: - SchemaNotFoundException() { - } - - explicit SchemaNotFoundException(const char* m) : IoTDBException(m) { - } - - explicit SchemaNotFoundException(const std::string& m) : IoTDBException(m) { - } -}; - -class StatementExecutionException : public IoTDBException { -public: - StatementExecutionException() { - } - - explicit StatementExecutionException(const char* m) : IoTDBException(m) { - } - - explicit StatementExecutionException(const std::string& m) : IoTDBException(m) { - } -}; - -enum LogLevelType { - LEVEL_DEBUG = 0, - LEVEL_INFO, - LEVEL_WARN, - LEVEL_ERROR -}; - -extern LogLevelType LOG_LEVEL; - -#define log_debug(fmt,...) do {if(LOG_LEVEL <= LEVEL_DEBUG) {string s=string("[DEBUG] %s:%d (%s) - ") + fmt + "\n"; printf(s.c_str(), __FILE__, __LINE__, __FUNCTION__, ##__VA_ARGS__);}} while(0) -#define log_info(fmt,...) do {if(LOG_LEVEL <= LEVEL_INFO) {string s=string("[INFO] %s:%d (%s) - ") + fmt + "\n"; printf(s.c_str(), __FILE__, __LINE__, __FUNCTION__, ##__VA_ARGS__);}} while(0) -#define log_warn(fmt,...) do {if(LOG_LEVEL <= LEVEL_WARN) {string s=string("[WARN] %s:%d (%s) - ") + fmt + "\n"; printf(s.c_str(), __FILE__, __LINE__, __FUNCTION__, ##__VA_ARGS__);}} while(0) -#define log_error(fmt,...) do {if(LOG_LEVEL <= LEVEL_ERROR) {string s=string("[ERROR] %s:%d (%s) - ") + fmt + "\n"; printf(s.c_str(), __FILE__, __LINE__, __FUNCTION__, ##__VA_ARGS__);}} while(0) - -class RpcUtils { -public: - std::shared_ptr SUCCESS_STATUS; - - RpcUtils() { - SUCCESS_STATUS = std::make_shared(); - SUCCESS_STATUS->__set_code(TSStatusCode::SUCCESS_STATUS); - } - - static void verifySuccess(const TSStatus& status); - - static void verifySuccessWithRedirection(const TSStatus& status); - - static void verifySuccessWithRedirectionForMultiDevices(const TSStatus& status, vector devices); - - static void verifySuccess(const std::vector& statuses); - - static TSStatus getStatus(TSStatusCode::TSStatusCode tsStatusCode); - - static TSStatus getStatus(int code, const std::string& message); - - static std::shared_ptr getTSExecuteStatementResp(TSStatusCode::TSStatusCode tsStatusCode); - - static std::shared_ptr - getTSExecuteStatementResp(TSStatusCode::TSStatusCode tsStatusCode, const std::string& message); - - static std::shared_ptr getTSExecuteStatementResp(const TSStatus& status); - - static std::shared_ptr getTSFetchResultsResp(TSStatusCode::TSStatusCode tsStatusCode); - - static std::shared_ptr - getTSFetchResultsResp(TSStatusCode::TSStatusCode tsStatusCode, const std::string& appendMessage); - - static std::shared_ptr getTSFetchResultsResp(const TSStatus& status); -}; - -class UrlUtils { -public: - UrlUtils() = delete; - - /** Parse host:port; aligns with Java UrlUtils.parseTEndPointIpv4AndIpv6Url plus test edge cases. */ - static TEndPoint parseTEndPointIpv4AndIpv6Url(const std::string& endPointUrl); -}; - - -#endif diff --git a/iotdb-client/client-cpp/src/main/DeviceID.h b/iotdb-client/client-cpp/src/main/DeviceID.h deleted file mode 100644 index df2682cd5199e..0000000000000 --- a/iotdb-client/client-cpp/src/main/DeviceID.h +++ /dev/null @@ -1,161 +0,0 @@ -/** -* 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. - */ -#ifndef IOTDB_DEVICEID_H -#define IOTDB_DEVICEID_H - -#include -#include -#include -#include -#include - -namespace storage { - -static const int DEFAULT_SEGMENT_NUM_FOR_TABLE_NAME = 3; -static const std::string PATH_SEPARATOR = "."; - -class IDeviceID { -public: - virtual ~IDeviceID() = default; - virtual std::string get_table_name() { return ""; } - virtual int segment_num() { return 0; } - virtual const std::vector& get_segments() const { - return empty_segments_; - } - virtual std::string get_device_name() const { return ""; }; - virtual bool operator<(const IDeviceID& other) { return 0; } - virtual bool operator==(const IDeviceID& other) { return false; } - virtual bool operator!=(const IDeviceID& other) { return false; } - -protected: - IDeviceID() : empty_segments_() {} - -private: - const std::vector empty_segments_; -}; - -struct IDeviceIDComparator { - bool operator()(const std::shared_ptr& lhs, - const std::shared_ptr& rhs) const { - return *lhs < *rhs; - } -}; - -class StringArrayDeviceID : public IDeviceID { -public: - explicit StringArrayDeviceID(const std::vector& segments) - : segments_(formalize(segments)) {} - - explicit StringArrayDeviceID() : segments_() {} - - ~StringArrayDeviceID() override = default; - - std::string get_device_name() const override { - return segments_.empty() ? "" : std::accumulate(std::next(segments_.begin()), segments_.end(), - segments_.front(), - [](std::string a, const std::string& b) { - return std::move(a) + "." + b; - }); - }; - - std::string get_table_name() override { - return segments_.empty() ? "" : segments_[0]; - } - - int segment_num() override { return static_cast(segments_.size()); } - - const std::vector& get_segments() const override { - return segments_; - } - - bool operator<(const IDeviceID& other) override { - auto other_segments = other.get_segments(); - return std::lexicographical_compare(segments_.begin(), segments_.end(), - other_segments.begin(), - other_segments.end()); - } - - bool operator==(const IDeviceID& other) override { - auto other_segments = other.get_segments(); - return (segments_.size() == other_segments.size()) && - std::equal(segments_.begin(), segments_.end(), - other_segments.begin()); - } - - bool operator!=(const IDeviceID& other) override { - return !(*this == other); - } - -private: - std::vector segments_; - - std::vector formalize( - const std::vector& segments) { - auto it = - std::find_if(segments.rbegin(), segments.rend(), - [](const std::string& seg) { return !seg.empty(); }); - return std::vector(segments.begin(), it.base()); - } - - std::vector split_device_id_string( - const std::vector& splits) { - size_t segment_cnt = splits.size(); - std::vector final_segments; - - if (segment_cnt == 0) { - return final_segments; - } - - if (segment_cnt == 1) { - // "root" -> {"root"} - final_segments.push_back(splits[0]); - } else if (segment_cnt < static_cast( - DEFAULT_SEGMENT_NUM_FOR_TABLE_NAME + 1)) { - // "root.a" -> {"root", "a"} - // "root.a.b" -> {"root.a", "b"} - std::string table_name = std::accumulate( - splits.begin(), splits.end() - 1, std::string(), - [](const std::string& a, const std::string& b) { - return a.empty() ? b : a + PATH_SEPARATOR + b; - }); - final_segments.push_back(table_name); - final_segments.push_back(splits.back()); - } else { - // "root.a.b.c" -> {"root.a.b", "c"} - // "root.a.b.c.d" -> {"root.a.b", "c", "d"} - std::string table_name = std::accumulate( - splits.begin(), - splits.begin() + DEFAULT_SEGMENT_NUM_FOR_TABLE_NAME, - std::string(), [](const std::string& a, const std::string& b) { - return a.empty() ? b : a + PATH_SEPARATOR + b; - }); - - final_segments.emplace_back(std::move(table_name)); - final_segments.insert( - final_segments.end(), - splits.begin() + DEFAULT_SEGMENT_NUM_FOR_TABLE_NAME, - splits.end()); - } - - return final_segments; - } -}; -} - -#endif \ No newline at end of file diff --git a/iotdb-client/client-cpp/src/main/IoTDBRpcDataSet.cpp b/iotdb-client/client-cpp/src/main/IoTDBRpcDataSet.cpp deleted file mode 100644 index 237bcb803586d..0000000000000 --- a/iotdb-client/client-cpp/src/main/IoTDBRpcDataSet.cpp +++ /dev/null @@ -1,558 +0,0 @@ -/** -* Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -#include -#include -#include - -#include "IoTDBRpcDataSet.h" -#include "Column.h" - -const int32_t IoTDBRpcDataSet::START_INDEX = 2; -const std::string IoTDBRpcDataSet::TIMESTAMP_STR = "Time"; -const std::string IoTDBRpcDataSet::DEFAULT_TIME_FORMAT = "default"; - -IoTDBRpcDataSet::IoTDBRpcDataSet(const std::string& sql, - const std::vector& columnNameList, - const std::vector& columnTypeList, - const std::map& columnNameIndex, - bool ignoreTimestamp, - bool moreData, - int64_t queryId, - int64_t statementId, - std::shared_ptr client, - int64_t sessionId, - const std::vector& queryResult, - int32_t fetchSize, - int64_t timeout, - const std::string& zoneId, - const std::string& timeFormat) - : sql_(sql), - isClosed_(false), - client_(client), - fetchSize_(fetchSize), - timeout_(timeout), - hasCachedRecord_(false), - lastReadWasNull_(false), - columnSize_(static_cast(columnNameList.size())), - sessionId_(sessionId), - queryId_(queryId), - statementId_(statementId), - time_(0), - ignoreTimestamp_(ignoreTimestamp), - moreData_(moreData), - queryResult_(queryResult), - curTsBlock_(nullptr), - queryResultSize_(static_cast(queryResult.size())), - queryResultIndex_(0), - tsBlockSize_(0), - tsBlockIndex_(-1), - timeZoneId_(zoneId), - timeFormat_(timeFormat) { - if (!ignoreTimestamp) { - columnNameList_.push_back(TIMESTAMP_STR); - columnTypeList_.emplace_back("INT64"); - columnOrdinalMap_[TIMESTAMP_STR] = 1; - } - - // Process column names and types - if (!columnNameIndex.empty()) { - // Deduplicate column types - std::set uniqueValues; - for (const auto& entry : columnNameIndex) { - uniqueValues.insert(entry.second); - } - int deduplicatedColumnSize = static_cast(uniqueValues.size()); - columnTypeDeduplicatedList_.resize(deduplicatedColumnSize); - for (size_t i = 0; i < columnNameList.size(); ++i) { - const std::string& name = columnNameList[i]; - columnNameList_.push_back(name); - columnTypeList_.push_back(columnTypeList[i]); - // Update ordinal map and deduplicated types - if (!columnOrdinalMap_.count(name)) { - int index = columnNameIndex.at(name); - if (std::none_of(columnOrdinalMap_.begin(), columnOrdinalMap_.end(), - [index](const std::pair& entry) { - return entry.second == (index + START_INDEX); - })) { - columnTypeDeduplicatedList_[index] = getDataTypeByStr(columnTypeList[i]); - } - columnOrdinalMap_[name] = index + START_INDEX; - } - } - } - else { - // Handle case without column name index - int32_t currentIndex = START_INDEX; - for (size_t i = 0; i < columnNameList.size(); ++i) { - std::string name = columnNameList[i]; - columnNameList_.push_back(name); - columnTypeList_.push_back(columnTypeList[i]); - if (!columnOrdinalMap_.count(name)) { - columnOrdinalMap_[name] = currentIndex++; - columnTypeDeduplicatedList_.push_back(getDataTypeByStr(columnTypeList[i])); - } - } - } - - columnSize_ = static_cast(columnNameList_.size()); -} - -IoTDBRpcDataSet::~IoTDBRpcDataSet() { - if (!isClosed_) { - close(); - } -} - -bool IoTDBRpcDataSet::next() { - if (hasCachedBlock()) { - lastReadWasNull_ = false; - constructOneRow(); - return true; - } - - if (hasCachedByteBuffer()) { - constructOneTsBlock(); - constructOneRow(); - return true; - } - - if (moreData_) { - bool hasResultSet = fetchResults(); - if (hasResultSet && hasCachedByteBuffer()) { - constructOneTsBlock(); - constructOneRow(); - return true; - } - } - - close(); - return false; -} - -void IoTDBRpcDataSet::close(bool forceClose) { - if ((!forceClose) && isClosed_) { - return; - } - TSCloseOperationReq closeReq; - closeReq.__set_sessionId(sessionId_); - closeReq.__set_statementId(statementId_); - closeReq.__set_queryId(queryId_); - TSStatus tsStatus; - try { - client_->closeOperation(tsStatus, closeReq); - RpcUtils::verifySuccess(tsStatus); - } - catch (const TTransportException& e) { - log_debug(e.what()); - throw IoTDBConnectionException(e.what()); - } catch (const IoTDBException& e) { - log_debug(e.what()); - throw; - } catch (exception& e) { - log_debug(e.what()); - throw IoTDBException(e.what()); - } - isClosed_ = true; - client_ = nullptr; -} - -bool IoTDBRpcDataSet::fetchResults() { - if (isClosed_) { - throw IoTDBException("This data set is already closed"); - } - - TSFetchResultsReq req; - req.__set_sessionId(sessionId_); - req.__set_statement(sql_); - req.__set_fetchSize(fetchSize_); - req.__set_queryId(queryId_); - req.__set_isAlign(true); - req.__set_timeout(timeout_); - TSFetchResultsResp resp; - client_->fetchResultsV2(resp, req); - RpcUtils::verifySuccess(resp.status); - moreData_ = resp.moreData; - if (!resp.hasResultSet) { - close(); - } - else { - queryResult_ = resp.queryResult; - queryResultIndex_ = 0; - if (!queryResult_.empty()) { - queryResultSize_ = queryResult_.size(); - } - else { - queryResultSize_ = 0; - } - tsBlockIndex_ = -1; - tsBlockSize_ = 0; - } - return resp.hasResultSet; -} - -void IoTDBRpcDataSet::constructOneRow() { - tsBlockIndex_++; - hasCachedRecord_ = true; - time_ = curTsBlock_->getTimeColumn()->getLong(tsBlockIndex_); -} - -void IoTDBRpcDataSet::constructOneTsBlock() { - lastReadWasNull_ = false; - const auto& curTsBlockBytes = queryResult_[queryResultIndex_]; - queryResultIndex_++; - curTsBlock_ = TsBlock::deserialize(curTsBlockBytes); - tsBlockIndex_ = -1; - tsBlockSize_ = curTsBlock_->getPositionCount(); -} - -bool IoTDBRpcDataSet::isNullByIndex(int32_t columnIndex) { - int index = columnOrdinalMap_[findColumnNameByIndex(columnIndex)] - START_INDEX; - // time column will never be null - if (index < 0) { - return false; - } - return isNull(index, tsBlockIndex_); -} - -bool IoTDBRpcDataSet::isNullByColumnName(const std::string& columnName) { - int index = columnOrdinalMap_[columnName] - START_INDEX; - // time column will never be null - if (index < 0) { - return false; - } - return isNull(index, tsBlockIndex_); -} - -bool IoTDBRpcDataSet::isNull(int32_t index, int32_t rowNum) { - return index >= 0 && curTsBlock_->getColumn(index)->isNull(rowNum); -} - -bool IoTDBRpcDataSet::getBooleanByIndex(int32_t columnIndex) { - return getBoolean(findColumnNameByIndex(columnIndex)); -} - -bool IoTDBRpcDataSet::getBoolean(const std::string& columnName) { - int index = columnOrdinalMap_[columnName] - START_INDEX; - return getBooleanByTsBlockColumnIndex(index); -} - -bool IoTDBRpcDataSet::getBooleanByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { - checkRecord(); - if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { - lastReadWasNull_ = false; - return curTsBlock_->getColumn(tsBlockColumnIndex)->getBoolean(tsBlockIndex_); - } - else { - lastReadWasNull_ = true; - return false; - } -} - -double IoTDBRpcDataSet::getDoubleByIndex(int32_t columnIndex) { - return getDouble(findColumnNameByIndex(columnIndex)); -} - -double IoTDBRpcDataSet::getDouble(const std::string& columnName) { - int index = columnOrdinalMap_[columnName] - START_INDEX; - return getDoubleByTsBlockColumnIndex(index); -} - -double IoTDBRpcDataSet::getDoubleByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { - checkRecord(); - if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { - lastReadWasNull_ = false; - return curTsBlock_->getColumn(tsBlockColumnIndex)->getDouble(tsBlockIndex_); - } - else { - lastReadWasNull_ = true; - return 0.0; - } -} - -float IoTDBRpcDataSet::getFloatByIndex(int32_t columnIndex) { - return getFloat(findColumnNameByIndex(columnIndex)); -} - -float IoTDBRpcDataSet::getFloat(const std::string& columnName) { - int index = columnOrdinalMap_[columnName] - START_INDEX; - return getFloatByTsBlockColumnIndex(index); -} - -float IoTDBRpcDataSet::getFloatByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { - checkRecord(); - if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { - lastReadWasNull_ = false; - return curTsBlock_->getColumn(tsBlockColumnIndex)->getFloat(tsBlockIndex_); - } - else { - lastReadWasNull_ = true; - return 0.0f; - } -} - -int32_t IoTDBRpcDataSet::getIntByIndex(int32_t columnIndex) { - return getInt(findColumnNameByIndex(columnIndex)); -} - -int32_t IoTDBRpcDataSet::getInt(const std::string& columnName) { - int index = columnOrdinalMap_[columnName] - START_INDEX; - return getIntByTsBlockColumnIndex(index); -} - -int32_t IoTDBRpcDataSet::getIntByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { - checkRecord(); - if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { - lastReadWasNull_ = false; - TSDataType::TSDataType dataType = curTsBlock_->getColumn(tsBlockColumnIndex)->getDataType(); - if (dataType == TSDataType::INT64) { - return static_cast(curTsBlock_->getColumn(tsBlockColumnIndex)->getLong(tsBlockIndex_)); - } - return curTsBlock_->getColumn(tsBlockColumnIndex)->getInt(tsBlockIndex_); - } - else { - lastReadWasNull_ = true; - return 0; - } -} - -int64_t IoTDBRpcDataSet::getLongByIndex(int32_t columnIndex) { - return getLong(findColumnNameByIndex(columnIndex)); -} - -int64_t IoTDBRpcDataSet::getLong(const std::string& columnName) { - int index = columnOrdinalMap_[columnName] - START_INDEX; - return getLongByTsBlockColumnIndex(index); -} - -int64_t IoTDBRpcDataSet::getLongByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { - checkRecord(); - if (tsBlockColumnIndex < 0) { - lastReadWasNull_ = false; - return curTsBlock_->getTimeByIndex(tsBlockIndex_); - } - if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { - lastReadWasNull_ = false; - TSDataType::TSDataType dataType = curTsBlock_->getColumn(tsBlockColumnIndex)->getDataType(); - if (dataType == TSDataType::INT32) { - return static_cast(curTsBlock_->getColumn(tsBlockColumnIndex)->getInt(tsBlockIndex_)); - } - return curTsBlock_->getColumn(tsBlockColumnIndex)->getLong(tsBlockIndex_); - } - else { - lastReadWasNull_ = true; - return 0; - } -} - -std::shared_ptr IoTDBRpcDataSet::getBinaryByIndex(int32_t columnIndex) { - return getBinary(findColumnNameByIndex(columnIndex)); -} - -std::shared_ptr IoTDBRpcDataSet::getBinary(const std::string& columnName) { - int index = columnOrdinalMap_[columnName] - START_INDEX; - return getBinaryByTsBlockColumnIndex(index); -} - -std::shared_ptr IoTDBRpcDataSet::getBinaryByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { - checkRecord(); - if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { - lastReadWasNull_ = false; - return curTsBlock_->getColumn(tsBlockColumnIndex)->getBinary(tsBlockIndex_); - } - else { - lastReadWasNull_ = true; - return nullptr; - } -} - -std::string IoTDBRpcDataSet::getStringByIndex(int32_t columnIndex) { - return getString(findColumnNameByIndex(columnIndex)); -} - -std::string IoTDBRpcDataSet::getString(const std::string& columnName) { - int index = columnOrdinalMap_[columnName] - START_INDEX; - return getStringByTsBlockColumnIndex(index); -} - -std::string IoTDBRpcDataSet::getStringByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { - checkRecord(); - if (tsBlockColumnIndex == -1) { - int64_t timestamp = curTsBlock_->getTimeByIndex(tsBlockIndex_); - return std::to_string(timestamp); - } - if (isNull(tsBlockColumnIndex, tsBlockIndex_)) { - lastReadWasNull_ = true; - return ""; - } - lastReadWasNull_ = false; - return getStringByTsBlockColumnIndexAndDataType(tsBlockColumnIndex, - getDataTypeByIndex(tsBlockColumnIndex)); -} - -std::string IoTDBRpcDataSet::getStringByTsBlockColumnIndexAndDataType(int32_t index, - TSDataType::TSDataType tsDataType) { - switch (tsDataType) { - case TSDataType::BOOLEAN: - return std::to_string(curTsBlock_->getColumn(index)->getBoolean(tsBlockIndex_)); - case TSDataType::INT32: - return std::to_string(curTsBlock_->getColumn(index)->getInt(tsBlockIndex_)); - case TSDataType::INT64: - return std::to_string(curTsBlock_->getColumn(index)->getLong(tsBlockIndex_)); - case TSDataType::TIMESTAMP: { - int64_t value = curTsBlock_->getColumn(index)->getLong(tsBlockIndex_); - return formatDatetime(timeFormat_, timePrecision_, value, timeZoneId_); - } - case TSDataType::FLOAT: - return std::to_string(curTsBlock_->getColumn(index)->getFloat(tsBlockIndex_)); - case TSDataType::DOUBLE: - return std::to_string(curTsBlock_->getColumn(index)->getDouble(tsBlockIndex_)); - case TSDataType::TEXT: - case TSDataType::STRING: - case TSDataType::BLOB: { - auto binary = curTsBlock_->getColumn(index)->getBinary(tsBlockIndex_); - return binary->getStringValue(); - } - case TSDataType::DATE: { - int32_t value = curTsBlock_->getColumn(index)->getInt(tsBlockIndex_); - auto date = parseIntToDate(value); - return boost::gregorian::to_iso_extended_string(date); - } - default: - return ""; - } -} - -int64_t IoTDBRpcDataSet::getTimestampByIndex(int32_t columnIndex) { - return getTimestamp(findColumnNameByIndex(columnIndex)); -} - -int64_t IoTDBRpcDataSet::getTimestamp(const std::string& columnName) { - return getLong(columnName); -} - -boost::gregorian::date IoTDBRpcDataSet::getDateByIndex(int32_t columnIndex) { - return getDate(findColumnNameByIndex(columnIndex)); -} - -boost::gregorian::date IoTDBRpcDataSet::getDate(const std::string& columnName) { - int32_t value = getInt(columnName); - return parseIntToDate(value); -} - -TSDataType::TSDataType IoTDBRpcDataSet::getDataTypeByIndex(int32_t columnIndex) { - return getDataType(findColumnNameByIndex(columnIndex)); -} - -TSDataType::TSDataType IoTDBRpcDataSet::getDataType(const std::string& columnName) { - if (columnName == TIMESTAMP_STR) { - return TSDataType::INT64; - } - int index = columnOrdinalMap_[columnName] - START_INDEX; - return index < 0 || index >= columnTypeDeduplicatedList_.size() - ? TSDataType::UNKNOWN - : columnTypeDeduplicatedList_[index]; -} - -int32_t IoTDBRpcDataSet::findColumn(const std::string& columnName) { - auto it = columnOrdinalMap_.find(columnName); - if (it != columnOrdinalMap_.end()) { - return it->second; - } - return -1; -} - -std::string IoTDBRpcDataSet::findColumnNameByIndex(int32_t columnIndex) { - if (columnIndex <= 0) { - throw IoTDBException("column index should start from 1"); - } - if (columnIndex > static_cast(columnNameList_.size())) { - throw IoTDBException( - "Column index " + std::to_string(columnIndex) + - " is out of range. Valid range is 0 to " + - std::to_string(columnNameList_.size() - 1) - ); - } - return columnNameList_[columnIndex - 1]; -} - -void IoTDBRpcDataSet::checkRecord() { - if (queryResultIndex_ > queryResultSize_ || - tsBlockIndex_ >= tsBlockSize_ || - queryResult_.empty() || - !curTsBlock_) { - throw IoTDBException("no record remains"); - } -} - -int32_t IoTDBRpcDataSet::getValueColumnStartIndex() const { - return ignoreTimestamp_ ? 0 : 1; -} - -int32_t IoTDBRpcDataSet::getColumnSize() const { - return static_cast(columnNameList_.size()); -} - -const std::vector& IoTDBRpcDataSet::getColumnTypeList() const { - return columnTypeList_; -} - -const std::vector& IoTDBRpcDataSet::getColumnNameList() const { - return columnNameList_; -} - -bool IoTDBRpcDataSet::isClosed() const { - return isClosed_; -} - -int32_t IoTDBRpcDataSet::getFetchSize() const { - return fetchSize_; -} - -void IoTDBRpcDataSet::setFetchSize(int32_t fetchSize) { - fetchSize_ = fetchSize; -} - -bool IoTDBRpcDataSet::hasCachedRecord() const { - return hasCachedRecord_; -} - -void IoTDBRpcDataSet::setHasCachedRecord(bool hasCachedRecord) { - hasCachedRecord_ = hasCachedRecord; -} - -bool IoTDBRpcDataSet::isLastReadWasNull() const { - return lastReadWasNull_; -} - -int64_t IoTDBRpcDataSet::getCurrentRowTime() const { - return time_; -} - -bool IoTDBRpcDataSet::isIgnoreTimestamp() const { - return ignoreTimestamp_; -} - -bool IoTDBRpcDataSet::hasCachedBlock() const { - return curTsBlock_ && tsBlockIndex_ < tsBlockSize_ - 1; -} - -bool IoTDBRpcDataSet::hasCachedByteBuffer() const { - return !queryResult_.empty() && queryResultIndex_ < queryResultSize_; -} diff --git a/iotdb-client/client-cpp/src/main/IoTDBRpcDataSet.h b/iotdb-client/client-cpp/src/main/IoTDBRpcDataSet.h deleted file mode 100644 index 8e3f0b3d628e4..0000000000000 --- a/iotdb-client/client-cpp/src/main/IoTDBRpcDataSet.h +++ /dev/null @@ -1,150 +0,0 @@ -/** -* 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. - */ - -#ifndef IOTDB_RPC_DATA_SET_H -#define IOTDB_RPC_DATA_SET_H - -#include -#include -#include -#include -#include -#include "IClientRPCService.h" -#include -#include "TsBlock.h" - -class IoTDBRpcDataSet { -public: - static const int32_t START_INDEX; - static const std::string TIMESTAMP_STR; - - static const std::string DEFAULT_TIME_FORMAT; - static const std::string TIME_PRECISION; - static const std::string MILLISECOND; - static const std::string MICROSECOND; - static const std::string NANOSECOND; - - IoTDBRpcDataSet(const std::string& sql, - const std::vector& columnNameList, - const std::vector& columnTypeList, - const std::map& columnNameIndex, - bool ignoreTimestamp, - bool moreData, - int64_t queryId, - int64_t statementId, - std::shared_ptr client, - int64_t sessionId, - const std::vector& queryResult, - int32_t fetchSize, - int64_t timeout, - const std::string& zoneId, - const std::string& timeFormat); - - ~IoTDBRpcDataSet(); - - bool next(); - void close(bool forceClose = false); - - bool hasCachedBlock() const; - bool hasCachedByteBuffer() const; - - bool isNull(int32_t index, int32_t rowNum); - bool isNullByIndex(int32_t columnIndex); - bool isNullByColumnName(const std::string& columnName); - bool getBooleanByIndex(int32_t columnIndex); - bool getBoolean(const std::string& columnName); - double getDoubleByIndex(int32_t columnIndex); - double getDouble(const std::string& columnName); - float getFloatByIndex(int32_t columnIndex); - float getFloat(const std::string& columnName); - int32_t getIntByIndex(int32_t columnIndex); - int32_t getInt(const std::string& columnName); - int64_t getLongByIndex(int32_t columnIndex); - int64_t getLong(const std::string& columnName); - std::shared_ptr getBinaryByIndex(int32_t columnIndex); - std::shared_ptr getBinary(const std::string& columnName); - std::string getStringByIndex(int32_t columnIndex); - std::string getString(const std::string& columnName); - int64_t getTimestampByIndex(int32_t columnIndex); - int64_t getTimestamp(const std::string& columnName); - boost::gregorian::date getDateByIndex(int32_t columnIndex); - boost::gregorian::date getDate(const std::string& columnName); - - TSDataType::TSDataType getDataTypeByIndex(int32_t columnIndex); - TSDataType::TSDataType getDataType(const std::string& columnName); - int32_t findColumn(const std::string& columnName); - std::string findColumnNameByIndex(int32_t columnIndex); - int32_t getValueColumnStartIndex() const; - int32_t getColumnSize() const; - const std::vector& getColumnTypeList() const; - const std::vector& getColumnNameList() const; - bool isClosed() const; - int32_t getFetchSize() const; - void setFetchSize(int32_t fetchSize); - bool hasCachedRecord() const; - void setHasCachedRecord(bool hasCachedRecord); - bool isLastReadWasNull() const; - int64_t getCurrentRowTime() const; - bool isIgnoreTimestamp() const; - -private: - bool fetchResults(); - void constructOneRow(); - void constructOneTsBlock(); - void checkRecord(); - bool getBooleanByTsBlockColumnIndex(int32_t tsBlockColumnIndex); - std::string getStringByTsBlockColumnIndexAndDataType(int32_t index, TSDataType::TSDataType tsDataType); - double getDoubleByTsBlockColumnIndex(int32_t tsBlockColumnIndex); - float getFloatByTsBlockColumnIndex(int32_t tsBlockColumnIndex); - int32_t getIntByTsBlockColumnIndex(int32_t tsBlockColumnIndex); - int64_t getLongByTsBlockColumnIndex(int32_t tsBlockColumnIndex); - std::shared_ptr getBinaryByTsBlockColumnIndex(int32_t tsBlockColumnIndex); - std::string getStringByTsBlockColumnIndex(int32_t tsBlockColumnIndex); - - std::string sql_; - bool isClosed_; - std::shared_ptr client_; - std::vector columnNameList_; - std::vector columnTypeList_; - std::map columnOrdinalMap_; - std::vector columnTypeDeduplicatedList_; - int32_t fetchSize_; - int64_t timeout_; - bool hasCachedRecord_; - bool lastReadWasNull_; - int32_t columnSize_; - int64_t sessionId_; - int64_t queryId_; - int64_t statementId_; - int64_t time_; - bool ignoreTimestamp_; - bool moreData_; - std::vector queryResult_; - std::shared_ptr curTsBlock_; - int32_t queryResultSize_; - int32_t queryResultIndex_; - int32_t tsBlockSize_; - int32_t tsBlockIndex_; - std::string timeZoneId_; - std::string timeFormat_; - int32_t timeFactor_; - std::string timePrecision_; -}; - -#endif // IOTDB_RPC_DATA_SET_H diff --git a/iotdb-client/client-cpp/src/main/NodesSupplier.cpp b/iotdb-client/client-cpp/src/main/NodesSupplier.cpp deleted file mode 100644 index 5f268f075a108..0000000000000 --- a/iotdb-client/client-cpp/src/main/NodesSupplier.cpp +++ /dev/null @@ -1,222 +0,0 @@ -/** -* Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -#include "NodesSupplier.h" -#include "Session.h" -#include "SessionDataSet.h" -#include -#include -#include - -const std::string NodesSupplier::SHOW_DATA_NODES_COMMAND = "SHOW DATANODES"; -const std::string NodesSupplier::STATUS_COLUMN_NAME = "Status"; -const std::string NodesSupplier::IP_COLUMN_NAME = "RpcAddress"; -const std::string NodesSupplier::PORT_COLUMN_NAME = "RpcPort"; -const std::string NodesSupplier::REMOVING_STATUS = "Removing"; - -const int64_t NodesSupplier::TIMEOUT_IN_MS = 60000; -const int NodesSupplier::FETCH_SIZE = 10000; -const int NodesSupplier::THRIFT_DEFAULT_BUFFER_SIZE = 4096; -const int NodesSupplier::THRIFT_MAX_FRAME_SIZE = 1048576; -const int NodesSupplier::CONNECTION_TIMEOUT_IN_MS = 1000; - -TEndPoint RoundRobinPolicy::select(const std::vector& nodes) { - static std::atomic_uint index{0}; - - if (nodes.empty()) { - throw IoTDBException("No available nodes"); - } - - return nodes[index++ % nodes.size()]; -} - -StaticNodesSupplier::StaticNodesSupplier(const std::vector& nodes, - NodeSelectionPolicy policy) - : availableNodes_(nodes), policy_(std::move(policy)) {} - -boost::optional StaticNodesSupplier::getQueryEndPoint() { - try { - if (availableNodes_.empty()) { - return boost::none; - } - return policy_(availableNodes_); - } catch (const IoTDBException& e) { - return boost::none; - } -} - -std::vector StaticNodesSupplier::getEndPointList() { - return availableNodes_; -} - -StaticNodesSupplier::~StaticNodesSupplier() = default; - -std::shared_ptr NodesSupplier::create( - std::vector endpoints, - std::string userName, std::string password, std::string zoneId, - int32_t thriftDefaultBufferSize, int32_t thriftMaxFrameSize, - int32_t connectionTimeoutInMs, bool useSSL, bool enableRPCCompression, - std::string version, std::chrono::milliseconds refreshInterval, - NodeSelectionPolicy policy) { - if (endpoints.empty()) { - return nullptr; - } - auto supplier = std::make_shared( - userName, password, zoneId, thriftDefaultBufferSize, - thriftMaxFrameSize, connectionTimeoutInMs, useSSL, - enableRPCCompression, version, std::move(endpoints), std::move(policy) - ); - supplier->startBackgroundRefresh(refreshInterval); - return supplier; -} - -NodesSupplier::NodesSupplier( - std::string userName, std::string password, const std::string& zoneId, - int32_t thriftDefaultBufferSize, int32_t thriftMaxFrameSize, - int32_t connectionTimeoutInMs, bool useSSL, bool enableRPCCompression, - std::string version, std::vector endpoints, NodeSelectionPolicy policy) : userName_(std::move(userName)), password_(std::move(password)), zoneId_(zoneId), - thriftDefaultBufferSize_(thriftDefaultBufferSize), thriftMaxFrameSize_(thriftMaxFrameSize), - connectionTimeoutInMs_(connectionTimeoutInMs), useSSL_(useSSL), enableRPCCompression_(enableRPCCompression), version(version), endpoints_(std::move(endpoints)), - selectionPolicy_(std::move(policy)) { - deduplicateEndpoints(); -} - -std::vector NodesSupplier::getEndPointList() { - std::lock_guard lock(mutex_); - return endpoints_; -} - -TEndPoint NodesSupplier::selectQueryEndpoint() { - std::lock_guard lock(mutex_); - try { - return selectionPolicy_(endpoints_); - } catch (const std::exception& e) { - log_error("NodesSupplier::selectQueryEndpoint exception: %s", e.what()); - throw IoTDBException("NodesSupplier::selectQueryEndpoint exception, " + std::string(e.what())); - } -} - -boost::optional NodesSupplier::getQueryEndPoint() { - try { - return selectQueryEndpoint(); - } catch (const IoTDBException& e) { - return boost::none; - } -} - -NodesSupplier::~NodesSupplier() { - stopBackgroundRefresh(); - client_->close(); -} - -void NodesSupplier::deduplicateEndpoints() { - std::vector uniqueEndpoints; - uniqueEndpoints.reserve(endpoints_.size()); - for (const auto& endpoint : endpoints_) { - if (std::find(uniqueEndpoints.begin(), uniqueEndpoints.end(), endpoint) == uniqueEndpoints.end()) { - uniqueEndpoints.push_back(endpoint); - } - } - endpoints_ = std::move(uniqueEndpoints); -} - -void NodesSupplier::startBackgroundRefresh(std::chrono::milliseconds interval) { - isRunning_ = true; - refreshThread_ = std::thread([this, interval] { - while (isRunning_) { - refreshEndpointList(); - std::unique_lock cvLock(this->mutex_); - refreshCondition_.wait_for(cvLock, interval, [this]() { - return !isRunning_.load(); - }); - } - }); -} - -std::vector NodesSupplier::fetchLatestEndpoints() { - try { - if (client_ == nullptr) { - client_ = std::make_shared(selectionPolicy_(endpoints_)); - client_->init(userName_, password_, enableRPCCompression_, zoneId_, version); - } - - auto sessionDataSet = client_->executeQueryStatement(SHOW_DATA_NODES_COMMAND); - - uint32_t columnAddrIdx = -1, columnPortIdx = -1, columnStatusIdx = -1; - auto columnNames = sessionDataSet->getColumnNames(); - for (uint32_t i = 0; i < columnNames.size(); i++) { - if (columnNames[i] == IP_COLUMN_NAME) { - columnAddrIdx = i; - } else if (columnNames[i] == PORT_COLUMN_NAME) { - columnPortIdx = i; - } else if (columnNames[i] == STATUS_COLUMN_NAME) { - columnStatusIdx = i; - } - } - - if (columnAddrIdx == -1 || columnPortIdx == -1 || columnStatusIdx == -1) { - throw IoTDBException("Required columns not found in query result."); - } - - std::vector ret; - while (sessionDataSet->hasNext()) { - auto record = sessionDataSet->next(); - std::string ip = record->fields.at(columnAddrIdx).stringV; - int32_t port = record->fields.at(columnPortIdx).intV; - std::string status = record->fields.at(columnStatusIdx).stringV; - - if (ip == "0.0.0.0" || status == REMOVING_STATUS) { - log_warn("Skipping invalid node: " + ip + ":" + to_string(port)); - continue; - } - TEndPoint endpoint; - endpoint.ip = ip; - endpoint.port = port; - ret.emplace_back(endpoint); - } - - return ret; - } catch (const IoTDBException& e) { - client_.reset(); - throw IoTDBException(std::string("NodesSupplier::fetchLatestEndpoints failed: ") + e.what()); - } -} - -void NodesSupplier::refreshEndpointList() { - try { - auto newEndpoints = fetchLatestEndpoints(); - if (newEndpoints.empty()) { - return; - } - - std::lock_guard lock(mutex_); - endpoints_.swap(newEndpoints); - deduplicateEndpoints(); - } catch (const IoTDBException& e) { - log_error(std::string("NodesSupplier::refreshEndpointList failed: ") + e.what()); - } -} - -void NodesSupplier::stopBackgroundRefresh() noexcept { - if (isRunning_.exchange(false)) { - refreshCondition_.notify_all(); - if (refreshThread_.joinable()) { - refreshThread_.join(); - } - } -} \ No newline at end of file diff --git a/iotdb-client/client-cpp/src/main/NodesSupplier.h b/iotdb-client/client-cpp/src/main/NodesSupplier.h deleted file mode 100644 index a3cda24deac69..0000000000000 --- a/iotdb-client/client-cpp/src/main/NodesSupplier.h +++ /dev/null @@ -1,137 +0,0 @@ -/** - * 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. - */ -#ifndef IOTDB_NODES_SUPPLIER_H -#define IOTDB_NODES_SUPPLIER_H - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "ThriftConnection.h" - -class TEndPoint; - -class RoundRobinPolicy { -public: - static TEndPoint select(const std::vector& nodes); -}; - -class INodesSupplier { -public: - virtual ~INodesSupplier() = default; - virtual boost::optional getQueryEndPoint() = 0; - virtual std::vector getEndPointList() = 0; - using NodeSelectionPolicy = std::function&)>; -}; - -class StaticNodesSupplier : public INodesSupplier { -public: - explicit StaticNodesSupplier(const std::vector& nodes, - NodeSelectionPolicy policy = RoundRobinPolicy::select); - - boost::optional getQueryEndPoint() override; - - std::vector getEndPointList() override; - - ~StaticNodesSupplier() override; - -private: - const std::vector availableNodes_; - NodeSelectionPolicy policy_; -}; - -class NodesSupplier : public INodesSupplier { -public: - static const std::string SHOW_DATA_NODES_COMMAND; - static const std::string STATUS_COLUMN_NAME; - static const std::string IP_COLUMN_NAME; - static const std::string PORT_COLUMN_NAME; - static const std::string REMOVING_STATUS; - - static const int64_t TIMEOUT_IN_MS; - static const int FETCH_SIZE; - static const int THRIFT_DEFAULT_BUFFER_SIZE; - static const int THRIFT_MAX_FRAME_SIZE; - static const int CONNECTION_TIMEOUT_IN_MS; - - static std::shared_ptr create( - std::vector endpoints, - std::string userName, std::string password, std::string zoneId = "", - int32_t thriftDefaultBufferSize = ThriftConnection::THRIFT_DEFAULT_BUFFER_SIZE, - int32_t thriftMaxFrameSize = ThriftConnection::THRIFT_MAX_FRAME_SIZE, - int32_t connectionTimeoutInMs = ThriftConnection::CONNECTION_TIMEOUT_IN_MS, - bool useSSL = false, bool enableRPCCompression = false, - std::string version = "V_1_0", - std::chrono::milliseconds refreshInterval = std::chrono::milliseconds(TIMEOUT_IN_MS), - NodeSelectionPolicy policy = RoundRobinPolicy::select - ); - - NodesSupplier( - std::string userName, std::string password, const std::string& zoneId, - int32_t thriftDefaultBufferSize, int32_t thriftMaxFrameSize, - int32_t connectionTimeoutInMs, bool useSSL, bool enableRPCCompression, - std::string version, std::vector endpoints, NodeSelectionPolicy policy - ); - std::vector getEndPointList() override; - - boost::optional getQueryEndPoint() override; - - ~NodesSupplier() override; - -private: - std::string userName_; - std::string password_; - int32_t thriftDefaultBufferSize_; - int32_t thriftMaxFrameSize_; - int32_t connectionTimeoutInMs_; - bool useSSL_; - bool enableRPCCompression_; - std::string version; - std::string zoneId_; - - std::mutex mutex_; - std::vector endpoints_; - NodeSelectionPolicy selectionPolicy_; - - std::atomic isRunning_{false}; - std::thread refreshThread_; - std::condition_variable refreshCondition_; - - std::shared_ptr client_; - - void deduplicateEndpoints(); - - void startBackgroundRefresh(std::chrono::milliseconds interval); - - std::vector fetchLatestEndpoints(); - - void refreshEndpointList(); - - TEndPoint selectQueryEndpoint(); - - void stopBackgroundRefresh() noexcept; -}; - -#endif \ No newline at end of file diff --git a/iotdb-client/client-cpp/src/main/Session.cpp b/iotdb-client/client-cpp/src/main/Session.cpp deleted file mode 100644 index cb7dbc581d9ee..0000000000000 --- a/iotdb-client/client-cpp/src/main/Session.cpp +++ /dev/null @@ -1,2048 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -#include "Session.h" -#include -#include -#include -#include -#include -#include "NodesSupplier.h" -#include "SessionDataSet.h" - -using namespace std; - -/** -* Timeout of query can be set by users. -* A negative number means using the default configuration of server. -* And value 0 will disable the function of query timeout. -*/ -static const int64_t QUERY_TIMEOUT_MS = -1; - -LogLevelType LOG_LEVEL = LEVEL_DEBUG; - -TSDataType::TSDataType getTSDataTypeFromString(const string& str) { - // BOOLEAN, INT32, INT64, FLOAT, DOUBLE, TEXT, STRING, BLOB, TIMESTAMP, DATE, NULLTYPE - if (str == "BOOLEAN") { - return TSDataType::BOOLEAN; - } else if (str == "INT32") { - return TSDataType::INT32; - } else if (str == "INT64") { - return TSDataType::INT64; - } else if (str == "FLOAT") { - return TSDataType::FLOAT; - } else if (str == "DOUBLE") { - return TSDataType::DOUBLE; - } else if (str == "TEXT") { - return TSDataType::TEXT; - } else if (str == "TIMESTAMP") { - return TSDataType::TIMESTAMP; - } else if (str == "DATE") { - return TSDataType::DATE; - } else if (str == "BLOB") { - return TSDataType::BLOB; - } else if (str == "STRING") { - return TSDataType::STRING; - } - return TSDataType::UNKNOWN; -} - -void Tablet::createColumns() { - for (size_t i = 0; i < schemas.size(); i++) { - TSDataType::TSDataType dataType = schemas[i].second; - switch (dataType) { - case TSDataType::BOOLEAN: - values[i] = new bool[maxRowNumber]; - break; - case TSDataType::DATE: - values[i] = new boost::gregorian::date[maxRowNumber]; - break; - case TSDataType::INT32: - values[i] = new int[maxRowNumber]; - break; - case TSDataType::TIMESTAMP: - case TSDataType::INT64: - values[i] = new int64_t[maxRowNumber]; - break; - case TSDataType::FLOAT: - values[i] = new float[maxRowNumber]; - break; - case TSDataType::DOUBLE: - values[i] = new double[maxRowNumber]; - break; - case TSDataType::STRING: - case TSDataType::BLOB: - case TSDataType::TEXT: - values[i] = new string[maxRowNumber]; - break; - default: - throw UnSupportedDataTypeException(string("Data type ") + to_string(dataType) + " is not supported."); - } - } -} - -void Tablet::deleteColumns() { - for (size_t i = 0; i < schemas.size(); i++) { - if (!values[i]) continue; - TSDataType::TSDataType dataType = schemas[i].second; - switch (dataType) { - case TSDataType::BOOLEAN: { - bool* valueBuf = (bool*)(values[i]); - delete[] valueBuf; - break; - } - case TSDataType::INT32: { - int* valueBuf = (int*)(values[i]); - delete[] valueBuf; - break; - } - case TSDataType::DATE: { - boost::gregorian::date* valueBuf = (boost::gregorian::date*)(values[i]); - delete[] valueBuf; - break; - } - case TSDataType::TIMESTAMP: - case TSDataType::INT64: { - int64_t* valueBuf = (int64_t*)(values[i]); - delete[] valueBuf; - break; - } - case TSDataType::FLOAT: { - float* valueBuf = (float*)(values[i]); - delete[] valueBuf; - break; - } - case TSDataType::DOUBLE: { - double* valueBuf = (double*)(values[i]); - delete[] valueBuf; - break; - } - case TSDataType::STRING: - case TSDataType::BLOB: - case TSDataType::TEXT: { - string* valueBuf = (string*)(values[i]); - delete[] valueBuf; - break; - } - default: - throw UnSupportedDataTypeException(string("Data type ") + to_string(dataType) + " is not supported."); - } - values[i] = nullptr; - } -} - -void Tablet::deepCopyTabletColValue(void* const* srcPtr, void** destPtr, TSDataType::TSDataType type, int maxRowNumber) { - void *src = *srcPtr; - switch (type) { - case TSDataType::BOOLEAN: - *destPtr = new bool[maxRowNumber]; - memcpy(*destPtr, src, maxRowNumber * sizeof(bool)); - break; - case TSDataType::INT32: - *destPtr = new int32_t[maxRowNumber]; - memcpy(*destPtr, src, maxRowNumber * sizeof(int32_t)); - break; - case TSDataType::INT64: - case TSDataType::TIMESTAMP: - *destPtr = new int64_t[maxRowNumber]; - memcpy(*destPtr, src, maxRowNumber * sizeof(int64_t)); - break; - case TSDataType::FLOAT: - *destPtr = new float[maxRowNumber]; - memcpy(*destPtr, src, maxRowNumber * sizeof(float)); - break; - case TSDataType::DOUBLE: - *destPtr = new double[maxRowNumber]; - memcpy(*destPtr, src, maxRowNumber * sizeof(double)); - break; - case TSDataType::DATE: { - *destPtr = new boost::gregorian::date[maxRowNumber]; - boost::gregorian::date* srcDate = static_cast(src); - boost::gregorian::date* destDate = static_cast(*destPtr); - for (size_t j = 0; j < maxRowNumber; ++j) { - destDate[j] = srcDate[j]; - } - break; - } - case TSDataType::STRING: - case TSDataType::TEXT: - case TSDataType::BLOB: { - *destPtr = new std::string[maxRowNumber]; - std::string* srcStr = static_cast(src); - std::string* destStr = static_cast(*destPtr); - for (size_t j = 0; j < maxRowNumber; ++j) { - destStr[j] = srcStr[j]; - } - break; - } - default: - break; - } -} - -void Tablet::reset() { - rowSize = 0; - for (size_t i = 0; i < schemas.size(); i++) { - bitMaps[i].reset(); - } -} - -size_t Tablet::getTimeBytesSize() { - return rowSize * 8; -} - -size_t Tablet::getValueByteSize() { - size_t valueOccupation = 0; - for (size_t i = 0; i < schemas.size(); i++) { - switch (schemas[i].second) { - case TSDataType::BOOLEAN: - valueOccupation += rowSize; - break; - case TSDataType::INT32: - valueOccupation += rowSize * 4; - break; - case TSDataType::DATE: - valueOccupation += rowSize * 4; - break; - case TSDataType::TIMESTAMP: - case TSDataType::INT64: - valueOccupation += rowSize * 8; - break; - case TSDataType::FLOAT: - valueOccupation += rowSize * 4; - break; - case TSDataType::DOUBLE: - valueOccupation += rowSize * 8; - break; - case TSDataType::STRING: - case TSDataType::BLOB: - case TSDataType::TEXT: { - valueOccupation += rowSize * 4; - string* valueBuf = (string*)(values[i]); - for (size_t j = 0; j < rowSize; j++) { - valueOccupation += valueBuf[j].size(); - } - break; - } - default: - throw UnSupportedDataTypeException( - string("Data type ") + to_string(schemas[i].second) + " is not supported."); - } - } - return valueOccupation; -} - -void Tablet::setAligned(bool isAligned) { - this->isAligned = isAligned; -} - -std::shared_ptr Tablet::getDeviceID(int row) { - std::vector id_array(idColumnIndexes.size() + 1); - size_t idArrayIdx = 0; - id_array[idArrayIdx++] = this->deviceId; - for (auto idColumnIndex : idColumnIndexes) { - void* strPtr = getValue(idColumnIndex, row, TSDataType::TEXT); - id_array[idArrayIdx++] = *static_cast(strPtr); - } - return std::make_shared(id_array); -} - -string SessionUtils::getTime(const Tablet& tablet) { - MyStringBuffer timeBuffer; - unsigned int n = 8u * tablet.rowSize; - if (n > timeBuffer.str.capacity()) { - timeBuffer.reserve(n); - } - - for (size_t i = 0; i < tablet.rowSize; i++) { - timeBuffer.putInt64(tablet.timestamps[i]); - } - return timeBuffer.str; -} - -string SessionUtils::getValue(const Tablet& tablet) { - MyStringBuffer valueBuffer; - unsigned int n = 8u * tablet.schemas.size() * tablet.rowSize; - if (n > valueBuffer.str.capacity()) { - valueBuffer.reserve(n); - } - for (size_t i = 0; i < tablet.schemas.size(); i++) { - TSDataType::TSDataType dataType = tablet.schemas[i].second; - const BitMap& bitMap = tablet.bitMaps[i]; - switch (dataType) { - case TSDataType::BOOLEAN: { - bool* valueBuf = (bool*)(tablet.values[i]); - for (size_t index = 0; index < tablet.rowSize; index++) { - if (!bitMap.isMarked(index)) { - valueBuffer.putBool(valueBuf[index]); - } - else { - valueBuffer.putBool(false); - } - } - break; - } - case TSDataType::INT32: { - int* valueBuf = (int*)(tablet.values[i]); - for (size_t index = 0; index < tablet.rowSize; index++) { - if (!bitMap.isMarked(index)) { - valueBuffer.putInt(valueBuf[index]); - } - else { - valueBuffer.putInt((numeric_limits::min)()); - } - } - break; - } - case TSDataType::DATE: { - boost::gregorian::date* valueBuf = (boost::gregorian::date*)(tablet.values[i]); - for (size_t index = 0; index < tablet.rowSize; index++) { - if (!bitMap.isMarked(index)) { - valueBuffer.putDate(valueBuf[index]); - } - else { - valueBuffer.putInt(EMPTY_DATE_INT); - } - } - break; - } - case TSDataType::TIMESTAMP: - case TSDataType::INT64: { - int64_t* valueBuf = (int64_t*)(tablet.values[i]); - for (size_t index = 0; index < tablet.rowSize; index++) { - if (!bitMap.isMarked(index)) { - valueBuffer.putInt64(valueBuf[index]); - } - else { - valueBuffer.putInt64((numeric_limits::min)()); - } - } - break; - } - case TSDataType::FLOAT: { - float* valueBuf = (float*)(tablet.values[i]); - for (size_t index = 0; index < tablet.rowSize; index++) { - if (!bitMap.isMarked(index)) { - valueBuffer.putFloat(valueBuf[index]); - } - else { - valueBuffer.putFloat((numeric_limits::min)()); - } - } - break; - } - case TSDataType::DOUBLE: { - double* valueBuf = (double*)(tablet.values[i]); - for (size_t index = 0; index < tablet.rowSize; index++) { - if (!bitMap.isMarked(index)) { - valueBuffer.putDouble(valueBuf[index]); - } - else { - valueBuffer.putDouble((numeric_limits::min)()); - } - } - break; - } - case TSDataType::STRING: - case TSDataType::BLOB: - case TSDataType::TEXT: { - string* valueBuf = (string*)(tablet.values[i]); - for (size_t index = 0; index < tablet.rowSize; index++) { - if (!bitMap.isMarked(index)) { - valueBuffer.putString(valueBuf[index]); - } - else { - valueBuffer.putString(""); - } - } - break; - } - default: - throw UnSupportedDataTypeException(string("Data type ") + to_string(dataType) + " is not supported."); - } - } - for (size_t i = 0; i < tablet.schemas.size(); i++) { - const BitMap& bitMap = tablet.bitMaps[i]; - bool columnHasNull = !bitMap.isAllUnmarked(); - valueBuffer.putChar(columnHasNull ? (char)1 : (char)0); - if (columnHasNull) { - const vector& bytes = bitMap.getByteArray(); - for (size_t index = 0; index < tablet.rowSize / 8 + 1; index++) { - valueBuffer.putChar(bytes[index]); - } - } - } - return valueBuffer.str; -} - -bool SessionUtils::isTabletContainsSingleDevice(Tablet tablet) { - if (tablet.rowSize == 1) { - return true; - } - auto firstDeviceId = tablet.getDeviceID(0); - for (int i = 1; i < tablet.rowSize; ++i) { - if (*firstDeviceId != *tablet.getDeviceID(i)) { - return false; - } - } - return true; -} - -string MeasurementNode::serialize() const { - MyStringBuffer buffer; - buffer.putString(getName()); - buffer.putChar(getDataType()); - buffer.putChar(getEncoding()); - buffer.putChar(getCompressionType()); - return buffer.str; -} - -string Template::serialize() const { - MyStringBuffer buffer; - stack>> stack; - unordered_set alignedPrefix; - buffer.putString(getName()); - buffer.putBool(isAligned()); - if (isAligned()) { - alignedPrefix.emplace(""); - } - - for (const auto& child : children_) { - stack.push(make_pair("", child.second)); - } - - while (!stack.empty()) { - auto cur = stack.top(); - stack.pop(); - - string prefix = cur.first; - shared_ptr cur_node_ptr = cur.second; - string fullPath(prefix); - - if (!cur_node_ptr->isMeasurement()) { - if (!prefix.empty()) { - fullPath.append("."); - } - fullPath.append(cur_node_ptr->getName()); - if (cur_node_ptr->isAligned()) { - alignedPrefix.emplace(fullPath); - } - for (const auto& child : cur_node_ptr->getChildren()) { - stack.push(make_pair(fullPath, child.second)); - } - } - else { - buffer.putString(prefix); - buffer.putBool(alignedPrefix.find(prefix) != alignedPrefix.end()); - buffer.concat(cur_node_ptr->serialize()); - } - } - - return buffer.str; -} - -/** - * When delete variable, make sure release all resource. - */ -Session::~Session() { - try { - close(); - } - catch (const exception& e) { - log_debug(e.what()); - } -} - -void Session::removeBrokenSessionConnection(shared_ptr sessionConnection) { - if (enableRedirection_) { - this->endPointToSessionConnection.erase(sessionConnection->getEndPoint()); - } - - auto it1 = deviceIdToEndpoint.begin(); - while (it1 != deviceIdToEndpoint.end()) { - if (it1->second == sessionConnection->getEndPoint()) { - it1 = deviceIdToEndpoint.erase(it1); - } - else { - ++it1; - } - } - - auto it2 = tableModelDeviceIdToEndpoint.begin(); - while (it2 != tableModelDeviceIdToEndpoint.end()) { - if (it2->second == sessionConnection->getEndPoint()) { - it2 = tableModelDeviceIdToEndpoint.erase(it2); - } - else { - ++it2; - } - } -} - -/** - * check whether the batch has been sorted - * - * @return whether the batch has been sorted - */ -bool Session::checkSorted(const Tablet& tablet) { - for (size_t i = 1; i < tablet.rowSize; i++) { - if (tablet.timestamps[i] < tablet.timestamps[i - 1]) { - return false; - } - } - return true; -} - -bool Session::checkSorted(const vector& times) { - for (size_t i = 1; i < times.size(); i++) { - if (times[i] < times[i - 1]) { - return false; - } - } - return true; -} - -template -std::vector sortList(const std::vector& valueList, const int* index, int indexLength) { - std::vector sortedValues(valueList.size()); - for (int i = 0; i < indexLength; i++) { - sortedValues[i] = valueList[index[i]]; - } - return sortedValues; -} - -template -void sortValuesList(T* valueList, const int* index, size_t indexLength) { - T* sortedValues = new T[indexLength]; - for (int i = 0; i < indexLength; i++) { - sortedValues[i] = valueList[index[i]]; - } - for (int i = 0; i < indexLength; i++) { - valueList[i] = sortedValues[i]; - } - delete[] sortedValues; -} - -void Session::sortTablet(Tablet& tablet) { - /* - * following part of code sort the batch data by time, - * so we can insert continuous data in value list to get a better performance - */ - // sort to get index, and use index to sort value list - int* index = new int[tablet.rowSize]; - for (size_t i = 0; i < tablet.rowSize; i++) { - index[i] = i; - } - - sortIndexByTimestamp(index, tablet.timestamps, tablet.rowSize); - tablet.timestamps = sortList(tablet.timestamps, index, tablet.rowSize); - for (size_t i = 0; i < tablet.schemas.size(); i++) { - TSDataType::TSDataType dataType = tablet.schemas[i].second; - switch (dataType) { - case TSDataType::BOOLEAN: { - sortValuesList((bool*)(tablet.values[i]), index, tablet.rowSize); - break; - } - case TSDataType::INT32: { - sortValuesList((int*)(tablet.values[i]), index, tablet.rowSize); - break; - } - case TSDataType::DATE: { - sortValuesList((boost::gregorian::date*)(tablet.values[i]), index, tablet.rowSize); - break; - } - case TSDataType::TIMESTAMP: - case TSDataType::INT64: { - sortValuesList((int64_t*)(tablet.values[i]), index, tablet.rowSize); - break; - } - case TSDataType::FLOAT: { - sortValuesList((float*)(tablet.values[i]), index, tablet.rowSize); - break; - } - case TSDataType::DOUBLE: { - sortValuesList((double*)(tablet.values[i]), index, tablet.rowSize); - break; - } - case TSDataType::STRING: - case TSDataType::BLOB: - case TSDataType::TEXT: { - sortValuesList((string*)(tablet.values[i]), index, tablet.rowSize); - break; - } - default: - throw UnSupportedDataTypeException(string("Data type ") + to_string(dataType) + " is not supported."); - } - } - - delete[] index; -} - -void Session::sortIndexByTimestamp(int* index, std::vector& timestamps, int length) { - if (length <= 1) { - return; - } - - TsCompare tsCompareObj(timestamps); - std::sort(&index[0], &index[length], tsCompareObj); -} - -/** - * Append value into buffer in Big Endian order to comply with IoTDB server - */ -void Session::appendValues(string& buffer, const char* value, int size) { - static bool hasCheckedEndianFlag = false; - static bool localCpuIsBigEndian = false; - if (!hasCheckedEndianFlag) { - hasCheckedEndianFlag = true; - int chk = 0x0201; //used to distinguish CPU's type (BigEndian or LittleEndian) - localCpuIsBigEndian = (0x01 != *(char*)(&chk)); - } - - if (localCpuIsBigEndian) { - buffer.append(value, size); - } - else { - for (int i = size - 1; i >= 0; i--) { - buffer.append(value + i, 1); - } - } -} - -void -Session::putValuesIntoBuffer(const vector& types, const vector& values, string& buf) { - int32_t date; - for (size_t i = 0; i < values.size(); i++) { - int8_t typeNum = getDataTypeNumber(types[i]); - buf.append((char*)(&typeNum), sizeof(int8_t)); - switch (types[i]) { - case TSDataType::BOOLEAN: - buf.append(values[i], 1); - break; - case TSDataType::INT32: - appendValues(buf, values[i], sizeof(int32_t)); - break; - case TSDataType::DATE: - date = parseDateExpressionToInt(*(boost::gregorian::date*)values[i]); - appendValues(buf, (char*)&date, sizeof(int32_t)); - break; - case TSDataType::TIMESTAMP: - case TSDataType::INT64: - appendValues(buf, values[i], sizeof(int64_t)); - break; - case TSDataType::FLOAT: - appendValues(buf, values[i], sizeof(float)); - break; - case TSDataType::DOUBLE: - appendValues(buf, values[i], sizeof(double)); - break; - case TSDataType::STRING: - case TSDataType::BLOB: - case TSDataType::TEXT: { - int32_t len = (uint32_t)strlen(values[i]); - appendValues(buf, (char*)(&len), sizeof(uint32_t)); - // no need to change the byte order of string value - buf.append(values[i], len); - break; - } - default: - break; - } - } -} - -int8_t Session::getDataTypeNumber(TSDataType::TSDataType type) { - switch (type) { - case TSDataType::BOOLEAN: - return 0; - case TSDataType::INT32: - return 1; - case TSDataType::INT64: - return 2; - case TSDataType::FLOAT: - return 3; - case TSDataType::DOUBLE: - return 4; - case TSDataType::TEXT: - return 5; - case TSDataType::TIMESTAMP: - return 8; - case TSDataType::DATE: - return 9; - case TSDataType::BLOB: - return 10; - case TSDataType::STRING: - return 11; - default: - return -1; - } -} - -string Session::getVersionString(Version::Version version) { - switch (version) { - case Version::V_0_12: - return "V_0_12"; - case Version::V_0_13: - return "V_0_13"; - case Version::V_1_0: - return "V_1_0"; - default: - return "V_0_12"; - } -} - -void Session::initZoneId() { - if (!zoneId_.empty()) { - return; - } - - time_t ts = 0; - struct tm tmv; -#if defined(_WIN64) || defined (WIN32) || defined (_WIN32) - localtime_s(&tmv, &ts); -#else - localtime_r(&ts, &tmv); -#endif - - char zoneStr[32]; - strftime(zoneStr, sizeof(zoneStr), "%z", &tmv); - zoneId_ = zoneStr; -} - -void Session::initNodesSupplier() { - std::vector endPoints; - TEndPoint endPoint; - endPoint.__set_ip(host_); - endPoint.__set_port(rpcPort_); - endPoints.emplace_back(endPoint); - if (enableAutoFetch_) { - nodesSupplier_ = NodesSupplier::create(endPoints, username_, password_); - } - else { - nodesSupplier_ = make_shared(endPoints); - } -} - -void Session::initDefaultSessionConnection() { - defaultEndPoint_.__set_ip(host_); - defaultEndPoint_.__set_port(rpcPort_); - defaultSessionConnection_ = make_shared(this, defaultEndPoint_, zoneId_, nodesSupplier_, fetchSize_, - 60, 500, - sqlDialect_, database_); -} - -void Session::insertStringRecordsWithLeaderCache(vector deviceIds, vector times, - vector> measurementsList, - vector> valuesList, bool isAligned) { - std::unordered_map, TSInsertStringRecordsReq> recordsGroup; - for (int i = 0; i < deviceIds.size(); i++) { - auto connection = getSessionConnection(deviceIds[i]); - if (recordsGroup.find(connection) == recordsGroup.end()) { - TSInsertStringRecordsReq request; - std::vector emptyPrefixPaths; - std::vector> emptyMeasurementsList; - vector> emptyValuesList; - std::vector emptyTimestamps; - request.__set_isAligned(isAligned); - request.__set_prefixPaths(emptyPrefixPaths); - request.__set_timestamps(emptyTimestamps); - request.__set_measurementsList(emptyMeasurementsList); - request.__set_valuesList(emptyValuesList); - recordsGroup.insert(make_pair(connection, request)); - } - TSInsertStringRecordsReq& existingReq = recordsGroup[connection]; - existingReq.prefixPaths.emplace_back(deviceIds[i]); - existingReq.timestamps.emplace_back(times[i]); - existingReq.measurementsList.emplace_back(measurementsList[i]); - existingReq.valuesList.emplace_back(valuesList[i]); - } - std::function, const TSInsertStringRecordsReq&)> consumer = - [](const std::shared_ptr& c, const TSInsertStringRecordsReq& r) { - c->insertStringRecords(r); - }; - if (recordsGroup.size() == 1) { - insertOnce(recordsGroup, consumer); - } - else { - insertByGroup(recordsGroup, consumer); - } -} - -void Session::insertRecordsWithLeaderCache(vector deviceIds, vector times, - vector> measurementsList, - const vector>& typesList, - vector> valuesList, bool isAligned) { - std::unordered_map, TSInsertRecordsReq> recordsGroup; - for (int i = 0; i < deviceIds.size(); i++) { - auto connection = getSessionConnection(deviceIds[i]); - if (recordsGroup.find(connection) == recordsGroup.end()) { - TSInsertRecordsReq request; - std::vector emptyPrefixPaths; - std::vector> emptyMeasurementsList; - std::vector emptyValuesList; - std::vector emptyTimestamps; - request.__set_isAligned(isAligned); - request.__set_prefixPaths(emptyPrefixPaths); - request.__set_timestamps(emptyTimestamps); - request.__set_measurementsList(emptyMeasurementsList); - request.__set_valuesList(emptyValuesList); - recordsGroup.insert(make_pair(connection, request)); - } - TSInsertRecordsReq& existingReq = recordsGroup[connection]; - existingReq.prefixPaths.emplace_back(deviceIds[i]); - existingReq.timestamps.emplace_back(times[i]); - existingReq.measurementsList.emplace_back(measurementsList[i]); - vector bufferList; - string buffer; - putValuesIntoBuffer(typesList[i], valuesList[i], buffer); - existingReq.valuesList.emplace_back(buffer); - recordsGroup[connection] = existingReq; - } - std::function, const TSInsertRecordsReq&)> consumer = - [](const std::shared_ptr& c, const TSInsertRecordsReq& r) { - c->insertRecords(r); - }; - if (recordsGroup.size() == 1) { - insertOnce(recordsGroup, consumer); - } - else { - insertByGroup(recordsGroup, consumer); - } -} - -void Session::insertTabletsWithLeaderCache(unordered_map tablets, bool sorted, bool isAligned) { - std::unordered_map, TSInsertTabletsReq> tabletsGroup; - if (tablets.empty()) { - throw BatchExecutionException("No tablet is inserting!"); - } - for (const auto& item : tablets) { - if (isAligned != item.second->isAligned) { - throw BatchExecutionException("The tablets should be all aligned or non-aligned!"); - } - if (!checkSorted(*(item.second))) { - sortTablet(*(item.second)); - } - auto deviceId = item.first; - auto tablet = item.second; - auto connection = getSessionConnection(deviceId); - auto it = tabletsGroup.find(connection); - if (it == tabletsGroup.end()) { - TSInsertTabletsReq request; - tabletsGroup[connection] = request; - } - TSInsertTabletsReq& existingReq = tabletsGroup[connection]; - existingReq.prefixPaths.emplace_back(tablet->deviceId); - existingReq.timestampsList.emplace_back(move(SessionUtils::getTime(*tablet))); - existingReq.valuesList.emplace_back(move(SessionUtils::getValue(*tablet))); - existingReq.sizeList.emplace_back(tablet->rowSize); - vector dataTypes; - vector measurements; - for (pair schema : tablet->schemas) { - measurements.push_back(schema.first); - dataTypes.push_back(schema.second); - } - existingReq.measurementsList.emplace_back(measurements); - existingReq.typesList.emplace_back(dataTypes); - } - - std::function, const TSInsertTabletsReq&)> consumer = - [](const std::shared_ptr& c, const TSInsertTabletsReq& r) { - c->insertTablets(r); - }; - if (tabletsGroup.size() == 1) { - insertOnce(tabletsGroup, consumer); - } - else { - insertByGroup(tabletsGroup, consumer); - } -} - -void Session::open() { - open(false, DEFAULT_TIMEOUT_MS); -} - -void Session::open(bool enableRPCCompression) { - open(enableRPCCompression, DEFAULT_TIMEOUT_MS); -} - -void Session::open(bool enableRPCCompression, int connectionTimeoutInMs) { - if (!isClosed_) { - return; - } - - try { - initDefaultSessionConnection(); - } - catch (const exception& e) { - log_debug(e.what()); - throw IoTDBException(e.what()); - } - zoneId_ = defaultSessionConnection_->zoneId; - - if (enableRedirection_) { - endPointToSessionConnection.insert(make_pair(defaultEndPoint_, defaultSessionConnection_)); - } - - isClosed_ = false; -} - - -void Session::close() { - if (isClosed_) { - return; - } - isClosed_ = true; -} - - -void Session::insertRecord(const string& deviceId, int64_t time, - const vector& measurements, - const vector& values) { - TSInsertStringRecordReq req; - req.__set_prefixPath(deviceId); - req.__set_timestamp(time); - req.__set_measurements(measurements); - req.__set_values(values); - req.__set_isAligned(false); - try { - getSessionConnection(deviceId)->insertStringRecord(req); - } - catch (RedirectException& e) { - handleRedirection(deviceId, e.endPoint); - } catch (const IoTDBConnectionException& e) { - if (enableRedirection_ && deviceIdToEndpoint.find(deviceId) != deviceIdToEndpoint.end()) { - deviceIdToEndpoint.erase(deviceId); - try { - defaultSessionConnection_->insertStringRecord(req); - } - catch (RedirectException& e) { - } - } - else { - throw e; - } - } -} - -void Session::insertRecord(const string& deviceId, int64_t time, - const vector& measurements, - const vector& types, - const vector& values) { - TSInsertRecordReq req; - req.__set_prefixPath(deviceId); - req.__set_timestamp(time); - req.__set_measurements(measurements); - string buffer; - putValuesIntoBuffer(types, values, buffer); - req.__set_values(buffer); - req.__set_isAligned(false); - try { - getSessionConnection(deviceId)->insertRecord(req); - } - catch (RedirectException& e) { - handleRedirection(deviceId, e.endPoint); - } catch (const IoTDBConnectionException& e) { - if (enableRedirection_ && deviceIdToEndpoint.find(deviceId) != deviceIdToEndpoint.end()) { - deviceIdToEndpoint.erase(deviceId); - try { - defaultSessionConnection_->insertRecord(req); - } - catch (RedirectException& e) { - } - } - else { - throw e; - } - } -} - -void Session::insertAlignedRecord(const string& deviceId, int64_t time, - const vector& measurements, - const vector& values) { - TSInsertStringRecordReq req; - req.__set_prefixPath(deviceId); - req.__set_timestamp(time); - req.__set_measurements(measurements); - req.__set_values(values); - req.__set_isAligned(true); - try { - getSessionConnection(deviceId)->insertStringRecord(req); - } - catch (RedirectException& e) { - handleRedirection(deviceId, e.endPoint); - } catch (const IoTDBConnectionException& e) { - if (enableRedirection_ && deviceIdToEndpoint.find(deviceId) != deviceIdToEndpoint.end()) { - deviceIdToEndpoint.erase(deviceId); - try { - defaultSessionConnection_->insertStringRecord(req); - } - catch (RedirectException& e) { - } - } - else { - throw e; - } - } -} - -void Session::insertAlignedRecord(const string& deviceId, int64_t time, - const vector& measurements, - const vector& types, - const vector& values) { - TSInsertRecordReq req; - req.__set_prefixPath(deviceId); - req.__set_timestamp(time); - req.__set_measurements(measurements); - string buffer; - putValuesIntoBuffer(types, values, buffer); - req.__set_values(buffer); - req.__set_isAligned(false); - try { - getSessionConnection(deviceId)->insertRecord(req); - } - catch (RedirectException& e) { - handleRedirection(deviceId, e.endPoint); - } catch (const IoTDBConnectionException& e) { - if (enableRedirection_ && deviceIdToEndpoint.find(deviceId) != deviceIdToEndpoint.end()) { - deviceIdToEndpoint.erase(deviceId); - try { - defaultSessionConnection_->insertRecord(req); - } - catch (RedirectException& e) { - } - } - else { - throw e; - } - } -} - -void Session::insertRecords(const vector& deviceIds, - const vector& times, - const vector>& measurementsList, - const vector>& valuesList) { - size_t len = deviceIds.size(); - if (len != times.size() || len != measurementsList.size() || len != valuesList.size()) { - logic_error e("deviceIds, times, measurementsList and valuesList's size should be equal"); - throw exception(e); - } - - if (enableRedirection_) { - insertStringRecordsWithLeaderCache(deviceIds, times, measurementsList, valuesList, false); - } - else { - TSInsertStringRecordsReq request; - request.__set_prefixPaths(deviceIds); - request.__set_timestamps(times); - request.__set_measurementsList(measurementsList); - request.__set_valuesList(valuesList); - request.__set_isAligned(false); - try { - defaultSessionConnection_->insertStringRecords(request); - } - catch (RedirectException& e) { - } - } -} - -void Session::insertRecords(const vector& deviceIds, - const vector& times, - const vector>& measurementsList, - const vector>& typesList, - const vector>& valuesList) { - size_t len = deviceIds.size(); - if (len != times.size() || len != measurementsList.size() || len != valuesList.size()) { - logic_error e("deviceIds, times, measurementsList and valuesList's size should be equal"); - throw exception(e); - } - - if (enableRedirection_) { - insertRecordsWithLeaderCache(deviceIds, times, measurementsList, typesList, valuesList, false); - } - else { - TSInsertRecordsReq request; - request.__set_prefixPaths(deviceIds); - request.__set_timestamps(times); - request.__set_measurementsList(measurementsList); - vector bufferList; - for (size_t i = 0; i < valuesList.size(); i++) { - string buffer; - putValuesIntoBuffer(typesList[i], valuesList[i], buffer); - bufferList.push_back(buffer); - } - request.__set_valuesList(bufferList); - request.__set_isAligned(false); - try { - defaultSessionConnection_->insertRecords(request); - } - catch (RedirectException& e) { - } - } -} - -void Session::insertAlignedRecords(const vector& deviceIds, - const vector& times, - const vector>& measurementsList, - const vector>& valuesList) { - size_t len = deviceIds.size(); - if (len != times.size() || len != measurementsList.size() || len != valuesList.size()) { - logic_error e("deviceIds, times, measurementsList and valuesList's size should be equal"); - throw exception(e); - } - - if (enableRedirection_) { - insertStringRecordsWithLeaderCache(deviceIds, times, measurementsList, valuesList, true); - } - else { - TSInsertStringRecordsReq request; - request.__set_prefixPaths(deviceIds); - request.__set_timestamps(times); - request.__set_measurementsList(measurementsList); - request.__set_valuesList(valuesList); - request.__set_isAligned(true); - try { - defaultSessionConnection_->insertStringRecords(request); - } - catch (RedirectException& e) { - } - } -} - -void Session::insertAlignedRecords(const vector& deviceIds, - const vector& times, - const vector>& measurementsList, - const vector>& typesList, - const vector>& valuesList) { - size_t len = deviceIds.size(); - if (len != times.size() || len != measurementsList.size() || len != valuesList.size()) { - logic_error e("deviceIds, times, measurementsList and valuesList's size should be equal"); - throw exception(e); - } - - if (enableRedirection_) { - insertRecordsWithLeaderCache(deviceIds, times, measurementsList, typesList, valuesList, true); - } - else { - TSInsertRecordsReq request; - request.__set_prefixPaths(deviceIds); - request.__set_timestamps(times); - request.__set_measurementsList(measurementsList); - vector bufferList; - for (size_t i = 0; i < valuesList.size(); i++) { - string buffer; - putValuesIntoBuffer(typesList[i], valuesList[i], buffer); - bufferList.push_back(buffer); - } - request.__set_valuesList(bufferList); - request.__set_isAligned(false); - try { - defaultSessionConnection_->insertRecords(request); - } - catch (RedirectException& e) { - } - } -} - -void Session::insertRecordsOfOneDevice(const string& deviceId, - vector& times, - vector>& measurementsList, - vector>& typesList, - vector>& valuesList) { - insertRecordsOfOneDevice(deviceId, times, measurementsList, typesList, valuesList, false); -} - -void Session::insertRecordsOfOneDevice(const string& deviceId, - vector& times, - vector>& measurementsList, - vector>& typesList, - vector>& valuesList, - bool sorted) { - if (!checkSorted(times)) { - int* index = new int[times.size()]; - for (size_t i = 0; i < times.size(); i++) { - index[i] = (int)i; - } - - sortIndexByTimestamp(index, times, (int)(times.size())); - times = sortList(times, index, (int)(times.size())); - measurementsList = sortList(measurementsList, index, (int)(times.size())); - typesList = sortList(typesList, index, (int)(times.size())); - valuesList = sortList(valuesList, index, (int)(times.size())); - delete[] index; - } - TSInsertRecordsOfOneDeviceReq request; - request.__set_prefixPath(deviceId); - request.__set_timestamps(times); - request.__set_measurementsList(measurementsList); - vector bufferList; - for (size_t i = 0; i < valuesList.size(); i++) { - string buffer; - putValuesIntoBuffer(typesList[i], valuesList[i], buffer); - bufferList.push_back(buffer); - } - request.__set_valuesList(bufferList); - request.__set_isAligned(false); - TSStatus respStatus; - try { - getSessionConnection(deviceId)->insertRecordsOfOneDevice(request); - } - catch (RedirectException& e) { - handleRedirection(deviceId, e.endPoint); - } catch (const IoTDBConnectionException& e) { - if (enableRedirection_ && deviceIdToEndpoint.find(deviceId) != deviceIdToEndpoint.end()) { - deviceIdToEndpoint.erase(deviceId); - try { - defaultSessionConnection_->insertRecordsOfOneDevice(request); - } - catch (RedirectException& e) { - } - } - else { - throw e; - } - } -} - -void Session::insertAlignedRecordsOfOneDevice(const string& deviceId, - vector& times, - vector>& measurementsList, - vector>& typesList, - vector>& valuesList) { - insertAlignedRecordsOfOneDevice(deviceId, times, measurementsList, typesList, valuesList, false); -} - -void Session::insertAlignedRecordsOfOneDevice(const string& deviceId, - vector& times, - vector>& measurementsList, - vector>& typesList, - vector>& valuesList, - bool sorted) { - if (!checkSorted(times)) { - int* index = new int[times.size()]; - for (size_t i = 0; i < times.size(); i++) { - index[i] = (int)i; - } - - sortIndexByTimestamp(index, times, (int)(times.size())); - times = sortList(times, index, (int)(times.size())); - measurementsList = sortList(measurementsList, index, (int)(times.size())); - typesList = sortList(typesList, index, (int)(times.size())); - valuesList = sortList(valuesList, index, (int)(times.size())); - delete[] index; - } - TSInsertRecordsOfOneDeviceReq request; - request.__set_prefixPath(deviceId); - request.__set_timestamps(times); - request.__set_measurementsList(measurementsList); - vector bufferList; - for (size_t i = 0; i < valuesList.size(); i++) { - string buffer; - putValuesIntoBuffer(typesList[i], valuesList[i], buffer); - bufferList.push_back(buffer); - } - request.__set_valuesList(bufferList); - request.__set_isAligned(true); - TSStatus respStatus; - try { - getSessionConnection(deviceId)->insertRecordsOfOneDevice(request); - } - catch (RedirectException& e) { - handleRedirection(deviceId, e.endPoint); - } catch (const IoTDBConnectionException& e) { - if (enableRedirection_ && deviceIdToEndpoint.find(deviceId) != deviceIdToEndpoint.end()) { - deviceIdToEndpoint.erase(deviceId); - try { - defaultSessionConnection_->insertRecordsOfOneDevice(request); - } - catch (RedirectException& e) { - } - } - else { - throw e; - } - } -} - -void Session::insertTablet(Tablet& tablet) { - try { - insertTablet(tablet, false); - } - catch (const exception& e) { - log_debug(e.what()); - logic_error error(e.what()); - throw exception(error); - } -} - -void Session::buildInsertTabletReq(TSInsertTabletReq& request, Tablet& tablet, bool sorted) { - if ((!sorted) && !checkSorted(tablet)) { - sortTablet(tablet); - } - - request.prefixPath = tablet.deviceId; - - request.measurements.reserve(tablet.schemas.size()); - request.types.reserve(tablet.schemas.size()); - for (pair schema : tablet.schemas) { - request.measurements.push_back(schema.first); - request.types.push_back(schema.second); - } - request.values = move(SessionUtils::getValue(tablet)); - request.timestamps = move(SessionUtils::getTime(tablet)); - request.__set_size(tablet.rowSize); - request.__set_isAligned(tablet.isAligned); -} - -void Session::insertTablet(TSInsertTabletReq request) { - auto deviceId = request.prefixPath; - try { - getSessionConnection(deviceId)->insertTablet(request); - } - catch (RedirectException& e) { - handleRedirection(deviceId, e.endPoint); - } catch (const IoTDBConnectionException& e) { - if (enableRedirection_ && deviceIdToEndpoint.find(deviceId) != deviceIdToEndpoint.end()) { - deviceIdToEndpoint.erase(deviceId); - try { - defaultSessionConnection_->insertTablet(request); - } - catch (RedirectException& e) { - } - } - else { - throw e; - } - } -} - -void Session::insertTablet(Tablet& tablet, bool sorted) { - TSInsertTabletReq request; - buildInsertTabletReq(request, tablet, sorted); - insertTablet(request); -} - -void Session::insertAlignedTablet(Tablet& tablet) { - insertAlignedTablet(tablet, false); -} - -void Session::insertAlignedTablet(Tablet& tablet, bool sorted) { - tablet.setAligned(true); - try { - insertTablet(tablet, sorted); - } - catch (const exception& e) { - log_debug(e.what()); - logic_error error(e.what()); - throw exception(error); - } -} - -void Session::insertTablets(unordered_map& tablets) { - try { - insertTablets(tablets, false); - } - catch (const exception& e) { - log_debug(e.what()); - logic_error error(e.what()); - throw exception(error); - } -} - -void Session::insertTablets(unordered_map& tablets, bool sorted) { - if (tablets.empty()) { - throw BatchExecutionException("No tablet is inserting!"); - } - auto beginIter = tablets.begin(); - bool isAligned = ((*beginIter).second)->isAligned; - if (enableRedirection_) { - insertTabletsWithLeaderCache(tablets, sorted, isAligned); - } - else { - TSInsertTabletsReq request; - for (const auto& item : tablets) { - if (isAligned != item.second->isAligned) { - throw BatchExecutionException("The tablets should be all aligned or non-aligned!"); - } - if (!checkSorted(*(item.second))) { - sortTablet(*(item.second)); - } - request.prefixPaths.push_back(item.second->deviceId); - vector measurements; - vector dataTypes; - for (pair schema : item.second->schemas) { - measurements.push_back(schema.first); - dataTypes.push_back(schema.second); - } - request.measurementsList.push_back(measurements); - request.typesList.push_back(dataTypes); - request.timestampsList.push_back(move(SessionUtils::getTime(*(item.second)))); - request.valuesList.push_back(move(SessionUtils::getValue(*(item.second)))); - request.sizeList.push_back(item.second->rowSize); - } - request.__set_isAligned(isAligned); - try { - TSStatus respStatus; - defaultSessionConnection_->insertTablets(request); - RpcUtils::verifySuccess(respStatus); - } - catch (RedirectException& e) { - } - } -} - - -void Session::insertAlignedTablets(unordered_map& tablets, bool sorted) { - for (auto iter = tablets.begin(); iter != tablets.end(); iter++) { - iter->second->setAligned(true); - } - try { - insertTablets(tablets, sorted); - } - catch (const exception& e) { - log_debug(e.what()); - logic_error error(e.what()); - throw exception(error); - } -} - -void Session::testInsertRecord(const string& deviceId, int64_t time, const vector& measurements, - const vector& values) { - TSInsertStringRecordReq req; - req.__set_prefixPath(deviceId); - req.__set_timestamp(time); - req.__set_measurements(measurements); - req.__set_values(values); - TSStatus tsStatus; - try { - defaultSessionConnection_->testInsertStringRecord(req); - RpcUtils::verifySuccess(tsStatus); - } - catch (const TTransportException& e) { - log_debug(e.what()); - throw IoTDBConnectionException(e.what()); - } catch (const IoTDBException& e) { - log_debug(e.what()); - throw; - } catch (const exception& e) { - log_debug(e.what()); - throw IoTDBException(e.what()); - } -} - -void Session::testInsertTablet(const Tablet& tablet) { - TSInsertTabletReq request; - request.prefixPath = tablet.deviceId; - for (pair schema : tablet.schemas) { - request.measurements.push_back(schema.first); - request.types.push_back(schema.second); - } - request.__set_timestamps(move(SessionUtils::getTime(tablet))); - request.__set_values(move(SessionUtils::getValue(tablet))); - request.__set_size(tablet.rowSize); - try { - TSStatus tsStatus; - defaultSessionConnection_->testInsertTablet(request); - RpcUtils::verifySuccess(tsStatus); - } - catch (const TTransportException& e) { - log_debug(e.what()); - throw IoTDBConnectionException(e.what()); - } catch (const IoTDBException& e) { - log_debug(e.what()); - throw; - } catch (const exception& e) { - log_debug(e.what()); - throw IoTDBException(e.what()); - } -} - -void Session::testInsertRecords(const vector& deviceIds, - const vector& times, - const vector>& measurementsList, - const vector>& valuesList) { - size_t len = deviceIds.size(); - if (len != times.size() || len != measurementsList.size() || len != valuesList.size()) { - logic_error error("deviceIds, times, measurementsList and valuesList's size should be equal"); - throw exception(error); - } - TSInsertStringRecordsReq request; - request.__set_prefixPaths(deviceIds); - request.__set_timestamps(times); - request.__set_measurementsList(measurementsList); - request.__set_valuesList(valuesList); - - try { - TSStatus tsStatus; - defaultSessionConnection_->getSessionClient()->insertStringRecords(tsStatus, request); - RpcUtils::verifySuccess(tsStatus); - } - catch (const TTransportException& e) { - log_debug(e.what()); - throw IoTDBConnectionException(e.what()); - } catch (const IoTDBException& e) { - log_debug(e.what()); - throw; - } catch (const exception& e) { - log_debug(e.what()); - throw IoTDBException(e.what()); - } -} - -void Session::deleteTimeseries(const string& path) { - vector paths; - paths.push_back(path); - deleteTimeseries(paths); -} - -void Session::deleteTimeseries(const vector& paths) { - defaultSessionConnection_->deleteTimeseries(paths); -} - -void Session::deleteData(const string& path, int64_t endTime) { - vector paths; - paths.push_back(path); - deleteData(paths, LONG_LONG_MIN, endTime); -} - -void Session::deleteData(const vector& paths, int64_t endTime) { - deleteData(paths, LONG_LONG_MIN, endTime); -} - -void Session::deleteData(const vector& paths, int64_t startTime, int64_t endTime) { - TSDeleteDataReq req; - req.__set_paths(paths); - req.__set_startTime(startTime); - req.__set_endTime(endTime); - defaultSessionConnection_->deleteData(req); -} - -void Session::setStorageGroup(const string& storageGroupId) { - defaultSessionConnection_->setStorageGroup(storageGroupId); -} - -void Session::deleteStorageGroup(const string& storageGroup) { - vector storageGroups; - storageGroups.push_back(storageGroup); - deleteStorageGroups(storageGroups); -} - -void Session::deleteStorageGroups(const vector& storageGroups) { - defaultSessionConnection_->deleteStorageGroups(storageGroups); -} - -void Session::createDatabase(const string& database) { - this->setStorageGroup(database); -} - -void Session::deleteDatabase(const string& database) { - this->deleteStorageGroups(vector{database}); -} - -void Session::deleteDatabases(const vector& databases) { - this->deleteStorageGroups(databases); -} - -void Session::createTimeseries(const string& path, - TSDataType::TSDataType dataType, - TSEncoding::TSEncoding encoding, - CompressionType::CompressionType compressor) { - try { - createTimeseries(path, dataType, encoding, compressor, nullptr, nullptr, nullptr, ""); - } - catch (const exception& e) { - log_debug(e.what()); - throw IoTDBException(e.what()); - } -} - -void Session::createTimeseries(const string& path, - TSDataType::TSDataType dataType, - TSEncoding::TSEncoding encoding, - CompressionType::CompressionType compressor, - map* props, - map* tags, - map* attributes, - const string& measurementAlias) { - TSCreateTimeseriesReq req; - req.__set_path(path); - req.__set_dataType(dataType); - req.__set_encoding(encoding); - req.__set_compressor(compressor); - if (props != nullptr) { - req.__set_props(*props); - } - - if (tags != nullptr) { - req.__set_tags(*tags); - } - if (attributes != nullptr) { - req.__set_attributes(*attributes); - } - if (!measurementAlias.empty()) { - req.__set_measurementAlias(measurementAlias); - } - defaultSessionConnection_->createTimeseries(req); -} - -void Session::createMultiTimeseries(const vector& paths, - const vector& dataTypes, - const vector& encodings, - const vector& compressors, - vector>* propsList, - vector>* tagsList, - vector>* attributesList, - vector* measurementAliasList) { - TSCreateMultiTimeseriesReq request; - request.__set_paths(paths); - - vector dataTypesOrdinal; - dataTypesOrdinal.reserve(dataTypes.size()); - for (TSDataType::TSDataType dataType : dataTypes) { - dataTypesOrdinal.push_back(dataType); - } - request.__set_dataTypes(dataTypesOrdinal); - - vector encodingsOrdinal; - encodingsOrdinal.reserve(encodings.size()); - for (TSEncoding::TSEncoding encoding : encodings) { - encodingsOrdinal.push_back(encoding); - } - request.__set_encodings(encodingsOrdinal); - - vector compressorsOrdinal; - compressorsOrdinal.reserve(compressors.size()); - for (CompressionType::CompressionType compressor : compressors) { - compressorsOrdinal.push_back(compressor); - } - request.__set_compressors(compressorsOrdinal); - - if (propsList != nullptr) { - request.__set_propsList(*propsList); - } - - if (tagsList != nullptr) { - request.__set_tagsList(*tagsList); - } - if (attributesList != nullptr) { - request.__set_attributesList(*attributesList); - } - if (measurementAliasList != nullptr) { - request.__set_measurementAliasList(*measurementAliasList); - } - - defaultSessionConnection_->createMultiTimeseries(request); -} - -void Session::createAlignedTimeseries(const std::string& deviceId, - const std::vector& measurements, - const std::vector& dataTypes, - const std::vector& encodings, - const std::vector& compressors) { - TSCreateAlignedTimeseriesReq request; - request.__set_prefixPath(deviceId); - request.__set_measurements(measurements); - - vector dataTypesOrdinal; - dataTypesOrdinal.reserve(dataTypes.size()); - for (TSDataType::TSDataType dataType : dataTypes) { - dataTypesOrdinal.push_back(dataType); - } - request.__set_dataTypes(dataTypesOrdinal); - - vector encodingsOrdinal; - encodingsOrdinal.reserve(encodings.size()); - for (TSEncoding::TSEncoding encoding : encodings) { - encodingsOrdinal.push_back(encoding); - } - request.__set_encodings(encodingsOrdinal); - - vector compressorsOrdinal; - compressorsOrdinal.reserve(compressors.size()); - for (CompressionType::CompressionType compressor : compressors) { - compressorsOrdinal.push_back(compressor); - } - request.__set_compressors(compressorsOrdinal); - - defaultSessionConnection_->createAlignedTimeseries(request); -} - -bool Session::checkTimeseriesExists(const string& path) { - try { - std::unique_ptr dataset = executeQueryStatement("SHOW TIMESERIES " + path); - bool isExisted = dataset->hasNext(); - dataset->closeOperationHandle(); - return isExisted; - } - catch (const exception& e) { - log_debug(e.what()); - throw IoTDBException(e.what()); - } -} - -shared_ptr Session::getQuerySessionConnection() { - auto endPoint = nodesSupplier_->getQueryEndPoint(); - if (!endPoint.is_initialized() || endPointToSessionConnection.empty()) { - return defaultSessionConnection_; - } - - auto it = endPointToSessionConnection.find(endPoint.value()); - if (it != endPointToSessionConnection.end()) { - return it->second; - } - - shared_ptr newConnection; - try { - newConnection = make_shared(this, endPoint.value(), zoneId_, nodesSupplier_, - fetchSize_, 60, 500, sqlDialect_, database_); - endPointToSessionConnection.emplace(endPoint.value(), newConnection); - return newConnection; - } - catch (exception& e) { - log_debug("Session::getQuerySessionConnection() exception: " + e.what()); - return newConnection; - } -} - -shared_ptr Session::getSessionConnection(std::string deviceId) { - if (!enableRedirection_ || - deviceIdToEndpoint.find(deviceId) == deviceIdToEndpoint.end() || - endPointToSessionConnection.find(deviceIdToEndpoint[deviceId]) == endPointToSessionConnection.end()) { - return defaultSessionConnection_; - } - return endPointToSessionConnection.find(deviceIdToEndpoint[deviceId])->second; -} - -shared_ptr Session::getSessionConnection(std::shared_ptr deviceId) { - if (!enableRedirection_ || - tableModelDeviceIdToEndpoint.find(deviceId) == tableModelDeviceIdToEndpoint.end() || - endPointToSessionConnection.find(tableModelDeviceIdToEndpoint[deviceId]) == endPointToSessionConnection.end()) { - return defaultSessionConnection_; - } - return endPointToSessionConnection.find(tableModelDeviceIdToEndpoint[deviceId])->second; -} - -string Session::getTimeZone() { - auto ret = defaultSessionConnection_->getTimeZone(); - return ret.timeZone; -} - -void Session::setTimeZone(const string& zoneId) { - TSSetTimeZoneReq req; - req.__set_sessionId(defaultSessionConnection_->sessionId); - req.__set_timeZone(zoneId); - defaultSessionConnection_->setTimeZone(req); -} - -unique_ptr Session::executeQueryStatement(const string& sql) { - return executeQueryStatementMayRedirect(sql, QUERY_TIMEOUT_MS); -} - -unique_ptr Session::executeQueryStatement(const string& sql, int64_t timeoutInMs) { - return executeQueryStatementMayRedirect(sql, timeoutInMs); -} - -void Session::handleQueryRedirection(TEndPoint endPoint) { - if (!enableRedirection_) return; - shared_ptr newConnection; - auto it = endPointToSessionConnection.find(endPoint); - if (it != endPointToSessionConnection.end()) { - newConnection = it->second; - } - else { - try { - newConnection = make_shared(this, endPoint, zoneId_, nodesSupplier_, - fetchSize_, 60, 500, sqlDialect_, database_); - - endPointToSessionConnection.emplace(endPoint, newConnection); - } - catch (exception& e) { - throw IoTDBConnectionException(e.what()); - } - } - defaultSessionConnection_ = newConnection; -} - -void Session::handleRedirection(const std::string& deviceId, TEndPoint endPoint) { - if (!enableRedirection_) return; - if (endPoint.ip == "0.0.0.0") return; - deviceIdToEndpoint[deviceId] = endPoint; - - shared_ptr newConnection; - auto it = endPointToSessionConnection.find(endPoint); - if (it != endPointToSessionConnection.end()) { - newConnection = it->second; - } - else { - try { - newConnection = make_shared(this, endPoint, zoneId_, nodesSupplier_, - fetchSize_, 60, 500, sqlDialect_, database_); - endPointToSessionConnection.emplace(endPoint, newConnection); - } - catch (exception& e) { - deviceIdToEndpoint.erase(deviceId); - throw IoTDBConnectionException(e.what()); - } - } -} - -void Session::handleRedirection(const std::shared_ptr& deviceId, TEndPoint endPoint) { - if (!enableRedirection_) return; - if (endPoint.ip == "0.0.0.0") return; - tableModelDeviceIdToEndpoint[deviceId] = endPoint; - - shared_ptr newConnection; - auto it = endPointToSessionConnection.find(endPoint); - if (it != endPointToSessionConnection.end()) { - newConnection = it->second; - } - else { - try { - newConnection = make_shared(this, endPoint, zoneId_, nodesSupplier_, - fetchSize_, 60, 500, sqlDialect_, database_); - endPointToSessionConnection.emplace(endPoint, newConnection); - } - catch (exception& e) { - tableModelDeviceIdToEndpoint.erase(deviceId); - throw IoTDBConnectionException(e.what()); - } - } -} - -std::unique_ptr Session::executeQueryStatementMayRedirect(const std::string& sql, int64_t timeoutInMs) { - auto sessionConnection = getQuerySessionConnection(); - if (!sessionConnection) { - log_warn("Session connection not found"); - return nullptr; - } - try { - return sessionConnection->executeQueryStatement(sql, timeoutInMs); - } - catch (RedirectException& e) { - log_warn("Session connection redirect exception: " + e.what()); - handleQueryRedirection(e.endPoint); - try { - return defaultSessionConnection_->executeQueryStatement(sql, timeoutInMs); - } - catch (exception& e) { - log_error("Exception while executing redirected query statement: %s", e.what()); - throw ExecutionException(e.what()); - } - } catch (exception& e) { - log_error("Exception while executing query statement: %s", e.what()); - throw e; - } -} - -void Session::executeNonQueryStatement(const string& sql) { - try { - defaultSessionConnection_->executeNonQueryStatement(sql); - } - catch (const exception& e) { - throw IoTDBException(e.what()); - } -} - -unique_ptr -Session::executeRawDataQuery(const vector& paths, int64_t startTime, int64_t endTime) { - return defaultSessionConnection_->executeRawDataQuery(paths, startTime, endTime); -} - - -unique_ptr Session::executeLastDataQuery(const vector& paths) { - return executeLastDataQuery(paths, LONG_LONG_MIN); -} - -unique_ptr Session::executeLastDataQuery(const vector& paths, int64_t lastTime) { - return defaultSessionConnection_->executeLastDataQuery(paths, lastTime); -} - -void Session::createSchemaTemplate(const Template& templ) { - TSCreateSchemaTemplateReq req; - req.__set_name(templ.getName()); - req.__set_serializedTemplate(templ.serialize()); - defaultSessionConnection_->createSchemaTemplate(req); -} - -void Session::setSchemaTemplate(const string& template_name, const string& prefix_path) { - TSSetSchemaTemplateReq req; - req.__set_templateName(template_name); - req.__set_prefixPath(prefix_path); - defaultSessionConnection_->setSchemaTemplate(req); -} - -void Session::unsetSchemaTemplate(const string& prefix_path, const string& template_name) { - TSUnsetSchemaTemplateReq req; - req.__set_templateName(template_name); - req.__set_prefixPath(prefix_path); - defaultSessionConnection_->unsetSchemaTemplate(req); -} - -void Session::addAlignedMeasurementsInTemplate(const string& template_name, const vector& measurements, - const vector& dataTypes, - const vector& encodings, - const vector& compressors) { - TSAppendSchemaTemplateReq req; - req.__set_name(template_name); - req.__set_measurements(measurements); - req.__set_isAligned(true); - - vector dataTypesOrdinal; - dataTypesOrdinal.reserve(dataTypes.size()); - for (TSDataType::TSDataType dataType : dataTypes) { - dataTypesOrdinal.push_back(dataType); - } - req.__set_dataTypes(dataTypesOrdinal); - - vector encodingsOrdinal; - encodingsOrdinal.reserve(encodings.size()); - for (TSEncoding::TSEncoding encoding : encodings) { - encodingsOrdinal.push_back(encoding); - } - req.__set_encodings(encodingsOrdinal); - - vector compressorsOrdinal; - compressorsOrdinal.reserve(compressors.size()); - for (CompressionType::CompressionType compressor : compressors) { - compressorsOrdinal.push_back(compressor); - } - req.__set_compressors(compressorsOrdinal); - - defaultSessionConnection_->appendSchemaTemplate(req); -} - -void Session::addAlignedMeasurementsInTemplate(const string& template_name, const string& measurement, - TSDataType::TSDataType dataType, TSEncoding::TSEncoding encoding, - CompressionType::CompressionType compressor) { - vector measurements(1, measurement); - vector dataTypes(1, dataType); - vector encodings(1, encoding); - vector compressors(1, compressor); - addAlignedMeasurementsInTemplate(template_name, measurements, dataTypes, encodings, compressors); -} - -void Session::addUnalignedMeasurementsInTemplate(const string& template_name, const vector& measurements, - const vector& dataTypes, - const vector& encodings, - const vector& compressors) { - TSAppendSchemaTemplateReq req; - req.__set_name(template_name); - req.__set_measurements(measurements); - req.__set_isAligned(false); - - vector dataTypesOrdinal; - dataTypesOrdinal.reserve(dataTypes.size()); - for (TSDataType::TSDataType dataType : dataTypes) { - dataTypesOrdinal.push_back(dataType); - } - req.__set_dataTypes(dataTypesOrdinal); - - vector encodingsOrdinal; - encodingsOrdinal.reserve(encodings.size()); - for (TSEncoding::TSEncoding encoding : encodings) { - encodingsOrdinal.push_back(encoding); - } - req.__set_encodings(encodingsOrdinal); - - vector compressorsOrdinal; - compressorsOrdinal.reserve(compressors.size()); - for (CompressionType::CompressionType compressor : compressors) { - compressorsOrdinal.push_back(compressor); - } - req.__set_compressors(compressorsOrdinal); - - defaultSessionConnection_->appendSchemaTemplate(req); -} - -void Session::addUnalignedMeasurementsInTemplate(const string& template_name, const string& measurement, - TSDataType::TSDataType dataType, TSEncoding::TSEncoding encoding, - CompressionType::CompressionType compressor) { - vector measurements(1, measurement); - vector dataTypes(1, dataType); - vector encodings(1, encoding); - vector compressors(1, compressor); - addUnalignedMeasurementsInTemplate(template_name, measurements, dataTypes, encodings, compressors); -} - -void Session::deleteNodeInTemplate(const string& template_name, const string& path) { - TSPruneSchemaTemplateReq req; - req.__set_name(template_name); - req.__set_path(path); - defaultSessionConnection_->pruneSchemaTemplate(req); -} - -int Session::countMeasurementsInTemplate(const string& template_name) { - TSQueryTemplateReq req; - req.__set_name(template_name); - req.__set_queryType(TemplateQueryType::COUNT_MEASUREMENTS); - TSQueryTemplateResp resp = defaultSessionConnection_->querySchemaTemplate(req); - return resp.count; -} - -bool Session::isMeasurementInTemplate(const string& template_name, const string& path) { - TSQueryTemplateReq req; - req.__set_name(template_name); - req.__set_measurement(path); - req.__set_queryType(TemplateQueryType::IS_MEASUREMENT); - TSQueryTemplateResp resp = defaultSessionConnection_->querySchemaTemplate(req); - return resp.result; -} - -bool Session::isPathExistInTemplate(const string& template_name, const string& path) { - TSQueryTemplateReq req; - req.__set_name(template_name); - req.__set_measurement(path); - req.__set_queryType(TemplateQueryType::PATH_EXIST); - TSQueryTemplateResp resp = defaultSessionConnection_->querySchemaTemplate(req); - return resp.result; -} - -std::vector Session::showMeasurementsInTemplate(const string& template_name) { - TSQueryTemplateReq req; - req.__set_name(template_name); - req.__set_measurement(""); - req.__set_queryType(TemplateQueryType::SHOW_MEASUREMENTS); - TSQueryTemplateResp resp = defaultSessionConnection_->querySchemaTemplate(req); - return resp.measurements; -} - -std::vector Session::showMeasurementsInTemplate(const string& template_name, const string& pattern) { - TSQueryTemplateReq req; - req.__set_name(template_name); - req.__set_measurement(pattern); - req.__set_queryType(TemplateQueryType::SHOW_MEASUREMENTS); - TSQueryTemplateResp resp = defaultSessionConnection_->querySchemaTemplate(req); - return resp.measurements; -} - -bool Session::checkTemplateExists(const string& template_name) { - try { - std::unique_ptr dataset = executeQueryStatement( - "SHOW NODES IN DEVICE TEMPLATE " + template_name); - bool isExisted = dataset->hasNext(); - dataset->closeOperationHandle(); - return isExisted; - } - catch (const exception& e) { - if (strstr(e.what(), "does not exist") != NULL) { - return false; - } - log_debug(e.what()); - throw IoTDBException(e.what()); - } -} diff --git a/iotdb-client/client-cpp/src/main/Session.h b/iotdb-client/client-cpp/src/main/Session.h deleted file mode 100644 index ecdc20e6e1648..0000000000000 --- a/iotdb-client/client-cpp/src/main/Session.h +++ /dev/null @@ -1,1017 +0,0 @@ -/** - * 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. - */ -#ifndef IOTDB_SESSION_H -#define IOTDB_SESSION_H - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "IClientRPCService.h" -#include "NodesSupplier.h" -#include "AbstractSessionBuilder.h" -#include "SessionConnection.h" -#include "SessionDataSet.h" -#include "DeviceID.h" -#include "Common.h" - -//== For compatible with Windows OS == -#ifndef LONG_LONG_MIN -#define LONG_LONG_MIN 0x8000000000000000 -#endif - -using namespace std; - -using ::apache::thrift::protocol::TBinaryProtocol; -using ::apache::thrift::protocol::TCompactProtocol; -using ::apache::thrift::transport::TSocket; -using ::apache::thrift::transport::TTransport; -using ::apache::thrift::transport::TTransportException; -using ::apache::thrift::transport::TBufferedTransport; -using ::apache::thrift::transport::TFramedTransport; -using ::apache::thrift::TException; - - -template -void safe_cast(const T& value, Target& target) { - /* - Target Allowed Source Types - BOOLEAN BOOLEAN - INT32 INT32 - INT64 INT32 INT64 - FLOAT INT32 FLOAT - DOUBLE INT32 INT64 FLOAT DOUBLE - TEXT TEXT - */ - if (std::is_same::value) { - target = *(Target*)&value; - } - else if (std::is_same::value && std::is_array::value && std::is_same< - char, typename std::remove_extent::type>::value) { - string tmp((const char*)&value); - target = *(Target*)&tmp; - } - else if (std::is_same::value && std::is_same::value) { - int64_t tmp = *(int32_t*)&value; - target = *(Target*)&tmp; - } - else if (std::is_same::value && std::is_same::value) { - float tmp = *(int32_t*)&value; - target = *(Target*)&tmp; - } - else if (std::is_same::value && std::is_same::value) { - double tmp = *(int32_t*)&value; - target = *(Target*)&tmp; - } - else if (std::is_same::value && std::is_same::value) { - double tmp = *(int64_t*)&value; - target = *(Target*)&tmp; - } - else if (std::is_same::value && std::is_same::value) { - double tmp = *(float*)&value; - target = *(Target*)&tmp; - } - else { - throw UnSupportedDataTypeException("Error: Parameter type " + - std::string(typeid(T).name()) + " cannot be converted to DataType" + - std::string(typeid(Target).name())); - } -} - -/* - * A tablet data of one device, the tablet contains multiple measurements of this device that share - * the same time column. - * - * for example: device root.sg1.d1 - * - * time, m1, m2, m3 - * 1, 1, 2, 3 - * 2, 1, 2, 3 - * 3, 1, 2, 3 - * - * Notice: The tablet should not have empty cell - * - */ -class Tablet { -private: - static const int DEFAULT_ROW_SIZE = 1024; - - void createColumns(); - void deleteColumns(); - -public: - std::string deviceId; // deviceId of this tablet - std::vector> schemas; - // the list of measurement schemas for creating the tablet - std::map schemaNameIndex; // the map of schema name to index - std::vector columnTypes; // the list of column types (used in table model) - std::vector timestamps; // timestamps in this tablet - std::vector values; // each object is a primitive type array, which represents values of one measurement - std::vector bitMaps; // each bitmap represents the existence of each value in the current column - size_t rowSize; //the number of rows to include in this tablet - size_t maxRowNumber; // the maximum number of rows for this tablet - bool isAligned; // whether this tablet store data of aligned timeseries or not - std::vector idColumnIndexes; - - Tablet() = default; - - /** - * Return a tablet with default specified row number. This is the standard - * constructor (all Tablet should be the same size). - * - * @param deviceId the name of the device specified to be written in - * @param timeseries the list of measurement schemas for creating the tablet - */ - Tablet(const std::string& deviceId, - const std::vector>& timeseries) - : Tablet(deviceId, timeseries, DEFAULT_ROW_SIZE) { - } - - Tablet(const std::string& deviceId, - const std::vector>& timeseries, - const std::vector& columnTypes) - : Tablet(deviceId, timeseries, columnTypes, DEFAULT_ROW_SIZE) { - } - - /** - * Return a tablet with the specified number of rows (maxBatchSize). Only - * call this constructor directly for testing purposes. Tablet should normally - * always be default size. - * - * @param deviceId the name of the device specified to be written in - * @param schemas the list of measurement schemas for creating the row - * batch - * @param columnTypes the list of column types (used in table model) - * @param maxRowNumber the maximum number of rows for this tablet - */ - Tablet(const std::string& deviceId, - const std::vector>& schemas, - int maxRowNumber) - : Tablet(deviceId, schemas, std::vector(schemas.size(), ColumnCategory::FIELD), maxRowNumber) { - } - - Tablet(const std::string& deviceId, const std::vector>& schemas, - const std::vector columnTypes, - size_t maxRowNumber, bool _isAligned = false) : deviceId(deviceId), schemas(schemas), - columnTypes(columnTypes), - maxRowNumber(maxRowNumber), isAligned(_isAligned) { - // create timestamp column - timestamps.resize(maxRowNumber); - // create value columns - values.resize(schemas.size()); - createColumns(); - // init idColumnIndexs - for (size_t i = 0; i < this->columnTypes.size(); i++) { - if (this->columnTypes[i] == ColumnCategory::TAG) { - idColumnIndexes.push_back(i); - } - } - // create bitMaps - bitMaps.resize(schemas.size()); - for (size_t i = 0; i < schemas.size(); i++) { - bitMaps[i].resize(maxRowNumber); - } - // create schemaNameIndex - for (size_t i = 0; i < schemas.size(); i++) { - schemaNameIndex[schemas[i].first] = i; - } - this->rowSize = 0; - } - - Tablet(const Tablet& other) - : deviceId(other.deviceId), - schemas(other.schemas), - schemaNameIndex(other.schemaNameIndex), - columnTypes(other.columnTypes), - timestamps(other.timestamps), - maxRowNumber(other.maxRowNumber), - bitMaps(other.bitMaps), - rowSize(other.rowSize), - isAligned(other.isAligned), - idColumnIndexes(other.idColumnIndexes) { - values.resize(other.values.size()); - for (size_t i = 0; i < other.values.size(); ++i) { - if (!other.values[i]) continue; - TSDataType::TSDataType type = schemas[i].second; - deepCopyTabletColValue(&(other.values[i]), &values[i], type, maxRowNumber); - } - } - - Tablet& operator=(const Tablet& other) { - if (this != &other) { - deleteColumns(); - deviceId = other.deviceId; - schemas = other.schemas; - schemaNameIndex = other.schemaNameIndex; - columnTypes = other.columnTypes; - timestamps = other.timestamps; - maxRowNumber = other.maxRowNumber; - rowSize = other.rowSize; - isAligned = other.isAligned; - idColumnIndexes = other.idColumnIndexes; - bitMaps = other.bitMaps; - values.resize(other.values.size()); - for (size_t i = 0; i < other.values.size(); ++i) { - if (!other.values[i]) continue; - TSDataType::TSDataType type = schemas[i].second; - deepCopyTabletColValue(&(other.values[i]), &values[i], type, maxRowNumber); - } - } - return *this; - } - - ~Tablet() { - try { - deleteColumns(); - } - catch (exception& e) { - log_debug(string("Tablet::~Tablet(), ") + e.what()); - } - } - - void addTimestamp(size_t rowIndex, int64_t timestamp) { - timestamps[rowIndex] = timestamp; - rowSize = max(rowSize, rowIndex + 1); - } - - static void deepCopyTabletColValue(void* const* srcPtr, void** destPtr, - TSDataType::TSDataType type, int maxRowNumber); - - template - void addValue(size_t schemaId, size_t rowIndex, const T& value) { - if (schemaId >= schemas.size()) { - char tmpStr[100]; - snprintf(tmpStr, sizeof(tmpStr), - "Tablet::addValue(), schemaId >= schemas.size(). schemaId=%ld, schemas.size()=%ld.", - (long)schemaId, (long)schemas.size()); - throw std::out_of_range(tmpStr); - } - - if (rowIndex >= rowSize) { - char tmpStr[100]; - snprintf(tmpStr, sizeof(tmpStr), - "Tablet::addValue(), rowIndex >= rowSize. rowIndex=%ld, rowSize.size()=%ld.", - (long)rowIndex, (long)rowSize); - throw std::out_of_range(tmpStr); - } - - TSDataType::TSDataType dataType = schemas[schemaId].second; - switch (dataType) { - case TSDataType::BOOLEAN: { - safe_cast(value, ((bool*)values[schemaId])[rowIndex]); - break; - } - case TSDataType::INT32: { - safe_cast(value, ((int*)values[schemaId])[rowIndex]); - break; - } - case TSDataType::DATE: { - safe_cast(value, ((boost::gregorian::date*)values[schemaId])[rowIndex]); - break; - } - case TSDataType::TIMESTAMP: - case TSDataType::INT64: { - safe_cast(value, ((int64_t*)values[schemaId])[rowIndex]); - break; - } - case TSDataType::FLOAT: { - safe_cast(value, ((float*)values[schemaId])[rowIndex]); - break; - } - case TSDataType::DOUBLE: { - safe_cast(value, ((double*)values[schemaId])[rowIndex]); - break; - } - case TSDataType::BLOB: - case TSDataType::STRING: - case TSDataType::TEXT: { - safe_cast(value, ((string*)values[schemaId])[rowIndex]); - break; - } - default: - throw UnSupportedDataTypeException(string("Data type ") + to_string(dataType) + " is not supported."); - } - } - - template - void addValue(const string& schemaName, size_t rowIndex, const T& value) { - if (schemaNameIndex.find(schemaName) == schemaNameIndex.end()) { - throw SchemaNotFoundException(string("Schema ") + schemaName + " not found."); - } - size_t schemaId = schemaNameIndex[schemaName]; - addValue(schemaId, rowIndex, value); - } - - - void* getValue(size_t schemaId, size_t rowIndex, TSDataType::TSDataType dataType) { - if (schemaId >= schemas.size()) { - throw std::out_of_range("Tablet::getValue schemaId out of range: " - + std::to_string(schemaId)); - } - if (rowIndex >= rowSize) { - throw std::out_of_range("Tablet::getValue rowIndex out of range: " - + std::to_string(rowIndex)); - } - - switch (dataType) { - case TSDataType::BOOLEAN: - return &(reinterpret_cast(values[schemaId])[rowIndex]); - case TSDataType::INT32: - return &(reinterpret_cast(values[schemaId])[rowIndex]); - case TSDataType::DATE: - return &(reinterpret_cast(values[schemaId])[rowIndex]); - case TSDataType::TIMESTAMP: - case TSDataType::INT64: - return &(reinterpret_cast(values[schemaId])[rowIndex]); - case TSDataType::FLOAT: - return &(reinterpret_cast(values[schemaId])[rowIndex]); - case TSDataType::DOUBLE: - return &(reinterpret_cast(values[schemaId])[rowIndex]); - case TSDataType::BLOB: - case TSDataType::STRING: - case TSDataType::TEXT: - return &(reinterpret_cast(values[schemaId])[rowIndex]); - default: - throw UnSupportedDataTypeException("Unsupported data type: " - + std::to_string(dataType)); - } - } - - std::shared_ptr getDeviceID(int i); - - std::vector> getSchemas() const { - return schemas; - } - - void reset(); // Reset Tablet to the default state - set the rowSize to 0 - - size_t getTimeBytesSize(); - - size_t getValueByteSize(); // total byte size that values occupies - - void setAligned(bool isAligned); -}; - -class SessionUtils { -public: - static std::string getTime(const Tablet& tablet); - - static std::string getValue(const Tablet& tablet); - - static bool isTabletContainsSingleDevice(Tablet tablet); -}; - -class TemplateNode { -public: - explicit TemplateNode(const std::string& name) : name_(name) { - } - - const std::string& getName() const { - return name_; - } - - virtual const std::unordered_map>& getChildren() const { - throw BatchExecutionException("Should call exact sub class!"); - } - - virtual bool isMeasurement() = 0; - - virtual bool isAligned() { - throw BatchExecutionException("Should call exact sub class!"); - } - - virtual std::string serialize() const { - throw BatchExecutionException("Should call exact sub class!"); - } - -private: - std::string name_; -}; - -class MeasurementNode : public TemplateNode { -public: - MeasurementNode(const std::string& name_, TSDataType::TSDataType data_type_, TSEncoding::TSEncoding encoding_, - CompressionType::CompressionType compression_type_) : TemplateNode(name_) { - this->data_type_ = data_type_; - this->encoding_ = encoding_; - this->compression_type_ = compression_type_; - } - - TSDataType::TSDataType getDataType() const { - return data_type_; - } - - TSEncoding::TSEncoding getEncoding() const { - return encoding_; - } - - CompressionType::CompressionType getCompressionType() const { - return compression_type_; - } - - bool isMeasurement() override { - return true; - } - - std::string serialize() const override; - -private: - TSDataType::TSDataType data_type_; - TSEncoding::TSEncoding encoding_; - CompressionType::CompressionType compression_type_; -}; - -class InternalNode : public TemplateNode { -public: - InternalNode(const std::string& name, bool is_aligned) : TemplateNode(name), is_aligned_(is_aligned) { - } - - void addChild(const InternalNode& node) { - if (this->children_.count(node.getName())) { - throw BatchExecutionException("Duplicated child of node in template."); - } - this->children_[node.getName()] = std::make_shared(node); - } - - void addChild(const MeasurementNode& node) { - if (this->children_.count(node.getName())) { - throw BatchExecutionException("Duplicated child of node in template."); - } - this->children_[node.getName()] = std::make_shared(node); - } - - void deleteChild(const TemplateNode& node) { - this->children_.erase(node.getName()); - } - - const std::unordered_map>& getChildren() const override { - return children_; - } - - bool isMeasurement() override { - return false; - } - - bool isAligned() override { - return is_aligned_; - } - -private: - std::unordered_map> children_; - bool is_aligned_; -}; - -namespace TemplateQueryType { -enum TemplateQueryType { - COUNT_MEASUREMENTS, IS_MEASUREMENT, PATH_EXIST, SHOW_MEASUREMENTS -}; -} - -class Template { -public: - Template(const std::string& name, bool is_aligned) : name_(name), is_aligned_(is_aligned) { - } - - const std::string& getName() const { - return name_; - } - - bool isAligned() const { - return is_aligned_; - } - - void addToTemplate(const InternalNode& child) { - if (this->children_.count(child.getName())) { - throw BatchExecutionException("Duplicated child of node in template."); - } - this->children_[child.getName()] = std::make_shared(child); - } - - void addToTemplate(const MeasurementNode& child) { - if (this->children_.count(child.getName())) { - throw BatchExecutionException("Duplicated child of node in template."); - } - this->children_[child.getName()] = std::make_shared(child); - } - - std::string serialize() const; - -private: - std::string name_; - std::unordered_map> children_; - bool is_aligned_; -}; - -class Session { -private: - std::string host_; - int rpcPort_; - std::string username_; - std::string password_; - const TSProtocolVersion::type protocolVersion_ = TSProtocolVersion::IOTDB_SERVICE_PROTOCOL_V3; - bool isClosed_ = true; - std::string zoneId_; - int fetchSize_; - const static int DEFAULT_FETCH_SIZE = 10000; - const static int DEFAULT_TIMEOUT_MS = 0; - Version::Version version; - std::string sqlDialect_ = "tree"; // default sql dialect - std::string database_; - bool enableAutoFetch_ = true; - bool enableRedirection_ = true; - std::shared_ptr nodesSupplier_; - friend class SessionConnection; - friend class TableSession; - std::shared_ptr defaultSessionConnection_; - - TEndPoint defaultEndPoint_; - - struct TEndPointHash { - size_t operator()(const TEndPoint& endpoint) const { - return std::hash()(endpoint.ip) ^ std::hash()(endpoint.port); - } - }; - - struct TEndPointEqual { - bool operator()(const TEndPoint& lhs, const TEndPoint& rhs) const { - return lhs.ip == rhs.ip && lhs.port == rhs.port; - } - }; - - using EndPointSessionMap = std::unordered_map< - TEndPoint, shared_ptr, TEndPointHash, TEndPointEqual>; - EndPointSessionMap endPointToSessionConnection; - std::unordered_map deviceIdToEndpoint; - std::unordered_map, TEndPoint> tableModelDeviceIdToEndpoint; - -private: - void removeBrokenSessionConnection(shared_ptr sessionConnection); - - static bool checkSorted(const Tablet& tablet); - - static bool checkSorted(const std::vector& times); - - static void sortTablet(Tablet& tablet); - - static void sortIndexByTimestamp(int* index, std::vector& timestamps, int length); - - void appendValues(std::string& buffer, const char* value, int size); - - void - putValuesIntoBuffer(const std::vector& types, const std::vector& values, - std::string& buf); - - int8_t getDataTypeNumber(TSDataType::TSDataType type); - - struct TsCompare { - std::vector& timestamps; - - explicit TsCompare(std::vector& inTimestamps) : timestamps(inTimestamps) { - }; - - bool operator()(int i, int j) { return (timestamps[i] < timestamps[j]); }; - }; - - std::string getVersionString(Version::Version version); - - void initZoneId(); - - void initNodesSupplier(); - - void initDefaultSessionConnection(); - - template - void insertByGroup(std::unordered_map, T>& insertGroup, - InsertConsumer insertConsumer); - - template - void insertOnce(std::unordered_map, T>& insertGroup, - InsertConsumer insertConsumer); - - void insertStringRecordsWithLeaderCache(vector deviceIds, vector times, - vector> measurementsList, vector> valuesList, - bool isAligned); - - void insertRecordsWithLeaderCache(vector deviceIds, vector times, - vector> measurementsList, - const vector>& typesList, - vector> valuesList, bool isAligned); - - void insertTabletsWithLeaderCache(unordered_map tablets, bool sorted, bool isAligned); - - shared_ptr getQuerySessionConnection(); - - shared_ptr getSessionConnection(std::string deviceId); - - shared_ptr getSessionConnection(std::shared_ptr deviceId); - - void handleQueryRedirection(TEndPoint endPoint); - - void handleRedirection(const std::string& deviceId, TEndPoint endPoint); - - void handleRedirection(const std::shared_ptr& deviceId, TEndPoint endPoint); - - void setSqlDialect(const std::string& dialect) { - this->sqlDialect_ = dialect; - } - - void setDatabase(const std::string& database) { - this->database_ = database; - } - - string getDatabase() { - return database_; - } - - void changeDatabase(string database) { - this->database_ = database; - } - -public: - Session(const std::string& host, int rpcPort) : username_("root"), password_("root"), version(Version::V_1_0) { - this->host_ = host; - this->rpcPort_ = rpcPort; - initZoneId(); - initNodesSupplier(); - } - - Session(const std::string& host, int rpcPort, const std::string& username, const std::string& password) - : fetchSize_(DEFAULT_FETCH_SIZE) { - this->host_ = host; - this->rpcPort_ = rpcPort; - this->username_ = username; - this->password_ = password; - this->version = Version::V_1_0; - initZoneId(); - initNodesSupplier(); - } - - Session(const std::string& host, int rpcPort, const std::string& username, const std::string& password, - const std::string& zoneId, int fetchSize = DEFAULT_FETCH_SIZE) { - this->host_ = host; - this->rpcPort_ = rpcPort; - this->username_ = username; - this->password_ = password; - this->zoneId_ = zoneId; - this->fetchSize_ = fetchSize; - this->version = Version::V_1_0; - initZoneId(); - initNodesSupplier(); - } - - Session(const std::string& host, const std::string& rpcPort, const std::string& username = "user", - const std::string& password = "password", const std::string& zoneId = "", - int fetchSize = DEFAULT_FETCH_SIZE) { - this->host_ = host; - this->rpcPort_ = stoi(rpcPort); - this->username_ = username; - this->password_ = password; - this->zoneId_ = zoneId; - this->fetchSize_ = fetchSize; - this->version = Version::V_1_0; - initZoneId(); - initNodesSupplier(); - } - - Session(AbstractSessionBuilder* builder) { - this->host_ = builder->host; - this->rpcPort_ = builder->rpcPort; - this->username_ = builder->username; - this->password_ = builder->password; - this->zoneId_ = builder->zoneId; - this->fetchSize_ = builder->fetchSize; - this->version = Version::V_1_0; - this->sqlDialect_ = builder->sqlDialect; - this->database_ = builder->database; - this->enableAutoFetch_ = builder->enableAutoFetch; - this->enableRedirection_ = builder->enableRedirections; - initZoneId(); - initNodesSupplier(); - } - - ~Session(); - - void open(); - - void open(bool enableRPCCompression); - - void open(bool enableRPCCompression, int connectionTimeoutInMs); - - void close(); - - void setTimeZone(const std::string& zoneId); - - std::string getTimeZone(); - - void insertRecord(const std::string& deviceId, int64_t time, const std::vector& measurements, - const std::vector& values); - - void insertRecord(const std::string& deviceId, int64_t time, const std::vector& measurements, - const std::vector& types, const std::vector& values); - - void insertAlignedRecord(const std::string& deviceId, int64_t time, const std::vector& measurements, - const std::vector& values); - - void insertAlignedRecord(const std::string& deviceId, int64_t time, const std::vector& measurements, - const std::vector& types, const std::vector& values); - - void insertRecords(const std::vector& deviceIds, - const std::vector& times, - const std::vector>& measurementsList, - const std::vector>& valuesList); - - void insertRecords(const std::vector& deviceIds, - const std::vector& times, - const std::vector>& measurementsList, - const std::vector>& typesList, - const std::vector>& valuesList); - - void insertAlignedRecords(const std::vector& deviceIds, - const std::vector& times, - const std::vector>& measurementsList, - const std::vector>& valuesList); - - void insertAlignedRecords(const std::vector& deviceIds, - const std::vector& times, - const std::vector>& measurementsList, - const std::vector>& typesList, - const std::vector>& valuesList); - - void insertRecordsOfOneDevice(const std::string& deviceId, - std::vector& times, - std::vector>& measurementsList, - std::vector>& typesList, - std::vector>& valuesList); - - void insertRecordsOfOneDevice(const std::string& deviceId, - std::vector& times, - std::vector>& measurementsList, - std::vector>& typesList, - std::vector>& valuesList, - bool sorted); - - void insertAlignedRecordsOfOneDevice(const std::string& deviceId, - std::vector& times, - std::vector>& measurementsList, - std::vector>& typesList, - std::vector>& valuesList); - - void insertAlignedRecordsOfOneDevice(const std::string& deviceId, - std::vector& times, - std::vector>& measurementsList, - std::vector>& typesList, - std::vector>& valuesList, - bool sorted); - - void insertTablet(Tablet& tablet); - - void insertTablet(Tablet& tablet, bool sorted); - - static void buildInsertTabletReq(TSInsertTabletReq& request, Tablet& tablet, bool sorted); - - void insertTablet(TSInsertTabletReq request); - - void insertAlignedTablet(Tablet& tablet); - - void insertAlignedTablet(Tablet& tablet, bool sorted); - - void insertTablets(std::unordered_map& tablets); - - void insertTablets(std::unordered_map& tablets, bool sorted); - - void insertAlignedTablets(std::unordered_map& tablets, bool sorted = false); - - void testInsertRecord(const std::string& deviceId, int64_t time, - const std::vector& measurements, - const std::vector& values); - - void testInsertTablet(const Tablet& tablet); - - void testInsertRecords(const std::vector& deviceIds, - const std::vector& times, - const std::vector>& measurementsList, - const std::vector>& valuesList); - - void deleteTimeseries(const std::string& path); - - void deleteTimeseries(const std::vector& paths); - - void deleteData(const std::string& path, int64_t endTime); - - void deleteData(const std::vector& paths, int64_t endTime); - - void deleteData(const std::vector& paths, int64_t startTime, int64_t endTime); - - void setStorageGroup(const std::string& storageGroupId); - - void deleteStorageGroup(const std::string& storageGroup); - - void deleteStorageGroups(const std::vector& storageGroups); - - void createDatabase(const std::string& database); - - void deleteDatabase(const std::string& database); - - void deleteDatabases(const std::vector& databases); - - void createTimeseries(const std::string& path, TSDataType::TSDataType dataType, TSEncoding::TSEncoding encoding, - CompressionType::CompressionType compressor); - - void createTimeseries(const std::string& path, TSDataType::TSDataType dataType, TSEncoding::TSEncoding encoding, - CompressionType::CompressionType compressor, - std::map* props, std::map* tags, - std::map* attributes, - const std::string& measurementAlias); - - void createMultiTimeseries(const std::vector& paths, - const std::vector& dataTypes, - const std::vector& encodings, - const std::vector& compressors, - std::vector>* propsList, - std::vector>* tagsList, - std::vector>* attributesList, - std::vector* measurementAliasList); - - void createAlignedTimeseries(const std::string& deviceId, - const std::vector& measurements, - const std::vector& dataTypes, - const std::vector& encodings, - const std::vector& compressors); - - bool checkTimeseriesExists(const std::string& path); - - std::unique_ptr executeQueryStatement(const std::string& sql); - - std::unique_ptr executeQueryStatement(const std::string& sql, int64_t timeoutInMs); - - std::unique_ptr executeQueryStatementMayRedirect(const std::string& sql, int64_t timeoutInMs); - - void executeNonQueryStatement(const std::string& sql); - - std::unique_ptr executeRawDataQuery(const std::vector& paths, int64_t startTime, - int64_t endTime); - - std::unique_ptr executeLastDataQuery(const std::vector& paths); - - std::unique_ptr executeLastDataQuery(const std::vector& paths, int64_t lastTime); - - void createSchemaTemplate(const Template& templ); - - void setSchemaTemplate(const std::string& template_name, const std::string& prefix_path); - - void unsetSchemaTemplate(const std::string& prefix_path, const std::string& template_name); - - void addAlignedMeasurementsInTemplate(const std::string& template_name, - const std::vector& measurements, - const std::vector& dataTypes, - const std::vector& encodings, - const std::vector& compressors); - - void addAlignedMeasurementsInTemplate(const std::string& template_name, - const std::string& measurement, - TSDataType::TSDataType dataType, - TSEncoding::TSEncoding encoding, - CompressionType::CompressionType compressor); - - void addUnalignedMeasurementsInTemplate(const std::string& template_name, - const std::vector& measurements, - const std::vector& dataTypes, - const std::vector& encodings, - const std::vector& compressors); - - void addUnalignedMeasurementsInTemplate(const std::string& template_name, - const std::string& measurement, - TSDataType::TSDataType dataType, - TSEncoding::TSEncoding encoding, - CompressionType::CompressionType compressor); - - void deleteNodeInTemplate(const std::string& template_name, const std::string& path); - - int countMeasurementsInTemplate(const std::string& template_name); - - bool isMeasurementInTemplate(const std::string& template_name, const std::string& path); - - bool isPathExistInTemplate(const std::string& template_name, const std::string& path); - - std::vector showMeasurementsInTemplate(const std::string& template_name); - - std::vector showMeasurementsInTemplate(const std::string& template_name, const std::string& pattern); - - bool checkTemplateExists(const std::string& template_name); -}; - -template -void Session::insertByGroup(std::unordered_map, T>& insertGroup, - InsertConsumer insertConsumer) { - std::vector> futures; - - for (auto& entry : insertGroup) { - auto connection = entry.first; - auto& req = entry.second; - futures.emplace_back(std::async(std::launch::async, [=, &req]() mutable { - try { - insertConsumer(connection, req); - } - catch (const RedirectException& e) { - for (const auto& deviceEndPoint : e.deviceEndPointMap) { - handleRedirection(deviceEndPoint.first, deviceEndPoint.second); - } - } catch (const IoTDBConnectionException& e) { - if (endPointToSessionConnection.size() > 1) { - removeBrokenSessionConnection(connection); - try { - insertConsumer(defaultSessionConnection_, req); - } - catch (const RedirectException&) { - } - } - else { - throw; - } - } catch (const std::exception& e) { - log_debug(e.what()); - throw IoTDBException(e.what()); - } - })); - } - - std::string errorMessages; - for (auto& f : futures) { - try { - f.get(); - } - catch (const IoTDBConnectionException& e) { - throw; - } catch (const std::exception& e) { - if (!errorMessages.empty()) { - errorMessages += ";"; - } - errorMessages += e.what(); - } - } - - if (!errorMessages.empty()) { - throw StatementExecutionException(errorMessages); - } -} - -template -void Session::insertOnce(std::unordered_map, T>& insertGroup, - InsertConsumer insertConsumer) { - auto connection = insertGroup.begin()->first; - auto req = insertGroup.begin()->second; - try { - insertConsumer(connection, req); - } - catch (RedirectException e) { - for (const auto& deviceEndPoint : e.deviceEndPointMap) { - handleRedirection(deviceEndPoint.first, deviceEndPoint.second); - } - } catch (IoTDBConnectionException e) { - if (endPointToSessionConnection.size() > 1) { - removeBrokenSessionConnection(connection); - try { - insertConsumer(defaultSessionConnection_, req); - } - catch (RedirectException e) { - } - } - else { - throw e; - } - } -} - -#endif // IOTDB_SESSION_H diff --git a/iotdb-client/client-cpp/src/main/SessionConnection.cpp b/iotdb-client/client-cpp/src/main/SessionConnection.cpp deleted file mode 100644 index 14032ae24c04a..0000000000000 --- a/iotdb-client/client-cpp/src/main/SessionConnection.cpp +++ /dev/null @@ -1,673 +0,0 @@ -/** -* Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -#include "SessionConnection.h" -#include "Session.h" -#include "common_types.h" -#include - -#include - -#include "SessionDataSet.h" - -using namespace apache::thrift; -using namespace apache::thrift::protocol; -using namespace apache::thrift::transport; - -SessionConnection::SessionConnection(Session* session_ptr, const TEndPoint& endpoint, - const std::string& zoneId, - std::shared_ptr nodeSupplier, - int fetchSize, - int maxRetries, - int64_t retryInterval, - std::string dialect, - std::string db) - : session(session_ptr), - zoneId(zoneId), - endPoint(endpoint), - availableNodes(std::move(nodeSupplier)), - fetchSize(fetchSize), - maxRetryCount(maxRetries), - retryIntervalMs(retryInterval), - sqlDialect(std::move(dialect)), - database(std::move(db)) { - this->zoneId = zoneId.empty() ? getSystemDefaultZoneId() : zoneId; - endPointList.push_back(endpoint); - init(endPoint); -} - -void SessionConnection::close() { - bool needThrowException = false; - string errMsg; - session = nullptr; - try { - TSCloseSessionReq req; - req.__set_sessionId(sessionId); - TSStatus tsStatus; - client->closeSession(tsStatus, req); - } - catch (const TTransportException& e) { - log_debug(e.what()); - throw IoTDBConnectionException(e.what()); - } catch (const exception& e) { - log_debug(e.what()); - errMsg = errMsg + "Session::close() client->closeSession() error, maybe remote server is down. " + e.what() + - "\n"; - needThrowException = true; - } - - try { - if (transport->isOpen()) { - transport->close(); - } - } - catch (const exception& e) { - log_debug(e.what()); - errMsg = errMsg + "Session::close() transport->close() error. " + e.what() + "\n"; - needThrowException = true; - } - - if (needThrowException) { - throw IoTDBException(errMsg); - } -} - -SessionConnection::~SessionConnection() { - try { - close(); - } - catch (const exception& e) { - log_debug(e.what()); - } -} - -void SessionConnection::init(const TEndPoint& endpoint) { - shared_ptr socket(new TSocket(endpoint.ip, endpoint.port)); - transport = std::make_shared(socket); - socket->setConnTimeout(connectionTimeoutInMs); - if (!transport->isOpen()) { - try { - transport->open(); - } - catch (TTransportException& e) { - log_debug(e.what()); - throw IoTDBConnectionException(e.what()); - } - } - if (enableRPCCompression) { - shared_ptr protocol(new TCompactProtocol(transport)); - client = std::make_shared(protocol); - } - else { - shared_ptr protocol(new TBinaryProtocol(transport)); - client = std::make_shared(protocol); - } - - std::map configuration; - configuration["version"] = session->getVersionString(session->version); - configuration["sql_dialect"] = sqlDialect; - if (database != "") { - configuration["db"] = database; - } - TSOpenSessionReq openReq; - openReq.__set_username(session->username_); - openReq.__set_password(session->password_); - openReq.__set_zoneId(zoneId); - openReq.__set_configuration(configuration); - try { - TSOpenSessionResp openResp; - client->openSession(openResp, openReq); - RpcUtils::verifySuccess(openResp.status); - if (session->protocolVersion_ != openResp.serverProtocolVersion) { - if (openResp.serverProtocolVersion == 0) { - // less than 0.10 - throw logic_error(string("Protocol not supported, Client version is ") + - to_string(session->protocolVersion_) + - ", but Server version is " + to_string(openResp.serverProtocolVersion)); - } - } - - sessionId = openResp.sessionId; - statementId = client->requestStatementId(sessionId); - - if (!zoneId.empty()) { - setTimeZone(zoneId); - } - } - catch (const TTransportException& e) { - log_debug(e.what()); - transport->close(); - throw IoTDBConnectionException(e.what()); - } catch (const IoTDBException& e) { - log_debug(e.what()); - transport->close(); - throw; - } catch (const exception& e) { - log_debug(e.what()); - transport->close(); - throw; - } -} - -std::unique_ptr SessionConnection::executeQueryStatement(const std::string& sql, int64_t timeoutInMs) { - TSExecuteStatementReq req; - req.__set_sessionId(sessionId); - req.__set_statementId(statementId); - req.__set_statement(sql); - req.__set_timeout(timeoutInMs); - req.__set_enableRedirectQuery(true); - - auto result = callWithRetryAndReconnect( - [this, &req]() { - TSExecuteStatementResp resp; - client->executeQueryStatementV2(resp, req); - return resp; - }, - [](const TSExecuteStatementResp& resp) { - return resp.status; - } - ); - TSExecuteStatementResp resp = result.getResult(); - if (result.getRetryAttempts() == 0) { - RpcUtils::verifySuccessWithRedirection(resp.status); - } - else { - RpcUtils::verifySuccess(resp.status); - } - - return std::unique_ptr(new SessionDataSet(sql, resp.columns, resp.dataTypeList, - resp.columnNameIndexMap, resp.queryId, statementId, - client, sessionId, resp.queryResult, resp.ignoreTimeStamp, - timeoutInMs, resp.moreData, fetchSize, zoneId)); -} - -std::unique_ptr SessionConnection::executeRawDataQuery(const std::vector& paths, - int64_t startTime, int64_t endTime) { - TSRawDataQueryReq req; - req.__set_sessionId(sessionId); - req.__set_statementId(statementId); - req.__set_fetchSize(fetchSize); - req.__set_paths(paths); - req.__set_startTime(startTime); - req.__set_endTime(endTime); - auto result = callWithRetryAndReconnect( - [this, &req]() { - TSExecuteStatementResp resp; - client->executeRawDataQueryV2(resp, req); - return resp; - }, - [](const TSExecuteStatementResp& resp) { - return resp.status; - } - ); - TSExecuteStatementResp resp = result.getResult(); - if (result.getRetryAttempts() == 0) { - RpcUtils::verifySuccessWithRedirection(resp.status); - } - else { - RpcUtils::verifySuccess(resp.status); - } - return std::unique_ptr(new SessionDataSet("", resp.columns, resp.dataTypeList, - resp.columnNameIndexMap, resp.queryId, statementId, - client, sessionId, resp.queryResult, resp.ignoreTimeStamp, - connectionTimeoutInMs, resp.moreData, fetchSize, zoneId)); -} - -std::unique_ptr SessionConnection::executeLastDataQuery(const std::vector& paths, - int64_t lastTime) { - TSLastDataQueryReq req; - req.__set_sessionId(sessionId); - req.__set_statementId(statementId); - req.__set_fetchSize(fetchSize); - req.__set_paths(paths); - req.__set_time(lastTime); - - auto result = callWithRetryAndReconnect( - [this, &req]() { - TSExecuteStatementResp resp; - client->executeLastDataQuery(resp, req); - return resp; - }, - [](const TSExecuteStatementResp& resp) { - return resp.status; - } - ); - TSExecuteStatementResp resp = result.getResult(); - if (result.getRetryAttempts() == 0) { - RpcUtils::verifySuccessWithRedirection(resp.status); - } - else { - RpcUtils::verifySuccess(resp.status); - } - return std::unique_ptr(new SessionDataSet("", resp.columns, resp.dataTypeList, - resp.columnNameIndexMap, resp.queryId, statementId, - client, sessionId, resp.queryResult, resp.ignoreTimeStamp, - connectionTimeoutInMs, resp.moreData, fetchSize, zoneId)); -} - -void SessionConnection::executeNonQueryStatement(const string& sql) { - TSExecuteStatementReq req; - req.__set_sessionId(sessionId); - req.__set_statementId(statementId); - req.__set_statement(sql); - req.__set_timeout(0); //0 means no timeout. This value keep consistent to JAVA SDK. - TSExecuteStatementResp resp; - try { - client->executeUpdateStatementV2(resp, req); - RpcUtils::verifySuccess(resp.status); - } - catch (const TTransportException& e) { - log_debug(e.what()); - throw IoTDBConnectionException(e.what()); - } catch (const IoTDBException& e) { - log_debug(e.what()); - throw; - } catch (const exception& e) { - throw IoTDBException(e.what()); - } -} - -const TEndPoint& SessionConnection::getEndPoint() { - return endPoint; -} - -void SessionConnection::setTimeZone(const std::string& newZoneId) { - TSSetTimeZoneReq req; - req.__set_sessionId(sessionId); - req.__set_timeZone(newZoneId); - - try { - TSStatus tsStatus; - client->setTimeZone(tsStatus, req); - zoneId = newZoneId; - } - catch (const TException& e) { - throw IoTDBConnectionException(e.what()); - } -} - -std::string SessionConnection::getSystemDefaultZoneId() { - time_t ts = 0; - struct tm tmv{}; -#if defined(_WIN64) || defined (WIN32) || defined (_WIN32) - localtime_s(&tmv, &ts); -#else - localtime_r(&ts, &tmv); -#endif - char zoneStr[32]; - strftime(zoneStr, sizeof(zoneStr), "%z", &tmv); - return zoneStr; -} - -bool SessionConnection::reconnect() { - bool reconnect = false; - for (int i = 1; i <= 3; i++) { - if (transport != nullptr) { - transport->close(); - endPointList = std::move(availableNodes->getEndPointList()); - int currHostIndex = rand() % endPointList.size(); - int tryHostNum = 0; - for (int j = currHostIndex; j < endPointList.size(); j++) { - if (tryHostNum == endPointList.size()) { - break; - } - this->endPoint = endPointList[j]; - if (j == endPointList.size() - 1) { - j = -1; - } - tryHostNum++; - try { - init(this->endPoint); - reconnect = true; - } - catch (const IoTDBConnectionException& e) { - log_warn("The current node may have been down, connection exception: %s", e.what()); - continue; - } catch (exception& e) { - log_warn("login in failed, because %s", e.what()); - } - break; - } - } - if (reconnect) { - session->removeBrokenSessionConnection(shared_from_this()); - session->defaultEndPoint_ = this->endPoint; - session->defaultSessionConnection_ = shared_from_this(); - session->endPointToSessionConnection.insert(make_pair(this->endPoint, shared_from_this())); - } - } - return reconnect; -} - -void SessionConnection::insertStringRecord(const TSInsertStringRecordReq& request) { - auto rpc = [this, request]() { - return this->insertStringRecordInternal(request); - }; - callWithRetryAndVerifyWithRedirection(rpc); -} - -void SessionConnection::insertRecord(const TSInsertRecordReq& request) { - auto rpc = [this, request]() { - return this->insertRecordInternal(request); - }; - callWithRetryAndVerifyWithRedirection(rpc); -} - -void SessionConnection::insertStringRecords(const TSInsertStringRecordsReq& request) { - auto rpc = [this, request]() { - return this->insertStringRecordsInternal(request); - }; - callWithRetryAndVerifyWithRedirection(rpc); -} - -void SessionConnection::insertRecords(const TSInsertRecordsReq& request) { - auto rpc = [this, request]() { - return this->insertRecordsInternal(request); - }; - callWithRetryAndVerifyWithRedirection(rpc); -} - -void SessionConnection::insertRecordsOfOneDevice(TSInsertRecordsOfOneDeviceReq request) { - auto rpc = [this, request]() { - return this->insertRecordsOfOneDeviceInternal(request); - }; - callWithRetryAndVerifyWithRedirection(rpc); -} - -void SessionConnection::insertStringRecordsOfOneDevice(TSInsertStringRecordsOfOneDeviceReq request) { - auto rpc = [this, request]() { - return this->insertStringRecordsOfOneDeviceInternal(request); - }; - callWithRetryAndVerifyWithRedirection(rpc); -} - -void SessionConnection::insertTablet(TSInsertTabletReq request) { - auto rpc = [this, request]() { - return this->insertTabletInternal(request); - }; - callWithRetryAndVerifyWithRedirection(rpc); -} - -void SessionConnection::insertTablets(TSInsertTabletsReq request) { - auto rpc = [this, request]() { - return this->insertTabletsInternal(request); - }; - callWithRetryAndVerifyWithRedirection(rpc); -} - -void SessionConnection::testInsertStringRecord(TSInsertStringRecordReq& request) { - auto rpc = [this, &request]() { - request.sessionId = sessionId; - TSStatus ret; - client->testInsertStringRecord(ret, request); - return ret; - }; - auto status = callWithRetryAndReconnect(rpc).getResult(); - RpcUtils::verifySuccess(status); -} - -void SessionConnection::testInsertTablet(TSInsertTabletReq& request) { - auto rpc = [this, &request]() { - request.sessionId = sessionId; - TSStatus ret; - client->testInsertTablet(ret, request); - return ret; - }; - auto status = callWithRetryAndReconnect(rpc).getResult(); - RpcUtils::verifySuccess(status); -} - -void SessionConnection::testInsertRecords(TSInsertRecordsReq& request) { - auto rpc = [this, &request]() { - request.sessionId = sessionId; - TSStatus ret; - client->testInsertRecords(ret, request); - return ret; - }; - auto status = callWithRetryAndReconnect(rpc).getResult(); - RpcUtils::verifySuccess(status); -} - -void SessionConnection::deleteTimeseries(const vector& paths) { - auto rpc = [this, &paths]() { - TSStatus ret; - client->deleteTimeseries(ret, sessionId, paths); - return ret; - }; - callWithRetryAndVerify(rpc); -} - -void SessionConnection::deleteData(const TSDeleteDataReq& request) { - auto rpc = [this, request]() { - return this->deleteDataInternal(request); - }; - callWithRetryAndVerify(rpc); -} - -void SessionConnection::setStorageGroup(const string& storageGroupId) { - auto rpc = [this, &storageGroupId]() { - TSStatus ret; - client->setStorageGroup(ret, sessionId, storageGroupId); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -void SessionConnection::deleteStorageGroups(const vector& storageGroups) { - auto rpc = [this, &storageGroups]() { - TSStatus ret; - client->deleteStorageGroups(ret, sessionId, storageGroups); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -void SessionConnection::createTimeseries(TSCreateTimeseriesReq& req) { - auto rpc = [this, &req]() { - TSStatus ret; - req.sessionId = sessionId; - client->createTimeseries(ret, req); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -void SessionConnection::createMultiTimeseries(TSCreateMultiTimeseriesReq& req) { - auto rpc = [this, &req]() { - TSStatus ret; - req.sessionId = sessionId; - client->createMultiTimeseries(ret, req); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -void SessionConnection::createAlignedTimeseries(TSCreateAlignedTimeseriesReq& req) { - auto rpc = [this, &req]() { - TSStatus ret; - req.sessionId = sessionId; - client->createAlignedTimeseries(ret, req); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -TSGetTimeZoneResp SessionConnection::getTimeZone() { - auto rpc = [this]() { - TSGetTimeZoneResp resp; - client->getTimeZone(resp, sessionId); - zoneId = resp.timeZone; - return resp; - }; - auto ret = callWithRetryAndReconnect(rpc, - [](const TSGetTimeZoneResp& resp) { - return resp.status; - }); - RpcUtils::verifySuccess(ret.getResult().status); - return ret.result; -} - -void SessionConnection::setTimeZone(TSSetTimeZoneReq& req) { - auto rpc = [this, &req]() { - TSStatus ret; - req.sessionId = sessionId; - client->setTimeZone(ret, req); - zoneId = req.timeZone; - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -void SessionConnection::createSchemaTemplate(TSCreateSchemaTemplateReq req) { - auto rpc = [this, &req]() { - TSStatus ret; - req.sessionId = sessionId; - client->createSchemaTemplate(ret, req); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -void SessionConnection::setSchemaTemplate(TSSetSchemaTemplateReq req) { - auto rpc = [this, &req]() { - TSStatus ret; - req.sessionId = sessionId; - client->setSchemaTemplate(ret, req); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -void SessionConnection::unsetSchemaTemplate(TSUnsetSchemaTemplateReq req) { - auto rpc = [this, &req]() { - TSStatus ret; - req.sessionId = sessionId; - client->unsetSchemaTemplate(ret, req); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -void SessionConnection::appendSchemaTemplate(TSAppendSchemaTemplateReq req) { - auto rpc = [this, &req]() { - TSStatus ret; - req.sessionId = sessionId; - client->appendSchemaTemplate(ret, req); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -void SessionConnection::pruneSchemaTemplate(TSPruneSchemaTemplateReq req) { - auto rpc = [this, &req]() { - TSStatus ret; - req.sessionId = sessionId; - client->pruneSchemaTemplate(ret, req); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc); - RpcUtils::verifySuccess(ret.getResult()); -} - -TSQueryTemplateResp SessionConnection::querySchemaTemplate(TSQueryTemplateReq req) { - auto rpc = [this, &req]() { - TSQueryTemplateResp ret; - req.sessionId = sessionId; - client->querySchemaTemplate(ret, req); - return ret; - }; - auto ret = callWithRetryAndReconnect(rpc, - [](const TSQueryTemplateResp& resp) { - return resp.status; - }); - RpcUtils::verifySuccess(ret.getResult().status); - return ret.getResult(); -} - -TSStatus SessionConnection::insertStringRecordInternal(TSInsertStringRecordReq request) { - request.sessionId = sessionId; - TSStatus ret; - client->insertStringRecord(ret, request); - return ret; -} - -TSStatus SessionConnection::insertRecordInternal(TSInsertRecordReq request) { - request.sessionId = sessionId; - TSStatus ret; - client->insertRecord(ret, request); - return ret; -} - -TSStatus SessionConnection::insertStringRecordsInternal(TSInsertStringRecordsReq request) { - request.sessionId = sessionId; - TSStatus ret; - client->insertStringRecords(ret, request); - return ret; -} - -TSStatus SessionConnection::insertRecordsInternal(TSInsertRecordsReq request) { - request.sessionId = sessionId; - TSStatus ret; - client->insertRecords(ret, request); - return ret; -} - -TSStatus SessionConnection::insertRecordsOfOneDeviceInternal(TSInsertRecordsOfOneDeviceReq request) { - request.sessionId = sessionId; - TSStatus ret; - client->insertRecordsOfOneDevice(ret, request); - return ret; -} - -TSStatus SessionConnection::insertStringRecordsOfOneDeviceInternal(TSInsertStringRecordsOfOneDeviceReq request) { - request.sessionId = sessionId; - TSStatus ret; - client->insertStringRecordsOfOneDevice(ret, request); - return ret; -} - -TSStatus SessionConnection::insertTabletInternal(TSInsertTabletReq request) { - request.sessionId = sessionId; - TSStatus ret; - client->insertTablet(ret, request); - return ret; -} - -TSStatus SessionConnection::insertTabletsInternal(TSInsertTabletsReq request) { - request.sessionId = sessionId; - TSStatus ret; - client->insertTablets(ret, request); - return ret; -} - -TSStatus SessionConnection::deleteDataInternal(TSDeleteDataReq request) { - request.sessionId = sessionId; - TSStatus ret; - client->deleteData(ret, request); - return ret; -} diff --git a/iotdb-client/client-cpp/src/main/SessionConnection.h b/iotdb-client/client-cpp/src/main/SessionConnection.h deleted file mode 100644 index 093c48aae452d..0000000000000 --- a/iotdb-client/client-cpp/src/main/SessionConnection.h +++ /dev/null @@ -1,364 +0,0 @@ -/** -* 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. - */ -#ifndef IOTDB_SESSIONCONNECTION_H -#define IOTDB_SESSIONCONNECTION_H - -#include -#include -#include -#include -#include "IClientRPCService.h" -#include "common_types.h" -#include "NodesSupplier.h" -#include "Common.h" - -class SessionDataSet; -class Session; - -class SessionConnection : public std::enable_shared_from_this { -public: - SessionConnection(Session* session_ptr, const TEndPoint& endpoint, - const std::string& zoneId, - std::shared_ptr nodeSupplier, - int fetchSize = 10000, - int maxRetries = 60, - int64_t retryInterval = 500, - std::string dialect = "tree", - std::string db = ""); - - ~SessionConnection(); - - void setTimeZone(const std::string& newZoneId); - - - const TEndPoint& getEndPoint(); - - void init(const TEndPoint& endpoint); - - void insertStringRecord(const TSInsertStringRecordReq& request); - - void insertRecord(const TSInsertRecordReq& request); - - void insertStringRecords(const TSInsertStringRecordsReq& request); - - void insertRecords(const TSInsertRecordsReq& request); - - void insertRecordsOfOneDevice(TSInsertRecordsOfOneDeviceReq request); - - void insertStringRecordsOfOneDevice(TSInsertStringRecordsOfOneDeviceReq request); - - void insertTablet(TSInsertTabletReq request); - - void insertTablets(TSInsertTabletsReq request); - - void testInsertStringRecord(TSInsertStringRecordReq& request); - - void testInsertTablet(TSInsertTabletReq& request); - - void testInsertRecords(TSInsertRecordsReq& request); - - void deleteTimeseries(const vector& paths); - - void deleteData(const TSDeleteDataReq& request); - - void setStorageGroup(const string& storageGroupId); - - void deleteStorageGroups(const vector& storageGroups); - - void createTimeseries(TSCreateTimeseriesReq& req); - - void createMultiTimeseries(TSCreateMultiTimeseriesReq& req); - - void createAlignedTimeseries(TSCreateAlignedTimeseriesReq& req); - - TSGetTimeZoneResp getTimeZone(); - - void setTimeZone(TSSetTimeZoneReq& req); - - void createSchemaTemplate(TSCreateSchemaTemplateReq req); - - void setSchemaTemplate(TSSetSchemaTemplateReq req); - - void unsetSchemaTemplate(TSUnsetSchemaTemplateReq req); - - void appendSchemaTemplate(TSAppendSchemaTemplateReq req); - - void pruneSchemaTemplate(TSPruneSchemaTemplateReq req); - - TSQueryTemplateResp querySchemaTemplate(TSQueryTemplateReq req); - - std::unique_ptr executeRawDataQuery(const std::vector& paths, int64_t startTime, - int64_t endTime); - - std::unique_ptr executeLastDataQuery(const std::vector& paths, int64_t lastTime); - - void executeNonQueryStatement(const std::string& sql); - - std::unique_ptr executeQueryStatement(const std::string& sql, int64_t timeoutInMs = -1); - - std::shared_ptr getSessionClient() { - return client; - } - - friend class Session; - -private: - void close(); - std::string getSystemDefaultZoneId(); - bool reconnect(); - - template - struct RetryResult { - T result; - std::exception_ptr exception; - int retryAttempts; - - RetryResult(T r, std::exception_ptr e, int a) - : result(r), exception(e), retryAttempts(a) { - } - - int getRetryAttempts() const { return retryAttempts; } - T getResult() const { return result; } - std::exception_ptr getException() const { return exception; } - }; - - template - void callWithRetryAndVerifyWithRedirection(std::function rpc); - - template - void callWithRetryAndVerifyWithRedirectionForMultipleDevices( - std::function rpc, const vector& deviceIds); - - template - RetryResult callWithRetryAndVerify(std::function rpc); - - template - RetryResult callWithRetry(std::function rpc); - - template - RetryResult callWithRetryAndReconnect(RpcFunc rpc); - - template - RetryResult callWithRetryAndReconnect(RpcFunc rpc, StatusGetter statusGetter); - - template - RetryResult callWithRetryAndReconnect(RpcFunc rpc, ShouldRetry shouldRetry, ForceReconnect forceReconnect); - - TSStatus insertStringRecordInternal(TSInsertStringRecordReq request); - - TSStatus insertRecordInternal(TSInsertRecordReq request); - - TSStatus insertStringRecordsInternal(TSInsertStringRecordsReq request); - - TSStatus insertRecordsInternal(TSInsertRecordsReq request); - - TSStatus insertRecordsOfOneDeviceInternal(TSInsertRecordsOfOneDeviceReq request); - - TSStatus insertStringRecordsOfOneDeviceInternal(TSInsertStringRecordsOfOneDeviceReq request); - - TSStatus insertTabletInternal(TSInsertTabletReq request); - - TSStatus insertTabletsInternal(TSInsertTabletsReq request); - - TSStatus deleteDataInternal(TSDeleteDataReq request); - - std::shared_ptr transport; - std::shared_ptr client; - Session* session; - int64_t sessionId{}; - int64_t statementId{}; - int64_t connectionTimeoutInMs{}; - bool enableRPCCompression = false; - std::string zoneId; - TEndPoint endPoint; - std::vector endPointList; - std::shared_ptr availableNodes; - int fetchSize; - int maxRetryCount; - int64_t retryIntervalMs; - std::string sqlDialect; - std::string database; - int timeFactor = 1000; -}; - -template -SessionConnection::RetryResult SessionConnection::callWithRetry(std::function rpc) { - std::exception_ptr lastException = nullptr; - TSStatus status; - int i; - for (i = 0; i <= maxRetryCount; i++) { - if (i > 0) { - lastException = nullptr; - status = TSStatus(); - try { - std::this_thread::sleep_for( - std::chrono::milliseconds(retryIntervalMs)); - } - catch (const std::exception& e) { - break; - } - if (!reconnect()) { - continue; - } - } - - try { - status = rpc(); - if (status.__isset.needRetry && status.needRetry) { - continue; - } - break; - } - catch (...) { - lastException = std::current_exception(); - } - } - return {status, lastException, i}; -} - -template -void SessionConnection::callWithRetryAndVerifyWithRedirection(std::function rpc) { - auto result = callWithRetry(rpc); - - auto status = result.getResult(); - if (result.getRetryAttempts() == 0) { - RpcUtils::verifySuccessWithRedirection(status); - } - else { - RpcUtils::verifySuccess(status); - } - - if (result.getException()) { - try { - std::rethrow_exception(result.getException()); - } - catch (const std::exception& e) { - throw IoTDBConnectionException(e.what()); - } - } -} - -template -void SessionConnection::callWithRetryAndVerifyWithRedirectionForMultipleDevices( - std::function rpc, const vector& deviceIds) { - auto result = callWithRetry(rpc); - auto status = result.getResult(); - if (result.getRetryAttempts() == 0) { - RpcUtils::verifySuccessWithRedirectionForMultiDevices(status, deviceIds); - } - else { - RpcUtils::verifySuccess(status); - } - if (result.getException()) { - try { - std::rethrow_exception(result.getException()); - } - catch (const std::exception& e) { - throw IoTDBConnectionException(e.what()); - } - } - result.exception = nullptr; -} - -template -SessionConnection::RetryResult SessionConnection::callWithRetryAndVerify(std::function rpc) { - auto result = callWithRetry(rpc); - RpcUtils::verifySuccess(result.getResult()); - if (result.getException()) { - try { - std::rethrow_exception(result.getException()); - } - catch (const std::exception& e) { - throw IoTDBConnectionException(e.what()); - } - } - return result; -} - -template -SessionConnection::RetryResult SessionConnection::callWithRetryAndReconnect(RpcFunc rpc) { - return callWithRetryAndReconnect(rpc, - [](const TSStatus& status) { - return status.__isset.needRetry && status.needRetry; - }, - [](const TSStatus& status) { - return status.code == TSStatusCode::PLAN_FAILED_NETWORK_PARTITION; - } - ); -} - -template -SessionConnection::RetryResult SessionConnection::callWithRetryAndReconnect(RpcFunc rpc, StatusGetter statusGetter) { - auto shouldRetry = [&statusGetter](const T& t) { - auto status = statusGetter(t); - return status.__isset.needRetry && status.needRetry; - }; - auto forceReconnect = [&statusGetter](const T& t) { - auto status = statusGetter(t); - return status.code == TSStatusCode::PLAN_FAILED_NETWORK_PARTITION;; - }; - return callWithRetryAndReconnect(rpc, shouldRetry, forceReconnect); -} - - -template -SessionConnection::RetryResult SessionConnection::callWithRetryAndReconnect(RpcFunc rpc, - ShouldRetry shouldRetry, - ForceReconnect forceReconnect) { - std::exception_ptr lastException = nullptr; - T result; - int retryAttempt; - for (retryAttempt = 0; retryAttempt <= maxRetryCount; retryAttempt++) { - try { - result = rpc(); - lastException = nullptr; - } - catch (...) { - result = T(); - lastException = std::current_exception(); - } - - if (!shouldRetry(result)) { - return {result, lastException, retryAttempt}; - } - - if (lastException != nullptr || - std::find(availableNodes->getEndPointList().begin(), availableNodes->getEndPointList().end(), - this->endPoint) == availableNodes->getEndPointList().end() || - forceReconnect(result)) { - reconnect(); - } - - try { - std::this_thread::sleep_for(std::chrono::milliseconds(retryIntervalMs)); - } - catch (const std::exception& e) { - log_debug("Thread was interrupted during retry " + - std::to_string(retryAttempt) + - " with wait time " + - std::to_string(retryIntervalMs) + - " ms. Exiting retry loop."); - break; - } - } - - return {result, lastException, retryAttempt}; -} - -#endif diff --git a/iotdb-client/client-cpp/src/main/SessionDataSet.cpp b/iotdb-client/client-cpp/src/main/SessionDataSet.cpp deleted file mode 100644 index fe28714419a83..0000000000000 --- a/iotdb-client/client-cpp/src/main/SessionDataSet.cpp +++ /dev/null @@ -1,254 +0,0 @@ -/** -* Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -#include "SessionDataSet.h" -#include -#include - -RowRecord::RowRecord(int64_t timestamp) { - this->timestamp = timestamp; -} - -RowRecord::RowRecord(int64_t timestamp, const std::vector& fields) - : timestamp(timestamp), fields(fields) { -} - -RowRecord::RowRecord(const std::vector& fields) - : timestamp(-1), fields(fields) { -} - -RowRecord::RowRecord() { - this->timestamp = -1; -} - -void RowRecord::addField(const Field& f) { - this->fields.push_back(f); -} - -std::string RowRecord::toString() { - std::string ret; - if (this->timestamp != -1) { - ret.append(std::to_string(timestamp)); - ret.append("\t"); - } - for (size_t i = 0; i < fields.size(); i++) { - if (i != 0) { - ret.append("\t"); - } - TSDataType::TSDataType dataType = fields[i].dataType; - switch (dataType) { - case TSDataType::BOOLEAN: - ret.append(fields[i].boolV ? "true" : "false"); - break; - case TSDataType::INT32: - ret.append(std::to_string(fields[i].intV)); - break; - case TSDataType::DATE: - ret.append(boost::gregorian::to_iso_extended_string(fields[i].dateV)); - break; - case TSDataType::TIMESTAMP: - case TSDataType::INT64: - ret.append(std::to_string(fields[i].longV)); - break; - case TSDataType::FLOAT: - ret.append(std::to_string(fields[i].floatV)); - break; - case TSDataType::DOUBLE: - ret.append(std::to_string(fields[i].doubleV)); - break; - case TSDataType::BLOB: - case TSDataType::STRING: - case TSDataType::TEXT: - ret.append(fields[i].stringV); - break; - default: - break; - } - } - ret.append("\n"); - return ret; -} - -bool SessionDataSet::hasNext() { - if (iotdbRpcDataSet_->hasCachedRecord()) { - return true; - } - return iotdbRpcDataSet_->next(); -} - -shared_ptr SessionDataSet::next() { - if (!iotdbRpcDataSet_->hasCachedRecord() && !hasNext()) { - return nullptr; - } - iotdbRpcDataSet_->setHasCachedRecord(false); - - return constructRowRecordFromValueArray(); -} - -int SessionDataSet::getFetchSize() { - return iotdbRpcDataSet_->getFetchSize(); -} - -void SessionDataSet::setFetchSize(int fetchSize) { - return iotdbRpcDataSet_->setFetchSize(fetchSize); -} - -const std::vector& SessionDataSet::getColumnNames() const { - return iotdbRpcDataSet_->getColumnNameList(); -} - -const std::vector& SessionDataSet::getColumnTypeList() const { - return iotdbRpcDataSet_->getColumnTypeList(); -} - -void SessionDataSet::closeOperationHandle(bool forceClose) { - iotdbRpcDataSet_->close(forceClose); -} - -bool SessionDataSet::DataIterator::next() { - return iotdbRpcDataSet_->next(); -} - -bool SessionDataSet::DataIterator::isNull(const std::string& columnName) { - return iotdbRpcDataSet_->isNullByColumnName(columnName); -} - -bool SessionDataSet::DataIterator::isNullByIndex(int32_t columnIndex) { - return iotdbRpcDataSet_->isNullByIndex(columnIndex); -} - -bool SessionDataSet::DataIterator::getBooleanByIndex(int32_t columnIndex) { - return iotdbRpcDataSet_->getBooleanByIndex(columnIndex); -} - -bool SessionDataSet::DataIterator::getBoolean(const std::string& columnName) { - return iotdbRpcDataSet_->getBoolean(columnName); -} - -double SessionDataSet::DataIterator::getDoubleByIndex(int32_t columnIndex) { - return iotdbRpcDataSet_->getDoubleByIndex(columnIndex); -} - -double SessionDataSet::DataIterator::getDouble(const std::string& columnName) { - return iotdbRpcDataSet_->getDouble(columnName); -} - -float SessionDataSet::DataIterator::getFloatByIndex(int32_t columnIndex) { - return iotdbRpcDataSet_->getFloatByIndex(columnIndex); -} - -float SessionDataSet::DataIterator::getFloat(const std::string& columnName) { - return iotdbRpcDataSet_->getFloat(columnName); -} - -int32_t SessionDataSet::DataIterator::getIntByIndex(int32_t columnIndex) { - return iotdbRpcDataSet_->getIntByIndex(columnIndex); -} - -int32_t SessionDataSet::DataIterator::getInt(const std::string& columnName) { - return iotdbRpcDataSet_->getInt(columnName); -} - -int64_t SessionDataSet::DataIterator::getLongByIndex(int32_t columnIndex) { - return iotdbRpcDataSet_->getLongByIndex(columnIndex); -} - -int64_t SessionDataSet::DataIterator::getLong(const std::string& columnName) { - return iotdbRpcDataSet_->getLong(columnName); -} - -std::string SessionDataSet::DataIterator::getStringByIndex(int32_t columnIndex) { - return iotdbRpcDataSet_->getStringByIndex(columnIndex); -} - -std::string SessionDataSet::DataIterator::getString(const std::string& columnName) { - return iotdbRpcDataSet_->getString(columnName); -} - -int64_t SessionDataSet::DataIterator::getTimestampByIndex(int32_t columnIndex) { - return iotdbRpcDataSet_->getTimestampByIndex(columnIndex); -} - -int64_t SessionDataSet::DataIterator::getTimestamp(const std::string& columnName) { - return iotdbRpcDataSet_->getTimestamp(columnName); -} - -boost::gregorian::date SessionDataSet::DataIterator::getDateByIndex(int32_t columnIndex) { - return iotdbRpcDataSet_->getDateByIndex(columnIndex); -} - -boost::gregorian::date SessionDataSet::DataIterator::getDate(const std::string& columnName) { - return iotdbRpcDataSet_->getDate(columnName); -} - -int32_t SessionDataSet::DataIterator::findColumn(const std::string& columnName) { - return iotdbRpcDataSet_->findColumn(columnName); -} - -const std::vector& SessionDataSet::DataIterator::getColumnNames() const { - return iotdbRpcDataSet_->getColumnNameList(); -} - -const std::vector& SessionDataSet::DataIterator::getColumnTypeList() const { - return iotdbRpcDataSet_->getColumnTypeList(); -} - -shared_ptr SessionDataSet::constructRowRecordFromValueArray() { - std::vector outFields; - const int32_t valueColumnStartIndex = iotdbRpcDataSet_->getValueColumnStartIndex(); - const int32_t columnSize = iotdbRpcDataSet_->getColumnSize(); - outFields.reserve(columnSize - valueColumnStartIndex); - for (int32_t columnIndex = valueColumnStartIndex + 1; columnIndex <= columnSize; ++columnIndex) { - Field field; - if (!iotdbRpcDataSet_->isNullByIndex(columnIndex)) { - TSDataType::TSDataType dataType = iotdbRpcDataSet_->getDataTypeByIndex(columnIndex); - field.dataType = dataType; - switch (dataType) { - case TSDataType::BOOLEAN: - field.boolV = iotdbRpcDataSet_->getBooleanByIndex(columnIndex); - break; - case TSDataType::INT32: - field.intV = iotdbRpcDataSet_->getIntByIndex(columnIndex); - break; - case TSDataType::DATE: - field.dateV = iotdbRpcDataSet_->getDateByIndex(columnIndex); - break; - case TSDataType::INT64: - case TSDataType::TIMESTAMP: - field.longV = iotdbRpcDataSet_->getLongByIndex(columnIndex); - break; - case TSDataType::FLOAT: - field.floatV = iotdbRpcDataSet_->getFloatByIndex(columnIndex); - break; - case TSDataType::DOUBLE: - field.doubleV = iotdbRpcDataSet_->getDoubleByIndex(columnIndex); - break; - case TSDataType::TEXT: - case TSDataType::BLOB: - case TSDataType::STRING: - field.stringV = iotdbRpcDataSet_->getBinaryByIndex(columnIndex)->getStringValue(); - break; - default: - throw UnSupportedDataTypeException("Data type %s is not supported." + dataType); - } - } - outFields.emplace_back(field); - } - return std::make_shared(iotdbRpcDataSet_->getCurrentRowTime(), outFields); -} diff --git a/iotdb-client/client-cpp/src/main/SessionDataSet.h b/iotdb-client/client-cpp/src/main/SessionDataSet.h deleted file mode 100644 index d6178d97fbafa..0000000000000 --- a/iotdb-client/client-cpp/src/main/SessionDataSet.h +++ /dev/null @@ -1,142 +0,0 @@ -/** -* 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. - */ - -#ifndef IOTDB_SESSION_DATA_SET_H -#define IOTDB_SESSION_DATA_SET_H - -#include -#include -#include -#include -#include -#include "IoTDBRpcDataSet.h" -#include "Column.h" - -class RowRecord { -public: - int64_t timestamp; - std::vector fields; - - explicit RowRecord(int64_t timestamp); - RowRecord(int64_t timestamp, const std::vector& fields); - explicit RowRecord(const std::vector& fields); - RowRecord(); - - void addField(const Field& f); - std::string toString(); -}; - -class SessionDataSet { -public: - SessionDataSet(const std::string& sql, - const std::vector& columnNameList, - const std::vector& columnTypeList, - const std::map& columnNameIndex, - int64_t queryId, - int64_t statementId, - std::shared_ptr client, - int64_t sessionId, - const std::vector& queryResult, - bool ignoreTimestamp, - int64_t timeout, - bool moreData, - int32_t fetchSize, - const std::string& zoneId) { - iotdbRpcDataSet_ = std::make_shared(sql, - columnNameList, - columnTypeList, - columnNameIndex, - ignoreTimestamp, - moreData, - queryId, - statementId, - client, - sessionId, - queryResult, - fetchSize, - timeout, - zoneId, - IoTDBRpcDataSet::DEFAULT_TIME_FORMAT); - } - - ~SessionDataSet() = default; - - bool hasNext(); - shared_ptr next(); - - int getFetchSize(); - void setFetchSize(int fetchSize); - - const std::vector& getColumnNames() const; - const std::vector& getColumnTypeList() const; - void closeOperationHandle(bool forceClose = false); - - class DataIterator { - std::shared_ptr iotdbRpcDataSet_; - - public: - DataIterator(std::shared_ptr iotdbRpcDataSet) : - iotdbRpcDataSet_(iotdbRpcDataSet) { - } - - bool next(); - - bool isNull(const std::string& columnName); - bool isNullByIndex(int32_t columnIndex); - - bool getBooleanByIndex(int32_t columnIndex); - bool getBoolean(const std::string& columnName); - - double getDoubleByIndex(int32_t columnIndex); - double getDouble(const std::string& columnName); - - float getFloatByIndex(int32_t columnIndex); - float getFloat(const std::string& columnName); - - int32_t getIntByIndex(int32_t columnIndex); - int32_t getInt(const std::string& columnName); - - int64_t getLongByIndex(int32_t columnIndex); - int64_t getLong(const std::string& columnName); - - std::string getStringByIndex(int32_t columnIndex); - std::string getString(const std::string& columnName); - - int64_t getTimestampByIndex(int32_t columnIndex); - int64_t getTimestamp(const std::string& columnName); - - boost::gregorian::date getDateByIndex(int32_t columnIndex); - boost::gregorian::date getDate(const std::string& columnName); - - int32_t findColumn(const std::string& columnName); - const std::vector& getColumnNames() const; - const std::vector& getColumnTypeList() const; - }; - - DataIterator getIterator() { - return {iotdbRpcDataSet_}; - }; - -private: - std::shared_ptr constructRowRecordFromValueArray(); - - std::shared_ptr iotdbRpcDataSet_; -}; - -#endif diff --git a/iotdb-client/client-cpp/src/main/ThriftConnection.cpp b/iotdb-client/client-cpp/src/main/ThriftConnection.cpp deleted file mode 100644 index 8cee0c6f0140d..0000000000000 --- a/iotdb-client/client-cpp/src/main/ThriftConnection.cpp +++ /dev/null @@ -1,170 +0,0 @@ -/** -* Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -#include "ThriftConnection.h" -#include -#include -#include - -#include "Session.h" -#include "SessionDataSet.h" - -const int ThriftConnection::THRIFT_DEFAULT_BUFFER_SIZE = 4096; -const int ThriftConnection::THRIFT_MAX_FRAME_SIZE = 1048576; -const int ThriftConnection::CONNECTION_TIMEOUT_IN_MS = 1000; -const int ThriftConnection::DEFAULT_FETCH_SIZE = 10000; - -ThriftConnection::ThriftConnection(const TEndPoint& endPoint, - int thriftDefaultBufferSize, - int thriftMaxFrameSize, - int connectionTimeoutInMs, - int fetchSize) - : endPoint_(endPoint), - thriftDefaultBufferSize_(thriftDefaultBufferSize), - thriftMaxFrameSize_(thriftMaxFrameSize), - connectionTimeoutInMs_(connectionTimeoutInMs), - fetchSize_(fetchSize){ -} - -ThriftConnection::~ThriftConnection() = default; - -void ThriftConnection::initZoneId() { - if (!zoneId_.empty()) { - return; - } - - time_t ts = 0; - struct tm tmv{}; -#if defined(_WIN64) || defined (WIN32) || defined (_WIN32) - localtime_s(&tmv, &ts); -#else - localtime_r(&ts, &tmv); -#endif - - char zoneStr[32]; - strftime(zoneStr, sizeof(zoneStr), "%z", &tmv); - zoneId_ = zoneStr; -} - -void ThriftConnection::init(const std::string& username, - const std::string& password, - bool enableRPCCompression, - const std::string& zoneId, - const std::string& version) { - std::shared_ptr socket(new TSocket(endPoint_.ip, endPoint_.port)); - socket->setConnTimeout(connectionTimeoutInMs_); - transport_ = std::make_shared(socket); - if (!transport_->isOpen()) { - try { - transport_->open(); - } - catch (TTransportException& e) { - throw IoTDBConnectionException(e.what()); - } - } - if (zoneId.empty()) { - initZoneId(); - } - else { - this->zoneId_ = zoneId; - } - - if (enableRPCCompression) { - std::shared_ptr protocol(new TCompactProtocol(transport_)); - client_ = std::make_shared(protocol); - } - else { - std::shared_ptr protocol(new TBinaryProtocol(transport_)); - client_ = std::make_shared(protocol); - } - - std::map configuration; - configuration["version"] = version; - TSOpenSessionReq openReq; - openReq.__set_username(username); - openReq.__set_password(password); - openReq.__set_zoneId(this->zoneId_); - openReq.__set_configuration(configuration); - try { - TSOpenSessionResp openResp; - client_->openSession(openResp, openReq); - RpcUtils::verifySuccess(openResp.status); - sessionId_ = openResp.sessionId; - statementId_ = client_->requestStatementId(sessionId_); - } - catch (const TTransportException& e) { - transport_->close(); - throw IoTDBConnectionException(e.what()); - } catch (const IoTDBException& e) { - transport_->close(); - throw IoTDBException(e.what()); - } catch (const std::exception& e) { - transport_->close(); - throw IoTDBException(e.what()); - } -} - -std::unique_ptr ThriftConnection::executeQueryStatement(const std::string& sql, int64_t timeoutInMs) { - TSExecuteStatementReq req; - req.__set_sessionId(sessionId_); - req.__set_statementId(statementId_); - req.__set_statement(sql); - req.__set_timeout(timeoutInMs); - TSExecuteStatementResp resp; - try { - client_->executeQueryStatementV2(resp, req); - RpcUtils::verifySuccess(resp.status); - } - catch (const TTransportException& e) { - throw IoTDBConnectionException(e.what()); - } catch (const IoTDBException& e) { - throw IoTDBConnectionException(e.what()); - } catch (const std::exception& e) { - throw IoTDBException(e.what()); - } - std::shared_ptr queryDataSet(new TSQueryDataSet(resp.queryDataSet)); - return std::unique_ptr(new SessionDataSet("", resp.columns, resp.dataTypeList, - resp.columnNameIndexMap, resp.queryId, statementId_, - client_, sessionId_, resp.queryResult, resp.ignoreTimeStamp, - connectionTimeoutInMs_, resp.moreData, fetchSize_, zoneId_)); -} - -void ThriftConnection::close() { - try { - if (client_) { - TSCloseSessionReq req; - req.__set_sessionId(sessionId_); - TSStatus tsStatus; - client_->closeSession(tsStatus, req); - } - } - catch (const TTransportException& e) { - throw IoTDBConnectionException(e.what()); - } catch (const std::exception& e) { - throw IoTDBConnectionException(e.what()); - } - - try { - if (transport_->isOpen()) { - transport_->close(); - } - } - catch (const std::exception& e) { - throw IoTDBConnectionException(e.what()); - } -} diff --git a/iotdb-client/client-cpp/src/main/ThriftConnection.h b/iotdb-client/client-cpp/src/main/ThriftConnection.h deleted file mode 100644 index d578e37b1b705..0000000000000 --- a/iotdb-client/client-cpp/src/main/ThriftConnection.h +++ /dev/null @@ -1,71 +0,0 @@ -/** - * 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. - */ -#ifndef IOTDB_THRIFTCONNECTION_H -#define IOTDB_THRIFTCONNECTION_H - -#include -#include -#include "IClientRPCService.h" - -class SessionDataSet; - -class ThriftConnection { -public: - static const int THRIFT_DEFAULT_BUFFER_SIZE; - static const int THRIFT_MAX_FRAME_SIZE; - static const int CONNECTION_TIMEOUT_IN_MS; - static const int DEFAULT_FETCH_SIZE; - - explicit ThriftConnection(const TEndPoint& endPoint, - int thriftDefaultBufferSize = THRIFT_DEFAULT_BUFFER_SIZE, - int thriftMaxFrameSize = THRIFT_MAX_FRAME_SIZE, - int connectionTimeoutInMs = CONNECTION_TIMEOUT_IN_MS, - int fetchSize = DEFAULT_FETCH_SIZE); - - ~ThriftConnection(); - - void init(const std::string& username, - const std::string& password, - bool enableRPCCompression = false, - const std::string& zoneId = std::string(), - const std::string& version = "V_1_0"); - - std::unique_ptr executeQueryStatement(const std::string& sql, int64_t timeoutInMs = -1); - - void close(); - -private: - TEndPoint endPoint_; - - int thriftDefaultBufferSize_; - int thriftMaxFrameSize_; - int connectionTimeoutInMs_; - int fetchSize_; - - std::shared_ptr transport_; - std::shared_ptr client_; - int64_t sessionId_{}; - int64_t statementId_{}; - std::string zoneId_; - int timeFactor_{}; - - void initZoneId(); -}; - -#endif diff --git a/iotdb-client/client-cpp/src/main/TsBlock.cpp b/iotdb-client/client-cpp/src/main/TsBlock.cpp deleted file mode 100644 index 92afbef3f270f..0000000000000 --- a/iotdb-client/client-cpp/src/main/TsBlock.cpp +++ /dev/null @@ -1,121 +0,0 @@ -/** -* Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -#include -#include -#include -#include "TsBlock.h" - -std::shared_ptr TsBlock::create(int32_t positionCount, - std::shared_ptr timeColumn, - std::vector> valueColumns) { - if (valueColumns.empty()) { - throw std::invalid_argument("valueColumns cannot be empty"); - } - return std::shared_ptr(new TsBlock(positionCount, std::move(timeColumn), std::move(valueColumns))); -} - -std::shared_ptr TsBlock::deserialize(const std::string& data) { - MyStringBuffer buffer(data); - - // Read value column count - int32_t valueColumnCount = buffer.getInt(); - if (valueColumnCount < 0) { - throw IoTDBException("TsBlock::deserialize: negative valueColumnCount"); - } - const int64_t minHeaderBytes = - 9LL + 2LL * static_cast(valueColumnCount); - if (minHeaderBytes > static_cast(data.size())) { - throw IoTDBException("TsBlock::deserialize: truncated header"); - } - - // Read value column data types - std::vector valueColumnDataTypes(valueColumnCount); - for (int32_t i = 0; i < valueColumnCount; i++) { - valueColumnDataTypes[i] = static_cast(buffer.getChar()); - } - - // Read position count - int32_t positionCount = buffer.getInt(); - if (positionCount < 0) { - throw IoTDBException("TsBlock::deserialize: negative positionCount"); - } - - // Read column encodings - std::vector columnEncodings(valueColumnCount + 1); - for (int32_t i = 0; i < valueColumnCount + 1; i++) { - columnEncodings[i] = static_cast(buffer.getChar()); - } - - // Read time column - auto timeColumnDecoder = getColumnDecoder(columnEncodings[0]); - auto timeColumn = timeColumnDecoder->readColumn(buffer, TSDataType::INT64, positionCount); - - // Read value columns - std::vector> valueColumns(valueColumnCount); - for (int32_t i = 0; i < valueColumnCount; i++) { - auto valueColumnDecoder = getColumnDecoder(columnEncodings[i + 1]); - valueColumns[i] = valueColumnDecoder->readColumn(buffer, valueColumnDataTypes[i], positionCount); - } - - return create(positionCount, std::move(timeColumn), std::move(valueColumns)); -} - -TsBlock::TsBlock(int32_t positionCount, - std::shared_ptr timeColumn, - std::vector> valueColumns) - : positionCount_(positionCount), - timeColumn_(std::move(timeColumn)), - valueColumns_(std::move(valueColumns)) { -} - -int32_t TsBlock::getPositionCount() const { - return positionCount_; -} - -int64_t TsBlock::getStartTime() const { - return timeColumn_->getLong(0); -} - -int64_t TsBlock::getEndTime() const { - return timeColumn_->getLong(positionCount_ - 1); -} - -bool TsBlock::isEmpty() const { - return positionCount_ == 0; -} - -int64_t TsBlock::getTimeByIndex(int32_t index) const { - return timeColumn_->getLong(index); -} - -int32_t TsBlock::getValueColumnCount() const { - return static_cast(valueColumns_.size()); -} - -const std::shared_ptr TsBlock::getTimeColumn() const { - return timeColumn_; -} - -const std::vector>& TsBlock::getValueColumns() const { - return valueColumns_; -} - -const std::shared_ptr TsBlock::getColumn(int32_t columnIndex) const { - return valueColumns_[columnIndex]; -} diff --git a/iotdb-client/client-cpp/src/main/TsBlock.h b/iotdb-client/client-cpp/src/main/TsBlock.h deleted file mode 100644 index 2ca6b0197445d..0000000000000 --- a/iotdb-client/client-cpp/src/main/TsBlock.h +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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. - */ - -#ifndef IOTDB_TS_BLOCK_H -#define IOTDB_TS_BLOCK_H - -#include -#include -#include "Column.h" - -class TsBlock { -public: - static std::shared_ptr create(int32_t positionCount, - std::shared_ptr timeColumn, - std::vector> valueColumns); - - static std::shared_ptr deserialize(const std::string& data); - - int32_t getPositionCount() const; - int64_t getStartTime() const; - int64_t getEndTime() const; - bool isEmpty() const; - int64_t getTimeByIndex(int32_t index) const; - int32_t getValueColumnCount() const; - const std::shared_ptr getTimeColumn() const; - const std::vector>& getValueColumns() const; - const std::shared_ptr getColumn(int32_t columnIndex) const; - -private: - TsBlock(int32_t positionCount, - std::shared_ptr timeColumn, - std::vector> valueColumns); - - std::shared_ptr timeColumn_; - std::vector> valueColumns_; - int32_t positionCount_; -}; - -#endif diff --git a/iotdb-client/client-cpp/src/rpc/IoTDBRpcDataSet.cpp b/iotdb-client/client-cpp/src/rpc/IoTDBRpcDataSet.cpp new file mode 100644 index 0000000000000..27ea6bca24877 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/IoTDBRpcDataSet.cpp @@ -0,0 +1,734 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include +#include +#include +#include + +#include "IoTDBRpcDataSet.h" + +#include + +#include "Column.h" +#include "Date.h" +#include "Optional.h" +#include "RpcCommon.h" + +using apache::thrift::transport::TTransportException; + +const int32_t IoTDBRpcDataSet::startIndex = 2; +const std::string IoTDBRpcDataSet::TimestampColumnName = "Time"; +const std::string IoTDBRpcDataSet::DEFAULT_TIME_FORMAT = "default"; +const std::string IoTDBRpcDataSet::TIME_PRECISION = "timestamp_precision"; +const std::string IoTDBRpcDataSet::MILLISECOND = "ms"; +const std::string IoTDBRpcDataSet::MICROSECOND = "us"; +const std::string IoTDBRpcDataSet::NANOSECOND = "ns"; + +IoTDBRpcDataSet::IoTDBRpcDataSet( + const std::string &sql, const std::vector &columnNameList, + const std::vector &columnTypeList, + const std::map &columnNameIndex, bool ignoreTimestamp, + bool moreData, int64_t queryId, int64_t statementId, + std::shared_ptr client, int64_t sessionId, + const std::vector &queryResult, int32_t fetchSize, + const int64_t timeout, const std::string &zoneId, + const std::string &timeFormat) + : sql_(sql), isClosed_(false), client_(client), fetchSize_(fetchSize), + timeout_(timeout), hasCachedRecord_(false), lastReadWasNull_(false), + columnSize_(static_cast(columnNameList.size())), + sessionId_(sessionId), queryId_(queryId), statementId_(statementId), + time_(0), ignoreTimestamp_(ignoreTimestamp), moreData_(moreData), + queryResult_(queryResult), curTsBlock_(nullptr), + queryResultSize_(static_cast(queryResult.size())), + queryResultIndex_(0), tsBlockSize_(0), tsBlockIndex_(-1), + timeZoneId_(zoneId), timeFormat_(timeFormat) { + std::vector columnIndex2TsBlockColumnIndexList; + int columnStartIndex = 1; + int resultSetColumnSize = columnNameList_.size(); + // newly generated or updated columnIndex2TsBlockColumnIndexList.size() may + // not be equal to columnNameList.size() so we need + // startIndexForColumnIndex2TsBlockColumnIndexList to adjust the mapping + // relation + int startIndexForColumnIndex2TsBlockColumnIndexList = 0; + const bool serverIncludesTime = + !columnNameList.empty() && columnNameList[0] == TimestampColumnName; + // for Time Column in tree model which should always be the first column and + // its index for TsBlockColumn is -1 + if (!ignoreTimestamp && !serverIncludesTime) { + columnNameList_.push_back(TimestampColumnName); + columnTypeList_.push_back("INT64"); + columnName2TsBlockColumnIndexMap_[TimestampColumnName] = -1; + columnOrdinalMap_[TimestampColumnName] = 1; + if (!columnIndex2TsBlockColumnIndexList.empty()) { + columnIndex2TsBlockColumnIndexList.insert( + columnIndex2TsBlockColumnIndexList.begin(), -1); + startIndexForColumnIndex2TsBlockColumnIndexList = 1; + } + columnStartIndex++; + resultSetColumnSize++; + } + + if (columnIndex2TsBlockColumnIndexList.empty()) { + columnIndex2TsBlockColumnIndexList.reserve(resultSetColumnSize); + if (!ignoreTimestamp && !serverIncludesTime) { + startIndexForColumnIndex2TsBlockColumnIndexList = 1; + columnIndex2TsBlockColumnIndexList.push_back(-1); + } + for (size_t i = 0; i < columnNameList.size(); ++i) { + if (!columnNameIndex.empty()) { + auto it = columnNameIndex.find(columnNameList[i]); + columnIndex2TsBlockColumnIndexList.push_back( + it != columnNameIndex.end() ? it->second : static_cast(i)); + } else { + columnIndex2TsBlockColumnIndexList.push_back(static_cast(i)); + } + } + } + + columnNameList_.insert(columnNameList_.end(), columnNameList.begin(), + columnNameList.end()); + columnTypeList_.insert(columnTypeList_.end(), columnTypeList.begin(), + columnTypeList.end()); + + // Initialize data types for TsBlock columns + int32_t tsBlockColumnSize = 0; + for (auto value : columnIndex2TsBlockColumnIndexList) { + if (value > tsBlockColumnSize) { + tsBlockColumnSize = value; + } + } + tsBlockColumnSize += 1; + dataTypeForTsBlockColumn_.resize(tsBlockColumnSize); + + // Populate data types and maps + for (size_t i = 0; i < columnNameList.size(); i++) { + auto columnName = columnNameList[i]; + int32_t tsBlockColumnIndex; + if (!columnNameIndex.empty()) { + auto it = columnNameIndex.find(columnName); + tsBlockColumnIndex = + it != columnNameIndex.end() ? it->second : static_cast(i); + } else { + tsBlockColumnIndex = columnIndex2TsBlockColumnIndexList + [i + startIndexForColumnIndex2TsBlockColumnIndexList]; + } + if (tsBlockColumnIndex != -1) { + if (static_cast(tsBlockColumnIndex) >= + dataTypeForTsBlockColumn_.size()) { + dataTypeForTsBlockColumn_.resize(tsBlockColumnIndex + 1); + } + dataTypeForTsBlockColumn_[tsBlockColumnIndex] = + getDataTypeByStr(columnTypeList[i]); + } + + if (columnName2TsBlockColumnIndexMap_.find(columnName) == + columnName2TsBlockColumnIndexMap_.end()) { + columnOrdinalMap_[columnName] = i + columnStartIndex; + columnName2TsBlockColumnIndexMap_[columnName] = tsBlockColumnIndex; + } + } + + timePrecision_ = getTimePrecision(timeFactor_); + columnIndex2TsBlockColumnIndexList_ = columnIndex2TsBlockColumnIndexList; + + if (serverIncludesTime) { + columnSize_--; + } +} + +IoTDBRpcDataSet::~IoTDBRpcDataSet() { + if (!isClosed_) { + close(); + } +} + +bool IoTDBRpcDataSet::next() { + if (hasCachedBlock()) { + lastReadWasNull_ = false; + constructOneRow(); + return true; + } + + if (hasCachedByteBuffer()) { + constructOneTsBlock(); + constructOneRow(); + return true; + } + + if (moreData_) { + bool hasResultSet = fetchResults(); + if (hasResultSet && hasCachedByteBuffer()) { + constructOneTsBlock(); + constructOneRow(); + return true; + } + } + + close(); + return false; +} + +void IoTDBRpcDataSet::close(bool forceClose) { + if ((!forceClose) && isClosed_) { + return; + } + TSCloseOperationReq closeReq; + closeReq.__set_sessionId(sessionId_); + closeReq.__set_statementId(statementId_); + closeReq.__set_queryId(queryId_); + TSStatus tsStatus; + try { + client_->closeOperation(tsStatus, closeReq); + RpcUtils::verifySuccess(tsStatus); + } catch (const TTransportException &e) { + log_debug(e.what()); + throw IoTDBConnectionException(e.what()); + } catch (const IoTDBException &e) { + log_debug(e.what()); + throw; + } catch (std::exception &e) { + log_debug(e.what()); + throw IoTDBException(e.what()); + } + isClosed_ = true; + client_ = nullptr; +} + +bool IoTDBRpcDataSet::fetchResults() { + if (isClosed_) { + throw IoTDBException("This data set is already closed"); + } + + TSFetchResultsReq req; + req.__set_sessionId(sessionId_); + req.__set_statement(sql_); + req.__set_fetchSize(fetchSize_); + req.__set_queryId(queryId_); + req.__set_isAlign(true); + req.__set_timeout(timeout_); + TSFetchResultsResp resp; + client_->fetchResultsV2(resp, req); + RpcUtils::verifySuccess(resp.status); + moreData_ = resp.moreData; + if (!resp.hasResultSet) { + close(); + } else { + queryResult_ = resp.queryResult; + queryResultIndex_ = 0; + if (!queryResult_.empty()) { + queryResultSize_ = queryResult_.size(); + } else { + queryResultSize_ = 0; + } + tsBlockIndex_ = -1; + tsBlockSize_ = 0; + } + return resp.hasResultSet; +} + +void IoTDBRpcDataSet::constructOneRow() { + tsBlockIndex_++; + hasCachedRecord_ = true; + time_ = curTsBlock_->getTimeColumn()->getLong(tsBlockIndex_); +} + +void IoTDBRpcDataSet::constructOneTsBlock() { + lastReadWasNull_ = false; + const auto &curTsBlockBytes = queryResult_[queryResultIndex_]; + queryResultIndex_++; + curTsBlock_ = TsBlock::deserialize(curTsBlockBytes); + tsBlockIndex_ = -1; + tsBlockSize_ = curTsBlock_->getPositionCount(); +} + +bool IoTDBRpcDataSet::isNullByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return isNull(index, tsBlockIndex_); +} + +bool IoTDBRpcDataSet::isNullByColumnName(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return isNull(index, tsBlockIndex_); +} + +bool IoTDBRpcDataSet::isNull(int32_t index, int32_t rowNum) { + return index >= 0 && curTsBlock_->getColumn(index)->isNull(rowNum); +} + +Optional IoTDBRpcDataSet::getBooleanByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getBooleanByTsBlockColumnIndex(index); +} + +Optional IoTDBRpcDataSet::getBoolean(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getBooleanByTsBlockColumnIndex(index); +} + +// Note: tsBlockColumnIndex < 0 indicates the time pseudo-column in tree model. +// Only getLong and getString support reading the time column directly. +// All other typed getters throw IoTDBException to prevent undefined behavior +// from accessing valueColumns_ with a negative index. +Optional +IoTDBRpcDataSet::getBooleanByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + checkRecord(); + if (tsBlockColumnIndex < 0) { + throw IoTDBException("Cannot read boolean from time column"); + } + if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { + lastReadWasNull_ = false; + return curTsBlock_->getColumn(tsBlockColumnIndex) + ->getBoolean(tsBlockIndex_); + } else { + lastReadWasNull_ = true; + return Optional::none(); + } +} + +Optional IoTDBRpcDataSet::getDoubleByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getDoubleByTsBlockColumnIndex(index); +} + +Optional IoTDBRpcDataSet::getDouble(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getDoubleByTsBlockColumnIndex(index); +} + +Optional +IoTDBRpcDataSet::getDoubleByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + checkRecord(); + if (tsBlockColumnIndex < 0) { + throw IoTDBException("Cannot read double from time column"); + } + if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { + lastReadWasNull_ = false; + return curTsBlock_->getColumn(tsBlockColumnIndex)->getDouble(tsBlockIndex_); + } else { + lastReadWasNull_ = true; + return Optional::none(); + } +} + +Optional IoTDBRpcDataSet::getFloatByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getFloatByTsBlockColumnIndex(index); +} + +Optional IoTDBRpcDataSet::getFloat(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getFloatByTsBlockColumnIndex(index); +} + +Optional +IoTDBRpcDataSet::getFloatByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + checkRecord(); + if (tsBlockColumnIndex < 0) { + throw IoTDBException("Cannot read float from time column"); + } + if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { + lastReadWasNull_ = false; + return curTsBlock_->getColumn(tsBlockColumnIndex)->getFloat(tsBlockIndex_); + } else { + lastReadWasNull_ = true; + return Optional::none(); + } +} + +Optional IoTDBRpcDataSet::getIntByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getIntByTsBlockColumnIndex(index); +} + +Optional IoTDBRpcDataSet::getInt(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getIntByTsBlockColumnIndex(index); +} + +Optional +IoTDBRpcDataSet::getIntByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + checkRecord(); + if (tsBlockColumnIndex < 0) { + throw IoTDBException("Cannot read int32 from time column"); + } + if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { + lastReadWasNull_ = false; + const auto column = curTsBlock_->getColumn(tsBlockColumnIndex); + switch (column->getDataType()) { + case TSDataType::INT64: + return static_cast(column->getLong(tsBlockIndex_)); + case TSDataType::DOUBLE: + return static_cast(column->getDouble(tsBlockIndex_)); + case TSDataType::FLOAT: + return static_cast(column->getFloat(tsBlockIndex_)); + case TSDataType::INT32: + return column->getInt(tsBlockIndex_); + default: + return column->getInt(tsBlockIndex_); + } + } else { + lastReadWasNull_ = true; + return Optional::none(); + } +} + +Optional IoTDBRpcDataSet::getLongByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getLongByTsBlockColumnIndex(index); +} + +Optional IoTDBRpcDataSet::getLong(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getLongByTsBlockColumnIndex(index); +} + +Optional +IoTDBRpcDataSet::getLongByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + checkRecord(); + if (tsBlockColumnIndex < 0) { + lastReadWasNull_ = false; + return curTsBlock_->getTimeByIndex(tsBlockIndex_); + } + if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { + lastReadWasNull_ = false; + const auto column = curTsBlock_->getColumn(tsBlockColumnIndex); + switch (column->getDataType()) { + case TSDataType::INT32: + return static_cast(column->getInt(tsBlockIndex_)); + case TSDataType::FLOAT: + return static_cast(column->getFloat(tsBlockIndex_)); + case TSDataType::DOUBLE: + return static_cast(column->getDouble(tsBlockIndex_)); + case TSDataType::INT64: + return column->getLong(tsBlockIndex_); + default: + return column->getLong(tsBlockIndex_); + } + } else { + lastReadWasNull_ = true; + return Optional::none(); + } +} + +std::shared_ptr IoTDBRpcDataSet::getBinaryByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getBinaryByTsBlockColumnIndex(index); +} + +std::shared_ptr +IoTDBRpcDataSet::getBinary(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getBinaryByTsBlockColumnIndex(index); +} + +std::shared_ptr +IoTDBRpcDataSet::getBinaryByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + checkRecord(); + if (tsBlockColumnIndex < 0) { + throw IoTDBException("Cannot read binary from time column"); + } + if (!isNull(tsBlockColumnIndex, tsBlockIndex_)) { + lastReadWasNull_ = false; + return curTsBlock_->getColumn(tsBlockColumnIndex)->getBinary(tsBlockIndex_); + } else { + lastReadWasNull_ = true; + return nullptr; + } +} + +Optional IoTDBRpcDataSet::getStringByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getStringByTsBlockColumnIndex(index); +} + +Optional +IoTDBRpcDataSet::getString(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getStringByTsBlockColumnIndex(index); +} + +Optional +IoTDBRpcDataSet::getStringByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + checkRecord(); + if (tsBlockColumnIndex < 0) { + int64_t timestamp = curTsBlock_->getTimeByIndex(tsBlockIndex_); + return std::to_string(timestamp); + } + if (isNull(tsBlockColumnIndex, tsBlockIndex_)) { + lastReadWasNull_ = true; + return Optional::none(); + } + lastReadWasNull_ = false; + return getStringByTsBlockColumnIndexAndDataType( + tsBlockColumnIndex, getDataTypeByTsBlockColumnIndex(tsBlockColumnIndex)); +} + +std::string IoTDBRpcDataSet::getStringByTsBlockColumnIndexAndDataType( + int32_t index, TSDataType::TSDataType tsDataType) { + const auto column = curTsBlock_->getColumn(index); + const TSDataType::TSDataType physicalType = column->getDataType(); + switch (physicalType) { + case TSDataType::BOOLEAN: + return std::to_string(column->getBoolean(tsBlockIndex_)); + case TSDataType::INT32: + return std::to_string(column->getInt(tsBlockIndex_)); + case TSDataType::INT64: + return std::to_string(column->getLong(tsBlockIndex_)); + case TSDataType::TIMESTAMP: { + int64_t value = column->getLong(tsBlockIndex_); + return formatDatetime(timeFormat_, timePrecision_, value, timeZoneId_); + } + case TSDataType::FLOAT: + return std::to_string(column->getFloat(tsBlockIndex_)); + case TSDataType::DOUBLE: + return std::to_string(column->getDouble(tsBlockIndex_)); + case TSDataType::TEXT: + case TSDataType::STRING: + case TSDataType::OBJECT: + case TSDataType::BLOB: + return column->getBinary(tsBlockIndex_)->getStringValue(); + case TSDataType::DATE: { + int32_t value = column->getInt(tsBlockIndex_); + return parseIntToDate(value).toIsoExtendedString(); + } + default: { + if (tsDataType == TSDataType::DATE) { + auto date = getDateByTsBlockColumnIndex(index); + if (date.is_initialized()) { + return date.value().toIsoExtendedString(); + } + } + return ""; + } + } +} + +Optional IoTDBRpcDataSet::getTimestampByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getTimestampByTsBlockColumnIndex(index); +} + +Optional IoTDBRpcDataSet::getTimestamp(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getTimestampByTsBlockColumnIndex(index); +} + +Optional +IoTDBRpcDataSet::getTimestampByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + return getLongByTsBlockColumnIndex(tsBlockColumnIndex); +} + +Optional IoTDBRpcDataSet::getDateByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getDateByTsBlockColumnIndex(index); +} + +Optional IoTDBRpcDataSet::getDate(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getDateByTsBlockColumnIndex(index); +} + +Optional +IoTDBRpcDataSet::getDateByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + checkRecord(); + if (tsBlockColumnIndex < 0) { + throw IoTDBException("Cannot read date from time column"); + } + if (isNull(tsBlockColumnIndex, tsBlockIndex_)) { + lastReadWasNull_ = true; + return Optional::none(); + } + lastReadWasNull_ = false; + const auto column = curTsBlock_->getColumn(tsBlockColumnIndex); + switch (column->getDataType()) { + case TSDataType::INT32: + return parseIntToDate(column->getInt(tsBlockIndex_)); + case TSDataType::INT64: + return parseIntToDate(static_cast(column->getLong(tsBlockIndex_))); + case TSDataType::DOUBLE: + return parseIntToDate( + static_cast(column->getDouble(tsBlockIndex_))); + case TSDataType::FLOAT: + return parseIntToDate( + static_cast(column->getFloat(tsBlockIndex_))); + case TSDataType::TEXT: + case TSDataType::STRING: + case TSDataType::BLOB: + case TSDataType::OBJECT: { + const auto binary = column->getBinary(tsBlockIndex_); + const std::string &text = binary->getStringValue(); + if (text.empty()) { + return IoTDBDate::notADate(); + } + // Server may return DATE as packed int in text or ISO-8601 string. + try { + return parseIntToDate(static_cast(std::stol(text))); + } catch (const std::exception &) { + int year = 0, month = 0, day = 0; + if (std::sscanf(text.c_str(), "%d-%d-%d", &year, &month, &day) == 3) { + return IoTDBDate(year, month, day); + } + throw IoTDBException("Cannot parse DATE value: " + text); + } + } + default: + return parseIntToDate( + getIntByTsBlockColumnIndex(tsBlockColumnIndex).value()); + } +} + +TSDataType::TSDataType +IoTDBRpcDataSet::getDataTypeByIndex(int32_t columnIndex) { + int32_t index = getTsBlockColumnIndexForColumnIndex(columnIndex); + return getDataTypeByTsBlockColumnIndex(index); +} + +TSDataType::TSDataType +IoTDBRpcDataSet::getDataType(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + return getDataTypeByTsBlockColumnIndex(index); +} + +TSDataType::TSDataType +IoTDBRpcDataSet::getColumnPhysicalDataType(const std::string &columnName) { + int32_t index = getTsBlockColumnIndexForColumnName(columnName); + if (index < 0) { + return TSDataType::TIMESTAMP; + } + checkRecord(); + return curTsBlock_->getColumn(index)->getDataType(); +} + +int32_t +IoTDBRpcDataSet::getTsBlockColumnIndexForColumnIndex(int32_t columnIndex) { + const int32_t adjusted_index = columnIndex - 1; + if (adjusted_index >= + static_cast(columnIndex2TsBlockColumnIndexList_.size()) || + adjusted_index < 0) { + throw std::out_of_range( + "Index " + std::to_string(adjusted_index) + " out of range [0, " + + std::to_string(columnIndex2TsBlockColumnIndexList_.size()) + ")"); + } + return columnIndex2TsBlockColumnIndexList_[adjusted_index]; +} + +TSDataType::TSDataType +IoTDBRpcDataSet::getDataTypeByTsBlockColumnIndex(int32_t tsBlockColumnIndex) { + if (tsBlockColumnIndex < 0) { + return TSDataType::TIMESTAMP; + } else { + return dataTypeForTsBlockColumn_[tsBlockColumnIndex]; + } +} + +int32_t IoTDBRpcDataSet::findColumn(const std::string &columnName) { + auto it = columnOrdinalMap_.find(columnName); + if (it != columnOrdinalMap_.end()) { + return it->second; + } + return -1; +} + +std::string IoTDBRpcDataSet::findColumnNameByIndex(int32_t columnIndex) { + if (columnIndex <= 0) { + throw IoTDBException("column index should start from 1"); + } + if (columnIndex > static_cast(columnNameList_.size())) { + throw IoTDBException("Column index " + std::to_string(columnIndex) + + " is out of range. Valid range is 0 to " + + std::to_string(columnNameList_.size() - 1)); + } + return columnNameList_[columnIndex - 1]; +} + +int32_t IoTDBRpcDataSet::getTsBlockColumnIndexForColumnName( + const std::string &columnName) { + auto it = columnName2TsBlockColumnIndexMap_.find(columnName); + if (it == columnName2TsBlockColumnIndexMap_.end()) { + throw IoTDBException("unknown column name: " + columnName); + } + return it->second; +} + +void IoTDBRpcDataSet::checkRecord() { + if (queryResultIndex_ > queryResultSize_ || tsBlockIndex_ >= tsBlockSize_ || + queryResult_.empty() || !curTsBlock_) { + throw IoTDBException("no record remains"); + } +} + +int32_t IoTDBRpcDataSet::getValueColumnStartIndex() const { + return getValueColumnNameListIndex(0); +} + +int32_t IoTDBRpcDataSet::getServerColumnCount() const { return columnSize_; } + +int32_t +IoTDBRpcDataSet::getValueColumnNameListIndex(int32_t valueColumnOrdinal) const { + if (valueColumnOrdinal < 0 || valueColumnOrdinal >= columnSize_) { + throw std::out_of_range( + "value column ordinal " + std::to_string(valueColumnOrdinal) + + " out of range [0, " + std::to_string(columnSize_) + ")"); + } + if (!columnNameList_.empty() && columnNameList_[0] == TimestampColumnName) { + return valueColumnOrdinal + 1; + } + return valueColumnOrdinal; +} + +int32_t IoTDBRpcDataSet::getColumnSize() const { + return static_cast(columnNameList_.size()); +} + +const std::vector &IoTDBRpcDataSet::getColumnTypeList() const { + return columnTypeList_; +} + +const std::vector &IoTDBRpcDataSet::getColumnNameList() const { + return columnNameList_; +} + +bool IoTDBRpcDataSet::isClosed() const { return isClosed_; } + +int32_t IoTDBRpcDataSet::getFetchSize() const { return fetchSize_; } + +void IoTDBRpcDataSet::setFetchSize(int32_t fetchSize) { + fetchSize_ = fetchSize; +} + +bool IoTDBRpcDataSet::hasCachedRecord() const { return hasCachedRecord_; } + +void IoTDBRpcDataSet::setHasCachedRecord(bool hasCachedRecord) { + hasCachedRecord_ = hasCachedRecord; +} + +bool IoTDBRpcDataSet::isLastReadWasNull() const { return lastReadWasNull_; } + +int64_t IoTDBRpcDataSet::getCurrentRowTime() const { return time_; } + +bool IoTDBRpcDataSet::isIgnoreTimestamp() const { return ignoreTimestamp_; } + +bool IoTDBRpcDataSet::hasCachedBlock() const { + return curTsBlock_ && tsBlockIndex_ < tsBlockSize_ - 1; +} + +bool IoTDBRpcDataSet::hasCachedByteBuffer() const { + return !queryResult_.empty() && queryResultIndex_ < queryResultSize_; +} diff --git a/iotdb-client/client-cpp/src/rpc/IoTDBRpcDataSet.h b/iotdb-client/client-cpp/src/rpc/IoTDBRpcDataSet.h new file mode 100644 index 0000000000000..2a880ae619212 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/IoTDBRpcDataSet.h @@ -0,0 +1,164 @@ +/** + * 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. + */ + +#ifndef IOTDB_RPC_DATA_SET_H +#define IOTDB_RPC_DATA_SET_H + +#include "Date.h" +#include "IClientRPCService.h" +#include "Optional.h" +#include "TsBlock.h" +#include +#include +#include +#include +#include + +class IoTDBRpcDataSet { +public: + static const int32_t startIndex; + static const std::string TimestampColumnName; + + static const std::string DEFAULT_TIME_FORMAT; + static const std::string TIME_PRECISION; + static const std::string MILLISECOND; + static const std::string MICROSECOND; + static const std::string NANOSECOND; + + IoTDBRpcDataSet(const std::string &sql, + const std::vector &columnNameList, + const std::vector &columnTypeList, + const std::map &columnNameIndex, + bool ignoreTimestamp, bool moreData, int64_t queryId, + int64_t statementId, + std::shared_ptr client, + int64_t sessionId, + const std::vector &queryResult, + int32_t fetchSize, int64_t timeout, const std::string &zoneId, + const std::string &timeFormat); + + ~IoTDBRpcDataSet(); + + bool next(); + void close(bool forceClose = false); + + bool hasCachedBlock() const; + bool hasCachedByteBuffer() const; + + bool isNull(int32_t index, int32_t rowNum); + bool isNullByIndex(int32_t columnIndex); + bool isNullByColumnName(const std::string &columnName); + Optional getBooleanByIndex(int32_t columnIndex); + Optional getBoolean(const std::string &columnName); + Optional getDoubleByIndex(int32_t columnIndex); + Optional getDouble(const std::string &columnName); + Optional getFloatByIndex(int32_t columnIndex); + Optional getFloat(const std::string &columnName); + Optional getIntByIndex(int32_t columnIndex); + Optional getInt(const std::string &columnName); + Optional getLongByIndex(int32_t columnIndex); + Optional getLong(const std::string &columnName); + std::shared_ptr getBinaryByIndex(int32_t columnIndex); + std::shared_ptr getBinary(const std::string &columnName); + Optional getStringByIndex(int32_t columnIndex); + Optional getString(const std::string &columnName); + Optional getTimestampByIndex(int32_t columnIndex); + Optional getTimestamp(const std::string &columnName); + Optional getDateByIndex(int32_t columnIndex); + Optional getDate(const std::string &columnName); + + TSDataType::TSDataType getDataTypeByIndex(int32_t columnIndex); + TSDataType::TSDataType getDataType(const std::string &columnName); + TSDataType::TSDataType + getColumnPhysicalDataType(const std::string &columnName); + int32_t findColumn(const std::string &columnName); + std::string findColumnNameByIndex(int32_t columnIndex); + int32_t getValueColumnStartIndex() const; + int32_t getServerColumnCount() const; + int32_t getValueColumnNameListIndex(int32_t valueColumnOrdinal) const; + int32_t getColumnSize() const; + const std::vector &getColumnTypeList() const; + const std::vector &getColumnNameList() const; + bool isClosed() const; + int32_t getFetchSize() const; + void setFetchSize(int32_t fetchSize); + bool hasCachedRecord() const; + void setHasCachedRecord(bool hasCachedRecord); + bool isLastReadWasNull() const; + int64_t getCurrentRowTime() const; + bool isIgnoreTimestamp() const; + +private: + bool fetchResults(); + void constructOneRow(); + void constructOneTsBlock(); + int32_t getTsBlockColumnIndexForColumnName(const std::string &columnName); + int32_t getTsBlockColumnIndexForColumnIndex(int32_t columnIndex); + void checkRecord(); + TSDataType::TSDataType + getDataTypeByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + Optional getBooleanByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + std::string + getStringByTsBlockColumnIndexAndDataType(int32_t index, + TSDataType::TSDataType tsDataType); + Optional getDoubleByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + Optional getFloatByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + Optional getIntByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + Optional getLongByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + std::shared_ptr + getBinaryByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + Optional + getStringByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + Optional getDateByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + Optional + getTimestampByTsBlockColumnIndex(int32_t tsBlockColumnIndex); + + std::string sql_; + bool isClosed_; + std::shared_ptr client_; + std::vector columnNameList_; + std::vector columnTypeList_; + std::map columnOrdinalMap_; + std::map columnName2TsBlockColumnIndexMap_; + std::vector columnIndex2TsBlockColumnIndexList_; + std::vector dataTypeForTsBlockColumn_; + int32_t fetchSize_; + int64_t timeout_; + bool hasCachedRecord_; + bool lastReadWasNull_; + int32_t columnSize_; + int64_t sessionId_; + int64_t queryId_; + int64_t statementId_; + int64_t time_; + bool ignoreTimestamp_; + bool moreData_; + std::vector queryResult_; + std::shared_ptr curTsBlock_; + int32_t queryResultSize_; + int32_t queryResultIndex_; + int32_t tsBlockSize_; + int32_t tsBlockIndex_; + std::string timeZoneId_; + std::string timeFormat_; + int32_t timeFactor_{1000}; + std::string timePrecision_; +}; + +#endif // IOTDB_RPC_DATA_SET_H diff --git a/iotdb-client/client-cpp/src/rpc/NodesSupplier.cpp b/iotdb-client/client-cpp/src/rpc/NodesSupplier.cpp new file mode 100644 index 0000000000000..5f79c89e14c88 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/NodesSupplier.cpp @@ -0,0 +1,247 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +#include "NodesSupplier.h" +#include "Session.h" +#include "SessionDataSet.h" +#include +#include +#include + +const std::string NodesSupplier::SHOW_AVAILABLE_URLS_COMMAND = + "SHOW AVAILABLE URLS"; +const std::string NodesSupplier::RUNNING_STATUS = "Running"; +const std::string NodesSupplier::STATUS_COLUMN_NAME = "Status"; +const std::string NodesSupplier::IP_COLUMN_NAME = "RpcAddress"; +const std::string NodesSupplier::PORT_COLUMN_NAME = "RpcPort"; +const std::string NodesSupplier::REMOVING_STATUS = "Removing"; + +const int64_t NodesSupplier::TIMEOUT_IN_MS = 60000; +const int NodesSupplier::THRIFT_DEFAULT_BUFFER_SIZE = 4096; +const int NodesSupplier::THRIFT_MAX_FRAME_SIZE = 1048576; +const int NodesSupplier::CONNECTION_TIMEOUT_IN_MS = 1000; + +TEndPoint RoundRobinPolicy::select(const std::vector &nodes) { + static std::atomic_uint index{0}; + + if (nodes.empty()) { + throw IoTDBException("No available nodes"); + } + + return nodes[index++ % nodes.size()]; +} + +StaticNodesSupplier::StaticNodesSupplier(const std::vector &nodes, + NodeSelectionPolicy policy) + : availableNodes_(nodes), policy_(std::move(policy)) {} + +Optional StaticNodesSupplier::getQueryEndPoint() { + try { + if (availableNodes_.empty()) { + return Optional::none(); + } + return policy_(availableNodes_); + } catch (const IoTDBException &e) { + return Optional::none(); + } +} + +std::vector StaticNodesSupplier::getEndPointList() { + return availableNodes_; +} + +StaticNodesSupplier::~StaticNodesSupplier() = default; + +std::shared_ptr NodesSupplier::create( + const std::vector &endpoints, const std::string &userName, + const std::string &password, bool useSSL, + const std::string &trustCertFilePath, const std::string &zoneId, + int32_t thriftDefaultBufferSize, int32_t thriftMaxFrameSize, + int32_t connectionTimeoutInMs, bool enableRPCCompression, + const std::string &version, std::chrono::milliseconds refreshInterval, + NodeSelectionPolicy policy) { + if (endpoints.empty()) { + return nullptr; + } + auto supplier = std::make_shared( + userName, password, useSSL, trustCertFilePath, zoneId, + thriftDefaultBufferSize, thriftMaxFrameSize, connectionTimeoutInMs, + enableRPCCompression, version, endpoints, policy); + supplier->startBackgroundRefresh(refreshInterval); + return supplier; +} + +NodesSupplier::NodesSupplier( + const std::string &userName, const std::string &password, bool useSSL, + const std::string &trustCertFilePath, const std::string &zoneId, + int32_t thriftDefaultBufferSize, int32_t thriftMaxFrameSize, + int32_t connectionTimeoutInMs, bool enableRPCCompression, + const std::string &version, const std::vector &endpoints, + NodeSelectionPolicy policy) + : userName_(userName), password_(password), zoneId_(zoneId), + thriftDefaultBufferSize_(thriftDefaultBufferSize), + thriftMaxFrameSize_(thriftMaxFrameSize), + connectionTimeoutInMs_(connectionTimeoutInMs), useSSL_(useSSL), + trustCertFilePath_(trustCertFilePath), + enableRPCCompression_(enableRPCCompression), version_(version), + endpoints_(endpoints), selectionPolicy_(policy) { + deduplicateEndpoints(); +} + +std::vector NodesSupplier::getEndPointList() { + std::lock_guard lock(mutex_); + return endpoints_; +} + +TEndPoint NodesSupplier::selectQueryEndpoint() { + std::lock_guard lock(mutex_); + try { + return selectionPolicy_(endpoints_); + } catch (const std::exception &e) { + log_error("NodesSupplier::selectQueryEndpoint exception: %s", e.what()); + throw IoTDBException("NodesSupplier::selectQueryEndpoint exception, " + + std::string(e.what())); + } +} + +Optional NodesSupplier::getQueryEndPoint() { + try { + return selectQueryEndpoint(); + } catch (const IoTDBException &e) { + return Optional::none(); + } +} + +NodesSupplier::~NodesSupplier() { + stopBackgroundRefresh(); + if (client_ != nullptr) { + client_->close(); + } +} + +void NodesSupplier::deduplicateEndpoints() { + std::vector uniqueEndpoints; + uniqueEndpoints.reserve(endpoints_.size()); + for (const auto &endpoint : endpoints_) { + if (std::find(uniqueEndpoints.begin(), uniqueEndpoints.end(), endpoint) == + uniqueEndpoints.end()) { + uniqueEndpoints.push_back(endpoint); + } + } + endpoints_ = std::move(uniqueEndpoints); +} + +void NodesSupplier::startBackgroundRefresh(std::chrono::milliseconds interval) { + isRunning_ = true; + refreshEndpointList(); + refreshThread_ = std::thread([this, interval] { + while (isRunning_) { + refreshEndpointList(); + std::unique_lock cvLock(this->mutex_); + refreshCondition_.wait_for(cvLock, interval, + [this]() { return !isRunning_.load(); }); + } + }); +} + +std::vector NodesSupplier::fetchLatestEndpoints() { + for (const auto &endpoint : endpoints_) { + try { + if (client_ == nullptr) { + client_ = std::make_shared(endpoint); + client_->init(userName_, password_, enableRPCCompression_, useSSL_, + trustCertFilePath_, zoneId_, version_); + } + + auto sessionDataSet = + client_->executeQueryStatement(SHOW_AVAILABLE_URLS_COMMAND); + + uint32_t columnAddrIdx = -1, columnPortIdx = -1; + auto columnNames = sessionDataSet->getColumnNames(); + for (uint32_t i = 0; i < columnNames.size(); i++) { + if (columnNames[i] == IP_COLUMN_NAME) { + columnAddrIdx = i; + } else if (columnNames[i] == PORT_COLUMN_NAME) { + columnPortIdx = i; + } + } + + if (columnAddrIdx == -1 || columnPortIdx == -1) { + throw IoTDBException("Required columns not found in query result."); + } + + std::vector ret; + while (sessionDataSet->hasNext()) { + auto record = sessionDataSet->next(); + std::string ip; + int32_t port = 0; + + if (record->fields.at(columnAddrIdx).stringV.is_initialized()) { + ip = record->fields.at(columnAddrIdx).stringV.value(); + } + if (record->fields.at(columnPortIdx).intV.is_initialized()) { + port = record->fields.at(columnPortIdx).intV.value(); + } + + if (ip == "0.0.0.0") { + log_warn("Skipping invalid node: " + ip + ":" + std::to_string(port)); + continue; + } + TEndPoint newEndpoint; + newEndpoint.ip = ip; + newEndpoint.port = port; + ret.emplace_back(newEndpoint); + } + return ret; // success + } catch (const std::exception &e) { + log_warn("Failed to fetch endpoints from " + endpoint.ip + ":" + + std::to_string(endpoint.port) + " , error=" + e.what()); + client_.reset(); // reset client before retrying next endpoint + continue; // try next endpoint + } + } + log_warn( + "NodesSupplier::fetchLatestEndpoints: SHOW AVAILABLE URLS unavailable, " + "keeping configured endpoints"); + return endpoints_; +} + +void NodesSupplier::refreshEndpointList() { + try { + auto newEndpoints = fetchLatestEndpoints(); + if (newEndpoints.empty()) { + return; + } + + std::lock_guard lock(mutex_); + endpoints_.swap(newEndpoints); + deduplicateEndpoints(); + } catch (const IoTDBException &e) { + log_error(std::string("NodesSupplier::refreshEndpointList failed: ") + + e.what()); + } +} + +void NodesSupplier::stopBackgroundRefresh() noexcept { + if (isRunning_.exchange(false)) { + refreshCondition_.notify_all(); + if (refreshThread_.joinable()) { + refreshThread_.join(); + } + } +} \ No newline at end of file diff --git a/iotdb-client/client-cpp/src/rpc/NodesSupplier.h b/iotdb-client/client-cpp/src/rpc/NodesSupplier.h new file mode 100644 index 0000000000000..57f16524d18dd --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/NodesSupplier.h @@ -0,0 +1,144 @@ +/** + * 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. + */ +#ifndef IOTDB_NODES_SUPPLIER_H +#define IOTDB_NODES_SUPPLIER_H + +#include "Optional.h" +#include +#include +#include +#include +#include +#include +#include +#include + +#include "ThriftConnection.h" + +class TEndPoint; + +class RoundRobinPolicy { +public: + static TEndPoint select(const std::vector &nodes); +}; + +class INodesSupplier { +public: + virtual ~INodesSupplier() = default; + virtual Optional getQueryEndPoint() = 0; + virtual std::vector getEndPointList() = 0; + using NodeSelectionPolicy = + std::function &)>; +}; + +class StaticNodesSupplier : public INodesSupplier { +public: + explicit StaticNodesSupplier( + const std::vector &nodes, + NodeSelectionPolicy policy = RoundRobinPolicy::select); + + Optional getQueryEndPoint() override; + + std::vector getEndPointList() override; + + ~StaticNodesSupplier() override; + +private: + const std::vector availableNodes_; + NodeSelectionPolicy policy_; +}; + +class NodesSupplier : public INodesSupplier { +public: + static const std::string SHOW_AVAILABLE_URLS_COMMAND; + static const std::string RUNNING_STATUS; + static const std::string STATUS_COLUMN_NAME; + static const std::string IP_COLUMN_NAME; + static const std::string PORT_COLUMN_NAME; + static const std::string REMOVING_STATUS; + + static const int64_t TIMEOUT_IN_MS; + static const int THRIFT_DEFAULT_BUFFER_SIZE; + static const int THRIFT_MAX_FRAME_SIZE; + static const int CONNECTION_TIMEOUT_IN_MS; + + static std::shared_ptr create( + const std::vector &endpoints, const std::string &userName, + const std::string &password, bool useSSL = false, + const std::string &trustCertFilePath = "", const std::string &zoneId = "", + int32_t thriftDefaultBufferSize = + ThriftConnection::THRIFT_DEFAULT_BUFFER_SIZE, + int32_t thriftMaxFrameSize = ThriftConnection::THRIFT_MAX_FRAME_SIZE, + int32_t connectionTimeoutInMs = + ThriftConnection::CONNECTION_TIMEOUT_IN_MS, + bool enableRPCCompression = false, const std::string &version = "V_1_0", + std::chrono::milliseconds refreshInterval = + std::chrono::milliseconds(TIMEOUT_IN_MS), + NodeSelectionPolicy policy = RoundRobinPolicy::select); + + NodesSupplier(const std::string &userName, const std::string &password, + bool useSSL, const std::string &trustCertFilePath, + const std::string &zoneId, int32_t thriftDefaultBufferSize, + int32_t thriftMaxFrameSize, int32_t connectionTimeoutInMs, + bool enableRPCCompression, const std::string &version, + const std::vector &endpoints, + NodeSelectionPolicy policy); + + std::vector getEndPointList() override; + + Optional getQueryEndPoint() override; + + ~NodesSupplier() override; + +private: + std::string userName_; + std::string password_; + int32_t thriftDefaultBufferSize_; + int32_t thriftMaxFrameSize_; + int32_t connectionTimeoutInMs_; + bool useSSL_; + std::string trustCertFilePath_; + bool enableRPCCompression_; + std::string version_; + std::string zoneId_; + + std::mutex mutex_; + std::vector endpoints_; + NodeSelectionPolicy selectionPolicy_; + + std::atomic isRunning_{false}; + std::thread refreshThread_; + std::condition_variable refreshCondition_; + + std::shared_ptr client_; + + void deduplicateEndpoints(); + + void startBackgroundRefresh(std::chrono::milliseconds interval); + + std::vector fetchLatestEndpoints(); + + void refreshEndpointList(); + + TEndPoint selectQueryEndpoint(); + + void stopBackgroundRefresh() noexcept; +}; + +#endif \ No newline at end of file diff --git a/iotdb-client/client-cpp/src/rpc/RpcCommon.cpp b/iotdb-client/client-cpp/src/rpc/RpcCommon.cpp new file mode 100644 index 0000000000000..0ec345a1c5ca8 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/RpcCommon.cpp @@ -0,0 +1,216 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "RpcCommon.h" + +#include +#include +#include + +#include "ThriftConvert.h" +#include "client_types.h" +#include "common_types.h" + +using namespace std; + +RpcUtils::RpcUtils() { + SUCCESS_STATUS = std::make_shared(); + SUCCESS_STATUS->__set_code(TSStatusCode::SUCCESS_STATUS); +} + +void RpcUtils::verifySuccess(const TSStatus &status) { + if (status.code == TSStatusCode::MULTIPLE_ERROR) { + verifySuccess(status.subStatus); + return; + } + if (status.code != TSStatusCode::SUCCESS_STATUS && + status.code != TSStatusCode::REDIRECTION_RECOMMEND) { + throw ExecutionException(to_string(status.code) + ": " + status.message, + statusFromThrift(status)); + } +} + +void RpcUtils::verifySuccessWithRedirection(const TSStatus &status) { + verifySuccess(status); + if (status.__isset.redirectNode) { + throw RedirectException(to_string(status.code) + ": " + status.message, + endpointFromThrift(status.redirectNode)); + } + if (status.__isset.subStatus) { + auto statusSubStatus = status.subStatus; + vector endPointList(statusSubStatus.size()); + int count = 0; + for (const TSStatus &subStatus : statusSubStatus) { + if (subStatus.__isset.redirectNode) { + endPointList[count++] = subStatus.redirectNode; + } else { + TEndPoint endPoint; + endPointList[count++] = endPoint; + } + } + if (!endPointList.empty()) { + throw RedirectException(to_string(status.code) + ": " + status.message, + endpointListFromThrift(endPointList)); + } + } +} + +void RpcUtils::verifySuccessWithRedirectionForMultiDevices( + const TSStatus &status, vector devices) { + verifySuccess(status); + + if (status.code == TSStatusCode::MULTIPLE_ERROR || + status.code == TSStatusCode::REDIRECTION_RECOMMEND) { + map deviceEndPointMap; + const vector &statusSubStatus = status.subStatus; + for (size_t i = 0; i < statusSubStatus.size() && i < devices.size(); i++) { + const TSStatus &subStatus = statusSubStatus[i]; + if (subStatus.__isset.redirectNode) { + deviceEndPointMap.insert(make_pair(devices[i], subStatus.redirectNode)); + } + } + throw RedirectException(to_string(status.code) + ": " + status.message, + endpointMapFromThrift(deviceEndPointMap)); + } + + if (status.__isset.redirectNode) { + throw RedirectException(to_string(status.code) + ": " + status.message, + endpointFromThrift(status.redirectNode)); + } + if (status.__isset.subStatus) { + auto statusSubStatus = status.subStatus; + vector endPointList(statusSubStatus.size()); + int count = 0; + for (const TSStatus &subStatus : statusSubStatus) { + if (subStatus.__isset.redirectNode) { + endPointList[count++] = subStatus.redirectNode; + } else { + TEndPoint endPoint; + endPointList[count++] = endPoint; + } + } + if (!endPointList.empty()) { + throw RedirectException(to_string(status.code) + ": " + status.message, + endpointListFromThrift(endPointList)); + } + } +} + +void RpcUtils::verifySuccess(const vector &statuses) { + for (const TSStatus &status : statuses) { + if (status.code != TSStatusCode::SUCCESS_STATUS) { + vector publicStatuses; + publicStatuses.reserve(statuses.size()); + for (const TSStatus &s : statuses) { + publicStatuses.push_back(statusFromThrift(s)); + } + throw BatchExecutionException(status.message, publicStatuses); + } + } +} + +TSStatus RpcUtils::getStatus(TSStatusCode::TSStatusCode tsStatusCode) { + TSStatus status; + status.__set_code(tsStatusCode); + return status; +} + +TSStatus RpcUtils::getStatus(int code, const string &message) { + TSStatus status; + status.__set_code(code); + status.__set_message(message); + return status; +} + +shared_ptr +RpcUtils::getTSExecuteStatementResp(TSStatusCode::TSStatusCode tsStatusCode) { + TSStatus status = getStatus(tsStatusCode); + return getTSExecuteStatementResp(status); +} + +shared_ptr +RpcUtils::getTSExecuteStatementResp(TSStatusCode::TSStatusCode tsStatusCode, + const string &message) { + TSStatus status = getStatus(tsStatusCode, message); + return getTSExecuteStatementResp(status); +} + +shared_ptr +RpcUtils::getTSExecuteStatementResp(const TSStatus &status) { + shared_ptr resp(new TSExecuteStatementResp()); + resp->__set_status(status); + return resp; +} + +shared_ptr +RpcUtils::getTSFetchResultsResp(TSStatusCode::TSStatusCode tsStatusCode) { + TSStatus status = getStatus(tsStatusCode); + return getTSFetchResultsResp(status); +} + +shared_ptr +RpcUtils::getTSFetchResultsResp(TSStatusCode::TSStatusCode tsStatusCode, + const string &appendMessage) { + TSStatus status = getStatus(tsStatusCode, appendMessage); + return getTSFetchResultsResp(status); +} + +shared_ptr +RpcUtils::getTSFetchResultsResp(const TSStatus &status) { + shared_ptr resp(new TSFetchResultsResp()); + resp->__set_status(status); + return resp; +} + +const string UrlUtils::PORT_SEPARATOR = ":"; +const string UrlUtils::ABB_COLON = "["; + +TEndPoint UrlUtils::parseTEndPointIpv4AndIpv6Url(const string &endPointUrl) { + TEndPoint endPoint; + + if (endPointUrl.empty()) { + return endPoint; + } + + size_t portSeparatorPos = endPointUrl.find_last_of(PORT_SEPARATOR); + + if (portSeparatorPos == string::npos) { + endPoint.__set_ip(endPointUrl); + return endPoint; + } + + string portStr = endPointUrl.substr(portSeparatorPos + 1); + string ip = endPointUrl.substr(0, portSeparatorPos); + + if (ip.find(ABB_COLON) != string::npos) { + if (ip.size() >= 2 && ip.front() == '[' && ip.back() == ']') { + ip = ip.substr(1, ip.size() - 2); + } + } + + try { + int port = stoi(portStr); + endPoint.__set_ip(ip); + endPoint.__set_port(port); + } catch (const exception &) { + endPoint.__set_ip(endPointUrl); + } + + return endPoint; +} diff --git a/iotdb-client/client-cpp/src/rpc/RpcCommon.h b/iotdb-client/client-cpp/src/rpc/RpcCommon.h new file mode 100644 index 0000000000000..0416321d5d1ba --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/RpcCommon.h @@ -0,0 +1,86 @@ +/** + * 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. + */ +#ifndef IOTDB_RPC_COMMON_H +#define IOTDB_RPC_COMMON_H + +#include +#include +#include + +#include "Common.h" + +class TSStatus; +class TSExecuteStatementResp; +class TSFetchResultsResp; +class TEndPoint; + +class RpcUtils { +public: + std::shared_ptr SUCCESS_STATUS; + + RpcUtils(); + + static void verifySuccess(const TSStatus &status); + + static void verifySuccessWithRedirection(const TSStatus &status); + + static void + verifySuccessWithRedirectionForMultiDevices(const TSStatus &status, + std::vector devices); + + static void verifySuccess(const std::vector &statuses); + + static TSStatus getStatus(TSStatusCode::TSStatusCode tsStatusCode); + + static TSStatus getStatus(int code, const std::string &message); + + static std::shared_ptr + getTSExecuteStatementResp(TSStatusCode::TSStatusCode tsStatusCode); + + static std::shared_ptr + getTSExecuteStatementResp(TSStatusCode::TSStatusCode tsStatusCode, + const std::string &message); + + static std::shared_ptr + getTSExecuteStatementResp(const TSStatus &status); + + static std::shared_ptr + getTSFetchResultsResp(TSStatusCode::TSStatusCode tsStatusCode); + + static std::shared_ptr + getTSFetchResultsResp(TSStatusCode::TSStatusCode tsStatusCode, + const std::string &appendMessage); + + static std::shared_ptr + getTSFetchResultsResp(const TSStatus &status); +}; + +class UrlUtils { +private: + static const std::string PORT_SEPARATOR; + static const std::string ABB_COLON; + + UrlUtils() = delete; + ~UrlUtils() = delete; + +public: + static TEndPoint parseTEndPointIpv4AndIpv6Url(const std::string &endPointUrl); +}; + +#endif diff --git a/iotdb-client/client-cpp/src/rpc/SessionConnection.cpp b/iotdb-client/client-cpp/src/rpc/SessionConnection.cpp new file mode 100644 index 0000000000000..bc56da04aac00 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/SessionConnection.cpp @@ -0,0 +1,680 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +#include "SessionConnection.h" +#include "RpcCommon.h" +#include "SessionImpl.h" +#include "common_types.h" +#include +#include +#include +#include +#include + +#include +#include + +#include "SessionDataSet.h" +#include "SessionDataSetFactory.h" + +using namespace apache::thrift; +using namespace apache::thrift::protocol; +using namespace apache::thrift::transport; + +SessionConnection::SessionConnection( + Session::Impl *session_ptr, const TEndPoint &endpoint, + const std::string &zoneId, std::shared_ptr nodeSupplier, + int fetchSize, int maxRetries, int64_t retryInterval, + int64_t connectionTimeout, std::string dialect, std::string db) + : session(session_ptr), zoneId(zoneId), endPoint(endpoint), + availableNodes(std::move(nodeSupplier)), fetchSize(fetchSize), + maxRetryCount(maxRetries), retryIntervalMs(retryInterval), + connectionTimeoutInMs(connectionTimeout), sqlDialect(std::move(dialect)), + database(std::move(db)) { + this->zoneId = zoneId.empty() ? getSystemDefaultZoneId() : zoneId; + endPointList.push_back(endpoint); + init(endPoint, session->useSSL_, session->trustCertFilePath_); +} + +void SessionConnection::close() { + bool needThrowException = false; + std::string errMsg; + session = nullptr; + try { + TSCloseSessionReq req; + req.__set_sessionId(sessionId); + TSStatus tsStatus; + client->closeSession(tsStatus, req); + } catch (const TTransportException &e) { + log_debug(e.what()); + throw IoTDBConnectionException(e.what()); + } catch (const std::exception &e) { + log_debug(e.what()); + errMsg = errMsg + + "Session::close() client->closeSession() error, maybe remote " + "server is down. " + + e.what() + "\n"; + needThrowException = true; + } + + try { + if (transport->isOpen()) { + transport->close(); + } + } catch (const std::exception &e) { + log_debug(e.what()); + errMsg = errMsg + "Session::close() transport->close() error. " + e.what() + + "\n"; + needThrowException = true; + } + + if (needThrowException) { + throw IoTDBException(errMsg); + } +} + +SessionConnection::~SessionConnection() { + try { + close(); + } catch (const std::exception &e) { + log_debug(e.what()); + } +} + +void SessionConnection::init(const TEndPoint &endpoint, bool useSSL, + const std::string &trustCertFilePath) { + if (useSSL) { +#if WITH_SSL + socketFactory_->loadTrustedCertificates(trustCertFilePath.c_str()); + socketFactory_->authenticate(false); + auto sslSocket = socketFactory_->createSocket(endPoint.ip, endPoint.port); + sslSocket->setConnTimeout(connectionTimeoutInMs); + transport = std::make_shared(sslSocket); +#else + throw IoTDBException("SSL/TLS support is not enabled in this build. " + "Please rebuild with -DWITH_SSL=ON flag " + "or use non-SSL connection."); +#endif + } else { + auto socket = std::make_shared(endPoint.ip, endPoint.port); + socket->setConnTimeout(connectionTimeoutInMs); + transport = std::make_shared(socket); + } + if (!transport->isOpen()) { + try { + transport->open(); + } catch (TTransportException &e) { + log_debug(e.what()); + throw IoTDBConnectionException(e.what()); + } + } + if (enableRPCCompression) { + std::shared_ptr protocol(new TCompactProtocol(transport)); + client = std::make_shared(protocol); + } else { + std::shared_ptr protocol(new TBinaryProtocol(transport)); + client = std::make_shared(protocol); + } + + std::map configuration; + configuration["version"] = session->getVersionString(session->version); + configuration["sql_dialect"] = sqlDialect; + if (database != "") { + configuration["db"] = database; + } + TSOpenSessionReq openReq; + openReq.__set_username(session->username_); + openReq.__set_password(session->password_); + openReq.__set_zoneId(zoneId); + openReq.__set_configuration(configuration); + try { + TSOpenSessionResp openResp; + client->openSession(openResp, openReq); + RpcUtils::verifySuccess(openResp.status); + if (session->protocolVersion_ != openResp.serverProtocolVersion) { + if (openResp.serverProtocolVersion == 0) { + // less than 0.10 + throw std::logic_error( + std::string("Protocol not supported, Client version is ") + + std::to_string(session->protocolVersion_) + + ", but Server version is " + + std::to_string(openResp.serverProtocolVersion)); + } + } + + sessionId = openResp.sessionId; + statementId = client->requestStatementId(sessionId); + + if (!zoneId.empty()) { + setTimeZone(zoneId); + } + } catch (const TTransportException &e) { + log_debug(e.what()); + transport->close(); + throw IoTDBConnectionException(e.what()); + } catch (const IoTDBException &e) { + log_debug(e.what()); + transport->close(); + throw; + } catch (const std::exception &e) { + log_debug(e.what()); + transport->close(); + throw; + } +} + +std::unique_ptr +SessionConnection::executeQueryStatement(const std::string &sql, + int64_t timeoutInMs) { + TSExecuteStatementReq req; + req.__set_sessionId(sessionId); + req.__set_statementId(statementId); + req.__set_statement(sql); + req.__set_timeout(timeoutInMs); + req.__set_enableRedirectQuery(true); + + auto result = callWithRetryAndReconnect( + [this, &req]() { + TSExecuteStatementResp resp; + client->executeQueryStatementV2(resp, req); + return resp; + }, + [](const TSExecuteStatementResp &resp) { return resp.status; }); + TSExecuteStatementResp resp = result.getResult(); + if (result.getRetryAttempts() == 0) { + RpcUtils::verifySuccessWithRedirection(resp.status); + } else { + RpcUtils::verifySuccess(resp.status); + } + + return createSessionDataSet( + sql, resp.columns, resp.dataTypeList, resp.columnNameIndexMap, + resp.queryId, statementId, client, sessionId, resp.queryResult, + resp.ignoreTimeStamp, timeoutInMs, resp.moreData, fetchSize, zoneId); +} + +std::unique_ptr +SessionConnection::executeRawDataQuery(const std::vector &paths, + int64_t startTime, int64_t endTime) { + TSRawDataQueryReq req; + req.__set_sessionId(sessionId); + req.__set_statementId(statementId); + req.__set_fetchSize(fetchSize); + req.__set_paths(paths); + req.__set_startTime(startTime); + req.__set_endTime(endTime); + auto result = callWithRetryAndReconnect( + [this, &req]() { + TSExecuteStatementResp resp; + client->executeRawDataQueryV2(resp, req); + return resp; + }, + [](const TSExecuteStatementResp &resp) { return resp.status; }); + TSExecuteStatementResp resp = result.getResult(); + if (result.getRetryAttempts() == 0) { + RpcUtils::verifySuccessWithRedirection(resp.status); + } else { + RpcUtils::verifySuccess(resp.status); + } + return createSessionDataSet("", resp.columns, resp.dataTypeList, + resp.columnNameIndexMap, resp.queryId, + statementId, client, sessionId, resp.queryResult, + resp.ignoreTimeStamp, connectionTimeoutInMs, + resp.moreData, fetchSize, zoneId); +} + +std::unique_ptr +SessionConnection::executeLastDataQuery(const std::vector &paths, + int64_t lastTime) { + TSLastDataQueryReq req; + req.__set_sessionId(sessionId); + req.__set_statementId(statementId); + req.__set_fetchSize(fetchSize); + req.__set_paths(paths); + req.__set_time(lastTime); + + auto result = callWithRetryAndReconnect( + [this, &req]() { + TSExecuteStatementResp resp; + client->executeLastDataQuery(resp, req); + return resp; + }, + [](const TSExecuteStatementResp &resp) { return resp.status; }); + TSExecuteStatementResp resp = result.getResult(); + if (result.getRetryAttempts() == 0) { + RpcUtils::verifySuccessWithRedirection(resp.status); + } else { + RpcUtils::verifySuccess(resp.status); + } + return createSessionDataSet("", resp.columns, resp.dataTypeList, + resp.columnNameIndexMap, resp.queryId, + statementId, client, sessionId, resp.queryResult, + resp.ignoreTimeStamp, connectionTimeoutInMs, + resp.moreData, fetchSize, zoneId); +} + +void SessionConnection::executeNonQueryStatement(const std::string &sql) { + TSExecuteStatementReq req; + req.__set_sessionId(sessionId); + req.__set_statementId(statementId); + req.__set_statement(sql); + req.__set_timeout( + 0); // 0 means no timeout. This value keep consistent to JAVA SDK. + TSExecuteStatementResp resp; + try { + client->executeUpdateStatementV2(resp, req); + RpcUtils::verifySuccess(resp.status); + } catch (const TTransportException &e) { + log_debug(e.what()); + throw IoTDBConnectionException(e.what()); + } catch (const IoTDBException &e) { + log_debug(e.what()); + throw; + } catch (const std::exception &e) { + throw IoTDBException(e.what()); + } +} + +const TEndPoint &SessionConnection::getEndPoint() { return endPoint; } + +void SessionConnection::setTimeZone(const std::string &newZoneId) { + TSSetTimeZoneReq req; + req.__set_sessionId(sessionId); + req.__set_timeZone(newZoneId); + + try { + TSStatus tsStatus; + client->setTimeZone(tsStatus, req); + zoneId = newZoneId; + } catch (const TException &e) { + throw IoTDBConnectionException(e.what()); + } +} + +std::string SessionConnection::getSystemDefaultZoneId() { + time_t ts = 0; + struct tm tmv {}; +#if defined(_WIN64) || defined(WIN32) || defined(_WIN32) + localtime_s(&tmv, &ts); +#else + localtime_r(&ts, &tmv); +#endif + char zoneStr[32]; + strftime(zoneStr, sizeof(zoneStr), "%z", &tmv); + return zoneStr; +} + +bool SessionConnection::reconnect() { + bool reconnect = false; + for (int i = 1; i <= 3; i++) { + if (transport != nullptr) { + transport->close(); + endPointList = std::move(availableNodes->getEndPointList()); + int currHostIndex = rand() % endPointList.size(); + int tryHostNum = 0; + for (int j = currHostIndex; j < endPointList.size(); j++) { + if (tryHostNum == endPointList.size()) { + break; + } + this->endPoint = endPointList[j]; + if (j == endPointList.size() - 1) { + j = -1; + } + tryHostNum++; + try { + init(this->endPoint, this->session->useSSL_, + this->session->trustCertFilePath_); + reconnect = true; + } catch (const IoTDBConnectionException &e) { + log_warn( + "The current node may have been down, connection exception: %s", + e.what()); + continue; + } catch (std::exception &e) { + log_warn("login in failed, because %s", e.what()); + } + break; + } + } + if (reconnect) { + session->removeBrokenSessionConnection(shared_from_this()); + session->defaultEndPoint_ = this->endPoint; + session->defaultSessionConnection_ = shared_from_this(); + session->endPointToSessionConnection.insert( + std::make_pair(this->endPoint, shared_from_this())); + } + } + return reconnect; +} + +void SessionConnection::insertStringRecord( + const TSInsertStringRecordReq &request) { + auto rpc = [this, request]() { + return this->insertStringRecordInternal(request); + }; + callWithRetryAndVerifyWithRedirection(rpc); +} + +void SessionConnection::insertRecord(const TSInsertRecordReq &request) { + auto rpc = [this, request]() { return this->insertRecordInternal(request); }; + callWithRetryAndVerifyWithRedirection(rpc); +} + +void SessionConnection::insertStringRecords( + const TSInsertStringRecordsReq &request) { + auto rpc = [this, request]() { + return this->insertStringRecordsInternal(request); + }; + callWithRetryAndVerifyWithRedirection(rpc); +} + +void SessionConnection::insertRecords(const TSInsertRecordsReq &request) { + auto rpc = [this, request]() { return this->insertRecordsInternal(request); }; + callWithRetryAndVerifyWithRedirection(rpc); +} + +void SessionConnection::insertRecordsOfOneDevice( + TSInsertRecordsOfOneDeviceReq request) { + auto rpc = [this, request]() { + return this->insertRecordsOfOneDeviceInternal(request); + }; + callWithRetryAndVerifyWithRedirection(rpc); +} + +void SessionConnection::insertStringRecordsOfOneDevice( + TSInsertStringRecordsOfOneDeviceReq request) { + auto rpc = [this, request]() { + return this->insertStringRecordsOfOneDeviceInternal(request); + }; + callWithRetryAndVerifyWithRedirection(rpc); +} + +void SessionConnection::insertTablet(TSInsertTabletReq request) { + auto rpc = [this, request]() { return this->insertTabletInternal(request); }; + callWithRetryAndVerifyWithRedirection(rpc); +} + +void SessionConnection::insertTablets(TSInsertTabletsReq request) { + auto rpc = [this, request]() { return this->insertTabletsInternal(request); }; + callWithRetryAndVerifyWithRedirection(rpc); +} + +void SessionConnection::testInsertStringRecord( + TSInsertStringRecordReq &request) { + auto rpc = [this, &request]() { + request.sessionId = sessionId; + TSStatus ret; + client->testInsertStringRecord(ret, request); + return ret; + }; + auto status = callWithRetryAndReconnect(rpc).getResult(); + RpcUtils::verifySuccess(status); +} + +void SessionConnection::testInsertTablet(TSInsertTabletReq &request) { + auto rpc = [this, &request]() { + request.sessionId = sessionId; + TSStatus ret; + client->testInsertTablet(ret, request); + return ret; + }; + auto status = callWithRetryAndReconnect(rpc).getResult(); + RpcUtils::verifySuccess(status); +} + +void SessionConnection::testInsertRecords(TSInsertRecordsReq &request) { + auto rpc = [this, &request]() { + request.sessionId = sessionId; + TSStatus ret; + client->testInsertRecords(ret, request); + return ret; + }; + auto status = callWithRetryAndReconnect(rpc).getResult(); + RpcUtils::verifySuccess(status); +} + +void SessionConnection::deleteTimeseries( + const std::vector &paths) { + auto rpc = [this, &paths]() { + TSStatus ret; + client->deleteTimeseries(ret, sessionId, paths); + return ret; + }; + callWithRetryAndVerify(rpc); +} + +void SessionConnection::deleteData(const TSDeleteDataReq &request) { + auto rpc = [this, request]() { return this->deleteDataInternal(request); }; + callWithRetryAndVerify(rpc); +} + +void SessionConnection::setStorageGroup(const std::string &storageGroupId) { + auto rpc = [this, &storageGroupId]() { + TSStatus ret; + client->setStorageGroup(ret, sessionId, storageGroupId); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +void SessionConnection::deleteStorageGroups( + const std::vector &storageGroups) { + auto rpc = [this, &storageGroups]() { + TSStatus ret; + client->deleteStorageGroups(ret, sessionId, storageGroups); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +void SessionConnection::createTimeseries(TSCreateTimeseriesReq &req) { + auto rpc = [this, &req]() { + TSStatus ret; + req.sessionId = sessionId; + client->createTimeseries(ret, req); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +void SessionConnection::createMultiTimeseries(TSCreateMultiTimeseriesReq &req) { + auto rpc = [this, &req]() { + TSStatus ret; + req.sessionId = sessionId; + client->createMultiTimeseries(ret, req); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +void SessionConnection::createAlignedTimeseries( + TSCreateAlignedTimeseriesReq &req) { + auto rpc = [this, &req]() { + TSStatus ret; + req.sessionId = sessionId; + client->createAlignedTimeseries(ret, req); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +TSGetTimeZoneResp SessionConnection::getTimeZone() { + auto rpc = [this]() { + TSGetTimeZoneResp resp; + client->getTimeZone(resp, sessionId); + zoneId = resp.timeZone; + return resp; + }; + auto ret = callWithRetryAndReconnect( + rpc, [](const TSGetTimeZoneResp &resp) { return resp.status; }); + RpcUtils::verifySuccess(ret.getResult().status); + return ret.result; +} + +void SessionConnection::setTimeZone(TSSetTimeZoneReq &req) { + auto rpc = [this, &req]() { + TSStatus ret; + req.sessionId = sessionId; + client->setTimeZone(ret, req); + zoneId = req.timeZone; + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +void SessionConnection::createSchemaTemplate(TSCreateSchemaTemplateReq req) { + auto rpc = [this, &req]() { + TSStatus ret; + req.sessionId = sessionId; + client->createSchemaTemplate(ret, req); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +void SessionConnection::setSchemaTemplate(TSSetSchemaTemplateReq req) { + auto rpc = [this, &req]() { + TSStatus ret; + req.sessionId = sessionId; + client->setSchemaTemplate(ret, req); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +void SessionConnection::unsetSchemaTemplate(TSUnsetSchemaTemplateReq req) { + auto rpc = [this, &req]() { + TSStatus ret; + req.sessionId = sessionId; + client->unsetSchemaTemplate(ret, req); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +void SessionConnection::appendSchemaTemplate(TSAppendSchemaTemplateReq req) { + auto rpc = [this, &req]() { + TSStatus ret; + req.sessionId = sessionId; + client->appendSchemaTemplate(ret, req); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +void SessionConnection::pruneSchemaTemplate(TSPruneSchemaTemplateReq req) { + auto rpc = [this, &req]() { + TSStatus ret; + req.sessionId = sessionId; + client->pruneSchemaTemplate(ret, req); + return ret; + }; + auto ret = callWithRetryAndReconnect(rpc); + RpcUtils::verifySuccess(ret.getResult()); +} + +TSQueryTemplateResp +SessionConnection::querySchemaTemplate(TSQueryTemplateReq req) { + auto rpc = [this, &req]() { + TSQueryTemplateResp ret; + req.sessionId = sessionId; + client->querySchemaTemplate(ret, req); + return ret; + }; + auto ret = callWithRetryAndReconnect( + rpc, [](const TSQueryTemplateResp &resp) { return resp.status; }); + RpcUtils::verifySuccess(ret.getResult().status); + return ret.getResult(); +} + +TSStatus +SessionConnection::insertStringRecordInternal(TSInsertStringRecordReq request) { + request.sessionId = sessionId; + TSStatus ret; + client->insertStringRecord(ret, request); + return ret; +} + +TSStatus SessionConnection::insertRecordInternal(TSInsertRecordReq request) { + request.sessionId = sessionId; + TSStatus ret; + client->insertRecord(ret, request); + return ret; +} + +TSStatus SessionConnection::insertStringRecordsInternal( + TSInsertStringRecordsReq request) { + request.sessionId = sessionId; + TSStatus ret; + client->insertStringRecords(ret, request); + return ret; +} + +TSStatus SessionConnection::insertRecordsInternal(TSInsertRecordsReq request) { + request.sessionId = sessionId; + TSStatus ret; + client->insertRecords(ret, request); + return ret; +} + +TSStatus SessionConnection::insertRecordsOfOneDeviceInternal( + TSInsertRecordsOfOneDeviceReq request) { + request.sessionId = sessionId; + TSStatus ret; + client->insertRecordsOfOneDevice(ret, request); + return ret; +} + +TSStatus SessionConnection::insertStringRecordsOfOneDeviceInternal( + TSInsertStringRecordsOfOneDeviceReq request) { + request.sessionId = sessionId; + TSStatus ret; + client->insertStringRecordsOfOneDevice(ret, request); + return ret; +} + +TSStatus SessionConnection::insertTabletInternal(TSInsertTabletReq request) { + request.sessionId = sessionId; + TSStatus ret; + client->insertTablet(ret, request); + return ret; +} + +TSStatus SessionConnection::insertTabletsInternal(TSInsertTabletsReq request) { + request.sessionId = sessionId; + TSStatus ret; + client->insertTablets(ret, request); + return ret; +} + +TSStatus SessionConnection::deleteDataInternal(TSDeleteDataReq request) { + request.sessionId = sessionId; + TSStatus ret; + client->deleteData(ret, request); + return ret; +} diff --git a/iotdb-client/client-cpp/src/rpc/SessionConnection.h b/iotdb-client/client-cpp/src/rpc/SessionConnection.h new file mode 100644 index 0000000000000..08525ae799272 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/SessionConnection.h @@ -0,0 +1,364 @@ +/** + * 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. + */ +#ifndef IOTDB_SESSIONCONNECTION_H +#define IOTDB_SESSIONCONNECTION_H + +#include +#include +#include +#include +#if WITH_SSL +#include +#endif + +#include "Common.h" +#include "IClientRPCService.h" +#include "NodesSupplier.h" +#include "RpcCommon.h" +#include "Session.h" +#include "SessionConfig.h" +#include "common_types.h" + +class SessionDataSet; + +class SessionConnection + : public std::enable_shared_from_this { +public: + SessionConnection( + Session::Impl *session_ptr, const TEndPoint &endpoint, + const std::string &zoneId, std::shared_ptr nodeSupplier, + int fetchSize = iotdb::session::DEFAULT_FETCH_SIZE, + int maxRetries = iotdb::session::DEFAULT_MAX_RETRIES, + int64_t retryInterval = iotdb::session::DEFAULT_RETRY_DELAY_MS, + int64_t connectionTimeoutMs = iotdb::session::DEFAULT_CONNECT_TIMEOUT_MS, + std::string dialect = "tree", std::string db = ""); + + ~SessionConnection(); + + void setTimeZone(const std::string &newZoneId); + + const TEndPoint &getEndPoint(); + + void init(const TEndPoint &endpoint, bool useSSL, + const std::string &trustCertFilePath); + + void insertStringRecord(const TSInsertStringRecordReq &request); + + void insertRecord(const TSInsertRecordReq &request); + + void insertStringRecords(const TSInsertStringRecordsReq &request); + + void insertRecords(const TSInsertRecordsReq &request); + + void insertRecordsOfOneDevice(TSInsertRecordsOfOneDeviceReq request); + + void + insertStringRecordsOfOneDevice(TSInsertStringRecordsOfOneDeviceReq request); + + void insertTablet(TSInsertTabletReq request); + + void insertTablets(TSInsertTabletsReq request); + + void testInsertStringRecord(TSInsertStringRecordReq &request); + + void testInsertTablet(TSInsertTabletReq &request); + + void testInsertRecords(TSInsertRecordsReq &request); + + void deleteTimeseries(const std::vector &paths); + + void deleteData(const TSDeleteDataReq &request); + + void setStorageGroup(const std::string &storageGroupId); + + void deleteStorageGroups(const std::vector &storageGroups); + + void createTimeseries(TSCreateTimeseriesReq &req); + + void createMultiTimeseries(TSCreateMultiTimeseriesReq &req); + + void createAlignedTimeseries(TSCreateAlignedTimeseriesReq &req); + + TSGetTimeZoneResp getTimeZone(); + + void setTimeZone(TSSetTimeZoneReq &req); + + void createSchemaTemplate(TSCreateSchemaTemplateReq req); + + void setSchemaTemplate(TSSetSchemaTemplateReq req); + + void unsetSchemaTemplate(TSUnsetSchemaTemplateReq req); + + void appendSchemaTemplate(TSAppendSchemaTemplateReq req); + + void pruneSchemaTemplate(TSPruneSchemaTemplateReq req); + + TSQueryTemplateResp querySchemaTemplate(TSQueryTemplateReq req); + + std::unique_ptr + executeRawDataQuery(const std::vector &paths, int64_t startTime, + int64_t endTime); + + std::unique_ptr + executeLastDataQuery(const std::vector &paths, int64_t lastTime); + + void executeNonQueryStatement(const std::string &sql); + + std::unique_ptr + executeQueryStatement(const std::string &sql, int64_t timeoutInMs = -1); + + std::shared_ptr getSessionClient() { return client; } + + friend class Session; + +private: + void close(); + std::string getSystemDefaultZoneId(); + bool reconnect(); + + template struct RetryResult { + T result; + std::exception_ptr exception; + int retryAttempts; + + RetryResult(T r, std::exception_ptr e, int a) + : result(r), exception(e), retryAttempts(a) {} + + int getRetryAttempts() const { return retryAttempts; } + T getResult() const { return result; } + std::exception_ptr getException() const { return exception; } + }; + + template + void callWithRetryAndVerifyWithRedirection(std::function rpc); + + template + void callWithRetryAndVerifyWithRedirectionForMultipleDevices( + std::function rpc, const std::vector &deviceIds); + + template + RetryResult callWithRetryAndVerify(std::function rpc); + + template RetryResult callWithRetry(std::function rpc); + + template + RetryResult callWithRetryAndReconnect(RpcFunc rpc); + + template + RetryResult callWithRetryAndReconnect(RpcFunc rpc, + StatusGetter statusGetter); + + template + RetryResult callWithRetryAndReconnect(RpcFunc rpc, ShouldRetry shouldRetry, + ForceReconnect forceReconnect); + + TSStatus insertStringRecordInternal(TSInsertStringRecordReq request); + + TSStatus insertRecordInternal(TSInsertRecordReq request); + + TSStatus insertStringRecordsInternal(TSInsertStringRecordsReq request); + + TSStatus insertRecordsInternal(TSInsertRecordsReq request); + + TSStatus + insertRecordsOfOneDeviceInternal(TSInsertRecordsOfOneDeviceReq request); + + TSStatus insertStringRecordsOfOneDeviceInternal( + TSInsertStringRecordsOfOneDeviceReq request); + + TSStatus insertTabletInternal(TSInsertTabletReq request); + + TSStatus insertTabletsInternal(TSInsertTabletsReq request); + + TSStatus deleteDataInternal(TSDeleteDataReq request); +#if WITH_SSL + std::shared_ptr socketFactory_ = + std::make_shared(); +#endif + std::shared_ptr transport; + std::shared_ptr client; + Session::Impl *session; + int64_t sessionId; + int64_t statementId; + int64_t connectionTimeoutInMs; + bool enableRPCCompression = false; + std::string zoneId; + TEndPoint endPoint; + std::vector endPointList; + std::shared_ptr availableNodes; + int fetchSize; + int maxRetryCount; + int64_t retryIntervalMs; + std::string sqlDialect; + std::string database; + int timeFactor = 1000; +}; + +template +SessionConnection::RetryResult +SessionConnection::callWithRetry(std::function rpc) { + std::exception_ptr lastException = nullptr; + TSStatus status; + int i; + for (i = 0; i <= maxRetryCount; i++) { + if (i > 0) { + lastException = nullptr; + status = TSStatus(); + try { + std::this_thread::sleep_for(std::chrono::milliseconds(retryIntervalMs)); + } catch (const std::exception &e) { + break; + } + if (!reconnect()) { + continue; + } + } + + try { + status = rpc(); + if (status.__isset.needRetry && status.needRetry) { + continue; + } + break; + } catch (...) { + lastException = std::current_exception(); + } + } + return {status, lastException, i}; +} + +template +void SessionConnection::callWithRetryAndVerifyWithRedirection( + std::function rpc) { + auto result = callWithRetry(rpc); + + auto status = result.getResult(); + if (result.getRetryAttempts() == 0) { + RpcUtils::verifySuccessWithRedirection(status); + } else { + RpcUtils::verifySuccess(status); + } + + if (result.getException()) { + throw IoTDBConnectionException( + extractExceptionMessage(result.getException())); + } +} + +template +void SessionConnection::callWithRetryAndVerifyWithRedirectionForMultipleDevices( + std::function rpc, const std::vector &deviceIds) { + auto result = callWithRetry(rpc); + auto status = result.getResult(); + if (result.getRetryAttempts() == 0) { + RpcUtils::verifySuccessWithRedirectionForMultiDevices(status, deviceIds); + } else { + RpcUtils::verifySuccess(status); + } + if (result.getException()) { + throw IoTDBConnectionException( + extractExceptionMessage(result.getException())); + } + result.exception = nullptr; +} + +template +SessionConnection::RetryResult +SessionConnection::callWithRetryAndVerify(std::function rpc) { + auto result = callWithRetry(rpc); + RpcUtils::verifySuccess(result.getResult()); + if (result.getException()) { + throw IoTDBConnectionException( + extractExceptionMessage(result.getException())); + } + return result; +} + +template +SessionConnection::RetryResult +SessionConnection::callWithRetryAndReconnect(RpcFunc rpc) { + return callWithRetryAndReconnect( + rpc, + [](const TSStatus &status) { + return status.__isset.needRetry && status.needRetry; + }, + [](const TSStatus &status) { + return status.code == TSStatusCode::PLAN_FAILED_NETWORK_PARTITION; + }); +} + +template +SessionConnection::RetryResult +SessionConnection::callWithRetryAndReconnect(RpcFunc rpc, + StatusGetter statusGetter) { + auto shouldRetry = [&statusGetter](const T &t) { + auto status = statusGetter(t); + return status.__isset.needRetry && status.needRetry; + }; + auto forceReconnect = [&statusGetter](const T &t) { + auto status = statusGetter(t); + return status.code == TSStatusCode::PLAN_FAILED_NETWORK_PARTITION; + ; + }; + return callWithRetryAndReconnect(rpc, shouldRetry, forceReconnect); +} + +template +SessionConnection::RetryResult SessionConnection::callWithRetryAndReconnect( + RpcFunc rpc, ShouldRetry shouldRetry, ForceReconnect forceReconnect) { + std::exception_ptr lastException = nullptr; + T result; + int retryAttempt; + for (retryAttempt = 0; retryAttempt <= maxRetryCount; retryAttempt++) { + try { + result = rpc(); + lastException = nullptr; + } catch (...) { + result = T(); + lastException = std::current_exception(); + } + + if (!shouldRetry(result)) { + return {result, lastException, retryAttempt}; + } + + if (lastException != nullptr || + std::find(availableNodes->getEndPointList().begin(), + availableNodes->getEndPointList().end(), + this->endPoint) == availableNodes->getEndPointList().end() || + forceReconnect(result)) { + reconnect(); + } + + try { + std::this_thread::sleep_for(std::chrono::milliseconds(retryIntervalMs)); + } catch (const std::exception &e) { + log_debug("Thread was interrupted during retry " + + std::to_string(retryAttempt) + " with wait time " + + std::to_string(retryIntervalMs) + " ms. Exiting retry loop."); + break; + } + } + + return {result, lastException, retryAttempt}; +} + +#endif diff --git a/iotdb-client/client-cpp/src/rpc/SessionDataSetFactory.h b/iotdb-client/client-cpp/src/rpc/SessionDataSetFactory.h new file mode 100644 index 0000000000000..3b7429d3e3d0e --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/SessionDataSetFactory.h @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +#ifndef IOTDB_SESSION_DATA_SET_FACTORY_H +#define IOTDB_SESSION_DATA_SET_FACTORY_H + +#include +#include +#include +#include +#include + +#include "IClientRPCService.h" + +class SessionDataSet; + +std::unique_ptr createSessionDataSet( + const std::string &sql, const std::vector &columnNameList, + const std::vector &columnTypeList, + const std::map &columnNameIndex, int64_t queryId, + int64_t statementId, std::shared_ptr client, + int64_t sessionId, const std::vector &queryResult, + bool ignoreTimestamp, int64_t timeout, bool moreData, int32_t fetchSize, + const std::string &zoneId); + +#endif diff --git a/iotdb-client/client-cpp/src/rpc/SessionImpl.h b/iotdb-client/client-cpp/src/rpc/SessionImpl.h new file mode 100644 index 0000000000000..6ccba7dd84056 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/SessionImpl.h @@ -0,0 +1,230 @@ +/** + * 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. + */ +#ifndef IOTDB_SESSION_IMPL_H +#define IOTDB_SESSION_IMPL_H + +#include +#include +#include +#include +#include +#include + +#include "AbstractSessionBuilder.h" +#include "Common.h" +#include "Endpoint.h" +#include "NodesSupplier.h" +#include "Session.h" +#include "SessionConnection.h" +#include "ThriftConvert.h" +#include "client_types.h" +#include "common_types.h" + +class Session::Impl { +public: + std::string host_; + int rpcPort_ = 6667; + bool useSSL_ = false; + std::string trustCertFilePath_; + std::vector nodeUrls_; + std::string username_ = "root"; + std::string password_ = "root"; + const TSProtocolVersion::type protocolVersion_ = + TSProtocolVersion::IOTDB_SERVICE_PROTOCOL_V3; + bool isClosed_ = true; + std::string zoneId_; + int fetchSize_ = iotdb::session::DEFAULT_FETCH_SIZE; + static const int DEFAULT_TIMEOUT_MS = 0; + int connectTimeoutMs_ = iotdb::session::DEFAULT_CONNECT_TIMEOUT_MS; + Version::Version version = Version::V_1_0; + std::string sqlDialect_ = "tree"; + std::string database_; + bool enableAutoFetch_ = true; + bool enableRedirection_ = true; + std::shared_ptr nodesSupplier_; + std::shared_ptr defaultSessionConnection_; + + TEndPoint defaultEndPoint_; + + struct TEndPointHash { + size_t operator()(const TEndPoint &endpoint) const { + return std::hash()(endpoint.ip) ^ + std::hash()(endpoint.port); + } + }; + + struct TEndPointEqual { + bool operator()(const TEndPoint &lhs, const TEndPoint &rhs) const { + return lhs.ip == rhs.ip && lhs.port == rhs.port; + } + }; + + using EndPointSessionMap = + std::unordered_map, + TEndPointHash, TEndPointEqual>; + EndPointSessionMap endPointToSessionConnection; + std::unordered_map deviceIdToEndpoint; + + void removeBrokenSessionConnection( + std::shared_ptr sessionConnection); + + static bool checkSorted(const Tablet &tablet); + static bool checkSorted(const std::vector ×); + static void sortTablet(Tablet &tablet); + static void sortIndexByTimestamp(int *index, std::vector ×tamps, + int length); + + void appendValues(std::string &buffer, const char *value, int size); + void putValuesIntoBuffer(const std::vector &types, + const std::vector &values, std::string &buf); + int8_t getDataTypeNumber(TSDataType::TSDataType type); + + struct TsCompare { + std::vector ×tamps; + explicit TsCompare(std::vector &inTimestamps) + : timestamps(inTimestamps) {} + bool operator()(int i, int j) { return timestamps[i] < timestamps[j]; } + }; + + std::string getVersionString(Version::Version version); + + void initZoneId(); + void initNodesSupplier( + const std::vector &nodeUrls = std::vector()); + void initDefaultSessionConnection(); + + template + void insertByGroup( + std::unordered_map, T> &insertGroup, + InsertConsumer insertConsumer); + + template + void insertOnce( + std::unordered_map, T> &insertGroup, + InsertConsumer insertConsumer); + + void insertStringRecordsWithLeaderCache( + std::vector deviceIds, std::vector times, + std::vector> measurementsList, + std::vector> valuesList, bool isAligned); + + void insertRecordsWithLeaderCache( + std::vector deviceIds, std::vector times, + std::vector> measurementsList, + const std::vector> &typesList, + std::vector> valuesList, bool isAligned); + + void insertTabletsWithLeaderCache( + std::unordered_map &tablets, bool sorted, + bool isAligned); + + std::shared_ptr getQuerySessionConnection(); + std::shared_ptr getSessionConnection(std::string deviceId); + + void handleQueryRedirection(TEndPoint endPoint); + void handleRedirection(const std::string &deviceId, TEndPoint endPoint); + + static void buildInsertTabletReq(TSInsertTabletReq &request, Tablet &tablet, + bool sorted); + void insertTablet(TSInsertTabletReq request); +}; + +template +void Session::Impl::insertByGroup( + std::unordered_map, T> &insertGroup, + InsertConsumer insertConsumer) { + std::vector> futures; + + for (auto &entry : insertGroup) { + auto connection = entry.first; + auto &req = entry.second; + futures.emplace_back(std::async(std::launch::async, [=, &req]() mutable { + try { + insertConsumer(connection, req); + } catch (const RedirectException &e) { + for (const auto &deviceEndPoint : e.deviceEndPointMap) { + handleRedirection(deviceEndPoint.first, + endpointToThrift(deviceEndPoint.second)); + } + } catch (const IoTDBConnectionException &) { + if (endPointToSessionConnection.size() > 1) { + removeBrokenSessionConnection(connection); + try { + insertConsumer(defaultSessionConnection_, req); + } catch (const RedirectException &) { + } + } else { + throw; + } + } catch (const std::exception &e) { + log_debug(e.what()); + throw IoTDBException(e.what()); + } + })); + } + + std::string errorMessages; + for (auto &f : futures) { + try { + f.get(); + } catch (const IoTDBConnectionException &) { + throw; + } catch (const std::exception &e) { + if (!errorMessages.empty()) { + errorMessages += ";"; + } + errorMessages += e.what(); + } + } + + if (!errorMessages.empty()) { + throw StatementExecutionException(errorMessages); + } +} + +template +void Session::Impl::insertOnce( + std::unordered_map, T> &insertGroup, + InsertConsumer insertConsumer) { + auto connection = insertGroup.begin()->first; + auto req = insertGroup.begin()->second; + try { + insertConsumer(connection, req); + } catch (const RedirectException &e) { + for (const auto &deviceEndPoint : e.deviceEndPointMap) { + handleRedirection(deviceEndPoint.first, + endpointToThrift(deviceEndPoint.second)); + } + } catch (const IoTDBConnectionException &) { + if (endPointToSessionConnection.size() > 1) { + removeBrokenSessionConnection(connection); + try { + insertConsumer(defaultSessionConnection_, req); + } catch (const RedirectException &) { + } + } else { + throw; + } + } catch (const std::exception &e) { + log_debug(e.what()); + throw IoTDBException(e.what()); + } +} + +#endif diff --git a/iotdb-client/client-cpp/src/rpc/ThriftConnection.cpp b/iotdb-client/client-cpp/src/rpc/ThriftConnection.cpp new file mode 100644 index 0000000000000..832d664cd94e2 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/ThriftConnection.cpp @@ -0,0 +1,187 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +#include "ThriftConnection.h" +#include +#include +#include +#include +#include +#include +#include + +#include "RpcCommon.h" +#include "SessionDataSet.h" +#include "SessionDataSetFactory.h" + +using namespace apache::thrift; +using namespace apache::thrift::protocol; +using namespace apache::thrift::transport; + +const int ThriftConnection::THRIFT_DEFAULT_BUFFER_SIZE = 4096; +const int ThriftConnection::THRIFT_MAX_FRAME_SIZE = 1048576; +const int ThriftConnection::CONNECTION_TIMEOUT_IN_MS = 1000; + +ThriftConnection::ThriftConnection(const TEndPoint &endPoint, + int thriftDefaultBufferSize, + int thriftMaxFrameSize, + int connectionTimeoutInMs, int fetchSize) + : endPoint_(endPoint), thriftDefaultBufferSize_(thriftDefaultBufferSize), + thriftMaxFrameSize_(thriftMaxFrameSize), + connectionTimeoutInMs_(connectionTimeoutInMs), fetchSize_(fetchSize) {} + +ThriftConnection::~ThriftConnection() = default; + +void ThriftConnection::initZoneId() { + if (!zoneId_.empty()) { + return; + } + + time_t ts = 0; + struct tm tmv {}; +#if defined(_WIN64) || defined(WIN32) || defined(_WIN32) + localtime_s(&tmv, &ts); +#else + localtime_r(&ts, &tmv); +#endif + + char zoneStr[32]; + strftime(zoneStr, sizeof(zoneStr), "%z", &tmv); + zoneId_ = zoneStr; +} + +void ThriftConnection::init(const std::string &username, + const std::string &password, + bool enableRPCCompression, bool useSSL, + const std::string &trustCertFilePath, + const std::string &zoneId, + const std::string &version) { + if (useSSL) { +#if WITH_SSL + socketFactory_->loadTrustedCertificates(trustCertFilePath.c_str()); + socketFactory_->authenticate(false); + auto sslSocket = socketFactory_->createSocket(endPoint_.ip, endPoint_.port); + sslSocket->setConnTimeout(connectionTimeoutInMs_); + transport_ = std::make_shared(sslSocket); +#else + throw IoTDBException("SSL/TLS support is not enabled in this build. " + "Please rebuild with -DWITH_SSL=ON flag " + "or use non-SSL connection."); +#endif + } else { + auto socket = std::make_shared(endPoint_.ip, endPoint_.port); + socket->setConnTimeout(connectionTimeoutInMs_); + transport_ = std::make_shared(socket); + } + if (!transport_->isOpen()) { + try { + transport_->open(); + } catch (TTransportException &e) { + throw IoTDBConnectionException(e.what()); + } + } + if (zoneId.empty()) { + initZoneId(); + } else { + this->zoneId_ = zoneId; + } + + if (enableRPCCompression) { + std::shared_ptr protocol( + new TCompactProtocol(transport_)); + client_ = std::make_shared(protocol); + } else { + std::shared_ptr protocol(new TBinaryProtocol(transport_)); + client_ = std::make_shared(protocol); + } + + std::map configuration; + configuration["version"] = version; + TSOpenSessionReq openReq; + openReq.__set_username(username); + openReq.__set_password(password); + openReq.__set_zoneId(this->zoneId_); + openReq.__set_configuration(configuration); + try { + TSOpenSessionResp openResp; + client_->openSession(openResp, openReq); + RpcUtils::verifySuccess(openResp.status); + sessionId_ = openResp.sessionId; + statementId_ = client_->requestStatementId(sessionId_); + } catch (const TTransportException &e) { + transport_->close(); + throw IoTDBConnectionException(e.what()); + } catch (const IoTDBException &e) { + transport_->close(); + throw IoTDBException(e.what()); + } catch (const std::exception &e) { + transport_->close(); + throw IoTDBException(e.what()); + } +} + +std::unique_ptr +ThriftConnection::executeQueryStatement(const std::string &sql, + int64_t timeoutInMs) { + TSExecuteStatementReq req; + req.__set_sessionId(sessionId_); + req.__set_statementId(statementId_); + req.__set_statement(sql); + req.__set_timeout(timeoutInMs); + TSExecuteStatementResp resp; + try { + client_->executeQueryStatementV2(resp, req); + RpcUtils::verifySuccess(resp.status); + } catch (const TTransportException &e) { + throw IoTDBConnectionException(e.what()); + } catch (const IoTDBException &e) { + throw IoTDBConnectionException(e.what()); + } catch (const std::exception &e) { + throw IoTDBException(e.what()); + } + std::shared_ptr queryDataSet( + new TSQueryDataSet(resp.queryDataSet)); + return createSessionDataSet( + "", resp.columns, resp.dataTypeList, resp.columnNameIndexMap, + resp.queryId, statementId_, client_, sessionId_, resp.queryResult, + resp.ignoreTimeStamp, connectionTimeoutInMs_, resp.moreData, fetchSize_, + zoneId_); +} + +void ThriftConnection::close() { + try { + if (client_) { + TSCloseSessionReq req; + req.__set_sessionId(sessionId_); + TSStatus tsStatus; + client_->closeSession(tsStatus, req); + } + } catch (const TTransportException &e) { + throw IoTDBConnectionException(e.what()); + } catch (const std::exception &e) { + throw IoTDBConnectionException(e.what()); + } + + try { + if (transport_->isOpen()) { + transport_->close(); + } + } catch (const std::exception &e) { + throw IoTDBConnectionException(e.what()); + } +} diff --git a/iotdb-client/client-cpp/src/rpc/ThriftConnection.h b/iotdb-client/client-cpp/src/rpc/ThriftConnection.h new file mode 100644 index 0000000000000..d79a63fb1be01 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/ThriftConnection.h @@ -0,0 +1,79 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +#ifndef IOTDB_THRIFTCONNECTION_H +#define IOTDB_THRIFTCONNECTION_H + +#include +#if WITH_SSL +#include +#endif +#include "IClientRPCService.h" +#include "SessionConfig.h" + +class SessionDataSet; + +class ThriftConnection { +public: + static const int THRIFT_DEFAULT_BUFFER_SIZE; + static const int THRIFT_MAX_FRAME_SIZE; + static const int CONNECTION_TIMEOUT_IN_MS; + + explicit ThriftConnection( + const TEndPoint &endPoint, + int thriftDefaultBufferSize = THRIFT_DEFAULT_BUFFER_SIZE, + int thriftMaxFrameSize = THRIFT_MAX_FRAME_SIZE, + int connectionTimeoutInMs = CONNECTION_TIMEOUT_IN_MS, + int fetchSize = iotdb::session::DEFAULT_FETCH_SIZE); + + ~ThriftConnection(); + + void init(const std::string &username, const std::string &password, + bool enableRPCCompression = false, bool useSSL = false, + const std::string &trustCertFilePath = "", + const std::string &zoneId = std::string(), + const std::string &version = "V_1_0"); + + std::unique_ptr + executeQueryStatement(const std::string &sql, int64_t timeoutInMs = -1); + + void close(); + +private: + TEndPoint endPoint_; + + int thriftDefaultBufferSize_; + int thriftMaxFrameSize_; + int connectionTimeoutInMs_; + int fetchSize_; + +#if WITH_SSL + std::shared_ptr socketFactory_ = + std::make_shared(); +#endif + std::shared_ptr transport_; + std::shared_ptr client_; + int64_t sessionId_{}; + int64_t statementId_{}; + std::string zoneId_; + int timeFactor_{}; + + void initZoneId(); +}; + +#endif diff --git a/iotdb-client/client-cpp/src/rpc/ThriftConvert.cpp b/iotdb-client/client-cpp/src/rpc/ThriftConvert.cpp new file mode 100644 index 0000000000000..714be579f8489 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/ThriftConvert.cpp @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "ThriftConvert.h" + +#include "common_types.h" + +Status statusFromThrift(const TSStatus &tsStatus) { + Status status; + status.code = tsStatus.code; + status.message = tsStatus.message; + return status; +} + +Endpoint endpointFromThrift(const TEndPoint &endPoint) { + Endpoint endpoint; + endpoint.host = endPoint.ip; + endpoint.port = endPoint.port; + return endpoint; +} + +TEndPoint endpointToThrift(const Endpoint &endpoint) { + TEndPoint endPoint; + endPoint.__set_ip(endpoint.host); + endPoint.__set_port(endpoint.port); + return endPoint; +} + +std::map endpointMapFromThrift( + const std::map &deviceEndPointMap) { + std::map result; + for (const auto &entry : deviceEndPointMap) { + result.emplace(entry.first, endpointFromThrift(entry.second)); + } + return result; +} + +std::vector +endpointListFromThrift(const std::vector &endPointList) { + std::vector result; + result.reserve(endPointList.size()); + for (const auto &endPoint : endPointList) { + result.push_back(endpointFromThrift(endPoint)); + } + return result; +} diff --git a/iotdb-client/client-cpp/src/rpc/ThriftConvert.h b/iotdb-client/client-cpp/src/rpc/ThriftConvert.h new file mode 100644 index 0000000000000..effb3da097344 --- /dev/null +++ b/iotdb-client/client-cpp/src/rpc/ThriftConvert.h @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +#ifndef IOTDB_THRIFT_CONVERT_H +#define IOTDB_THRIFT_CONVERT_H + +#include +#include +#include + +#include "Endpoint.h" +#include "Status.h" + +class TEndPoint; +class TSStatus; + +Status statusFromThrift(const TSStatus &tsStatus); +Endpoint endpointFromThrift(const TEndPoint &endPoint); +TEndPoint endpointToThrift(const Endpoint &endpoint); + +std::map endpointMapFromThrift( + const std::map &deviceEndPointMap); +std::vector +endpointListFromThrift(const std::vector &endPointList); + +#endif diff --git a/iotdb-client/client-cpp/src/session/Column.cpp b/iotdb-client/client-cpp/src/session/Column.cpp new file mode 100644 index 0000000000000..d01921598c606 --- /dev/null +++ b/iotdb-client/client-cpp/src/session/Column.cpp @@ -0,0 +1,459 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "Column.h" +#include "ColumnDecoder.h" + +TimeColumn::TimeColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &values) + : arrayOffset_(arrayOffset), positionCount_(positionCount), + values_(values) { + if (arrayOffset < 0) + throw IoTDBException("arrayOffset is negative"); + if (positionCount < 0) + throw IoTDBException("positionCount is negative"); + if (static_cast(values.size()) - arrayOffset < positionCount) { + throw IoTDBException("values length is less than positionCount"); + } +} + +TSDataType::TSDataType TimeColumn::getDataType() const { + return TSDataType::INT64; +} +ColumnEncoding TimeColumn::getEncoding() const { + return ColumnEncoding::Int64Array; +} + +int64_t TimeColumn::getLong(int32_t position) const { + return values_[position + arrayOffset_]; +} + +bool TimeColumn::mayHaveNull() const { return false; } +bool TimeColumn::isNull(int32_t position) const { return false; } +std::vector TimeColumn::isNulls() const { return {}; } + +int32_t TimeColumn::getPositionCount() const { return positionCount_; } + +int64_t TimeColumn::getStartTime() const { return values_[arrayOffset_]; } +int64_t TimeColumn::getEndTime() const { + return values_[positionCount_ + arrayOffset_ - 1]; +} + +const std::vector &TimeColumn::getTimes() const { return values_; } +std::vector TimeColumn::getLongs() const { return getTimes(); } + +BinaryColumn::BinaryColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector> &values) + : arrayOffset_(arrayOffset), positionCount_(positionCount), + valueIsNull_(valueIsNull), values_(values) { + if (arrayOffset < 0) + throw IoTDBException("arrayOffset is negative"); + if (positionCount < 0) + throw IoTDBException("positionCount is negative"); + if (static_cast(values.size()) - arrayOffset < positionCount) { + throw IoTDBException("values length is less than positionCount"); + } + if (!valueIsNull.empty() && + static_cast(valueIsNull.size()) - arrayOffset < positionCount) { + throw IoTDBException("isNull length is less than positionCount"); + } +} + +TSDataType::TSDataType BinaryColumn::getDataType() const { + return TSDataType::TSDataType::TEXT; +} +ColumnEncoding BinaryColumn::getEncoding() const { + return ColumnEncoding::BinaryArray; +} + +std::shared_ptr BinaryColumn::getBinary(int32_t position) const { + return values_[position + arrayOffset_]; +} + +std::vector> BinaryColumn::getBinaries() const { + return values_; +} + +bool BinaryColumn::mayHaveNull() const { return !valueIsNull_.empty(); } + +bool BinaryColumn::isNull(int32_t position) const { + return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; +} + +std::vector BinaryColumn::isNulls() const { + if (!valueIsNull_.empty()) + return valueIsNull_; + + std::vector result(positionCount_, false); + return result; +} + +int32_t BinaryColumn::getPositionCount() const { return positionCount_; } + +IntColumn::IntColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values) + : arrayOffset_(arrayOffset), positionCount_(positionCount), + valueNull_(valueIsNull), values_(values) { + if (arrayOffset < 0) + throw IoTDBException("arrayOffset is negative"); + if (positionCount < 0) + throw IoTDBException("positionCount is negative"); + if (static_cast(values.size()) - arrayOffset < positionCount) { + throw IoTDBException("values length is less than positionCount"); + } + if (!valueIsNull.empty() && + static_cast(valueIsNull.size()) - arrayOffset < positionCount) { + throw IoTDBException("isNull length is less than positionCount"); + } +} + +TSDataType::TSDataType IntColumn::getDataType() const { + return TSDataType::INT32; +} +ColumnEncoding IntColumn::getEncoding() const { + return ColumnEncoding::Int32Array; +} + +int32_t IntColumn::getInt(int32_t position) const { + return values_[position + arrayOffset_]; +} + +int64_t IntColumn::getLong(int32_t position) const { + return values_[position + arrayOffset_]; +} + +float IntColumn::getFloat(int32_t position) const { + return values_[position + arrayOffset_]; +} + +double IntColumn::getDouble(int32_t position) const { + return values_[position + arrayOffset_]; +} + +std::vector IntColumn::getInts() const { return values_; } + +bool IntColumn::mayHaveNull() const { return !valueNull_.empty(); } + +bool IntColumn::isNull(int32_t position) const { + return !valueNull_.empty() && valueNull_[position + arrayOffset_]; +} + +std::vector IntColumn::isNulls() const { + if (!valueNull_.empty()) + return valueNull_; + + std::vector result(positionCount_, false); + return result; +} + +int32_t IntColumn::getPositionCount() const { return positionCount_; } + +FloatColumn::FloatColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values) + : arrayOffset_(arrayOffset), positionCount_(positionCount), + valueIsNull_(valueIsNull), values_(values) { + if (arrayOffset < 0) + throw IoTDBException("arrayOffset is negative"); + if (positionCount < 0) + throw IoTDBException("positionCount is negative"); + if (static_cast(values.size()) - arrayOffset < positionCount) { + throw IoTDBException("values length is less than positionCount"); + } + if (!valueIsNull.empty() && + static_cast(valueIsNull.size()) - arrayOffset < positionCount) { + throw IoTDBException("isNull length is less than positionCount"); + } +} + +TSDataType::TSDataType FloatColumn::getDataType() const { + return TSDataType::TSDataType::FLOAT; +} +ColumnEncoding FloatColumn::getEncoding() const { + return ColumnEncoding::Int32Array; +} + +float FloatColumn::getFloat(int32_t position) const { + return values_[position + arrayOffset_]; +} + +double FloatColumn::getDouble(int32_t position) const { + return values_[position + arrayOffset_]; +} + +std::vector FloatColumn::getFloats() const { return values_; } + +bool FloatColumn::mayHaveNull() const { return !valueIsNull_.empty(); } + +bool FloatColumn::isNull(int32_t position) const { + return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; +} + +std::vector FloatColumn::isNulls() const { + if (!valueIsNull_.empty()) + return valueIsNull_; + + std::vector result(positionCount_, false); + return result; +} + +int32_t FloatColumn::getPositionCount() const { return positionCount_; } + +LongColumn::LongColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values) + : arrayOffset_(arrayOffset), positionCount_(positionCount), + valueIsNull_(valueIsNull), values_(values) { + if (arrayOffset < 0) + throw IoTDBException("arrayOffset is negative"); + if (positionCount < 0) + throw IoTDBException("positionCount is negative"); + if (static_cast(values.size()) - arrayOffset < positionCount) { + throw IoTDBException("values length is less than positionCount"); + } + if (!valueIsNull.empty() && + static_cast(valueIsNull.size()) - arrayOffset < positionCount) { + throw IoTDBException("isNull length is less than positionCount"); + } +} + +TSDataType::TSDataType LongColumn::getDataType() const { + return TSDataType::TSDataType::INT64; +} +ColumnEncoding LongColumn::getEncoding() const { + return ColumnEncoding::Int64Array; +} + +int64_t LongColumn::getLong(int32_t position) const { + return values_[position + arrayOffset_]; +} + +double LongColumn::getDouble(int32_t position) const { + return values_[position + arrayOffset_]; +} + +std::vector LongColumn::getLongs() const { return values_; } + +bool LongColumn::mayHaveNull() const { return !valueIsNull_.empty(); } + +bool LongColumn::isNull(int32_t position) const { + return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; +} + +std::vector LongColumn::isNulls() const { + if (!valueIsNull_.empty()) + return valueIsNull_; + + std::vector result(positionCount_, false); + return result; +} + +int32_t LongColumn::getPositionCount() const { return positionCount_; } + +DoubleColumn::DoubleColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values) + : arrayOffset_(arrayOffset), positionCount_(positionCount), + valueIsNull_(valueIsNull), values_(values) { + if (arrayOffset < 0) + throw IoTDBException("arrayOffset is negative"); + if (positionCount < 0) + throw IoTDBException("positionCount is negative"); + if (static_cast(values.size()) - arrayOffset < positionCount) { + throw IoTDBException("values length is less than positionCount"); + } + if (!valueIsNull.empty() && + static_cast(valueIsNull.size()) - arrayOffset < positionCount) { + throw IoTDBException("isNull length is less than positionCount"); + } +} + +TSDataType::TSDataType DoubleColumn::getDataType() const { + return TSDataType::TSDataType::DOUBLE; +} +ColumnEncoding DoubleColumn::getEncoding() const { + return ColumnEncoding::Int64Array; +} + +double DoubleColumn::getDouble(int32_t position) const { + return values_[position + arrayOffset_]; +} + +std::vector DoubleColumn::getDoubles() const { return values_; } + +bool DoubleColumn::mayHaveNull() const { return !valueIsNull_.empty(); } + +bool DoubleColumn::isNull(int32_t position) const { + return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; +} + +std::vector DoubleColumn::isNulls() const { + if (!valueIsNull_.empty()) + return valueIsNull_; + + std::vector result(positionCount_, false); + return result; +} + +int32_t DoubleColumn::getPositionCount() const { return positionCount_; } + +BooleanColumn::BooleanColumn(int32_t arrayOffset, int32_t positionCount, + const std::vector &valueIsNull, + const std::vector &values) + : arrayOffset_(arrayOffset), positionCount_(positionCount), + valueIsNull_(valueIsNull), values_(values) { + if (arrayOffset < 0) + throw IoTDBException("arrayOffset is negative"); + if (positionCount < 0) + throw IoTDBException("positionCount is negative"); + if (static_cast(values.size()) - arrayOffset < positionCount) { + throw IoTDBException("values length is less than positionCount"); + } + if (!valueIsNull.empty() && + static_cast(valueIsNull.size()) - arrayOffset < positionCount) { + throw IoTDBException("isNull length is less than positionCount"); + } +} + +TSDataType::TSDataType BooleanColumn::getDataType() const { + return TSDataType::TSDataType::BOOLEAN; +} +ColumnEncoding BooleanColumn::getEncoding() const { + return ColumnEncoding::ByteArray; +} + +bool BooleanColumn::getBoolean(int32_t position) const { + return values_[position + arrayOffset_]; +} + +std::vector BooleanColumn::getBooleans() const { return values_; } + +bool BooleanColumn::mayHaveNull() const { return !valueIsNull_.empty(); } + +bool BooleanColumn::isNull(int32_t position) const { + return !valueIsNull_.empty() && valueIsNull_[position + arrayOffset_]; +} + +std::vector BooleanColumn::isNulls() const { + if (!valueIsNull_.empty()) + return valueIsNull_; + + std::vector result(positionCount_, false); + return result; +} + +int32_t BooleanColumn::getPositionCount() const { return positionCount_; } + +RunLengthEncodedColumn::RunLengthEncodedColumn(std::shared_ptr value, + int32_t positionCount) + : value_(value), positionCount_(positionCount) { + if (!value) + throw IoTDBException("value is null"); + if (value->getPositionCount() != 1) { + throw IoTDBException("Expected value to contain a single position"); + } + if (positionCount < 0) + throw IoTDBException("positionCount is negative"); +} + +std::shared_ptr RunLengthEncodedColumn::getValue() const { + return value_; +} + +TSDataType::TSDataType RunLengthEncodedColumn::getDataType() const { + return value_->getDataType(); +} +ColumnEncoding RunLengthEncodedColumn::getEncoding() const { + return ColumnEncoding::Rle; +} + +bool RunLengthEncodedColumn::getBoolean(int32_t position) const { + return value_->getBoolean(0); +} + +int32_t RunLengthEncodedColumn::getInt(int32_t position) const { + return value_->getInt(0); +} + +int64_t RunLengthEncodedColumn::getLong(int32_t position) const { + return value_->getLong(0); +} + +float RunLengthEncodedColumn::getFloat(int32_t position) const { + return value_->getFloat(0); +} + +double RunLengthEncodedColumn::getDouble(int32_t position) const { + return value_->getDouble(0); +} + +std::shared_ptr +RunLengthEncodedColumn::getBinary(int32_t position) const { + return value_->getBinary(0); +} + +std::vector RunLengthEncodedColumn::getBooleans() const { + bool v = value_->getBoolean(0); + return std::vector(positionCount_, v); +} + +std::vector RunLengthEncodedColumn::getInts() const { + int32_t v = value_->getInt(0); + return std::vector(positionCount_, v); +} + +std::vector RunLengthEncodedColumn::getLongs() const { + int64_t v = value_->getLong(0); + return std::vector(positionCount_, v); +} + +std::vector RunLengthEncodedColumn::getFloats() const { + float v = value_->getFloat(0); + return std::vector(positionCount_, v); +} + +std::vector RunLengthEncodedColumn::getDoubles() const { + double v = value_->getDouble(0); + return std::vector(positionCount_, v); +} + +std::vector> +RunLengthEncodedColumn::getBinaries() const { + auto v = value_->getBinary(0); + return std::vector>(positionCount_, v); +} + +bool RunLengthEncodedColumn::mayHaveNull() const { + return value_->mayHaveNull(); +} + +bool RunLengthEncodedColumn::isNull(int32_t position) const { + return value_->isNull(0); +} + +std::vector RunLengthEncodedColumn::isNulls() const { + bool v = value_->isNull(0); + return std::vector(positionCount_, v); +} + +int32_t RunLengthEncodedColumn::getPositionCount() const { + return positionCount_; +} diff --git a/iotdb-client/client-cpp/src/session/ColumnDecoder.cpp b/iotdb-client/client-cpp/src/session/ColumnDecoder.cpp new file mode 100644 index 0000000000000..f455ac5a8ce5a --- /dev/null +++ b/iotdb-client/client-cpp/src/session/ColumnDecoder.cpp @@ -0,0 +1,207 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "ColumnDecoder.h" + +#include "Column.h" + +std::vector deserializeNullIndicators(MyStringBuffer &buffer, + int32_t positionCount) { + uint8_t mayHaveNullByte = buffer.getChar(); + + bool mayHaveNull = mayHaveNullByte != 0; + if (!mayHaveNull) { + return {}; + } + + return deserializeBooleanArray(buffer, positionCount); +} + +std::vector deserializeBooleanArray(MyStringBuffer &buffer, + int32_t size) { + const int32_t packedSize = (size + 7) / 8; + std::vector packedBytes(packedSize); + for (int i = 0; i < packedSize; i++) { + packedBytes[i] = buffer.getChar(); + } + + std::vector output(size); + int currentByte = 0; + const int fullGroups = size & ~0b111; + + for (int pos = 0; pos < fullGroups; pos += 8) { + const uint8_t b = packedBytes[currentByte++]; + output[pos + 0] = (b & 0b10000000) != 0; + output[pos + 1] = (b & 0b01000000) != 0; + output[pos + 2] = (b & 0b00100000) != 0; + output[pos + 3] = (b & 0b00010000) != 0; + output[pos + 4] = (b & 0b00001000) != 0; + output[pos + 5] = (b & 0b00000100) != 0; + output[pos + 6] = (b & 0b00000010) != 0; + output[pos + 7] = (b & 0b00000001) != 0; + } + + if ((size & 0b111) > 0) { + const uint8_t b = packedBytes.back(); + uint8_t mask = 0b10000000; + + for (int pos = fullGroups; pos < size; pos++) { + output[pos] = (b & mask) != 0; + mask >>= 1; + } + } + + return output; +} + +std::unique_ptr +BaseColumnDecoder::readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) { + return nullptr; +} + +std::unique_ptr +Int32ArrayColumnDecoder::readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) { + auto nullIndicators = deserializeNullIndicators(buffer, positionCount); + + switch (dataType) { + case TSDataType::INT32: + case TSDataType::DATE: { + std::vector intValues(positionCount); + for (int32_t i = 0; i < positionCount; i++) { + if (!nullIndicators.empty() && nullIndicators[i]) + continue; + intValues[i] = buffer.getInt(); + } + return std::unique_ptr( + new IntColumn(0, positionCount, nullIndicators, intValues)); + } + case TSDataType::FLOAT: { + std::vector floatValues(positionCount); + for (int32_t i = 0; i < positionCount; i++) { + if (!nullIndicators.empty() && nullIndicators[i]) + continue; + floatValues[i] = buffer.getFloat(); + } + return std::unique_ptr( + new FloatColumn(0, positionCount, nullIndicators, floatValues)); + } + default: + throw IoTDBException("Invalid data type for Int32ArrayColumnDecoder"); + } +} + +std::unique_ptr +Int64ArrayColumnDecoder::readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) { + auto nullIndicators = deserializeNullIndicators(buffer, positionCount); + + switch (dataType) { + case TSDataType::INT64: + case TSDataType::TIMESTAMP: { + std::vector values(positionCount); + for (int32_t i = 0; i < positionCount; i++) { + if (!nullIndicators.empty() && nullIndicators[i]) + continue; + values[i] = buffer.getInt64(); + } + return std::unique_ptr( + new LongColumn(0, positionCount, nullIndicators, values)); + } + case TSDataType::DOUBLE: { + std::vector values(positionCount); + for (int32_t i = 0; i < positionCount; i++) { + if (!nullIndicators.empty() && nullIndicators[i]) + continue; + values[i] = buffer.getDouble(); + } + return std::unique_ptr( + new DoubleColumn(0, positionCount, nullIndicators, values)); + } + default: + throw IoTDBException("Invalid data type for Int64ArrayColumnDecoder"); + } +} + +std::unique_ptr +ByteArrayColumnDecoder::readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) { + if (dataType != TSDataType::BOOLEAN) { + throw IoTDBException("Invalid data type for ByteArrayColumnDecoder"); + } + + auto nullIndicators = deserializeNullIndicators(buffer, positionCount); + auto values = deserializeBooleanArray(buffer, positionCount); + return std::unique_ptr( + new BooleanColumn(0, positionCount, nullIndicators, values)); +} + +std::unique_ptr +BinaryArrayColumnDecoder::readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) { + if (dataType != TSDataType::TEXT) { + throw IoTDBException("Invalid data type for BinaryArrayColumnDecoder"); + } + + auto nullIndicators = deserializeNullIndicators(buffer, positionCount); + std::vector> values(positionCount); + + for (int32_t i = 0; i < positionCount; i++) { + if (!nullIndicators.empty() && nullIndicators[i]) + continue; + + int32_t length = buffer.getInt(); + if (length < 0) { + throw IoTDBException("BinaryArrayColumnDecoder: negative TEXT length"); + } + + std::vector value(length); + for (int32_t j = 0; j < length; j++) { + value[j] = buffer.getChar(); + } + + values[i] = std::make_shared(value); + } + + return std::unique_ptr( + new BinaryColumn(0, positionCount, nullIndicators, values)); +} + +std::unique_ptr +RunLengthColumnDecoder::readColumn(MyStringBuffer &buffer, + TSDataType::TSDataType dataType, + int32_t positionCount) { + uint8_t encodingByte = buffer.getChar(); + + auto columnEncoding = static_cast(encodingByte); + auto decoder = getColumnDecoder(columnEncoding); + + auto column = decoder->readColumn(buffer, dataType, 1); + if (!column) { + throw IoTDBException("Failed to read inner column"); + } + return std::unique_ptr( + new RunLengthEncodedColumn(move(column), positionCount)); +} diff --git a/iotdb-client/client-cpp/src/session/Common.cpp b/iotdb-client/client-cpp/src/session/Common.cpp new file mode 100644 index 0000000000000..6f449ea840af8 --- /dev/null +++ b/iotdb-client/client-cpp/src/session/Common.cpp @@ -0,0 +1,315 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "Common.h" + +#include +#include +#include + +LogLevelType LOG_LEVEL = LEVEL_WARN; + +std::string extractExceptionMessage(const std::exception &exception) { + const char *what = exception.what(); + if (what != nullptr) { + std::string message(what); + if (!message.empty() && message != "std::exception") { + return message; + } + } + return std::string("Unhandled exception type: ") + typeid(exception).name(); +} + +std::string extractExceptionMessage(const std::exception_ptr &exceptionPtr) { + if (exceptionPtr == nullptr) { + return "Unknown exception"; + } + try { + std::rethrow_exception(exceptionPtr); + } catch (const std::exception &exception) { + return extractExceptionMessage(exception); + } catch (...) { + return "Unknown non-std exception"; + } +} + +std::string getTimePrecision(int32_t timeFactor) { + if (timeFactor >= 1000000) + return "us"; + if (timeFactor >= 1000) + return "ms"; + return "s"; +} + +std::string formatDatetime(const std::string &format, + const std::string &precision, int64_t timestamp, + const std::string &zoneId) { + (void)precision; + (void)zoneId; + std::time_t time = static_cast(timestamp); + std::tm *tm = std::localtime(&time); + char buffer[80]; + strftime(buffer, sizeof(buffer), format.c_str(), tm); + return std::string(buffer); +} + +std::tm convertToTimestamp(int64_t value, int32_t timeFactor) { + std::time_t time = static_cast(value / timeFactor); + return *std::localtime(&time); +} + +TSDataType::TSDataType getDataTypeByStr(const std::string &typeStr) { + if (typeStr == "BOOLEAN") + return TSDataType::BOOLEAN; + if (typeStr == "INT32") + return TSDataType::INT32; + if (typeStr == "INT64") + return TSDataType::INT64; + if (typeStr == "FLOAT") + return TSDataType::FLOAT; + if (typeStr == "DOUBLE") + return TSDataType::DOUBLE; + if (typeStr == "TEXT") + return TSDataType::TEXT; + if (typeStr == "TIMESTAMP") + return TSDataType::TIMESTAMP; + if (typeStr == "DATE") + return TSDataType::DATE; + if (typeStr == "BLOB") + return TSDataType::BLOB; + if (typeStr == "STRING") + return TSDataType::STRING; + if (typeStr == "OBJECT") + return TSDataType::OBJECT; + return TSDataType::UNKNOWN; +} + +std::tm int32ToDate(int32_t value) { + std::time_t time = static_cast(value) * 86400; + return *std::localtime(&time); +} + +MyStringBuffer::MyStringBuffer() : pos(0) { checkBigEndian(); } + +MyStringBuffer::MyStringBuffer(const std::string &str) : str(str), pos(0) { + checkBigEndian(); +} + +void MyStringBuffer::reserve(size_t n) { str.reserve(n); } + +void MyStringBuffer::clear() { + str.clear(); + pos = 0; +} + +bool MyStringBuffer::hasRemaining() { return pos < str.size(); } + +int MyStringBuffer::getInt() { return *(int *)getOrderedByte(4); } + +IoTDBDate MyStringBuffer::getDate() { return parseIntToDate(getInt()); } + +int64_t MyStringBuffer::getInt64() { +#ifdef ARCH32 + const char *buf_addr = getOrderedByte(8); + if (reinterpret_cast(buf_addr) % 4 == 0) { + return *(int64_t *)buf_addr; + } else { + char tmp_buf[8]; + memcpy(tmp_buf, buf_addr, 8); + return *(int64_t *)tmp_buf; + } +#else + return *(int64_t *)getOrderedByte(8); +#endif +} + +float MyStringBuffer::getFloat() { return *(float *)getOrderedByte(4); } + +double MyStringBuffer::getDouble() { +#ifdef ARCH32 + const char *buf_addr = getOrderedByte(8); + if (reinterpret_cast(buf_addr) % 4 == 0) { + return *(double *)buf_addr; + } else { + char tmp_buf[8]; + memcpy(tmp_buf, buf_addr, 8); + return *(double *)tmp_buf; + } +#else + return *(double *)getOrderedByte(8); +#endif +} + +char MyStringBuffer::getChar() { + if (pos >= str.size()) { + throw IoTDBException( + "MyStringBuffer::getChar: read past end (pos=" + std::to_string(pos) + + ", size=" + std::to_string(str.size()) + ")"); + } + return str[pos++]; +} + +bool MyStringBuffer::getBool() { return getChar() == 1; } + +std::string MyStringBuffer::getString() { + const int lenInt = getInt(); + if (lenInt < 0) { + throw IoTDBException("MyStringBuffer::getString: negative length"); + } + const size_t len = static_cast(lenInt); + if (pos > str.size() || len > str.size() - pos) { + throw IoTDBException( + "MyStringBuffer::getString: length exceeds buffer (pos=" + + std::to_string(pos) + ", len=" + std::to_string(len) + + ", size=" + std::to_string(str.size()) + ")"); + } + const size_t tmpPos = pos; + pos += len; + return str.substr(tmpPos, len); +} + +void MyStringBuffer::putInt(int ins) { putOrderedByte((char *)&ins, 4); } + +void MyStringBuffer::putDate(IoTDBDate date) { + putInt(parseDateExpressionToInt(date)); +} + +void MyStringBuffer::putInt64(int64_t ins) { putOrderedByte((char *)&ins, 8); } + +void MyStringBuffer::putFloat(float ins) { putOrderedByte((char *)&ins, 4); } + +void MyStringBuffer::putDouble(double ins) { putOrderedByte((char *)&ins, 8); } + +void MyStringBuffer::putChar(char ins) { str += ins; } + +void MyStringBuffer::putBool(bool ins) { + char tmp = ins ? 1 : 0; + str += tmp; +} + +void MyStringBuffer::putString(const std::string &ins) { + putInt((int)(ins.size())); + str += ins; +} + +void MyStringBuffer::concat(const std::string &ins) { str.append(ins); } + +void MyStringBuffer::checkBigEndian() { + static int chk = 0x0201; + isBigEndian = (0x01 != *(char *)(&chk)); +} + +const char *MyStringBuffer::getOrderedByte(size_t len) { + if (pos > str.size() || len > str.size() - pos) { + throw IoTDBException("MyStringBuffer::getOrderedByte: read past end (pos=" + + std::to_string(pos) + ", len=" + std::to_string(len) + + ", size=" + std::to_string(str.size()) + ")"); + } + const char *p = nullptr; + if (isBigEndian) { + p = str.c_str() + pos; + } else { + const char *tmp = str.c_str(); + for (size_t i = pos; i < pos + len; i++) { + numericBuf[pos + len - 1 - i] = tmp[i]; + } + p = numericBuf; + } + pos += len; + return p; +} + +void MyStringBuffer::putOrderedByte(char *buf, int len) { + if (isBigEndian) { + str.append(buf, len); + } else { + for (int i = len - 1; i > -1; i--) { + str += buf[i]; + } + } +} + +BitMap::BitMap(size_t size) { resize(size); } + +void BitMap::resize(size_t size) { + this->size = size; + this->bits.resize((size >> 3) + 1); + reset(); +} + +bool BitMap::mark(size_t position) { + if (position >= size) + return false; + + bits[position >> 3] |= (char)1 << (position % 8); + return true; +} + +bool BitMap::unmark(size_t position) { + if (position >= size) + return false; + + bits[position >> 3] &= ~((char)1 << (position % 8)); + return true; +} + +void BitMap::markAll() { std::fill(bits.begin(), bits.end(), (char)0XFF); } + +void BitMap::reset() { std::fill(bits.begin(), bits.end(), (char)0); } + +bool BitMap::isMarked(size_t position) const { + if (position >= size) + return false; + + return (bits[position >> 3] & ((char)1 << (position % 8))) != 0; +} + +bool BitMap::isAllUnmarked() const { + size_t j; + for (j = 0; j < size >> 3; j++) { + if (bits[j] != (char)0) { + return false; + } + } + for (j = 0; j < size % 8; j++) { + if ((bits[size >> 3] & ((char)1 << j)) != 0) { + return false; + } + } + return true; +} + +bool BitMap::isAllMarked() const { + size_t j; + for (j = 0; j < size >> 3; j++) { + if (bits[j] != (char)0XFF) { + return false; + } + } + for (j = 0; j < size % 8; j++) { + if ((bits[size >> 3] & ((char)1 << j)) == 0) { + return false; + } + } + return true; +} + +const std::vector &BitMap::getByteArray() const { return this->bits; } + +size_t BitMap::getSize() const { return this->size; } diff --git a/iotdb-client/client-cpp/src/session/Date.cpp b/iotdb-client/client-cpp/src/session/Date.cpp new file mode 100644 index 0000000000000..094dad6b6aba6 --- /dev/null +++ b/iotdb-client/client-cpp/src/session/Date.cpp @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "Date.h" + +#include +#include +#include + +#include "Common.h" + +std::string IoTDBDate::toIsoExtendedString() const { + if (!valid_) { + return ""; + } + char buf[16]; + std::snprintf(buf, sizeof(buf), "%04d-%02d-%02d", year_, month_, day_); + return std::string(buf); +} + +int32_t parseDateExpressionToInt(const IoTDBDate &date) { + if (date.is_not_a_date()) { + throw IoTDBException("Date expression is null or empty."); + } + + const int year = date.year(); + if (year < 1000 || year > 9999) { + throw DateTimeParseException("Year must be between 1000 and 9999.", + date.toIsoExtendedString(), 0); + } + + const int64_t result = + static_cast(year) * 10000 + date.month() * 100 + date.day(); + if (result > INT32_MAX || result < INT32_MIN) { + throw DateTimeParseException("Date value overflow. ", + date.toIsoExtendedString(), 0); + } + return static_cast(result); +} + +IoTDBDate parseIntToDate(int32_t dateInt) { + if (dateInt == EMPTY_DATE_INT) { + return IoTDBDate::notADate(); + } + const int year = dateInt / 10000; + const int month = (dateInt % 10000) / 100; + const int day = dateInt % 100; + return IoTDBDate(year, month, day); +} diff --git a/iotdb-client/client-cpp/src/session/Session.cpp b/iotdb-client/client-cpp/src/session/Session.cpp new file mode 100644 index 0000000000000..d9b9edb1ebf18 --- /dev/null +++ b/iotdb-client/client-cpp/src/session/Session.cpp @@ -0,0 +1,2174 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "Session.h" +#include "SessionDataSet.h" +#include "SessionImpl.h" +#include "ThriftConvert.h" +#include +#include +#include +#include +#include +#include +#include +#include + +using apache::thrift::transport::TTransportException; + +using namespace std; + +/** + * Timeout of query can be set by users. + * A negative number means using the default configuration of server. + * And value 0 will disable the function of query timeout. + */ +static const int64_t QUERY_TIMEOUT_MS = -1; + +TSDataType::TSDataType getTSDataTypeFromString(const string &str) { + // BOOLEAN, INT32, INT64, FLOAT, DOUBLE, TEXT, STRING, BLOB, TIMESTAMP, DATE, + // NULLTYPE + if (str == "BOOLEAN") { + return TSDataType::BOOLEAN; + } else if (str == "INT32") { + return TSDataType::INT32; + } else if (str == "INT64") { + return TSDataType::INT64; + } else if (str == "FLOAT") { + return TSDataType::FLOAT; + } else if (str == "DOUBLE") { + return TSDataType::DOUBLE; + } else if (str == "TEXT") { + return TSDataType::TEXT; + } else if (str == "TIMESTAMP") { + return TSDataType::TIMESTAMP; + } else if (str == "DATE") { + return TSDataType::DATE; + } else if (str == "BLOB") { + return TSDataType::BLOB; + } else if (str == "STRING") { + return TSDataType::STRING; + } else if (str == "OBJECT") { + return TSDataType::OBJECT; + } + return TSDataType::UNKNOWN; +} + +void Tablet::createColumns() { + for (size_t i = 0; i < schemas.size(); i++) { + TSDataType::TSDataType dataType = schemas[i].second; + switch (dataType) { + case TSDataType::BOOLEAN: + values[i] = new bool[maxRowNumber]; + break; + case TSDataType::DATE: + values[i] = new IoTDBDate[maxRowNumber]; + break; + case TSDataType::INT32: + values[i] = new int[maxRowNumber]; + break; + case TSDataType::TIMESTAMP: + case TSDataType::INT64: + values[i] = new int64_t[maxRowNumber]; + break; + case TSDataType::FLOAT: + values[i] = new float[maxRowNumber]; + break; + case TSDataType::DOUBLE: + values[i] = new double[maxRowNumber]; + break; + case TSDataType::STRING: + case TSDataType::BLOB: + case TSDataType::OBJECT: + case TSDataType::TEXT: + values[i] = new string[maxRowNumber]; + break; + default: + throw UnSupportedDataTypeException( + string("Data type ") + to_string(dataType) + " is not supported."); + } + } +} + +void Tablet::deleteColumns() { + for (size_t i = 0; i < schemas.size(); i++) { + if (!values[i]) + continue; + TSDataType::TSDataType dataType = schemas[i].second; + switch (dataType) { + case TSDataType::BOOLEAN: { + bool *valueBuf = (bool *)(values[i]); + delete[] valueBuf; + break; + } + case TSDataType::INT32: { + int *valueBuf = (int *)(values[i]); + delete[] valueBuf; + break; + } + case TSDataType::DATE: { + IoTDBDate *valueBuf = (IoTDBDate *)(values[i]); + delete[] valueBuf; + break; + } + case TSDataType::TIMESTAMP: + case TSDataType::INT64: { + int64_t *valueBuf = (int64_t *)(values[i]); + delete[] valueBuf; + break; + } + case TSDataType::FLOAT: { + float *valueBuf = (float *)(values[i]); + delete[] valueBuf; + break; + } + case TSDataType::DOUBLE: { + double *valueBuf = (double *)(values[i]); + delete[] valueBuf; + break; + } + case TSDataType::STRING: + case TSDataType::BLOB: + case TSDataType::OBJECT: + case TSDataType::TEXT: { + string *valueBuf = (string *)(values[i]); + delete[] valueBuf; + break; + } + default: + throw UnSupportedDataTypeException( + string("Data type ") + to_string(dataType) + " is not supported."); + } + values[i] = nullptr; + } +} + +void Tablet::deepCopyTabletColValue(void *const *srcPtr, void **destPtr, + TSDataType::TSDataType type, + int maxRowNumber) { + void *src = *srcPtr; + switch (type) { + case TSDataType::BOOLEAN: + *destPtr = new bool[maxRowNumber]; + memcpy(*destPtr, src, maxRowNumber * sizeof(bool)); + break; + case TSDataType::INT32: + *destPtr = new int32_t[maxRowNumber]; + memcpy(*destPtr, src, maxRowNumber * sizeof(int32_t)); + break; + case TSDataType::INT64: + case TSDataType::TIMESTAMP: + *destPtr = new int64_t[maxRowNumber]; + memcpy(*destPtr, src, maxRowNumber * sizeof(int64_t)); + break; + case TSDataType::FLOAT: + *destPtr = new float[maxRowNumber]; + memcpy(*destPtr, src, maxRowNumber * sizeof(float)); + break; + case TSDataType::DOUBLE: + *destPtr = new double[maxRowNumber]; + memcpy(*destPtr, src, maxRowNumber * sizeof(double)); + break; + case TSDataType::DATE: { + *destPtr = new IoTDBDate[maxRowNumber]; + IoTDBDate *srcDate = static_cast(src); + IoTDBDate *destDate = static_cast(*destPtr); + for (size_t j = 0; j < maxRowNumber; ++j) { + destDate[j] = srcDate[j]; + } + break; + } + case TSDataType::STRING: + case TSDataType::TEXT: + case TSDataType::OBJECT: + case TSDataType::BLOB: { + *destPtr = new std::string[maxRowNumber]; + std::string *srcStr = static_cast(src); + std::string *destStr = static_cast(*destPtr); + for (size_t j = 0; j < maxRowNumber; ++j) { + destStr[j] = srcStr[j]; + } + break; + } + default: + break; + } +} + +void Tablet::reset() { + rowSize = 0; + for (size_t i = 0; i < schemas.size(); i++) { + bitMaps[i].reset(); + } +} + +size_t Tablet::getTimeBytesSize() { return rowSize * 8; } + +size_t Tablet::getValueByteSize() { + size_t valueOccupation = 0; + for (size_t i = 0; i < schemas.size(); i++) { + switch (schemas[i].second) { + case TSDataType::BOOLEAN: + valueOccupation += rowSize; + break; + case TSDataType::INT32: + valueOccupation += rowSize * 4; + break; + case TSDataType::DATE: + valueOccupation += rowSize * 4; + break; + case TSDataType::TIMESTAMP: + case TSDataType::INT64: + valueOccupation += rowSize * 8; + break; + case TSDataType::FLOAT: + valueOccupation += rowSize * 4; + break; + case TSDataType::DOUBLE: + valueOccupation += rowSize * 8; + break; + case TSDataType::STRING: + case TSDataType::BLOB: + case TSDataType::OBJECT: + case TSDataType::TEXT: { + valueOccupation += rowSize * 4; + string *valueBuf = (string *)(values[i]); + for (size_t j = 0; j < rowSize; j++) { + valueOccupation += valueBuf[j].size(); + } + break; + } + default: + throw UnSupportedDataTypeException(string("Data type ") + + to_string(schemas[i].second) + + " is not supported."); + } + } + return valueOccupation; +} + +void Tablet::setAligned(bool isAligned) { this->isAligned = isAligned; } + +string SessionUtils::getTime(const Tablet &tablet) { + MyStringBuffer timeBuffer; + unsigned int n = 8u * tablet.rowSize; + if (n > timeBuffer.str.capacity()) { + timeBuffer.reserve(n); + } + + for (size_t i = 0; i < tablet.rowSize; i++) { + timeBuffer.putInt64(tablet.timestamps[i]); + } + return timeBuffer.str; +} + +string SessionUtils::getValue(const Tablet &tablet) { + MyStringBuffer valueBuffer; + unsigned int n = 8u * tablet.schemas.size() * tablet.rowSize; + if (n > valueBuffer.str.capacity()) { + valueBuffer.reserve(n); + } + for (size_t i = 0; i < tablet.schemas.size(); i++) { + TSDataType::TSDataType dataType = tablet.schemas[i].second; + const BitMap &bitMap = tablet.bitMaps[i]; + switch (dataType) { + case TSDataType::BOOLEAN: { + bool *valueBuf = (bool *)(tablet.values[i]); + for (size_t index = 0; index < tablet.rowSize; index++) { + if (!bitMap.isMarked(index)) { + valueBuffer.putBool(valueBuf[index]); + } else { + valueBuffer.putBool(false); + } + } + break; + } + case TSDataType::INT32: { + int *valueBuf = (int *)(tablet.values[i]); + for (size_t index = 0; index < tablet.rowSize; index++) { + if (!bitMap.isMarked(index)) { + valueBuffer.putInt(valueBuf[index]); + } else { + valueBuffer.putInt((numeric_limits::min)()); + } + } + break; + } + case TSDataType::DATE: { + IoTDBDate *valueBuf = (IoTDBDate *)(tablet.values[i]); + for (size_t index = 0; index < tablet.rowSize; index++) { + if (!bitMap.isMarked(index)) { + valueBuffer.putDate(valueBuf[index]); + } else { + valueBuffer.putInt(EMPTY_DATE_INT); + } + } + break; + } + case TSDataType::TIMESTAMP: + case TSDataType::INT64: { + int64_t *valueBuf = (int64_t *)(tablet.values[i]); + for (size_t index = 0; index < tablet.rowSize; index++) { + if (!bitMap.isMarked(index)) { + valueBuffer.putInt64(valueBuf[index]); + } else { + valueBuffer.putInt64((numeric_limits::min)()); + } + } + break; + } + case TSDataType::FLOAT: { + float *valueBuf = (float *)(tablet.values[i]); + for (size_t index = 0; index < tablet.rowSize; index++) { + if (!bitMap.isMarked(index)) { + valueBuffer.putFloat(valueBuf[index]); + } else { + valueBuffer.putFloat((numeric_limits::min)()); + } + } + break; + } + case TSDataType::DOUBLE: { + double *valueBuf = (double *)(tablet.values[i]); + for (size_t index = 0; index < tablet.rowSize; index++) { + if (!bitMap.isMarked(index)) { + valueBuffer.putDouble(valueBuf[index]); + } else { + valueBuffer.putDouble((numeric_limits::min)()); + } + } + break; + } + case TSDataType::STRING: + case TSDataType::BLOB: + case TSDataType::OBJECT: + case TSDataType::TEXT: { + string *valueBuf = (string *)(tablet.values[i]); + for (size_t index = 0; index < tablet.rowSize; index++) { + if (!bitMap.isMarked(index)) { + valueBuffer.putString(valueBuf[index]); + } else { + valueBuffer.putString(""); + } + } + break; + } + default: + throw UnSupportedDataTypeException( + string("Data type ") + to_string(dataType) + " is not supported."); + } + } + for (size_t i = 0; i < tablet.schemas.size(); i++) { + const BitMap &bitMap = tablet.bitMaps[i]; + bool columnHasNull = !bitMap.isAllUnmarked(); + valueBuffer.putChar(columnHasNull ? (char)1 : (char)0); + if (columnHasNull) { + const vector &bytes = bitMap.getByteArray(); + for (size_t index = 0; index < tablet.rowSize / 8 + 1; index++) { + valueBuffer.putChar(bytes[index]); + } + } + } + return valueBuffer.str; +} + +string MeasurementNode::serialize() const { + MyStringBuffer buffer; + buffer.putString(getName()); + buffer.putChar(getDataType()); + buffer.putChar(getEncoding()); + buffer.putChar(getCompressionType()); + return buffer.str; +} + +string Template::serialize() const { + MyStringBuffer buffer; + std::stack>> nodeStack; + unordered_set alignedPrefix; + buffer.putString(getName()); + buffer.putBool(isAligned()); + if (isAligned()) { + alignedPrefix.emplace(""); + } + + for (const auto &child : children_) { + nodeStack.push(make_pair("", child.second)); + } + + while (!nodeStack.empty()) { + auto cur = nodeStack.top(); + nodeStack.pop(); + + string prefix = cur.first; + shared_ptr cur_node_ptr = cur.second; + string fullPath(prefix); + + if (!cur_node_ptr->isMeasurement()) { + if (!prefix.empty()) { + fullPath.append("."); + } + fullPath.append(cur_node_ptr->getName()); + if (cur_node_ptr->isAligned()) { + alignedPrefix.emplace(fullPath); + } + for (const auto &child : cur_node_ptr->getChildren()) { + nodeStack.push(make_pair(fullPath, child.second)); + } + } else { + buffer.putString(prefix); + buffer.putBool(alignedPrefix.find(prefix) != alignedPrefix.end()); + buffer.concat(cur_node_ptr->serialize()); + } + } + + return buffer.str; +} + +Session::Session(const std::string &host, int rpcPort) : impl_(new Impl()) { + impl_->username_ = "root"; + impl_->password_ = "root"; + impl_->version = Version::V_1_0; + impl_->host_ = host; + impl_->rpcPort_ = rpcPort; + impl_->initZoneId(); + impl_->initNodesSupplier(); +} + +Session::Session(const std::vector &nodeUrls, + const std::string &username, const std::string &password) + : impl_(new Impl()) { + impl_->nodeUrls_ = nodeUrls; + impl_->username_ = username; + impl_->password_ = password; + impl_->version = Version::V_1_0; + impl_->initZoneId(); + impl_->initNodesSupplier(impl_->nodeUrls_); +} + +Session::Session(const std::string &host, int rpcPort, + const std::string &username, const std::string &password) + : impl_(new Impl()) { + impl_->host_ = host; + impl_->rpcPort_ = rpcPort; + impl_->username_ = username; + impl_->password_ = password; + impl_->fetchSize_ = iotdb::session::DEFAULT_FETCH_SIZE; + impl_->version = Version::V_1_0; + impl_->initZoneId(); + impl_->initNodesSupplier(); +} + +Session::Session(const std::string &host, int rpcPort, + const std::string &username, const std::string &password, + const std::string &zoneId, int fetchSize) + : impl_(new Impl()) { + impl_->host_ = host; + impl_->rpcPort_ = rpcPort; + impl_->username_ = username; + impl_->password_ = password; + impl_->zoneId_ = zoneId; + impl_->fetchSize_ = fetchSize; + impl_->version = Version::V_1_0; + impl_->initZoneId(); + impl_->initNodesSupplier(); +} + +Session::Session(const std::string &host, const std::string &rpcPort, + const std::string &username, const std::string &password, + const std::string &zoneId, int fetchSize) + : impl_(new Impl()) { + impl_->host_ = host; + impl_->rpcPort_ = stoi(rpcPort); + impl_->username_ = username; + impl_->password_ = password; + impl_->zoneId_ = zoneId; + impl_->fetchSize_ = fetchSize; + impl_->version = Version::V_1_0; + impl_->initZoneId(); + impl_->initNodesSupplier(); +} + +Session::Session(AbstractSessionBuilder *builder) : impl_(new Impl()) { + impl_->host_ = builder->host; + impl_->rpcPort_ = builder->rpcPort; + impl_->username_ = builder->username; + impl_->password_ = builder->password; + impl_->zoneId_ = builder->zoneId; + impl_->fetchSize_ = builder->fetchSize; + impl_->version = Version::V_1_0; + impl_->sqlDialect_ = builder->sqlDialect; + impl_->database_ = builder->database; + impl_->enableAutoFetch_ = builder->enableAutoFetch; + impl_->enableRedirection_ = builder->enableRedirections; + impl_->connectTimeoutMs_ = builder->connectTimeoutMs; + impl_->nodeUrls_ = builder->nodeUrls; + impl_->useSSL_ = builder->useSSL; + impl_->trustCertFilePath_ = builder->trustCertFilePath; + impl_->initZoneId(); + impl_->initNodesSupplier(impl_->nodeUrls_); +} + +/** + * When delete variable, make sure release all resource. + */ +Session::~Session() { + try { + close(); + } catch (const exception &e) { + log_debug(e.what()); + } +} + +void Session::Impl::removeBrokenSessionConnection( + shared_ptr sessionConnection) { + if (enableRedirection_) { + this->endPointToSessionConnection.erase(sessionConnection->getEndPoint()); + } + + auto it1 = deviceIdToEndpoint.begin(); + while (it1 != deviceIdToEndpoint.end()) { + if (it1->second == sessionConnection->getEndPoint()) { + it1 = deviceIdToEndpoint.erase(it1); + } else { + ++it1; + } + } +} + +/** + * check whether the batch has been sorted + * + * @return whether the batch has been sorted + */ +bool Session::Impl::checkSorted(const Tablet &tablet) { + for (size_t i = 1; i < tablet.rowSize; i++) { + if (tablet.timestamps[i] < tablet.timestamps[i - 1]) { + return false; + } + } + return true; +} + +bool Session::Impl::checkSorted(const vector ×) { + for (size_t i = 1; i < times.size(); i++) { + if (times[i] < times[i - 1]) { + return false; + } + } + return true; +} + +template +std::vector sortList(const std::vector &valueList, const int *index, + int indexLength) { + std::vector sortedValues(valueList.size()); + for (int i = 0; i < indexLength; i++) { + sortedValues[i] = valueList[index[i]]; + } + return sortedValues; +} + +template +void sortValuesList(T *valueList, const int *index, size_t indexLength) { + T *sortedValues = new T[indexLength]; + for (int i = 0; i < indexLength; i++) { + sortedValues[i] = valueList[index[i]]; + } + for (int i = 0; i < indexLength; i++) { + valueList[i] = sortedValues[i]; + } + delete[] sortedValues; +} + +void Session::Impl::sortTablet(Tablet &tablet) { + /* + * following part of code sort the batch data by time, + * so we can insert continuous data in value list to get a better performance + */ + // sort to get index, and use index to sort value list + int *index = new int[tablet.rowSize]; + for (size_t i = 0; i < tablet.rowSize; i++) { + index[i] = i; + } + + sortIndexByTimestamp(index, tablet.timestamps, tablet.rowSize); + tablet.timestamps = sortList(tablet.timestamps, index, tablet.rowSize); + for (size_t i = 0; i < tablet.schemas.size(); i++) { + TSDataType::TSDataType dataType = tablet.schemas[i].second; + switch (dataType) { + case TSDataType::BOOLEAN: { + sortValuesList((bool *)(tablet.values[i]), index, tablet.rowSize); + break; + } + case TSDataType::INT32: { + sortValuesList((int *)(tablet.values[i]), index, tablet.rowSize); + break; + } + case TSDataType::DATE: { + sortValuesList((IoTDBDate *)(tablet.values[i]), index, tablet.rowSize); + break; + } + case TSDataType::TIMESTAMP: + case TSDataType::INT64: { + sortValuesList((int64_t *)(tablet.values[i]), index, tablet.rowSize); + break; + } + case TSDataType::FLOAT: { + sortValuesList((float *)(tablet.values[i]), index, tablet.rowSize); + break; + } + case TSDataType::DOUBLE: { + sortValuesList((double *)(tablet.values[i]), index, tablet.rowSize); + break; + } + case TSDataType::STRING: + case TSDataType::BLOB: + case TSDataType::OBJECT: + case TSDataType::TEXT: { + sortValuesList((string *)(tablet.values[i]), index, tablet.rowSize); + break; + } + default: + throw UnSupportedDataTypeException( + string("Data type ") + to_string(dataType) + " is not supported."); + } + } + + delete[] index; +} + +void Session::Impl::sortIndexByTimestamp(int *index, + std::vector ×tamps, + int length) { + if (length <= 1) { + return; + } + + TsCompare tsCompareObj(timestamps); + std::sort(&index[0], &index[length], tsCompareObj); +} + +/** + * Append value into buffer in Big Endian order to comply with IoTDB server + */ +void Session::Impl::appendValues(string &buffer, const char *value, int size) { + static bool hasCheckedEndianFlag = false; + static bool localCpuIsBigEndian = false; + if (!hasCheckedEndianFlag) { + hasCheckedEndianFlag = true; + int chk = + 0x0201; // used to distinguish CPU's type (BigEndian or LittleEndian) + localCpuIsBigEndian = (0x01 != *(char *)(&chk)); + } + + if (localCpuIsBigEndian) { + buffer.append(value, size); + } else { + for (int i = size - 1; i >= 0; i--) { + buffer.append(value + i, 1); + } + } +} + +void Session::Impl::putValuesIntoBuffer( + const vector &types, const vector &values, + string &buf) { + int32_t date; + for (size_t i = 0; i < values.size(); i++) { + int8_t typeNum = getDataTypeNumber(types[i]); + buf.append((char *)(&typeNum), sizeof(int8_t)); + switch (types[i]) { + case TSDataType::BOOLEAN: + buf.append(values[i], 1); + break; + case TSDataType::INT32: + appendValues(buf, values[i], sizeof(int32_t)); + break; + case TSDataType::DATE: + date = parseDateExpressionToInt(*(IoTDBDate *)values[i]); + appendValues(buf, (char *)&date, sizeof(int32_t)); + break; + case TSDataType::TIMESTAMP: + case TSDataType::INT64: + appendValues(buf, values[i], sizeof(int64_t)); + break; + case TSDataType::FLOAT: + appendValues(buf, values[i], sizeof(float)); + break; + case TSDataType::DOUBLE: + appendValues(buf, values[i], sizeof(double)); + break; + case TSDataType::STRING: + case TSDataType::BLOB: + case TSDataType::OBJECT: + case TSDataType::TEXT: { + int32_t len = (uint32_t)strlen(values[i]); + appendValues(buf, (char *)(&len), sizeof(uint32_t)); + // no need to change the byte order of string value + buf.append(values[i], len); + break; + } + default: + break; + } + } +} + +int8_t Session::Impl::getDataTypeNumber(TSDataType::TSDataType type) { + switch (type) { + case TSDataType::BOOLEAN: + return 0; + case TSDataType::INT32: + return 1; + case TSDataType::INT64: + return 2; + case TSDataType::FLOAT: + return 3; + case TSDataType::DOUBLE: + return 4; + case TSDataType::TEXT: + return 5; + case TSDataType::TIMESTAMP: + return 8; + case TSDataType::DATE: + return 9; + case TSDataType::BLOB: + return 10; + case TSDataType::STRING: + return 11; + case TSDataType::OBJECT: + return 12; + default: + return -1; + } +} + +string Session::Impl::getVersionString(Version::Version version) { + switch (version) { + case Version::V_0_12: + return "V_0_12"; + case Version::V_0_13: + return "V_0_13"; + case Version::V_1_0: + return "V_1_0"; + default: + return "V_0_12"; + } +} + +void Session::Impl::initZoneId() { + if (!zoneId_.empty()) { + return; + } + + time_t ts = 0; + struct tm tmv; +#if defined(_WIN64) || defined(WIN32) || defined(_WIN32) + localtime_s(&tmv, &ts); +#else + localtime_r(&ts, &tmv); +#endif + + char zoneStr[32]; + strftime(zoneStr, sizeof(zoneStr), "%z", &tmv); + zoneId_ = zoneStr; +} + +void Session::Impl::initNodesSupplier( + const std::vector &nodeUrls) { + std::vector endPoints; + std::unordered_set uniqueEndpoints; + + if (nodeUrls.empty() && host_.empty()) { + throw IoTDBException("No available nodes"); + } + + // Process provided node URLs + if (!nodeUrls.empty()) { + for (auto &url : nodeUrls) { + try { + TEndPoint endPoint = UrlUtils::parseTEndPointIpv4AndIpv6Url(url); + if (endPoint.port == 0) + continue; // Skip invalid endpoints + + std::string endpointKey = + endPoint.ip + ":" + std::to_string(endPoint.port); + if (uniqueEndpoints.find(endpointKey) == uniqueEndpoints.end()) { + endPoints.emplace_back(std::move(endPoint)); + uniqueEndpoints.insert(std::move(endpointKey)); + } + } catch (...) { + continue; // Skip malformed URLs + } + } + } + + // Fallback to local endpoint if no valid endpoints found + if (endPoints.empty()) { + if (host_.empty() || rpcPort_ == 0) { + throw IoTDBException("No valid endpoints available"); + } + TEndPoint endPoint; + endPoint.__set_ip(host_); + endPoint.__set_port(rpcPort_); + endPoints.emplace_back(std::move(endPoint)); + } + + if (enableAutoFetch_) { + nodesSupplier_ = NodesSupplier::create(endPoints, username_, password_, + useSSL_, trustCertFilePath_); + } else { + nodesSupplier_ = make_shared(endPoints); + } +} + +void Session::Impl::initDefaultSessionConnection() { + // Try all endpoints from supplier until a connection is established. + auto endpoints = nodesSupplier_->getEndPointList(); + bool connected = false; + + for (const auto &endpoint : endpoints) { + try { + host_ = endpoint.ip; + rpcPort_ = endpoint.port; + + defaultEndPoint_.__set_ip(host_); + defaultEndPoint_.__set_port(rpcPort_); + + defaultSessionConnection_ = std::make_shared( + this, defaultEndPoint_, zoneId_, nodesSupplier_, fetchSize_, 3, 500, + connectTimeoutMs_, sqlDialect_, database_); + + connected = true; + break; + } catch (const IoTDBException &e) { + log_debug(e.what()); + throw; + } catch (const std::exception &e) { + log_warn(e.what()); + } + } + + if (!connected) { + throw std::runtime_error( + "No available node to establish SessionConnection."); + } +} + +void Session::Impl::insertStringRecordsWithLeaderCache( + vector deviceIds, vector times, + vector> measurementsList, vector> valuesList, + bool isAligned) { + std::unordered_map, + TSInsertStringRecordsReq> + recordsGroup; + for (int i = 0; i < deviceIds.size(); i++) { + auto connection = getSessionConnection(deviceIds[i]); + if (recordsGroup.find(connection) == recordsGroup.end()) { + TSInsertStringRecordsReq request; + std::vector emptyPrefixPaths; + std::vector> emptyMeasurementsList; + vector> emptyValuesList; + std::vector emptyTimestamps; + request.__set_isAligned(isAligned); + request.__set_prefixPaths(emptyPrefixPaths); + request.__set_timestamps(emptyTimestamps); + request.__set_measurementsList(emptyMeasurementsList); + request.__set_valuesList(emptyValuesList); + recordsGroup.insert(make_pair(connection, request)); + } + TSInsertStringRecordsReq &existingReq = recordsGroup[connection]; + existingReq.prefixPaths.emplace_back(deviceIds[i]); + existingReq.timestamps.emplace_back(times[i]); + existingReq.measurementsList.emplace_back(measurementsList[i]); + existingReq.valuesList.emplace_back(valuesList[i]); + } + std::function, + const TSInsertStringRecordsReq &)> + consumer = + [](const std::shared_ptr &c, + const TSInsertStringRecordsReq &r) { c->insertStringRecords(r); }; + if (recordsGroup.size() == 1) { + insertOnce(recordsGroup, consumer); + } else { + insertByGroup(recordsGroup, consumer); + } +} + +void Session::Impl::insertRecordsWithLeaderCache( + vector deviceIds, vector times, + vector> measurementsList, + const vector> &typesList, + vector> valuesList, bool isAligned) { + std::unordered_map, TSInsertRecordsReq> + recordsGroup; + for (int i = 0; i < deviceIds.size(); i++) { + auto connection = getSessionConnection(deviceIds[i]); + if (recordsGroup.find(connection) == recordsGroup.end()) { + TSInsertRecordsReq request; + std::vector emptyPrefixPaths; + std::vector> emptyMeasurementsList; + std::vector emptyValuesList; + std::vector emptyTimestamps; + request.__set_isAligned(isAligned); + request.__set_prefixPaths(emptyPrefixPaths); + request.__set_timestamps(emptyTimestamps); + request.__set_measurementsList(emptyMeasurementsList); + request.__set_valuesList(emptyValuesList); + recordsGroup.insert(make_pair(connection, request)); + } + TSInsertRecordsReq &existingReq = recordsGroup[connection]; + existingReq.prefixPaths.emplace_back(deviceIds[i]); + existingReq.timestamps.emplace_back(times[i]); + existingReq.measurementsList.emplace_back(measurementsList[i]); + vector bufferList; + string buffer; + putValuesIntoBuffer(typesList[i], valuesList[i], buffer); + existingReq.valuesList.emplace_back(buffer); + recordsGroup[connection] = existingReq; + } + std::function, + const TSInsertRecordsReq &)> + consumer = [](const std::shared_ptr &c, + const TSInsertRecordsReq &r) { c->insertRecords(r); }; + if (recordsGroup.size() == 1) { + insertOnce(recordsGroup, consumer); + } else { + insertByGroup(recordsGroup, consumer); + } +} + +void Session::Impl::insertTabletsWithLeaderCache( + unordered_map &tablets, bool sorted, bool isAligned) { + std::unordered_map, TSInsertTabletsReq> + tabletsGroup; + if (tablets.empty()) { + throw BatchExecutionException("No tablet is inserting!"); + } + for (const auto &item : tablets) { + if (isAligned != item.second->isAligned) { + throw BatchExecutionException( + "The tablets should be all aligned or non-aligned!"); + } + if (!checkSorted(*(item.second))) { + sortTablet(*(item.second)); + } + auto deviceId = item.first; + auto tablet = item.second; + auto connection = getSessionConnection(deviceId); + auto it = tabletsGroup.find(connection); + if (it == tabletsGroup.end()) { + TSInsertTabletsReq request; + tabletsGroup[connection] = request; + } + TSInsertTabletsReq &existingReq = tabletsGroup[connection]; + existingReq.prefixPaths.emplace_back(tablet->deviceId); + existingReq.timestampsList.emplace_back( + move(SessionUtils::getTime(*tablet))); + existingReq.valuesList.emplace_back(move(SessionUtils::getValue(*tablet))); + existingReq.sizeList.emplace_back(tablet->rowSize); + vector dataTypes; + vector measurements; + for (pair schema : tablet->schemas) { + measurements.push_back(schema.first); + dataTypes.push_back(schema.second); + } + existingReq.measurementsList.emplace_back(measurements); + existingReq.typesList.emplace_back(dataTypes); + } + + std::function, + const TSInsertTabletsReq &)> + consumer = [](const std::shared_ptr &c, + const TSInsertTabletsReq &r) { c->insertTablets(r); }; + if (tabletsGroup.size() == 1) { + insertOnce(tabletsGroup, consumer); + } else { + insertByGroup(tabletsGroup, consumer); + } +} + +void Session::open() { open(false, Impl::DEFAULT_TIMEOUT_MS); } + +void Session::open(bool enableRPCCompression) { + open(enableRPCCompression, Impl::DEFAULT_TIMEOUT_MS); +} + +void Session::open(bool enableRPCCompression, int connectionTimeoutInMs) { + if (!impl_->isClosed_) { + return; + } + + try { + impl_->initDefaultSessionConnection(); + } catch (const exception &e) { + log_debug(e.what()); + throw IoTDBException(e.what()); + } + impl_->zoneId_ = impl_->defaultSessionConnection_->zoneId; + + if (impl_->enableRedirection_) { + impl_->endPointToSessionConnection.insert( + make_pair(impl_->defaultEndPoint_, impl_->defaultSessionConnection_)); + } + + impl_->isClosed_ = false; +} + +void Session::close() { + if (impl_->isClosed_) { + return; + } + impl_->isClosed_ = true; +} + +void Session::insertRecord(const string &deviceId, int64_t time, + const vector &measurements, + const vector &values) { + TSInsertStringRecordReq req; + req.__set_prefixPath(deviceId); + req.__set_timestamp(time); + req.__set_measurements(measurements); + req.__set_values(values); + req.__set_isAligned(false); + try { + impl_->getSessionConnection(deviceId)->insertStringRecord(req); + } catch (RedirectException &e) { + impl_->handleRedirection(deviceId, endpointToThrift(e.endPoint)); + } catch (const IoTDBConnectionException &e) { + if (impl_->enableRedirection_ && impl_->deviceIdToEndpoint.find(deviceId) != + impl_->deviceIdToEndpoint.end()) { + impl_->deviceIdToEndpoint.erase(deviceId); + try { + impl_->defaultSessionConnection_->insertStringRecord(req); + } catch (RedirectException &e) { + } + } else { + throw; + } + } +} + +void Session::insertRecord(const string &deviceId, int64_t time, + const vector &measurements, + const vector &types, + const vector &values) { + TSInsertRecordReq req; + req.__set_prefixPath(deviceId); + req.__set_timestamp(time); + req.__set_measurements(measurements); + string buffer; + impl_->putValuesIntoBuffer(types, values, buffer); + req.__set_values(buffer); + req.__set_isAligned(false); + try { + impl_->getSessionConnection(deviceId)->insertRecord(req); + } catch (RedirectException &e) { + impl_->handleRedirection(deviceId, endpointToThrift(e.endPoint)); + } catch (const IoTDBConnectionException &e) { + if (impl_->enableRedirection_ && impl_->deviceIdToEndpoint.find(deviceId) != + impl_->deviceIdToEndpoint.end()) { + impl_->deviceIdToEndpoint.erase(deviceId); + try { + impl_->defaultSessionConnection_->insertRecord(req); + } catch (RedirectException &e) { + } + } else { + throw; + } + } +} + +void Session::insertAlignedRecord(const string &deviceId, int64_t time, + const vector &measurements, + const vector &values) { + TSInsertStringRecordReq req; + req.__set_prefixPath(deviceId); + req.__set_timestamp(time); + req.__set_measurements(measurements); + req.__set_values(values); + req.__set_isAligned(true); + try { + impl_->getSessionConnection(deviceId)->insertStringRecord(req); + } catch (RedirectException &e) { + impl_->handleRedirection(deviceId, endpointToThrift(e.endPoint)); + } catch (const IoTDBConnectionException &e) { + if (impl_->enableRedirection_ && impl_->deviceIdToEndpoint.find(deviceId) != + impl_->deviceIdToEndpoint.end()) { + impl_->deviceIdToEndpoint.erase(deviceId); + try { + impl_->defaultSessionConnection_->insertStringRecord(req); + } catch (RedirectException &e) { + } + } else { + throw; + } + } +} + +void Session::insertAlignedRecord(const string &deviceId, int64_t time, + const vector &measurements, + const vector &types, + const vector &values) { + TSInsertRecordReq req; + req.__set_prefixPath(deviceId); + req.__set_timestamp(time); + req.__set_measurements(measurements); + string buffer; + impl_->putValuesIntoBuffer(types, values, buffer); + req.__set_values(buffer); + req.__set_isAligned(false); + try { + impl_->getSessionConnection(deviceId)->insertRecord(req); + } catch (RedirectException &e) { + impl_->handleRedirection(deviceId, endpointToThrift(e.endPoint)); + } catch (const IoTDBConnectionException &e) { + if (impl_->enableRedirection_ && impl_->deviceIdToEndpoint.find(deviceId) != + impl_->deviceIdToEndpoint.end()) { + impl_->deviceIdToEndpoint.erase(deviceId); + try { + impl_->defaultSessionConnection_->insertRecord(req); + } catch (RedirectException &e) { + } + } else { + throw; + } + } +} + +void Session::insertRecords(const vector &deviceIds, + const vector ×, + const vector> &measurementsList, + const vector> &valuesList) { + size_t len = deviceIds.size(); + if (len != times.size() || len != measurementsList.size() || + len != valuesList.size()) { + logic_error e("deviceIds, times, measurementsList and valuesList's size " + "should be equal"); + throw exception(e); + } + + if (impl_->enableRedirection_) { + impl_->insertStringRecordsWithLeaderCache( + deviceIds, times, measurementsList, valuesList, false); + } else { + TSInsertStringRecordsReq request; + request.__set_prefixPaths(deviceIds); + request.__set_timestamps(times); + request.__set_measurementsList(measurementsList); + request.__set_valuesList(valuesList); + request.__set_isAligned(false); + try { + impl_->defaultSessionConnection_->insertStringRecords(request); + } catch (RedirectException &e) { + } + } +} + +void Session::insertRecords( + const vector &deviceIds, const vector ×, + const vector> &measurementsList, + const vector> &typesList, + const vector> &valuesList) { + size_t len = deviceIds.size(); + if (len != times.size() || len != measurementsList.size() || + len != valuesList.size()) { + logic_error e("deviceIds, times, measurementsList and valuesList's size " + "should be equal"); + throw exception(e); + } + + if (impl_->enableRedirection_) { + impl_->insertRecordsWithLeaderCache(deviceIds, times, measurementsList, + typesList, valuesList, false); + } else { + TSInsertRecordsReq request; + request.__set_prefixPaths(deviceIds); + request.__set_timestamps(times); + request.__set_measurementsList(measurementsList); + vector bufferList; + for (size_t i = 0; i < valuesList.size(); i++) { + string buffer; + impl_->putValuesIntoBuffer(typesList[i], valuesList[i], buffer); + bufferList.push_back(buffer); + } + request.__set_valuesList(bufferList); + request.__set_isAligned(false); + try { + impl_->defaultSessionConnection_->insertRecords(request); + } catch (RedirectException &e) { + } + } +} + +void Session::insertAlignedRecords( + const vector &deviceIds, const vector ×, + const vector> &measurementsList, + const vector> &valuesList) { + size_t len = deviceIds.size(); + if (len != times.size() || len != measurementsList.size() || + len != valuesList.size()) { + logic_error e("deviceIds, times, measurementsList and valuesList's size " + "should be equal"); + throw exception(e); + } + + if (impl_->enableRedirection_) { + impl_->insertStringRecordsWithLeaderCache( + deviceIds, times, measurementsList, valuesList, true); + } else { + TSInsertStringRecordsReq request; + request.__set_prefixPaths(deviceIds); + request.__set_timestamps(times); + request.__set_measurementsList(measurementsList); + request.__set_valuesList(valuesList); + request.__set_isAligned(true); + try { + impl_->defaultSessionConnection_->insertStringRecords(request); + } catch (RedirectException &e) { + } + } +} + +void Session::insertAlignedRecords( + const vector &deviceIds, const vector ×, + const vector> &measurementsList, + const vector> &typesList, + const vector> &valuesList) { + size_t len = deviceIds.size(); + if (len != times.size() || len != measurementsList.size() || + len != valuesList.size()) { + logic_error e("deviceIds, times, measurementsList and valuesList's size " + "should be equal"); + throw exception(e); + } + + if (impl_->enableRedirection_) { + impl_->insertRecordsWithLeaderCache(deviceIds, times, measurementsList, + typesList, valuesList, true); + } else { + TSInsertRecordsReq request; + request.__set_prefixPaths(deviceIds); + request.__set_timestamps(times); + request.__set_measurementsList(measurementsList); + vector bufferList; + for (size_t i = 0; i < valuesList.size(); i++) { + string buffer; + impl_->putValuesIntoBuffer(typesList[i], valuesList[i], buffer); + bufferList.push_back(buffer); + } + request.__set_valuesList(bufferList); + request.__set_isAligned(false); + try { + impl_->defaultSessionConnection_->insertRecords(request); + } catch (RedirectException &e) { + } + } +} + +void Session::insertRecordsOfOneDevice( + const string &deviceId, vector ×, + vector> &measurementsList, + vector> &typesList, + vector> &valuesList) { + insertRecordsOfOneDevice(deviceId, times, measurementsList, typesList, + valuesList, false); +} + +void Session::insertRecordsOfOneDevice( + const string &deviceId, vector ×, + vector> &measurementsList, + vector> &typesList, + vector> &valuesList, bool sorted) { + if (!impl_->checkSorted(times)) { + int *index = new int[times.size()]; + for (size_t i = 0; i < times.size(); i++) { + index[i] = (int)i; + } + + impl_->sortIndexByTimestamp(index, times, (int)(times.size())); + times = sortList(times, index, (int)(times.size())); + measurementsList = sortList(measurementsList, index, (int)(times.size())); + typesList = sortList(typesList, index, (int)(times.size())); + valuesList = sortList(valuesList, index, (int)(times.size())); + delete[] index; + } + TSInsertRecordsOfOneDeviceReq request; + request.__set_prefixPath(deviceId); + request.__set_timestamps(times); + request.__set_measurementsList(measurementsList); + vector bufferList; + for (size_t i = 0; i < valuesList.size(); i++) { + string buffer; + impl_->putValuesIntoBuffer(typesList[i], valuesList[i], buffer); + bufferList.push_back(buffer); + } + request.__set_valuesList(bufferList); + request.__set_isAligned(false); + TSStatus respStatus; + try { + impl_->getSessionConnection(deviceId)->insertRecordsOfOneDevice(request); + } catch (RedirectException &e) { + impl_->handleRedirection(deviceId, endpointToThrift(e.endPoint)); + } catch (const IoTDBConnectionException &e) { + if (impl_->enableRedirection_ && impl_->deviceIdToEndpoint.find(deviceId) != + impl_->deviceIdToEndpoint.end()) { + impl_->deviceIdToEndpoint.erase(deviceId); + try { + impl_->defaultSessionConnection_->insertRecordsOfOneDevice(request); + } catch (RedirectException &e) { + } + } else { + throw; + } + } +} + +void Session::insertAlignedRecordsOfOneDevice( + const string &deviceId, vector ×, + vector> &measurementsList, + vector> &typesList, + vector> &valuesList) { + insertAlignedRecordsOfOneDevice(deviceId, times, measurementsList, typesList, + valuesList, false); +} + +void Session::insertAlignedRecordsOfOneDevice( + const string &deviceId, vector ×, + vector> &measurementsList, + vector> &typesList, + vector> &valuesList, bool sorted) { + if (!impl_->checkSorted(times)) { + int *index = new int[times.size()]; + for (size_t i = 0; i < times.size(); i++) { + index[i] = (int)i; + } + + impl_->sortIndexByTimestamp(index, times, (int)(times.size())); + times = sortList(times, index, (int)(times.size())); + measurementsList = sortList(measurementsList, index, (int)(times.size())); + typesList = sortList(typesList, index, (int)(times.size())); + valuesList = sortList(valuesList, index, (int)(times.size())); + delete[] index; + } + TSInsertRecordsOfOneDeviceReq request; + request.__set_prefixPath(deviceId); + request.__set_timestamps(times); + request.__set_measurementsList(measurementsList); + vector bufferList; + for (size_t i = 0; i < valuesList.size(); i++) { + string buffer; + impl_->putValuesIntoBuffer(typesList[i], valuesList[i], buffer); + bufferList.push_back(buffer); + } + request.__set_valuesList(bufferList); + request.__set_isAligned(true); + TSStatus respStatus; + try { + impl_->getSessionConnection(deviceId)->insertRecordsOfOneDevice(request); + } catch (RedirectException &e) { + impl_->handleRedirection(deviceId, endpointToThrift(e.endPoint)); + } catch (const IoTDBConnectionException &e) { + if (impl_->enableRedirection_ && impl_->deviceIdToEndpoint.find(deviceId) != + impl_->deviceIdToEndpoint.end()) { + impl_->deviceIdToEndpoint.erase(deviceId); + try { + impl_->defaultSessionConnection_->insertRecordsOfOneDevice(request); + } catch (RedirectException &e) { + } + } else { + throw; + } + } +} + +void Session::insertTablet(Tablet &tablet) { + try { + insertTablet(tablet, false); + } catch (const exception &e) { + log_debug(e.what()); + logic_error error(e.what()); + throw exception(error); + } +} + +void Session::Impl::buildInsertTabletReq(TSInsertTabletReq &request, + Tablet &tablet, bool sorted) { + if ((!sorted) && !checkSorted(tablet)) { + sortTablet(tablet); + } + + request.__set_prefixPath(tablet.deviceId); + + std::vector reqMeasurements; + reqMeasurements.reserve(tablet.schemas.size()); + std::vector types; + types.reserve(tablet.schemas.size()); + for (pair schema : tablet.schemas) { + reqMeasurements.push_back(schema.first); + types.push_back(schema.second); + } + request.__set_measurements(reqMeasurements); + request.__set_types(types); + request.__set_values(SessionUtils::getValue(tablet)); + request.__set_timestamps(SessionUtils::getTime(tablet)); + request.__set_size(tablet.rowSize); + request.__set_isAligned(tablet.isAligned); +} + +void Session::Impl::insertTablet(TSInsertTabletReq request) { + auto deviceId = request.prefixPath; + try { + getSessionConnection(deviceId)->insertTablet(request); + } catch (RedirectException &e) { + handleRedirection(deviceId, endpointToThrift(e.endPoint)); + } catch (const IoTDBConnectionException &e) { + if (enableRedirection_ && + deviceIdToEndpoint.find(deviceId) != deviceIdToEndpoint.end()) { + deviceIdToEndpoint.erase(deviceId); + try { + defaultSessionConnection_->insertTablet(request); + } catch (RedirectException &e) { + } + } else { + throw; + } + } +} + +void Session::insertTablet(Tablet &tablet, bool sorted) { + TSInsertTabletReq request; + impl_->buildInsertTabletReq(request, tablet, sorted); + impl_->insertTablet(request); +} + +void Session::insertAlignedTablet(Tablet &tablet) { + insertAlignedTablet(tablet, false); +} + +void Session::insertAlignedTablet(Tablet &tablet, bool sorted) { + tablet.setAligned(true); + try { + insertTablet(tablet, sorted); + } catch (const exception &e) { + log_debug(e.what()); + logic_error error(e.what()); + throw exception(error); + } +} + +void Session::insertTablets(unordered_map &tablets) { + try { + insertTablets(tablets, false); + } catch (const exception &e) { + log_debug(e.what()); + logic_error error(e.what()); + throw exception(error); + } +} + +void Session::insertTablets(unordered_map &tablets, + bool sorted) { + if (tablets.empty()) { + throw BatchExecutionException("No tablet is inserting!"); + } + auto beginIter = tablets.begin(); + bool isAligned = ((*beginIter).second)->isAligned; + if (impl_->enableRedirection_) { + impl_->insertTabletsWithLeaderCache(tablets, sorted, isAligned); + } else { + TSInsertTabletsReq request; + for (const auto &item : tablets) { + if (isAligned != item.second->isAligned) { + throw BatchExecutionException( + "The tablets should be all aligned or non-aligned!"); + } + if (!impl_->checkSorted(*(item.second))) { + impl_->sortTablet(*(item.second)); + } + request.prefixPaths.push_back(item.second->deviceId); + vector measurements; + vector dataTypes; + for (pair schema : item.second->schemas) { + measurements.push_back(schema.first); + dataTypes.push_back(schema.second); + } + request.measurementsList.push_back(measurements); + request.typesList.push_back(dataTypes); + request.timestampsList.push_back( + move(SessionUtils::getTime(*(item.second)))); + request.valuesList.push_back( + move(SessionUtils::getValue(*(item.second)))); + request.sizeList.push_back(item.second->rowSize); + } + request.__set_isAligned(isAligned); + try { + TSStatus respStatus; + impl_->defaultSessionConnection_->insertTablets(request); + RpcUtils::verifySuccess(respStatus); + } catch (RedirectException &e) { + } + } +} + +void Session::insertAlignedTablets(unordered_map &tablets, + bool sorted) { + for (auto iter = tablets.begin(); iter != tablets.end(); iter++) { + iter->second->setAligned(true); + } + try { + insertTablets(tablets, sorted); + } catch (const exception &e) { + log_debug(e.what()); + logic_error error(e.what()); + throw exception(error); + } +} + +void Session::testInsertRecord(const string &deviceId, int64_t time, + const vector &measurements, + const vector &values) { + TSInsertStringRecordReq req; + req.__set_prefixPath(deviceId); + req.__set_timestamp(time); + req.__set_measurements(measurements); + req.__set_values(values); + TSStatus tsStatus; + try { + impl_->defaultSessionConnection_->testInsertStringRecord(req); + RpcUtils::verifySuccess(tsStatus); + } catch (const TTransportException &e) { + log_debug(e.what()); + throw IoTDBConnectionException(e.what()); + } catch (const IoTDBException &e) { + log_debug(e.what()); + throw; + } catch (const exception &e) { + log_debug(e.what()); + throw IoTDBException(e.what()); + } +} + +void Session::testInsertTablet(const Tablet &tablet) { + TSInsertTabletReq request; + request.prefixPath = tablet.deviceId; + for (pair schema : tablet.schemas) { + request.measurements.push_back(schema.first); + request.types.push_back(schema.second); + } + request.__set_timestamps(move(SessionUtils::getTime(tablet))); + request.__set_values(move(SessionUtils::getValue(tablet))); + request.__set_size(tablet.rowSize); + try { + TSStatus tsStatus; + impl_->defaultSessionConnection_->testInsertTablet(request); + RpcUtils::verifySuccess(tsStatus); + } catch (const TTransportException &e) { + log_debug(e.what()); + throw IoTDBConnectionException(e.what()); + } catch (const IoTDBException &e) { + log_debug(e.what()); + throw; + } catch (const exception &e) { + log_debug(e.what()); + throw IoTDBException(e.what()); + } +} + +void Session::testInsertRecords(const vector &deviceIds, + const vector ×, + const vector> &measurementsList, + const vector> &valuesList) { + size_t len = deviceIds.size(); + if (len != times.size() || len != measurementsList.size() || + len != valuesList.size()) { + logic_error error("deviceIds, times, measurementsList and valuesList's " + "size should be equal"); + throw exception(error); + } + TSInsertStringRecordsReq request; + request.__set_prefixPaths(deviceIds); + request.__set_timestamps(times); + request.__set_measurementsList(measurementsList); + request.__set_valuesList(valuesList); + + try { + TSStatus tsStatus; + impl_->defaultSessionConnection_->getSessionClient()->insertStringRecords( + tsStatus, request); + RpcUtils::verifySuccess(tsStatus); + } catch (const TTransportException &e) { + log_debug(e.what()); + throw IoTDBConnectionException(e.what()); + } catch (const IoTDBException &e) { + log_debug(e.what()); + throw; + } catch (const exception &e) { + log_debug(e.what()); + throw IoTDBException(e.what()); + } +} + +void Session::deleteTimeseries(const string &path) { + vector paths; + paths.push_back(path); + deleteTimeseries(paths); +} + +void Session::deleteTimeseries(const vector &paths) { + impl_->defaultSessionConnection_->deleteTimeseries(paths); +} + +void Session::deleteData(const string &path, int64_t endTime) { + vector paths; + paths.push_back(path); + deleteData(paths, LONG_LONG_MIN, endTime); +} + +void Session::deleteData(const vector &paths, int64_t endTime) { + deleteData(paths, LONG_LONG_MIN, endTime); +} + +void Session::deleteData(const vector &paths, int64_t startTime, + int64_t endTime) { + TSDeleteDataReq req; + req.__set_paths(paths); + req.__set_startTime(startTime); + req.__set_endTime(endTime); + impl_->defaultSessionConnection_->deleteData(req); +} + +void Session::setStorageGroup(const string &storageGroupId) { + impl_->defaultSessionConnection_->setStorageGroup(storageGroupId); +} + +void Session::deleteStorageGroup(const string &storageGroup) { + vector storageGroups; + storageGroups.push_back(storageGroup); + deleteStorageGroups(storageGroups); +} + +void Session::deleteStorageGroups(const vector &storageGroups) { + impl_->defaultSessionConnection_->deleteStorageGroups(storageGroups); +} + +void Session::createDatabase(const string &database) { + this->setStorageGroup(database); +} + +void Session::deleteDatabase(const string &database) { + this->deleteStorageGroups(vector{database}); +} + +void Session::deleteDatabases(const vector &databases) { + this->deleteStorageGroups(databases); +} + +void Session::createTimeseries(const string &path, + TSDataType::TSDataType dataType, + TSEncoding::TSEncoding encoding, + CompressionType::CompressionType compressor) { + try { + createTimeseries(path, dataType, encoding, compressor, nullptr, nullptr, + nullptr, ""); + } catch (const exception &e) { + log_debug(e.what()); + throw IoTDBException(e.what()); + } +} + +void Session::createTimeseries(const string &path, + TSDataType::TSDataType dataType, + TSEncoding::TSEncoding encoding, + CompressionType::CompressionType compressor, + map *props, + map *tags, + map *attributes, + const string &measurementAlias) { + TSCreateTimeseriesReq req; + req.__set_path(path); + req.__set_dataType(dataType); + req.__set_encoding(encoding); + req.__set_compressor(compressor); + if (props != nullptr) { + req.__set_props(*props); + } + + if (tags != nullptr) { + req.__set_tags(*tags); + } + if (attributes != nullptr) { + req.__set_attributes(*attributes); + } + if (!measurementAlias.empty()) { + req.__set_measurementAlias(measurementAlias); + } + impl_->defaultSessionConnection_->createTimeseries(req); +} + +void Session::createMultiTimeseries( + const vector &paths, + const vector &dataTypes, + const vector &encodings, + const vector &compressors, + vector> *propsList, + vector> *tagsList, + vector> *attributesList, + vector *measurementAliasList) { + TSCreateMultiTimeseriesReq request; + request.__set_paths(paths); + + vector dataTypesOrdinal; + dataTypesOrdinal.reserve(dataTypes.size()); + for (TSDataType::TSDataType dataType : dataTypes) { + dataTypesOrdinal.push_back(dataType); + } + request.__set_dataTypes(dataTypesOrdinal); + + vector encodingsOrdinal; + encodingsOrdinal.reserve(encodings.size()); + for (TSEncoding::TSEncoding encoding : encodings) { + encodingsOrdinal.push_back(encoding); + } + request.__set_encodings(encodingsOrdinal); + + vector compressorsOrdinal; + compressorsOrdinal.reserve(compressors.size()); + for (CompressionType::CompressionType compressor : compressors) { + compressorsOrdinal.push_back(compressor); + } + request.__set_compressors(compressorsOrdinal); + + if (propsList != nullptr) { + request.__set_propsList(*propsList); + } + + if (tagsList != nullptr) { + request.__set_tagsList(*tagsList); + } + if (attributesList != nullptr) { + request.__set_attributesList(*attributesList); + } + if (measurementAliasList != nullptr) { + request.__set_measurementAliasList(*measurementAliasList); + } + + impl_->defaultSessionConnection_->createMultiTimeseries(request); +} + +void Session::createAlignedTimeseries( + const std::string &deviceId, const std::vector &measurements, + const std::vector &dataTypes, + const std::vector &encodings, + const std::vector &compressors) { + TSCreateAlignedTimeseriesReq request; + request.__set_prefixPath(deviceId); + request.__set_measurements(measurements); + + vector dataTypesOrdinal; + dataTypesOrdinal.reserve(dataTypes.size()); + for (TSDataType::TSDataType dataType : dataTypes) { + dataTypesOrdinal.push_back(dataType); + } + request.__set_dataTypes(dataTypesOrdinal); + + vector encodingsOrdinal; + encodingsOrdinal.reserve(encodings.size()); + for (TSEncoding::TSEncoding encoding : encodings) { + encodingsOrdinal.push_back(encoding); + } + request.__set_encodings(encodingsOrdinal); + + vector compressorsOrdinal; + compressorsOrdinal.reserve(compressors.size()); + for (CompressionType::CompressionType compressor : compressors) { + compressorsOrdinal.push_back(compressor); + } + request.__set_compressors(compressorsOrdinal); + + impl_->defaultSessionConnection_->createAlignedTimeseries(request); +} + +bool Session::checkTimeseriesExists(const string &path) { + try { + std::unique_ptr dataset = + executeQueryStatement("SHOW TIMESERIES " + path); + if (dataset == nullptr) { + throw IoTDBException("executeQueryStatement failed"); + } + bool isExisted = dataset->hasNext(); + dataset->closeOperationHandle(); + return isExisted; + } catch (const exception &e) { + log_debug(e.what()); + throw IoTDBException(e.what()); + } +} + +shared_ptr Session::Impl::getQuerySessionConnection() { + auto endPoint = nodesSupplier_->getQueryEndPoint(); + if (!endPoint.is_initialized() || endPointToSessionConnection.empty()) { + return defaultSessionConnection_; + } + + auto it = endPointToSessionConnection.find(endPoint.value()); + if (it != endPointToSessionConnection.end()) { + return it->second; + } + + shared_ptr newConnection; + try { + newConnection = make_shared( + this, endPoint.value(), zoneId_, nodesSupplier_, fetchSize_, 60, 500, + connectTimeoutMs_, sqlDialect_, database_); + endPointToSessionConnection.emplace(endPoint.value(), newConnection); + return newConnection; + } catch (exception &e) { + log_debug("Session::Impl::getQuerySessionConnection() exception: " + + e.what()); + return newConnection; + } +} + +shared_ptr +Session::Impl::getSessionConnection(std::string deviceId) { + if (!enableRedirection_ || + deviceIdToEndpoint.find(deviceId) == deviceIdToEndpoint.end() || + endPointToSessionConnection.find(deviceIdToEndpoint[deviceId]) == + endPointToSessionConnection.end()) { + return defaultSessionConnection_; + } + return endPointToSessionConnection.find(deviceIdToEndpoint[deviceId])->second; +} + +string Session::getTimeZone() { + auto ret = impl_->defaultSessionConnection_->getTimeZone(); + return ret.timeZone; +} + +void Session::setTimeZone(const string &zoneId) { + TSSetTimeZoneReq req; + req.__set_sessionId(impl_->defaultSessionConnection_->sessionId); + req.__set_timeZone(zoneId); + impl_->defaultSessionConnection_->setTimeZone(req); +} + +unique_ptr Session::executeQueryStatement(const string &sql) { + return executeQueryStatementMayRedirect(sql, QUERY_TIMEOUT_MS); +} + +unique_ptr Session::executeQueryStatement(const string &sql, + int64_t timeoutInMs) { + return executeQueryStatementMayRedirect(sql, timeoutInMs); +} + +void Session::Impl::handleQueryRedirection(TEndPoint endPoint) { + if (!enableRedirection_) + return; + shared_ptr newConnection; + auto it = endPointToSessionConnection.find(endPoint); + if (it != endPointToSessionConnection.end()) { + newConnection = it->second; + } else { + try { + newConnection = make_shared( + this, endPoint, zoneId_, nodesSupplier_, fetchSize_, 60, 500, + connectTimeoutMs_, sqlDialect_, database_); + + endPointToSessionConnection.emplace(endPoint, newConnection); + } catch (exception &e) { + throw IoTDBConnectionException(e.what()); + } + } + defaultSessionConnection_ = newConnection; +} + +void Session::Impl::handleRedirection(const std::string &deviceId, + TEndPoint endPoint) { + if (!enableRedirection_) + return; + if (endPoint.ip == "0.0.0.0") + return; + deviceIdToEndpoint[deviceId] = endPoint; + + shared_ptr newConnection; + auto it = endPointToSessionConnection.find(endPoint); + if (it != endPointToSessionConnection.end()) { + newConnection = it->second; + } else { + try { + newConnection = make_shared( + this, endPoint, zoneId_, nodesSupplier_, fetchSize_, 60, 500, 1000, + sqlDialect_, database_); + endPointToSessionConnection.emplace(endPoint, newConnection); + } catch (exception &e) { + deviceIdToEndpoint.erase(deviceId); + throw IoTDBConnectionException(e.what()); + } + } +} + +std::unique_ptr +Session::executeQueryStatementMayRedirect(const std::string &sql, + int64_t timeoutInMs) { + auto sessionConnection = impl_->getQuerySessionConnection(); + if (!sessionConnection) { + log_warn("Session connection not found"); + return nullptr; + } + try { + return sessionConnection->executeQueryStatement(sql, timeoutInMs); + } catch (RedirectException &e) { + log_warn("Session connection redirect exception: " + e.what()); + impl_->handleQueryRedirection(endpointToThrift(e.endPoint)); + try { + return impl_->defaultSessionConnection_->executeQueryStatement( + sql, timeoutInMs); + } catch (exception &e) { + log_error("Exception while executing redirected query statement: %s", + e.what()); + throw ExecutionException(e.what()); + } + } catch (exception &e) { + log_error("Exception while executing query statement: %s", e.what()); + throw; + } +} + +void Session::executeNonQueryStatement(const string &sql) { + try { + impl_->defaultSessionConnection_->executeNonQueryStatement(sql); + } catch (const exception &e) { + throw IoTDBException(e.what()); + } +} + +unique_ptr +Session::executeRawDataQuery(const vector &paths, int64_t startTime, + int64_t endTime) { + return impl_->defaultSessionConnection_->executeRawDataQuery(paths, startTime, + endTime); +} + +unique_ptr +Session::executeLastDataQuery(const vector &paths) { + return executeLastDataQuery(paths, LONG_LONG_MIN); +} + +unique_ptr +Session::executeLastDataQuery(const vector &paths, int64_t lastTime) { + return impl_->defaultSessionConnection_->executeLastDataQuery(paths, + lastTime); +} + +void Session::createSchemaTemplate(const Template &templ) { + TSCreateSchemaTemplateReq req; + req.__set_name(templ.getName()); + req.__set_serializedTemplate(templ.serialize()); + impl_->defaultSessionConnection_->createSchemaTemplate(req); +} + +void Session::setSchemaTemplate(const string &template_name, + const string &prefix_path) { + TSSetSchemaTemplateReq req; + req.__set_templateName(template_name); + req.__set_prefixPath(prefix_path); + impl_->defaultSessionConnection_->setSchemaTemplate(req); +} + +void Session::unsetSchemaTemplate(const string &prefix_path, + const string &template_name) { + TSUnsetSchemaTemplateReq req; + req.__set_templateName(template_name); + req.__set_prefixPath(prefix_path); + impl_->defaultSessionConnection_->unsetSchemaTemplate(req); +} + +void Session::addAlignedMeasurementsInTemplate( + const string &template_name, const vector &measurements, + const vector &dataTypes, + const vector &encodings, + const vector &compressors) { + TSAppendSchemaTemplateReq req; + req.__set_name(template_name); + req.__set_measurements(measurements); + req.__set_isAligned(true); + + vector dataTypesOrdinal; + dataTypesOrdinal.reserve(dataTypes.size()); + for (TSDataType::TSDataType dataType : dataTypes) { + dataTypesOrdinal.push_back(dataType); + } + req.__set_dataTypes(dataTypesOrdinal); + + vector encodingsOrdinal; + encodingsOrdinal.reserve(encodings.size()); + for (TSEncoding::TSEncoding encoding : encodings) { + encodingsOrdinal.push_back(encoding); + } + req.__set_encodings(encodingsOrdinal); + + vector compressorsOrdinal; + compressorsOrdinal.reserve(compressors.size()); + for (CompressionType::CompressionType compressor : compressors) { + compressorsOrdinal.push_back(compressor); + } + req.__set_compressors(compressorsOrdinal); + + impl_->defaultSessionConnection_->appendSchemaTemplate(req); +} + +void Session::addAlignedMeasurementsInTemplate( + const string &template_name, const string &measurement, + TSDataType::TSDataType dataType, TSEncoding::TSEncoding encoding, + CompressionType::CompressionType compressor) { + vector measurements(1, measurement); + vector dataTypes(1, dataType); + vector encodings(1, encoding); + vector compressors(1, compressor); + addAlignedMeasurementsInTemplate(template_name, measurements, dataTypes, + encodings, compressors); +} + +void Session::addUnalignedMeasurementsInTemplate( + const string &template_name, const vector &measurements, + const vector &dataTypes, + const vector &encodings, + const vector &compressors) { + TSAppendSchemaTemplateReq req; + req.__set_name(template_name); + req.__set_measurements(measurements); + req.__set_isAligned(false); + + vector dataTypesOrdinal; + dataTypesOrdinal.reserve(dataTypes.size()); + for (TSDataType::TSDataType dataType : dataTypes) { + dataTypesOrdinal.push_back(dataType); + } + req.__set_dataTypes(dataTypesOrdinal); + + vector encodingsOrdinal; + encodingsOrdinal.reserve(encodings.size()); + for (TSEncoding::TSEncoding encoding : encodings) { + encodingsOrdinal.push_back(encoding); + } + req.__set_encodings(encodingsOrdinal); + + vector compressorsOrdinal; + compressorsOrdinal.reserve(compressors.size()); + for (CompressionType::CompressionType compressor : compressors) { + compressorsOrdinal.push_back(compressor); + } + req.__set_compressors(compressorsOrdinal); + + impl_->defaultSessionConnection_->appendSchemaTemplate(req); +} + +void Session::addUnalignedMeasurementsInTemplate( + const string &template_name, const string &measurement, + TSDataType::TSDataType dataType, TSEncoding::TSEncoding encoding, + CompressionType::CompressionType compressor) { + vector measurements(1, measurement); + vector dataTypes(1, dataType); + vector encodings(1, encoding); + vector compressors(1, compressor); + addUnalignedMeasurementsInTemplate(template_name, measurements, dataTypes, + encodings, compressors); +} + +void Session::deleteNodeInTemplate(const string &template_name, + const string &path) { + TSPruneSchemaTemplateReq req; + req.__set_name(template_name); + req.__set_path(path); + impl_->defaultSessionConnection_->pruneSchemaTemplate(req); +} + +int Session::countMeasurementsInTemplate(const string &template_name) { + TSQueryTemplateReq req; + req.__set_name(template_name); + req.__set_queryType(TemplateQueryType::COUNT_MEASUREMENTS); + TSQueryTemplateResp resp = + impl_->defaultSessionConnection_->querySchemaTemplate(req); + return resp.count; +} + +bool Session::isMeasurementInTemplate(const string &template_name, + const string &path) { + TSQueryTemplateReq req; + req.__set_name(template_name); + req.__set_measurement(path); + req.__set_queryType(TemplateQueryType::IS_MEASUREMENT); + TSQueryTemplateResp resp = + impl_->defaultSessionConnection_->querySchemaTemplate(req); + return resp.result; +} + +bool Session::isPathExistInTemplate(const string &template_name, + const string &path) { + TSQueryTemplateReq req; + req.__set_name(template_name); + req.__set_measurement(path); + req.__set_queryType(TemplateQueryType::PATH_EXIST); + TSQueryTemplateResp resp = + impl_->defaultSessionConnection_->querySchemaTemplate(req); + return resp.result; +} + +std::vector +Session::showMeasurementsInTemplate(const string &template_name) { + TSQueryTemplateReq req; + req.__set_name(template_name); + req.__set_measurement(""); + req.__set_queryType(TemplateQueryType::SHOW_MEASUREMENTS); + TSQueryTemplateResp resp = + impl_->defaultSessionConnection_->querySchemaTemplate(req); + return resp.measurements; +} + +std::vector +Session::showMeasurementsInTemplate(const string &template_name, + const string &pattern) { + TSQueryTemplateReq req; + req.__set_name(template_name); + req.__set_measurement(pattern); + req.__set_queryType(TemplateQueryType::SHOW_MEASUREMENTS); + TSQueryTemplateResp resp = + impl_->defaultSessionConnection_->querySchemaTemplate(req); + return resp.measurements; +} + +bool Session::checkTemplateExists(const string &template_name) { + try { + std::unique_ptr dataset = + executeQueryStatement("SHOW NODES IN DEVICE TEMPLATE " + template_name); + if (dataset == nullptr) { + throw IoTDBException("executeQueryStatement failed"); + } + bool isExisted = dataset->hasNext(); + dataset->closeOperationHandle(); + return isExisted; + } catch (const exception &e) { + if (strstr(e.what(), "does not exist") != NULL) { + return false; + } + log_debug(e.what()); + throw IoTDBException(e.what()); + } +} diff --git a/iotdb-client/client-cpp/src/session/SessionC.cpp b/iotdb-client/client-cpp/src/session/SessionC.cpp new file mode 100644 index 0000000000000..58eed24f2d2dd --- /dev/null +++ b/iotdb-client/client-cpp/src/session/SessionC.cpp @@ -0,0 +1,1410 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "SessionC.h" +#include "Session.h" +#include "SessionBuilder.h" +#include "SessionDataSet.h" + +#include +#include +#include +#include +#include +#include + +/* ============================================================ + * Internal wrapper structs — the opaque handles point to these + * ============================================================ */ + +struct CSession_ { + std::shared_ptr cpp; +}; + +struct CTablet_ { + Tablet cpp; +}; + +struct CSessionDataSet_ { + std::unique_ptr cpp; +}; + +struct CRowRecord_ { + std::shared_ptr cpp; +}; + +/* ============================================================ + * Thread-local error message buffer + * ============================================================ */ + +static thread_local std::string g_lastError; + +static void clearError() { g_lastError.clear(); } + +static TsStatus setError(TsStatus code, const std::string &msg) { + g_lastError = msg; + return code; +} + +static TsStatus setError(TsStatus code, const std::exception &e) { + g_lastError = e.what(); + return code; +} + +static TsStatus handleException(const std::exception &e) { +#if defined(_CPPRTTI) || defined(__GXX_RTTI) + // Try to classify exception type (requires RTTI enabled). + if (dynamic_cast(&e)) { + return setError(TS_ERR_CONNECTION, e); + } + if (dynamic_cast(&e) || + dynamic_cast(&e) || + dynamic_cast(&e)) { + return setError(TS_ERR_EXECUTION, e); + } +#endif + return setError(TS_ERR_UNKNOWN, e); +} + +extern "C" { + +const char *ts_get_last_error(void) { return g_lastError.c_str(); } + +} /* extern "C" */ + +/* ============================================================ + * Helpers — convert C arrays to C++ vectors + * ============================================================ */ + +static std::vector toStringVec(const char *const *arr, int count) { + std::vector v; + v.reserve(count); + for (int i = 0; i < count; i++) { + v.emplace_back(arr[i]); + } + return v; +} + +static std::vector toTypeVec(const TSDataType_C *arr, + int count) { + std::vector v; + v.reserve(count); + for (int i = 0; i < count; i++) { + v.push_back(static_cast(arr[i])); + } + return v; +} + +static std::vector +toEncodingVec(const TSEncoding_C *arr, int count) { + std::vector v; + v.reserve(count); + for (int i = 0; i < count; i++) { + v.push_back(static_cast(arr[i])); + } + return v; +} + +static std::vector +toCompressionVec(const TSCompressionType_C *arr, int count) { + std::vector v; + v.reserve(count); + for (int i = 0; i < count; i++) { + v.push_back(static_cast(arr[i])); + } + return v; +} + +static std::map +toStringMap(int count, const char *const *keys, const char *const *values) { + std::map m; + for (int i = 0; i < count; i++) { + m[keys[i]] = values[i]; + } + return m; +} + +/** + * Convert C typed values (void* const* values, TSDataType_C* types, int count) + * to C++ vector that Session expects. + * The caller must free the returned char* pointers using freeCharPtrVec(). + */ +static std::vector toCharPtrVec(const TSDataType_C *types, + const void *const *values, int count) { + std::vector result(count); + for (int i = 0; i < count; i++) { + switch (types[i]) { + case TS_TYPE_BOOLEAN: { + bool *p = new bool(*static_cast(values[i])); + result[i] = reinterpret_cast(p); + break; + } + case TS_TYPE_INT32: { + int32_t *p = new int32_t(*static_cast(values[i])); + result[i] = reinterpret_cast(p); + break; + } + case TS_TYPE_INT64: + case TS_TYPE_TIMESTAMP: { + int64_t *p = new int64_t(*static_cast(values[i])); + result[i] = reinterpret_cast(p); + break; + } + case TS_TYPE_FLOAT: { + float *p = new float(*static_cast(values[i])); + result[i] = reinterpret_cast(p); + break; + } + case TS_TYPE_DOUBLE: { + double *p = new double(*static_cast(values[i])); + result[i] = reinterpret_cast(p); + break; + } + case TS_TYPE_TEXT: + case TS_TYPE_STRING: + case TS_TYPE_BLOB: + default: { + const char *src = static_cast(values[i]); + size_t len = strlen(src) + 1; + char *p = new char[len]; + memcpy(p, src, len); + result[i] = p; + break; + } + } + } + return result; +} + +static void freeCharPtrVec(std::vector &vec, const TSDataType_C *types, + int count) { + for (int i = 0; i < count; i++) { + switch (types[i]) { + case TS_TYPE_BOOLEAN: + delete reinterpret_cast(vec[i]); + break; + case TS_TYPE_INT32: + delete reinterpret_cast(vec[i]); + break; + case TS_TYPE_INT64: + case TS_TYPE_TIMESTAMP: + delete reinterpret_cast(vec[i]); + break; + case TS_TYPE_FLOAT: + delete reinterpret_cast(vec[i]); + break; + case TS_TYPE_DOUBLE: + delete reinterpret_cast(vec[i]); + break; + default: + delete[] vec[i]; + break; + } + } +} + +/* ============================================================ + * Session Lifecycle — Tree Model + * ============================================================ */ + +extern "C" { + +CSession *ts_session_new(const char *host, int rpcPort, const char *username, + const char *password) { + clearError(); + try { + SessionBuilder builder; + builder.host(host) + ->rpcPort(rpcPort) + ->username(username) + ->password(password) + ->enableAutoFetch(false); + auto cpp = std::make_shared(&builder); + auto *cs = new CSession_(); + cs->cpp = std::move(cpp); + return cs; + } catch (const std::exception &e) { + handleException(e); + return nullptr; + } +} + +CSession *ts_session_new_with_zone(const char *host, int rpcPort, + const char *username, const char *password, + const char *zoneId, int fetchSize) { + clearError(); + try { + SessionBuilder builder; + builder.host(host) + ->rpcPort(rpcPort) + ->username(username) + ->password(password) + ->zoneId(zoneId) + ->fetchSize(fetchSize) + ->enableAutoFetch(false); + auto cpp = std::make_shared(&builder); + auto *cs = new CSession_(); + cs->cpp = std::move(cpp); + return cs; + } catch (const std::exception &e) { + handleException(e); + return nullptr; + } +} + +CSession *ts_session_new_multi_node(const char *const *nodeUrls, int urlCount, + const char *username, + const char *password) { + clearError(); + try { + auto urls = toStringVec(nodeUrls, urlCount); + SessionBuilder builder; + builder.nodeUrls(urls) + ->username(username) + ->password(password) + ->enableAutoFetch(false); + auto cpp = std::make_shared(&builder); + auto *cs = new CSession_(); + cs->cpp = std::move(cpp); + return cs; + } catch (const std::exception &e) { + handleException(e); + return nullptr; + } +} + +void ts_session_destroy(CSession *session) { delete session; } + +TsStatus ts_session_open(CSession *session) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->open(); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_open_with_compression(CSession *session, + bool enableRPCCompression) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->open(enableRPCCompression); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_close(CSession *session) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->close(); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Timezone + * ============================================================ */ + +TsStatus ts_session_set_timezone(CSession *session, const char *zoneId) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->setTimeZone(std::string(zoneId)); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_get_timezone(CSession *session, char *buf, int bufLen) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + if (!buf || bufLen <= 0) + return setError(TS_ERR_INVALID_PARAM, "invalid buffer"); + try { + std::string tz = session->cpp->getTimeZone(); + if ((int)tz.size() >= bufLen) { + return setError(TS_ERR_INVALID_PARAM, "buffer too small"); + } + strncpy(buf, tz.c_str(), bufLen); + buf[bufLen - 1] = '\0'; + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Database Management (Tree Model) + * ============================================================ */ + +TsStatus ts_session_create_database(CSession *session, const char *database) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->createDatabase(std::string(database)); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_delete_database(CSession *session, const char *database) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->deleteDatabase(std::string(database)); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_delete_databases(CSession *session, + const char *const *databases, int count) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto dbs = toStringVec(databases, count); + session->cpp->deleteDatabases(dbs); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Timeseries Management (Tree Model) + * ============================================================ */ + +TsStatus ts_session_create_timeseries(CSession *session, const char *path, + TSDataType_C dataType, + TSEncoding_C encoding, + TSCompressionType_C compressor) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->createTimeseries( + std::string(path), static_cast(dataType), + static_cast(encoding), + static_cast(compressor)); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_create_timeseries_ex( + CSession *session, const char *path, TSDataType_C dataType, + TSEncoding_C encoding, TSCompressionType_C compressor, int propsCount, + const char *const *propKeys, const char *const *propValues, int tagsCount, + const char *const *tagKeys, const char *const *tagValues, int attrsCount, + const char *const *attrKeys, const char *const *attrValues, + const char *measurementAlias) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + std::map props = + propsCount > 0 ? toStringMap(propsCount, propKeys, propValues) + : std::map(); + std::map tags = + tagsCount > 0 ? toStringMap(tagsCount, tagKeys, tagValues) + : std::map(); + std::map attrs = + attrsCount > 0 ? toStringMap(attrsCount, attrKeys, attrValues) + : std::map(); + session->cpp->createTimeseries( + std::string(path), static_cast(dataType), + static_cast(encoding), + static_cast(compressor), &props, + &tags, &attrs, std::string(measurementAlias ? measurementAlias : "")); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_create_multi_timeseries( + CSession *session, int count, const char *const *paths, + const TSDataType_C *dataTypes, const TSEncoding_C *encodings, + const TSCompressionType_C *compressors) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto pathsVec = toStringVec(paths, count); + auto typesVec = toTypeVec(dataTypes, count); + auto encVec = toEncodingVec(encodings, count); + auto compVec = toCompressionVec(compressors, count); + session->cpp->createMultiTimeseries(pathsVec, typesVec, encVec, compVec, + nullptr, nullptr, nullptr, nullptr); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_create_aligned_timeseries( + CSession *session, const char *deviceId, int count, + const char *const *measurements, const TSDataType_C *dataTypes, + const TSEncoding_C *encodings, const TSCompressionType_C *compressors) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto measurementsVec = toStringVec(measurements, count); + auto typesVec = toTypeVec(dataTypes, count); + auto encVec = toEncodingVec(encodings, count); + auto compVec = toCompressionVec(compressors, count); + session->cpp->createAlignedTimeseries( + std::string(deviceId), measurementsVec, typesVec, encVec, compVec); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_check_timeseries_exists(CSession *session, const char *path, + bool *exists) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + if (!exists) + return setError(TS_ERR_INVALID_PARAM, "exists pointer is null"); + try { + *exists = session->cpp->checkTimeseriesExists(std::string(path)); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_delete_timeseries(CSession *session, const char *path) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->deleteTimeseries(std::string(path)); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_delete_timeseries_batch(CSession *session, + const char *const *paths, + int count) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto pathsVec = toStringVec(paths, count); + session->cpp->deleteTimeseries(pathsVec); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Tablet Operations + * ============================================================ */ + +CTablet *ts_tablet_new(const char *deviceId, int columnCount, + const char *const *columnNames, + const TSDataType_C *dataTypes, int maxRowNumber) { + try { + std::vector> schemas; + schemas.reserve(columnCount); + for (int i = 0; i < columnCount; i++) { + schemas.emplace_back(std::string(columnNames[i]), + static_cast(dataTypes[i])); + } + Tablet tablet(std::string(deviceId), schemas, maxRowNumber); + auto *ct = new CTablet_(); + ct->cpp = std::move(tablet); + return ct; + } catch (const std::exception &e) { + handleException(e); + return nullptr; + } +} + +void ts_tablet_destroy(CTablet *tablet) { delete tablet; } + +void ts_tablet_reset(CTablet *tablet) { + if (tablet) { + tablet->cpp.reset(); + } +} + +int ts_tablet_get_row_count(CTablet *tablet) { + if (!tablet) + return 0; + return static_cast(tablet->cpp.rowSize); +} + +TsStatus ts_tablet_set_row_count(CTablet *tablet, int rowCount) { + clearError(); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + tablet->cpp.rowSize = rowCount; + return TS_OK; +} + +TsStatus ts_tablet_add_timestamp(CTablet *tablet, int rowIndex, + int64_t timestamp) { + clearError(); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + try { + tablet->cpp.addTimestamp(static_cast(rowIndex), timestamp); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_tablet_add_value_bool(CTablet *tablet, int colIndex, int rowIndex, + bool value) { + clearError(); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + try { + tablet->cpp.addValue(static_cast(colIndex), + static_cast(rowIndex), value); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_tablet_add_value_int32(CTablet *tablet, int colIndex, int rowIndex, + int32_t value) { + clearError(); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + try { + tablet->cpp.addValue(static_cast(colIndex), + static_cast(rowIndex), value); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_tablet_add_value_int64(CTablet *tablet, int colIndex, int rowIndex, + int64_t value) { + clearError(); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + try { + tablet->cpp.addValue(static_cast(colIndex), + static_cast(rowIndex), value); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_tablet_add_value_float(CTablet *tablet, int colIndex, int rowIndex, + float value) { + clearError(); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + try { + tablet->cpp.addValue(static_cast(colIndex), + static_cast(rowIndex), value); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_tablet_add_value_double(CTablet *tablet, int colIndex, int rowIndex, + double value) { + clearError(); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + try { + tablet->cpp.addValue(static_cast(colIndex), + static_cast(rowIndex), value); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_tablet_add_value_string(CTablet *tablet, int colIndex, int rowIndex, + const char *value) { + clearError(); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + try { + std::string str(value); + tablet->cpp.addValue(static_cast(colIndex), + static_cast(rowIndex), str); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Data Insertion — Tree Model (Record, string values) + * ============================================================ */ + +TsStatus ts_session_insert_record_str(CSession *session, const char *deviceId, + int64_t time, int count, + const char *const *measurements, + const char *const *values) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto measurementsVec = toStringVec(measurements, count); + auto valuesVec = toStringVec(values, count); + session->cpp->insertRecord(std::string(deviceId), time, measurementsVec, + valuesVec); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_insert_record(CSession *session, const char *deviceId, + int64_t time, int count, + const char *const *measurements, + const TSDataType_C *types, + const void *const *values) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto measurementsVec = toStringVec(measurements, count); + auto typesVec = toTypeVec(types, count); + auto charVec = toCharPtrVec(types, values, count); + session->cpp->insertRecord(std::string(deviceId), time, measurementsVec, + typesVec, charVec); + freeCharPtrVec(charVec, types, count); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_insert_aligned_record_str(CSession *session, + const char *deviceId, + int64_t time, int count, + const char *const *measurements, + const char *const *values) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto measurementsVec = toStringVec(measurements, count); + auto valuesVec = toStringVec(values, count); + session->cpp->insertAlignedRecord(std::string(deviceId), time, + measurementsVec, valuesVec); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_insert_aligned_record(CSession *session, + const char *deviceId, int64_t time, + int count, + const char *const *measurements, + const TSDataType_C *types, + const void *const *values) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto measurementsVec = toStringVec(measurements, count); + auto typesVec = toTypeVec(types, count); + auto charVec = toCharPtrVec(types, values, count); + session->cpp->insertAlignedRecord(std::string(deviceId), time, + measurementsVec, typesVec, charVec); + freeCharPtrVec(charVec, types, count); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Data Insertion — Batch, multiple devices (string values) + * ============================================================ */ + +TsStatus ts_session_insert_records_str( + CSession *session, int deviceCount, const char *const *deviceIds, + const int64_t *times, const int *measurementCounts, + const char *const *const *measurementsList, + const char *const *const *valuesList) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto devVec = toStringVec(deviceIds, deviceCount); + std::vector timesVec(times, times + deviceCount); + std::vector> mList, vList; + mList.reserve(deviceCount); + vList.reserve(deviceCount); + for (int i = 0; i < deviceCount; i++) { + mList.push_back(toStringVec(measurementsList[i], measurementCounts[i])); + vList.push_back(toStringVec(valuesList[i], measurementCounts[i])); + } + session->cpp->insertRecords(devVec, timesVec, mList, vList); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_insert_aligned_records_str( + CSession *session, int deviceCount, const char *const *deviceIds, + const int64_t *times, const int *measurementCounts, + const char *const *const *measurementsList, + const char *const *const *valuesList) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto devVec = toStringVec(deviceIds, deviceCount); + std::vector timesVec(times, times + deviceCount); + std::vector> mList, vList; + mList.reserve(deviceCount); + vList.reserve(deviceCount); + for (int i = 0; i < deviceCount; i++) { + mList.push_back(toStringVec(measurementsList[i], measurementCounts[i])); + vList.push_back(toStringVec(valuesList[i], measurementCounts[i])); + } + session->cpp->insertAlignedRecords(devVec, timesVec, mList, vList); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Data Insertion — Batch, multiple devices (typed values) + * ============================================================ */ + +TsStatus ts_session_insert_records(CSession *session, int deviceCount, + const char *const *deviceIds, + const int64_t *times, + const int *measurementCounts, + const char *const *const *measurementsList, + const TSDataType_C *const *typesList, + const void *const *const *valuesList) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto devVec = toStringVec(deviceIds, deviceCount); + std::vector timesVec(times, times + deviceCount); + std::vector> mList; + std::vector> tList; + std::vector> vList; + mList.reserve(deviceCount); + tList.reserve(deviceCount); + vList.reserve(deviceCount); + for (int i = 0; i < deviceCount; i++) { + mList.push_back(toStringVec(measurementsList[i], measurementCounts[i])); + tList.push_back(toTypeVec(typesList[i], measurementCounts[i])); + vList.push_back( + toCharPtrVec(typesList[i], valuesList[i], measurementCounts[i])); + } + session->cpp->insertRecords(devVec, timesVec, mList, tList, vList); + for (int i = 0; i < deviceCount; i++) { + freeCharPtrVec(vList[i], typesList[i], measurementCounts[i]); + } + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_insert_aligned_records( + CSession *session, int deviceCount, const char *const *deviceIds, + const int64_t *times, const int *measurementCounts, + const char *const *const *measurementsList, + const TSDataType_C *const *typesList, + const void *const *const *valuesList) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto devVec = toStringVec(deviceIds, deviceCount); + std::vector timesVec(times, times + deviceCount); + std::vector> mList; + std::vector> tList; + std::vector> vList; + mList.reserve(deviceCount); + tList.reserve(deviceCount); + vList.reserve(deviceCount); + for (int i = 0; i < deviceCount; i++) { + mList.push_back(toStringVec(measurementsList[i], measurementCounts[i])); + tList.push_back(toTypeVec(typesList[i], measurementCounts[i])); + vList.push_back( + toCharPtrVec(typesList[i], valuesList[i], measurementCounts[i])); + } + session->cpp->insertAlignedRecords(devVec, timesVec, mList, tList, vList); + for (int i = 0; i < deviceCount; i++) { + freeCharPtrVec(vList[i], typesList[i], measurementCounts[i]); + } + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Data Insertion — Batch, single device (typed values) + * ============================================================ */ + +TsStatus ts_session_insert_records_of_one_device( + CSession *session, const char *deviceId, int rowCount, const int64_t *times, + const int *measurementCounts, const char *const *const *measurementsList, + const TSDataType_C *const *typesList, const void *const *const *valuesList, + bool sorted) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + std::vector timesVec(times, times + rowCount); + std::vector> mList; + std::vector> tList; + std::vector> vList; + mList.reserve(rowCount); + tList.reserve(rowCount); + vList.reserve(rowCount); + for (int i = 0; i < rowCount; i++) { + mList.push_back(toStringVec(measurementsList[i], measurementCounts[i])); + tList.push_back(toTypeVec(typesList[i], measurementCounts[i])); + vList.push_back( + toCharPtrVec(typesList[i], valuesList[i], measurementCounts[i])); + } + session->cpp->insertRecordsOfOneDevice(std::string(deviceId), timesVec, + mList, tList, vList, sorted); + for (int i = 0; i < rowCount; i++) { + freeCharPtrVec(vList[i], typesList[i], measurementCounts[i]); + } + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_insert_aligned_records_of_one_device( + CSession *session, const char *deviceId, int rowCount, const int64_t *times, + const int *measurementCounts, const char *const *const *measurementsList, + const TSDataType_C *const *typesList, const void *const *const *valuesList, + bool sorted) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + std::vector timesVec(times, times + rowCount); + std::vector> mList; + std::vector> tList; + std::vector> vList; + mList.reserve(rowCount); + tList.reserve(rowCount); + vList.reserve(rowCount); + for (int i = 0; i < rowCount; i++) { + mList.push_back(toStringVec(measurementsList[i], measurementCounts[i])); + tList.push_back(toTypeVec(typesList[i], measurementCounts[i])); + vList.push_back( + toCharPtrVec(typesList[i], valuesList[i], measurementCounts[i])); + } + session->cpp->insertAlignedRecordsOfOneDevice( + std::string(deviceId), timesVec, mList, tList, vList, sorted); + for (int i = 0; i < rowCount; i++) { + freeCharPtrVec(vList[i], typesList[i], measurementCounts[i]); + } + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Data Insertion — Tree Model (Tablet) + * ============================================================ */ + +TsStatus ts_session_insert_tablet(CSession *session, CTablet *tablet, + bool sorted) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + try { + session->cpp->insertTablet(tablet->cpp, sorted); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_insert_aligned_tablet(CSession *session, CTablet *tablet, + bool sorted) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + if (!tablet) + return setError(TS_ERR_NULL_PTR, "tablet is null"); + try { + session->cpp->insertAlignedTablet(tablet->cpp, sorted); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_insert_tablets(CSession *session, int tabletCount, + const char *const *deviceIds, + CTablet **tablets, bool sorted) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + std::unordered_map tabletMap; + for (int i = 0; i < tabletCount; i++) { + tabletMap[std::string(deviceIds[i])] = &(tablets[i]->cpp); + } + session->cpp->insertTablets(tabletMap, sorted); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_insert_aligned_tablets(CSession *session, int tabletCount, + const char *const *deviceIds, + CTablet **tablets, bool sorted) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + std::unordered_map tabletMap; + for (int i = 0; i < tabletCount; i++) { + tabletMap[std::string(deviceIds[i])] = &(tablets[i]->cpp); + } + session->cpp->insertAlignedTablets(tabletMap, sorted); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +/* ============================================================ + * Query — Tree Model + * ============================================================ */ + +TsStatus ts_session_execute_query(CSession *session, const char *sql, + CSessionDataSet **dataSet) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + if (!dataSet) + return setError(TS_ERR_INVALID_PARAM, "dataSet pointer is null"); + try { + auto ds = session->cpp->executeQueryStatement(std::string(sql)); + CSessionDataSet_ tmp{}; + tmp.cpp = std::move(ds); + auto *cds = new CSessionDataSet_(); + cds->cpp = std::move(tmp.cpp); + *dataSet = cds; + return TS_OK; + } catch (const std::exception &e) { + *dataSet = nullptr; + return handleException(e); + } +} + +TsStatus ts_session_execute_query_with_timeout(CSession *session, + const char *sql, + int64_t timeoutInMs, + CSessionDataSet **dataSet) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + if (!dataSet) + return setError(TS_ERR_INVALID_PARAM, "dataSet pointer is null"); + try { + auto ds = + session->cpp->executeQueryStatement(std::string(sql), timeoutInMs); + CSessionDataSet_ tmp{}; + tmp.cpp = std::move(ds); + auto *cds = new CSessionDataSet_(); + cds->cpp = std::move(tmp.cpp); + *dataSet = cds; + return TS_OK; + } catch (const std::exception &e) { + *dataSet = nullptr; + return handleException(e); + } +} + +TsStatus ts_session_execute_non_query(CSession *session, const char *sql) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->executeNonQueryStatement(std::string(sql)); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_execute_raw_data_query(CSession *session, int pathCount, + const char *const *paths, + int64_t startTime, int64_t endTime, + CSessionDataSet **dataSet) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + if (!dataSet) + return setError(TS_ERR_INVALID_PARAM, "dataSet pointer is null"); + try { + auto pathsVec = toStringVec(paths, pathCount); + auto ds = session->cpp->executeRawDataQuery(pathsVec, startTime, endTime); + CSessionDataSet_ tmp{}; + tmp.cpp = std::move(ds); + auto *cds = new CSessionDataSet_(); + cds->cpp = std::move(tmp.cpp); + *dataSet = cds; + return TS_OK; + } catch (const std::exception &e) { + *dataSet = nullptr; + return handleException(e); + } +} + +TsStatus ts_session_execute_last_data_query(CSession *session, int pathCount, + const char *const *paths, + CSessionDataSet **dataSet) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + if (!dataSet) + return setError(TS_ERR_INVALID_PARAM, "dataSet pointer is null"); + try { + auto pathsVec = toStringVec(paths, pathCount); + auto ds = session->cpp->executeLastDataQuery(pathsVec); + CSessionDataSet_ tmp{}; + tmp.cpp = std::move(ds); + auto *cds = new CSessionDataSet_(); + cds->cpp = std::move(tmp.cpp); + *dataSet = cds; + return TS_OK; + } catch (const std::exception &e) { + *dataSet = nullptr; + return handleException(e); + } +} + +TsStatus ts_session_execute_last_data_query_with_time( + CSession *session, int pathCount, const char *const *paths, + int64_t lastTime, CSessionDataSet **dataSet) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + if (!dataSet) + return setError(TS_ERR_INVALID_PARAM, "dataSet pointer is null"); + try { + auto pathsVec = toStringVec(paths, pathCount); + auto ds = session->cpp->executeLastDataQuery(pathsVec, lastTime); + CSessionDataSet_ tmp{}; + tmp.cpp = std::move(ds); + auto *cds = new CSessionDataSet_(); + cds->cpp = std::move(tmp.cpp); + *dataSet = cds; + return TS_OK; + } catch (const std::exception &e) { + *dataSet = nullptr; + return handleException(e); + } +} + +/* ============================================================ + * SessionDataSet & RowRecord — Result Iteration + * ============================================================ */ + +void ts_dataset_destroy(CSessionDataSet *dataSet) { + if (dataSet) { + if (dataSet->cpp) { + dataSet->cpp->closeOperationHandle(); + } + delete dataSet; + } +} + +bool ts_dataset_has_next(CSessionDataSet *dataSet) { + clearError(); + if (!dataSet) { + (void)setError(TS_ERR_NULL_PTR, "dataSet is null"); + return false; + } + if (!dataSet->cpp) { + (void)setError(TS_ERR_NULL_PTR, "dataSet is not initialized"); + return false; + } + try { + return dataSet->cpp->hasNext(); + } catch (const std::exception &e) { + (void)handleException(e); + return false; + } catch (...) { + (void)setError(TS_ERR_UNKNOWN, "non-standard exception"); + return false; + } +} + +CRowRecord *ts_dataset_next(CSessionDataSet *dataSet) { + clearError(); + if (!dataSet) { + (void)setError(TS_ERR_NULL_PTR, "dataSet is null"); + return nullptr; + } + if (!dataSet->cpp) { + (void)setError(TS_ERR_NULL_PTR, "dataSet is not initialized"); + return nullptr; + } + try { + auto row = dataSet->cpp->next(); + if (!row) + return nullptr; + CRowRecord_ tmp{}; + tmp.cpp = std::move(row); + auto *crr = new CRowRecord_(); + crr->cpp = std::move(tmp.cpp); + return crr; + } catch (const std::exception &e) { + (void)handleException(e); + return nullptr; + } catch (...) { + (void)setError(TS_ERR_UNKNOWN, "non-standard exception"); + return nullptr; + } +} + +int ts_dataset_get_column_count(CSessionDataSet *dataSet) { + if (!dataSet || !dataSet->cpp) + return 0; + return static_cast(dataSet->cpp->getColumnNames().size()); +} + +static thread_local std::string g_colNameBuf; + +const char *ts_dataset_get_column_name(CSessionDataSet *dataSet, int index) { + if (!dataSet || !dataSet->cpp) + return ""; + const auto &names = dataSet->cpp->getColumnNames(); + if (index < 0 || index >= (int)names.size()) + return ""; + g_colNameBuf = names[index]; + return g_colNameBuf.c_str(); +} + +static thread_local std::string g_colTypeBuf; + +const char *ts_dataset_get_column_type(CSessionDataSet *dataSet, int index) { + if (!dataSet || !dataSet->cpp) + return ""; + const auto &types = dataSet->cpp->getColumnTypeList(); + if (index < 0 || index >= (int)types.size()) + return ""; + g_colTypeBuf = types[index]; + return g_colTypeBuf.c_str(); +} + +void ts_dataset_set_fetch_size(CSessionDataSet *dataSet, int fetchSize) { + if (dataSet && dataSet->cpp) { + dataSet->cpp->setFetchSize(fetchSize); + } +} + +void ts_row_record_destroy(CRowRecord *record) { delete record; } + +int64_t ts_row_record_get_timestamp(CRowRecord *record) { + if (!record || !record->cpp) + return -1; + return record->cpp->timestamp; +} + +int ts_row_record_get_field_count(CRowRecord *record) { + if (!record || !record->cpp) + return 0; + return static_cast(record->cpp->fields.size()); +} + +bool ts_row_record_is_null(CRowRecord *record, int index) { + if (!record || !record->cpp) + return true; + if (index < 0 || index >= (int)record->cpp->fields.size()) + return true; + return record->cpp->fields[index].isNull(); +} + +bool ts_row_record_get_bool(CRowRecord *record, int index) { + if (!record || !record->cpp) + return false; + if (index < 0 || index >= (int)record->cpp->fields.size()) + return false; + const Field &f = record->cpp->fields[index]; + return f.boolV.is_initialized() ? f.boolV.value() : false; +} + +int32_t ts_row_record_get_int32(CRowRecord *record, int index) { + if (!record || !record->cpp) + return 0; + if (index < 0 || index >= (int)record->cpp->fields.size()) + return 0; + const Field &f = record->cpp->fields[index]; + return f.intV.is_initialized() ? f.intV.value() : 0; +} + +int64_t ts_row_record_get_int64(CRowRecord *record, int index) { + if (!record || !record->cpp) + return 0; + if (index < 0 || index >= (int)record->cpp->fields.size()) + return 0; + const Field &f = record->cpp->fields[index]; + if (f.longV.is_initialized()) + return f.longV.value(); + if (f.intV.is_initialized()) + return f.intV.value(); + if (f.doubleV.is_initialized()) + return static_cast(f.doubleV.value()); + return 0; +} + +float ts_row_record_get_float(CRowRecord *record, int index) { + if (!record || !record->cpp) + return 0.0f; + if (index < 0 || index >= (int)record->cpp->fields.size()) + return 0.0f; + const Field &f = record->cpp->fields[index]; + return f.floatV.is_initialized() ? f.floatV.value() : 0.0f; +} + +double ts_row_record_get_double(CRowRecord *record, int index) { + if (!record || !record->cpp) + return 0.0; + if (index < 0 || index >= (int)record->cpp->fields.size()) + return 0.0; + const Field &f = record->cpp->fields[index]; + return f.doubleV.is_initialized() ? f.doubleV.value() : 0.0; +} + +static thread_local std::string g_stringBuf; + +const char *ts_row_record_get_string(CRowRecord *record, int index) { + if (!record || !record->cpp) + return ""; + if (index < 0 || index >= (int)record->cpp->fields.size()) + return ""; + const Field &f = record->cpp->fields[index]; + if (f.stringV.is_initialized()) { + g_stringBuf = f.stringV.value(); + return g_stringBuf.c_str(); + } + return ""; +} + +TSDataType_C ts_row_record_get_data_type(CRowRecord *record, int index) { + if (!record || !record->cpp) + return TS_TYPE_INVALID; + if (index < 0 || index >= (int)record->cpp->fields.size()) + return TS_TYPE_INVALID; + return static_cast(record->cpp->fields[index].dataType); +} + +/* ============================================================ + * Data Deletion (Tree Model) + * ============================================================ */ + +TsStatus ts_session_delete_data(CSession *session, const char *path, + int64_t endTime) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + session->cpp->deleteData(std::string(path), endTime); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_delete_data_batch(CSession *session, int pathCount, + const char *const *paths, + int64_t endTime) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto pathsVec = toStringVec(paths, pathCount); + session->cpp->deleteData(pathsVec, endTime); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +TsStatus ts_session_delete_data_range(CSession *session, int pathCount, + const char *const *paths, + int64_t startTime, int64_t endTime) { + clearError(); + if (!session) + return setError(TS_ERR_NULL_PTR, "session is null"); + try { + auto pathsVec = toStringVec(paths, pathCount); + session->cpp->deleteData(pathsVec, startTime, endTime); + return TS_OK; + } catch (const std::exception &e) { + return handleException(e); + } +} + +} /* extern "C" */ diff --git a/iotdb-client/client-cpp/src/session/SessionDataSet.cpp b/iotdb-client/client-cpp/src/session/SessionDataSet.cpp new file mode 100644 index 0000000000000..94a81d3c58d65 --- /dev/null +++ b/iotdb-client/client-cpp/src/session/SessionDataSet.cpp @@ -0,0 +1,341 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "SessionDataSet.h" + +#include + +#include "IoTDBRpcDataSet.h" +#include "SessionDataSetFactory.h" + +using namespace std; + +struct SessionDataSet::Impl { + std::shared_ptr iotdbRpcDataSet_; +}; + +std::unique_ptr createSessionDataSet( + const std::string &sql, const std::vector &columnNameList, + const std::vector &columnTypeList, + const std::map &columnNameIndex, int64_t queryId, + int64_t statementId, std::shared_ptr client, + int64_t sessionId, const std::vector &queryResult, + bool ignoreTimestamp, int64_t timeout, bool moreData, int32_t fetchSize, + const std::string &zoneId) { + auto dataSet = std::unique_ptr(new SessionDataSet()); + dataSet->impl_ = + std::unique_ptr(new SessionDataSet::Impl()); + dataSet->impl_->iotdbRpcDataSet_ = std::make_shared( + sql, columnNameList, columnTypeList, columnNameIndex, ignoreTimestamp, + moreData, queryId, statementId, client, sessionId, queryResult, fetchSize, + timeout, zoneId, IoTDBRpcDataSet::DEFAULT_TIME_FORMAT); + return dataSet; +} + +RowRecord::RowRecord(int64_t timestamp) { this->timestamp = timestamp; } + +RowRecord::RowRecord(int64_t timestamp, const std::vector &fields) + : timestamp(timestamp), fields(fields) {} + +RowRecord::RowRecord(const std::vector &fields) + : timestamp(-1), fields(fields) {} + +RowRecord::RowRecord() { this->timestamp = -1; } + +void RowRecord::addField(const Field &f) { this->fields.push_back(f); } + +std::string RowRecord::toString() { + std::string ret; + if (this->timestamp != -1) { + ret.append(std::to_string(timestamp)); + ret.append("\t"); + } + for (size_t i = 0; i < fields.size(); i++) { + if (i != 0) { + ret.append("\t"); + } + const Field &f = fields[i]; + switch (f.dataType) { + case TSDataType::BOOLEAN: + if (f.isNull()) { + ret.append("null"); + } else { + ret.append(f.boolV.value() ? "true" : "false"); + } + break; + case TSDataType::INT32: + if (f.isNull()) { + ret.append("null"); + } else { + ret.append(std::to_string(f.intV.value())); + } + break; + case TSDataType::DATE: + if (f.isNull()) { + ret.append("null"); + } else { + ret.append(f.dateV.value().toIsoExtendedString()); + } + break; + case TSDataType::TIMESTAMP: + case TSDataType::INT64: + if (f.isNull()) { + ret.append("null"); + } else { + ret.append(std::to_string(f.longV.value())); + } + break; + case TSDataType::FLOAT: + if (f.isNull()) { + ret.append("null"); + } else { + ret.append(std::to_string(f.floatV.value())); + } + break; + case TSDataType::DOUBLE: + if (f.isNull()) { + ret.append("null"); + } else { + ret.append(std::to_string(f.doubleV.value())); + } + break; + case TSDataType::BLOB: + case TSDataType::STRING: + case TSDataType::TEXT: + if (f.isNull()) { + ret.append("null"); + } else { + ret.append(f.stringV.value()); + } + break; + case TSDataType::OBJECT: + if (!f.stringV.is_initialized()) { + ret.append("null"); + } else { + ret.append(f.stringV.value()); + } + break; + default: + break; + } + } + ret.append("\n"); + return ret; +} + +SessionDataSet::~SessionDataSet() = default; + +bool SessionDataSet::hasNext() { + if (impl_->iotdbRpcDataSet_->hasCachedRecord()) { + return true; + } + return impl_->iotdbRpcDataSet_->next(); +} + +shared_ptr SessionDataSet::next() { + if (!impl_->iotdbRpcDataSet_->hasCachedRecord() && !hasNext()) { + return nullptr; + } + impl_->iotdbRpcDataSet_->setHasCachedRecord(false); + return constructRowRecordFromValueArray(); +} + +int SessionDataSet::getFetchSize() { + return impl_->iotdbRpcDataSet_->getFetchSize(); +} + +void SessionDataSet::setFetchSize(int fetchSize) { + impl_->iotdbRpcDataSet_->setFetchSize(fetchSize); +} + +const std::vector &SessionDataSet::getColumnNames() const { + return impl_->iotdbRpcDataSet_->getColumnNameList(); +} + +const std::vector &SessionDataSet::getColumnTypeList() const { + return impl_->iotdbRpcDataSet_->getColumnTypeList(); +} + +void SessionDataSet::closeOperationHandle(bool forceClose) { + impl_->iotdbRpcDataSet_->close(forceClose); +} + +SessionDataSet::DataIterator::DataIterator(std::shared_ptr impl) + : impl_(std::move(impl)) {} + +bool SessionDataSet::DataIterator::next() { + return impl_->iotdbRpcDataSet_->next(); +} + +bool SessionDataSet::DataIterator::isNull(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->isNullByColumnName(columnName); +} + +bool SessionDataSet::DataIterator::isNullByIndex(int32_t columnIndex) { + return impl_->iotdbRpcDataSet_->isNullByIndex(columnIndex); +} + +Optional +SessionDataSet::DataIterator::getBooleanByIndex(int32_t columnIndex) { + return impl_->iotdbRpcDataSet_->getBooleanByIndex(columnIndex); +} + +Optional +SessionDataSet::DataIterator::getBoolean(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->getBoolean(columnName); +} + +Optional +SessionDataSet::DataIterator::getDoubleByIndex(int32_t columnIndex) { + return impl_->iotdbRpcDataSet_->getDoubleByIndex(columnIndex); +} + +Optional +SessionDataSet::DataIterator::getDouble(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->getDouble(columnName); +} + +Optional +SessionDataSet::DataIterator::getFloatByIndex(int32_t columnIndex) { + return impl_->iotdbRpcDataSet_->getFloatByIndex(columnIndex); +} + +Optional +SessionDataSet::DataIterator::getFloat(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->getFloat(columnName); +} + +Optional +SessionDataSet::DataIterator::getIntByIndex(int32_t columnIndex) { + return impl_->iotdbRpcDataSet_->getIntByIndex(columnIndex); +} + +Optional +SessionDataSet::DataIterator::getInt(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->getInt(columnName); +} + +Optional +SessionDataSet::DataIterator::getLongByIndex(int32_t columnIndex) { + return impl_->iotdbRpcDataSet_->getLongByIndex(columnIndex); +} + +Optional +SessionDataSet::DataIterator::getLong(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->getLong(columnName); +} + +Optional +SessionDataSet::DataIterator::getStringByIndex(int32_t columnIndex) { + return impl_->iotdbRpcDataSet_->getStringByIndex(columnIndex); +} + +Optional +SessionDataSet::DataIterator::getString(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->getString(columnName); +} + +Optional +SessionDataSet::DataIterator::getTimestampByIndex(int32_t columnIndex) { + return impl_->iotdbRpcDataSet_->getTimestampByIndex(columnIndex); +} + +Optional +SessionDataSet::DataIterator::getTimestamp(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->getTimestamp(columnName); +} + +Optional +SessionDataSet::DataIterator::getDateByIndex(int32_t columnIndex) { + return impl_->iotdbRpcDataSet_->getDateByIndex(columnIndex); +} + +Optional +SessionDataSet::DataIterator::getDate(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->getDate(columnName); +} + +int32_t +SessionDataSet::DataIterator::findColumn(const std::string &columnName) { + return impl_->iotdbRpcDataSet_->findColumn(columnName); +} + +const std::vector & +SessionDataSet::DataIterator::getColumnNames() const { + return impl_->iotdbRpcDataSet_->getColumnNameList(); +} + +const std::vector & +SessionDataSet::DataIterator::getColumnTypeList() const { + return impl_->iotdbRpcDataSet_->getColumnTypeList(); +} + +SessionDataSet::DataIterator SessionDataSet::getIterator() { + return DataIterator(std::shared_ptr(impl_.get(), [](Impl *) {})); +} + +shared_ptr SessionDataSet::constructRowRecordFromValueArray() { + std::vector outFields; + const auto &dataSet = impl_->iotdbRpcDataSet_; + const int32_t valueColumnCount = dataSet->getServerColumnCount(); + for (int i = 0; i < valueColumnCount; i++) { + const int32_t listIndex = dataSet->getValueColumnNameListIndex(i); + const std::string &columnName = dataSet->getColumnNameList().at(listIndex); + Field field; + if (!dataSet->isNullByColumnName(columnName)) { + TSDataType::TSDataType dataType = dataSet->getDataType(columnName); + field.dataType = dataType; + switch (dataType) { + case TSDataType::BOOLEAN: + field.boolV = dataSet->getBoolean(columnName); + break; + case TSDataType::INT32: + field.intV = dataSet->getInt(columnName); + break; + case TSDataType::DATE: + field.dateV = dataSet->getDate(columnName); + break; + case TSDataType::INT64: + case TSDataType::TIMESTAMP: + field.longV = dataSet->getLong(columnName); + break; + case TSDataType::FLOAT: + field.floatV = dataSet->getFloat(columnName); + break; + case TSDataType::DOUBLE: + field.doubleV = dataSet->getDouble(columnName); + break; + case TSDataType::TEXT: + case TSDataType::BLOB: + case TSDataType::STRING: + case TSDataType::OBJECT: { + auto stringValue = dataSet->getString(columnName); + if (stringValue.is_initialized()) { + field.stringV = stringValue.value(); + } + break; + } + default: + throw UnSupportedDataTypeException("Data type is not supported."); + } + } + outFields.emplace_back(field); + } + return std::make_shared(dataSet->getCurrentRowTime(), outFields); +} diff --git a/iotdb-client/client-cpp/src/main/SessionPool.cpp b/iotdb-client/client-cpp/src/session/SessionPool.cpp similarity index 66% rename from iotdb-client/client-cpp/src/main/SessionPool.cpp rename to iotdb-client/client-cpp/src/session/SessionPool.cpp index 7f673a0171a8d..67df6ca1a40e5 100644 --- a/iotdb-client/client-cpp/src/main/SessionPool.cpp +++ b/iotdb-client/client-cpp/src/session/SessionPool.cpp @@ -28,8 +28,8 @@ void PooledSession::reset() { broken_ = false; } -SessionPool::SessionPool(std::string host, int rpcPort, std::string username, std::string password, - size_t maxSize) +SessionPool::SessionPool(std::string host, int rpcPort, std::string username, + std::string password, size_t maxSize) : host_(std::move(host)), rpcPort_(rpcPort), username_(std::move(username)), password_(std::move(password)), maxSize_(maxSize) { if (maxSize_ == 0) { @@ -40,52 +40,52 @@ SessionPool::SessionPool(std::string host, int rpcPort, std::string username, st SessionPool::~SessionPool() { try { close(); - } catch (const std::exception& e) { + } catch (const std::exception &e) { log_debug(std::string("SessionPool::~SessionPool(), ") + e.what()); } } -SessionPool& SessionPool::setFetchSize(int fetchSize) { +SessionPool &SessionPool::setFetchSize(int fetchSize) { fetchSize_ = fetchSize; return *this; } -SessionPool& SessionPool::setZoneId(std::string zoneId) { +SessionPool &SessionPool::setZoneId(std::string zoneId) { zoneId_ = std::move(zoneId); return *this; } -SessionPool& SessionPool::setSqlDialect(std::string sqlDialect) { +SessionPool &SessionPool::setSqlDialect(std::string sqlDialect) { sqlDialect_ = std::move(sqlDialect); return *this; } -SessionPool& SessionPool::setDatabase(std::string database) { +SessionPool &SessionPool::setDatabase(std::string database) { database_ = std::move(database); return *this; } -SessionPool& SessionPool::setEnableRedirection(bool enable) { +SessionPool &SessionPool::setEnableRedirection(bool enable) { enableRedirection_ = enable; return *this; } -SessionPool& SessionPool::setEnableAutoFetch(bool enable) { +SessionPool &SessionPool::setEnableAutoFetch(bool enable) { enableAutoFetch_ = enable; return *this; } -SessionPool& SessionPool::setEnableRPCCompression(bool enable) { +SessionPool &SessionPool::setEnableRPCCompression(bool enable) { enableRPCCompression_ = enable; return *this; } -SessionPool& SessionPool::setConnectTimeoutMs(int connectTimeoutMs) { +SessionPool &SessionPool::setConnectTimeoutMs(int connectTimeoutMs) { connectTimeoutMs_ = connectTimeoutMs; return *this; } -SessionPool& SessionPool::setWaitToGetSessionTimeoutMs(int64_t timeoutMs) { +SessionPool &SessionPool::setWaitToGetSessionTimeoutMs(int64_t timeoutMs) { waitTimeoutMs_ = timeoutMs; return *this; } @@ -112,8 +112,8 @@ std::shared_ptr SessionPool::constructNewSession() { std::shared_ptr SessionPool::acquire(int64_t timeoutMs) { const int64_t effectiveTimeout = timeoutMs > 0 ? timeoutMs : waitTimeoutMs_; std::unique_lock lock(mutex_); - const auto deadline = - std::chrono::steady_clock::now() + std::chrono::milliseconds(effectiveTimeout); + const auto deadline = std::chrono::steady_clock::now() + + std::chrono::milliseconds(effectiveTimeout); while (true) { if (closed_) { @@ -140,8 +140,8 @@ std::shared_ptr SessionPool::acquire(int64_t timeoutMs) { } lock.lock(); if (closed_) { - // The pool was closed while this session was being built; do not hand it - // out. Release its slot and let it be torn down outside the lock. + // The pool was closed while this session was being built; do not hand + // it out. Release its slot and let it be torn down outside the lock. --size_; lock.unlock(); throw IoTDBException("SessionPool is closed."); @@ -153,22 +153,25 @@ std::shared_ptr SessionPool::acquire(int64_t timeoutMs) { if (effectiveTimeout <= 0) { cv_.wait(lock); } else { - if (cv_.wait_until(lock, deadline) == std::cv_status::timeout && idleQueue_.empty() && - size_ >= maxSize_ && !closed_) { + if (cv_.wait_until(lock, deadline) == std::cv_status::timeout && + idleQueue_.empty() && size_ >= maxSize_ && !closed_) { throw IoTDBException( - "Wait to get session timeout in SessionPool, maxSize=" + std::to_string(maxSize_) + + "Wait to get session timeout in SessionPool, maxSize=" + + std::to_string(maxSize_) + ", waitTimeoutMs=" + std::to_string(effectiveTimeout) + "."); } } } } -void SessionPool::putBack(const std::shared_ptr& session, bool broken) { +void SessionPool::putBack(const std::shared_ptr &session, + bool broken) { std::lock_guard lock(mutex_); if (broken || closed_) { - // Drop the Session and free its slot so a healthy replacement can be created - // on demand. The caller (PooledSession::reset) still holds the last reference - // and tears the connection down after we return, i.e. outside this lock. + // Drop the Session and free its slot so a healthy replacement can be + // created on demand. The caller (PooledSession::reset) still holds the last + // reference and tears the connection down after we return, i.e. outside + // this lock. --size_; } else { idleQueue_.push_back(session); @@ -176,61 +179,66 @@ void SessionPool::putBack(const std::shared_ptr& session, bool broken) cv_.notify_one(); } -PooledSession SessionPool::getSession() { - return getSession(waitTimeoutMs_); -} +PooledSession SessionPool::getSession() { return getSession(waitTimeoutMs_); } PooledSession SessionPool::getSession(int64_t timeoutMs) { return PooledSession(this, acquire(timeoutMs)); } -void SessionPool::insertTablet(Tablet& tablet, bool sorted) { - execute([&](Session& s) { s.insertTablet(tablet, sorted); }); +void SessionPool::insertTablet(Tablet &tablet, bool sorted) { + execute([&](Session &s) { s.insertTablet(tablet, sorted); }); } -void SessionPool::insertAlignedTablet(Tablet& tablet, bool sorted) { - execute([&](Session& s) { s.insertAlignedTablet(tablet, sorted); }); +void SessionPool::insertAlignedTablet(Tablet &tablet, bool sorted) { + execute([&](Session &s) { s.insertAlignedTablet(tablet, sorted); }); } -void SessionPool::insertTablets(std::unordered_map& tablets, bool sorted) { - execute([&](Session& s) { s.insertTablets(tablets, sorted); }); +void SessionPool::insertTablets( + std::unordered_map &tablets, bool sorted) { + execute([&](Session &s) { s.insertTablets(tablets, sorted); }); } -void SessionPool::insertRecord(const std::string& deviceId, int64_t time, - const std::vector& measurements, - const std::vector& values) { - execute([&](Session& s) { s.insertRecord(deviceId, time, measurements, values); }); +void SessionPool::insertRecord(const std::string &deviceId, int64_t time, + const std::vector &measurements, + const std::vector &values) { + execute([&](Session &s) { + s.insertRecord(deviceId, time, measurements, values); + }); } -void SessionPool::insertRecords(const std::vector& deviceIds, - const std::vector& times, - const std::vector>& measurementsList, - const std::vector>& valuesList) { - execute([&](Session& s) { s.insertRecords(deviceIds, times, measurementsList, valuesList); }); +void SessionPool::insertRecords( + const std::vector &deviceIds, + const std::vector ×, + const std::vector> &measurementsList, + const std::vector> &valuesList) { + execute([&](Session &s) { + s.insertRecords(deviceIds, times, measurementsList, valuesList); + }); } -void SessionPool::executeNonQueryStatement(const std::string& sql) { - execute([&](Session& s) { s.executeNonQueryStatement(sql); }); +void SessionPool::executeNonQueryStatement(const std::string &sql) { + execute([&](Session &s) { s.executeNonQueryStatement(sql); }); } -PooledSessionDataSet SessionPool::executeQueryStatement(const std::string& sql) { +PooledSessionDataSet +SessionPool::executeQueryStatement(const std::string &sql) { PooledSession lease = getSession(); try { auto dataSet = lease->executeQueryStatement(sql); return PooledSessionDataSet(std::move(lease), std::move(dataSet)); - } catch (const IoTDBConnectionException&) { + } catch (const IoTDBConnectionException &) { lease.markBroken(); throw; } } -PooledSessionDataSet SessionPool::executeQueryStatement(const std::string& sql, +PooledSessionDataSet SessionPool::executeQueryStatement(const std::string &sql, int64_t timeoutInMs) { PooledSession lease = getSession(); try { auto dataSet = lease->executeQueryStatement(sql, timeoutInMs); return PooledSessionDataSet(std::move(lease), std::move(dataSet)); - } catch (const IoTDBConnectionException&) { + } catch (const IoTDBConnectionException &) { lease.markBroken(); throw; } diff --git a/iotdb-client/client-cpp/src/session/TsBlock.cpp b/iotdb-client/client-cpp/src/session/TsBlock.cpp new file mode 100644 index 0000000000000..b4c3cc2da82c6 --- /dev/null +++ b/iotdb-client/client-cpp/src/session/TsBlock.cpp @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +#include "TsBlock.h" +#include +#include +#include + +std::shared_ptr +TsBlock::create(int32_t positionCount, std::shared_ptr timeColumn, + std::vector> valueColumns) { + if (valueColumns.empty()) { + throw std::invalid_argument("valueColumns cannot be empty"); + } + return std::shared_ptr(new TsBlock( + positionCount, std::move(timeColumn), std::move(valueColumns))); +} + +std::shared_ptr TsBlock::deserialize(const std::string &data) { + MyStringBuffer buffer(data); + + // Read value column count + int32_t valueColumnCount = buffer.getInt(); + if (valueColumnCount < 0) { + throw IoTDBException("TsBlock::deserialize: negative valueColumnCount"); + } + const int64_t minHeaderBytes = + 9LL + 2LL * static_cast(valueColumnCount); + if (minHeaderBytes > static_cast(data.size())) { + throw IoTDBException("TsBlock::deserialize: truncated header"); + } + + // Read value column data types + std::vector valueColumnDataTypes(valueColumnCount); + for (int32_t i = 0; i < valueColumnCount; i++) { + valueColumnDataTypes[i] = + static_cast(buffer.getChar()); + } + + // Read position count + int32_t positionCount = buffer.getInt(); + if (positionCount < 0) { + throw IoTDBException("TsBlock::deserialize: negative positionCount"); + } + + // Read column encodings + std::vector columnEncodings(valueColumnCount + 1); + for (int32_t i = 0; i < valueColumnCount + 1; i++) { + columnEncodings[i] = static_cast(buffer.getChar()); + } + + // Read time column + auto timeColumnDecoder = getColumnDecoder(columnEncodings[0]); + auto timeColumn = + timeColumnDecoder->readColumn(buffer, TSDataType::INT64, positionCount); + + // Read value columns + std::vector> valueColumns(valueColumnCount); + for (int32_t i = 0; i < valueColumnCount; i++) { + auto valueColumnDecoder = getColumnDecoder(columnEncodings[i + 1]); + valueColumns[i] = valueColumnDecoder->readColumn( + buffer, valueColumnDataTypes[i], positionCount); + } + + return create(positionCount, std::move(timeColumn), std::move(valueColumns)); +} + +TsBlock::TsBlock(int32_t positionCount, std::shared_ptr timeColumn, + std::vector> valueColumns) + : positionCount_(positionCount), timeColumn_(std::move(timeColumn)), + valueColumns_(std::move(valueColumns)) {} + +int32_t TsBlock::getPositionCount() const { return positionCount_; } + +int64_t TsBlock::getStartTime() const { return timeColumn_->getLong(0); } + +int64_t TsBlock::getEndTime() const { + return timeColumn_->getLong(positionCount_ - 1); +} + +bool TsBlock::isEmpty() const { return positionCount_ == 0; } + +int64_t TsBlock::getTimeByIndex(int32_t index) const { + return timeColumn_->getLong(index); +} + +int32_t TsBlock::getValueColumnCount() const { + return static_cast(valueColumns_.size()); +} + +const std::shared_ptr TsBlock::getTimeColumn() const { + return timeColumn_; +} + +const std::vector> &TsBlock::getValueColumns() const { + return valueColumns_; +} + +const std::shared_ptr TsBlock::getColumn(int32_t columnIndex) const { + return valueColumns_[columnIndex]; +} diff --git a/iotdb-client/client-cpp/src/test/CMakeLists.txt b/iotdb-client/client-cpp/src/test/CMakeLists.txt deleted file mode 100644 index d38c975de201c..0000000000000 --- a/iotdb-client/client-cpp/src/test/CMakeLists.txt +++ /dev/null @@ -1,64 +0,0 @@ -# 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.7) -INCLUDE( CTest ) -SET(CMAKE_CXX_STANDARD 11) -SET(CMAKE_CXX_STANDARD_REQUIRED ON) -SET(TARGET_NAME session_tests) -SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11 -Wall -g -O2") -ENABLE_TESTING() - -# Add Boost include path for MacOS -INCLUDE_DIRECTORIES(/usr/local/include) -# Add Session related include files -INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/../main/generated-sources-cpp) -# Add Thrift include directory -INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/../../thrift/include) - -# Add Boost library headers for MaxOS -FIND_PACKAGE(Boost REQUIRED) -IF (DEFINED BOOST_INCLUDEDIR) - include_directories("${Boost_INCLUDE_DIR}") -ENDIF() - -# Link directories are different for Windows and Linux/Mac -IF(MSVC) - LINK_DIRECTORIES(${CMAKE_SOURCE_DIR}/../main/Release) - SET(THRIFT_STATIC_LIB "${CMAKE_SOURCE_DIR}/../../thrift/lib/Release/thriftmd.lib") -ELSE() - LINK_DIRECTORIES(${CMAKE_SOURCE_DIR}/../main) - SET(THRIFT_STATIC_LIB "${CMAKE_SOURCE_DIR}/../../thrift/lib/libthrift.a") -ENDIF() - -ADD_EXECUTABLE(${TARGET_NAME} main.cpp cpp/sessionIT.cpp) - -# Link with shared library iotdb_session and pthread -IF(MSVC) - TARGET_LINK_LIBRARIES(${TARGET_NAME} iotdb_session ${THRIFT_STATIC_LIB}) -ELSE() - TARGET_LINK_LIBRARIES(${TARGET_NAME} iotdb_session pthread) -ENDIF() -TARGET_INCLUDE_DIRECTORIES(${TARGET_NAME} PUBLIC ./catch2/) - -# Add 'sessionIT' to the project to be run by ctest -IF(MSVC) - ADD_TEST(NAME sessionIT CONFIGURATIONS Release COMMAND ${TARGET_NAME}) -ELSE() - ADD_TEST(NAME sessionIT COMMAND ${TARGET_NAME}) -ENDIF() diff --git a/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp b/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp deleted file mode 100644 index a527975589435..0000000000000 --- a/iotdb-client/client-cpp/src/test/cpp/sessionIT.cpp +++ /dev/null @@ -1,933 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -#include "catch.hpp" -#include "Session.h" -#include "SessionPool.h" -#include "TsBlock.h" -#include - -#include -#include - -using namespace std; - -extern std::shared_ptr session; - -static vector testTimeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3"}; - -void prepareTimeseries() { - for (const string ×eries: testTimeseries) { - if (session->checkTimeseriesExists(timeseries)) { - session->deleteTimeseries(timeseries); - } - session->createTimeseries(timeseries, TSDataType::INT64, TSEncoding::RLE, CompressionType::SNAPPY); - } -} - -static int global_test_id = 0; -class CaseReporter -{ -public: - CaseReporter(const char *caseNameArg) : caseName(caseNameArg) - { - test_id = global_test_id++; - std::cout << "Test " << test_id << ": " << caseName << std::endl; - } - ~CaseReporter() - { - std::cout << "Test " << test_id << ": " << caseName << " Done"<< std::endl << std::endl; - } -private: - const char *caseName; - int test_id; -}; - -TEST_CASE("Create timeseries success", "[createTimeseries]") { - CaseReporter cr("createTimeseries"); - if (!session->checkTimeseriesExists("root.test.d1.s1")) { - session->createTimeseries("root.test.d1.s1", TSDataType::INT64, TSEncoding::RLE, CompressionType::SNAPPY); - } - REQUIRE(session->checkTimeseriesExists("root.test.d1.s1") == true); - session->deleteTimeseries("root.test.d1.s1"); -} - -TEST_CASE("Delete timeseries success", "[deleteTimeseries]") { - CaseReporter cr("deleteTimeseries"); - if (!session->checkTimeseriesExists("root.test.d1.s1")) { - session->createTimeseries("root.test.d1.s1", TSDataType::INT64, TSEncoding::RLE, CompressionType::SNAPPY); - } - REQUIRE(session->checkTimeseriesExists("root.test.d1.s1") == true); - session->deleteTimeseries("root.test.d1.s1"); - REQUIRE(session->checkTimeseriesExists("root.test.d1.s1") == false); -} - -TEST_CASE("Test insertRecord by string", "[testInsertRecord]") { - CaseReporter cr("testInsertRecord"); - prepareTimeseries(); - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3"}; - - for (long time = 0; time < 100; time++) { - vector values = {"1", "2", "3"}; - session->insertRecord(deviceId, time, measurements, values); - } - - session->executeNonQueryStatement("insert into root.test.d1(timestamp,s1, s2, s3) values(100, 1,2,3)"); - - - unique_ptr sessionDataSet = session->executeQueryStatement("select * from root.ln.设备"); - - sessionDataSet->setFetchSize(1024); - while (sessionDataSet->hasNext()) { - auto record = sessionDataSet->next(); - std::cout << record->toString() << std::endl; - } - - // unique_ptr sessionDataSet = session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); - // sessionDataSet->setFetchSize(1024); - // int count = 0; - // while (sessionDataSet->hasNext()) { - // long index = 1; - // count++; - // for (const Field &f: sessionDataSet->next()->fields) { - // REQUIRE(f.longV == index); - // index++; - // } - // } - // REQUIRE(count == 101); -} - -TEST_CASE("Test insertRecords ", "[testInsertRecords]") { - CaseReporter cr("testInsertRecords"); - prepareTimeseries(); - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3"}; - vector deviceIds; - vector> measurementsList; - vector> valuesList; - vector timestamps; - - int64_t COUNT = 500; - for (int64_t time = 1; time <= COUNT; time++) { - vector values = {"1", "2", "3"}; - - deviceIds.push_back(deviceId); - measurementsList.push_back(measurements); - valuesList.push_back(values); - timestamps.push_back(time); - if (time != 0 && time % 100 == 0) { - session->insertRecords(deviceIds, timestamps, measurementsList, valuesList); - deviceIds.clear(); - measurementsList.clear(); - valuesList.clear(); - timestamps.clear(); - } - } - - if (timestamps.size() > 0) { - session->insertRecords(deviceIds, timestamps, measurementsList, valuesList); - } - - unique_ptr sessionDataSet = session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); - sessionDataSet->setFetchSize(1024); - int count = 0; - while (sessionDataSet->hasNext()) { - long index = 1; - count++; - for (const Field &f: sessionDataSet->next()->fields) { - REQUIRE(f.longV == index); - index++; - } - } - REQUIRE(count == COUNT); -} - -TEST_CASE("Test insertRecord with types ", "[testTypedInsertRecord]") { - CaseReporter cr("testTypedInsertRecord"); - vector timeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3"}; - vector types = {TSDataType::INT32, TSDataType::DOUBLE, TSDataType::INT64}; - - for (size_t i = 0; i < timeseries.size(); i++) { - if (session->checkTimeseriesExists(timeseries[i])) { - session->deleteTimeseries(timeseries[i]); - } - session->createTimeseries(timeseries[i], types[i], TSEncoding::RLE, CompressionType::SNAPPY); - } - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3"}; - vector value1(100, 1); - vector value2(100, 2.2); - vector value3(100, 3); - - for (long time = 0; time < 100; time++) { - vector values = {(char *) (&value1[time]), (char *) (&value2[time]), (char *) (&value3[time])}; - session->insertRecord(deviceId, time, measurements, types, values); - } - - unique_ptr sessionDataSet = session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); - sessionDataSet->setFetchSize(1024); - long count = 0; - while (sessionDataSet->hasNext()) { - sessionDataSet->next(); - count++; - } - REQUIRE(count == 100); -} - - -TEST_CASE("Test insertRecord with new datatypes ", "[testTypedInsertRecordNewDatatype]") { - CaseReporter cr("testTypedInsertRecordNewDatatype"); - vector timeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3", "root.test.d1.s4"}; - std::vector types = {TSDataType::TIMESTAMP, - TSDataType::DATE, TSDataType::BLOB, TSDataType::STRING}; - - for (size_t i = 0; i < timeseries.size(); i++) { - if (session->checkTimeseriesExists(timeseries[i])) { - session->deleteTimeseries(timeseries[i]); - } - session->createTimeseries(timeseries[i], types[i], TSEncoding::PLAIN, CompressionType::SNAPPY); - } - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3", "s4"}; - int64_t value1 = 20250507; - boost::gregorian::date value2 = boost::gregorian::date(2025, 5, 7); - string value3 = "20250507"; - string value4 = "20250507"; - - for (long time = 0; time < 100; time++) { - vector values = {(char *) (&value1), (char *) (&value2), - const_cast(value3.c_str()), const_cast(value4.c_str())}; - session->insertRecord(deviceId, time, measurements, types, values); - } - - unique_ptr sessionDataSet = session->executeQueryStatement("select s1,s2,s3,s4 from root.test.d1"); - sessionDataSet->setFetchSize(1024); - long count = 0; - while (sessionDataSet->hasNext()) { - auto record = sessionDataSet->next(); - REQUIRE(record->fields.size() == 4); - for (int i = 0; i < 4; i++) { - REQUIRE(types[i] == record->fields[i].dataType); - } - REQUIRE(record->fields[0].longV == value1); - REQUIRE(record->fields[1].dateV == value2); - REQUIRE(record->fields[2].stringV == value3); - REQUIRE(record->fields[3].stringV == value4); - count++; - } - REQUIRE(count == 100); -} - -TEST_CASE("Test insertRecords with types ", "[testTypedInsertRecords]") { - CaseReporter cr("testTypedInsertRecords"); - vector timeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3"}; - vector types = {TSDataType::INT32, TSDataType::DOUBLE, TSDataType::INT64}; - - for (size_t i = 0; i < timeseries.size(); i++) { - if (session->checkTimeseriesExists(timeseries[i])) { - session->deleteTimeseries(timeseries[i]); - } - session->createTimeseries(timeseries[i], types[i], TSEncoding::RLE, CompressionType::SNAPPY); - } - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3"}; - vector deviceIds; - vector> measurementsList; - vector> typesList; - vector> valuesList; - vector timestamps; - vector value1(100, 1); - vector value2(100, 2.2); - vector value3(100, 3); - - for (int64_t time = 0; time < 100; time++) { - vector values = {(char *) (&value1[time]), (char *) (&value2[time]), (char *) (&value3[time])}; - deviceIds.push_back(deviceId); - measurementsList.push_back(measurements); - typesList.push_back(types); - valuesList.push_back(values); - timestamps.push_back(time); - } - - session->insertRecords(deviceIds, timestamps, measurementsList, typesList, valuesList); - - unique_ptr sessionDataSet = session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); - sessionDataSet->setFetchSize(1024); - int count = 0; - while (sessionDataSet->hasNext()) { - sessionDataSet->next(); - count++; - } - REQUIRE(count == 100); -} - -TEST_CASE("Test insertRecordsOfOneDevice", "[testInsertRecordsOfOneDevice]") { - CaseReporter cr("testInsertRecordsOfOneDevice"); - vector timeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3"}; - vector types = {TSDataType::INT32, TSDataType::DOUBLE, TSDataType::INT64}; - - for (size_t i = 0; i < timeseries.size(); i++) { - if (session->checkTimeseriesExists(timeseries[i])) { - session->deleteTimeseries(timeseries[i]); - } - session->createTimeseries(timeseries[i], types[i], TSEncoding::RLE, CompressionType::SNAPPY); - } - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3"}; - vector> measurementsList; - vector> typesList; - vector> valuesList; - vector timestamps; - vector value1(100, 1); - vector value2(100, 2.2); - vector value3(100, 3); - - for (int64_t time = 0; time < 100; time++) { - vector values = {(char *) (&value1[time]), (char *) (&value2[time]), (char *) (&value3[time])}; - measurementsList.push_back(measurements); - typesList.push_back(types); - valuesList.push_back(values); - timestamps.push_back(time); - } - - session->insertRecordsOfOneDevice(deviceId, timestamps, measurementsList, typesList, valuesList); - - unique_ptr sessionDataSet = session->executeQueryStatement("select * from root.test.d1"); - sessionDataSet->setFetchSize(1024); - int count = 0; - while (sessionDataSet->hasNext()) { - sessionDataSet->next(); - count++; - } - REQUIRE(count == 100); -} - -TEST_CASE("Test insertTablet ", "[testInsertTablet]") { - CaseReporter cr("testInsertTablet"); - prepareTimeseries(); - string deviceId = "root.test.d1"; - vector> schemaList; - schemaList.emplace_back("s1", TSDataType::INT64); - schemaList.emplace_back("s2", TSDataType::INT64); - schemaList.emplace_back("s3", TSDataType::INT64); - - Tablet tablet(deviceId, schemaList, 100); - for (int64_t time = 0; time < 100; time++) { - int row = tablet.rowSize++; - tablet.timestamps[row] = time; - for (int64_t i = 0; i < 3; i++) { - tablet.addValue(i, row, i); - } - if (tablet.rowSize == tablet.maxRowNumber) { - session->insertTablet(tablet); - tablet.reset(); - } - } - - if (tablet.rowSize != 0) { - session->insertTablet(tablet); - tablet.reset(); - } - unique_ptr sessionDataSet = session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); - sessionDataSet->setFetchSize(1024); - int count = 0; - while (sessionDataSet->hasNext()) { - long index = 0; - count++; - for (const Field& f: sessionDataSet->next()->fields) { - REQUIRE(f.longV == index); - index++; - } - } - REQUIRE(count == 100); -} - -TEST_CASE("Test insertTablets ", "[testInsertTablets]") { - CaseReporter cr("testInsertTablets"); - vector testTimeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3", - "root.test.d2.s1", "root.test.d2.s2", "root.test.d2.s3"}; - for (const string ×eries: testTimeseries) { - if (session->checkTimeseriesExists(timeseries)) { - session->deleteTimeseries(timeseries); - } - session->createTimeseries(timeseries, TSDataType::INT64, TSEncoding::RLE, CompressionType::SNAPPY); - } - vector> schemaList; - schemaList.emplace_back("s1", TSDataType::INT64); - schemaList.emplace_back("s2", TSDataType::INT64); - schemaList.emplace_back("s3", TSDataType::INT64); - - int maxRowNumber = 100; - vector deviceIds = {"root.test.d1", "root.test.d2"}; - vector tablets; - for (const auto& deviceId: deviceIds) { - tablets.emplace_back(deviceId, schemaList, maxRowNumber); - } - for (auto& tablet : tablets) { - for (int64_t time = 0; time < maxRowNumber; time++) { - int row = tablet.rowSize++; - tablet.timestamps[row] = time; - for (int64_t i = 0; i < 3; i++) { - tablet.addValue(i, row, i); - } - } - } - unordered_map tabletsMap; - for (auto& tablet : tablets) { - tabletsMap[tablet.deviceId] = &tablet; - } - session->insertTablets(tabletsMap); - - unique_ptr sessionDataSet = session->executeQueryStatement("select s1,s2,s3 from root.test.d2"); - sessionDataSet->setFetchSize(1024); - int count = 0; - while (sessionDataSet->hasNext()) { - long index = 0; - count++; - for (const Field& f: sessionDataSet->next()->fields) { - REQUIRE(f.longV == index); - index++; - } - } - REQUIRE(count == 100); -} - -TEST_CASE("Test insertTablet new datatype", "[testInsertTabletNewDatatype]") { - CaseReporter cr("testInsertTabletNewDatatype"); - string deviceId = "root.test.d2"; - vector> schemaList; - std::vector measurements = {"s1", "s2", "s3", "s4"}; - std::vector dataTypes = {TSDataType::TIMESTAMP, - TSDataType::DATE, TSDataType::BLOB, TSDataType::STRING}; - for (int i = 0; i < 4; i++) { - schemaList.emplace_back(measurements[i], dataTypes[i]); - } - - for (int i = 0; i < 4; i++) { - auto timeseries = deviceId + "." + measurements[i]; - if (session->checkTimeseriesExists(timeseries)) { - session->deleteTimeseries(timeseries); - } - session->createTimeseries(timeseries, dataTypes[i], TSEncoding::PLAIN, CompressionType::UNCOMPRESSED); - } - - int64_t s1Value = 20250507; - boost::gregorian::date s2Value(2025, 5, 7); - std::string s3Value("20250507"); - std::string s4Value("20250507"); - - Tablet tablet(deviceId, schemaList, 100); - for (int64_t time = 0; time < 100; time++) { - int row = tablet.rowSize++; - tablet.timestamps[row] = time; - tablet.addValue(0, row, s1Value); - tablet.addValue(1, row, s2Value); - tablet.addValue(2, row, s3Value); - tablet.addValue(3, row, s4Value); - if (tablet.rowSize == tablet.maxRowNumber) { - session->insertTablet(tablet); - tablet.reset(); - } - } - - if (tablet.rowSize != 0) { - session->insertTablet(tablet); - tablet.reset(); - } - unique_ptr sessionDataSet = session->executeQueryStatement("select s1,s2,s3,s4 from root.test.d2"); - sessionDataSet->setFetchSize(1024); - int count = 0; - while (sessionDataSet->hasNext()) { - auto record = sessionDataSet->next(); - REQUIRE(record->fields.size() == 4); - for (int i = 0; i < 4; i++) { - REQUIRE(dataTypes[i] == record->fields[i].dataType); - } - REQUIRE(record->fields[0].longV == s1Value); - REQUIRE(record->fields[1].dateV == s2Value); - REQUIRE(record->fields[2].stringV == s3Value); - REQUIRE(record->fields[3].stringV == s4Value); - count++; - } - REQUIRE(count == 100); -} - -TEST_CASE("Test Last query ", "[testLastQuery]") { - CaseReporter cr("testLastQuery"); - prepareTimeseries(); - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3"}; - - for (long time = 0; time < 100; time++) { - vector values = {"1", "2", "3"}; - session->insertRecord(deviceId, time, measurements, values); - } - - vector measurementValues = {"1", "2", "3"}; - unique_ptr sessionDataSet = session->executeQueryStatement( - "select last s1,s2,s3 from root.test.d1"); - sessionDataSet->setFetchSize(1024); - long index = 0; - while (sessionDataSet->hasNext()) { - vector fields = sessionDataSet->next()->fields; - REQUIRE("1" <= fields[1].stringV); - REQUIRE(fields[1].stringV <= "3"); - index++; - } -} - -TEST_CASE("Test Huge query ", "[testHugeQuery]") { - CaseReporter cr("testHugeQuery"); - prepareTimeseries(); - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3"}; - vector types = {TSDataType::INT64, TSDataType::INT64, TSDataType::INT64}; - int64_t value1 = 1, value2 = 2, value3 = 3; - vector values = {(char*)&value1, (char*)&value2, (char*)&value3}; - - long total_count = 500000; - int print_count = 0; - std::cout.width(7); - std::cout << "inserting " << total_count << " rows:" << std::endl; - for (long time = 0; time < total_count; time++) { - session->insertRecord(deviceId, time, measurements, types, values); - if (time != 0 && time % 1000 == 0) { - std::cout << time << "\t" << std::flush; - if (++print_count % 20 == 0) { - std::cout << std::endl; - } - } - } - - unique_ptr sessionDataSet = session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); - sessionDataSet->setFetchSize(1024); - int count = 0; - print_count = 0; - std::cout << "\n\niterating " << total_count << " rows:" << std::endl; - while (sessionDataSet->hasNext()) { - auto rowRecord = sessionDataSet->next(); - REQUIRE(rowRecord->timestamp == count); - REQUIRE(rowRecord->fields[0].longV== 1); - REQUIRE(rowRecord->fields[1].longV == 2); - REQUIRE(rowRecord->fields[2].longV == 3); - count++; - if (count % 1000 == 0) { - std::cout << count << "\t" << std::flush; - if (++print_count % 20 == 0) { - std::cout << std::endl; - } - } - } - - REQUIRE(count == total_count); -} - - -TEST_CASE("Test executeRawDataQuery ", "[executeRawDataQuery]") { - CaseReporter cr("executeRawDataQuery"); - prepareTimeseries(); - - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3"}; - vector types = {TSDataType::INT64, TSDataType::INT64, TSDataType::INT64}; - - long total_count = 5000; - vector values; - int64_t valueArray[3]; - for (long time = -total_count; time < total_count; time++) { - valueArray[0] = time; - valueArray[1] = time * 2; - valueArray[2] = time * 3; - values.clear(); - values.push_back((char*)&valueArray[0]); - values.push_back((char*)&valueArray[1]); - values.push_back((char*)&valueArray[2]); - session->insertRecord(deviceId, time, measurements, types, values); - if (time == 100) { //insert 1 big timestamp data for generate un-seq data. - valueArray[0] = 9; - valueArray[2] = 999; - values.clear(); - values.push_back((char*)&valueArray[0]); - values.push_back((char*)&valueArray[2]); - vector measurements2 = {"s1", "s3"}; - vector types2 = {TSDataType::INT64, TSDataType::INT64}; - session->insertRecord(deviceId, 99999, measurements2, types2, values); - } - } - - vector paths; - paths.push_back("root.test.d1.s1"); - paths.push_back("root.test.d1.s2"); - paths.push_back("root.test.d1.s3"); - - //== Test executeRawDataQuery() with negative timestamp - int startTs = -total_count, endTs = total_count; - unique_ptr sessionDataSet = session->executeRawDataQuery(paths, startTs, endTs); - sessionDataSet->setFetchSize(10); - vector columns = sessionDataSet->getColumnNames(); - columns = sessionDataSet->getColumnNames(); - for (const string &column : columns) { - cout << column << " " ; - } - cout << endl; - REQUIRE(columns[0] == "Time"); - REQUIRE(columns[1] == paths[0]); - REQUIRE(columns[2] == paths[1]); - REQUIRE(columns[3] == paths[2]); - - int ts = startTs; - while (sessionDataSet->hasNext()) { - auto rowRecordPtr = sessionDataSet->next(); - //cout << rowRecordPtr->toString(); - - vector fields = rowRecordPtr->fields; - REQUIRE(rowRecordPtr->timestamp == ts); - REQUIRE(fields[0].dataType == TSDataType::INT64); - REQUIRE(fields[0].longV == ts); - REQUIRE(fields[1].dataType == TSDataType::INT64); - REQUIRE(fields[1].longV == ts * 2); - REQUIRE(fields[2].dataType == TSDataType::INT64); - REQUIRE(fields[2].longV == ts *3); - ts++; - } - - - //== Test executeRawDataQuery() with null field - startTs = 99999; - endTs = 99999 + 10; - sessionDataSet = session->executeRawDataQuery(paths, startTs, endTs); - - sessionDataSet->setFetchSize(10); - columns = sessionDataSet->getColumnNames(); - for (const string &column : columns) { - cout << column << " " ; - } - cout << endl; - REQUIRE(columns[0] == "Time"); - REQUIRE(columns[1] == paths[0]); - REQUIRE(columns[2] == paths[1]); - REQUIRE(columns[3] == paths[2]); - ts = startTs; - while (sessionDataSet->hasNext()) { - auto rowRecordPtr = sessionDataSet->next(); - cout << rowRecordPtr->toString(); - - vector fields = rowRecordPtr->fields; - REQUIRE(rowRecordPtr->timestamp == ts); - REQUIRE(fields[0].dataType == TSDataType::INT64); - REQUIRE(fields[0].longV == 9); - REQUIRE(fields[1].dataType == TSDataType::UNKNOWN); - REQUIRE(fields[2].dataType == TSDataType::INT64); - REQUIRE(fields[2].longV == 999); - } - - //== Test executeRawDataQuery() with empty data - sessionDataSet = session->executeRawDataQuery(paths, 100000, 110000); - sessionDataSet->setFetchSize(1); - REQUIRE(sessionDataSet->hasNext() == false); -} - -TEST_CASE("Test executeLastDataQuery ", "[testExecuteLastDataQuery]") { - CaseReporter cr("testExecuteLastDataQuery"); - prepareTimeseries(); - - string deviceId = "root.test.d1"; - vector measurements = {"s1", "s2", "s3"}; - vector types = {TSDataType::INT64, TSDataType::INT64, TSDataType::INT64}; - - long total_count = 5000; - vector values; - int64_t valueArray[3]; - for (long time = -total_count; time < total_count; time++) { - valueArray[0] = time; - valueArray[1] = time * 2; - valueArray[2] = time * 3; - values.clear(); - values.push_back((char*)&valueArray[0]); - values.push_back((char*)&valueArray[1]); - values.push_back((char*)&valueArray[2]); - session->insertRecord(deviceId, time, measurements, types, values); - if (time == 100) { //insert 1 big timestamp data for gen unseq data. - valueArray[0] = 9; - valueArray[2] = 999; - values.clear(); - values.push_back((char*)&valueArray[0]); - values.push_back((char*)&valueArray[2]); - vector measurements2 = {"s1", "s3"}; - vector types2 = {TSDataType::INT64, TSDataType::INT64}; - session->insertRecord(deviceId, 99999, measurements2, types2, values); - } - } - - int64_t tsCheck[3] = {99999, 4999, 99999}; - std::vector valueCheck = {"9", "9998", "999"}; - - vector paths; - paths.push_back("root.test.d1.s1"); - paths.push_back("root.test.d1.s2"); - paths.push_back("root.test.d1.s3"); - - //== Test executeLastDataQuery() without lastTime - unique_ptr sessionDataSet = session->executeLastDataQuery(paths); - sessionDataSet->setFetchSize(1); - - vector columns = sessionDataSet->getColumnNames(); - for (const string &column : columns) { - cout << column << " " ; - } - cout << endl; - - int index = 0; - while (sessionDataSet->hasNext()) { - auto rowRecordPtr = sessionDataSet->next(); - cout << rowRecordPtr->toString(); - - vector fields = rowRecordPtr->fields; - REQUIRE(rowRecordPtr->timestamp == tsCheck[index]); - REQUIRE(fields[0].stringV == paths[index]); - REQUIRE(fields[1].stringV == valueCheck[index]); - REQUIRE(fields[2].stringV == "INT64"); - index++; - } - - //== Test executeLastDataQuery() with negative lastTime - sessionDataSet = session->executeLastDataQuery(paths, -200); - sessionDataSet->setFetchSize(1); - columns = sessionDataSet->getColumnNames(); - for (const string &column : columns) { - cout << column << " " ; - } - cout << endl; - - index = 0; - while (sessionDataSet->hasNext()) { - auto rowRecordPtr = sessionDataSet->next(); - cout << rowRecordPtr->toString(); - - vector fields = rowRecordPtr->fields; - REQUIRE(rowRecordPtr->timestamp == tsCheck[index]); - REQUIRE(fields[0].stringV == paths[index]); - REQUIRE(fields[1].stringV == valueCheck[index]); - REQUIRE(fields[2].stringV == "INT64"); - index++; - } - - //== Test executeLastDataQuery() with the lastTime that is > largest timestamp. - sessionDataSet = session->executeLastDataQuery(paths, 100000); - sessionDataSet->setFetchSize(1024); - REQUIRE(sessionDataSet->hasNext() == false); -} - -// Helper function for comparing TEndPoint with detailed error message -void assertTEndPointEqual(const TEndPoint& actual, - const std::string& expectedIp, - int expectedPort, - const char* file, - int line) { - if (actual.ip != expectedIp || actual.port != expectedPort) { - std::stringstream ss; - ss << "\nTEndPoint mismatch:\nExpected: " << expectedIp << ":" << expectedPort - << "\nActual: " << actual.ip << ":" << actual.port; - Catch::SourceLineInfo location(file, line); - Catch::AssertionHandler handler("TEndPoint comparison", location, ss.str(), Catch::ResultDisposition::Normal); - handler.handleMessage(Catch::ResultWas::ExplicitFailure, ss.str()); - handler.complete(); - } -} - -// Macro to simplify test assertions -#define REQUIRE_TENDPOINT(actual, expectedIp, expectedPort) \ - assertTEndPointEqual(actual, expectedIp, expectedPort, __FILE__, __LINE__) - -TEST_CASE("UrlUtils - parseTEndPointIpv4AndIpv6Url", "[UrlUtils]") { - // Test valid IPv4 addresses - SECTION("Valid IPv4") { - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("192.168.1.1:8080"), "192.168.1.1", 8080); - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("10.0.0.1:80"), "10.0.0.1", 80); - } - - // Test valid IPv6 addresses - SECTION("Valid IPv6") { - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("[2001:db8::1]:8080"), "2001:db8::1", 8080); - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("[::1]:80"), "::1", 80); - } - - // Test hostnames - SECTION("Hostnames") { - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("localhost:8080"), "localhost", 8080); - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("example.com:443"), "example.com", 443); - } - - // Test edge cases - SECTION("Edge cases") { - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url(""), "", 0); - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("127.0.0.1"), "127.0.0.1", 0); - } - - // Test invalid inputs - SECTION("Invalid inputs") { - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("192.168.1.1:abc"), "192.168.1.1:abc", 0); - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("]invalid[:80"), "]invalid[", 80); - } - - // Test port ranges - SECTION("Port ranges") { - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("localhost:0"), "localhost", 0); - REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("127.0.0.1:65535"), "127.0.0.1", 65535); - } -} - -TEST_CASE("TsBlock deserialize rejects truncated malicious payload", "[TsBlockDeserialize]") { - std::string data(18, '\0'); - data[3] = '\x10'; - REQUIRE_THROWS_AS(TsBlock::deserialize(data), IoTDBException); -} - -TEST_CASE("SessionPool basic borrow/insert/query via RAII lease", "[sessionPool]") { - CaseReporter cr("SessionPool basic"); - auto pool = SessionPoolBuilder() - .host("127.0.0.1") - ->rpcPort(6667) - ->username("root") - ->password("root") - ->maxSize(3) - ->build(); - - { - PooledSession s = pool->getSession(); - try { - s->executeNonQueryStatement("delete timeseries root.test.pool.d1.*"); - } catch (const std::exception&) { - // Ignore: the timeseries may not exist yet on a fresh database. - } - } - - const int rows = 50; - for (int i = 0; i < rows; i++) { - PooledSession s = pool->getSession(); - s->insertRecord("root.test.pool.d1", i, {"s1"}, {to_string(i)}); - } - - int count = 0; - { - PooledSessionDataSet ds = pool->executeQueryStatement("select s1 from root.test.pool.d1"); - while (ds->hasNext()) { - ds->next(); - count++; - } - } - REQUIRE(count == rows); - - { - PooledSession s = pool->getSession(); - s->executeNonQueryStatement("delete timeseries root.test.pool.d1.*"); - } - pool->close(); -} - -TEST_CASE("SessionPool is safe under concurrent writers", "[sessionPool]") { - CaseReporter cr("SessionPool concurrency"); - auto pool = SessionPoolBuilder() - .host("127.0.0.1") - ->rpcPort(6667) - ->username("root") - ->password("root") - ->maxSize(4) - ->build(); - - { - PooledSession s = pool->getSession(); - try { - s->executeNonQueryStatement("delete timeseries root.test.pool.d2.*"); - } catch (const std::exception&) { - // Ignore: the timeseries may not exist yet on a fresh database. - } - } - - const int threadCount = 8; - const int rowsPerThread = 100; - std::atomic failures(0); - std::vector threads; - for (int t = 0; t < threadCount; t++) { - threads.emplace_back([&pool, t, rowsPerThread, &failures]() { - try { - for (int i = 0; i < rowsPerThread; i++) { - int64_t ts = static_cast(t) * rowsPerThread + i; - // Mix RAII and convenience APIs to exercise both borrow paths. - if (i % 2 == 0) { - pool->insertRecord("root.test.pool.d2", ts, {"s1"}, {to_string(ts)}); - } else { - PooledSession s = pool->getSession(); - s->insertRecord("root.test.pool.d2", ts, {"s1"}, {to_string(ts)}); - } - } - } catch (const std::exception& e) { - std::cerr << "writer thread failed: " << e.what() << std::endl; - failures++; - } - }); - } - for (auto& th : threads) { - th.join(); - } - REQUIRE(failures.load() == 0); - REQUIRE(pool->getMaxSize() == 4); - - int count = 0; - { - PooledSessionDataSet ds = pool->executeQueryStatement("select s1 from root.test.pool.d2"); - while (ds->hasNext()) { - ds->next(); - count++; - } - } - REQUIRE(count == threadCount * rowsPerThread); - - { - PooledSession s = pool->getSession(); - s->executeNonQueryStatement("delete timeseries root.test.pool.d2.*"); - } - pool->close(); -} - -TEST_CASE("SessionPool getSession times out when exhausted", "[sessionPool]") { - CaseReporter cr("SessionPool exhaustion timeout"); - auto pool = SessionPoolBuilder() - .host("127.0.0.1") - ->rpcPort(6667) - ->username("root") - ->password("root") - ->maxSize(1) - ->waitToGetSessionTimeoutMs(200) - ->build(); - - PooledSession held = pool->getSession(); - REQUIRE(static_cast(held)); - REQUIRE(pool->activeCount() == 1); - // The only Session is checked out, so a second borrow must time out. - REQUIRE_THROWS_AS(pool->getSession(), IoTDBException); - - held.release(); - // After returning it, a borrow succeeds again. - PooledSession reused = pool->getSession(); - REQUIRE(static_cast(reused)); - reused.release(); - pool->close(); -} diff --git a/iotdb-client/client-cpp/test/CMakeLists.txt b/iotdb-client/client-cpp/test/CMakeLists.txt new file mode 100644 index 0000000000000..921b9bddd1bd6 --- /dev/null +++ b/iotdb-client/client-cpp/test/CMakeLists.txt @@ -0,0 +1,94 @@ +# 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. +# +# ============================================================================= +# C++ client integration tests (driven by ctest). +# The parent CMakeLists.txt include()s this folder when BUILD_TESTING=ON; +# it relies on the iotdb_session target (and iotdb_thrift_static) already +# being defined at the parent scope. +# ============================================================================= + +include(CTest) + +set(CATCH2_URL + "https://github.com/catchorg/Catch2/releases/download/v2.13.7/catch.hpp") +if(CATCH2_INCLUDE_DIR) + set(_catch2_include_dir "${CATCH2_INCLUDE_DIR}") +else() + set(_catch2_include_dir "${CMAKE_CURRENT_BINARY_DIR}/catch2") +endif() +set(_catch2_header "${_catch2_include_dir}/catch.hpp") +if(NOT EXISTS "${_catch2_header}") + file(MAKE_DIRECTORY "${_catch2_include_dir}") + message(STATUS "Downloading Catch2 from ${CATCH2_URL}") + file(DOWNLOAD "${CATCH2_URL}" "${_catch2_header}" SHOW_PROGRESS TLS_VERIFY ON) +endif() + +set(_test_targets + session_tests + session_c_tests) + +add_executable(session_tests main.cpp cpp/sessionIT.cpp) +add_executable(session_c_tests main_c.cpp cpp/sessionCIT.cpp) + +foreach(_t IN LISTS _test_targets) + target_include_directories(${_t} PRIVATE + "${_catch2_include_dir}" + "${CMAKE_CURRENT_SOURCE_DIR}/../src/rpc" + "${THRIFT_GEN_CPP_DIR}" + "${THRIFT_INCLUDE_DIR}") + if(BOOST_INCLUDE_DIR) + target_include_directories(${_t} PRIVATE "${BOOST_INCLUDE_DIR}") + endif() + target_link_libraries(${_t} PRIVATE iotdb_session) + if(WITH_SSL) + target_link_libraries(${_t} PRIVATE OpenSSL::SSL OpenSSL::Crypto) + endif() +endforeach() + +if(CMAKE_CXX_COMPILER_ID MATCHES "GNU|Clang" AND NOT MSVC) + foreach(_t IN LISTS _test_targets) + target_compile_options(${_t} PRIVATE -fsanitize=address -fno-omit-frame-pointer) + target_link_options(${_t} PRIVATE -fsanitize=address) + endforeach() +endif() + +# Linux: keep iotdb_session in the executable's needed-list even when there +# is no direct symbol reference from main.cpp (the IT helpers pull it in +# transitively); without this, ld may drop the lib at link time. +if(UNIX AND NOT APPLE) + foreach(_t IN LISTS _test_targets) + target_link_options(${_t} PRIVATE -Wl,--no-as-needed) + endforeach() +endif() + +# Register ctest cases (multi-config on MSVC). +if(MSVC) + add_test(NAME sessionIT CONFIGURATIONS Release COMMAND session_tests) + add_test(NAME sessionCIT CONFIGURATIONS Release COMMAND session_c_tests) + foreach(_t IN LISTS _test_targets) + add_custom_command(TARGET ${_t} POST_BUILD + COMMAND ${CMAKE_COMMAND} -E copy_if_different + $ $) + endforeach() +else() + add_test(NAME sessionIT COMMAND session_tests) + add_test(NAME sessionCIT COMMAND session_c_tests) +endif() + +# Run sequentially: parallel ctest overloads the single local IoTDB instance. +set_tests_properties(sessionIT sessionCIT PROPERTIES RUN_SERIAL TRUE) diff --git a/iotdb-client/client-cpp/test/catch2/.gitignore b/iotdb-client/client-cpp/test/catch2/.gitignore new file mode 100644 index 0000000000000..39e8e4e792225 --- /dev/null +++ b/iotdb-client/client-cpp/test/catch2/.gitignore @@ -0,0 +1,4 @@ +# Catch2 single-header (downloaded at build time by Maven or CMake) +catch.hpp +* +!.gitignore diff --git a/iotdb-client/client-cpp/test/cpp/sessionCIT.cpp b/iotdb-client/client-cpp/test/cpp/sessionCIT.cpp new file mode 100644 index 0000000000000..48ce21aeba256 --- /dev/null +++ b/iotdb-client/client-cpp/test/cpp/sessionCIT.cpp @@ -0,0 +1,748 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "catch.hpp" +#include "SessionC.h" +#include +#include +#include +#include +#include + +extern CSession* g_session; + +static int global_test_id = 0; + +class CaseReporter { +public: + CaseReporter(const char* caseNameArg) : caseName(caseNameArg) { + test_id = global_test_id++; + std::cout << "C-API Test " << test_id << ": " << caseName << std::endl; + } + ~CaseReporter() { + std::cout << "C-API Test " << test_id << ": " << caseName << " Done" << std::endl << std::endl; + } + +private: + const char* caseName; + int test_id; +}; + +static const char* testTimeseries[] = {"root.ctest.d1.s1", "root.ctest.d1.s2", "root.ctest.d1.s3"}; +static const int testTimeseriesCount = 3; + +static void dropTimeseriesIfExists(CSession* session, const char* path) { + bool exists = false; + REQUIRE(ts_session_check_timeseries_exists(session, path, &exists) == TS_OK); + if (exists) { + REQUIRE(ts_session_delete_timeseries(session, path) == TS_OK); + } +} + +static void ensureTimeseries(CSession* session, const char* path, TSDataType_C type, + TSEncoding_C encoding, TSCompressionType_C compression) { + dropTimeseriesIfExists(session, path); + REQUIRE(ts_session_create_timeseries(session, path, type, encoding, compression) == TS_OK); +} + +static int queryRowCount(CSession* session, const char* sql, int fetchSize = 1024) { + CSessionDataSet* dataSet = nullptr; + REQUIRE(ts_session_execute_query(session, sql, &dataSet) == TS_OK); + REQUIRE(dataSet != nullptr); + ts_dataset_set_fetch_size(dataSet, fetchSize); + int count = 0; + while (ts_dataset_has_next(dataSet)) { + CRowRecord* record = ts_dataset_next(dataSet); + REQUIRE(record != nullptr); + ++count; + ts_row_record_destroy(record); + } + ts_dataset_destroy(dataSet); + return count; +} + +static void dropDatabaseIfExists(CSession* session, const char* database) { + TsStatus status = ts_session_delete_database(session, database); + (void)status; +} + +static void prepareTimeseries() { + for (int i = 0; i < testTimeseriesCount; i++) { + ensureTimeseries(g_session, testTimeseries[i], TS_TYPE_INT64, TS_ENCODING_RLE, + TS_COMPRESSION_SNAPPY); + } +} + +/* ============================================================ + * Timeseries CRUD + * ============================================================ */ + +TEST_CASE("C API - Create timeseries", "[c_createTimeseries]") { + CaseReporter cr("c_createTimeseries"); + const char* path = "root.ctest.d1.s1"; + ensureTimeseries(g_session, path, TS_TYPE_INT64, TS_ENCODING_RLE, TS_COMPRESSION_SNAPPY); + bool exists = false; + REQUIRE(ts_session_check_timeseries_exists(g_session, path, &exists) == TS_OK); + REQUIRE(exists); + REQUIRE(ts_session_delete_timeseries(g_session, path) == TS_OK); +} + +TEST_CASE("C API - Delete timeseries", "[c_deleteTimeseries]") { + CaseReporter cr("c_deleteTimeseries"); + const char* path = "root.ctest.d1.s1"; + ensureTimeseries(g_session, path, TS_TYPE_INT64, TS_ENCODING_RLE, TS_COMPRESSION_SNAPPY); + REQUIRE(ts_session_delete_timeseries(g_session, path) == TS_OK); + bool exists = true; + REQUIRE(ts_session_check_timeseries_exists(g_session, path, &exists) == TS_OK); + REQUIRE_FALSE(exists); +} + +TEST_CASE("C API - Login failure", "[c_Authentication]") { + CaseReporter cr("c_LoginTest"); + CSession* badSession = ts_session_new("127.0.0.1", 6667, "root", "wrong-password"); + REQUIRE(badSession != nullptr); + TsStatus status = ts_session_open(badSession); + REQUIRE(status != TS_OK); + const char* err = ts_get_last_error(); + REQUIRE((std::string(err).find("801") != std::string::npos || + std::string(err).find("Authentication") != std::string::npos)); + ts_session_destroy(badSession); +} + +/* ============================================================ + * Insert Record (string values) + * ============================================================ */ + +TEST_CASE("C API - Insert record by string", "[c_insertRecordStr]") { + CaseReporter cr("c_insertRecordStr"); + prepareTimeseries(); + const char* deviceId = "root.ctest.d1"; + const char* measurements[] = {"s1", "s2", "s3"}; + + for (int64_t time = 0; time < 100; time++) { + const char* values[] = {"1", "2", "3"}; + TsStatus status = + ts_session_insert_record_str(g_session, deviceId, time, 3, measurements, values); + REQUIRE(status == TS_OK); + } + + CSessionDataSet* dataSet = nullptr; + TsStatus status = + ts_session_execute_query(g_session, "select s1,s2,s3 from root.ctest.d1", &dataSet); + REQUIRE(status == TS_OK); + REQUIRE(dataSet != nullptr); + ts_dataset_set_fetch_size(dataSet, 1024); + int count = 0; + while (ts_dataset_has_next(dataSet)) { + CRowRecord* record = ts_dataset_next(dataSet); + REQUIRE(record != nullptr); + REQUIRE(ts_row_record_get_int64(record, 0) == 1); + REQUIRE(ts_row_record_get_int64(record, 1) == 2); + REQUIRE(ts_row_record_get_int64(record, 2) == 3); + ++count; + ts_row_record_destroy(record); + } + REQUIRE(count == 100); + ts_dataset_destroy(dataSet); +} + +/* ============================================================ + * Insert Record (typed values) + * ============================================================ */ + +TEST_CASE("C API - Insert record with types", "[c_insertRecordTyped]") { + CaseReporter cr("c_insertRecordTyped"); + + const char* timeseries[] = {"root.ctest.d1.s1", "root.ctest.d1.s2", "root.ctest.d1.s3"}; + TSDataType_C types[] = {TS_TYPE_INT32, TS_TYPE_DOUBLE, TS_TYPE_INT64}; + TSEncoding_C encodings[] = {TS_ENCODING_RLE, TS_ENCODING_RLE, TS_ENCODING_RLE}; + TSCompressionType_C compressions[] = {TS_COMPRESSION_SNAPPY, TS_COMPRESSION_SNAPPY, + TS_COMPRESSION_SNAPPY}; + + for (int i = 0; i < 3; i++) { + ensureTimeseries(g_session, timeseries[i], types[i], encodings[i], compressions[i]); + } + + const char* deviceId = "root.ctest.d1"; + const char* measurements[] = {"s1", "s2", "s3"}; + + for (int64_t time = 0; time < 100; time++) { + int32_t v1 = 1; + double v2 = 2.2; + int64_t v3 = 3; + const void* values[] = {&v1, &v2, &v3}; + TsStatus status = + ts_session_insert_record(g_session, deviceId, time, 3, measurements, types, values); + REQUIRE(status == TS_OK); + } + + REQUIRE(queryRowCount(g_session, "select s1,s2,s3 from root.ctest.d1") == 100); +} + +/* ============================================================ + * Insert Records (batch, string values) + * ============================================================ */ + +TEST_CASE("C API - Insert records batch", "[c_insertRecordsBatch]") { + CaseReporter cr("c_insertRecordsBatch"); + prepareTimeseries(); + + const int BATCH = 100; + const char* deviceId = "root.ctest.d1"; + const char* measurements[] = {"s1", "s2", "s3"}; + + const char* deviceIds[BATCH]; + int64_t times[BATCH]; + int measurementCounts[BATCH]; + const char* const* measurementsList[BATCH]; + const char* values[] = {"1", "2", "3"}; + const char* const* valuesList[BATCH]; + + for (int i = 0; i < BATCH; i++) { + deviceIds[i] = deviceId; + times[i] = i; + measurementCounts[i] = 3; + measurementsList[i] = measurements; + valuesList[i] = values; + } + + TsStatus status = ts_session_insert_records_str(g_session, BATCH, deviceIds, times, + measurementCounts, measurementsList, valuesList); + REQUIRE(status == TS_OK); + + REQUIRE(queryRowCount(g_session, "select s1,s2,s3 from root.ctest.d1") == BATCH); +} + +/* ============================================================ + * Insert Tablet + * ============================================================ */ + +TEST_CASE("C API - Insert tablet", "[c_insertTablet]") { + CaseReporter cr("c_insertTablet"); + prepareTimeseries(); + + const char* columnNames[] = {"s1", "s2", "s3"}; + TSDataType_C dataTypes[] = {TS_TYPE_INT64, TS_TYPE_INT64, TS_TYPE_INT64}; + + CTablet* tablet = ts_tablet_new("root.ctest.d1", 3, columnNames, dataTypes, 100); + REQUIRE(tablet != nullptr); + + for (int64_t time = 0; time < 100; time++) { + ts_tablet_add_timestamp(tablet, (int)time, time); + for (int col = 0; col < 3; col++) { + int64_t val = col; + ts_tablet_add_value_int64(tablet, col, (int)time, val); + } + } + ts_tablet_set_row_count(tablet, 100); + + TsStatus status = ts_session_insert_tablet(g_session, tablet, false); + REQUIRE(status == TS_OK); + + CSessionDataSet* dataSet = nullptr; + REQUIRE(ts_session_execute_query(g_session, "select s1,s2,s3 from root.ctest.d1", &dataSet) == + TS_OK); + REQUIRE(dataSet != nullptr); + ts_dataset_set_fetch_size(dataSet, 1024); + int count = 0; + while (ts_dataset_has_next(dataSet)) { + CRowRecord* record = ts_dataset_next(dataSet); + REQUIRE(ts_row_record_get_int64(record, 0) == 0); + REQUIRE(ts_row_record_get_int64(record, 1) == 1); + REQUIRE(ts_row_record_get_int64(record, 2) == 2); + ++count; + ts_row_record_destroy(record); + } + REQUIRE(count == 100); + ts_dataset_destroy(dataSet); + ts_tablet_destroy(tablet); +} + +/* ============================================================ + * Execute SQL directly + * ============================================================ */ + +TEST_CASE("C API - Execute non-query SQL", "[c_executeNonQuery]") { + CaseReporter cr("c_executeNonQuery"); + prepareTimeseries(); + + TsStatus status = ts_session_execute_non_query( + g_session, "insert into root.ctest.d1(timestamp,s1,s2,s3) values(200,10,20,30)"); + REQUIRE(status == TS_OK); + + CSessionDataSet* dataSet = nullptr; + ts_session_execute_query(g_session, "select s1 from root.ctest.d1 where time=200", &dataSet); + REQUIRE(ts_dataset_has_next(dataSet)); + CRowRecord* record = ts_dataset_next(dataSet); + REQUIRE(ts_row_record_get_int64(record, 0) == 10); + ts_row_record_destroy(record); + ts_dataset_destroy(dataSet); +} + +/* ============================================================ + * Raw data query + * ============================================================ */ + +TEST_CASE("C API - Execute raw data query", "[c_executeRawDataQuery]") { + CaseReporter cr("c_executeRawDataQuery"); + prepareTimeseries(); + + const char* deviceId = "root.ctest.d1"; + const char* measurements[] = {"s1", "s2", "s3"}; + + for (int64_t time = 0; time < 50; time++) { + const char* values[] = {"1", "2", "3"}; + ts_session_insert_record_str(g_session, deviceId, time, 3, measurements, values); + } + + const char* paths[] = {"root.ctest.d1.s1", "root.ctest.d1.s2", "root.ctest.d1.s3"}; + CSessionDataSet* dataSet = nullptr; + TsStatus status = ts_session_execute_raw_data_query(g_session, 3, paths, 0, 50, &dataSet); + REQUIRE(status == TS_OK); + + int count = 0; + while (ts_dataset_has_next(dataSet)) { + CRowRecord* record = ts_dataset_next(dataSet); + count++; + ts_row_record_destroy(record); + } + REQUIRE(count == 50); + ts_dataset_destroy(dataSet); +} + +/* ============================================================ + * Data deletion + * ============================================================ */ + +TEST_CASE("C API - Delete data", "[c_deleteData]") { + CaseReporter cr("c_deleteData"); + prepareTimeseries(); + + const char* deviceId = "root.ctest.d1"; + const char* measurements[] = {"s1", "s2", "s3"}; + for (int64_t time = 0; time < 100; time++) { + const char* values[] = {"1", "2", "3"}; + ts_session_insert_record_str(g_session, deviceId, time, 3, measurements, values); + } + + const char* paths[] = {"root.ctest.d1.s1", "root.ctest.d1.s2", "root.ctest.d1.s3"}; + TsStatus status = ts_session_delete_data_batch(g_session, 3, paths, 49); + REQUIRE(status == TS_OK); + + REQUIRE(queryRowCount(g_session, "select s1,s2,s3 from root.ctest.d1") == 50); +} + +/* ============================================================ + * Timezone + * ============================================================ */ + +TEST_CASE("C API - Timezone", "[c_timezone]") { + CaseReporter cr("c_timezone"); + char buf[64] = {0}; + TsStatus status = ts_session_get_timezone(g_session, buf, sizeof(buf)); + REQUIRE(status == TS_OK); + REQUIRE(strlen(buf) > 0); + + status = ts_session_set_timezone(g_session, "Asia/Shanghai"); + REQUIRE(status == TS_OK); + + memset(buf, 0, sizeof(buf)); + ts_session_get_timezone(g_session, buf, sizeof(buf)); + REQUIRE(std::string(buf) == "Asia/Shanghai"); +} + +/* ============================================================ + * Multi-node constructor + * ============================================================ */ + +TEST_CASE("C API - Multi-node session", "[c_multiNode]") { + CaseReporter cr("c_multiNode"); + const char* urls[] = {"127.0.0.1:6667"}; + CSession* localSession = ts_session_new_multi_node(urls, 1, "root", "root"); + REQUIRE(localSession != nullptr); + + TsStatus status = ts_session_open(localSession); + REQUIRE(status == TS_OK); + + const char* path = "root.ctest.d1.s1"; + ensureTimeseries(localSession, path, TS_TYPE_INT64, TS_ENCODING_RLE, TS_COMPRESSION_SNAPPY); + bool exists = false; + REQUIRE(ts_session_check_timeseries_exists(localSession, path, &exists) == TS_OK); + REQUIRE(exists); + REQUIRE(ts_session_delete_timeseries(localSession, path) == TS_OK); + + ts_session_close(localSession); + ts_session_destroy(localSession); +} + +/* ============================================================ + * Dataset column info + * ============================================================ */ + +TEST_CASE("C API - Dataset column info", "[c_datasetColumns]") { + CaseReporter cr("c_datasetColumns"); + prepareTimeseries(); + + const char* deviceId = "root.ctest.d1"; + const char* measurements[] = {"s1", "s2", "s3"}; + const char* values[] = {"1", "2", "3"}; + ts_session_insert_record_str(g_session, deviceId, 0, 3, measurements, values); + + CSessionDataSet* dataSet = nullptr; + ts_session_execute_query(g_session, "select s1,s2,s3 from root.ctest.d1", &dataSet); + REQUIRE(dataSet != nullptr); + + int colCount = ts_dataset_get_column_count(dataSet); + REQUIRE(colCount == 4); // Time + s1 + s2 + s3 + + const char* col0 = ts_dataset_get_column_name(dataSet, 0); + REQUIRE(std::string(col0) == "Time"); + + int n = ts_dataset_get_column_count(dataSet); + for (int i = 0; i < n; i++) { + const char* ct = ts_dataset_get_column_type(dataSet, i); + REQUIRE(ct != nullptr); + REQUIRE(strlen(ct) > 0); + } + + ts_dataset_destroy(dataSet); +} + +/* ============================================================ + * SessionC.h API coverage (tree model) — additional smoke tests + * ============================================================ */ + +TEST_CASE("C API - Session lifecycle variants", "[c_sessionLifecycle]") { + CaseReporter cr("c_sessionLifecycle"); + + CSession* s1 = ts_session_new_with_zone("127.0.0.1", 6667, "root", "root", "Asia/Shanghai", 1024); + REQUIRE(s1 != nullptr); + REQUIRE(ts_session_open(s1) == TS_OK); + ts_session_close(s1); + ts_session_destroy(s1); + + CSession* s2 = ts_session_new("127.0.0.1", 6667, "root", "root"); + REQUIRE(s2 != nullptr); + REQUIRE(ts_session_open_with_compression(s2, true) == TS_OK); + ts_session_close(s2); + ts_session_destroy(s2); +} + +TEST_CASE("C API - Database and extended timeseries APIs", "[c_dbTimeseries]") { + CaseReporter cr("c_dbTimeseries"); + + const char* sg1 = "root.cov_sg_a"; + const char* sg2 = "root.cov_sg_b"; + dropDatabaseIfExists(g_session, sg1); + dropDatabaseIfExists(g_session, sg2); + REQUIRE(ts_session_create_database(g_session, sg1) == TS_OK); + REQUIRE(ts_session_create_database(g_session, sg2) == TS_OK); + const char* dbs[] = {sg1, sg2}; + REQUIRE(ts_session_delete_databases(g_session, dbs, 2) == TS_OK); + + const char* sg3 = "root.cov_sg_c"; + dropDatabaseIfExists(g_session, sg3); + REQUIRE(ts_session_create_database(g_session, sg3) == TS_OK); + REQUIRE(ts_session_delete_database(g_session, sg3) == TS_OK); + + const char* sgEx = "root.cov_sg_ex"; + dropDatabaseIfExists(g_session, sgEx); + REQUIRE(ts_session_create_database(g_session, sgEx) == TS_OK); + + const char* pathEx = "root.cov_sg_ex.d1.s_ex"; + dropTimeseriesIfExists(g_session, pathEx); + REQUIRE(ts_session_create_timeseries_ex(g_session, pathEx, TS_TYPE_INT64, TS_ENCODING_RLE, + TS_COMPRESSION_SNAPPY, 0, nullptr, nullptr, 0, nullptr, + nullptr, 0, nullptr, nullptr, nullptr) == TS_OK); + + const char* pathsM[] = {"root.cov_sg_ex.d1.s_m1", "root.cov_sg_ex.d1.s_m2"}; + TSDataType_C tsM[] = {TS_TYPE_INT64, TS_TYPE_DOUBLE}; + TSEncoding_C encM[] = {TS_ENCODING_RLE, TS_ENCODING_RLE}; + TSCompressionType_C compM[] = {TS_COMPRESSION_SNAPPY, TS_COMPRESSION_SNAPPY}; + for (int i = 0; i < 2; i++) { + dropTimeseriesIfExists(g_session, pathsM[i]); + } + REQUIRE(ts_session_create_multi_timeseries(g_session, 2, pathsM, tsM, encM, compM) == TS_OK); + REQUIRE(ts_session_delete_timeseries_batch(g_session, pathsM, 2) == TS_OK); + REQUIRE(ts_session_delete_timeseries(g_session, pathEx) == TS_OK); + REQUIRE(ts_session_delete_database(g_session, sgEx) == TS_OK); +} + +TEST_CASE("C API - Tablet row count and reset", "[c_tabletReset]") { + CaseReporter cr("c_tabletReset"); + const char* colNames[] = {"s1"}; + TSDataType_C dts[] = {TS_TYPE_INT64}; + CTablet* tablet = ts_tablet_new("root.ctest.d1", 1, colNames, dts, 10); + REQUIRE(tablet != nullptr); + REQUIRE(ts_tablet_get_row_count(tablet) == 0); + REQUIRE(ts_tablet_set_row_count(tablet, 1) == TS_OK); + REQUIRE(ts_tablet_get_row_count(tablet) == 1); + ts_tablet_reset(tablet); + REQUIRE(ts_tablet_get_row_count(tablet) == 0); + ts_tablet_destroy(tablet); +} + +TEST_CASE("C API - Aligned timeseries and aligned writes", "[c_aligned]") { + CaseReporter cr("c_aligned"); + + const char* sg = "root.cov_al"; + dropDatabaseIfExists(g_session, sg); + REQUIRE(ts_session_create_database(g_session, sg) == TS_OK); + + const char* alDev = "root.cov_al.dev"; + const char* meas[] = {"m1", "m2"}; + TSDataType_C adt[] = {TS_TYPE_INT64, TS_TYPE_INT64}; + TSEncoding_C aenc[] = {TS_ENCODING_RLE, TS_ENCODING_RLE}; + TSCompressionType_C acomp[] = {TS_COMPRESSION_SNAPPY, TS_COMPRESSION_SNAPPY}; + REQUIRE(ts_session_create_aligned_timeseries(g_session, alDev, 2, meas, adt, aenc, acomp) == + TS_OK); + + const char* mstr[] = {"m1", "m2"}; + const char* vstr[] = {"1", "2"}; + REQUIRE(ts_session_insert_aligned_record_str(g_session, alDev, 100LL, 2, mstr, vstr) == TS_OK); + + int64_t v1 = 3; + int64_t v2 = 4; + const void* vals[] = {&v1, &v2}; + REQUIRE(ts_session_insert_aligned_record(g_session, alDev, 101LL, 2, mstr, adt, vals) == TS_OK); + + const char* devs1[] = {alDev}; + int64_t times1[] = {102LL}; + int mc1[] = {2}; + const char* const* mlist1[] = {mstr}; + const char* const* vlist1[] = {vstr}; + REQUIRE(ts_session_insert_aligned_records_str(g_session, 1, devs1, times1, mc1, mlist1, vlist1) == + TS_OK); + + const TSDataType_C* trows[] = {adt}; + const void* const* vrows[] = {vals}; + REQUIRE(ts_session_insert_aligned_records(g_session, 1, devs1, times1, mc1, mlist1, trows, + vrows) == TS_OK); + + int64_t tRows[] = {104LL, 105LL}; + int mcRows[] = {2, 2}; + const char* const* mRows[] = {mstr, mstr}; + const TSDataType_C* tRowsList[] = {adt, adt}; + int64_t v1a = 5, v1b = 6; + int64_t v2a = 7, v2b = 8; + const void* row0[] = {&v1a, &v2a}; + const void* row1[] = {&v1b, &v2b}; + const void* const* vRowsList[] = {row0, row1}; + REQUIRE(ts_session_insert_aligned_records_of_one_device(g_session, alDev, 2, tRows, mcRows, mRows, + tRowsList, vRowsList, true) == TS_OK); + + const char* alDev2 = "root.cov_al.dev2"; + REQUIRE(ts_session_create_aligned_timeseries(g_session, alDev2, 2, meas, adt, aenc, acomp) == + TS_OK); + CTablet* tab = ts_tablet_new(alDev, 2, meas, adt, 10); + CTablet* tab2 = ts_tablet_new(alDev2, 2, meas, adt, 5); + REQUIRE(tab != nullptr); + REQUIRE(tab2 != nullptr); + ts_tablet_add_timestamp(tab, 0, 106LL); + ts_tablet_add_value_int64(tab, 0, 0, 9); + ts_tablet_add_value_int64(tab, 1, 0, 10); + ts_tablet_set_row_count(tab, 1); + ts_tablet_add_timestamp(tab2, 0, 107LL); + ts_tablet_add_value_int64(tab2, 0, 0, 11); + ts_tablet_add_value_int64(tab2, 1, 0, 12); + ts_tablet_set_row_count(tab2, 1); + const char* devIds[] = {alDev, alDev2}; + CTablet* tabs[] = {tab, tab2}; + REQUIRE(ts_session_insert_aligned_tablets(g_session, 2, devIds, tabs, false) == TS_OK); + + ts_tablet_reset(tab); + ts_tablet_add_timestamp(tab, 0, 200LL); + ts_tablet_add_value_int64(tab, 0, 0, 13); + ts_tablet_add_value_int64(tab, 1, 0, 14); + ts_tablet_set_row_count(tab, 1); + REQUIRE(ts_session_insert_aligned_tablet(g_session, tab, false) == TS_OK); + + ts_tablet_destroy(tab2); + ts_tablet_destroy(tab); + + REQUIRE(ts_session_delete_database(g_session, sg) == TS_OK); +} + +TEST_CASE("C API - Typed batch inserts and insert_tablets", "[c_batchTablet]") { + CaseReporter cr("c_batchTablet"); + + const char* sg = "root.cov_batch"; + dropDatabaseIfExists(g_session, sg); + REQUIRE(ts_session_create_database(g_session, sg) == TS_OK); + + const char* p1 = "root.cov_batch.da.s1"; + const char* p2 = "root.cov_batch.db.s1"; + ensureTimeseries(g_session, p1, TS_TYPE_INT64, TS_ENCODING_RLE, TS_COMPRESSION_SNAPPY); + ensureTimeseries(g_session, p2, TS_TYPE_INT64, TS_ENCODING_RLE, TS_COMPRESSION_SNAPPY); + + const char* devIds[] = {"root.cov_batch.da", "root.cov_batch.db"}; + int64_t tt[] = {1LL, 2LL}; + int mmc[] = {1, 1}; + const char* mda[] = {"s1"}; + const char* mdb[] = {"s1"}; + const char* const* mlist[] = {mda, mdb}; + int64_t va = 11; + int64_t vb = 22; + const void* vva[] = {&va}; + const void* vvb[] = {&vb}; + const void* const* vlist[] = {vva, vvb}; + TSDataType_C ta[] = {TS_TYPE_INT64}; + TSDataType_C tb[] = {TS_TYPE_INT64}; + const TSDataType_C* tlist[] = {ta, tb}; + REQUIRE(ts_session_insert_records(g_session, 2, devIds, tt, mmc, mlist, tlist, vlist) == TS_OK); + + const char* dc = "root.cov_batch.dc"; + const char* p3 = "root.cov_batch.dc.s1"; + ensureTimeseries(g_session, p3, TS_TYPE_INT64, TS_ENCODING_RLE, TS_COMPRESSION_SNAPPY); + int64_t tdc[] = {3LL, 4LL}; + int mcdc[] = {1, 1}; + const char* const* mdcList[] = {mda, mda}; + const TSDataType_C* tdcList[] = {ta, ta}; + int64_t vc = 30, vd = 40; + const void* rv0[] = {&vc}; + const void* rv1[] = {&vd}; + const void* const* vdcList[] = {rv0, rv1}; + REQUIRE(ts_session_insert_records_of_one_device(g_session, dc, 2, tdc, mcdc, mdcList, tdcList, + vdcList, true) == TS_OK); + + const char* col1[] = {"s1"}; + TSDataType_C dt1[] = {TS_TYPE_INT64}; + CTablet* tb1 = ts_tablet_new("root.cov_batch.ta", 1, col1, dt1, 5); + CTablet* tb2 = ts_tablet_new("root.cov_batch.tb", 1, col1, dt1, 5); + REQUIRE(tb1 != nullptr); + REQUIRE(tb2 != nullptr); + const char* pta = "root.cov_batch.ta.s1"; + const char* ptb = "root.cov_batch.tb.s1"; + ensureTimeseries(g_session, pta, TS_TYPE_INT64, TS_ENCODING_RLE, TS_COMPRESSION_SNAPPY); + ensureTimeseries(g_session, ptb, TS_TYPE_INT64, TS_ENCODING_RLE, TS_COMPRESSION_SNAPPY); + ts_tablet_add_timestamp(tb1, 0, 1LL); + ts_tablet_add_value_int64(tb1, 0, 0, 100); + ts_tablet_set_row_count(tb1, 1); + ts_tablet_add_timestamp(tb2, 0, 2LL); + ts_tablet_add_value_int64(tb2, 0, 0, 200); + ts_tablet_set_row_count(tb2, 1); + const char* tabDevs[] = {"root.cov_batch.ta", "root.cov_batch.tb"}; + CTablet* tbs[] = {tb1, tb2}; + REQUIRE(ts_session_insert_tablets(g_session, 2, tabDevs, tbs, false) == TS_OK); + ts_tablet_destroy(tb2); + ts_tablet_destroy(tb1); + + REQUIRE(ts_session_delete_database(g_session, sg) == TS_OK); +} + +TEST_CASE("C API - Query timeout and last data queries", "[c_queryLast]") { + CaseReporter cr("c_queryLast"); + prepareTimeseries(); + + const char* deviceId = "root.ctest.d1"; + const char* measurements[] = {"s1", "s2", "s3"}; + for (int64_t time = 300; time < 310; time++) { + const char* values[] = {"7", "8", "9"}; + REQUIRE(ts_session_insert_record_str(g_session, deviceId, time, 3, measurements, values) == + TS_OK); + } + + const char* paths[] = {"root.ctest.d1.s1", "root.ctest.d1.s2"}; + CSessionDataSet* ds = nullptr; + REQUIRE(ts_session_execute_query_with_timeout( + g_session, "select s1 from root.ctest.d1 where time>=300", 60000, &ds) == TS_OK); + REQUIRE(ds != nullptr); + ts_dataset_destroy(ds); + + ds = nullptr; + REQUIRE(ts_session_execute_last_data_query(g_session, 2, paths, &ds) == TS_OK); + REQUIRE(ds != nullptr); + ts_dataset_destroy(ds); + + ds = nullptr; + REQUIRE(ts_session_execute_last_data_query_with_time(g_session, 2, paths, 305LL, &ds) == TS_OK); + REQUIRE(ds != nullptr); + ts_dataset_destroy(ds); +} + +TEST_CASE("C API - RowRecord and delete data APIs", "[c_rowDelete]") { + CaseReporter cr("c_rowDelete"); + + const char* sg = "root.cov_types"; + dropDatabaseIfExists(g_session, sg); + REQUIRE(ts_session_create_database(g_session, sg) == TS_OK); + + const char* pb = "root.cov_types.d1.sb"; + const char* pi = "root.cov_types.d1.si"; + const char* pf = "root.cov_types.d1.sf"; + const char* pd = "root.cov_types.d1.sd"; + const char* pt = "root.cov_types.d1.st"; + const char* tpaths[] = {pb, pi, pf, pd, pt}; + for (const char* tp : tpaths) { + dropTimeseriesIfExists(g_session, tp); + } + REQUIRE(ts_session_create_timeseries(g_session, pb, TS_TYPE_BOOLEAN, TS_ENCODING_RLE, + TS_COMPRESSION_SNAPPY) == TS_OK); + REQUIRE(ts_session_create_timeseries(g_session, pi, TS_TYPE_INT32, TS_ENCODING_RLE, + TS_COMPRESSION_SNAPPY) == TS_OK); + REQUIRE(ts_session_create_timeseries(g_session, pf, TS_TYPE_FLOAT, TS_ENCODING_RLE, + TS_COMPRESSION_SNAPPY) == TS_OK); + REQUIRE(ts_session_create_timeseries(g_session, pd, TS_TYPE_DOUBLE, TS_ENCODING_RLE, + TS_COMPRESSION_SNAPPY) == TS_OK); + REQUIRE(ts_session_create_timeseries(g_session, pt, TS_TYPE_TEXT, TS_ENCODING_PLAIN, + TS_COMPRESSION_SNAPPY) == TS_OK); + + const char* dev = "root.cov_types.d1"; + const char* names[] = {"sb", "si", "sf", "sd", "st"}; + TSDataType_C types[] = {TS_TYPE_BOOLEAN, TS_TYPE_INT32, TS_TYPE_FLOAT, TS_TYPE_DOUBLE, + TS_TYPE_TEXT}; + bool bv = true; + int32_t iv = 42; + float fv = 2.5f; + double dv = 3.25; + const char* tv = "hi"; + const void* vals[] = {&bv, &iv, &fv, &dv, tv}; + REQUIRE(ts_session_insert_record(g_session, dev, 500LL, 5, names, types, vals) == TS_OK); + + CSessionDataSet* dataSet = nullptr; + REQUIRE(ts_session_execute_query(g_session, + "select sb,si,sf,sd,st from root.cov_types.d1 where time=500", + &dataSet) == TS_OK); + REQUIRE(dataSet != nullptr); + REQUIRE(ts_dataset_has_next(dataSet)); + CRowRecord* record = ts_dataset_next(dataSet); + REQUIRE(record != nullptr); + REQUIRE(ts_row_record_get_timestamp(record) == 500LL); + REQUIRE(ts_row_record_get_field_count(record) == 5); + REQUIRE_FALSE(ts_row_record_is_null(record, 0)); + REQUIRE(ts_row_record_get_bool(record, 0) == true); + REQUIRE(ts_row_record_get_int32(record, 1) == 42); + REQUIRE(std::fabs(ts_row_record_get_float(record, 2) - 2.5f) < 1e-4f); + REQUIRE(std::fabs(ts_row_record_get_double(record, 3) - 3.25) < 1e-9); + REQUIRE(std::string(ts_row_record_get_string(record, 4)) == "hi"); + REQUIRE(ts_row_record_get_data_type(record, 0) == TS_TYPE_BOOLEAN); + ts_row_record_destroy(record); + ts_dataset_destroy(dataSet); + + REQUIRE(ts_session_delete_data(g_session, pb, 500LL) == TS_OK); + const char* delPaths[] = {pi, pf}; + REQUIRE(ts_session_delete_data_range(g_session, 2, delPaths, 400LL, 600LL) == TS_OK); + + REQUIRE(ts_session_delete_timeseries(g_session, pb) == TS_OK); + REQUIRE(ts_session_delete_timeseries(g_session, pi) == TS_OK); + REQUIRE(ts_session_delete_timeseries(g_session, pf) == TS_OK); + REQUIRE(ts_session_delete_timeseries(g_session, pd) == TS_OK); + REQUIRE(ts_session_delete_timeseries(g_session, pt) == TS_OK); + REQUIRE(ts_session_delete_database(g_session, sg) == TS_OK); +} diff --git a/iotdb-client/client-cpp/test/cpp/sessionIT.cpp b/iotdb-client/client-cpp/test/cpp/sessionIT.cpp new file mode 100644 index 0000000000000..fdea5a8d07d8a --- /dev/null +++ b/iotdb-client/client-cpp/test/cpp/sessionIT.cpp @@ -0,0 +1,1042 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#include "catch.hpp" +#include "Column.h" +#include "Date.h" +#include "RpcCommon.h" +#include "Session.h" +#include "SessionBuilder.h" +#include "SessionPool.h" +#include "TsBlock.h" +#include "common_types.h" +#include +#include + +using namespace std; + +extern std::shared_ptr session; + +static vector testTimeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3"}; + +void prepareTimeseries() { + for (const string& timeseries : testTimeseries) { + if (session->checkTimeseriesExists(timeseries)) { + session->deleteTimeseries(timeseries); + } + session->createTimeseries(timeseries, TSDataType::INT64, TSEncoding::RLE, + CompressionType::SNAPPY); + } +} + +static int global_test_id = 0; +class CaseReporter { +public: + CaseReporter(const char* caseNameArg) : caseName(caseNameArg) { + test_id = global_test_id++; + std::cout << "Test " << test_id << ": " << caseName << std::endl; + } + ~CaseReporter() { + std::cout << "Test " << test_id << ": " << caseName << " Done" << std::endl << std::endl; + } + +private: + const char* caseName; + int test_id; +}; + +TEST_CASE("Create timeseries success", "[createTimeseries]") { + CaseReporter cr("createTimeseries"); + if (!session->checkTimeseriesExists("root.test.d1.s1")) { + session->createTimeseries("root.test.d1.s1", TSDataType::INT64, TSEncoding::RLE, + CompressionType::SNAPPY); + } + REQUIRE(session->checkTimeseriesExists("root.test.d1.s1") == true); + session->deleteTimeseries("root.test.d1.s1"); +} + +TEST_CASE("Login Test - Authentication failed with error code 801", "[Authentication]") { + CaseReporter cr("Login Test"); + + try { + Session session("127.0.0.1", 6667, "root", "wrong-password"); + session.open(false); + FAIL("Expected authentication exception"); // Test fails if no exception + } catch (const std::exception& e) { + // Verify exception contains error code 801 + REQUIRE(std::string(e.what()).find("801") != std::string::npos); + } +} + +TEST_CASE("Test Session constructor with nodeUrls", "[SessionInitAndOperate]") { + CaseReporter cr("SessionInitWithNodeUrls"); + + std::vector nodeUrls = {"127.0.0.1:6667"}; + std::shared_ptr localSession = std::make_shared(nodeUrls, "root", "root"); + localSession->open(); + if (!localSession->checkTimeseriesExists("root.test.d1.s1")) { + localSession->createTimeseries("root.test.d1.s1", TSDataType::INT64, TSEncoding::RLE, + CompressionType::SNAPPY); + } + REQUIRE(localSession->checkTimeseriesExists("root.test.d1.s1") == true); + localSession->deleteTimeseries("root.test.d1.s1"); + localSession->close(); +} + +TEST_CASE("Test Session builder with nodeUrls", "[SessionBuilderInit]") { + CaseReporter cr("SessionInitWithNodeUrls"); + + std::vector nodeUrls = {"127.0.0.1:6667"}; + auto builder = std::unique_ptr(new SessionBuilder()); + std::shared_ptr session = std::shared_ptr( + builder->username("root")->password("root")->nodeUrls(nodeUrls)->build()); + session->open(); + if (!session->checkTimeseriesExists("root.test.d1.s1")) { + session->createTimeseries("root.test.d1.s1", TSDataType::INT64, TSEncoding::RLE, + CompressionType::SNAPPY); + } + REQUIRE(session->checkTimeseriesExists("root.test.d1.s1") == true); + session->deleteTimeseries("root.test.d1.s1"); + session->close(); +} + +TEST_CASE("Delete timeseries success", "[deleteTimeseries]") { + CaseReporter cr("deleteTimeseries"); + if (!session->checkTimeseriesExists("root.test.d1.s1")) { + session->createTimeseries("root.test.d1.s1", TSDataType::INT64, TSEncoding::RLE, + CompressionType::SNAPPY); + } + REQUIRE(session->checkTimeseriesExists("root.test.d1.s1") == true); + session->deleteTimeseries("root.test.d1.s1"); + REQUIRE(session->checkTimeseriesExists("root.test.d1.s1") == false); +} + +TEST_CASE("Test insertRecord by string", "[testInsertRecord]") { + CaseReporter cr("testInsertRecord"); + prepareTimeseries(); + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3"}; + + for (long time = 0; time < 100; time++) { + vector values = {"1", "2", "3"}; + session->insertRecord(deviceId, time, measurements, values); + } + + session->executeNonQueryStatement( + "insert into root.test.d1(timestamp,s1, s2, s3) values(100, 1,2,3)"); + + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); + sessionDataSet->setFetchSize(1024); + int count = 0; + while (sessionDataSet->hasNext()) { + long index = 1; + count++; + auto fields = sessionDataSet->next()->fields; + for (const Field& f : fields) { + REQUIRE(f.longV.value() == index); + index++; + } + } + REQUIRE(count == 101); +} + +TEST_CASE("Test insertRecords ", "[testInsertRecords]") { + CaseReporter cr("testInsertRecords"); + prepareTimeseries(); + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3"}; + vector deviceIds; + vector> measurementsList; + vector> valuesList; + vector timestamps; + + int64_t COUNT = 500; + for (int64_t time = 1; time <= COUNT; time++) { + vector values = {"1", "2", "3"}; + + deviceIds.push_back(deviceId); + measurementsList.push_back(measurements); + valuesList.push_back(values); + timestamps.push_back(time); + if (time != 0 && time % 100 == 0) { + session->insertRecords(deviceIds, timestamps, measurementsList, valuesList); + deviceIds.clear(); + measurementsList.clear(); + valuesList.clear(); + timestamps.clear(); + } + } + + if (timestamps.size() > 0) { + session->insertRecords(deviceIds, timestamps, measurementsList, valuesList); + } + + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); + sessionDataSet->setFetchSize(1024); + int count = 0; + while (sessionDataSet->hasNext()) { + long index = 1; + count++; + auto fields = sessionDataSet->next()->fields; + for (const Field& f : fields) { + REQUIRE(f.longV.value() == index); + index++; + } + } + REQUIRE(count == COUNT); +} + +TEST_CASE("Test insertRecord with types ", "[testTypedInsertRecord]") { + CaseReporter cr("testTypedInsertRecord"); + vector timeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3"}; + vector types = {TSDataType::INT32, TSDataType::DOUBLE, TSDataType::INT64}; + + for (size_t i = 0; i < timeseries.size(); i++) { + if (session->checkTimeseriesExists(timeseries[i])) { + session->deleteTimeseries(timeseries[i]); + } + session->createTimeseries(timeseries[i], types[i], TSEncoding::RLE, CompressionType::SNAPPY); + } + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3"}; + vector value1(100, 1); + vector value2(100, 2.2); + vector value3(100, 3); + + for (long time = 0; time < 100; time++) { + vector values = {(char*)(&value1[time]), (char*)(&value2[time]), (char*)(&value3[time])}; + session->insertRecord(deviceId, time, measurements, types, values); + } + + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); + sessionDataSet->setFetchSize(1024); + long count = 0; + while (sessionDataSet->hasNext()) { + sessionDataSet->next(); + count++; + } + REQUIRE(count == 100); +} + +TEST_CASE("Test insertRecord with new datatypes ", "[testTypedInsertRecordNewDatatype]") { + CaseReporter cr("testTypedInsertRecordNewDatatype"); + vector timeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3", + "root.test.d1.s4"}; + std::vector types = {TSDataType::TIMESTAMP, TSDataType::DATE, + TSDataType::BLOB, TSDataType::STRING}; + + for (size_t i = 0; i < timeseries.size(); i++) { + if (session->checkTimeseriesExists(timeseries[i])) { + session->deleteTimeseries(timeseries[i]); + } + session->createTimeseries(timeseries[i], types[i], TSEncoding::PLAIN, CompressionType::SNAPPY); + } + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3", "s4"}; + int64_t value1 = 20250507; + IoTDBDate value2 = IoTDBDate(2025, 5, 7); + string value3 = "20250507"; + string value4 = "20250507"; + + for (long time = 0; time < 100; time++) { + vector values = {(char*)(&value1), (char*)(&value2), const_cast(value3.c_str()), + const_cast(value4.c_str())}; + session->insertRecord(deviceId, time, measurements, types, values); + } + + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1,s2,s3,s4 from root.test.d1"); + sessionDataSet->setFetchSize(1024); + long count = 0; + while (sessionDataSet->hasNext()) { + auto record = sessionDataSet->next(); + REQUIRE(record->fields.size() == 4); + for (int i = 0; i < 4; i++) { + REQUIRE(types[i] == record->fields[i].dataType); + } + REQUIRE(record->fields[0].longV.value() == value1); + REQUIRE(record->fields[1].dateV.value() == value2); + REQUIRE(record->fields[2].stringV.value() == value3); + REQUIRE(record->fields[3].stringV.value() == value4); + count++; + } + REQUIRE(count == 100); +} + +TEST_CASE("Test insertRecords with types ", "[testTypedInsertRecords]") { + CaseReporter cr("testTypedInsertRecords"); + vector timeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3"}; + vector types = {TSDataType::INT32, TSDataType::DOUBLE, TSDataType::INT64}; + + for (size_t i = 0; i < timeseries.size(); i++) { + if (session->checkTimeseriesExists(timeseries[i])) { + session->deleteTimeseries(timeseries[i]); + } + session->createTimeseries(timeseries[i], types[i], TSEncoding::RLE, CompressionType::SNAPPY); + } + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3"}; + vector deviceIds; + vector> measurementsList; + vector> typesList; + vector> valuesList; + vector timestamps; + vector value1(100, 1); + vector value2(100, 2.2); + vector value3(100, 3); + + for (int64_t time = 0; time < 100; time++) { + vector values = {(char*)(&value1[time]), (char*)(&value2[time]), (char*)(&value3[time])}; + deviceIds.push_back(deviceId); + measurementsList.push_back(measurements); + typesList.push_back(types); + valuesList.push_back(values); + timestamps.push_back(time); + } + + session->insertRecords(deviceIds, timestamps, measurementsList, typesList, valuesList); + + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); + sessionDataSet->setFetchSize(1024); + int count = 0; + while (sessionDataSet->hasNext()) { + sessionDataSet->next(); + count++; + } + REQUIRE(count == 100); +} + +TEST_CASE("Test insertRecordsOfOneDevice", "[testInsertRecordsOfOneDevice]") { + CaseReporter cr("testInsertRecordsOfOneDevice"); + vector timeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3"}; + vector types = {TSDataType::INT32, TSDataType::DOUBLE, TSDataType::INT64}; + + for (size_t i = 0; i < timeseries.size(); i++) { + if (session->checkTimeseriesExists(timeseries[i])) { + session->deleteTimeseries(timeseries[i]); + } + session->createTimeseries(timeseries[i], types[i], TSEncoding::RLE, CompressionType::SNAPPY); + } + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3"}; + vector> measurementsList; + vector> typesList; + vector> valuesList; + vector timestamps; + vector value1(100, 1); + vector value2(100, 2.2); + vector value3(100, 3); + + for (int64_t time = 0; time < 100; time++) { + vector values = {(char*)(&value1[time]), (char*)(&value2[time]), (char*)(&value3[time])}; + measurementsList.push_back(measurements); + typesList.push_back(types); + valuesList.push_back(values); + timestamps.push_back(time); + } + + session->insertRecordsOfOneDevice(deviceId, timestamps, measurementsList, typesList, valuesList); + + unique_ptr sessionDataSet = + session->executeQueryStatement("select * from root.test.d1"); + sessionDataSet->setFetchSize(1024); + int count = 0; + while (sessionDataSet->hasNext()) { + sessionDataSet->next(); + count++; + } + REQUIRE(count == 100); +} + +TEST_CASE("Test insertTablet ", "[testInsertTablet]") { + CaseReporter cr("testInsertTablet"); + prepareTimeseries(); + string deviceId = "root.test.d1"; + vector> schemaList; + schemaList.emplace_back("s1", TSDataType::INT64); + schemaList.emplace_back("s2", TSDataType::INT64); + schemaList.emplace_back("s3", TSDataType::INT64); + + Tablet tablet(deviceId, schemaList, 100); + for (int64_t time = 0; time < 100; time++) { + int row = tablet.rowSize++; + tablet.timestamps[row] = time; + for (int64_t i = 0; i < 3; i++) { + tablet.addValue(i, row, i); + } + if (tablet.rowSize == tablet.maxRowNumber) { + session->insertTablet(tablet); + tablet.reset(); + } + } + + if (tablet.rowSize != 0) { + session->insertTablet(tablet); + tablet.reset(); + } + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); + sessionDataSet->setFetchSize(1024); + int count = 0; + while (sessionDataSet->hasNext()) { + long index = 0; + count++; + auto fields = sessionDataSet->next()->fields; + for (const Field& f : fields) { + REQUIRE(f.longV.value() == index); + index++; + } + } + REQUIRE(count == 100); +} + +TEST_CASE("Test insertTablets ", "[testInsertTablets]") { + CaseReporter cr("testInsertTablets"); + vector testTimeseries = {"root.test.d1.s1", "root.test.d1.s2", "root.test.d1.s3", + "root.test.d2.s1", "root.test.d2.s2", "root.test.d2.s3"}; + for (const string& timeseries : testTimeseries) { + if (session->checkTimeseriesExists(timeseries)) { + session->deleteTimeseries(timeseries); + } + session->createTimeseries(timeseries, TSDataType::INT64, TSEncoding::RLE, + CompressionType::SNAPPY); + } + vector> schemaList; + schemaList.emplace_back("s1", TSDataType::INT64); + schemaList.emplace_back("s2", TSDataType::INT64); + schemaList.emplace_back("s3", TSDataType::INT64); + + int maxRowNumber = 100; + vector deviceIds = {"root.test.d1", "root.test.d2"}; + vector tablets; + for (const auto& deviceId : deviceIds) { + tablets.emplace_back(deviceId, schemaList, maxRowNumber); + } + for (auto& tablet : tablets) { + for (int64_t time = 0; time < maxRowNumber; time++) { + int row = tablet.rowSize++; + tablet.timestamps[row] = time; + for (int64_t i = 0; i < 3; i++) { + tablet.addValue(i, row, i); + } + } + } + unordered_map tabletsMap; + for (auto& tablet : tablets) { + tabletsMap[tablet.deviceId] = &tablet; + } + session->insertTablets(tabletsMap); + + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1,s2,s3 from root.test.d2"); + sessionDataSet->setFetchSize(1024); + int count = 0; + while (sessionDataSet->hasNext()) { + long index = 0; + count++; + auto fields = sessionDataSet->next()->fields; + for (const Field& f : fields) { + REQUIRE(f.longV.value() == index); + index++; + } + } + REQUIRE(count == 100); +} + +TEST_CASE("Test insertTablet multi datatype", "[testInsertTabletMultiDatatype]") { + CaseReporter cr("testInsertTabletNewDatatype"); + string deviceId = "root.test.d2"; + vector> schemaList; + std::vector measurements = {"s1", "s2", "s3", "s4"}; + std::vector dataTypes = {TSDataType::TIMESTAMP, TSDataType::DATE, + TSDataType::BLOB, TSDataType::STRING}; + for (int i = 0; i < 4; i++) { + schemaList.emplace_back(measurements[i], dataTypes[i]); + } + + for (int i = 0; i < 4; i++) { + auto timeseries = deviceId + "." + measurements[i]; + if (session->checkTimeseriesExists(timeseries)) { + session->deleteTimeseries(timeseries); + } + session->createTimeseries(timeseries, dataTypes[i], TSEncoding::PLAIN, + CompressionType::UNCOMPRESSED); + } + + int64_t s1Value = 20250507; + IoTDBDate s2Value(2025, 5, 7); + std::string s3Value("20250507"); + std::string s4Value("20250507"); + + Tablet tablet(deviceId, schemaList, 100); + for (int64_t time = 0; time < 100; time++) { + int row = tablet.rowSize++; + tablet.timestamps[row] = time; + tablet.addValue(0, row, s1Value); + tablet.addValue(1, row, s2Value); + tablet.addValue(2, row, s3Value); + tablet.addValue(3, row, s4Value); + if (tablet.rowSize == tablet.maxRowNumber) { + session->insertTablet(tablet); + tablet.reset(); + } + } + + if (tablet.rowSize != 0) { + session->insertTablet(tablet); + tablet.reset(); + } + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1,s2,s3,s4 from root.test.d2"); + auto dataIter = sessionDataSet->getIterator(); + sessionDataSet->setFetchSize(1024); + int count = 0; + while (dataIter.next()) { + REQUIRE(dataIter.getLongByIndex(2).value() == s1Value); + REQUIRE(dataIter.getDateByIndex(3).value() == s2Value); + REQUIRE(dataIter.getStringByIndex(4).value() == s3Value); + REQUIRE(dataIter.getStringByIndex(5).value() == s4Value); + count++; + } + REQUIRE(count == 100); +} + +TEST_CASE("Test boolean column via DataIterator", "[testBooleanColumnDataIterator]") { + CaseReporter cr("testBooleanColumnDataIterator"); + string deviceId = "root.test.d1"; + string timeseries = "root.test.d1.s1"; + + if (session->checkTimeseriesExists(timeseries)) { + session->deleteTimeseries(timeseries); + } + session->createTimeseries(timeseries, TSDataType::BOOLEAN, TSEncoding::PLAIN, + CompressionType::SNAPPY); + + // Insert boolean values: even timestamps get true, odd get false + vector measurements = {"s1"}; + vector types = {TSDataType::BOOLEAN}; + for (int64_t time = 0; time < 10; time++) { + bool val = (time % 2 == 0); + vector values = {(char*)&val}; + session->insertRecord(deviceId, time, measurements, types, values); + } + + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1 from root.test.d1"); + auto dataIter = sessionDataSet->getIterator(); + sessionDataSet->setFetchSize(1024); + int count = 0; + while (dataIter.next()) { + bool expected = (count % 2 == 0); + // Column 1 is Time, column 2 is s1 + REQUIRE(dataIter.getBooleanByIndex(2).value() == expected); + // Accessing time column (index 1) as boolean should throw + REQUIRE_THROWS_AS(dataIter.getBooleanByIndex(1), IoTDBException); + count++; + } + REQUIRE(count == 10); + session->deleteTimeseries(timeseries); +} + +TEST_CASE("Test Last query ", "[testLastQuery]") { + CaseReporter cr("testLastQuery"); + prepareTimeseries(); + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3"}; + + for (long time = 0; time < 100; time++) { + vector values = {"1", "2", "3"}; + session->insertRecord(deviceId, time, measurements, values); + } + + vector measurementValues = {"1", "2", "3"}; + unique_ptr sessionDataSet = + session->executeQueryStatement("select last s1,s2,s3 from root.test.d1"); + sessionDataSet->setFetchSize(1024); + long index = 0; + while (sessionDataSet->hasNext()) { + vector fields = sessionDataSet->next()->fields; + REQUIRE("1" <= fields[1].stringV.value()); + REQUIRE(fields[1].stringV.value() <= "3"); + index++; + } +} + +TEST_CASE("Test Huge query ", "[testHugeQuery]") { + CaseReporter cr("testHugeQuery"); + prepareTimeseries(); + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3"}; + vector types = {TSDataType::INT64, TSDataType::INT64, TSDataType::INT64}; + int64_t value1 = 1, value2 = 2, value3 = 3; + vector values = {(char*)&value1, (char*)&value2, (char*)&value3}; + + long total_count = 500000; + int print_count = 0; + std::cout.width(7); + std::cout << "inserting " << total_count << " rows:" << std::endl; + for (long time = 0; time < total_count; time++) { + session->insertRecord(deviceId, time, measurements, types, values); + if (time != 0 && time % 1000 == 0) { + std::cout << time << "\t" << std::flush; + if (++print_count % 20 == 0) { + std::cout << std::endl; + } + } + } + + unique_ptr sessionDataSet = + session->executeQueryStatement("select s1,s2,s3 from root.test.d1"); + sessionDataSet->setFetchSize(1024); + int count = 0; + print_count = 0; + std::cout << "\n\niterating " << total_count << " rows:" << std::endl; + while (sessionDataSet->hasNext()) { + auto rowRecord = sessionDataSet->next(); + REQUIRE(rowRecord->timestamp == count); + REQUIRE(rowRecord->fields[0].longV.value() == 1); + REQUIRE(rowRecord->fields[1].longV.value() == 2); + REQUIRE(rowRecord->fields[2].longV.value() == 3); + count++; + if (count % 1000 == 0) { + std::cout << count << "\t" << std::flush; + if (++print_count % 20 == 0) { + std::cout << std::endl; + } + } + } + + REQUIRE(count == total_count); +} + +TEST_CASE("Test executeRawDataQuery ", "[executeRawDataQuery]") { + CaseReporter cr("executeRawDataQuery"); + prepareTimeseries(); + + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3"}; + vector types = {TSDataType::INT64, TSDataType::INT64, TSDataType::INT64}; + + long total_count = 5000; + vector values; + int64_t valueArray[3]; + for (long time = -total_count; time < total_count; time++) { + valueArray[0] = time; + valueArray[1] = time * 2; + valueArray[2] = time * 3; + values.clear(); + values.push_back((char*)&valueArray[0]); + values.push_back((char*)&valueArray[1]); + values.push_back((char*)&valueArray[2]); + session->insertRecord(deviceId, time, measurements, types, values); + if (time == 100) { //insert 1 big timestamp data for generate un-seq data. + valueArray[0] = 9; + valueArray[2] = 999; + values.clear(); + values.push_back((char*)&valueArray[0]); + values.push_back((char*)&valueArray[2]); + vector measurements2 = {"s1", "s3"}; + vector types2 = {TSDataType::INT64, TSDataType::INT64}; + session->insertRecord(deviceId, 99999, measurements2, types2, values); + } + } + + vector paths; + paths.push_back("root.test.d1.s1"); + paths.push_back("root.test.d1.s2"); + paths.push_back("root.test.d1.s3"); + + //== Test executeRawDataQuery() with negative timestamp + int startTs = -total_count, endTs = total_count; + unique_ptr sessionDataSet = session->executeRawDataQuery(paths, startTs, endTs); + sessionDataSet->setFetchSize(10); + vector columns = sessionDataSet->getColumnNames(); + columns = sessionDataSet->getColumnNames(); + REQUIRE(columns[0] == "Time"); + REQUIRE(columns[1] == paths[0]); + REQUIRE(columns[2] == paths[1]); + REQUIRE(columns[3] == paths[2]); + + int ts = startTs; + while (sessionDataSet->hasNext()) { + auto rowRecordPtr = sessionDataSet->next(); + //cout << rowRecordPtr->toString(); + + vector fields = rowRecordPtr->fields; + REQUIRE(rowRecordPtr->timestamp == ts); + REQUIRE(fields[0].dataType == TSDataType::INT64); + REQUIRE(fields[0].longV.value() == ts); + REQUIRE(fields[1].dataType == TSDataType::INT64); + REQUIRE(fields[1].longV.value() == ts * 2); + REQUIRE(fields[2].dataType == TSDataType::INT64); + REQUIRE(fields[2].longV.value() == ts * 3); + ts++; + } + + //== Test executeRawDataQuery() with null field + startTs = 99999; + endTs = 99999 + 10; + sessionDataSet = session->executeRawDataQuery(paths, startTs, endTs); + + sessionDataSet->setFetchSize(10); + columns = sessionDataSet->getColumnNames(); + for (const string& column : columns) { + cout << column << " "; + } + cout << endl; + REQUIRE(columns[0] == "Time"); + REQUIRE(columns[1] == paths[0]); + REQUIRE(columns[2] == paths[1]); + REQUIRE(columns[3] == paths[2]); + ts = startTs; + while (sessionDataSet->hasNext()) { + auto rowRecordPtr = sessionDataSet->next(); + cout << rowRecordPtr->toString(); + + vector fields = rowRecordPtr->fields; + REQUIRE(rowRecordPtr->timestamp == ts); + REQUIRE(fields[0].dataType == TSDataType::INT64); + REQUIRE(fields[0].longV.value() == 9); + REQUIRE(fields[1].dataType == TSDataType::UNKNOWN); + REQUIRE(fields[2].dataType == TSDataType::INT64); + REQUIRE(fields[2].longV.value() == 999); + } + + //== Test executeRawDataQuery() with empty data + sessionDataSet = session->executeRawDataQuery(paths, 100000, 110000); + sessionDataSet->setFetchSize(1); + REQUIRE(sessionDataSet->hasNext() == false); +} + +TEST_CASE("Test executeLastDataQuery ", "[testExecuteLastDataQuery]") { + CaseReporter cr("testExecuteLastDataQuery"); + prepareTimeseries(); + + string deviceId = "root.test.d1"; + vector measurements = {"s1", "s2", "s3"}; + vector types = {TSDataType::INT64, TSDataType::INT64, TSDataType::INT64}; + + long total_count = 5000; + vector values; + int64_t valueArray[3]; + for (long time = -total_count; time < total_count; time++) { + valueArray[0] = time; + valueArray[1] = time * 2; + valueArray[2] = time * 3; + values.clear(); + values.push_back((char*)&valueArray[0]); + values.push_back((char*)&valueArray[1]); + values.push_back((char*)&valueArray[2]); + session->insertRecord(deviceId, time, measurements, types, values); + if (time == 100) { //insert 1 big timestamp data for gen unseq data. + valueArray[0] = 9; + valueArray[2] = 999; + values.clear(); + values.push_back((char*)&valueArray[0]); + values.push_back((char*)&valueArray[2]); + vector measurements2 = {"s1", "s3"}; + vector types2 = {TSDataType::INT64, TSDataType::INT64}; + session->insertRecord(deviceId, 99999, measurements2, types2, values); + } + } + + int64_t tsCheck[3] = {99999, 4999, 99999}; + std::vector valueCheck = {"9", "9998", "999"}; + + vector paths; + paths.push_back("root.test.d1.s1"); + paths.push_back("root.test.d1.s2"); + paths.push_back("root.test.d1.s3"); + + //== Test executeLastDataQuery() without lastTime + unique_ptr sessionDataSet = session->executeLastDataQuery(paths); + sessionDataSet->setFetchSize(1); + + vector columns = sessionDataSet->getColumnNames(); + for (const string& column : columns) { + cout << column << " "; + } + cout << endl; + + int index = 0; + while (sessionDataSet->hasNext()) { + auto rowRecordPtr = sessionDataSet->next(); + cout << rowRecordPtr->toString(); + + vector fields = rowRecordPtr->fields; + REQUIRE(rowRecordPtr->timestamp == tsCheck[index]); + REQUIRE(fields[0].stringV.value() == paths[index]); + REQUIRE(fields[1].stringV.value() == valueCheck[index]); + REQUIRE(fields[2].stringV.value() == "INT64"); + index++; + } + + //== Test executeLastDataQuery() with negative lastTime + sessionDataSet = session->executeLastDataQuery(paths, -200); + sessionDataSet->setFetchSize(1); + columns = sessionDataSet->getColumnNames(); + for (const string& column : columns) { + cout << column << " "; + } + cout << endl; + + index = 0; + while (sessionDataSet->hasNext()) { + auto rowRecordPtr = sessionDataSet->next(); + cout << rowRecordPtr->toString(); + + vector fields = rowRecordPtr->fields; + REQUIRE(rowRecordPtr->timestamp == tsCheck[index]); + REQUIRE(fields[0].stringV.value() == paths[index]); + REQUIRE(fields[1].stringV.value() == valueCheck[index]); + REQUIRE(fields[2].stringV.value() == "INT64"); + index++; + } + + //== Test executeLastDataQuery() with the lastTime that is > largest timestamp. + sessionDataSet = session->executeLastDataQuery(paths, 100000); + sessionDataSet->setFetchSize(1024); + REQUIRE(sessionDataSet->hasNext() == false); +} + +// Helper function for comparing TEndPoint with detailed error message +void assertTEndPointEqual(const TEndPoint& actual, const std::string& expectedIp, int expectedPort, + const char* file, int line) { + if (actual.ip != expectedIp || actual.port != expectedPort) { + std::stringstream ss; + ss << "\nTEndPoint mismatch:\nExpected: " << expectedIp << ":" << expectedPort + << "\nActual: " << actual.ip << ":" << actual.port; + Catch::SourceLineInfo location(file, line); + Catch::AssertionHandler handler("TEndPoint comparison", location, ss.str(), + Catch::ResultDisposition::Normal); + handler.handleMessage(Catch::ResultWas::ExplicitFailure, ss.str()); + handler.complete(); + } +} + +// Macro to simplify test assertions +#define REQUIRE_TENDPOINT(actual, expectedIp, expectedPort) \ + assertTEndPointEqual(actual, expectedIp, expectedPort, __FILE__, __LINE__) + +TEST_CASE("UrlUtils - parseTEndPointIpv4AndIpv6Url", "[UrlUtils]") { + // Test valid IPv4 addresses + SECTION("Valid IPv4") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("192.168.1.1:8080"), "192.168.1.1", + 8080); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("10.0.0.1:80"), "10.0.0.1", 80); + } + + // Test valid IPv6 addresses + SECTION("Valid IPv6") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("[2001:db8::1]:8080"), "2001:db8::1", + 8080); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("[::1]:80"), "::1", 80); + } + + // Test hostnames + SECTION("Hostnames") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("localhost:8080"), "localhost", 8080); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("example.com:443"), "example.com", + 443); + } + + // Test edge cases + SECTION("Edge cases") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url(""), "", 0); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("127.0.0.1"), "127.0.0.1", 0); + } + + // Test invalid inputs + SECTION("Invalid inputs") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("192.168.1.1:abc"), "192.168.1.1:abc", + 0); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("]invalid[:80"), "]invalid[", 80); + } + + // Test port ranges + SECTION("Port ranges") { + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("localhost:0"), "localhost", 0); + REQUIRE_TENDPOINT(UrlUtils::parseTEndPointIpv4AndIpv6Url("127.0.0.1:65535"), "127.0.0.1", + 65535); + } +} + +TEST_CASE("TsBlock deserialize rejects truncated malicious payload", "[TsBlockDeserialize]") { + std::string data(18, '\0'); + data[3] = '\x10'; + REQUIRE_THROWS_AS(TsBlock::deserialize(data), IoTDBException); +} + +TEST_CASE("Numeric column widening getters align with Java TsFile", "[column]") { + std::vector valueIsNull(1, false); + + std::vector floatValues = {120.00000762939453f}; + auto floatColumn = std::make_shared(0, 1, valueIsNull, floatValues); + auto rleColumn = std::make_shared(floatColumn, 20); + REQUIRE(floatColumn->getDouble(0) == Approx(120.0).margin(0.01)); + REQUIRE(rleColumn->getDouble(0) == Approx(120.0).margin(0.01)); + + std::vector intValues = {42}; + auto intColumn = std::make_shared(0, 1, valueIsNull, intValues); + REQUIRE(intColumn->getLong(0) == 42); + REQUIRE(intColumn->getDouble(0) == Approx(42.0)); + + std::vector longValues = {1000}; + auto longColumn = std::make_shared(0, 1, valueIsNull, longValues); + REQUIRE(longColumn->getDouble(0) == Approx(1000.0)); +} + +TEST_CASE("SessionPool basic borrow/insert/query via RAII lease", "[sessionPool]") { + CaseReporter cr("SessionPool basic"); + auto pool = SessionPoolBuilder() + .host("127.0.0.1") + ->rpcPort(6667) + ->username("root") + ->password("root") + ->maxSize(3) + ->build(); + + { + PooledSession s = pool->getSession(); + try { + s->executeNonQueryStatement("delete timeseries root.test.pool.d1.*"); + } catch (const std::exception &) { + // Ignore: the timeseries may not exist yet on a fresh database. + } + } + + const int rows = 50; + for (int i = 0; i < rows; i++) { + PooledSession s = pool->getSession(); + s->insertRecord("root.test.pool.d1", i, {"s1"}, {to_string(i)}); + } + + int count = 0; + { + PooledSessionDataSet ds = + pool->executeQueryStatement("select s1 from root.test.pool.d1"); + while (ds->hasNext()) { + ds->next(); + count++; + } + } + REQUIRE(count == rows); + + { + PooledSession s = pool->getSession(); + s->executeNonQueryStatement("delete timeseries root.test.pool.d1.*"); + } + pool->close(); +} + +TEST_CASE("SessionPool is safe under concurrent writers", "[sessionPool]") { + CaseReporter cr("SessionPool concurrency"); + auto pool = SessionPoolBuilder() + .host("127.0.0.1") + ->rpcPort(6667) + ->username("root") + ->password("root") + ->maxSize(4) + ->build(); + + { + PooledSession s = pool->getSession(); + try { + s->executeNonQueryStatement("delete timeseries root.test.pool.d2.*"); + } catch (const std::exception &) { + // Ignore: the timeseries may not exist yet on a fresh database. + } + } + + const int threadCount = 8; + const int rowsPerThread = 100; + std::atomic failures(0); + std::vector threads; + for (int t = 0; t < threadCount; t++) { + threads.emplace_back([&pool, t, rowsPerThread, &failures]() { + try { + for (int i = 0; i < rowsPerThread; i++) { + int64_t ts = static_cast(t) * rowsPerThread + i; + // Mix RAII and convenience APIs to exercise both borrow paths. + if (i % 2 == 0) { + pool->insertRecord("root.test.pool.d2", ts, {"s1"}, + {to_string(ts)}); + } else { + PooledSession s = pool->getSession(); + s->insertRecord("root.test.pool.d2", ts, {"s1"}, {to_string(ts)}); + } + } + } catch (const std::exception &e) { + std::cerr << "writer thread failed: " << e.what() << std::endl; + failures++; + } + }); + } + for (auto &th : threads) { + th.join(); + } + REQUIRE(failures.load() == 0); + REQUIRE(pool->getMaxSize() == 4); + + int count = 0; + { + PooledSessionDataSet ds = + pool->executeQueryStatement("select s1 from root.test.pool.d2"); + while (ds->hasNext()) { + ds->next(); + count++; + } + } + REQUIRE(count == threadCount * rowsPerThread); + + { + PooledSession s = pool->getSession(); + s->executeNonQueryStatement("delete timeseries root.test.pool.d2.*"); + } + pool->close(); +} + +TEST_CASE("SessionPool getSession times out when exhausted", "[sessionPool]") { + CaseReporter cr("SessionPool exhaustion timeout"); + auto pool = SessionPoolBuilder() + .host("127.0.0.1") + ->rpcPort(6667) + ->username("root") + ->password("root") + ->maxSize(1) + ->waitToGetSessionTimeoutMs(200) + ->build(); + + PooledSession held = pool->getSession(); + REQUIRE(static_cast(held)); + REQUIRE(pool->activeCount() == 1); + // The only Session is checked out, so a second borrow must time out. + REQUIRE_THROWS_AS(pool->getSession(), IoTDBException); + + held.release(); + // After returning it, a borrow succeeds again. + PooledSession reused = pool->getSession(); + REQUIRE(static_cast(reused)); + reused.release(); + pool->close(); +} diff --git a/iotdb-client/client-cpp/test/main.cpp b/iotdb-client/client-cpp/test/main.cpp new file mode 100644 index 0000000000000..256b733b291f5 --- /dev/null +++ b/iotdb-client/client-cpp/test/main.cpp @@ -0,0 +1,59 @@ +/** + * 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. + */ + +#define CATCH_CONFIG_MAIN + +#include +#include "Session.h" +#include "SessionBuilder.h" + +std::shared_ptr session; + +struct SessionListener : Catch::TestEventListenerBase { + + using TestEventListenerBase::TestEventListenerBase; + + void testCaseStarting(Catch::TestCaseInfo const& testInfo) override { + if (!session) { + SessionBuilder builder; + session = builder.host("127.0.0.1") + ->rpcPort(6667) + ->username("root") + ->password("root") + ->useSSL(false) + ->enableAutoFetch(false) + ->build(); + } else { + session->open(false); + } + } + + void testCaseEnded(Catch::TestCaseStats const& testCaseStats) override { + if (session) { + session->close(); + } + } + + void testRunEnded(Catch::TestRunStats const& testRunStats) override { + // Release session before static/global teardown on Windows. + session.reset(); + } +}; + +CATCH_REGISTER_LISTENER(SessionListener) \ No newline at end of file diff --git a/iotdb-client/client-cpp/test/main_c.cpp b/iotdb-client/client-cpp/test/main_c.cpp new file mode 100644 index 0000000000000..056b1f49b4ca1 --- /dev/null +++ b/iotdb-client/client-cpp/test/main_c.cpp @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +#define CATCH_CONFIG_MAIN + +#include +#include "SessionC.h" + +// Global session handle used by the C API tree-model tests +CSession* g_session = nullptr; + +struct CSessionListener : Catch::TestEventListenerBase { + using TestEventListenerBase::TestEventListenerBase; + + void testCaseStarting(Catch::TestCaseInfo const& testInfo) override { + g_session = ts_session_new("127.0.0.1", 6667, "root", "root"); + REQUIRE(g_session != nullptr); + TsStatus st = ts_session_open(g_session); + if (st != TS_OK) { + ts_session_destroy(g_session); + g_session = nullptr; + FAIL("ts_session_open failed; ensure distribution is built and IoTDB listens on " + "127.0.0.1:6667"); + } + } + + void testCaseEnded(Catch::TestCaseStats const& testCaseStats) override { + if (g_session) { + ts_session_close(g_session); + ts_session_destroy(g_session); + g_session = nullptr; + } + } +}; + +CATCH_REGISTER_LISTENER(CSessionListener) diff --git a/iotdb-client/client-cpp/third-party/.gitignore b/iotdb-client/client-cpp/third-party/.gitignore new file mode 100644 index 0000000000000..c8badffab8dc0 --- /dev/null +++ b/iotdb-client/client-cpp/third-party/.gitignore @@ -0,0 +1,7 @@ +# Ignore all cached archives; keep directory structure and README tracked. +* +!.gitignore +!README.md +!linux/ +!mac/ +!windows/ diff --git a/iotdb-client/client-cpp/third-party/README.md b/iotdb-client/client-cpp/third-party/README.md new file mode 100644 index 0000000000000..313a6fb79a1d6 --- /dev/null +++ b/iotdb-client/client-cpp/third-party/README.md @@ -0,0 +1,75 @@ + +# Third-party dependency cache (`client-cpp/third-party`) + +CMake downloads (or reuses) build-time tarballs and archives here. The +directory ships with the source tree so you can **stage dependencies on a +networked machine, copy the whole IoTDB checkout to an offline host, and +build `client-cpp` with `-DIOTDB_OFFLINE=ON`**. + +Tarballs themselves are **not** committed to Git (see per-platform +`.gitignore` files). Only this README and the empty platform folders are +tracked. Each `linux/`, `mac/`, and `windows/` sub-folder ships a minimal +`.gitignore` (`*` with `!.gitignore`) so Git keeps the directory in the +tree while ignoring downloaded archives. + +## Layout + +``` +third-party/ +├── linux/ # tarballs for Linux offline builds +├── mac/ # tarballs for macOS offline builds +└── windows/ # tarballs / zips for Windows offline builds +``` + +Override the root with `-DIOTDB_DEPS_DIR=` (Maven: `-Diotdb.deps.dir=...`). +The platform sub-folder (`linux/`, `mac/`, `windows/`) is selected automatically. + +## Staging dependencies (online machine) + +Run a normal **online** configure once; CMake caches everything under the +matching `/` folder: + +```bash +cmake -S iotdb-client/client-cpp -B build +# or: mvn -P with-cpp -pl iotdb-client/client-cpp -am -DskipTests package +``` + +Alternatively copy files manually from the URLs listed in +[`README.md`](../README.md) (*Offline build* section). + +## Offline machine (copy whole IoTDB tree) + +1. Copy the entire IoTDB repository (including `iotdb-client/client-cpp/third-party//`). +2. Configure with offline mode: + + ```bash + cmake -S iotdb-client/client-cpp -B build -DIOTDB_OFFLINE=ON + cmake --build build --config Release --target install + ``` + +## Per-platform files (offline minimum) + +| Platform | Typical files | +|------------|---------------| +| `linux/` | `thrift-0.21.0.tar.gz`, `boost_1_60_0.tar.gz`, `m4-1.4.19.tar.gz`, `flex-2.6.4.tar.gz`, `bison-3.8.tar.gz` (+ `openssl-3.5.0.tar.gz` when `WITH_SSL=ON`) | +| `mac/` | `thrift-0.21.0.tar.gz`, `boost_1_60_0.tar.gz` (Xcode CLT usually provides m4/flex/bison) | +| `windows/` | `thrift-0.21.0.tar.gz`, `boost_1_60_0.tar.gz`, `win_flex_bison-2.5.25.zip` (or any `win_flex_bison*.zip`; skip if flex/bison already on `PATH`) | + +Download URLs: see the *Offline build* table in [`README.md`](../README.md). diff --git a/iotdb-client/client-cpp/third-party/linux/.gitignore b/iotdb-client/client-cpp/third-party/linux/.gitignore new file mode 100644 index 0000000000000..d6b7ef32c8478 --- /dev/null +++ b/iotdb-client/client-cpp/third-party/linux/.gitignore @@ -0,0 +1,2 @@ +* +!.gitignore diff --git a/iotdb-client/client-cpp/third-party/mac/.gitignore b/iotdb-client/client-cpp/third-party/mac/.gitignore new file mode 100644 index 0000000000000..d6b7ef32c8478 --- /dev/null +++ b/iotdb-client/client-cpp/third-party/mac/.gitignore @@ -0,0 +1,2 @@ +* +!.gitignore diff --git a/iotdb-client/client-cpp/third-party/windows/.gitignore b/iotdb-client/client-cpp/third-party/windows/.gitignore new file mode 100644 index 0000000000000..d6b7ef32c8478 --- /dev/null +++ b/iotdb-client/client-cpp/third-party/windows/.gitignore @@ -0,0 +1,2 @@ +* +!.gitignore diff --git a/pom.xml b/pom.xml index a83a66e135c5a..11ffc5335b293 100644 --- a/pom.xml +++ b/pom.xml @@ -157,6 +157,7 @@ false 2.43.0 + 17.0.6 1.6.14 chmod @@ -863,9 +864,9 @@ 1.7.1 - com.googlecode.cmake-maven-project + io.github.cmake-maven-plugin cmake-maven-plugin - 3.29.3-b2 + 4.2.3-b1 org.apache.maven.plugins From 290842e77df127b0b02130bdde5fa180c562be62 Mon Sep 17 00:00:00 2001 From: Jackie Tien Date: Thu, 11 Jun 2026 19:38:27 +0800 Subject: [PATCH 100/102] [To dev/1.3] Fix driver scheduler ready queue reservation leak (#17920) --- .../execution/schedule/DriverScheduler.java | 15 +++++-- .../queue/IndexedBlockingReserveQueue.java | 36 ++++++++++++++-- .../MultilevelPriorityQueue.java | 10 +++++ .../execution/schedule/task/DriverTask.java | 13 ++++++ .../metric/DriverSchedulerMetricSet.java | 13 ++++++ .../schedule/DefaultDriverSchedulerTest.java | 41 +++++++++++++++++++ 6 files changed, 121 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/DriverScheduler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/DriverScheduler.java index 183e51b7e1ef1..239ebb11329ae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/DriverScheduler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/DriverScheduler.java @@ -336,16 +336,18 @@ private void clearDriverTask(DriverTask task) { return; case READY: task.setStatus(DriverTaskStatus.ABORTED); - readyQueue.remove(task.getDriverTaskId()); + if (readyQueue.remove(task.getDriverTaskId()) == null) { + readyQueue.decreaseReservedSize(task); + } break; case BLOCKED: task.setStatus(DriverTaskStatus.ABORTED); blockedTasks.remove(task); - readyQueue.decreaseReservedSize(); + readyQueue.decreaseReservedSize(task); break; case RUNNING: task.setStatus(DriverTaskStatus.ABORTED); - readyQueue.decreaseReservedSize(); + readyQueue.decreaseReservedSize(task); break; case FINISHED: break; @@ -414,6 +416,10 @@ public long getReadyQueueTaskCount() { return readyQueue.size(); } + public long getReadyQueueReservedTaskCount() { + return readyQueue.getReservedSize(); + } + public long getBlockQueueTaskCount() { return blockedTasks.size(); } @@ -489,6 +495,7 @@ public boolean readyToRunning(DriverTask task) { task.lock(); try { if (task.getStatus() != DriverTaskStatus.READY) { + readyQueue.decreaseReservedSize(task); return false; } @@ -545,7 +552,7 @@ public void runningToFinished(DriverTask task, ExecutionContext context) { } task.updateSchedulePriority(context); task.setStatus(DriverTaskStatus.FINISHED); - readyQueue.decreaseReservedSize(); + readyQueue.decreaseReservedSize(task); } finally { task.unlock(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/queue/IndexedBlockingReserveQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/queue/IndexedBlockingReserveQueue.java index 1891fb6355677..b0f0edbd0e1aa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/queue/IndexedBlockingReserveQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/queue/IndexedBlockingReserveQueue.java @@ -47,6 +47,7 @@ public synchronized E poll() throws InterruptedException { E output = pollFirst(); size--; reservedSize++; + markReserved(output); return output; } @@ -66,7 +67,7 @@ public synchronized void repush(E element) { throw new NullPointerException("pushed element is null"); } pushToQueue(element); - reservedSize--; + decreaseReservedSizeIfNecessary(element); size++; this.notifyAll(); } @@ -74,7 +75,36 @@ public synchronized void repush(E element) { /** * For task that is not in readyQueue when it's cleared, it won't be added into the queue again. */ - public synchronized void decreaseReservedSize() { - this.reservedSize--; + public synchronized boolean decreaseReservedSize(E element) { + if (element == null) { + throw new NullPointerException("pushed element is null"); + } + return decreaseReservedSizeIfNecessary(element); + } + + public final synchronized int getReservedSize() { + return reservedSize; + } + + @Override + public synchronized void clear() { + super.clear(); + this.reservedSize = 0; + } + + protected void markReserved(E element) { + // Do nothing by default. + } + + protected boolean releaseReserved(E element) { + return true; + } + + private boolean decreaseReservedSizeIfNecessary(E element) { + if (!releaseReserved(element) || reservedSize <= 0) { + return false; + } + reservedSize--; + return true; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/queue/multilevelqueue/MultilevelPriorityQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/queue/multilevelqueue/MultilevelPriorityQueue.java index c5f3cb3e6df1a..06c7275473b4e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/queue/multilevelqueue/MultilevelPriorityQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/queue/multilevelqueue/MultilevelPriorityQueue.java @@ -191,6 +191,16 @@ protected DriverTask get(DriverTask driverTask) { "MultilevelPriorityQueue does not support access element by get."); } + @Override + protected void markReserved(DriverTask task) { + task.markReservedInReadyQueue(); + } + + @Override + protected boolean releaseReserved(DriverTask task) { + return task.releaseReservedInReadyQueue(); + } + @Override protected void clearAllElements() { highestPriorityLevelQueue.clear(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/task/DriverTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/task/DriverTask.java index f320c6cad6c2f..85bada856caa2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/task/DriverTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/schedule/task/DriverTask.java @@ -59,6 +59,7 @@ public class DriverTask implements IDIndexedAccessible { private final DriverTaskHandle driverTaskHandle; private long lastEnterReadyQueueTime; private long lastEnterBlockQueueTime; + private boolean reservedInReadyQueue; private long estimatedMemorySize; @@ -210,6 +211,18 @@ public void setLastEnterBlockQueueTime(long lastEnterBlockQueueTime) { this.lastEnterBlockQueueTime = lastEnterBlockQueueTime; } + public void markReservedInReadyQueue() { + reservedInReadyQueue = true; + } + + public boolean releaseReservedInReadyQueue() { + if (!reservedInReadyQueue) { + return false; + } + reservedInReadyQueue = false; + return true; + } + /** a comparator of ddl, the less the ddl is, the low order it has. */ public static class TimeoutComparator implements Comparator { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/metric/DriverSchedulerMetricSet.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/metric/DriverSchedulerMetricSet.java index 0b9e0c04f050d..94aaccfd9b7d6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/metric/DriverSchedulerMetricSet.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/metric/DriverSchedulerMetricSet.java @@ -39,6 +39,7 @@ private DriverSchedulerMetricSet() { public static final String READY_QUEUED_TIME = "ready_queued_time"; public static final String BLOCK_QUEUED_TIME = "block_queued_time"; public static final String READY_QUEUE_TASK_COUNT = "ready_queue_task_count"; + public static final String READY_QUEUE_RESERVED_TASK_COUNT = "ready_queue_reserved_task_count"; public static final String BLOCK_QUEUE_TASK_COUNT = "block_queue_task_count"; private static final String TIMEOUT_QUEUE_SIZE = "timeout_queue_task_count"; private static final String QUERY_MAP_SIZE = "query_map_size"; @@ -67,6 +68,13 @@ public void bindTo(AbstractMetricService metricService) { DriverScheduler::getReadyQueueTaskCount, Tag.NAME.toString(), READY_QUEUE_TASK_COUNT); + metricService.createAutoGauge( + Metric.DRIVER_SCHEDULER.toString(), + MetricLevel.IMPORTANT, + DriverScheduler.getInstance(), + DriverScheduler::getReadyQueueReservedTaskCount, + Tag.NAME.toString(), + READY_QUEUE_RESERVED_TASK_COUNT); metricService.createAutoGauge( Metric.DRIVER_SCHEDULER.toString(), MetricLevel.IMPORTANT, @@ -109,6 +117,11 @@ public void unbindFrom(AbstractMetricService metricService) { Metric.DRIVER_SCHEDULER.toString(), Tag.NAME.toString(), READY_QUEUE_TASK_COUNT); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.DRIVER_SCHEDULER.toString(), + Tag.NAME.toString(), + READY_QUEUE_RESERVED_TASK_COUNT); metricService.remove( MetricType.AUTO_GAUGE, Metric.DRIVER_SCHEDULER.toString(), diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/schedule/DefaultDriverSchedulerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/schedule/DefaultDriverSchedulerTest.java index 0cc0e5edd71d5..595d5febe4c6e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/schedule/DefaultDriverSchedulerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/execution/schedule/DefaultDriverSchedulerTest.java @@ -197,6 +197,47 @@ public void testReadyToRunning() { clear(); } + @Test + public void testAbortReadyTaskAfterPollReleasesReadyQueueReservation() throws Exception { + IMPPDataExchangeManager mockMPPDataExchangeManager = + Mockito.mock(IMPPDataExchangeManager.class); + manager.setBlockManager(mockMPPDataExchangeManager); + ITaskScheduler defaultScheduler = manager.getScheduler(); + IDriver mockDriver = Mockito.mock(IDriver.class); + DriverTaskHandle driverTaskHandle = + new DriverTaskHandle( + 1, + (MultilevelPriorityQueue) manager.getReadyQueue(), + OptionalInt.of(Integer.MAX_VALUE)); + + QueryId queryId = new QueryId("test"); + FragmentInstanceId instanceId = + new FragmentInstanceId(new PlanFragmentId(queryId, 0), "inst-0"); + DriverTaskId driverTaskID = new DriverTaskId(instanceId, 0); + Mockito.when(mockDriver.getDriverTaskId()).thenReturn(driverTaskID); + DriverTask testTask = + new DriverTask(mockDriver, 100L, DriverTaskStatus.READY, driverTaskHandle, 0, false); + + try { + manager.registerTaskToQueryMap(queryId, testTask); + manager.submitTaskToReadyQueue(testTask); + + DriverTask polledTask = manager.getReadyQueue().poll(); + Assert.assertSame(testTask, polledTask); + Assert.assertEquals(1, manager.getReadyQueueReservedTaskCount()); + + manager.abortFragmentInstance(instanceId); + + Assert.assertEquals(DriverTaskStatus.ABORTED, testTask.getStatus()); + Assert.assertEquals(0, manager.getReadyQueue().size()); + Assert.assertEquals(0, manager.getReadyQueueReservedTaskCount()); + Assert.assertFalse(defaultScheduler.readyToRunning(polledTask)); + Assert.assertEquals(0, manager.getReadyQueueReservedTaskCount()); + } finally { + clear(); + } + } + @Test public void testRunningToReady() { IMPPDataExchangeManager mockMPPDataExchangeManager = From cef9184c294496063957ac047b5133fef33f8392 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 16 Jun 2026 09:36:03 +0800 Subject: [PATCH 101/102] Pipe: avoid blocking startup on hardlink dir cleanup (#17932) (#17943) (cherry picked from commit 8fd6b0490dacec1f4034ed427c9062ab10447d1a) --- .../runtime/PipeDataNodeRuntimeAgent.java | 2 +- ...HardlinkOrCopiedFileDirStartupCleaner.java | 257 +++++++++++++++++- 2 files changed, 248 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java index 437c15bdc1c6d..d47a8c4dbbd8a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java @@ -75,7 +75,7 @@ public class PipeDataNodeRuntimeAgent implements IService { public synchronized void preparePipeResources( final ResourcesInformationHolder resourcesInformationHolder) throws StartupException { // Clean sender (connector) hardlink file dir and snapshot dir - PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.clean(); + PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.clean(this::registerPeriodicalJob); // Clean receiver file dir PipeDataNodeAgent.receiver().cleanPipeReceiverDirs(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java index b58d934988f1a..009daea6c243b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java @@ -28,33 +28,270 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; public class PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner { private static final Logger LOGGER = LoggerFactory.getLogger(PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.class); + private static final String STALE_PIPE_DIR_SUFFIX = ".startup-cleaning-"; + private static final String PERIODICAL_CLEANUP_JOB_ID = + "PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner#cleanTsFileDir()"; + private static final long DELETE_MAX_PATH_COUNT_PER_ROUND = 100_000L; + private static final long DELETE_MAX_TIME_PER_ROUND_MS = 1_000L; /** * Delete the data directory and all of its subdirectories that contain the * PipeConfig.PIPE_TSFILE_DIR_NAME directory. */ - public static void clean() { - cleanTsFileDir(); + public static void clean(final PeriodicalJobRegistrar periodicalJobRegistrar) { + cleanTsFileDir(periodicalJobRegistrar); cleanSnapshotDir(); } - private static void cleanTsFileDir() { + private static void cleanTsFileDir(final PeriodicalJobRegistrar periodicalJobRegistrar) { + final String pipeHardlinkBaseDirName = PipeConfig.getInstance().getPipeHardlinkBaseDirName(); + final List stalePipeDirs = new ArrayList<>(); for (final String dataDir : IoTDBDescriptor.getInstance().getConfig().getDataDirs()) { - final File pipeHardLinkDir = - new File( - dataDir + File.separator + PipeConfig.getInstance().getPipeHardlinkBaseDirName()); + final File localDataDir = new File(dataDir); + collectInterruptedStalePipeDirs(localDataDir, pipeHardlinkBaseDirName, stalePipeDirs); + + final File pipeHardLinkDir = new File(localDataDir, pipeHardlinkBaseDirName); if (pipeHardLinkDir.isDirectory()) { - LOGGER.info( - "Pipe hardlink dir found, deleting it: {}, result: {}", - pipeHardLinkDir, - FileUtils.deleteQuietly(pipeHardLinkDir)); + moveAsideAndCollect(pipeHardLinkDir, pipeHardlinkBaseDirName, stalePipeDirs); } } + registerPeriodicalCleanupJob(periodicalJobRegistrar, stalePipeDirs); + } + + private static void collectInterruptedStalePipeDirs( + final File localDataDir, + final String pipeHardlinkBaseDirName, + final List stalePipeDirs) { + final File[] stalePipeDirFiles = + localDataDir.listFiles( + file -> + file.isDirectory() + && file.getName().startsWith(pipeHardlinkBaseDirName + STALE_PIPE_DIR_SUFFIX)); + if (stalePipeDirFiles == null) { + return; + } + + for (final File stalePipeDir : stalePipeDirFiles) { + LOGGER.info( + "Stale pipe hardlink dir found, registering it for throttled periodical deletion: {}", + stalePipeDir); + stalePipeDirs.add(stalePipeDir); + } + } + + private static void moveAsideAndCollect( + final File pipeHardLinkDir, + final String pipeHardlinkBaseDirName, + final List stalePipeDirs) { + try { + final File stalePipeDir = moveAside(pipeHardLinkDir, pipeHardlinkBaseDirName); + LOGGER.info( + "Pipe hardlink dir found, moved it from {} to {} for throttled periodical deletion.", + pipeHardLinkDir, + stalePipeDir); + stalePipeDirs.add(stalePipeDir); + } catch (final IOException e) { + LOGGER.warn( + "Failed to move pipe hardlink dir {} for periodical deletion, deleting it synchronously.", + pipeHardLinkDir, + e); + LOGGER.info( + "Pipe hardlink dir found, deleting it: {}, result: {}", + pipeHardLinkDir, + FileUtils.deleteQuietly(pipeHardLinkDir)); + } + } + + private static File moveAside(final File pipeHardLinkDir, final String pipeHardlinkBaseDirName) + throws IOException { + final File parentDir = pipeHardLinkDir.getParentFile(); + if (parentDir == null) { + throw new IOException("Failed to get parent dir of " + pipeHardLinkDir); + } + + final long timestamp = System.currentTimeMillis(); + for (int i = 0; ; ++i) { + final File stalePipeDir = + new File( + parentDir, pipeHardlinkBaseDirName + STALE_PIPE_DIR_SUFFIX + timestamp + "-" + i); + if (!stalePipeDir.exists()) { + Files.move(pipeHardLinkDir.toPath(), stalePipeDir.toPath()); + return stalePipeDir; + } + } + } + + private static void registerPeriodicalCleanupJob( + final PeriodicalJobRegistrar periodicalJobRegistrar, final List stalePipeDirs) { + if (stalePipeDirs.isEmpty()) { + return; + } + + periodicalJobRegistrar.register( + PERIODICAL_CLEANUP_JOB_ID, + new PeriodicalStalePipeDirCleaner(stalePipeDirs)::cleanOneRound, + PipeConfig.getInstance().getPipeSubtaskExecutorCronHeartbeatEventIntervalSeconds()); + } + + private static CleanupRoundResult deleteQuietlyWithThrottle(final File stalePipeDir) { + if (!stalePipeDir.exists()) { + return CleanupRoundResult.finished(); + } + + final AtomicBoolean deleteResult = new AtomicBoolean(true); + final AtomicLong deletedPathCount = new AtomicLong(0); + final long deadlineNanos = System.nanoTime() + DELETE_MAX_TIME_PER_ROUND_MS * 1_000_000L; + try { + Files.walkFileTree( + stalePipeDir.toPath(), + new SimpleFileVisitor() { + @Override + public FileVisitResult preVisitDirectory( + final Path dir, final BasicFileAttributes attrs) { + return shouldStop(deletedPathCount, deadlineNanos) + ? FileVisitResult.TERMINATE + : FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult visitFile(final Path file, final BasicFileAttributes attrs) { + return deletePath(file, deleteResult, deletedPathCount, deadlineNanos); + } + + @Override + public FileVisitResult visitFileFailed(final Path file, final IOException exc) { + deleteResult.set(false); + return deletePath(file, deleteResult, deletedPathCount, deadlineNanos); + } + + @Override + public FileVisitResult postVisitDirectory(final Path dir, final IOException exc) { + if (exc != null) { + deleteResult.set(false); + } + return deletePath(dir, deleteResult, deletedPathCount, deadlineNanos); + } + }); + } catch (final IOException e) { + deleteResult.set(false); + } + + return new CleanupRoundResult( + deletedPathCount.get(), deleteResult.get() && !stalePipeDir.exists(), deleteResult.get()); + } + + private static FileVisitResult deletePath( + final Path path, + final AtomicBoolean deleteResult, + final AtomicLong deletedPathCount, + final long deadlineNanos) { + if (shouldStop(deletedPathCount, deadlineNanos)) { + return FileVisitResult.TERMINATE; + } + + try { + if (Files.deleteIfExists(path)) { + deletedPathCount.incrementAndGet(); + } + } catch (final IOException e) { + deleteResult.set(false); + } + return shouldStop(deletedPathCount, deadlineNanos) + ? FileVisitResult.TERMINATE + : FileVisitResult.CONTINUE; + } + + private static boolean shouldStop(final AtomicLong deletedPathCount, final long deadlineNanos) { + return Thread.currentThread().isInterrupted() + || deletedPathCount.get() >= DELETE_MAX_PATH_COUNT_PER_ROUND + || System.nanoTime() >= deadlineNanos; + } + + @FunctionalInterface + public interface PeriodicalJobRegistrar { + + void register(String id, Runnable periodicalJob, long intervalInSeconds); + } + + private static class PeriodicalStalePipeDirCleaner { + + private final List stalePipeDirs; + private int currentDirIndex; + private boolean finished; + + private PeriodicalStalePipeDirCleaner(final List stalePipeDirs) { + this.stalePipeDirs = stalePipeDirs; + currentDirIndex = 0; + finished = false; + } + + private void cleanOneRound() { + if (finished) { + return; + } + + long deletedPathCount = 0; + while (currentDirIndex < stalePipeDirs.size()) { + final File stalePipeDir = stalePipeDirs.get(currentDirIndex); + final CleanupRoundResult result = deleteQuietlyWithThrottle(stalePipeDir); + deletedPathCount += result.deletedPathCount; + + if (result.finished) { + LOGGER.info( + "Finished deleting stale pipe hardlink dir {} by periodical job, result: {}", + stalePipeDir, + result.success); + ++currentDirIndex; + continue; + } + + if (deletedPathCount > 0 || !result.success) { + LOGGER.info( + "Periodically deleted {} paths from stale pipe hardlink dirs, current dir: {}, " + + "current round result: {}", + deletedPathCount, + stalePipeDir, + result.success); + } + return; + } + + finished = true; + LOGGER.info("Finished deleting all stale pipe hardlink dirs by periodical job."); + } + } + + private static class CleanupRoundResult { + + private final long deletedPathCount; + private final boolean finished; + private final boolean success; + + private CleanupRoundResult( + final long deletedPathCount, final boolean finished, final boolean success) { + this.deletedPathCount = deletedPathCount; + this.finished = finished; + this.success = success; + } + + private static CleanupRoundResult finished() { + return new CleanupRoundResult(0, true, true); + } } private static void cleanSnapshotDir() { From c1ee977bed1b9e3252c1a238e435639045290923 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 16 Jun 2026 18:18:59 +0800 Subject: [PATCH 102/102] Fix duplicate scheduling in procedure execution (#17902) * Fix duplicate scheduling in procedure execution * Fix delayed procedure deduplication and semaphore release * Fix SQL parser error handler traversal * Fix pipe procedure lock release race * Fix procedure lock wait scheduling (cherry picked from commit c25849a093b466966353ecd7b63f722535b8d5b3) (cherry picked from commit 0a45a3b7b708c3b436e6b9b4c6aa9b65bede7076) --- .../task/PipeTaskCoordinatorLock.java | 13 +- .../iotdb/confignode/procedure/Procedure.java | 20 ++++ .../procedure/ProcedureExecutor.java | 113 ++++++++++++------ .../procedure/TimeoutExecutorThread.java | 32 ++++- .../procedure/impl/StateMachineProcedure.java | 3 +- .../procedure/impl/cq/CreateCQProcedure.java | 2 +- .../impl/node/AbstractNodeProcedure.java | 12 ++ .../pipe/AbstractOperatePipeProcedureV2.java | 23 ++-- .../plugin/CreatePipePluginProcedure.java | 2 +- .../pipe/plugin/DropPipePluginProcedure.java | 2 +- .../PipeHandleLeaderChangeProcedure.java | 2 +- .../PipeHandleMetaChangeProcedure.java | 2 +- .../impl/pipe/task/DropPipeProcedureV2.java | 2 +- .../impl/pipe/task/StartPipeProcedureV2.java | 2 +- .../impl/pipe/task/StopPipeProcedureV2.java | 2 +- .../impl/schema/DeleteDatabaseProcedure.java | 2 +- .../schema/DeleteLogicalViewProcedure.java | 2 +- .../schema/DeleteTimeSeriesProcedure.java | 2 +- .../impl/trigger/CreateTriggerProcedure.java | 2 +- .../impl/trigger/DropTriggerProcedure.java | 2 +- .../procedure/scheduler/LockQueue.java | 10 +- .../scheduler/SimpleProcedureScheduler.java | 37 +++++- .../task/PipeTaskCoordinatorLockTest.java | 60 ++++++++++ .../confignode/procedure/TestLockRegime.java | 23 ++++ .../procedure/TestProcedureExecutor.java | 92 ++++++++++++++ .../procedure/entity/SimpleLockProcedure.java | 11 +- .../PipeHandleLeaderChangeProcedureTest.java | 47 ++++++++ 27 files changed, 433 insertions(+), 89 deletions(-) create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinatorLockTest.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinatorLock.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinatorLock.java index b86c556f20df8..58347f35af5a6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinatorLock.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinatorLock.java @@ -43,16 +43,9 @@ public class PipeTaskCoordinatorLock { public void lock() { LOGGER.debug( "PipeTaskCoordinator lock waiting for thread {}", Thread.currentThread().getName()); - try { - semaphore.acquire(); - LOGGER.debug( - "PipeTaskCoordinator lock acquired by thread {}", Thread.currentThread().getName()); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - LOGGER.error( - "Interrupted while waiting for PipeTaskCoordinator lock, current thread: {}", - Thread.currentThread().getName()); - } + semaphore.acquireUninterruptibly(); + LOGGER.debug( + "PipeTaskCoordinator lock acquired by thread {}", Thread.currentThread().getName()); } public boolean tryLock() { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/Procedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/Procedure.java index 91af03d3971be..862bc449c7319 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/Procedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/Procedure.java @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; /** @@ -60,6 +61,7 @@ public abstract class Procedure implements Comparable> { private volatile long lastUpdate; private final AtomicReference result = new AtomicReference<>(); + private final AtomicBoolean executing = new AtomicBoolean(false); private volatile boolean locked = false; private boolean lockedWhenLoading = false; @@ -233,6 +235,16 @@ protected void releaseLock(Env env) { // no op } + /** + * Called after an execution attempt returns {@link ProcedureLockState#LOCK_EVENT_WAIT}. Override + * it to put the procedure into the corresponding lock wait queue. + * + * @param env env + */ + protected void waitForLock(Env env) { + // no op + } + /** * Used to keep procedure lock even when the procedure is yielded or suspended. * @@ -254,6 +266,14 @@ protected boolean isYieldAfterExecution(Env env) { } // -------------------------Internal methods - called by the procedureExecutor------------------ + final boolean tryAcquireExecution() { + return executing.compareAndSet(false, true); + } + + final void releaseExecution() { + executing.set(false); + } + /** * Internal method called by the ProcedureExecutor that starts the user-level code execute(). * diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/ProcedureExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/ProcedureExecutor.java index 0d8368583b4eb..1633f78eec903 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/ProcedureExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/ProcedureExecutor.java @@ -21,7 +21,6 @@ import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.utils.TestOnly; -import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.scheduler.ProcedureScheduler; import org.apache.iotdb.confignode.procedure.scheduler.SimpleProcedureScheduler; @@ -40,7 +39,6 @@ import java.util.Deque; import java.util.HashSet; import java.util.List; -import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; @@ -82,6 +80,16 @@ public class ProcedureExecutor { private final Env environment; private final IProcedureStore store; + private static final class LockStateResult { + private final ProcedureLockState lockState; + private final Procedure procedure; + + private LockStateResult(ProcedureLockState lockState, Procedure procedure) { + this.lockState = lockState; + this.procedure = procedure; + } + } + public ProcedureExecutor( final Env environment, final IProcedureStore store, final ProcedureScheduler scheduler) { this.environment = environment; @@ -320,32 +328,38 @@ private void executeProcedure(Procedure proc) { return; } ProcedureLockState lockState = null; + Procedure lockEventWaitProcedure = null; try { do { if (!rootProcStack.acquire()) { if (rootProcStack.setRollback()) { - lockState = executeRootStackRollback(rootProcId, rootProcStack); + LockStateResult lockStateResult = + executeRootStackRollback(rootProcId, rootProcStack); + lockState = lockStateResult.lockState; switch (lockState) { case LOCK_ACQUIRED: break; case LOCK_EVENT_WAIT: - LOG.info("LOCK_EVENT_WAIT rollback {}", proc); + LOG.info("LOCK_EVENT_WAIT rollback {}", lockStateResult.procedure); rootProcStack.unsetRollback(); + lockEventWaitProcedure = lockStateResult.procedure; break; case LOCK_YIELD_WAIT: rootProcStack.unsetRollback(); - scheduler.yield(proc); + scheduler.yield(lockStateResult.procedure); break; default: throw new UnsupportedOperationException(); } } else { if (!proc.wasExecuted()) { - switch (executeRollback(proc)) { + lockState = executeRollback(proc); + switch (lockState) { case LOCK_ACQUIRED: break; case LOCK_EVENT_WAIT: LOG.info("LOCK_EVENT_WAIT can't rollback child running for {}", proc); + lockEventWaitProcedure = proc; break; case LOCK_YIELD_WAIT: scheduler.yield(proc); @@ -357,19 +371,25 @@ private void executeProcedure(Procedure proc) { } break; } - lockState = acquireLock(proc); - switch (lockState) { - case LOCK_ACQUIRED: - executeProcedure(rootProcStack, proc); - break; - case LOCK_YIELD_WAIT: - case LOCK_EVENT_WAIT: - LOG.info("{} lockstate is {}", proc, lockState); - break; - default: - throw new UnsupportedOperationException(); + try { + lockState = acquireLock(proc); + switch (lockState) { + case LOCK_ACQUIRED: + executeProcedure(rootProcStack, proc); + break; + case LOCK_YIELD_WAIT: + case LOCK_EVENT_WAIT: + LOG.info("{} lockstate is {}", proc, lockState); + if (lockState == ProcedureLockState.LOCK_EVENT_WAIT) { + lockEventWaitProcedure = proc; + } + break; + default: + throw new UnsupportedOperationException(); + } + } finally { + rootProcStack.release(); } - rootProcStack.release(); if (proc.isSuccess()) { // update metrics on finishing the procedure @@ -387,9 +407,9 @@ private void executeProcedure(Procedure proc) { } finally { // Only after procedure has completed execution can it be allowed to be rescheduled to prevent // data races - if (Objects.equals(lockState, ProcedureLockState.LOCK_EVENT_WAIT)) { - LOG.info("procedureId {} wait for lock.", proc.getProcId()); - ((ConfigNodeProcedureEnv) this.environment).getNodeLock().waitProcedure(proc); + if (lockEventWaitProcedure != null) { + LOG.info("procedureId {} wait for lock.", lockEventWaitProcedure.getProcId()); + lockEventWaitProcedure.waitForLock(this.environment); } } } @@ -404,6 +424,7 @@ private void executeProcedure(RootProcedureStack rootProcStack, Procedure p if (proc.getState() != ProcedureState.RUNNABLE) { LOG.error( "The executing procedure should in RUNNABLE state, but it's not. Procedure is {}", proc); + releaseLock(proc, false); return; } boolean reExecute; @@ -570,8 +591,8 @@ private void yieldProcedure(Procedure proc) { * @param procedureStack root procedure stack * @return lock state */ - private ProcedureLockState executeRootStackRollback( - Long rootProcId, RootProcedureStack procedureStack) { + private LockStateResult executeRootStackRollback( + Long rootProcId, RootProcedureStack procedureStack) { Procedure rootProcedure = procedures.get(rootProcId); ProcedureException exception = rootProcedure.getException(); if (exception == null) { @@ -590,7 +611,7 @@ private ProcedureLockState executeRootStackRollback( } ProcedureLockState lockState = acquireLock(procedure); if (lockState != ProcedureLockState.LOCK_ACQUIRED) { - return lockState; + return new LockStateResult<>(lockState, procedure); } lockState = executeRollback(procedure); releaseLock(procedure, false); @@ -598,11 +619,11 @@ private ProcedureLockState executeRootStackRollback( boolean abortRollback = lockState != ProcedureLockState.LOCK_ACQUIRED; abortRollback |= !isRunning() || !store.isRunning(); if (abortRollback) { - return lockState; + return new LockStateResult<>(lockState, procedure); } if (!procedure.isFinished() && procedure.isYieldAfterExecution(this.environment)) { - return ProcedureLockState.LOCK_YIELD_WAIT; + return new LockStateResult<>(ProcedureLockState.LOCK_YIELD_WAIT, procedure); } if (procedure != rootProcedure) { @@ -612,7 +633,7 @@ private ProcedureLockState executeRootStackRollback( LOG.info("Rolled back {}, time duration is {}", rootProcedure, rootProcedure.elapsedTime()); rootProcedureCleanup(rootProcedure); - return ProcedureLockState.LOCK_ACQUIRED; + return new LockStateResult<>(ProcedureLockState.LOCK_ACQUIRED, rootProcedure); } private ProcedureLockState acquireLock(Procedure proc) { @@ -728,16 +749,33 @@ public void run() { Thread.sleep(1000); continue; } - this.activeProcedure.set(procedure); - activeExecutorCount.incrementAndGet(); - startTime.set(System.currentTimeMillis()); - executeProcedure(procedure); - activeExecutorCount.decrementAndGet(); - LOG.trace( - "Halt pid={}, activeCount={}", procedure.getProcId(), activeExecutorCount.get()); - this.activeProcedure.set(null); - lastUpdated = System.currentTimeMillis(); - startTime.set(lastUpdated); + boolean executionAcquired = false; + while (isRunning() && !(executionAcquired = procedure.tryAcquireExecution())) { + Thread.sleep(10); + } + if (!executionAcquired) { + continue; + } + try { + this.activeProcedure.set(procedure); + activeExecutorCount.incrementAndGet(); + startTime.set(System.currentTimeMillis()); + try { + executeProcedure(procedure); + } finally { + procedure.releaseExecution(); + activeExecutorCount.decrementAndGet(); + LOG.trace( + "Halt pid={}, activeCount={}", procedure.getProcId(), activeExecutorCount.get()); + this.activeProcedure.set(null); + lastUpdated = System.currentTimeMillis(); + startTime.set(lastUpdated); + } + } catch (Exception e) { + LOG.warn( + "Exception happened when worker {} execute procedure {}", getName(), procedure, e); + throw e; + } } } catch (Exception e) { @@ -748,6 +786,7 @@ public void run() { this.activeProcedure.get(), e); } + this.activeProcedure.set(null); } finally { LOG.info("Procedure worker {} terminated.", getName()); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/TimeoutExecutorThread.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/TimeoutExecutorThread.java index 5aaf9a623f523..c998ad903c208 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/TimeoutExecutorThread.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/TimeoutExecutorThread.java @@ -37,11 +37,13 @@ public TimeoutExecutorThread( } public void add(Procedure procedure) { - queue.add(new ProcedureDelayContainer<>(procedure)); + ProcedureDelayContainer delayTask = new ProcedureDelayContainer<>(procedure); + queue.remove(delayTask); + queue.add(delayTask); } public boolean remove(Procedure procedure) { - return queue.remove(new ProcedureDelayContainer<>(procedure)); + return queue.remove(new ProcedureDelayContainer<>(procedure)) || procedure.isFinished(); } private ProcedureDelayContainer takeQuietly() { @@ -62,10 +64,15 @@ public void run() { } Procedure procedure = delayTask.getProcedure(); if (procedure instanceof InternalProcedure) { + if (procedure.isFinished()) { + continue; + } InternalProcedure internal = (InternalProcedure) procedure; internal.periodicExecute(executor.getEnvironment()); - procedure.updateTimestamp(); - queue.add(delayTask); + if (!procedure.isFinished()) { + procedure.updateTimestamp(); + queue.add(delayTask); + } } else { if (procedure.setTimeoutFailure(executor.getEnvironment())) { long rootProcId = executor.getRootProcedureId(procedure); @@ -92,6 +99,23 @@ public Procedure getProcedure() { return procedure; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ProcedureDelayContainer)) { + return false; + } + ProcedureDelayContainer that = (ProcedureDelayContainer) o; + return procedure == that.procedure; + } + + @Override + public int hashCode() { + return System.identityHashCode(procedure); + } + @Override public long getDelay(TimeUnit unit) { long delay = procedure.getTimeoutTimestamp() - System.currentTimeMillis(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java index f698735ee95be..400da5653181c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java @@ -195,7 +195,8 @@ private void addNextStateAndCalculateCycles() { nextState); } } - if (getStateId(getCurrentState()) == stateToBeAdded) { + final TState currentState = getCurrentState(); + if (currentState != null && getStateId(currentState) == stateToBeAdded) { cycles++; } else { cycles = 0; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java index af7f968e8a57c..76df7765c9a57 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/cq/CreateCQProcedure.java @@ -295,7 +295,7 @@ public boolean equals(Object o) { } CreateCQProcedure that = (CreateCQProcedure) o; return getProcId() == that.getProcId() - && getCurrentState().equals(that.getCurrentState()) + && Objects.equals(getCurrentState(), that.getCurrentState()) && getCycles() == that.getCycles() && isGeneratedByPipe == that.isGeneratedByPipe && firstExecutionTime == that.firstExecutionTime diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/AbstractNodeProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/AbstractNodeProcedure.java index b141027917366..6cade537f1fbb 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/AbstractNodeProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/AbstractNodeProcedure.java @@ -56,6 +56,18 @@ protected ProcedureLockState acquireLock(ConfigNodeProcedureEnv configNodeProced } } + @Override + protected void waitForLock(ConfigNodeProcedureEnv configNodeProcedureEnv) { + configNodeProcedureEnv.getSchedulerLock().lock(); + try { + configNodeProcedureEnv + .getNodeLock() + .waitProcedure(this, configNodeProcedureEnv.getScheduler()); + } finally { + configNodeProcedureEnv.getSchedulerLock().unlock(); + } + } + @Override protected void releaseLock(ConfigNodeProcedureEnv configNodeProcedureEnv) { configNodeProcedureEnv.getSchedulerLock().lock(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/AbstractOperatePipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/AbstractOperatePipeProcedureV2.java index 857d5733f6a7a..43620ac915796 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/AbstractOperatePipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/AbstractOperatePipeProcedureV2.java @@ -133,12 +133,7 @@ protected ProcedureLockState acquireLock(ConfigNodeProcedureEnv configNodeProced LOGGER.warn("ProcedureId {}: LOCK_EVENT_WAIT. Without acquiring pipe lock.", getProcId()); } else { LOGGER.debug("ProcedureId {}: LOCK_EVENT_WAIT. Pipe lock will be released.", getProcId()); - configNodeProcedureEnv - .getConfigManager() - .getPipeManager() - .getPipeTaskCoordinator() - .unlock(); - pipeTaskInfo = null; + releasePipeTaskCoordinatorLock(configNodeProcedureEnv); } break; default: @@ -152,12 +147,7 @@ protected ProcedureLockState acquireLock(ConfigNodeProcedureEnv configNodeProced "ProcedureId {}: {}. Invalid lock state. Pipe lock will be released.", getProcId(), procedureLockState); - configNodeProcedureEnv - .getConfigManager() - .getPipeManager() - .getPipeTaskCoordinator() - .unlock(); - pipeTaskInfo = null; + releasePipeTaskCoordinatorLock(configNodeProcedureEnv); } break; } @@ -181,11 +171,16 @@ protected void releaseLock(ConfigNodeProcedureEnv configNodeProcedureEnv) { } PipeProcedureMetrics.getInstance() .updateTimer(this.getOperation().getName(), this.elapsedTime()); - configNodeProcedureEnv.getConfigManager().getPipeManager().getPipeTaskCoordinator().unlock(); - pipeTaskInfo = null; + releasePipeTaskCoordinatorLock(configNodeProcedureEnv); } } + private void releasePipeTaskCoordinatorLock(ConfigNodeProcedureEnv configNodeProcedureEnv) { + // Clear before releasing the semaphore to avoid clobbering a re-scheduled execution's marker. + pipeTaskInfo = null; + configNodeProcedureEnv.getConfigManager().getPipeManager().getPipeTaskCoordinator().unlock(); + } + protected abstract PipeTaskOperation getOperation(); /** diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/plugin/CreatePipePluginProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/plugin/CreatePipePluginProcedure.java index f4fa738428d43..22fe0ef40f279 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/plugin/CreatePipePluginProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/plugin/CreatePipePluginProcedure.java @@ -314,7 +314,7 @@ public boolean equals(Object that) { if (that instanceof CreatePipePluginProcedure) { CreatePipePluginProcedure thatProcedure = (CreatePipePluginProcedure) that; return thatProcedure.getProcId() == getProcId() - && thatProcedure.getCurrentState().equals(getCurrentState()) + && Objects.equals(thatProcedure.getCurrentState(), getCurrentState()) && thatProcedure.getCycles() == getCycles() && thatProcedure.pipePluginMeta.equals(pipePluginMeta); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/plugin/DropPipePluginProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/plugin/DropPipePluginProcedure.java index efbe1ee6ccdda..5deb0ee4bf31f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/plugin/DropPipePluginProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/plugin/DropPipePluginProcedure.java @@ -275,7 +275,7 @@ public boolean equals(Object that) { if (that instanceof DropPipePluginProcedure) { final DropPipePluginProcedure thatProcedure = (DropPipePluginProcedure) that; return thatProcedure.getProcId() == getProcId() - && thatProcedure.getCurrentState().equals(this.getCurrentState()) + && Objects.equals(thatProcedure.getCurrentState(), this.getCurrentState()) && thatProcedure.getCycles() == this.getCycles() && (thatProcedure.pluginName).equals(pluginName); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedure.java index 61f6f3cae2aaf..598d8192e06af 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedure.java @@ -183,7 +183,7 @@ public boolean equals(Object o) { } PipeHandleLeaderChangeProcedure that = (PipeHandleLeaderChangeProcedure) o; return getProcId() == that.getProcId() - && getCurrentState().equals(that.getCurrentState()) + && Objects.equals(getCurrentState(), that.getCurrentState()) && getCycles() == that.getCycles() && this.regionGroupToOldAndNewLeaderPairMap.equals( that.regionGroupToOldAndNewLeaderPairMap); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleMetaChangeProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleMetaChangeProcedure.java index 401859f0a7e0a..def062359e4c4 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleMetaChangeProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleMetaChangeProcedure.java @@ -186,7 +186,7 @@ public boolean equals(Object o) { } PipeHandleMetaChangeProcedure that = (PipeHandleMetaChangeProcedure) o; return getProcId() == that.getProcId() - && getCurrentState().equals(that.getCurrentState()) + && Objects.equals(getCurrentState(), that.getCurrentState()) && getCycles() == that.getCycles() && needWriteConsensusOnConfigNodes == that.needWriteConsensusOnConfigNodes && needPushPipeMetaToDataNodes == that.needPushPipeMetaToDataNodes; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/DropPipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/DropPipeProcedureV2.java index 0c7042caf3f07..0f9a8bc0d8c84 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/DropPipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/DropPipeProcedureV2.java @@ -165,7 +165,7 @@ public boolean equals(Object o) { } DropPipeProcedureV2 that = (DropPipeProcedureV2) o; return getProcId() == that.getProcId() - && getCurrentState().equals(that.getCurrentState()) + && Objects.equals(getCurrentState(), that.getCurrentState()) && getCycles() == that.getCycles() && pipeName.equals(that.pipeName); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StartPipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StartPipeProcedureV2.java index fe36137b35f47..29cb2b51ab4e5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StartPipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StartPipeProcedureV2.java @@ -184,7 +184,7 @@ public boolean equals(Object o) { } StartPipeProcedureV2 that = (StartPipeProcedureV2) o; return getProcId() == that.getProcId() - && getCurrentState().equals(that.getCurrentState()) + && Objects.equals(getCurrentState(), that.getCurrentState()) && getCycles() == that.getCycles() && pipeName.equals(that.pipeName); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java index b2e1a584ec54a..44ca8a7bd5c1a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java @@ -187,7 +187,7 @@ public boolean equals(Object o) { } StopPipeProcedureV2 that = (StopPipeProcedureV2) o; return getProcId() == that.getProcId() - && getCurrentState().equals(that.getCurrentState()) + && Objects.equals(getCurrentState(), that.getCurrentState()) && getCycles() == that.getCycles() && isStoppedByRuntimeExceptionBeforeStop == that.isStoppedByRuntimeExceptionBeforeStop && pipeName.equals(that.pipeName); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java index 7288090e81559..b6ad21128af3b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java @@ -303,7 +303,7 @@ public boolean equals(Object that) { if (that instanceof DeleteDatabaseProcedure) { DeleteDatabaseProcedure thatProc = (DeleteDatabaseProcedure) that; return thatProc.getProcId() == this.getProcId() - && thatProc.getCurrentState().equals(this.getCurrentState()) + && Objects.equals(thatProc.getCurrentState(), this.getCurrentState()) && thatProc.getCycles() == this.getCycles() && thatProc.isGeneratedByPipe == this.isGeneratedByPipe && thatProc.deleteDatabaseSchema.equals(this.getDeleteDatabaseSchema()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java index aaf7bc1a7bff0..fd5d0d52f227c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java @@ -303,7 +303,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; DeleteLogicalViewProcedure that = (DeleteLogicalViewProcedure) o; return this.getProcId() == that.getProcId() - && this.getCurrentState().equals(that.getCurrentState()) + && Objects.equals(this.getCurrentState(), that.getCurrentState()) && this.getCycles() == that.getCycles() && isGeneratedByPipe == that.isGeneratedByPipe && patternTree.equals(that.patternTree); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java index 74f21abc72da2..0a223de1525f3 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java @@ -367,7 +367,7 @@ public boolean equals(final Object o) { if (o == null || getClass() != o.getClass()) return false; final DeleteTimeSeriesProcedure that = (DeleteTimeSeriesProcedure) o; return this.getProcId() == that.getProcId() - && this.getCurrentState().equals(that.getCurrentState()) + && Objects.equals(this.getCurrentState(), that.getCurrentState()) && this.getCycles() == getCycles() && this.isGeneratedByPipe == that.isGeneratedByPipe && this.patternTree.equals(that.patternTree); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java index 74f4112075a7e..548ba1ee22a35 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java @@ -304,7 +304,7 @@ public boolean equals(Object that) { if (that instanceof CreateTriggerProcedure) { CreateTriggerProcedure thatProc = (CreateTriggerProcedure) that; return thatProc.getProcId() == this.getProcId() - && thatProc.getCurrentState().equals(this.getCurrentState()) + && Objects.equals(thatProc.getCurrentState(), this.getCurrentState()) && thatProc.getCycles() == this.getCycles() && thatProc.isGeneratedByPipe == this.isGeneratedByPipe && thatProc.triggerInformation.equals(this.triggerInformation); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java index 19bfcdc30d2ba..958000d2b3fd6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java @@ -178,7 +178,7 @@ public boolean equals(Object that) { if (that instanceof DropTriggerProcedure) { DropTriggerProcedure thatProc = (DropTriggerProcedure) that; return thatProc.getProcId() == this.getProcId() - && thatProc.getCurrentState().equals(this.getCurrentState()) + && Objects.equals(thatProc.getCurrentState(), this.getCurrentState()) && thatProc.getCycles() == this.getCycles() && thatProc.isGeneratedByPipe == this.isGeneratedByPipe && (thatProc.triggerName).equals(this.triggerName); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/scheduler/LockQueue.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/scheduler/LockQueue.java index 832e339c0aede..e2f5935a909ab 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/scheduler/LockQueue.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/scheduler/LockQueue.java @@ -45,7 +45,15 @@ public boolean releaseLock(Procedure procedure) { return true; } - public void waitProcedure(Procedure procedure) { + public void waitProcedure(Procedure procedure, ProcedureScheduler procedureScheduler) { + if (lockOwnerProcedure == null) { + procedureScheduler.addFront(procedure); + return; + } + if (deque.stream() + .anyMatch(waitingProcedure -> waitingProcedure.getProcId() == procedure.getProcId())) { + return; + } deque.addLast(procedure); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/scheduler/SimpleProcedureScheduler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/scheduler/SimpleProcedureScheduler.java index 3cd5ceacf4b6d..94b6f3119308a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/scheduler/SimpleProcedureScheduler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/scheduler/SimpleProcedureScheduler.java @@ -22,6 +22,7 @@ import org.apache.iotdb.confignode.procedure.Procedure; import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.locks.ReentrantLock; /** Simple scheduler for procedures */ public class SimpleProcedureScheduler extends AbstractProcedureScheduler { @@ -48,6 +49,7 @@ public void clear() { schedLock(); try { runnables.clear(); + waitings.clear(); } finally { schedUnlock(); } @@ -68,12 +70,37 @@ public int queueSize() { return runnables.size(); } - public void addWaiting(Procedure proc) { - waitings.add(proc); + public void waitProcedure(Procedure proc, ReentrantLock lock) { + boolean signal = false; + schedLock(); + try { + if (lock.isLocked()) { + waitings.add(proc); + } else { + runnables.addFirst(proc); + signal = true; + } + } finally { + schedUnlock(); + } + if (signal) { + signalAll(); + } } - public void releaseWaiting() { - runnables.addAll(waitings); - waitings.clear(); + public void releaseWaiting(ReentrantLock lock) { + boolean signal; + schedLock(); + try { + lock.unlock(); + signal = !waitings.isEmpty(); + runnables.addAll(waitings); + waitings.clear(); + } finally { + schedUnlock(); + } + if (signal) { + signalAll(); + } } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinatorLockTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinatorLockTest.java new file mode 100644 index 0000000000000..74d5d821d76c3 --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/coordinator/task/PipeTaskCoordinatorLockTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.manager.pipe.coordinator.task; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +public class PipeTaskCoordinatorLockTest { + + @Test + public void testInterruptedThreadDoesNotAcquireWithoutPermit() throws Exception { + PipeTaskCoordinatorLock lock = new PipeTaskCoordinatorLock(); + lock.lock(); + + CountDownLatch waiting = new CountDownLatch(1); + AtomicBoolean acquired = new AtomicBoolean(false); + Thread thread = + new Thread( + () -> { + Thread.currentThread().interrupt(); + waiting.countDown(); + lock.lock(); + acquired.set(true); + lock.unlock(); + }); + thread.start(); + + Assert.assertTrue(waiting.await(3, TimeUnit.SECONDS)); + TimeUnit.MILLISECONDS.sleep(200); + Assert.assertFalse(acquired.get()); + + lock.unlock(); + thread.join(TimeUnit.SECONDS.toMillis(3)); + + Assert.assertFalse(thread.isAlive()); + Assert.assertTrue(acquired.get()); + Assert.assertFalse(lock.isLocked()); + } +} diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/TestLockRegime.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/TestLockRegime.java index 500a51e9e3d07..967611ca8eef1 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/TestLockRegime.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/TestLockRegime.java @@ -19,7 +19,10 @@ package org.apache.iotdb.confignode.procedure; +import org.apache.iotdb.confignode.procedure.entity.NoopProcedure; import org.apache.iotdb.confignode.procedure.entity.SimpleLockProcedure; +import org.apache.iotdb.confignode.procedure.scheduler.LockQueue; +import org.apache.iotdb.confignode.procedure.scheduler.SimpleProcedureScheduler; import org.apache.iotdb.confignode.procedure.util.ProcedureTestUtil; import org.junit.Assert; @@ -43,4 +46,24 @@ public void testAcquireLock() { this.procExecutor, procIdList.stream().mapToLong(Long::longValue).toArray()); Assert.assertEquals(env.lockAcquireSeq.toString(), env.executeSeq.toString()); } + + @Test + public void testLockQueueDoesNotWakeDuplicateProcedure() { + LockQueue lockQueue = new LockQueue(); + SimpleProcedureScheduler scheduler = new SimpleProcedureScheduler(); + scheduler.start(); + + NoopProcedure lockOwner = new NoopProcedure(); + lockOwner.setProcId(0); + Assert.assertTrue(lockQueue.tryLock(lockOwner)); + + NoopProcedure procedure = new NoopProcedure(); + procedure.setProcId(1); + lockQueue.waitProcedure(procedure, scheduler); + lockQueue.waitProcedure(procedure, scheduler); + + Assert.assertEquals(1, lockQueue.wakeWaitingProcedures(scheduler)); + Assert.assertEquals(1, scheduler.size()); + scheduler.stop(); + } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/TestProcedureExecutor.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/TestProcedureExecutor.java index dce7f2ba5dc48..ba5f635507a4a 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/TestProcedureExecutor.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/TestProcedureExecutor.java @@ -23,11 +23,14 @@ import org.apache.iotdb.confignode.procedure.entity.NoopProcedure; import org.apache.iotdb.confignode.procedure.entity.StuckProcedure; import org.apache.iotdb.confignode.procedure.env.TestProcEnv; +import org.apache.iotdb.confignode.procedure.exception.ProcedureException; +import org.apache.iotdb.confignode.procedure.state.ProcedureState; import org.apache.iotdb.confignode.procedure.util.ProcedureTestUtil; import org.junit.Assert; import org.junit.Test; +import java.io.IOException; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -96,6 +99,37 @@ public void testWorkerThreadStuck() throws InterruptedException { ProcedureTestUtil.waitForProcedure(procExecutor, busyProcId2); } + @Test + public void testDuplicatedSchedulingDoesNotExecuteConcurrently() throws InterruptedException { + BlockingProcedure blockingProcedure = new BlockingProcedure(); + long procId = procExecutor.submitProcedure(blockingProcedure); + + Assert.assertTrue(blockingProcedure.awaitExecution(30, TimeUnit.SECONDS)); + + procExecutor.getScheduler().addFront(blockingProcedure); + boolean duplicated = blockingProcedure.awaitExecution(3, TimeUnit.SECONDS); + + blockingProcedure.releaseExecutions(duplicated ? 2 : 1); + ProcedureTestUtil.waitForProcedure(procExecutor, procId); + + Assert.assertFalse(duplicated); + Assert.assertEquals(1, blockingProcedure.getExecutionCount()); + } + + @Test + public void testInternalProcedureCanBeDeduplicatedAndRemoved() throws InterruptedException { + CompletingInternalProcedure internalProcedure = new CompletingInternalProcedure(); + + procExecutor.addInternalProcedure(internalProcedure); + procExecutor.addInternalProcedure(internalProcedure); + + Assert.assertTrue(internalProcedure.awaitExecution(30, TimeUnit.SECONDS)); + Assert.assertFalse(internalProcedure.awaitExecution(300, TimeUnit.MILLISECONDS)); + Assert.assertEquals(1, internalProcedure.getExecutionCount()); + + Assert.assertTrue(procExecutor.removeInternalProcedure(internalProcedure)); + } + private int waitThreadCount(final int expectedThreads) { long startTime = System.currentTimeMillis(); while (procExecutor.isRunning() @@ -107,4 +141,62 @@ private int waitThreadCount(final int expectedThreads) { } return procExecutor.getWorkerThreadCount(); } + + private static class BlockingProcedure extends Procedure { + + private final Semaphore entered = new Semaphore(0); + private final Semaphore finish = new Semaphore(0); + private final AtomicInteger executionCount = new AtomicInteger(); + + @Override + protected Procedure[] execute(TestProcEnv env) throws InterruptedException { + executionCount.incrementAndGet(); + entered.release(); + finish.acquire(); + return null; + } + + @Override + protected void rollback(TestProcEnv env) + throws IOException, InterruptedException, ProcedureException { + // No state to roll back. + } + + private boolean awaitExecution(long timeout, TimeUnit unit) throws InterruptedException { + return entered.tryAcquire(timeout, unit); + } + + private void releaseExecutions(int permits) { + finish.release(permits); + } + + private int getExecutionCount() { + return executionCount.get(); + } + } + + private static class CompletingInternalProcedure extends InternalProcedure { + + private final Semaphore entered = new Semaphore(0); + private final AtomicInteger executionCount = new AtomicInteger(); + + private CompletingInternalProcedure() { + super(0); + } + + @Override + protected void periodicExecute(TestProcEnv env) { + executionCount.incrementAndGet(); + entered.release(); + setState(ProcedureState.SUCCESS); + } + + private boolean awaitExecution(long timeout, TimeUnit unit) throws InterruptedException { + return entered.tryAcquire(timeout, unit); + } + + private int getExecutionCount() { + return executionCount.get(); + } + } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/entity/SimpleLockProcedure.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/entity/SimpleLockProcedure.java index ce9fea39d5589..42badd700799e 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/entity/SimpleLockProcedure.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/entity/SimpleLockProcedure.java @@ -53,18 +53,21 @@ protected ProcedureLockState acquireLock(TestProcEnv testProcEnv) { return ProcedureLockState.LOCK_ACQUIRED; } - SimpleProcedureScheduler scheduler = (SimpleProcedureScheduler) testProcEnv.getScheduler(); - scheduler.addWaiting(this); System.out.println(procName + " wait for lock."); return ProcedureLockState.LOCK_EVENT_WAIT; } + @Override + protected void waitForLock(TestProcEnv testProcEnv) { + SimpleProcedureScheduler scheduler = (SimpleProcedureScheduler) testProcEnv.getScheduler(); + scheduler.waitProcedure(this, testProcEnv.getEnvLock()); + } + @Override protected void releaseLock(TestProcEnv testProcEnv) { System.out.println(procName + " release lock."); - testProcEnv.getEnvLock().unlock(); SimpleProcedureScheduler scheduler = (SimpleProcedureScheduler) testProcEnv.getScheduler(); - scheduler.releaseWaiting(); + scheduler.releaseWaiting(testProcEnv.getEnvLock()); } @Override diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedureTest.java index 75c0963a27fab..b2ec615fbcd5b 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/runtime/PipeHandleLeaderChangeProcedureTest.java @@ -106,4 +106,51 @@ public void deserializeOldFormatConfigRegionTest() { fail(); } } + + @Test + public void completedProcedureEqualsTest() { + Map> leaderMap = new HashMap<>(); + leaderMap.put(new TConsensusGroupId(TConsensusGroupType.DataRegion, 1), new Pair<>(1, 2)); + + try { + PipeHandleLeaderChangeProcedure proc = deserializeCompletedProcedure(leaderMap); + PipeHandleLeaderChangeProcedure proc2 = deserializeCompletedProcedure(leaderMap); + + assertEquals(proc, proc2); + assertEquals(proc.hashCode(), proc2.hashCode()); + } catch (Exception e) { + fail(); + } + } + + private PipeHandleLeaderChangeProcedure deserializeCompletedProcedure( + Map> leaderMap) throws Exception { + PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); + + outputStream.writeShort(ProcedureType.PIPE_HANDLE_LEADER_CHANGE_PROCEDURE.getTypeCode()); + outputStream.writeLong(Procedure.NO_PROC_ID); + outputStream.writeInt(ProcedureState.SUCCESS.ordinal()); + outputStream.writeLong(0L); + outputStream.writeLong(0L); + outputStream.writeLong(Procedure.NO_PROC_ID); + outputStream.writeLong(Procedure.NO_TIMEOUT); + outputStream.writeInt(-1); + outputStream.write((byte) 0); + outputStream.writeInt(-1); + outputStream.write((byte) 0); + outputStream.writeInt(1); + outputStream.writeInt(Integer.MIN_VALUE); + outputStream.write((byte) 0); + outputStream.writeInt(leaderMap.size()); + for (Map.Entry> entry : leaderMap.entrySet()) { + outputStream.writeInt(entry.getKey().getId()); + outputStream.writeInt(entry.getValue().getLeft()); + outputStream.writeInt(entry.getValue().getRight()); + } + + ByteBuffer buffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + return (PipeHandleLeaderChangeProcedure) ProcedureFactory.getInstance().create(buffer); + } }