/
SnowflakeSinkServiceV1.java
892 lines (801 loc) · 23.2 KB
/
SnowflakeSinkServiceV1.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
package com.snowflake.kafka.connector.internal;
import com.snowflake.kafka.connector.SnowflakeSinkConnectorConfig;
import com.snowflake.kafka.connector.Utils;
import com.snowflake.kafka.connector.records.RecordService;
import com.snowflake.kafka.connector.records.SnowflakeMetadataConfig;
import com.snowflake.kafka.connector.records.SnowflakeRecordContent;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.connect.sink.SinkRecord;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
class SnowflakeSinkServiceV1 extends Logging implements SnowflakeSinkService
{
private static final long ONE_HOUR = 60 * 60 * 1000L;
private static final long TEN_MINUTES = 10 * 60 * 1000L;
private static final long CLEAN_TIME = 60 * 1000L; //one minutes
private long flushTime; // in seconds
private long fileSize;
private long recordNum;
private final SnowflakeConnectionService conn;
private final Map<String, ServiceContext> pipes;
private final RecordService recordService;
private boolean isStopped;
private final SnowflakeTelemetryService telemetryService;
private Map<String, String> topic2TableMap;
SnowflakeSinkServiceV1(SnowflakeConnectionService conn)
{
if (conn == null || conn.isClosed())
{
throw SnowflakeErrors.ERROR_5010.getException();
}
this.fileSize = SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_DEFAULT;
this.recordNum = SnowflakeSinkConnectorConfig.BUFFER_COUNT_RECORDS_DEFAULT;
this.flushTime = SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_DEFAULT;
this.pipes = new HashMap<>();
this.conn = conn;
this.recordService = new RecordService();
isStopped = false;
this.telemetryService = conn.getTelemetryClient();
this.topic2TableMap = new HashMap<>();
}
@Override
public void startTask(final String tableName, final String topic,
final int partition)
{
String stageName = Utils.stageName(conn.getConnectorName(), tableName);
String nameIndex = getNameIndex(topic, partition);
if (pipes.containsKey(nameIndex))
{
logError("task is already registered, name: {}", nameIndex);
}
else
{
String pipeName = Utils.pipeName(conn.getConnectorName(), tableName,
partition);
pipes.put(nameIndex, new ServiceContext(tableName, stageName, pipeName,
conn, partition));
}
}
@Override
public void insert(final SinkRecord record)
{
String nameIndex = getNameIndex(record.topic(), record.kafkaPartition());
//init a new topic partition
if (!pipes.containsKey(nameIndex))
{
logWarn("Topic: {} Partition: {} hasn't been initialized by OPEN " +
"function", record.topic(), record.kafkaPartition());
startTask(Utils.tableName(record.topic(), this.topic2TableMap),
record.topic(), record.kafkaPartition());
}
pipes.get(nameIndex).insert(record);
}
@Override
public long getOffset(final TopicPartition topicPartition)
{
String name = getNameIndex(topicPartition.topic(),
topicPartition.partition());
if (pipes.containsKey(name))
{
return pipes.get(name).getOffset();
}
else
{
logError("Failed to find offset of Topic: {}, Partition: {}, sink " +
"service hasn't been initialized", topicPartition.topic(),
topicPartition.partition());
return 0;
}
}
@Override
public void close(Collection<TopicPartition> partitions)
{
partitions.forEach(tp -> {
String name = getNameIndex(tp.topic(), tp.partition());
ServiceContext sc = pipes.remove(name);
if (sc != null)
{
try
{
sc.close();
} catch (Exception e)
{
logError("Failed to close sink service for Topic: {}, Partition: " +
"{}\nMessage:{}", tp.topic(), tp.partition(), e.getMessage());
}
}
else
{
logError("Failed to close sink service for Topic: {}, Partition: {}, " +
"sink service hasn't been initialized");
}
});
}
@Override
public void closeAll()
{
this.isStopped = true; // release all cleaner and flusher threads
pipes.forEach(
(name, context) -> context.close()
);
pipes.clear();
}
@Override
public boolean isClosed()
{
return this.isStopped;
}
@Override
public void setRecordNumber(final long num)
{
if (num < 0)
{
logError("number of record in each file is {}, it is negative, reset to" +
" 0");
this.recordNum = 0;
}
else
{
this.recordNum = num;
logInfo("set number of record limitation to {}", num);
}
}
@Override
public void setFileSize(final long size)
{
if (size > SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_MAX)
{
logError("file size is {} bytes, it is larger than the maximum file " +
"size {} bytes, reset to the maximum file size",
size, SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_MAX
);
this.fileSize = SnowflakeSinkConnectorConfig.BUFFER_SIZE_BYTES_MAX;
}
else
{
this.fileSize = size;
logInfo("set file size limitation to {} bytes", size);
}
}
@Override
public void setFlushTime(final long time)
{
if (time < SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_MIN)
{
logError("flush time is {} seconds, it is smaller than the minimum " +
"flush time {} seconds, reset to the minimum flush time",
time, SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_MIN
);
this.flushTime = SnowflakeSinkConnectorConfig.BUFFER_FLUSH_TIME_SEC_MIN;
}
else
{
this.flushTime = time;
logInfo("set flush time to {} seconds", time);
}
}
@Override
public void setTopic2TableMap(Map<String, String> topic2TableMap)
{
this.topic2TableMap = topic2TableMap;
}
@Override
public void setMetadataConfig(SnowflakeMetadataConfig configMap)
{
this.recordService.setMetadataConfig(configMap);
}
@Override
public long getRecordNumber()
{
return this.recordNum;
}
@Override
public long getFlushTime()
{
return this.flushTime;
}
@Override
public long getFileSize()
{
return this.fileSize;
}
private static String getNameIndex(String topic, int partition)
{
return topic + "_" + partition;
}
private class ServiceContext
{
private final String tableName;
private final String stageName;
private final String pipeName;
private final SnowflakeConnectionService conn;
private final SnowflakeIngestionService ingestionService;
private List<String> fileNames;
private PartitionBuffer buffer;
private final String prefix;
private long committedOffset; // loaded offset + 1
private long processedOffset; // processed offset
//threads
private final ExecutorService cleanerExecutor;
private final ExecutorService flusherExecutor;
private final Lock bufferLock;
private final Lock fileListLock;
private final Lock usageDataLock;
//telemetry
private long startTime;
private long totalNumberOfRecord;
private long totalSizeOfData;
//make the initialization lazy
private boolean hasInitialized = false;
private ServiceContext(String tableName, String stageName,
String pipeName, SnowflakeConnectionService conn,
int partition)
{
this.pipeName = pipeName;
this.tableName = tableName;
this.stageName = stageName;
this.conn = conn;
this.fileNames = new LinkedList<>();
this.buffer = new PartitionBuffer();
this.ingestionService = conn.buildIngestService(stageName, pipeName);
this.prefix = FileNameUtils.filePrefix(conn.getConnectorName(),
tableName, partition);
this.processedOffset = -1;
this.committedOffset = 0;
this.bufferLock = new ReentrantLock();
this.fileListLock = new ReentrantLock();
this.usageDataLock = new ReentrantLock();
this.totalNumberOfRecord = 0;
this.totalSizeOfData = 0;
this.startTime = System.currentTimeMillis();
this.cleanerExecutor = Executors.newSingleThreadExecutor();
this.flusherExecutor = Executors.newSingleThreadExecutor();
logInfo("pipe: {} - service started", pipeName);
}
private void init()
{
logInfo("init pipe: {}", pipeName);
//wait sinkConnector start
createTableAndStage();
recover();
try
{
startCleaner();
startFlusher();
} catch (Exception e)
{
logWarn("Cleaner and Flusher threads shut down before initialization");
}
}
private void startCleaner()
{
cleanerExecutor.submit(
() ->
{
logInfo("pipe {}: cleaner started", pipeName);
while (!isStopped)
{
try
{
Thread.sleep(CLEAN_TIME);
checkStatus();
} catch (InterruptedException e)
{
logError("cleaner error:\n{}", e.getMessage());
}
if (System.currentTimeMillis() - startTime > ONE_HOUR)
{
sendUsageReport();
}
}
}
);
}
private void stopCleaner()
{
cleanerExecutor.shutdown();
logInfo("pipe {}: cleaner terminated", pipeName);
}
private void startFlusher()
{
flusherExecutor.submit(
() ->
{
logInfo("pipe {}: flusher started", pipeName);
while (!isStopped)
{
try
{
Thread.sleep(getFlushTime() * 1000);
PartitionBuffer tmpBuff;
bufferLock.lock();
try
{
tmpBuff = buffer;
buffer = new PartitionBuffer();
} finally
{
bufferLock.unlock();
}
flush(tmpBuff);
logDebug("pipe {}: flusher flushed", pipeName);
} catch (InterruptedException e)
{
logError("flusher error:\n{}", e.getMessage());
}
}
}
);
}
private void stopFlusher()
{
flusherExecutor.shutdown();
logInfo("pipe {}: flusher terminated", pipeName);
}
private void insert(final SinkRecord record)
{
//init pipe
if (!hasInitialized)
{
init();
this.hasInitialized = true;
}
//ignore ingested files
if (record.kafkaOffset() > processedOffset)
{
if (!(record.value() instanceof SnowflakeRecordContent))
{
throw SnowflakeErrors.ERROR_0019.getException();
}
//broken record
else if (((SnowflakeRecordContent) record.value()).isBroken())
{
writeBrokenDataToTableStage(record);
//don't move committed offset in this case
//only move it in the normal cases
}
else
{
PartitionBuffer tmpBuff = null;
bufferLock.lock();
try
{
processedOffset = record.kafkaOffset();
buffer.insert(record);
if (buffer.getBufferSize() >= getFileSize() ||
(getRecordNumber() != 0 && buffer.getNumOfRecord() >= getRecordNumber()))
{
tmpBuff = buffer;
this.buffer = new PartitionBuffer();
}
} finally
{
bufferLock.unlock();
}
if (tmpBuff != null)
{
flush(tmpBuff);
}
}
}
}
private void writeBrokenDataToTableStage(SinkRecord record)
{
String fileName = FileNameUtils.brokenRecordFileName(prefix,
record.kafkaOffset());
conn.putToTableStage(tableName, fileName,
((SnowflakeRecordContent) record.value()).getBrokenData());
}
private long getOffset()
{
return committedOffset;
}
private void flush(PartitionBuffer buff)
{
if (buff == null || buff.isEmpty())
{
return;
}
String fileName = FileNameUtils.fileName(prefix, buff.getFirstOffset(),
buff.getLastOffset());
String content = buff.getData();
conn.put(stageName, fileName, content);
ingestionService.ingestFile(fileName);
fileListLock.lock();
try
{
fileNames.add(fileName);
} finally
{
fileListLock.unlock();
}
logInfo("pipe {}, flush pipe: {}", pipeName, fileName);
}
private void checkStatus()
{
List<String> tmpFileNames;
fileListLock.lock();
try
{
tmpFileNames = fileNames;
fileNames = new LinkedList<>();
} finally
{
fileListLock.unlock();
}
long currentTime = System.currentTimeMillis();
List<String> loadedFiles = new LinkedList<>();
List<String> failedFiles = new LinkedList<>();
//ingest report
filterResult(ingestionService.readIngestReport(tmpFileNames),
tmpFileNames,
loadedFiles, failedFiles);
//old files
List<String> oldFiles = new LinkedList<>();
tmpFileNames.forEach(
name ->
{
long time = FileNameUtils.fileNameToTimeIngested(name);
if (time < currentTime - ONE_HOUR)
{
failedFiles.add(name);
tmpFileNames.remove(name);
}
else if (time < currentTime - TEN_MINUTES)
{
oldFiles.add(name);
}
}
);
//load history
if (!oldFiles.isEmpty())
{
filterResult(ingestionService.readOneHourHistory(tmpFileNames,
currentTime - ONE_HOUR), tmpFileNames, loadedFiles, failedFiles);
}
updateOffset(tmpFileNames, loadedFiles, failedFiles);
purge(loadedFiles);
moveToTableStage(failedFiles);
fileListLock.lock();
try
{
fileNames.addAll(tmpFileNames);
} finally
{
fileListLock.unlock();
}
}
private void updateOffset(List<String> allFiles,
List<String> loadedFiles,
List<String> failedFiles)
{
if (allFiles.isEmpty())
{
if (loadedFiles.isEmpty() && failedFiles.isEmpty())
{
return;
}
long result = 0;
for (String name : loadedFiles)
{
long endOffset = FileNameUtils.fileNameToEndOffset(name) + 1;
if (endOffset > result)
{
result = endOffset;
}
}
for (String name : failedFiles)
{
long endOffset = FileNameUtils.fileNameToEndOffset(name) + 1;
if (endOffset > result)
{
result = endOffset;
}
}
committedOffset = result;
}
else
{
long result = Long.MAX_VALUE;
for (String name : allFiles)
{
long startOffset = FileNameUtils.fileNameToStartOffset(name);
if (startOffset < result)
{
result = startOffset;
}
}
committedOffset = result;
}
}
private void filterResult(Map<String, InternalUtils.IngestedFileStatus> fileStatus,
List<String> allFiles,
List<String> loadedFiles,
List<String> failedFiles)
{
fileStatus.forEach(
(name, status) ->
{
switch (status)
{
case LOADED:
loadedFiles.add(name);
allFiles.remove(name);
break;
case FAILED:
case PARTIALLY_LOADED:
failedFiles.add(name);
allFiles.remove(name);
break;
default:
//otherwise, do nothing
}
}
);
}
private void purge(List<String> files)
{
if (!files.isEmpty())
{
conn.purgeStage(stageName, files);
}
}
private void moveToTableStage(List<String> files)
{
if (!files.isEmpty())
{
conn.moveToTableStage(tableName, stageName, files);
telemetryService.reportKafkaFileFailure(tableName, stageName, files);
}
}
private void recover()
{
if (conn.pipeExist(pipeName))
{
if (!conn.isPipeCompatible(tableName, stageName, pipeName))
{
throw SnowflakeErrors.ERROR_5005.getException("pipe name: " + pipeName,
conn.getTelemetryClient());
}
fileListLock.lock();
try
{
recoverFileStatues().forEach(
(name, status) -> fileNames.add(name)
);
} finally
{
fileListLock.unlock();
}
logInfo("pipe {}, recovered from existing pipe", pipeName);
}
else
{
conn.createPipe(tableName, stageName, pipeName);
}
}
private Map<String, InternalUtils.IngestedFileStatus> recoverFileStatues()
{
List<String> files = conn.listStage(stageName, prefix);
if (files.isEmpty())
{
return new HashMap<>(); //no file on stage
}
Map<String, InternalUtils.IngestedFileStatus> result = new HashMap<>();
List<String> loadedFiles = new LinkedList<>();
List<String> failedFiles = new LinkedList<>();
//sort by time
//may be an issue when continuously recovering
// because this time is time when file uploaded.
// if files ingested again, this time will not be
// updated. So the real ingestion time maybe different
// in the second time recovery.
files.sort(Comparator.comparingLong(FileNameUtils::fileNameToTimeIngested));
long startTime = FileNameUtils.fileNameToTimeIngested(files.get(0));
committedOffset = Long.MAX_VALUE;
processedOffset = -1;
ingestionService.readOneHourHistory(files, startTime).forEach(
(name, status) ->
{
long startOffset = FileNameUtils.fileNameToStartOffset(name);
long endOffset = FileNameUtils.fileNameToEndOffset(name);
if (processedOffset < endOffset)
{
processedOffset = endOffset;
}
switch (status)
{
case NOT_FOUND:
//re ingest
ingestionService.ingestFile(name);
result.put(name, status);
if (committedOffset > startOffset)
{
committedOffset = startOffset;
}
break;
case LOAD_IN_PROGRESS:
result.put(name, status);
if (committedOffset > startOffset)
{
committedOffset = startOffset;
}
break;
case LOADED:
loadedFiles.add(name);
break;
default:
failedFiles.add(name);
}
}
);
if (!loadedFiles.isEmpty())
{
purge(loadedFiles);
}
if (!failedFiles.isEmpty())
{
moveToTableStage(failedFiles);
}
logInfo("pipe {} : Recovered {} files", pipeName, files.size());
return result;
}
private void close()
{
try
{
stopCleaner();
stopFlusher();
} catch (Exception e)
{
logWarn("Failed to terminate Cleaner or Flusher");
}
ingestionService.close();
sendUsageReport();
logInfo("pipe {}: service closed", pipeName);
}
/**
* SinkConnector ans SinkTasks start at the same time, however, SinkTasks
* need create table and wait SinkConnector to create stage.
* This method checks table and stage existence for at most 120 times(10
* min)
* And then throws exceptions if table or stage doesn't exit
*/
private void createTableAndStage()
{
//create table if not exists
if (conn.tableExist(tableName))
{
if (conn.isTableCompatible(tableName))
{
logInfo("Using existing table {}.", tableName);
telemetryService.reportKafkaReuseTable(tableName);
}
else
{
throw SnowflakeErrors.ERROR_5003.getException("table name: " + tableName, telemetryService);
}
}
else
{
logInfo("Creating new table {}.", tableName);
conn.createTable(tableName);
}
if (conn.stageExist(stageName))
{
if (conn.isStageCompatible(stageName))
{
logInfo("Using existing stage {}.", stageName);
telemetryService.reportKafkaReuseStage(stageName);
}
else
{
throw SnowflakeErrors.ERROR_5004.getException("stage name: " + stageName, telemetryService);
}
}
else
{
logInfo("Creating new stage {}.", stageName);
conn.createStage(stageName);
}
}
private void sendUsageReport()
{
usageDataLock.lock();
long numOfRecord;
long sizeOfData;
long start;
long end;
try
{
numOfRecord = totalNumberOfRecord;
this.totalNumberOfRecord = 0;
sizeOfData = totalSizeOfData;
this.totalSizeOfData = 0;
start = this.startTime;
this.startTime = System.currentTimeMillis();
end = this.startTime;
} finally
{
usageDataLock.unlock();
}
telemetryService.reportKafkaUsage(start, end, numOfRecord, sizeOfData);
}
private void updateUsageData(long numOfRecord, long sizeOfData)
{
usageDataLock.lock();
try
{
this.totalSizeOfData += sizeOfData;
this.totalNumberOfRecord += numOfRecord;
} finally
{
usageDataLock.unlock();
}
}
private class PartitionBuffer
{
private StringBuilder stringBuilder;
private int numOfRecord;
private int bufferSize;
private long firstOffset;
private long lastOffset;
private int getNumOfRecord()
{
return numOfRecord;
}
private int getBufferSize()
{
return bufferSize;
}
private long getFirstOffset()
{
return firstOffset;
}
private long getLastOffset()
{
return lastOffset;
}
private PartitionBuffer()
{
stringBuilder = new StringBuilder();
numOfRecord = 0;
bufferSize = 0;
firstOffset = -1;
lastOffset = -1;
}
private void insert(SinkRecord record)
{
String data = recordService.processRecord(record);
if (bufferSize == 0)
{
firstOffset = record.kafkaOffset();
}
stringBuilder.append(data);
numOfRecord++;
bufferSize += data.length() * 2; //1 char = 2 bytes
lastOffset = record.kafkaOffset();
}
private boolean isEmpty()
{
return numOfRecord == 0;
}
private String getData()
{
String result = stringBuilder.toString();
logDebug("flush buffer: {} records, {} bytes, offset {} - {}",
numOfRecord, bufferSize, firstOffset, lastOffset);
updateUsageData(this.numOfRecord, this.bufferSize);
return result;
}
}
}
}