Skip to content

Commit 6fd0a83

Browse files
authored
Merge pull request #261 from guofei/output-partition-option
Add options to output partition range
2 parents ed0d750 + b8e7be0 commit 6fd0a83

File tree

13 files changed

+142
-36
lines changed

13 files changed

+142
-36
lines changed

README.md

Lines changed: 23 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -96,14 +96,34 @@ Each line above represents a partition-range (`min,max`). Alternatively, you can
9696
```
9797
./spark-submit --properties-file cdm.properties \
9898
--conf spark.cdm.schema.origin.keyspaceTable="<keyspacename>.<tablename>" \
99-
--conf spark.cdm.tokenrange.partitionFile="/<path-to-file>/<csv-input-filename>" \
99+
--conf spark.cdm.tokenrange.partitionFile.input="/<path-to-file>/<csv-input-filename>" \
100100
--master "local[*]" --driver-memory 25G --executor-memory 25G \
101101
--class com.datastax.cdm.job.<Migrate|DiffData> cassandra-data-migrator-4.x.x.jar &> logfile_name_$(date +%Y%m%d_%H_%M).txt
102102
```
103103
This mode is specifically useful to processes a subset of partition-ranges that may have failed during a previous run.
104104

105-
> **Note:**
106-
> A file named `./<keyspacename>.<tablename>_partitions.csv` is auto generated by the Migration & Validation jobs in the above format containing any failed partition ranges. No file is created if there are no failed partitions. You can use this file as an input to process any failed partition in a following run.
105+
A file named `./<keyspacename>.<tablename>_partitions.csv` is auto-generated by the Migration & Validation jobs in the above format containing any failed partition ranges. No file is created if there are no failed partitions. This file can be used as an input to process any failed partition in a following run. You can also specify a different output file using the `spark.cdm.tokenrange.partitionFile.output` option.
106+
```
107+
./spark-submit --properties-file cdm.properties \
108+
--conf spark.cdm.schema.origin.keyspaceTable="<keyspacename>.<tablename>" \
109+
--conf spark.cdm.tokenrange.partitionFile.input="/<path-to-file>/<csv-input-filename>" \
110+
--conf spark.cdm.tokenrange.partitionFile.output="/<path-to-file>/<csv-output-filename>" \
111+
--master "local[*]" --driver-memory 25G --executor-memory 25G \
112+
--class com.datastax.cdm.job.<Migrate|DiffData> cassandra-data-migrator-4.x.x.jar &> logfile_name_$(date +%Y%m%d_%H_%M).txt
113+
```
114+
115+
For the Data-Validation step, use the conf option `-conf spark.cdm.tokenrange.partitionFile.appendOnDiff` as shown below. This allows the partition range to be outputted whenever there are differences, not just fails.
116+
```
117+
./spark-submit --properties-file cdm.properties \
118+
--conf spark.cdm.schema.origin.keyspaceTable="<keyspacename>.<tablename>" \
119+
--conf spark.cdm.tokenrange.partitionFile.input="/<path-to-file>/<csv-input-filename>" \
120+
--conf spark.cdm.tokenrange.partitionFile.output="/<path-to-file>/<csv-output-filename>" \
121+
--conf spark.cdm.tokenrange.partitionFile.appendOnDiff=true \
122+
--master "local[*]" --driver-memory 25G --executor-memory 25G \
123+
--class com.datastax.cdm.job.<Migrate|DiffData> cassandra-data-migrator-4.x.x.jar &> logfile_name_$(date +%Y%m%d_%H_%M).txt
124+
```
125+
126+
If `spark.cdm.tokenrange.partitionFile.input` or `spark.cdm.tokenrange.partitionFile.output` are not specified, the system will use `./<keyspacename>.<tablename>_partitions.csv` as the default file.
107127

108128
# Perform large-field Guardrail violation checks
109129
- The tool can be used to identify large fields from a table that may break you cluster guardrails (e.g. AstraDB has a 10MB limit for a single large field) `--class com.datastax.cdm.job.GuardrailCheck` as shown below

SIT/features/06_partition_range/migrate_with_partitionfile.properties

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,5 +18,6 @@ spark.cdm.perfops.numParts 1
1818
spark.cdm.autocorrect.missing true
1919
spark.cdm.autocorrect.mismatch true
2020

21-
spark.cdm.tokenrange.partitionFile ./partitions.csv
21+
spark.cdm.tokenrange.partitionFile.input ./partitions.csv
22+
spark.cdm.tokenrange.partitionFile.output ./partitions.csv
2223

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

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,9 @@ public abstract class AbstractJobSession<T> extends BaseJobSession {
3434
protected EnhancedSession targetSession;
3535
protected Guardrail guardrailFeature;
3636
protected boolean guardrailEnabled;
37-
protected String partitionFile = SplitPartitions.getPartitionFile(propertyHelper);
37+
protected boolean appendPartitionOnDiff = SplitPartitions.appendPartitionOnDiff(propertyHelper);
38+
protected String partitionFileInput = SplitPartitions.getPartitionFileInput(propertyHelper);
39+
protected String partitionFileOutput = SplitPartitions.getPartitionFileOutput(propertyHelper);
3840
protected JobCounter jobCounter;
3941
protected Long printStatsAfter;
4042

@@ -62,7 +64,8 @@ protected AbstractJobSession(CqlSession originSession, CqlSession targetSession,
6264
maxRetries = propertyHelper.getInteger(KnownProperties.MAX_RETRIES);
6365

6466
logger.info("PARAM -- Max Retries: {}", maxRetries);
65-
logger.info("PARAM -- Partition file: {}", partitionFile);
67+
logger.info("PARAM -- Partition file input: {}", partitionFileInput);
68+
logger.info("PARAM -- Partition file output: {}", partitionFileOutput);
6669
logger.info("PARAM -- Origin Rate Limit: {}", rateLimiterOrigin.getRate());
6770
logger.info("PARAM -- Target Rate Limit: {}", rateLimiterTarget.getRate());
6871

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,7 @@ private void appendToFile(String filePath, String content) throws IOException {
9999
StandardOpenOption.APPEND);
100100
}
101101

102-
protected void logFailedPartitionsInFile(String partitionFile, BigInteger min, BigInteger max) {
102+
protected void logPartitionsInFile(String partitionFile, BigInteger min, BigInteger max) {
103103
try {
104104
appendToFile(partitionFile, min + "," + max);
105105
} catch (Exception ee) {

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -125,7 +125,7 @@ public void getDataAndInsert(BigInteger min, BigInteger max) {
125125
} catch (Exception e) {
126126
if (attempts == maxAttempts) {
127127
jobCounter.threadIncrement(JobCounter.CounterType.ERROR, jobCounter.getCount(JobCounter.CounterType.READ) - jobCounter.getCount(JobCounter.CounterType.WRITE) - jobCounter.getCount(JobCounter.CounterType.SKIPPED));
128-
logFailedPartitionsInFile(partitionFile, min, max);
128+
logPartitionsInFile(partitionFileOutput, min, max);
129129
}
130130
logger.error("Error occurred during Attempt#: {}", attempts, e);
131131
logger.error("Error with PartitionRange -- ThreadID: {} Processing min: {} max: {} -- Attempt# {}",

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

Lines changed: 25 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,7 @@
4242
import java.util.Collections;
4343
import java.util.List;
4444
import java.util.concurrent.CompletionStage;
45+
import java.util.concurrent.atomic.AtomicBoolean;
4546
import java.util.stream.Collectors;
4647
import java.util.stream.IntStream;
4748
import java.util.stream.StreamSupport;
@@ -112,6 +113,7 @@ public void getDataAndDiff(BigInteger min, BigInteger max) {
112113
ThreadContext.put(THREAD_CONTEXT_LABEL, getThreadLabel(min, max));
113114
logger.info("ThreadID: {} Processing min: {} max: {}", Thread.currentThread().getId(), min, max);
114115
boolean done = false;
116+
AtomicBoolean hasDiff = new AtomicBoolean(false);
115117
int maxAttempts = maxRetries + 1;
116118
for (int attempts = 1; attempts <= maxAttempts && !done; attempts++) {
117119
try {
@@ -152,19 +154,27 @@ public void getDataAndDiff(BigInteger min, BigInteger max) {
152154
r.setAsyncTargetRow(targetResult);
153155
recordsToDiff.add(r);
154156
if (recordsToDiff.size() > fetchSizeInRows) {
155-
diffAndClear(recordsToDiff);
157+
if(diffAndClear(recordsToDiff)) {
158+
hasDiff.set(true);
159+
}
156160
}
157161
} // targetRecord!=null
158162
} // recordSet iterator
159163
} // shouldFilterRecord
160164
});
161-
diffAndClear(recordsToDiff);
165+
if (diffAndClear(recordsToDiff)) {
166+
hasDiff.set(true);
167+
}
162168
done = true;
169+
170+
if (hasDiff.get() && appendPartitionOnDiff) {
171+
logPartitionsInFile(partitionFileOutput, min, max);
172+
}
163173
} catch (Exception e) {
164174
logger.error("Error with PartitionRange -- ThreadID: {} Processing min: {} max: {} -- Attempt# {}",
165175
Thread.currentThread().getId(), min, max, attempts, e);
166176
if (attempts == maxAttempts) {
167-
logFailedPartitionsInFile(partitionFile, min, max);
177+
logPartitionsInFile(partitionFileOutput, min, max);
168178
}
169179
} finally {
170180
jobCounter.globalIncrement();
@@ -173,18 +183,22 @@ public void getDataAndDiff(BigInteger min, BigInteger max) {
173183
}
174184
}
175185

176-
private void diffAndClear(List<Record> recordsToDiff) {
186+
private boolean diffAndClear(List<Record> recordsToDiff) {
187+
boolean isDiff = false;
177188
for (Record record : recordsToDiff) {
178189
try {
179-
diff(record);
190+
if (diff(record)) {
191+
isDiff = true;
192+
}
180193
} catch (Exception e) {
181194
logger.error("Could not perform diff for key {}: {}", record.getPk(), e);
182195
}
183196
}
184197
recordsToDiff.clear();
198+
return isDiff;
185199
}
186200

187-
private void diff(Record record) {
201+
private boolean diff(Record record) {
188202
EnhancedPK originPK = record.getPk();
189203
Row originRow = record.getOriginRow();
190204
Row targetRow = record.getTargetRow();
@@ -194,7 +208,7 @@ private void diff(Record record) {
194208
logger.error("Missing target row found for key: {}", record.getPk());
195209
if (autoCorrectMissing && isCounterTable && !forceCounterWhenMissing) {
196210
logger.error("{} is true, but not Inserting as {} is not enabled; key : {}", KnownProperties.AUTOCORRECT_MISSING, KnownProperties.AUTOCORRECT_MISSING_COUNTER, record.getPk());
197-
return;
211+
return true;
198212
}
199213

200214
//correct data
@@ -204,7 +218,7 @@ private void diff(Record record) {
204218
jobCounter.threadIncrement(JobCounter.CounterType.CORRECTED_MISSING);
205219
logger.error("Inserted missing row in target: {}", record.getPk());
206220
}
207-
return;
221+
return true;
208222
}
209223

210224
String diffData = isDifferent(originPK, originRow, targetRow);
@@ -218,8 +232,11 @@ private void diff(Record record) {
218232
jobCounter.threadIncrement(JobCounter.CounterType.CORRECTED_MISMATCH);
219233
logger.error("Corrected mismatch row in target: {}", record.getPk());
220234
}
235+
236+
return true;
221237
} else {
222238
jobCounter.threadIncrement(JobCounter.CounterType.VALID);
239+
return false;
223240
}
224241
}
225242

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

Lines changed: 16 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -168,13 +168,24 @@ private static BufferedReader getfileReader(String fileName) {
168168
}
169169
}
170170

171-
public static String getPartitionFile(PropertyHelper propertyHelper) {
172-
String filePath = propertyHelper.getString(KnownProperties.TOKEN_RANGE_PARTITION_FILE);
173-
if (StringUtils.isAllBlank(filePath)) {
174-
filePath = "./" + propertyHelper.getString(KnownProperties.ORIGIN_KEYSPACE_TABLE) + "_partitions.csv";
171+
public static boolean appendPartitionOnDiff(PropertyHelper propertyHelper) {
172+
return Boolean.TRUE.equals(propertyHelper.getBoolean(KnownProperties.TOKEN_RANGE_PARTITION_FILE_APPEND_ON_DIFF));
173+
}
174+
175+
public static String getPartitionFileInput(PropertyHelper propertyHelper) {
176+
if (!StringUtils.isAllBlank(propertyHelper.getString(KnownProperties.TOKEN_RANGE_PARTITION_FILE_INPUT))) {
177+
return propertyHelper.getString(KnownProperties.TOKEN_RANGE_PARTITION_FILE_INPUT);
178+
}
179+
180+
return "./" + propertyHelper.getString(KnownProperties.ORIGIN_KEYSPACE_TABLE) + "_partitions.csv";
181+
}
182+
183+
public static String getPartitionFileOutput(PropertyHelper propertyHelper) {
184+
if (!StringUtils.isAllBlank(propertyHelper.getString(KnownProperties.TOKEN_RANGE_PARTITION_FILE_OUTPUT))) {
185+
return propertyHelper.getString(KnownProperties.TOKEN_RANGE_PARTITION_FILE_OUTPUT);
175186
}
176187

177-
return filePath;
188+
return "./" + propertyHelper.getString(KnownProperties.ORIGIN_KEYSPACE_TABLE) + "_partitions.csv";
178189
}
179190

180191
public static class PKRows implements Serializable {

src/main/java/com/datastax/cdm/properties/KnownProperties.java

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -152,12 +152,17 @@ public enum PropertyType {
152152
}
153153

154154
//==========================================================================
155-
// Error handling
155+
// Partition File
156156
//==========================================================================
157-
public static final String TOKEN_RANGE_PARTITION_FILE = "spark.cdm.tokenrange.partitionFile";
157+
public static final String TOKEN_RANGE_PARTITION_FILE_APPEND_ON_DIFF = "spark.cdm.tokenrange.partitionFile.appendOnDiff";
158+
public static final String TOKEN_RANGE_PARTITION_FILE_INPUT = "spark.cdm.tokenrange.partitionFile.input";
159+
public static final String TOKEN_RANGE_PARTITION_FILE_OUTPUT = "spark.cdm.tokenrange.partitionFile.output";
158160
static {
159-
types.put(TOKEN_RANGE_PARTITION_FILE, PropertyType.STRING);
161+
types.put(TOKEN_RANGE_PARTITION_FILE_APPEND_ON_DIFF, PropertyType.BOOLEAN);
162+
types.put(TOKEN_RANGE_PARTITION_FILE_INPUT, PropertyType.STRING);
163+
types.put(TOKEN_RANGE_PARTITION_FILE_OUTPUT, PropertyType.STRING);
160164
}
165+
161166
//==========================================================================
162167
// Guardrails and Transformations
163168
//==========================================================================

src/main/scala/com/datastax/cdm/job/BaseJob.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
*/
1616
package com.datastax.cdm.job
1717

18-
import com.datastax.cdm.job.SplitPartitions.getPartitionFile
18+
import com.datastax.cdm.job.SplitPartitions.getPartitionFileInput
1919
import com.datastax.cdm.properties.{KnownProperties, PropertyHelper}
2020
import com.datastax.spark.connector.cql.CassandraConnector
2121
import org.apache.spark.{SparkConf, SparkContext}
@@ -53,7 +53,7 @@ abstract class BaseJob[T: ClassTag] extends App {
5353

5454
var originConnection: CassandraConnector = _
5555
var targetConnection: CassandraConnector = _
56-
var partitionFileName: String = ""
56+
var partitionFileNameInput: String = ""
5757

5858
def setup(jobName: String, jobFactory: IJobSessionFactory[T]): Unit = {
5959
logBanner(jobName + " - Starting")
@@ -66,7 +66,7 @@ abstract class BaseJob[T: ClassTag] extends App {
6666
sContext = spark.sparkContext
6767
sc = sContext.getConf
6868
propertyHelper = PropertyHelper.getInstance(sc);
69-
this.partitionFileName = getPartitionFile(propertyHelper);
69+
this.partitionFileNameInput = getPartitionFileInput(propertyHelper);
7070

7171
consistencyLevel = propertyHelper.getString(KnownProperties.READ_CL)
7272
val connectionFetcher = new ConnectionFetcher(sContext, propertyHelper)

src/main/scala/com/datastax/cdm/job/BasePKJob.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,6 @@ abstract class BasePKJob extends BaseJob[SplitPartitions.PKRows] {
2121
override def getParts(pieces: Int): util.Collection[SplitPartitions.PKRows] = {
2222
// This takes a file with N rows and divides it into pieces of size N/pieces
2323
// Each PKRows object contains a list of Strings that contain the PK to be parsed
24-
SplitPartitions.getRowPartsFromFile(pieces, this.partitionFileName)
24+
SplitPartitions.getRowPartsFromFile(pieces, this.partitionFileNameInput)
2525
}
2626
}

0 commit comments

Comments
 (0)