Skip to content

Commit 1fc5fc6

Browse files
mieslepmsmygit
andauthored
fixing progress counter bug (#197)
* fixing progress counter bug; consolidating count printing logic to a single class; * need to reset unflushed counter after flushing * refactor to HashMap and change names for clearer code * implementing spacing suggestion * address review comments to consolidate the pritprogress function --------- Co-authored-by: Phil Miesle <[email protected]> Co-authored-by: Madhavan Sridharan <[email protected]>
1 parent 5f0222f commit 1fc5fc6

File tree

18 files changed

+385
-158
lines changed

18 files changed

+385
-158
lines changed

SIT/Makefile

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,8 @@ test_features: reset test_features_cmd
2121
test_features_cmd:
2222
./test.sh -p features
2323

24+
test: test_smoke test_regression_cmd test_features_cmd
25+
2426
# Local tests are not included in automated tests, but provide a means
2527
# to use the test harness to validate project-specific work
2628
test_local: reset test_local_cmd

SIT/cdm-assert.sh

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,6 @@
11
#!/bin/bash
22

3-
assertCmd="egrep 'JobSession.* Final ' \${OUTPUT_FILE} | sed 's/^.*Final //'"
3+
assertCmd="egrep 'JobCounter.* Final ' \${OUTPUT_FILE} | sed 's/^.*Final //'"
44

55
_usage() {
66
cat <<EOF
Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,4 @@
11
Read Record Count: 4
22
Valid Record Count: 10
3+
Skipped Record Count: 0
34
Large Record Count: 6

SIT/regression/03_performance/migrate.properties

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,3 +6,6 @@ spark.cdm.schema.target.keyspaceTable target.regression_performance
66

77
spark.cdm.perfops.numParts 32
88
spark.cdm.perfops.batchSize 1
9+
10+
spark.cdm.perfops.printStatsAfter 450
11+
spark.cdm.perfops.printStatsPerPart true

pom.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -326,7 +326,7 @@
326326
<counter>LINE</counter>
327327
<value>MISSEDCOUNT</value>
328328
<!-- <maximum>1544</maximum>-->
329-
<maximum>3052</maximum>
329+
<maximum>3073</maximum>
330330
</limit>
331331
</limits>
332332
</rule>

src/main/java/com/datastax/cdm/job/AbstractJobSession.java

Lines changed: 13 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,8 @@ public abstract class AbstractJobSession<T> extends BaseJobSession {
2020
protected Guardrail guardrailFeature;
2121
protected boolean guardrailEnabled;
2222
protected String partitionFile = SplitPartitions.getPartitionFile(propertyHelper);
23+
protected JobCounter jobCounter;
24+
protected Long printStatsAfter;
2325

2426
protected AbstractJobSession(CqlSession originSession, CqlSession targetSession, SparkConf sc) {
2527
this(originSession, targetSession, sc, false);
@@ -32,12 +34,13 @@ protected AbstractJobSession(CqlSession originSession, CqlSession targetSession,
3234
return;
3335
}
3436

35-
printStatsAfter = propertyHelper.getInteger(KnownProperties.PRINT_STATS_AFTER);
36-
if (!propertyHelper.meetsMinimum(KnownProperties.PRINT_STATS_AFTER, printStatsAfter, 1)) {
37+
this.printStatsAfter = propertyHelper.getLong(KnownProperties.PRINT_STATS_AFTER);
38+
if (!propertyHelper.meetsMinimum(KnownProperties.PRINT_STATS_AFTER, printStatsAfter, 1L)) {
3739
logger.warn(KnownProperties.PRINT_STATS_AFTER + " must be greater than 0. Setting to default value of " + KnownProperties.getDefaultAsString(KnownProperties.PRINT_STATS_AFTER));
3840
propertyHelper.setProperty(KnownProperties.PRINT_STATS_AFTER, KnownProperties.getDefault(KnownProperties.PRINT_STATS_AFTER));
39-
printStatsAfter = propertyHelper.getInteger(KnownProperties.PRINT_STATS_AFTER);
41+
printStatsAfter = propertyHelper.getLong(KnownProperties.PRINT_STATS_AFTER);
4042
}
43+
this.jobCounter = new JobCounter(printStatsAfter, propertyHelper.getBoolean(KnownProperties.PRINT_STATS_PER_PART));
4144

4245
rateLimiterOrigin = RateLimiter.create(propertyHelper.getInteger(KnownProperties.PERF_RATELIMIT_ORIGIN));
4346
rateLimiterTarget = RateLimiter.create(propertyHelper.getInteger(KnownProperties.PERF_RATELIMIT_TARGET));
@@ -77,5 +80,11 @@ protected AbstractJobSession(CqlSession originSession, CqlSession targetSession,
7780

7881
public abstract void processSlice(T slice);
7982

80-
public abstract void printCounts(boolean isFinal);
83+
public synchronized void printCounts(boolean isFinal) {
84+
if (isFinal) {
85+
jobCounter.printFinal();
86+
} else {
87+
jobCounter.printProgress();
88+
}
89+
}
8190
}

src/main/java/com/datastax/cdm/job/BaseJobSession.java

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@ public abstract class BaseJobSession {
3232
protected RateLimiter rateLimiterOrigin;
3333
protected RateLimiter rateLimiterTarget;
3434
protected Integer maxRetries = 10;
35-
protected Integer printStatsAfter = 100000;
3635

3736
protected BaseJobSession(SparkConf sc) {
3837
propertyHelper.initializeSparkConf(sc);

src/main/java/com/datastax/cdm/job/CopyJobSession.java

Lines changed: 19 additions & 48 deletions
Original file line numberDiff line numberDiff line change
@@ -17,25 +17,20 @@
1717
import java.util.ArrayList;
1818
import java.util.Collection;
1919
import java.util.concurrent.CompletionStage;
20-
import java.util.concurrent.atomic.AtomicLong;
2120

2221
public class CopyJobSession extends AbstractJobSession<SplitPartitions.Partition> {
2322

24-
private static CopyJobSession copyJobSession;
2523
private final PKFactory pkFactory;
2624
private final boolean isCounterTable;
2725
private final Integer fetchSize;
2826
private final Integer batchSize;
2927
public Logger logger = LoggerFactory.getLogger(this.getClass().getName());
30-
protected AtomicLong readCounter = new AtomicLong(0);
31-
protected AtomicLong skippedCounter = new AtomicLong(0);
32-
protected AtomicLong writeCounter = new AtomicLong(0);
33-
protected AtomicLong errorCounter = new AtomicLong(0);
3428
private TargetUpsertStatement targetUpsertStatement;
3529
private TargetSelectByPKStatement targetSelectByPKStatement;
3630

3731
protected CopyJobSession(CqlSession originSession, CqlSession targetSession, SparkConf sc) {
3832
super(originSession, targetSession, sc);
33+
this.jobCounter.setRegisteredTypes(JobCounter.CounterType.READ, JobCounter.CounterType.WRITE, JobCounter.CounterType.SKIPPED, JobCounter.CounterType.ERROR, JobCounter.CounterType.UNFLUSHED);
3934

4035
pkFactory = this.originSession.getPKFactory();
4136
isCounterTable = this.originSession.getCqlTable().isCounterTable();
@@ -60,11 +55,8 @@ public void getDataAndInsert(BigInteger min, BigInteger max) {
6055
int maxAttempts = maxRetries + 1;
6156
String guardrailCheck;
6257
for (int attempts = 1; attempts <= maxAttempts && !done; attempts++) {
63-
long readCnt = 0;
64-
long flushedWriteCnt = 0;
65-
long skipCnt = 0;
66-
long errCnt = 0;
67-
long unflushedWrites = 0;
58+
jobCounter.threadReset();
59+
6860
try {
6961
OriginSelectByPartitionRangeStatement originSelectByPartitionRangeStatement = this.originSession.getOriginSelectByPartitionRangeStatement();
7062
targetUpsertStatement = this.targetSession.getTargetUpsertStatement();
@@ -74,14 +66,11 @@ public void getDataAndInsert(BigInteger min, BigInteger max) {
7466

7567
for (Row originRow : resultSet) {
7668
rateLimiterOrigin.acquire(1);
77-
readCnt++;
78-
if (readCnt % printStatsAfter == 0) {
79-
printCounts(false);
80-
}
69+
jobCounter.threadIncrement(JobCounter.CounterType.READ);
8170

8271
Record record = new Record(pkFactory.getTargetPK(originRow), originRow, null);
8372
if (originSelectByPartitionRangeStatement.shouldFilterRecord(record)) {
84-
skipCnt++;
73+
jobCounter.threadIncrement(JobCounter.CounterType.SKIPPED);
8574
continue;
8675
}
8776

@@ -90,66 +79,48 @@ public void getDataAndInsert(BigInteger min, BigInteger max) {
9079
guardrailCheck = guardrailFeature.guardrailChecks(r);
9180
if (guardrailCheck != null && guardrailCheck != Guardrail.CLEAN_CHECK) {
9281
logger.error("Guardrails failed for PrimaryKey {}; {}", r.getPk(), guardrailCheck);
93-
skipCnt++;
82+
jobCounter.threadIncrement(JobCounter.CounterType.SKIPPED);
9483
continue;
9584
}
9685
}
9786

9887
BoundStatement boundUpsert = bind(r);
9988
if (null == boundUpsert) {
100-
skipCnt++; // TODO: this previously skipped, why not errCnt?
89+
jobCounter.threadIncrement(JobCounter.CounterType.SKIPPED); // TODO: this previously skipped, why not errCnt?
10190
continue;
10291
}
10392

10493
rateLimiterTarget.acquire(1);
10594
batch = writeAsync(batch, writeResults, boundUpsert);
106-
unflushedWrites++;
95+
jobCounter.threadIncrement(JobCounter.CounterType.UNFLUSHED);
10796

108-
if (unflushedWrites > fetchSize) {
97+
if (jobCounter.getCount(JobCounter.CounterType.UNFLUSHED) > fetchSize) {
10998
flushAndClearWrites(batch, writeResults);
110-
flushedWriteCnt += unflushedWrites;
111-
unflushedWrites = 0;
99+
jobCounter.threadIncrement(JobCounter.CounterType.WRITE, jobCounter.getCount(JobCounter.CounterType.UNFLUSHED));
100+
jobCounter.threadReset(JobCounter.CounterType.UNFLUSHED);
112101
}
113102
}
114103
}
115104

116105
flushAndClearWrites(batch, writeResults);
117-
flushedWriteCnt += unflushedWrites;
118-
119-
readCounter.addAndGet(readCnt);
120-
writeCounter.addAndGet(flushedWriteCnt);
121-
skippedCounter.addAndGet(skipCnt);
106+
jobCounter.threadIncrement(JobCounter.CounterType.WRITE, jobCounter.getCount(JobCounter.CounterType.UNFLUSHED));
107+
jobCounter.threadReset(JobCounter.CounterType.UNFLUSHED);
122108
done = true;
123109

124110
} catch (Exception e) {
125111
if (attempts == maxAttempts) {
126-
readCounter.addAndGet(readCnt);
127-
writeCounter.addAndGet(flushedWriteCnt);
128-
skippedCounter.addAndGet(skipCnt);
129-
errorCounter.addAndGet(readCnt - flushedWriteCnt - skipCnt);
112+
jobCounter.threadIncrement(JobCounter.CounterType.ERROR, jobCounter.getCount(JobCounter.CounterType.READ) - jobCounter.getCount(JobCounter.CounterType.WRITE) - jobCounter.getCount(JobCounter.CounterType.SKIPPED));
130113
logFailedPartitionsInFile(partitionFile, min, max);
131114
}
132115
logger.error("Error occurred during Attempt#: {}", attempts, e);
133116
logger.error("Error with PartitionRange -- ThreadID: {} Processing min: {} max: {} -- Attempt# {}",
134117
Thread.currentThread().getId(), min, max, attempts);
135-
logger.error("Error stats Read#: {}, Wrote#: {}, Skipped#: {}, Error#: {}", readCnt, flushedWriteCnt, skipCnt, (readCnt - flushedWriteCnt - skipCnt));
118+
logger.error("Error stats " + jobCounter.getThreadCounters(false));
119+
}
120+
finally {
121+
jobCounter.globalIncrement();
122+
printCounts(false);
136123
}
137-
}
138-
}
139-
140-
@Override
141-
public synchronized void printCounts(boolean isFinal) {
142-
String msg = "ThreadID: " + Thread.currentThread().getId();
143-
if (isFinal) {
144-
msg += " Final";
145-
logger.info("################################################################################################");
146-
}
147-
logger.info("{} Read Record Count: {}", msg, readCounter.get());
148-
logger.info("{} Skipped Record Count: {}", msg, skippedCounter.get());
149-
logger.info("{} Write Record Count: {}", msg, writeCounter.get());
150-
logger.info("{} Error Record Count: {}", msg, errorCounter.get());
151-
if (isFinal) {
152-
logger.info("################################################################################################");
153124
}
154125
}
155126

src/main/java/com/datastax/cdm/job/CopyPKJobSession.java

Lines changed: 9 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -15,23 +15,18 @@
1515
import java.beans.PropertyEditorManager;
1616
import java.util.ArrayList;
1717
import java.util.List;
18-
import java.util.concurrent.atomic.AtomicLong;
1918

2019
public class CopyPKJobSession extends AbstractJobSession<SplitPartitions.PKRows> {
2120

22-
private static CopyPKJobSession copyJobSession;
2321
private final PKFactory pkFactory;
2422
private final List<Class> originPKClasses;
2523
private final boolean isCounterTable;
2624
public Logger logger = LoggerFactory.getLogger(this.getClass().getName());
27-
protected AtomicLong readCounter = new AtomicLong(0);
28-
protected AtomicLong missingCounter = new AtomicLong(0);
29-
protected AtomicLong skipCounter = new AtomicLong(0);
30-
protected AtomicLong writeCounter = new AtomicLong(0);
3125
private OriginSelectByPKStatement originSelectByPKStatement;
3226

3327
protected CopyPKJobSession(CqlSession originSession, CqlSession targetSession, SparkConf sc) {
3428
super(originSession, targetSession, sc, true);
29+
this.jobCounter.setRegisteredTypes(JobCounter.CounterType.READ, JobCounter.CounterType.WRITE, JobCounter.CounterType.SKIPPED, JobCounter.CounterType.MISSING);
3530
pkFactory = this.originSession.getPKFactory();
3631
isCounterTable = this.originSession.getCqlTable().isCounterTable();
3732
originPKClasses = this.originSession.getCqlTable().getPKClasses();
@@ -47,64 +42,49 @@ public void processSlice(SplitPartitions.PKRows slice) {
4742
public void getRowAndInsert(SplitPartitions.PKRows rowsList) {
4843
originSelectByPKStatement = originSession.getOriginSelectByPKStatement();
4944
for (String row : rowsList.getPkRows()) {
50-
readCounter.incrementAndGet();
45+
jobCounter.threadIncrement(JobCounter.CounterType.READ);
5146
EnhancedPK pk = toEnhancedPK(row);
5247
if (null == pk || pk.isError()) {
53-
missingCounter.incrementAndGet();
48+
jobCounter.threadIncrement(JobCounter.CounterType.MISSING);
5449
logger.error("Could not build PK object with value <{}>; error is: {}", row, (null == pk ? "null" : pk.getMessages()));
5550
return;
5651
}
5752

5853
rateLimiterOrigin.acquire(1);
5954
Record recordFromOrigin = originSelectByPKStatement.getRecord(pk);
6055
if (null == recordFromOrigin) {
61-
missingCounter.incrementAndGet();
56+
jobCounter.threadIncrement(JobCounter.CounterType.MISSING);
6257
logger.error("Could not find origin row with primary-key: {}", row);
6358
return;
6459
}
6560
Row originRow = recordFromOrigin.getOriginRow();
6661

6762
Record record = new Record(pkFactory.getTargetPK(originRow), originRow, null);
6863
if (originSelectByPKStatement.shouldFilterRecord(record)) {
69-
skipCounter.incrementAndGet();
64+
jobCounter.threadIncrement(JobCounter.CounterType.SKIPPED);
7065
return;
7166
}
7267

7368
if (guardrailEnabled) {
7469
String guardrailCheck = guardrailFeature.guardrailChecks(record);
7570
if (guardrailCheck != null && guardrailCheck != Guardrail.CLEAN_CHECK) {
7671
logger.error("Guardrails failed for PrimaryKey {}; {}", record.getPk(), guardrailCheck);
77-
skipCounter.incrementAndGet();
72+
jobCounter.threadIncrement(JobCounter.CounterType.SKIPPED);
7873
return;
7974
}
8075
}
8176

8277
rateLimiterTarget.acquire(1);
8378
targetSession.getTargetUpsertStatement().putRecord(record);
84-
writeCounter.incrementAndGet();
79+
jobCounter.threadIncrement(JobCounter.CounterType.WRITE);
8580

86-
if (readCounter.get() % printStatsAfter == 0) {
87-
printCounts(false);
88-
}
81+
jobCounter.globalIncrement();
82+
printCounts(false);
8983
}
9084

9185
printCounts(true);
9286
}
9387

94-
@Override
95-
public void printCounts(boolean isFinal) {
96-
if (isFinal) {
97-
logger.info("################################################################################################");
98-
}
99-
logger.info("ThreadID: {} Read Record Count: {}", Thread.currentThread().getId(), readCounter.get());
100-
logger.info("ThreadID: {} Missing Record Count: {}", Thread.currentThread().getId(), missingCounter.get());
101-
logger.info("ThreadID: {} Skipped Record Count: {}", Thread.currentThread().getId(), skipCounter.get());
102-
logger.info("ThreadID: {} Inserted Record Count: {}", Thread.currentThread().getId(), writeCounter.get());
103-
if (isFinal) {
104-
logger.info("################################################################################################");
105-
}
106-
}
107-
10888
private EnhancedPK toEnhancedPK(String rowString) {
10989
String[] pkFields = rowString.split(" %% ");
11090
List<Object> values = new ArrayList<>(originPKClasses.size());

0 commit comments

Comments
 (0)