3232import org .apache .iotdb .db .pipe .agent .task .PipeDataNodeTask ;
3333import org .apache .iotdb .db .pipe .event .common .tsfile .PipeTsFileInsertionEvent ;
3434
35+ import org .slf4j .Logger ;
36+ import org .slf4j .LoggerFactory ;
37+
38+ import java .util .Objects ;
3539import java .util .concurrent .ArrayBlockingQueue ;
40+ import java .util .concurrent .ConcurrentHashMap ;
41+ import java .util .concurrent .ConcurrentMap ;
3642import java .util .concurrent .ExecutorService ;
3743import java .util .concurrent .TimeUnit ;
44+ import java .util .concurrent .atomic .AtomicLong ;
3845
3946/**
4047 * The {@link PipeTerminateEvent} is an {@link EnrichedEvent} that controls the termination of pipe,
4451 */
4552public class PipeTerminateEvent extends EnrichedEvent {
4653
54+ private static final Logger LOGGER = LoggerFactory .getLogger (PipeTerminateEvent .class );
55+
4756 private final int dataRegionId ;
4857
4958 private final boolean shouldMark ;
@@ -57,6 +66,9 @@ public class PipeTerminateEvent extends EnrichedEvent {
5766 // Do not use call run policy to avoid deadlock
5867 private static final ExecutorService terminateExecutor = createTerminateExecutor ();
5968
69+ private static final ConcurrentMap <HistoricalTransferKey , HistoricalTransferSummaryCounter >
70+ HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP = new ConcurrentHashMap <>();
71+
6072 private static ExecutorService createTerminateExecutor () {
6173 final WrappedThreadPoolExecutor executor =
6274 new WrappedThreadPoolExecutor (
@@ -128,6 +140,18 @@ public boolean mayEventPathsOverlappedWithPattern() {
128140 }
129141
130142 public void markCompleted () {
143+ final HistoricalTransferSummary summary =
144+ snapshotAndClearHistoricalTransferSummary (pipeName , creationTime , dataRegionId );
145+ if (Objects .nonNull (summary )) {
146+ LOGGER .info (
147+ "Pipe {}@{}: terminate event committed for historical transfer. creationTime: {}, shouldMark: {}. {}" ,
148+ pipeName ,
149+ dataRegionId ,
150+ creationTime ,
151+ shouldMark ,
152+ summary .toReportMessage ());
153+ }
154+
131155 // To avoid deadlock
132156 if (shouldMark ) {
133157 terminateExecutor .submit (
@@ -142,4 +166,159 @@ public String toString() {
142166 + " - "
143167 + super .toString ();
144168 }
169+
170+ public static void initializeHistoricalTransferSummary (
171+ final String pipeName ,
172+ final long creationTime ,
173+ final int dataRegionId ,
174+ final long extractedHistoricalTsFileCount ,
175+ final long extractedHistoricalDeletionCount ) {
176+ HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP
177+ .computeIfAbsent (
178+ new HistoricalTransferKey (pipeName , creationTime , dataRegionId ),
179+ ignored -> new HistoricalTransferSummaryCounter ())
180+ .initialize (extractedHistoricalTsFileCount , extractedHistoricalDeletionCount );
181+ }
182+
183+ public static void markHistoricalTsFileSkipped (
184+ final String pipeName , final long creationTime , final int dataRegionId ) {
185+ getOrCreateHistoricalTransferSummaryCounter (pipeName , creationTime , dataRegionId )
186+ .skippedHistoricalTsFileCount
187+ .incrementAndGet ();
188+ }
189+
190+ public static void markHistoricalTsFileSplit (
191+ final String pipeName , final long creationTime , final int dataRegionId ) {
192+ getOrCreateHistoricalTransferSummaryCounter (pipeName , creationTime , dataRegionId )
193+ .splitHistoricalTsFileCount
194+ .incrementAndGet ();
195+ }
196+
197+ public static void markHistoricalTsFileUnsplit (
198+ final String pipeName , final long creationTime , final int dataRegionId ) {
199+ getOrCreateHistoricalTransferSummaryCounter (pipeName , creationTime , dataRegionId )
200+ .unsplitHistoricalTsFileCount
201+ .incrementAndGet ();
202+ }
203+
204+ public static HistoricalTransferSummary snapshotHistoricalTransferSummary (
205+ final String pipeName , final long creationTime , final int dataRegionId ) {
206+ final HistoricalTransferSummaryCounter counter =
207+ HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP .get (
208+ new HistoricalTransferKey (pipeName , creationTime , dataRegionId ));
209+ return Objects .nonNull (counter ) ? counter .snapshot () : null ;
210+ }
211+
212+ public static void clearHistoricalTransferSummary (
213+ final String pipeName , final long creationTime , final int dataRegionId ) {
214+ HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP .remove (
215+ new HistoricalTransferKey (pipeName , creationTime , dataRegionId ));
216+ }
217+
218+ private static HistoricalTransferSummary snapshotAndClearHistoricalTransferSummary (
219+ final String pipeName , final long creationTime , final int dataRegionId ) {
220+ final HistoricalTransferSummaryCounter counter =
221+ HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP .remove (
222+ new HistoricalTransferKey (pipeName , creationTime , dataRegionId ));
223+ return Objects .nonNull (counter ) ? counter .snapshot () : null ;
224+ }
225+
226+ private static HistoricalTransferSummaryCounter getOrCreateHistoricalTransferSummaryCounter (
227+ final String pipeName , final long creationTime , final int dataRegionId ) {
228+ return HISTORICAL_TRANSFER_SUMMARY_COUNTER_MAP .computeIfAbsent (
229+ new HistoricalTransferKey (pipeName , creationTime , dataRegionId ),
230+ ignored -> new HistoricalTransferSummaryCounter ());
231+ }
232+
233+ public static final class HistoricalTransferSummary {
234+
235+ private final long extractedHistoricalTsFileCount ;
236+ private final long skippedHistoricalTsFileCount ;
237+ private final long splitHistoricalTsFileCount ;
238+ private final long unsplitHistoricalTsFileCount ;
239+ private final long extractedHistoricalDeletionCount ;
240+
241+ private HistoricalTransferSummary (
242+ final long extractedHistoricalTsFileCount ,
243+ final long skippedHistoricalTsFileCount ,
244+ final long splitHistoricalTsFileCount ,
245+ final long unsplitHistoricalTsFileCount ,
246+ final long extractedHistoricalDeletionCount ) {
247+ this .extractedHistoricalTsFileCount = extractedHistoricalTsFileCount ;
248+ this .skippedHistoricalTsFileCount = skippedHistoricalTsFileCount ;
249+ this .splitHistoricalTsFileCount = splitHistoricalTsFileCount ;
250+ this .unsplitHistoricalTsFileCount = unsplitHistoricalTsFileCount ;
251+ this .extractedHistoricalDeletionCount = extractedHistoricalDeletionCount ;
252+ }
253+
254+ public String toReportMessage () {
255+ return String .format (
256+ "historical summary: extractedTsFileCount=%s, skippedTsFileCount=%s, splitTsFileCount=%s, unsplitTsFileCount=%s, deletionCount=%s" ,
257+ extractedHistoricalTsFileCount ,
258+ skippedHistoricalTsFileCount ,
259+ splitHistoricalTsFileCount ,
260+ unsplitHistoricalTsFileCount ,
261+ extractedHistoricalDeletionCount );
262+ }
263+ }
264+
265+ private static final class HistoricalTransferSummaryCounter {
266+
267+ private final AtomicLong extractedHistoricalTsFileCount = new AtomicLong (0 );
268+ private final AtomicLong skippedHistoricalTsFileCount = new AtomicLong (0 );
269+ private final AtomicLong splitHistoricalTsFileCount = new AtomicLong (0 );
270+ private final AtomicLong unsplitHistoricalTsFileCount = new AtomicLong (0 );
271+ private final AtomicLong extractedHistoricalDeletionCount = new AtomicLong (0 );
272+
273+ private void initialize (
274+ final long extractedHistoricalTsFileCount , final long extractedHistoricalDeletionCount ) {
275+ this .extractedHistoricalTsFileCount .set (extractedHistoricalTsFileCount );
276+ this .skippedHistoricalTsFileCount .set (0 );
277+ this .splitHistoricalTsFileCount .set (0 );
278+ this .unsplitHistoricalTsFileCount .set (0 );
279+ this .extractedHistoricalDeletionCount .set (extractedHistoricalDeletionCount );
280+ }
281+
282+ private HistoricalTransferSummary snapshot () {
283+ return new HistoricalTransferSummary (
284+ extractedHistoricalTsFileCount .get (),
285+ skippedHistoricalTsFileCount .get (),
286+ splitHistoricalTsFileCount .get (),
287+ unsplitHistoricalTsFileCount .get (),
288+ extractedHistoricalDeletionCount .get ());
289+ }
290+ }
291+
292+ private static final class HistoricalTransferKey {
293+
294+ private final String pipeName ;
295+ private final long creationTime ;
296+ private final int dataRegionId ;
297+
298+ private HistoricalTransferKey (
299+ final String pipeName , final long creationTime , final int dataRegionId ) {
300+ this .pipeName = pipeName ;
301+ this .creationTime = creationTime ;
302+ this .dataRegionId = dataRegionId ;
303+ }
304+
305+ @ Override
306+ public boolean equals (final Object obj ) {
307+ if (this == obj ) {
308+ return true ;
309+ }
310+ if (!(obj instanceof HistoricalTransferKey )) {
311+ return false ;
312+ }
313+ final HistoricalTransferKey that = (HistoricalTransferKey ) obj ;
314+ return creationTime == that .creationTime
315+ && dataRegionId == that .dataRegionId
316+ && Objects .equals (pipeName , that .pipeName );
317+ }
318+
319+ @ Override
320+ public int hashCode () {
321+ return Objects .hash (pipeName , creationTime , dataRegionId );
322+ }
323+ }
145324}
0 commit comments