diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java index 4055398ddb7ec..790fd637d616a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java @@ -37,6 +37,11 @@ public enum CnToDnSyncRequestType { DELETE_OLD_REGION_PEER, RESET_PEER_LIST, + // Data Partition Table Maintenance + COLLECT_EARLIEST_TIMESLOTS, + GENERATE_DATA_PARTITION_TABLE, + GENERATE_DATA_PARTITION_TABLE_HEART_BEAT, + // PartitionCache INVALIDATE_PARTITION_CACHE, INVALIDATE_PERMISSION_CACHE, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java index d63d5a74f6095..9f5729ef06dfd 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java @@ -32,6 +32,7 @@ import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq; import org.apache.iotdb.mpp.rpc.thrift.TCreatePeerReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq; +import org.apache.iotdb.mpp.rpc.thrift.TGenerateDataPartitionTableReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TKillQueryInstanceReq; @@ -139,6 +140,15 @@ private void buildActionMap() { actionMapBuilder.put( CnToDnSyncRequestType.SHOW_APPLIED_CONFIGURATIONS, (req, client) -> client.showAppliedConfigurations()); + actionMapBuilder.put( + CnToDnSyncRequestType.COLLECT_EARLIEST_TIMESLOTS, + (req, client) -> client.getEarliestTimeslots()); + actionMapBuilder.put( + CnToDnSyncRequestType.GENERATE_DATA_PARTITION_TABLE, + (req, client) -> client.generateDataPartitionTable((TGenerateDataPartitionTableReq) req)); + actionMapBuilder.put( + CnToDnSyncRequestType.GENERATE_DATA_PARTITION_TABLE_HEART_BEAT, + (req, client) -> client.generateDataPartitionTableHeartbeat()); actionMap = actionMapBuilder.build(); } 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 3abb322d08472..c682107698a3a 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 @@ -319,6 +319,8 @@ public class ConfigNodeConfig { private long forceWalPeriodForConfigNodeSimpleInMs = 100; + private long partitionTableRecoverWaitAllDnUpTimeout = 60000; + public ConfigNodeConfig() { // empty constructor } @@ -1286,4 +1288,13 @@ public long getFailureDetectorPhiAcceptablePauseInMs() { public void setFailureDetectorPhiAcceptablePauseInMs(long failureDetectorPhiAcceptablePauseInMs) { this.failureDetectorPhiAcceptablePauseInMs = failureDetectorPhiAcceptablePauseInMs; } + + public long getPartitionTableRecoverWaitAllDnUpTimeout() { + return partitionTableRecoverWaitAllDnUpTimeout; + } + + public void setPartitionTableRecoverWaitAllDnUpTimeout( + long partitionTableRecoverWaitAllDnUpTimeout) { + this.partitionTableRecoverWaitAllDnUpTimeout = partitionTableRecoverWaitAllDnUpTimeout; + } } 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 77790dae1a903..e7d39fd3bcb87 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 @@ -322,6 +322,12 @@ private void loadProperties(TrimProperties properties) throws BadNodeUrlExceptio "failure_detector_phi_acceptable_pause_in_ms", String.valueOf(conf.getFailureDetectorPhiAcceptablePauseInMs())))); + conf.setPartitionTableRecoverWaitAllDnUpTimeout( + Long.parseLong( + properties.getProperty( + "partition_table_recover_wait_all_dn_up_timeout", + String.valueOf(conf.getPartitionTableRecoverWaitAllDnUpTimeout())))); + String leaderDistributionPolicy = properties.getProperty("leader_distribution_policy", conf.getLeaderDistributionPolicy()); if (AbstractLeaderBalancer.GREEDY_POLICY.equals(leaderDistributionPolicy) 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 646aaf66daf4f..1a69044d37d3d 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 @@ -67,6 +67,7 @@ import org.apache.iotdb.confignode.procedure.impl.node.RemoveAINodeProcedure; import org.apache.iotdb.confignode.procedure.impl.node.RemoveConfigNodeProcedure; import org.apache.iotdb.confignode.procedure.impl.node.RemoveDataNodesProcedure; +import org.apache.iotdb.confignode.procedure.impl.partition.DataPartitionTableIntegrityCheckProcedure; import org.apache.iotdb.confignode.procedure.impl.pipe.plugin.CreatePipePluginProcedure; import org.apache.iotdb.confignode.procedure.impl.pipe.plugin.DropPipePluginProcedure; import org.apache.iotdb.confignode.procedure.impl.pipe.runtime.PipeHandleLeaderChangeProcedure; @@ -1374,6 +1375,16 @@ public TSStatus createRegionGroups( } } + /** Used to repair the lost data partition table */ + public TSStatus dataPartitionTableIntegrityCheck() { + DataPartitionTableIntegrityCheckProcedure procedure; + synchronized (this) { + procedure = new DataPartitionTableIntegrityCheckProcedure(); + executor.submitProcedure(procedure); + } + return waitingProcedureFinished(procedure); + } + /** * Generate {@link CreateTriggerProcedure} and wait until it finished. * 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 e3d775259d626..fdf8ef89f65d7 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 @@ -352,6 +352,9 @@ public DataSet registerDataNode(TDataNodeRegisterReq req) { // Adjust the maximum RegionGroup number of each Database getClusterSchemaManager().adjustMaxRegionGroupNum(); + // Check if all DataNodes are registered and trigger integrity check if needed + checkAndTriggerIntegrityCheck(); + resp.setStatus(ClusterNodeStartUtils.ACCEPT_NODE_REGISTRATION); resp.setDataNodeId( registerDataNodePlan.getDataNodeConfiguration().getLocation().getDataNodeId()); @@ -1346,4 +1349,56 @@ private TTLManager getTTLManager() { private ExternalServiceManager getServiceManager() { return configManager.getExternalServiceManager(); } + + /** + * Check if all DataNodes are registered and running, then trigger integrity check. This method + * should be called after each DataNode registration. + */ + private void checkAndTriggerIntegrityCheck() { + // Only trigger integrity check if this ConfigNode is the leader + if (!configManager.getConsensusManager().isLeader()) { + return; + } + + // Get all registered DataNodes + List registeredDataNodes = getRegisteredDataNodes(); + + // Check if all registered DataNodes are running + boolean allDataNodesRunning = + registeredDataNodes.stream() + .allMatch( + dataNode -> { + Integer dataNodeId = dataNode.getLocation().getDataNodeId(); + NodeStatus status = getLoadManager().getLoadCache().getNodeStatus(dataNodeId); + return status == NodeStatus.Running; + }); + + if (allDataNodesRunning && !registeredDataNodes.isEmpty()) { + LOGGER.info( + "All {} DataNodes are registered and running, triggering data partition table integrity check", + registeredDataNodes.size()); + + // Trigger integrity check asynchronously + try { + configManager.getProcedureManager().dataPartitionTableIntegrityCheck(); + LOGGER.info("Data partition table integrity check procedure submitted successfully"); + } catch (Exception e) { + LOGGER.error("Failed to submit data partition table integrity check procedure", e); + } + } else { + LOGGER.debug( + "Not all DataNodes are ready yet. Registered: {}, Running: {}", + registeredDataNodes.size(), + (int) + registeredDataNodes.stream() + .filter( + dataNode -> { + Integer dataNodeId = dataNode.getLocation().getDataNodeId(); + NodeStatus status = + getLoadManager().getLoadCache().getNodeStatus(dataNodeId); + return status == NodeStatus.Running; + }) + .count()); + } + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/ConfigNodeProcedureEnv.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/ConfigNodeProcedureEnv.java new file mode 100644 index 0000000000000..c1ebd7ffccde1 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/ConfigNodeProcedureEnv.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.impl.partition; + +import org.apache.iotdb.confignode.manager.ConfigManager; + +/** + * Environment object for ConfigNode procedures. Provides access to ConfigManager and other + * necessary components. + */ +public class ConfigNodeProcedureEnv { + + private final ConfigManager configManager; + + public ConfigNodeProcedureEnv(ConfigManager configManager) { + this.configManager = configManager; + } + + public ConfigManager getConfigManager() { + return configManager; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/DataPartitionTableIntegrityCheckProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/DataPartitionTableIntegrityCheckProcedure.java new file mode 100644 index 0000000000000..a417e78ec6afd --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/DataPartitionTableIntegrityCheckProcedure.java @@ -0,0 +1,878 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT 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.partition; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot; +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil; +import org.apache.iotdb.commons.enums.DataPartitionTableGeneratorState; +import org.apache.iotdb.commons.partition.DataPartitionTable; +import org.apache.iotdb.commons.partition.SeriesPartitionTable; +import org.apache.iotdb.commons.utils.TimePartitionUtils; +import org.apache.iotdb.confignode.client.sync.CnToDnSyncRequestType; +import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool; +import org.apache.iotdb.confignode.consensus.request.read.partition.GetDataPartitionPlan; +import org.apache.iotdb.confignode.consensus.request.write.partition.CreateDataPartitionPlan; +import org.apache.iotdb.confignode.manager.node.NodeManager; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; +import org.apache.iotdb.confignode.procedure.exception.ProcedureException; +import org.apache.iotdb.confignode.procedure.impl.StateMachineProcedure; +import org.apache.iotdb.confignode.procedure.state.DataPartitionTableIntegrityCheckProcedureState; +import org.apache.iotdb.confignode.rpc.thrift.TTimeSlotList; +import org.apache.iotdb.mpp.rpc.thrift.TGenerateDataPartitionTableHeartbeatResp; +import org.apache.iotdb.mpp.rpc.thrift.TGenerateDataPartitionTableReq; +import org.apache.iotdb.mpp.rpc.thrift.TGenerateDataPartitionTableResp; +import org.apache.iotdb.mpp.rpc.thrift.TGetEarliestTimeslotsResp; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.transport.TIOStreamTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +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.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * Procedure for checking and restoring data partition table integrity. This procedure scans all + * DataNodes to detect missing data partitions and restores the DataPartitionTable on the ConfigNode + * Leader. + */ +public class DataPartitionTableIntegrityCheckProcedure + extends StateMachineProcedure< + ConfigNodeProcedureEnv, DataPartitionTableIntegrityCheckProcedureState> { + + private static final Logger LOG = + LoggerFactory.getLogger(DataPartitionTableIntegrityCheckProcedure.class); + + private static final int MAX_RETRY_COUNT = 3; + private static final long HEART_BEAT_REQUEST_RATE = 60000; + + NodeManager dataNodeManager; + private List allDataNodes = new ArrayList<>(); + + // ============Need serialize BEGIN=============/ + /** Collected earliest timeslots from DataNodes: database -> earliest timeslot */ + private Map earliestTimeslots = new ConcurrentHashMap<>(); + + /** DataPartitionTables collected from DataNodes: dataNodeId -> DataPartitionTable */ + private Map dataPartitionTables = new ConcurrentHashMap<>(); + + private Set lostDataPartitionsOfDatabases = new HashSet<>(); + + /** Final merged DataPartitionTable */ + private DataPartitionTable finalDataPartitionTable; + + private static Set skipDataNodes = new HashSet<>(); + private static Set failedDataNodes = new HashSet<>(); + + private static ScheduledExecutorService heartBeatExecutor; + + // ============Need serialize END=============/ + + public DataPartitionTableIntegrityCheckProcedure() { + super(); + } + + @Override + protected Flow executeFromState( + final ConfigNodeProcedureEnv env, final DataPartitionTableIntegrityCheckProcedureState state) + throws InterruptedException { + try { + // Ensure to get the real-time DataNodes in the current cluster at every step + dataNodeManager = env.getConfigManager().getNodeManager(); + allDataNodes = dataNodeManager.getRegisteredDataNodes(); + + switch (state) { + case COLLECT_EARLIEST_TIMESLOTS: + failedDataNodes = new HashSet<>(); + return collectEarliestTimeslots(); + case ANALYZE_MISSING_PARTITIONS: + lostDataPartitionsOfDatabases = new HashSet<>(); + return analyzeMissingPartitions(env); + case REQUEST_PARTITION_TABLES: + heartBeatExecutor = Executors.newScheduledThreadPool(1); + return requestPartitionTables(env); + case MERGE_PARTITION_TABLES: + return mergePartitionTables(env); + case WRITE_PARTITION_TABLE_TO_RAFT: + return writePartitionTableToRaft(env); + default: + throw new ProcedureException("Unknown state: " + state); + } + } catch (Exception e) { + LOG.error("Error executing state {}: {}", state, e.getMessage(), e); + setFailure("DataPartitionTableIntegrityCheckProcedure", e); + return Flow.NO_MORE_STATE; + } + } + + @Override + protected void rollbackState( + final ConfigNodeProcedureEnv env, final DataPartitionTableIntegrityCheckProcedureState state) + throws IOException, InterruptedException, ProcedureException { + switch (state) { + case COLLECT_EARLIEST_TIMESLOTS: + case ANALYZE_MISSING_PARTITIONS: + case REQUEST_PARTITION_TABLES: + case MERGE_PARTITION_TABLES: + case WRITE_PARTITION_TABLE_TO_RAFT: + // Cleanup resources + earliestTimeslots.clear(); + dataPartitionTables.clear(); + allDataNodes.clear(); + finalDataPartitionTable = null; + break; + default: + throw new ProcedureException("Unknown state for rollback: " + state); + } + } + + @Override + protected DataPartitionTableIntegrityCheckProcedureState getState(final int stateId) { + return DataPartitionTableIntegrityCheckProcedureState.values()[stateId]; + } + + @Override + protected int getStateId(final DataPartitionTableIntegrityCheckProcedureState state) { + return state.ordinal(); + } + + @Override + protected DataPartitionTableIntegrityCheckProcedureState getInitialState() { + skipDataNodes = new HashSet<>(); + failedDataNodes = new HashSet<>(); + return DataPartitionTableIntegrityCheckProcedureState.COLLECT_EARLIEST_TIMESLOTS; + } + + /** + * Collect earliest timeslot information from all DataNodes. Each DataNode returns a Map where key is database name and value is the earliest timeslot id. + */ + private Flow collectEarliestTimeslots() { + if (LOG.isDebugEnabled()) { + LOG.debug("Collecting earliest timeslots from all DataNodes..."); + } + + if (allDataNodes.isEmpty()) { + LOG.error( + "No DataNodes registered, no way to collect earliest timeslots, terminating procedure"); + setNextState(DataPartitionTableIntegrityCheckProcedureState.COLLECT_EARLIEST_TIMESLOTS); + return Flow.HAS_MORE_STATE; + } + + // Collect earliest timeslots from all DataNodes + allDataNodes.removeAll(skipDataNodes); + for (TDataNodeConfiguration dataNode : allDataNodes) { + try { + TGetEarliestTimeslotsResp resp = + (TGetEarliestTimeslotsResp) + SyncDataNodeClientPool.getInstance() + .sendSyncRequestToDataNodeWithGivenRetry( + dataNode.getLocation().getInternalEndPoint(), + null, + CnToDnSyncRequestType.COLLECT_EARLIEST_TIMESLOTS, + MAX_RETRY_COUNT); + if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + failedDataNodes.add(dataNode); + LOG.error( + "Failed to collected earliest timeslots from the DataNode[id={}], response status is {}", + dataNode.getLocation().getDataNodeId(), + resp.getStatus()); + continue; + } + + Map nodeTimeslots = resp.getDatabaseToEarliestTimeslot(); + + // Merge with existing timeslots (take minimum) + for (Map.Entry entry : nodeTimeslots.entrySet()) { + earliestTimeslots.merge(entry.getKey(), entry.getValue(), Math::min); + } + + if (LOG.isDebugEnabled()) { + LOG.debug( + "Collected earliest timeslots from the DataNode[id={}]: {}", + dataNode.getLocation().getDataNodeId(), + nodeTimeslots); + } + } catch (Exception e) { + LOG.error( + "Failed to collect earliest timeslots from the DataNode[id={}]: {}", + dataNode.getLocation().getDataNodeId(), + e.getMessage(), + e); + failedDataNodes.add(dataNode); + } + } + + if (LOG.isDebugEnabled()) { + LOG.info( + "Collected earliest timeslots from {} DataNodes: {}, the number of successful DataNodes is {}", + allDataNodes.size(), + earliestTimeslots, + allDataNodes.size() - failedDataNodes.size()); + } + + if (failedDataNodes.size() == allDataNodes.size() + && new HashSet<>(allDataNodes).containsAll(failedDataNodes)) { + setNextState(DataPartitionTableIntegrityCheckProcedureState.COLLECT_EARLIEST_TIMESLOTS); + } else { + setNextState(DataPartitionTableIntegrityCheckProcedureState.ANALYZE_MISSING_PARTITIONS); + } + return Flow.HAS_MORE_STATE; + } + + /** + * Analyze which data partitions are missing based on earliest timeslots. Identify data partitions + * of databases need to be repaired. + */ + private Flow analyzeMissingPartitions(final ConfigNodeProcedureEnv env) { + if (LOG.isDebugEnabled()) { + LOG.debug("Analyzing missing data partitions..."); + } + + if (earliestTimeslots.isEmpty()) { + LOG.error( + "No missing data partitions detected, nothing needs to be repaired, terminating procedure"); + setNextState(DataPartitionTableIntegrityCheckProcedureState.COLLECT_EARLIEST_TIMESLOTS); + return Flow.HAS_MORE_STATE; + } + + // Find all databases that have lost data partition tables + for (Map.Entry entry : earliestTimeslots.entrySet()) { + String database = entry.getKey(); + long earliestTimeslot = entry.getValue(); + + // Get current DataPartitionTable from ConfigManager + Map>>> + localDataPartitionTable = getLocalDataPartitionTable(env, database); + + // Check if ConfigNode has a data partition that is associated with the earliestTimeslot + if (localDataPartitionTable == null + || localDataPartitionTable.isEmpty() + || localDataPartitionTable.get(database) == null + || localDataPartitionTable.get(database).isEmpty()) { + lostDataPartitionsOfDatabases.add(database); + LOG.warn( + "No data partition table related to database {} was found from the ConfigNode, and this issue needs to be repaired", + database); + continue; + } + + Map>> + seriesPartitionMap = localDataPartitionTable.get(database); + for (Map.Entry>> + seriesPartitionEntry : seriesPartitionMap.entrySet()) { + Map> tTimePartitionSlotListMap = + seriesPartitionEntry.getValue(); + tTimePartitionSlotListMap + .keySet() + .forEach( + slot -> { + if (!TimePartitionUtils.satisfyPartitionId( + slot.getStartTime(), earliestTimeslot)) { + lostDataPartitionsOfDatabases.add(database); + LOG.warn( + "Database {} has lost timeslot {} in its data table partition, and this issue needs to be repaired", + database, + earliestTimeslot); + } + }); + } + } + + if (lostDataPartitionsOfDatabases.isEmpty()) { + LOG.info("No databases have lost data partitions, terminating procedure"); + return Flow.NO_MORE_STATE; + } + + LOG.info( + "Identified {} databases have lost data partitions, will request DataPartitionTable generation from {} DataNodes", + lostDataPartitionsOfDatabases.size(), + allDataNodes.size() - failedDataNodes.size()); + setNextState(DataPartitionTableIntegrityCheckProcedureState.REQUEST_PARTITION_TABLES); + return Flow.HAS_MORE_STATE; + } + + private Map>>> + getLocalDataPartitionTable(final ConfigNodeProcedureEnv env, final String database) { + Map> schemaPartitionTable = + env.getConfigManager() + .getSchemaPartition(Collections.singletonMap(database, Collections.emptyList())) + .getSchemaPartitionTable(); + + // Construct request for getting data partition + final Map> partitionSlotsMap = new HashMap<>(); + schemaPartitionTable.forEach( + (key, value) -> { + Map slotListMap = new HashMap<>(); + value + .keySet() + .forEach( + slot -> + slotListMap.put( + slot, new TTimeSlotList(Collections.emptyList(), true, true))); + partitionSlotsMap.put(key, slotListMap); + }); + final GetDataPartitionPlan getDataPartitionPlan = new GetDataPartitionPlan(partitionSlotsMap); + return env.getConfigManager().getDataPartition(getDataPartitionPlan).getDataPartitionTable(); + } + + /** + * Request DataPartitionTable generation from target DataNodes. Each DataNode scans its tsfile + * resources and generates a DataPartitionTable. + */ + private Flow requestPartitionTables(final ConfigNodeProcedureEnv env) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "Requesting DataPartitionTable generation from {} DataNodes...", allDataNodes.size()); + } + + if (allDataNodes.isEmpty()) { + LOG.error( + "No DataNodes registered, no way to requested DataPartitionTable generation, terminating procedure"); + setNextState(DataPartitionTableIntegrityCheckProcedureState.COLLECT_EARLIEST_TIMESLOTS); + return Flow.HAS_MORE_STATE; + } + + ScheduledExecutorUtil.safelyScheduleAtFixedRate( + heartBeatExecutor, + this::checkPartitionTableGenerationStatus, + 0, + HEART_BEAT_REQUEST_RATE, + TimeUnit.MILLISECONDS); + + allDataNodes.removeAll(skipDataNodes); + allDataNodes.removeAll(failedDataNodes); + for (TDataNodeConfiguration dataNode : allDataNodes) { + int dataNodeId = dataNode.getLocation().getDataNodeId(); + if (!dataPartitionTables.containsKey(dataNodeId)) { + try { + TGenerateDataPartitionTableReq req = new TGenerateDataPartitionTableReq(); + req.setDatabases(lostDataPartitionsOfDatabases); + TGenerateDataPartitionTableResp resp = + (TGenerateDataPartitionTableResp) + SyncDataNodeClientPool.getInstance() + .sendSyncRequestToDataNodeWithGivenRetry( + dataNode.getLocation().getInternalEndPoint(), + req, + CnToDnSyncRequestType.GENERATE_DATA_PARTITION_TABLE, + MAX_RETRY_COUNT); + if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + failedDataNodes.add(dataNode); + LOG.error( + "Failed to request DataPartitionTable generation from the DataNode[id={}], response status is {}", + dataNode.getLocation().getDataNodeId(), + resp.getStatus()); + continue; + } + + byte[] bytes = resp.getDataPartitionTable(); + DataPartitionTable dataPartitionTable = new DataPartitionTable(); + dataPartitionTable.deserialize(ByteBuffer.wrap(bytes)); + dataPartitionTables.put(dataNodeId, dataPartitionTable); + } catch (Exception e) { + failedDataNodes.add(dataNode); + LOG.error( + "Failed to request DataPartitionTable generation from DataNode[id={}]: {}", + dataNodeId, + e.getMessage(), + e); + } + } + } + + if (failedDataNodes.size() == allDataNodes.size() + && new HashSet<>(allDataNodes).containsAll(failedDataNodes)) { + setNextState(DataPartitionTableIntegrityCheckProcedureState.COLLECT_EARLIEST_TIMESLOTS); + return Flow.HAS_MORE_STATE; + } + + setNextState(DataPartitionTableIntegrityCheckProcedureState.MERGE_PARTITION_TABLES); + return Flow.HAS_MORE_STATE; + } + + /** Check completion status of DataPartitionTable generation tasks. */ + private void checkPartitionTableGenerationStatus() { + if (LOG.isDebugEnabled()) { + LOG.info("Checking DataPartitionTable generation completion status..."); + } + + int completeCount = 0; + for (TDataNodeConfiguration dataNode : allDataNodes) { + int dataNodeId = dataNode.getLocation().getDataNodeId(); + + if (!dataPartitionTables.containsKey(dataNodeId)) { + try { + TGenerateDataPartitionTableHeartbeatResp resp = + (TGenerateDataPartitionTableHeartbeatResp) + SyncDataNodeClientPool.getInstance() + .sendSyncRequestToDataNodeWithGivenRetry( + dataNode.getLocation().getInternalEndPoint(), + null, + CnToDnSyncRequestType.GENERATE_DATA_PARTITION_TABLE_HEART_BEAT, + MAX_RETRY_COUNT); + DataPartitionTableGeneratorState state = + DataPartitionTableGeneratorState.getStateByCode(resp.getErrorCode()); + + if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + LOG.error( + "Failed to request DataPartitionTable generation heart beat from the DataNode[id={}], state is {}, response status is {}", + dataNode.getLocation().getDataNodeId(), + state, + resp.getStatus()); + continue; + } + switch (state) { + case SUCCESS: + LOG.info( + "DataNode {} completed DataPartitionTable generation, terminating heart beat", + dataNodeId); + completeCount++; + break; + case IN_PROGRESS: + LOG.info("DataNode {} still generating DataPartitionTable", dataNodeId); + break; + case FAILED: + LOG.error( + "DataNode {} failed to generate DataPartitionTable, terminating heart beat", + dataNodeId); + completeCount++; + break; + default: + LOG.error( + "DataNode {} returned unknown error code: {}", dataNodeId, resp.getErrorCode()); + break; + } + } catch (Exception e) { + LOG.error( + "Error checking DataPartitionTable status from DataNode {}: {}, terminating heart beat", + dataNodeId, + e.getMessage(), + e); + completeCount++; + } + } else { + completeCount++; + } + } + + if (completeCount >= allDataNodes.size()) { + heartBeatExecutor.shutdown(); + } + } + + /** Merge DataPartitionTables from all DataNodes into a final table. */ + private Flow mergePartitionTables(final ConfigNodeProcedureEnv env) { + if (LOG.isDebugEnabled()) { + LOG.info("Merging DataPartitionTables from {} DataNodes...", dataPartitionTables.size()); + } + + if (dataPartitionTables.isEmpty()) { + LOG.error("No DataPartitionTables to merge, dataPartitionTables is empty"); + setNextState(DataPartitionTableIntegrityCheckProcedureState.COLLECT_EARLIEST_TIMESLOTS); + return Flow.HAS_MORE_STATE; + } + + Map finalDataPartitionMap = new HashMap<>(); + + for (String database : lostDataPartitionsOfDatabases) { + // Get current DataPartitionTable from ConfigManager + Map>>> + localDataPartitionTableMap = getLocalDataPartitionTable(env, database); + + // Check if ConfigNode has a data partition that is associated with the earliestTimeslot + if (localDataPartitionTableMap == null + || localDataPartitionTableMap.isEmpty() + || localDataPartitionTableMap.get(database) == null + || localDataPartitionTableMap.get(database).isEmpty()) { + LOG.warn( + "No data partition table related to database {} was found from the ConfigNode, use data partition table of DataNode directly", + database); + continue; + } + + localDataPartitionTableMap + .values() + .forEach( + map -> + map.forEach( + (tSeriesPartitionSlot, seriesPartitionTableMap) -> { + if (tSeriesPartitionSlot == null + || seriesPartitionTableMap == null + || seriesPartitionTableMap.isEmpty()) { + return; + } + finalDataPartitionMap.computeIfAbsent( + tSeriesPartitionSlot, + k -> new SeriesPartitionTable(seriesPartitionTableMap)); + })); + } + + if (finalDataPartitionMap.isEmpty()) { + dataPartitionTables + .values() + .forEach( + dataPartitionTable -> { + if (dataPartitionTable == null + || dataPartitionTable.getDataPartitionMap() == null + || dataPartitionTable.getDataPartitionMap().isEmpty()) { + return; + } + dataPartitionTable + .getDataPartitionMap() + .forEach( + (dnSeriesPartitionSlot, dnSeriesPartitionTable) -> { + if (dnSeriesPartitionSlot == null || dnSeriesPartitionTable == null) { + return; + } + finalDataPartitionMap.computeIfAbsent( + dnSeriesPartitionSlot, k -> dnSeriesPartitionTable); + }); + }); + } else { + finalDataPartitionMap.forEach( + (tSeriesPartitionSlot, seriesPartitionTable) -> { + dataPartitionTables + .values() + .forEach( + dataPartitionTable -> { + if (dataPartitionTable == null + || dataPartitionTable.getDataPartitionMap() == null + || dataPartitionTable.getDataPartitionMap().isEmpty()) { + return; + } + dataPartitionTable + .getDataPartitionMap() + .forEach( + (dnSeriesPartitionSlot, dnSeriesPartitionTable) -> { + if (!tSeriesPartitionSlot.equals(dnSeriesPartitionSlot)) { + return; + } + + if (seriesPartitionTable == null + || seriesPartitionTable.getSeriesPartitionMap() == null + || seriesPartitionTable.getSeriesPartitionMap().isEmpty()) { + finalDataPartitionMap.put( + tSeriesPartitionSlot, dnSeriesPartitionTable); + } + + // dnDataPartitionTable merged to seriesPartitionTable + dnSeriesPartitionTable + .getSeriesPartitionMap() + .forEach( + (k, v) -> + v.forEach( + tConsensusGroupId -> { + if (seriesPartitionTable == null) { + return; + } + seriesPartitionTable.putDataPartition( + k, tConsensusGroupId); + })); + }); + }); + }); + } + + finalDataPartitionTable = new DataPartitionTable(finalDataPartitionMap); + + LOG.info("DataPartitionTable merge completed successfully"); + setNextState(DataPartitionTableIntegrityCheckProcedureState.WRITE_PARTITION_TABLE_TO_RAFT); + return Flow.HAS_MORE_STATE; + } + + /** Write the final DataPartitionTable to raft log. */ + private Flow writePartitionTableToRaft(final ConfigNodeProcedureEnv env) { + if (LOG.isDebugEnabled()) { + LOG.info("Writing DataPartitionTable to raft log..."); + } + + if (lostDataPartitionsOfDatabases.isEmpty()) { + LOG.error("No database lost data partition table"); + setFailure( + "DataPartitionTableIntegrityCheckProcedure", + new ProcedureException("No database lost data partition table for raft write")); + return getFlow(); + } + + if (finalDataPartitionTable == null) { + LOG.error("No DataPartitionTable to write to raft"); + setFailure( + "DataPartitionTableIntegrityCheckProcedure", + new ProcedureException("No DataPartitionTable available for raft write")); + return getFlow(); + } + + int failedCnt = 0; + while (failedCnt < MAX_RETRY_COUNT) { + try { + CreateDataPartitionPlan createPlan = new CreateDataPartitionPlan(); + Map assignedDataPartition = new HashMap<>(); + assignedDataPartition.put( + lostDataPartitionsOfDatabases.stream().findFirst().get(), finalDataPartitionTable); + createPlan.setAssignedDataPartition(assignedDataPartition); + TSStatus tsStatus = env.getConfigManager().getConsensusManager().write(createPlan); + + if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + LOG.info("DataPartitionTable successfully written to raft log"); + break; + } else { + LOG.error("Failed to write DataPartitionTable to raft log"); + setFailure( + "DataPartitionTableIntegrityCheckProcedure", + new ProcedureException("Failed to write DataPartitionTable to raft log")); + } + } catch (Exception e) { + LOG.error("Error writing DataPartitionTable to raft log", e); + setFailure("DataPartitionTableIntegrityCheckProcedure", e); + } + failedCnt++; + } + + return getFlow(); + } + + private Flow getFlow() { + if (!failedDataNodes.isEmpty()) { + allDataNodes.removeAll(failedDataNodes); + skipDataNodes = new HashSet<>(allDataNodes); + setNextState(DataPartitionTableIntegrityCheckProcedureState.COLLECT_EARLIEST_TIMESLOTS); + return Flow.HAS_MORE_STATE; + } else { + skipDataNodes.clear(); + return Flow.NO_MORE_STATE; + } + } + + @Override + public void serialize(final DataOutputStream stream) throws IOException { + super.serialize(stream); + + // Serialize earliestTimeslots + stream.writeInt(earliestTimeslots.size()); + for (Map.Entry entry : earliestTimeslots.entrySet()) { + stream.writeUTF(entry.getKey()); + stream.writeLong(entry.getValue()); + } + + // Serialize dataPartitionTables count + stream.writeInt(dataPartitionTables.size()); + for (Map.Entry entry : dataPartitionTables.entrySet()) { + stream.writeInt(entry.getKey()); + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos)) { + TTransport transport = new TIOStreamTransport(oos); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + entry.getValue().serialize(oos, protocol); + + // Write the size and data for byte array after serialize + byte[] data = baos.toByteArray(); + stream.writeInt(data.length); + stream.write(data); + } catch (IOException | TException e) { + LOG.error("{} serialize failed", this.getClass().getSimpleName(), e); + throw new IOException("Failed to serialize dataPartitionTables", e); + } + } + + stream.writeInt(lostDataPartitionsOfDatabases.size()); + for (String database : lostDataPartitionsOfDatabases) { + stream.writeUTF(database); + } + + if (finalDataPartitionTable != null) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos)) { + TTransport transport = new TIOStreamTransport(oos); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + finalDataPartitionTable.serialize(oos, protocol); + + // Write the size and data for byte array after serialize + byte[] data = baos.toByteArray(); + stream.writeInt(data.length); + stream.write(data); + } catch (IOException | TException e) { + LOG.error("{} serialize failed", this.getClass().getSimpleName(), e); + throw new IOException("Failed to serialize finalDataPartitionTable", e); + } + } else { + stream.writeInt(0); + } + + stream.writeInt(skipDataNodes.size()); + for (TDataNodeConfiguration skipDataNode : skipDataNodes) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + TTransport transport = new TIOStreamTransport(baos); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + skipDataNode.write(protocol); + + byte[] data = baos.toByteArray(); + stream.writeInt(data.length); + stream.write(data); + } catch (TException e) { + LOG.error("Failed to serialize skipDataNode", e); + throw new IOException("Failed to serialize skipDataNode", e); + } + } + + stream.writeInt(failedDataNodes.size()); + for (TDataNodeConfiguration failedDataNode : failedDataNodes) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + TTransport transport = new TIOStreamTransport(baos); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + failedDataNode.write(protocol); + + byte[] data = baos.toByteArray(); + stream.writeInt(data.length); + stream.write(data); + } catch (TException e) { + LOG.error("Failed to serialize failedDataNode", e); + throw new IOException("Failed to serialize failedDataNode", e); + } + } + } + + @Override + public void deserialize(final ByteBuffer byteBuffer) { + super.deserialize(byteBuffer); + + // Deserialize earliestTimeslots + int earliestTimeslotsSize = byteBuffer.getInt(); + earliestTimeslots = new ConcurrentHashMap<>(); + for (int i = 0; i < earliestTimeslotsSize; i++) { + String database = String.valueOf(byteBuffer.getChar()); + long timeslot = byteBuffer.getLong(); + earliestTimeslots.put(database, timeslot); + } + + // Deserialize dataPartitionTables count + int dataPartitionTablesSize = byteBuffer.getInt(); + dataPartitionTables = new HashMap<>(); + for (int i = 0; i < dataPartitionTablesSize; i++) { + int key = byteBuffer.getInt(); + int size = byteBuffer.getInt(); + byte[] bytes = new byte[size]; + byteBuffer.get(bytes); + try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes); + ObjectInputStream ois = new ObjectInputStream(bais)) { + TTransport transport = new TIOStreamTransport(ois); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + + // Deserialize by input stream and protocol + DataPartitionTable value = new DataPartitionTable(); + value.deserialize(ois, protocol); + dataPartitionTables.put(key, value); + } catch (IOException | TException e) { + LOG.error("{} deserialize failed", this.getClass().getSimpleName(), e); + throw new RuntimeException(e); + } + } + + int lostDataPartitionsOfDatabasesSize = byteBuffer.getInt(); + for (int i = 0; i < lostDataPartitionsOfDatabasesSize; i++) { + String database = ReadWriteIOUtils.readString(byteBuffer); + lostDataPartitionsOfDatabases.add(database); + } + + // Deserialize finalDataPartitionTable size + int finalDataPartitionTableSize = byteBuffer.getInt(); + if (finalDataPartitionTableSize > 0) { + byte[] finalDataPartitionTableBytes = new byte[finalDataPartitionTableSize]; + byteBuffer.get(finalDataPartitionTableBytes); + try (ByteArrayInputStream bais = new ByteArrayInputStream(finalDataPartitionTableBytes); + ObjectInputStream ois = new ObjectInputStream(bais)) { + TTransport transport = new TIOStreamTransport(ois); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + + // Deserialize by input stream and protocol + finalDataPartitionTable = new DataPartitionTable(); + finalDataPartitionTable.deserialize(ois, protocol); + } catch (IOException | TException e) { + LOG.error("{} deserialize failed", this.getClass().getSimpleName(), e); + throw new RuntimeException(e); + } + } else { + finalDataPartitionTable = null; + } + + skipDataNodes = new HashSet<>(); + int skipDataNodesSize = byteBuffer.getInt(); + for (int i = 0; i < skipDataNodesSize; i++) { + int size = byteBuffer.getInt(); + byte[] bytes = new byte[size]; + byteBuffer.get(bytes); + + try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes)) { + TTransport transport = new TIOStreamTransport(bais); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + + TDataNodeConfiguration dataNode = new TDataNodeConfiguration(); + dataNode.read(protocol); + skipDataNodes.add(dataNode); + } catch (TException | IOException e) { + LOG.error("Failed to deserialize skipDataNode", e); + throw new RuntimeException(e); + } + } + + failedDataNodes = new HashSet<>(); + int failedDataNodesSize = byteBuffer.getInt(); + for (int i = 0; i < failedDataNodesSize; i++) { + int size = byteBuffer.getInt(); + byte[] bytes = new byte[size]; + byteBuffer.get(bytes); + + try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes)) { + TTransport transport = new TIOStreamTransport(bais); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + + TDataNodeConfiguration dataNode = new TDataNodeConfiguration(); + dataNode.read(protocol); + failedDataNodes.add(dataNode); + } catch (TException | IOException e) { + LOG.error("Failed to deserialize failedDataNode", e); + throw new RuntimeException(e); + } + } + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DataPartitionTableIntegrityCheckProcedureState.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DataPartitionTableIntegrityCheckProcedureState.java new file mode 100644 index 0000000000000..2173ea8ef4589 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/DataPartitionTableIntegrityCheckProcedureState.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.state; + +public enum DataPartitionTableIntegrityCheckProcedureState { + /** Collect earliest timeslot information from all DataNodes */ + COLLECT_EARLIEST_TIMESLOTS, + /** Analyze missing data partitions */ + ANALYZE_MISSING_PARTITIONS, + /** Request DataPartitionTable generation from DataNodes */ + REQUEST_PARTITION_TABLES, + /** Merge DataPartitionTables from all DataNodes */ + MERGE_PARTITION_TABLES, + /** Write final DataPartitionTable to raft log */ + WRITE_PARTITION_TABLE_TO_RAFT +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java index dd15558608718..140fffa852ccc 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java @@ -26,6 +26,7 @@ import org.apache.iotdb.confignode.procedure.impl.node.RemoveAINodeProcedure; import org.apache.iotdb.confignode.procedure.impl.node.RemoveConfigNodeProcedure; import org.apache.iotdb.confignode.procedure.impl.node.RemoveDataNodesProcedure; +import org.apache.iotdb.confignode.procedure.impl.partition.DataPartitionTableIntegrityCheckProcedure; import org.apache.iotdb.confignode.procedure.impl.pipe.plugin.CreatePipePluginProcedure; import org.apache.iotdb.confignode.procedure.impl.pipe.plugin.DropPipePluginProcedure; import org.apache.iotdb.confignode.procedure.impl.pipe.runtime.PipeHandleLeaderChangeProcedure; @@ -404,6 +405,9 @@ public Procedure create(ByteBuffer buffer) throws IOException { case ADD_NEVER_FINISH_SUB_PROCEDURE_PROCEDURE: procedure = new AddNeverFinishSubProcedureProcedure(); break; + case DATA_PARTITION_TABLE_INTEGRITY_CHECK_PROCEDURE: + procedure = new DataPartitionTableIntegrityCheckProcedure(); + break; default: LOGGER.error("Unknown Procedure type: {}", typeCode); throw new IOException("Unknown Procedure type: " + typeCode); @@ -554,6 +558,8 @@ public static ProcedureType getProcedureType(final Procedure procedure) { return ProcedureType.NEVER_FINISH_PROCEDURE; } else if (procedure instanceof AddNeverFinishSubProcedureProcedure) { return ProcedureType.ADD_NEVER_FINISH_SUB_PROCEDURE_PROCEDURE; + } else if (procedure instanceof DataPartitionTableIntegrityCheckProcedure) { + return ProcedureType.DATA_PARTITION_TABLE_INTEGRITY_CHECK_PROCEDURE; } throw new UnsupportedOperationException( "Procedure type " + procedure.getClass() + " is not supported"); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java index 820a90f7ebfb9..839c8ace0984d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java @@ -172,7 +172,10 @@ public enum ProcedureType { @TestOnly NEVER_FINISH_PROCEDURE((short) 30000), @TestOnly - ADD_NEVER_FINISH_SUB_PROCEDURE_PROCEDURE((short) 30001); + ADD_NEVER_FINISH_SUB_PROCEDURE_PROCEDURE((short) 30001), + + /** Data Partition Table Integrity Check */ + DATA_PARTITION_TABLE_INTEGRITY_CHECK_PROCEDURE((short) 1600); private final short typeCode; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java index f20f77095d97a..01ae2499a9e80 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java @@ -24,6 +24,8 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.ServerCommandLine; import org.apache.iotdb.commons.client.ClientManagerMetrics; +import org.apache.iotdb.commons.cluster.NodeStatus; +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; import org.apache.iotdb.commons.concurrent.ThreadModule; import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.concurrent.ThreadPoolMetrics; @@ -79,6 +81,10 @@ import java.util.Arrays; import java.util.List; import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; public class ConfigNode extends ServerCommandLine implements ConfigNodeMBean { @@ -110,6 +116,13 @@ public class ConfigNode extends ServerCommandLine implements ConfigNodeMBean { private int exitStatusCode = 0; + private Future dataPartitionTableCheckFuture; + + private ExecutorService dataPartitionTableCheckExecutor = + IoTDBThreadPoolFactory.newSingleThreadExecutor("DATA_PARTITION_TABLE_CHECK"); + + private final CountDownLatch latch = new CountDownLatch(1); + public ConfigNode() { super("ConfigNode"); // We do not init anything here, so that we can re-initialize the instance in IT. @@ -147,6 +160,11 @@ protected void start() throws IoTDBException { } active(); LOGGER.info("IoTDB started"); + try { + dataPartitionTableCheckFuture.get(); + } catch (ExecutionException | InterruptedException e) { + LOGGER.error("Data partition table check task execute failed", e); + } } @Override @@ -203,6 +221,34 @@ public void active() { } loadSecretKey(); loadHardwareCode(); + + dataPartitionTableCheckFuture = + dataPartitionTableCheckExecutor.submit( + () -> { + LOGGER.info( + "Prepare to start dataPartitionTableIntegrityCheck after all datanodes are started up"); + // Thread.sleep(CONF.getPartitionTableRecoverWaitAllDnUpTimeout()); + + while (latch.getCount() > 0) { + List dnList = + configManager + .getLoadManager() + .filterDataNodeThroughStatus(NodeStatus.Running); + if (dnList != null && !dnList.isEmpty()) { + LOGGER.info("Starting dataPartitionTableIntegrityCheck..."); + TSStatus status = + configManager.getProcedureManager().dataPartitionTableIntegrityCheck(); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + LOGGER.error("Data partition table integrity check failed!"); + } + latch.countDown(); + } else { + LOGGER.info("No running datanodes found, waiting..."); + Thread.sleep(5000); // 等待5秒后重新检查 + } + } + return null; + }); return; } else { saveSecretKey(); 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 98c15a2d9bf06..2ce50415549e3 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 @@ -1219,6 +1219,10 @@ public class IoTDBConfig { private long maxObjectSizeInByte = 4 * 1024 * 1024 * 1024L; + /* Need use these parameters when repair data partition table */ + private int partitionTableRecoverWorkerNum = 10; + private int partitionTableRecoverMaxReadBytesPerSecond = 1000; + IoTDBConfig() {} public int getMaxLogEntriesNumPerBatch() { @@ -4367,4 +4371,20 @@ public long getMaxObjectSizeInByte() { public void setMaxObjectSizeInByte(long maxObjectSizeInByte) { this.maxObjectSizeInByte = maxObjectSizeInByte; } + + public int getPartitionTableRecoverWorkerNum() { + return partitionTableRecoverWorkerNum; + } + + public void setPartitionTableRecoverWorkerNum(int partitionTableRecoverWorkerNum) { + this.partitionTableRecoverWorkerNum = partitionTableRecoverWorkerNum; + } + + public int getPartitionTableRecoverMaxReadBytesPerSecond() { + return partitionTableRecoverMaxReadBytesPerSecond; + } + + public void setPartitionTableRecoverMaxReadBytesPerSecond(int partitionTableRecoverWorkerNum) { + this.partitionTableRecoverWorkerNum = partitionTableRecoverWorkerNum; + } } 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 6730138b2af5c..a5e89bb250dfb 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 @@ -1139,6 +1139,17 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException // update trusted_uri_pattern loadTrustedUriPattern(properties); + conf.setPartitionTableRecoverWorkerNum( + Integer.parseInt( + properties.getProperty( + "partition_table_recover_worker_num", + String.valueOf(conf.getPartitionTableRecoverWorkerNum())))); + conf.setPartitionTableRecoverMaxReadBytesPerSecond( + Integer.parseInt( + properties.getProperty( + "partition_table_recover_max_read_bytes_per_second", + String.valueOf(conf.getPartitionTableRecoverMaxReadBytesPerSecond())))); + conf.setIncludeNullValueInWriteThroughputMetric( Boolean.parseBoolean( properties.getProperty( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/partition/DataPartitionTableGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/partition/DataPartitionTableGenerator.java new file mode 100644 index 0000000000000..4f9a326f05223 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/partition/DataPartitionTableGenerator.java @@ -0,0 +1,498 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT 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.partition; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot; +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.commons.partition.DataPartitionTable; +import org.apache.iotdb.commons.partition.SeriesPartitionTable; +import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor; +import org.apache.iotdb.commons.utils.TimePartitionUtils; +import org.apache.iotdb.commons.utils.rateLimiter.LeakyBucketRateLimiter; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.file.metadata.IDeviceID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Generator for DataPartitionTable by scanning tsfile resources. This class scans the data + * directory structure and builds a complete DataPartitionTable based on existing tsfiles. + */ +public class DataPartitionTableGenerator { + + private static final Logger LOG = LoggerFactory.getLogger(DataPartitionTableGenerator.class); + + // Task status + private volatile TaskStatus status = TaskStatus.NOT_STARTED; + private volatile String errorMessage; + private volatile DataPartitionTable dataPartitionTable; + + // Progress tracking + private final AtomicInteger processedFiles = new AtomicInteger(0); + private final AtomicInteger failedFiles = new AtomicInteger(0); + private final AtomicLong totalFiles = new AtomicLong(0); + + // Configuration + private String[] dataDirectories; + private final ExecutorService executor; + private final Set databases; + private final int seriesSlotNum; + private final String seriesPartitionExecutorClass; + + private static final int EXECUTOR_MAX_TIMEOUT = 60; + + private static final LeakyBucketRateLimiter limiter = + new LeakyBucketRateLimiter( + (long) + IoTDBDescriptor.getInstance() + .getConfig() + .getPartitionTableRecoverMaxReadBytesPerSecond() + * 1024 + * 1024); + + public static final String SCAN_FILE_SUFFIX_NAME = ".tsfile"; + public static final Set IGNORE_DATABASE = + new HashSet() { + { + add("root.__audit"); + add("root.__system"); + } + }; + + public DataPartitionTableGenerator( + ExecutorService executor, + Set databases, + int seriesSlotNum, + String seriesPartitionExecutorClass) { + this.executor = executor; + this.databases = databases; + this.seriesSlotNum = seriesSlotNum; + this.seriesPartitionExecutorClass = seriesPartitionExecutorClass; + } + + public DataPartitionTableGenerator( + String dataDirectory, + ExecutorService executor, + Set databases, + int seriesSlotNum, + String seriesPartitionExecutorClass) { + this.dataDirectories = new String[] {dataDirectory}; + this.executor = executor; + this.databases = databases; + this.seriesSlotNum = seriesSlotNum; + this.seriesPartitionExecutorClass = seriesPartitionExecutorClass; + } + + public DataPartitionTableGenerator( + String[] dataDirectories, + ExecutorService executor, + Set databases, + int seriesSlotNum, + String seriesPartitionExecutorClass) { + this.dataDirectories = dataDirectories; + this.executor = executor; + this.databases = databases; + this.seriesSlotNum = seriesSlotNum; + this.seriesPartitionExecutorClass = seriesPartitionExecutorClass; + } + + public enum TaskStatus { + NOT_STARTED, + IN_PROGRESS, + COMPLETED, + FAILED + } + + /** Start generating DataPartitionTable asynchronously. */ + public CompletableFuture startGeneration() { + if (status != TaskStatus.NOT_STARTED) { + throw new IllegalStateException("Task is already started or completed"); + } + + status = TaskStatus.IN_PROGRESS; + return CompletableFuture.runAsync(this::generateDataPartitionTableByMemory); + } + + private void generateDataPartitionTableByMemory() { + Map dataPartitionMap = new ConcurrentHashMap<>(); + List> futures = new ArrayList<>(); + + SeriesPartitionExecutor seriesPartitionExecutor = + SeriesPartitionExecutor.getSeriesPartitionExecutor( + seriesPartitionExecutorClass, seriesSlotNum); + + for (DataRegion dataRegion : StorageEngine.getInstance().getAllDataRegions()) { + CompletableFuture regionFuture = + CompletableFuture.runAsync( + () -> { + TsFileManager tsFileManager = dataRegion.getTsFileManager(); + String databaseName = dataRegion.getDatabaseName(); + if (!databases.contains(databaseName) || IGNORE_DATABASE.contains(databaseName)) { + return; + } + + tsFileManager.readLock(); + List seqTsFileList = tsFileManager.getTsFileList(true); + List unseqTsFileList = tsFileManager.getTsFileList(false); + tsFileManager.readUnlock(); + + constructDataPartitionMap(seqTsFileList, seriesPartitionExecutor, dataPartitionMap); + constructDataPartitionMap( + unseqTsFileList, seriesPartitionExecutor, dataPartitionMap); + }, + executor); + futures.add(regionFuture); + } + + // Wait for all tasks to complete + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join(); + + if (dataPartitionMap.isEmpty()) { + LOG.error("Failed to generate DataPartitionTable, dataPartitionMap is empty"); + status = TaskStatus.FAILED; + return; + } + + dataPartitionTable = new DataPartitionTable(dataPartitionMap); + status = TaskStatus.COMPLETED; + } + + private static void constructDataPartitionMap( + List seqTsFileList, + SeriesPartitionExecutor seriesPartitionExecutor, + Map dataPartitionMap) { + for (TsFileResource tsFileResource : seqTsFileList) { + Set devices = tsFileResource.getDevices(limiter); + long timeSlotId = tsFileResource.getTsFileID().timePartitionId; + int regionId = tsFileResource.getTsFileID().regionId; + + TConsensusGroupId consensusGroupId = new TConsensusGroupId(); + consensusGroupId.setId(regionId); + consensusGroupId.setType(TConsensusGroupType.DataRegion); + + for (IDeviceID deviceId : devices) { + TSeriesPartitionSlot seriesSlotId = + seriesPartitionExecutor.getSeriesPartitionSlot(deviceId); + TTimePartitionSlot timePartitionSlot = + new TTimePartitionSlot(TimePartitionUtils.getTimeByPartitionId(timeSlotId)); + dataPartitionMap + .computeIfAbsent( + seriesSlotId, empty -> newSeriesPartitionTable(consensusGroupId, timeSlotId)) + .putDataPartition(timePartitionSlot, consensusGroupId); + } + } + } + + /** Generate DataPartitionTable by scanning all resource files. */ + private void generateDataPartitionTable() throws IOException { + LOG.info("Starting DataPartitionTable generation from {} directories", dataDirectories.length); + + List> futures = new ArrayList<>(); + + Map dataPartitionMap = new ConcurrentHashMap<>(); + + try { + // Count total files first for progress tracking + countTotalFiles(); + + // Process all data directories + for (String dataDirectory : dataDirectories) { + LOG.info("Processing data directory: {}", dataDirectory); + + // First layer: database directories + Files.list(Paths.get(dataDirectory)) + .filter(Files::isDirectory) + .forEach( + sequenceTypePath -> { + try { + Files.list(sequenceTypePath) + .filter(Files::isDirectory) + .forEach( + dbPath -> { + String databaseName = dbPath.getFileName().toString(); + if (!databases.contains(databaseName) + || IGNORE_DATABASE.contains(databaseName)) { + return; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Processing database: {}", databaseName); + } + + try { + Files.list(dbPath) + .filter(Files::isDirectory) + .forEach( + regionPath -> { + processRegionDirectory( + regionPath, + databaseName, + dataPartitionMap, + executor, + futures); + }); + } catch (IOException e) { + LOG.error("Failed to process database directory: {}", dbPath, e); + failedFiles.incrementAndGet(); + } + }); + } catch (IOException e) { + LOG.error("Failed to process database directory: {}", sequenceTypePath, e); + failedFiles.incrementAndGet(); + } + }); + } + + // Wait for all tasks to complete + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join(); + + dataPartitionTable = new DataPartitionTable(dataPartitionMap); + + LOG.info( + "DataPartitionTable generation completed. Processed: {}, Failed: {}", + processedFiles.get(), + failedFiles.get()); + + } finally { + executor.shutdown(); + try { + if (!executor.awaitTermination(EXECUTOR_MAX_TIMEOUT, TimeUnit.SECONDS)) { + executor.shutdownNow(); + } + } catch (InterruptedException e) { + executor.shutdownNow(); + Thread.currentThread().interrupt(); + } + } + } + + /** Process a region directory. */ + private void processRegionDirectory( + java.nio.file.Path regionPath, + String databaseName, + Map dataPartitionMap, + ExecutorService executor, + List> futures) { + + int regionId; + try { + regionId = Integer.parseInt(regionPath.getFileName().toString()); + LOG.debug("Processing region: {}", regionId); + } catch (NumberFormatException e) { + LOG.error("Invalid region directory: {}", regionPath); + return; + } + + TConsensusGroupId consensusGroupId = new TConsensusGroupId(); + consensusGroupId.setId(regionId); + consensusGroupId.setType(TConsensusGroupType.DataRegion); + + // Process time partitions asynchronously + CompletableFuture regionFuture = + CompletableFuture.runAsync( + () -> { + try { + Files.list(regionPath) + .filter(Files::isDirectory) + .forEach( + timeSlotPath -> { + processTimeSlotDirectory( + timeSlotPath, databaseName, consensusGroupId, dataPartitionMap); + }); + } catch (IOException e) { + LOG.error("Failed to list region directory: {}", regionPath, e); + } + }, + executor); + + futures.add(regionFuture); + } + + /** Process a time slot directory. */ + private void processTimeSlotDirectory( + java.nio.file.Path timeSlotPath, + String databaseName, + TConsensusGroupId consensusGroupId, + Map dataPartitionMap) { + + long timeSlotLong; + try { + timeSlotLong = Long.parseLong(timeSlotPath.getFileName().toString()); + LOG.debug("Processing time slot: {}", timeSlotLong); + } catch (NumberFormatException e) { + LOG.error("Invalid time slot directory: {}", timeSlotPath); + return; + } + + try { + // Fourth layer: .resource files + Files.walk(timeSlotPath) + .filter(Files::isRegularFile) + .filter(p -> p.toString().endsWith(SCAN_FILE_SUFFIX_NAME)) + .forEach( + tsFilePath -> { + processTsFile( + tsFilePath.toFile(), consensusGroupId, timeSlotLong, dataPartitionMap); + }); + } catch (IOException e) { + LOG.error("Failed to walk time slot directory: {}", timeSlotPath, e); + } + } + + /** Process a single tsfile. */ + private void processTsFile( + File tsFile, + TConsensusGroupId consensusGroupId, + long timeSlotId, + Map dataPartitionMap) { + try { + TsFileResource tsFileResource = new TsFileResource(tsFile.getAbsoluteFile()); + tsFileResource.deserialize(); + + Set devices = tsFileResource.getDevices(limiter); + processedFiles.incrementAndGet(); + + SeriesPartitionExecutor seriesPartitionExecutor = + SeriesPartitionExecutor.getSeriesPartitionExecutor( + seriesPartitionExecutorClass, seriesSlotNum); + + for (org.apache.tsfile.file.metadata.IDeviceID deviceId : devices) { + TSeriesPartitionSlot seriesSlotId = + seriesPartitionExecutor.getSeriesPartitionSlot(deviceId); + TTimePartitionSlot timePartitionSlot = + new TTimePartitionSlot(TimePartitionUtils.getTimeByPartitionId(timeSlotId)); + dataPartitionMap + .computeIfAbsent( + seriesSlotId, empty -> newSeriesPartitionTable(consensusGroupId, timeSlotId)) + .putDataPartition(timePartitionSlot, consensusGroupId); + } + + if (processedFiles.get() % 1000 == 0) { + LOG.info("Processed {} files, current: {}", processedFiles.get(), tsFile.getName()); + } + } catch (IOException e) { + failedFiles.incrementAndGet(); + LOG.error("Failed to process tsfile: {} -> {}", tsFile.getAbsolutePath(), e.getMessage()); + } + } + + private static SeriesPartitionTable newSeriesPartitionTable( + TConsensusGroupId consensusGroupId, long timeSlotId) { + SeriesPartitionTable seriesPartitionTable = new SeriesPartitionTable(); + TTimePartitionSlot timePartitionSlot = + new TTimePartitionSlot(TimePartitionUtils.getTimeByPartitionId(timeSlotId)); + seriesPartitionTable.putDataPartition(timePartitionSlot, consensusGroupId); + return seriesPartitionTable; + } + + /** Count total files for progress tracking. */ + private void countTotalFiles() throws IOException { + AtomicLong fileCount = new AtomicLong(0); + + for (String dataDirectory : dataDirectories) { + Files.list(Paths.get(dataDirectory)) + .filter(Files::isDirectory) + .forEach( + sequenceTypePath -> { + try { + Files.list(sequenceTypePath) + .filter(Files::isDirectory) + .forEach( + dbPath -> { + String databaseName = dbPath.getFileName().toString(); + if (!databases.contains(databaseName) + || IGNORE_DATABASE.contains(databaseName)) { + return; + } + + try { + Files.walk(dbPath) + .filter(Files::isRegularFile) + .filter(p -> p.toString().endsWith(SCAN_FILE_SUFFIX_NAME)) + .forEach(p -> fileCount.incrementAndGet()); + } catch (IOException e) { + LOG.error("countTotalFiles failed when scan {}", dbPath, e); + } + }); + } catch (IOException e) { + LOG.error("countTotalFiles failed when scan {}", sequenceTypePath, e); + } + }); + } + + totalFiles.set(fileCount.get()); + LOG.info("Found {} resource files to process", totalFiles.get()); + } + + // Getters + public TaskStatus getStatus() { + return status; + } + + public String getErrorMessage() { + return errorMessage; + } + + public DataPartitionTable getDataPartitionTable() { + return dataPartitionTable; + } + + public int getProcessedFiles() { + return processedFiles.get(); + } + + public int getFailedFiles() { + return failedFiles.get(); + } + + public long getTotalFiles() { + return totalFiles.get(); + } + + public double getProgress() { + if (totalFiles.get() == 0) { + return 0.0; + } + return (double) (processedFiles.get() + failedFiles.get()) / totalFiles.get(); + } +} 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 9c44de9f5fdca..881e823ef2d67 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 @@ -55,7 +55,10 @@ public enum OperationType { WRITE_AUDIT_LOG("writeAuditLog"), PREPARE_STATEMENT("prepareStatement"), EXECUTE_PREPARED_STATEMENT("executePreparedStatement"), - DEALLOCATE_PREPARED_STATEMENT("deallocatePreparedStatement"); + DEALLOCATE_PREPARED_STATEMENT("deallocatePreparedStatement"), + GET_EARLIEST_TIMESLOTS("getEarliestTimeslots"), + GENERATE_DATA_PARTITION_TABLE("generateDataPartitionTable"), + CHECK_DATA_PARTITION_TABLE_STATUS("checkDataPartitionTableStatus"); private final String name; OperationType(String name) { 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 42929be741819..ab4f1523516aa 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 @@ -61,8 +61,10 @@ import org.apache.iotdb.commons.consensus.SchemaRegionId; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.ProgressIndexType; +import org.apache.iotdb.commons.enums.DataPartitionTableGeneratorState; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.commons.partition.DataPartitionTable; import org.apache.iotdb.commons.path.ExtendedPartialPath; import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; @@ -102,6 +104,7 @@ import org.apache.iotdb.db.consensus.SchemaRegionConsensusImpl; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.query.QueryProcessException; +import org.apache.iotdb.db.partition.DataPartitionTableGenerator; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.protocol.client.cn.DnToCnInternalServiceAsyncRequestManager; @@ -260,6 +263,10 @@ import org.apache.iotdb.mpp.rpc.thrift.TFireTriggerReq; import org.apache.iotdb.mpp.rpc.thrift.TFireTriggerResp; import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceInfoResp; +import org.apache.iotdb.mpp.rpc.thrift.TGenerateDataPartitionTableHeartbeatResp; +import org.apache.iotdb.mpp.rpc.thrift.TGenerateDataPartitionTableReq; +import org.apache.iotdb.mpp.rpc.thrift.TGenerateDataPartitionTableResp; +import org.apache.iotdb.mpp.rpc.thrift.TGetEarliestTimeslotsResp; import org.apache.iotdb.mpp.rpc.thrift.TInactiveTriggerInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidateColumnCacheReq; @@ -317,11 +324,15 @@ import com.google.common.collect.ImmutableList; import org.apache.thrift.TException; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.transport.TIOStreamTransport; +import org.apache.thrift.transport.TTransport; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.exception.NotImplementedException; import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.read.common.block.TsBlock; import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.PublicBAOS; import org.apache.tsfile.utils.RamUsageEstimator; import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.write.record.Tablet; @@ -331,9 +342,12 @@ import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; +import java.io.File; import java.io.IOException; import java.net.URL; import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Path; import java.time.ZoneId; import java.util.ArrayList; import java.util.Arrays; @@ -370,7 +384,6 @@ import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onQueryException; public class DataNodeInternalRPCServiceImpl implements IDataNodeRPCService.Iface { - private static final Logger LOGGER = LoggerFactory.getLogger(DataNodeInternalRPCServiceImpl.class); @@ -414,6 +427,34 @@ public class DataNodeInternalRPCServiceImpl implements IDataNodeRPCService.Iface private static final String SYSTEM = "system"; + private final ExecutorService findEarliestTimeSlotExecutor = + new WrappedThreadPoolExecutor( + 0, + IoTDBDescriptor.getInstance().getConfig().getPartitionTableRecoverWorkerNum(), + 0L, + TimeUnit.SECONDS, + new ArrayBlockingQueue<>( + IoTDBDescriptor.getInstance().getConfig().getPartitionTableRecoverWorkerNum()), + new IoTThreadFactory(ThreadName.FIND_EARLIEST_TIME_SLOT_PARALLEL_POOL.getName()), + ThreadName.FIND_EARLIEST_TIME_SLOT_PARALLEL_POOL.getName(), + new ThreadPoolExecutor.CallerRunsPolicy()); + + private final ExecutorService partitionTableRecoverExecutor = + new WrappedThreadPoolExecutor( + 0, + IoTDBDescriptor.getInstance().getConfig().getPartitionTableRecoverWorkerNum(), + 0L, + TimeUnit.SECONDS, + new ArrayBlockingQueue<>( + IoTDBDescriptor.getInstance().getConfig().getPartitionTableRecoverWorkerNum()), + new IoTThreadFactory(ThreadName.DATA_PARTITION_RECOVER_PARALLEL_POOL.getName()), + ThreadName.DATA_PARTITION_RECOVER_PARALLEL_POOL.getName(), + new ThreadPoolExecutor.CallerRunsPolicy()); + + private Map databaseEarliestRegionMap = new ConcurrentHashMap<>(); + + private static final long timeoutMs = 600000; // 600 seconds timeout + public DataNodeInternalRPCServiceImpl() { super(); partitionFetcher = ClusterPartitionFetcher.getInstance(); @@ -3117,4 +3158,292 @@ public TSStatus writeAuditLog(TAuditLogReq req) { public void handleClientExit() { // Do nothing } + + // ==================================================== + // Data Partition Table Integrity Check Implementation + // ==================================================== + + private volatile DataPartitionTableGenerator currentGenerator; + private volatile long currentTaskId = 0; + + @Override + public TGetEarliestTimeslotsResp getEarliestTimeslots() { + TGetEarliestTimeslotsResp resp = new TGetEarliestTimeslotsResp(); + + try { + Map earliestTimeslots = new HashMap<>(); + + // Get data directories from configuration + String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs(); + + for (String dataDir : dataDirs) { + File dir = new File(dataDir); + if (dir.exists() && dir.isDirectory()) { + processDataDirectoryForEarliestTimeslots(dir, earliestTimeslots); + } + } + + resp.setStatus(RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS)); + resp.setDatabaseToEarliestTimeslot(earliestTimeslots); + + LOGGER.info("Retrieved earliest timeslots for {} databases", earliestTimeslots.size()); + + } catch (Exception e) { + LOGGER.error("Failed to get earliest timeslots", e); + resp.setStatus( + onIoTDBException( + e, + OperationType.GET_EARLIEST_TIMESLOTS, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode())); + } + + return resp; + } + + @Override + public TGenerateDataPartitionTableResp generateDataPartitionTable( + TGenerateDataPartitionTableReq req) { + TGenerateDataPartitionTableResp resp = new TGenerateDataPartitionTableResp(); + byte[] empty = new byte[0]; + + try { + // Check if there's already a task in the progress + if (currentGenerator != null + && currentGenerator.getStatus() == DataPartitionTableGenerator.TaskStatus.IN_PROGRESS) { + resp.setDataPartitionTable(empty); + resp.setErrorCode(DataPartitionTableGeneratorState.IN_PROGRESS.getCode()); + resp.setMessage("DataPartitionTable generation is already in the progress"); + resp.setStatus(RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR)); + return resp; + } + + // Create generator for all data directories + int seriesSlotNum = IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionSlotNum(); + String seriesPartitionExecutorClass = + IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionExecutorClass(); + + currentGenerator = + new DataPartitionTableGenerator( + partitionTableRecoverExecutor, + req.getDatabases(), + seriesSlotNum, + seriesPartitionExecutorClass); + currentTaskId = System.currentTimeMillis(); + + // Start generation synchronously for now to return the data partition table immediately + currentGenerator.startGeneration().get(timeoutMs, TimeUnit.MILLISECONDS); + + if (currentGenerator != null) { + switch (currentGenerator.getStatus()) { + case IN_PROGRESS: + resp.setDataPartitionTable(empty); + resp.setErrorCode(DataPartitionTableGeneratorState.FAILED.getCode()); + resp.setMessage("DataPartitionTable generation interrupted"); + resp.setStatus(RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR)); + break; + case COMPLETED: + DataPartitionTable dataPartitionTable = currentGenerator.getDataPartitionTable(); + if (dataPartitionTable != null) { + byte[] result = serializeDataPartitionTable(dataPartitionTable); + resp.setDataPartitionTable(result); + } + + resp.setErrorCode(DataPartitionTableGeneratorState.SUCCESS.getCode()); + resp.setMessage("DataPartitionTable generation completed successfully"); + resp.setStatus(RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS)); + LOGGER.info("DataPartitionTable generation completed with task ID: {}", currentTaskId); + break; + default: + resp.setDataPartitionTable(empty); + resp.setErrorCode(DataPartitionTableGeneratorState.FAILED.getCode()); + resp.setMessage( + "DataPartitionTable generation failed: " + currentGenerator.getErrorMessage()); + resp.setStatus(RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR)); + break; + } + } + + // Clear current generator + currentGenerator = null; + } catch (Exception e) { + LOGGER.error("Failed to generate DataPartitionTable", e); + resp.setStatus( + onIoTDBException( + e, + OperationType.GENERATE_DATA_PARTITION_TABLE, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode())); + } + + return resp; + } + + @Override + public TGenerateDataPartitionTableHeartbeatResp generateDataPartitionTableHeartbeat() { + TGenerateDataPartitionTableHeartbeatResp resp = new TGenerateDataPartitionTableHeartbeatResp(); + + try { + if (currentGenerator == null) { + resp.setErrorCode(DataPartitionTableGeneratorState.UNKNOWN.getCode()); + resp.setMessage("No DataPartitionTable generation task found"); + resp.setStatus(RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR)); + return resp; + } + + DataPartitionTableGenerator.TaskStatus status = currentGenerator.getStatus(); + + switch (status) { + case IN_PROGRESS: + resp.setErrorCode(DataPartitionTableGeneratorState.IN_PROGRESS.getCode()); + resp.setMessage( + String.format( + "DataPartitionTable generation in progress: %.1f%%", + currentGenerator.getProgress() * 100)); + resp.setStatus(RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR)); + break; + case COMPLETED: + resp.setErrorCode(DataPartitionTableGeneratorState.SUCCESS.getCode()); + resp.setMessage("DataPartitionTable generation completed successfully"); + resp.setStatus(RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS)); + break; + case FAILED: + resp.setErrorCode(DataPartitionTableGeneratorState.FAILED.getCode()); + resp.setMessage( + "DataPartitionTable generation failed: " + currentGenerator.getErrorMessage()); + resp.setStatus(RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR)); + break; + default: + resp.setErrorCode(DataPartitionTableGeneratorState.UNKNOWN.getCode()); + resp.setMessage("Unknown task status: " + status); + resp.setStatus(RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR)); + break; + } + } catch (Exception e) { + LOGGER.error("Failed to check DataPartitionTable generation status", e); + resp.setStatus( + onIoTDBException( + e, + OperationType.CHECK_DATA_PARTITION_TABLE_STATUS, + TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode())); + } + + return resp; + } + + /** Process data directory to find the earliest timeslots for each database. */ + private void processDataDirectoryForEarliestTimeslots( + File dataDir, Map earliestTimeslots) { + try { + Files.list(dataDir.toPath()) + .filter(Files::isDirectory) + .forEach( + sequenceTypePath -> { + try { + Files.list(sequenceTypePath) + .filter(Files::isDirectory) + .forEach( + dbPath -> { + String databaseName = dbPath.getFileName().toString(); + if (DataPartitionTableGenerator.IGNORE_DATABASE.contains( + databaseName)) { + return; + } + databaseEarliestRegionMap.computeIfAbsent( + databaseName, key -> Long.MAX_VALUE); + long earliestTimeslot = findEarliestTimeslotInDatabase(dbPath.toFile()); + + if (earliestTimeslot != Long.MAX_VALUE) { + earliestTimeslots.merge(databaseName, earliestTimeslot, Math::min); + } + }); + } catch (IOException e) { + LOGGER.error( + "Failed to process data directory: {}", sequenceTypePath.toFile(), e); + } + }); + } catch (IOException e) { + LOGGER.error("Failed to process data directory: {}", dataDir, e); + } + } + + /** Find the earliest timeslot in a database directory. */ + private long findEarliestTimeslotInDatabase(File databaseDir) { + String databaseName = databaseDir.getName(); + List> futureList = new ArrayList<>(); + + try { + Files.list(databaseDir.toPath()) + .filter(Files::isDirectory) + .forEach( + regionPath -> { + Future future = + findEarliestTimeSlotExecutor.submit( + () -> { + try { + Files.list(regionPath) + .filter(Files::isDirectory) + .forEach( + timeSlotPath -> { + try { + Optional matchedFile = + Files.find( + timeSlotPath, + 1, + (path, attrs) -> + attrs.isRegularFile() + && path.toString() + .endsWith( + DataPartitionTableGenerator + .SCAN_FILE_SUFFIX_NAME)) + .findFirst(); + if (!matchedFile.isPresent()) { + return; + } + String timeSlotName = timeSlotPath.getFileName().toString(); + long timeslot = Long.parseLong(timeSlotName); + if (timeslot + < databaseEarliestRegionMap.get(databaseName)) { + databaseEarliestRegionMap.put(databaseName, timeslot); + } + } catch (IOException e) { + LOGGER.error( + "Failed to find any {} files in the {} directory", + DataPartitionTableGenerator.SCAN_FILE_SUFFIX_NAME, + timeSlotPath, + e); + } + }); + } catch (IOException e) { + LOGGER.error("Failed to scan {}", regionPath, e); + } + }); + futureList.add(future); + }); + } catch (IOException e) { + LOGGER.error("Failed to walk database directory: {}", databaseDir, e); + } + + for (Future future : futureList) { + try { + future.get(); + } catch (InterruptedException | ExecutionException e) { + LOGGER.error("Failed to wait for task completion", e); + Thread.currentThread().interrupt(); + } + } + return databaseEarliestRegionMap.get(databaseName); + } + + /** Serialize DataPartitionTable to ByteBuffer for RPC transmission. */ + private byte[] serializeDataPartitionTable(DataPartitionTable dataPartitionTable) { + try (PublicBAOS baos = new PublicBAOS(); + DataOutputStream oos = new DataOutputStream(baos)) { + TTransport transport = new TIOStreamTransport(oos); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + dataPartitionTable.serialize(oos, protocol); + return baos.getBuf(); + } catch (IOException | TException e) { + LOGGER.error("Failed to serialize DataPartitionTable", e); + return ByteBuffer.allocate(0).array(); + } + } } 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 b84cce9e8d21b..f4a950a72afd6 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 @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.pipe.datastructure.resource.PersistentResource; import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.commons.utils.rateLimiter.LeakyBucketRateLimiter; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.load.PartitionViolationException; @@ -677,6 +678,10 @@ public Set getDevices() { return timeIndex.getDevices(file.getPath(), this); } + public Set getDevices(LeakyBucketRateLimiter limiter) { + return timeIndex.getDevices(file.getPath(), this, limiter); + } + public ArrayDeviceTimeIndex buildDeviceTimeIndex(IDeviceID.Deserializer deserializer) throws IOException { readLock(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ArrayDeviceTimeIndex.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ArrayDeviceTimeIndex.java index 8499b6d6b3d3e..a3262ddd37a1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ArrayDeviceTimeIndex.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ArrayDeviceTimeIndex.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.commons.utils.TimePartitionUtils; +import org.apache.iotdb.commons.utils.rateLimiter.LeakyBucketRateLimiter; import org.apache.iotdb.db.exception.load.PartitionViolationException; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -171,6 +172,12 @@ public Set getDevices(String tsFilePath, TsFileResource tsFileResourc return deviceToIndex.keySet(); } + @Override + public Set getDevices( + String tsFilePath, TsFileResource tsFileResource, LeakyBucketRateLimiter limiter) { + return deviceToIndex.keySet(); + } + public Map getDeviceToIndex() { return deviceToIndex; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java index e4a812012a8e3..059663c5a6aea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.commons.utils.TimePartitionUtils; +import org.apache.iotdb.commons.utils.rateLimiter.LeakyBucketRateLimiter; import org.apache.iotdb.db.exception.load.PartitionViolationException; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -120,6 +121,41 @@ public Set getDevices(String tsFilePath, TsFileResource tsFileResourc } } + @Override + public Set getDevices( + String tsFilePath, TsFileResource tsFileResource, LeakyBucketRateLimiter limiter) { + tsFileResource.readLock(); + try (InputStream inputStream = + FSFactoryProducer.getFSFactory() + .getBufferedInputStream(tsFilePath + TsFileResource.RESOURCE_SUFFIX)) { + // The first byte is VERSION_NUMBER, second byte is timeIndexType. + byte[] bytes = ReadWriteIOUtils.readBytes(inputStream, 2); + limiter.acquire(bytes.length); + if (bytes[1] == ARRAY_DEVICE_TIME_INDEX_TYPE) { + return ArrayDeviceTimeIndex.getDevices(inputStream); + } else { + return PlainDeviceTimeIndex.getDevices(inputStream); + } + } catch (NoSuchFileException e) { + // deleted by ttl + if (tsFileResource.isDeleted()) { + return Collections.emptySet(); + } else { + logger.error( + "Can't read file {} from disk ", tsFilePath + TsFileResource.RESOURCE_SUFFIX, e); + throw new RuntimeException( + "Can't read file " + tsFilePath + TsFileResource.RESOURCE_SUFFIX + " from disk"); + } + } catch (Exception e) { + logger.error( + "Failed to get devices from tsfile: {}", tsFilePath + TsFileResource.RESOURCE_SUFFIX, e); + throw new RuntimeException( + "Failed to get devices from tsfile: " + tsFilePath + TsFileResource.RESOURCE_SUFFIX); + } finally { + tsFileResource.readUnlock(); + } + } + @Override public boolean endTimeEmpty() { return endTime == Long.MIN_VALUE; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ITimeIndex.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ITimeIndex.java index d705a2417d7c6..400c478df5054 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ITimeIndex.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ITimeIndex.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.utils.rateLimiter.LeakyBucketRateLimiter; import org.apache.iotdb.db.exception.load.PartitionViolationException; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -74,6 +75,14 @@ ITimeIndex deserialize(InputStream inputStream, IDeviceID.Deserializer deseriali */ Set getDevices(String tsFilePath, TsFileResource tsFileResource); + /** + * get devices in TimeIndex and limit files reading rate + * + * @return device names + */ + Set getDevices( + String tsFilePath, TsFileResource tsFileResource, LeakyBucketRateLimiter limiter); + /** * @return whether end time is empty (Long.MIN_VALUE) */ diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/service/DataNodeInternalRPCServiceImplTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/service/DataNodeInternalRPCServiceImplTest.java index 622c2c4ebbfe7..066e1bea5bfd7 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/service/DataNodeInternalRPCServiceImplTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/service/DataNodeInternalRPCServiceImplTest.java @@ -68,6 +68,8 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; @@ -81,6 +83,8 @@ public class DataNodeInternalRPCServiceImplTest { + private static final Logger LOG = + LoggerFactory.getLogger(DataNodeInternalRPCServiceImplTest.class); private static final IoTDBConfig conf = IoTDBDescriptor.getInstance().getConfig(); DataNodeInternalRPCServiceImpl dataNodeInternalRPCServiceImpl; private static IConsensus instance; 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 f4ebae2fb807e..c36f35cd5778c 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 @@ -742,6 +742,21 @@ failure_detector_phi_acceptable_pause_in_ms=10000 # Datatype: double(percentage) disk_space_warning_threshold=0.05 +# The number of threads used for parallel scanning in the partition table recovery +# effectiveMode: restart +# Datatype: Integer +partition_table_recover_worker_num=10 + +# Limit the number of bytes read per second from a file, the unit is MB +# effectiveMode: restart +# Datatype: Integer +partition_table_recover_max_read_bytes_per_second=10 + +# Set a timeout to wait for all datanodes complete startup, the unit is ms +# effectiveMode: restart +# Datatype: Integer +partition_table_recover_wait_all_dn_up_timeout=60000 + #################### ### Memory Control Configuration #################### 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 6f9f95ca8fe88..39bc7eebfa92b 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 @@ -202,6 +202,8 @@ public enum ThreadName { FILE_TIME_INDEX_RECORD("FileTimeIndexRecord"), BINARY_ALLOCATOR_SAMPLE_EVICTOR("BinaryAllocator-SampleEvictor"), BINARY_ALLOCATOR_AUTO_RELEASER("BinaryAllocator-Auto-Releaser"), + FIND_EARLIEST_TIME_SLOT_PARALLEL_POOL("FindEarliestTimeSlot-Parallel-Pool"), + DATA_PARTITION_RECOVER_PARALLEL_POOL("DataPartitionRecover-Parallel-Pool"), // the unknown thread name is used for metrics UNKNOWN("UNKNOWN"); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/enums/DataPartitionTableGeneratorState.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/enums/DataPartitionTableGeneratorState.java new file mode 100644 index 0000000000000..a07f6e313cdb2 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/enums/DataPartitionTableGeneratorState.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.commons.enums; + +public enum DataPartitionTableGeneratorState { + SUCCESS(0), + FAILED(1), + IN_PROGRESS(2), + UNKNOWN(-1); + + private final int code; + + DataPartitionTableGeneratorState(int code) { + this.code = code; + } + + public int getCode() { + return code; + } + + /** + * get DataNodeRemoveState by code + * + * @param code code + * @return DataNodeRemoveState + */ + public static DataPartitionTableGeneratorState getStateByCode(int code) { + for (DataPartitionTableGeneratorState state : DataPartitionTableGeneratorState.values()) { + if (state.code == code) { + return state; + } + } + return UNKNOWN; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionTable.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionTable.java index f46344566dc32..ffb0413bc87e7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionTable.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionTable.java @@ -73,7 +73,11 @@ public Map> getSeriesPartitionMap() } public void putDataPartition(TTimePartitionSlot timePartitionSlot, TConsensusGroupId groupId) { - seriesPartitionMap.computeIfAbsent(timePartitionSlot, empty -> new Vector<>()).add(groupId); + seriesPartitionMap.computeIfAbsent(timePartitionSlot, empty -> new Vector<>()); + List groupList = seriesPartitionMap.get(timePartitionSlot); + if (!groupList.contains(groupId)) { + groupList.add(groupId); + } } /** diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java index eb53cdb2798dd..4eeddff9db7f9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java @@ -112,6 +112,14 @@ public static long getTimePartitionId(long time) { : time / timePartitionInterval - 1; } + public static long getTime(long partitionId) { + long time = partitionId * timePartitionInterval; + if (time > 0 || time % timePartitionInterval == 0) { + return time + timePartitionOrigin; + } + return ((partitionId + 1) * timePartitionInterval) + timePartitionOrigin; + } + public static long getTimePartitionIdWithoutOverflow(long time) { BigInteger bigTime = BigInteger.valueOf(time).subtract(bigTimePartitionOrigin); BigInteger partitionId = @@ -122,6 +130,23 @@ public static long getTimePartitionIdWithoutOverflow(long time) { return partitionId.longValue(); } + public static long getTimeWithoutOverflow(long partitionId) { + BigInteger bigTime = bigTimePartitionInterval.multiply(BigInteger.valueOf(partitionId)); + if (bigTime.compareTo(BigInteger.ZERO) > 0 + || bigTime.remainder(bigTimePartitionInterval).equals(BigInteger.ZERO)) { + return bigTime.add(bigTimePartitionOrigin).longValue(); + } + return BigInteger.valueOf(partitionId) + .add(BigInteger.ONE) + .multiply(bigTimePartitionInterval) + .add(bigTimePartitionOrigin) + .longValue(); + } + + public static long getTimeByPartitionId(long partitionId) { + return originMayCauseOverflow ? getTimeWithoutOverflow(partitionId) : getTime(partitionId); + } + public static boolean satisfyPartitionId(long startTime, long endTime, long partitionId) { long startPartition = originMayCauseOverflow @@ -134,6 +159,14 @@ public static boolean satisfyPartitionId(long startTime, long endTime, long part return startPartition <= partitionId && endPartition >= partitionId; } + public static boolean satisfyPartitionId(long startTime, long partitionId) { + long endTime = + startTime >= timePartitionLowerBoundWithoutOverflow + ? Long.MAX_VALUE + : (startTime + timePartitionInterval - 1); + return satisfyPartitionId(startTime, endTime, partitionId); + } + public static boolean satisfyPartitionStartTime(Filter timeFilter, long partitionStartTime) { if (timeFilter == null) { return true; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/rateLimiter/LeakyBucketRateLimiter.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/rateLimiter/LeakyBucketRateLimiter.java new file mode 100644 index 0000000000000..7af863db614b4 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/rateLimiter/LeakyBucketRateLimiter.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT 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.rateLimiter; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.LockSupport; + +/** + * A global leaky-bucket rate limiter for bytes throughput. Features: - Strict throughput limiting + * (no burst) - Smooth bandwidth shaping - Thread-safe - Fair for multi-thread - Low contention + */ +public class LeakyBucketRateLimiter { + /** bytes per second */ + private volatile long bytesPerSecond; + + /** start time */ + private final long startTimeNs; + + /** total consumed bytes */ + private final AtomicLong totalBytes = new AtomicLong(0); + + public LeakyBucketRateLimiter(long bytesPerSecond) { + if (bytesPerSecond <= 0) { + throw new IllegalArgumentException("bytesPerSecond must be > 0"); + } + this.bytesPerSecond = bytesPerSecond; + this.startTimeNs = System.nanoTime(); + } + + /** + * Acquire permission for reading bytes. + * + *

This method will block if reading too fast. + */ + public void acquire(long bytes) { + if (bytes <= 0) { + return; + } + + long currentTotal = totalBytes.addAndGet(bytes); + + long expectedTimeNs = expectedTimeNs(currentTotal); + long now = System.nanoTime(); + + long sleepNs = expectedTimeNs - now; + + if (sleepNs > 0) { + LockSupport.parkNanos(sleepNs); + } + } + + /** + * Try acquire without blocking. + * + * @return true if allowed immediately + */ + public boolean tryAcquire(long bytes) { + if (bytes <= 0) { + return true; + } + + long currentTotal = totalBytes.addAndGet(bytes); + + long expectedTimeNs = expectedTimeNs(currentTotal); + long now = System.nanoTime(); + + if (expectedTimeNs <= now) { + return true; + } + + // rollback + totalBytes.addAndGet(-bytes); + return false; + } + + /** Update rate dynamically. */ + public void setRate(long newBytesPerSecond) { + if (newBytesPerSecond <= 0) { + throw new IllegalArgumentException("bytesPerSecond must be > 0"); + } + this.bytesPerSecond = newBytesPerSecond; + } + + /** Current rate. */ + public long getRate() { + return bytesPerSecond; + } + + /** Total bytes processed. */ + public long getTotalBytes() { + return totalBytes.get(); + } + + /** Expected time based on bytes processed. */ + private long expectedTimeNs(long totalBytes) { + return startTimeNs + (totalBytes * 1_000_000_000L) / bytesPerSecond; + } +} diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index cca7110f28d40..b248599f59cc4 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -678,6 +678,36 @@ struct TAuditLogReq { 11: required i32 cnId } +/** +* BEGIN: Data Partition Table Integrity Check Structures +**/ + +struct TGetEarliestTimeslotsResp { + 1: required common.TSStatus status + 2: optional map databaseToEarliestTimeslot +} + +struct TGenerateDataPartitionTableReq { + 1: required set databases +} + +struct TGenerateDataPartitionTableResp { + 1: required common.TSStatus status + 2: required i32 errorCode + 3: optional string message + 4: optional binary dataPartitionTable +} + +struct TGenerateDataPartitionTableHeartbeatResp { + 1: required common.TSStatus status + 2: required i32 errorCode + 3: optional string message +} + +/** +* END: Data Partition Table Integrity Check Structures +**/ + /** * BEGIN: Used for EXPLAIN ANALYZE **/ @@ -1276,6 +1306,30 @@ service IDataNodeRPCService { * Write an audit log entry to the DataNode's AuditEventLogger */ common.TSStatus writeAuditLog(TAuditLogReq req); + + /** + * BEGIN: Data Partition Table Integrity Check + **/ + + /** + * Get earliest timeslot information from DataNode + * Returns map of database name to earliest timeslot id + */ + TGetEarliestTimeslotsResp getEarliestTimeslots() + + /** + * Request DataNode to generate DataPartitionTable by scanning tsfile resources + */ + TGenerateDataPartitionTableResp generateDataPartitionTable(TGenerateDataPartitionTableReq req) + + /** + * Check the status of DataPartitionTable generation task + */ + TGenerateDataPartitionTableHeartbeatResp generateDataPartitionTableHeartbeat() + + /** + * END: Data Partition Table Integrity Check + **/ } service MPPDataExchangeService {