Skip to content

Commit baaf92c

Browse files
authored
Merge pull request #38 from datastax/feature/frametoolongexception
Added capability to customize page-size (fixes FrameTooLongException)
2 parents 869b8fc + 02e6061 commit baaf92c

File tree

9 files changed

+41
-33
lines changed

9 files changed

+41
-33
lines changed

pom.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,7 @@
33

44
<groupId>datastax.astra.migrate</groupId>
55
<artifactId>cassandra-data-migrator</artifactId>
6-
<version>2.7</version>
6+
<version>2.8</version>
77
<packaging>jar</packaging>
88

99
<properties>

src/main/java/datastax/astra/migrate/AbstractJobSession.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
import org.slf4j.Logger;
1111
import org.slf4j.LoggerFactory;
1212

13+
import java.time.Duration;
1314
import java.util.Map;
1415
import java.util.stream.IntStream;
1516

@@ -26,6 +27,7 @@ protected AbstractJobSession(CqlSession sourceSession, CqlSession astraSession,
2627
this.astraSession = astraSession;
2728

2829
batchSize = new Integer(Util.getSparkPropOr(sc, "spark.batchSize", "1"));
30+
fetchSizeInRows = new Integer(Util.getSparkPropOr(sc, "spark.read.fetch.sizeInRows", "1000"));
2931
printStatsAfter = new Integer(Util.getSparkPropOr(sc, "spark.printStatsAfter", "100000"));
3032
if (printStatsAfter < 1) {
3133
printStatsAfter = 100000;
@@ -77,6 +79,7 @@ protected AbstractJobSession(CqlSession sourceSession, CqlSession astraSession,
7779
}
7880

7981
logger.info("PARAM -- Write Batch Size: {}", batchSize);
82+
logger.info("PARAM -- Read Fetch Size: {}", fetchSizeInRows);
8083
logger.info("PARAM -- Source Keyspace Table: {}", sourceKeyspaceTable);
8184
logger.info("PARAM -- Destination Keyspace Table: {}", astraKeyspaceTable);
8285
logger.info("PARAM -- ReadRateLimit: {}", readLimiter.getRate());
@@ -207,7 +210,8 @@ public BoundStatement bindInsert(PreparedStatement insertStatement, Row sourceRo
207210
}
208211
}
209212

210-
return boundInsertStatement;
213+
// Batch insert for large records may take longer, hence 10 secs to avoid timeout errors
214+
return boundInsertStatement.setTimeout(Duration.ofSeconds(10));
211215
}
212216

213217
public int getLargestTTL(Row sourceRow) {

src/main/java/datastax/astra/migrate/BaseJobSession.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ public abstract class BaseJobSession {
3333
protected List<Integer> updateSelectMapping = new ArrayList<Integer>();
3434

3535
protected Integer batchSize = 1;
36+
protected Integer fetchSizeInRows = 1000;
3637
protected Integer printStatsAfter = 100000;
3738

3839
protected Boolean writeTimeStampFilter = Boolean.FALSE;
@@ -67,6 +68,7 @@ public String getKey(Row sourceRow) {
6768

6869
return key.toString();
6970
}
71+
7072
public List<MigrateDataType> getTypes(String types) {
7173
List<MigrateDataType> dataTypes = new ArrayList<MigrateDataType>();
7274
for (String type : types.split(",")) {

src/main/java/datastax/astra/migrate/CopyJobSession.java

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,8 @@ public void getDataAndInsert(BigInteger min, BigInteger max) {
4747
for (int retryCount = 1; retryCount <= maxAttempts; retryCount++) {
4848

4949
try {
50-
ResultSet resultSet = sourceSession.execute(sourceSelectStatement.bind(hasRandomPartitioner ? min : min.longValueExact(), hasRandomPartitioner ? max : max.longValueExact()));
50+
ResultSet resultSet = sourceSession.execute(sourceSelectStatement.bind(hasRandomPartitioner ? min : min.longValueExact(),
51+
hasRandomPartitioner ? max : max.longValueExact()).setPageSize(fetchSizeInRows));
5152
Collection<CompletionStage<AsyncResultSet>> writeResults = new ArrayList<CompletionStage<AsyncResultSet>>();
5253

5354
// cannot do batching if the writeFilter is greater than 0 or
@@ -87,15 +88,16 @@ public void getDataAndInsert(BigInteger min, BigInteger max) {
8788
astraRow = astraReadResultSet.one();
8889
}
8990

91+
9092
CompletionStage<AsyncResultSet> astraWriteResultSet = astraSession
9193
.executeAsync(bindInsert(astraInsertStatement, sourceRow, astraRow));
9294
writeResults.add(astraWriteResultSet);
93-
if (writeResults.size() > 1000) {
95+
if (writeResults.size() > fetchSizeInRows) {
9496
iterateAndClearWriteResults(writeResults, 1);
9597
}
9698
}
9799

98-
// clear the write resultset in-case it didnt mod at 1000 above
100+
// clear the write resultset
99101
iterateAndClearWriteResults(writeResults, 1);
100102
} else {
101103
BatchStatement batchStatement = BatchStatement.newInstance(BatchType.UNLOGGED);
@@ -124,12 +126,12 @@ public void getDataAndInsert(BigInteger min, BigInteger max) {
124126
batchStatement = BatchStatement.newInstance(BatchType.UNLOGGED);
125127
}
126128

127-
if (writeResults.size() * batchSize > 1000) {
129+
if (writeResults.size() * batchSize > fetchSizeInRows) {
128130
iterateAndClearWriteResults(writeResults, batchSize);
129131
}
130132
}
131133

132-
// clear the write resultset in-case it didnt mod at 1000 above
134+
// clear the write resultset
133135
iterateAndClearWriteResults(writeResults, batchSize);
134136

135137
// if there are any pending writes because the batchSize threshold was not met, then write and clear them
@@ -173,4 +175,4 @@ private void iterateAndClearWriteResults(Collection<CompletionStage<AsyncResultS
173175
writeResults.clear();
174176
}
175177

176-
}
178+
}

src/main/java/datastax/astra/migrate/DiffJobSession.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,8 @@ public void getDataAndDiff(BigInteger min, BigInteger max) {
6262
try {
6363
// cannot do batching if the writeFilter is greater than 0
6464
ResultSet resultSet = sourceSession.execute(
65-
sourceSelectStatement.bind(hasRandomPartitioner ? min : min.longValueExact(), hasRandomPartitioner ? max : max.longValueExact()).setConsistencyLevel(ConsistencyLevel.LOCAL_QUORUM));
65+
sourceSelectStatement.bind(hasRandomPartitioner ? min : min.longValueExact(), hasRandomPartitioner ? max : max.longValueExact())
66+
.setConsistencyLevel(ConsistencyLevel.LOCAL_QUORUM).setPageSize(fetchSizeInRows));
6667

6768
Map<Row, CompletionStage<AsyncResultSet>> srcToTargetRowMap = new HashMap<Row, CompletionStage<AsyncResultSet>>();
6869
StreamSupport.stream(resultSet.spliterator(), false).forEach(srcRow -> {
@@ -77,7 +78,7 @@ public void getDataAndDiff(BigInteger min, BigInteger max) {
7778
CompletionStage<AsyncResultSet> targetRowFuture = astraSession
7879
.executeAsync(selectFromAstra(astraSelectStatement, srcRow));
7980
srcToTargetRowMap.put(srcRow, targetRowFuture);
80-
if (srcToTargetRowMap.size() > 1000) {
81+
if (srcToTargetRowMap.size() > fetchSizeInRows) {
8182
diffAndClear(srcToTargetRowMap);
8283
}
8384
} else {

src/main/scala/datastax/astra/migrate/AbstractJob.scala

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -10,13 +10,13 @@ class AbstractJob extends BaseJob {
1010
abstractLogger.info("PARAM -- Split Size: " + splitSize)
1111
abstractLogger.info("PARAM -- Coverage Percent: " + coveragePercent)
1212

13-
var sourceConnection = getConnection(true, sourceIsAstra, sourceScbPath, sourceHost, sourceUsername, sourcePassword, sourceReadConsistencyLevel,
13+
var sourceConnection = getConnection(true, sourceIsAstra, sourceScbPath, sourceHost, sourceUsername, sourcePassword,
1414
sourceTrustStorePath, sourceTrustStorePassword, sourceTrustStoreType, sourceKeyStorePath, sourceKeyStorePassword, sourceEnabledAlgorithms);
1515

16-
var destinationConnection = getConnection(false, destinationIsAstra, destinationScbPath, destinationHost, destinationUsername, destinationPassword, destinationReadConsistencyLevel,
16+
var destinationConnection = getConnection(false, destinationIsAstra, destinationScbPath, destinationHost, destinationUsername, destinationPassword,
1717
destinationTrustStorePath, destinationTrustStorePassword, destinationTrustStoreType, destinationKeyStorePath, destinationKeyStorePassword, destinationEnabledAlgorithms);
1818

19-
private def getConnection(isSource: Boolean, isAstra: String, scbPath: String, host: String, username: String, password: String, readConsistencyLevel: String,
19+
private def getConnection(isSource: Boolean, isAstra: String, scbPath: String, host: String, username: String, password: String,
2020
trustStorePath: String, trustStorePassword: String, trustStoreType: String,
2121
keyStorePath: String, keyStorePassword: String, enabledAlgorithms: String): CassandraConnector = {
2222
var connType: String = "Source"
@@ -31,7 +31,7 @@ class AbstractJob extends BaseJob {
3131
return CassandraConnector(config
3232
.set("spark.cassandra.auth.username", username)
3333
.set("spark.cassandra.auth.password", password)
34-
.set("spark.cassandra.input.consistency.level", readConsistencyLevel)
34+
.set("spark.cassandra.input.consistency.level", consistencyLevel)
3535
.set("spark.cassandra.connection.config.cloud.path", scbPath))
3636
} else if (null != trustStorePath && !trustStorePath.trim.isEmpty) {
3737
abstractLogger.info(connType + ": Connecting to Cassandra (or DSE) with SSL host: " + host);
@@ -45,7 +45,7 @@ class AbstractJob extends BaseJob {
4545
return CassandraConnector(config
4646
.set("spark.cassandra.auth.username", username)
4747
.set("spark.cassandra.auth.password", password)
48-
.set("spark.cassandra.input.consistency.level", readConsistencyLevel)
48+
.set("spark.cassandra.input.consistency.level", consistencyLevel)
4949
.set("spark.cassandra.connection.host", host)
5050
.set("spark.cassandra.connection.ssl.enabled", "true")
5151
.set("spark.cassandra.connection.ssl.enabledAlgorithms", enabledAlgorithmsVar)
@@ -61,7 +61,7 @@ class AbstractJob extends BaseJob {
6161

6262
return CassandraConnector(config.set("spark.cassandra.auth.username", username)
6363
.set("spark.cassandra.auth.password", password)
64-
.set("spark.cassandra.input.consistency.level", readConsistencyLevel)
64+
.set("spark.cassandra.input.consistency.level", consistencyLevel)
6565
.set("spark.cassandra.connection.host", host))
6666
}
6767

src/main/scala/datastax/astra/migrate/BaseJob.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,13 @@ class BaseJob extends App {
1818
val sContext = spark.sparkContext
1919
val sc = sContext.getConf
2020

21+
val consistencyLevel = Util.getSparkPropOr(sc, "spark.read.consistency.level", "LOCAL_QUORUM")
22+
2123
val sourceIsAstra = Util.getSparkPropOr(sc, "spark.origin.isAstra", "false")
2224
val sourceScbPath = Util.getSparkPropOrEmpty(sc, "spark.origin.scb")
2325
val sourceHost = Util.getSparkPropOrEmpty(sc, "spark.origin.host")
2426
val sourceUsername = Util.getSparkPropOrEmpty(sc, "spark.origin.username")
2527
val sourcePassword = Util.getSparkPropOrEmpty(sc, "spark.origin.password")
26-
val sourceReadConsistencyLevel = Util.getSparkPropOr(sc, "spark.origin.read.consistency.level", "LOCAL_QUORUM")
2728
val sourceTrustStorePath = Util.getSparkPropOrEmpty(sc, "spark.origin.trustStore.path")
2829
val sourceTrustStorePassword = Util.getSparkPropOrEmpty(sc, "spark.origin.trustStore.password")
2930
val sourceTrustStoreType = Util.getSparkPropOr(sc, "spark.origin.trustStore.type", "JKS")
@@ -36,7 +37,6 @@ class BaseJob extends App {
3637
val destinationHost = Util.getSparkPropOrEmpty(sc, "spark.target.host")
3738
val destinationUsername = Util.getSparkProp(sc, "spark.target.username")
3839
val destinationPassword = Util.getSparkProp(sc, "spark.target.password")
39-
val destinationReadConsistencyLevel = Util.getSparkPropOr(sc, "spark.target.read.consistency.level", "LOCAL_QUORUM")
4040
val destinationTrustStorePath = Util.getSparkPropOrEmpty(sc, "spark.target.trustStore.path")
4141
val destinationTrustStorePassword = Util.getSparkPropOrEmpty(sc, "spark.target.trustStore.password")
4242
val destinationTrustStoreType = Util.getSparkPropOr(sc, "spark.target.trustStore.type", "JKS")
@@ -48,7 +48,7 @@ class BaseJob extends App {
4848
val maxPartition = new BigInteger(Util.getSparkPropOr(sc, "spark.origin.maxPartition", "9223372036854775807"))
4949
val coveragePercent = Util.getSparkPropOr(sc, "spark.coveragePercent", "100")
5050
val splitSize = Integer.parseInt(Util.getSparkPropOr(sc, "spark.splitSize", "10000"))
51-
51+
5252
protected def exitSpark() = {
5353
spark.stop()
5454
abstractLogger.info("################################################################################################")

src/main/scala/datastax/astra/migrate/OriginData.scala

Lines changed: 5 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -9,13 +9,13 @@ object OriginData extends BaseJob {
99

1010
val logger = LoggerFactory.getLogger(this.getClass.getName)
1111
logger.info("Started Migration App")
12-
var sourceConnection = getConnection(true, sourceIsAstra, sourceScbPath, sourceHost, sourceUsername, sourcePassword, sourceReadConsistencyLevel,
12+
var sourceConnection = getConnection(true, sourceIsAstra, sourceScbPath, sourceHost, sourceUsername, sourcePassword,
1313
sourceTrustStorePath, sourceTrustStorePassword, sourceTrustStoreType, sourceKeyStorePath, sourceKeyStorePassword, sourceEnabledAlgorithms);
1414
analyzeSourceTable(sourceConnection)
1515
exitSpark
1616

1717

18-
private def getConnection(isSource: Boolean, isAstra: String, scbPath: String, host: String, username: String, password: String, readConsistencyLevel: String,
18+
private def getConnection(isSource: Boolean, isAstra: String, scbPath: String, host: String, username: String, password: String,
1919
trustStorePath: String, trustStorePassword: String, trustStoreType: String,
2020
keyStorePath: String, keyStorePassword: String, enabledAlgorithms: String): CassandraConnector = {
2121
var connType: String = "Source"
@@ -26,7 +26,7 @@ object OriginData extends BaseJob {
2626
return CassandraConnector(sc
2727
.set("spark.cassandra.auth.username", username)
2828
.set("spark.cassandra.auth.password", password)
29-
.set("spark.cassandra.input.consistency.level", readConsistencyLevel)
29+
.set("spark.cassandra.input.consistency.level", consistencyLevel)
3030
.set("spark.cassandra.connection.config.cloud.path", scbPath))
3131
} else if (null != trustStorePath && !trustStorePath.trim.isEmpty) {
3232
abstractLogger.info(connType + ": Connected to Cassandra (or DSE) with SSL!");
@@ -40,7 +40,7 @@ object OriginData extends BaseJob {
4040
return CassandraConnector(sc
4141
.set("spark.cassandra.auth.username", username)
4242
.set("spark.cassandra.auth.password", password)
43-
.set("spark.cassandra.input.consistency.level", readConsistencyLevel)
43+
.set("spark.cassandra.input.consistency.level", consistencyLevel)
4444
.set("spark.cassandra.connection.host", host)
4545
.set("spark.cassandra.connection.ssl.enabled", "true")
4646
.set("spark.cassandra.connection.ssl.enabledAlgorithms", enabledAlgorithmsVar)
@@ -56,7 +56,7 @@ object OriginData extends BaseJob {
5656

5757
return CassandraConnector(sc.set("spark.cassandra.auth.username", username)
5858
.set("spark.cassandra.auth.password", password)
59-
.set("spark.cassandra.input.consistency.level", readConsistencyLevel)
59+
.set("spark.cassandra.input.consistency.level", consistencyLevel)
6060
.set("spark.cassandra.connection.host", host))
6161
}
6262

@@ -78,6 +78,3 @@ object OriginData extends BaseJob {
7878

7979
}
8080

81-
82-
83-

src/resources/sparkConf.properties

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -2,27 +2,21 @@ spark.origin.isAstra false
22
spark.origin.host localhost
33
spark.origin.username some-username
44
spark.origin.password some-secret-password
5-
spark.origin.read.consistency.level LOCAL_QUORUM
65
spark.origin.keyspaceTable test.a1
76

87
spark.target.isAstra true
98
spark.target.scb file:///aaa/bbb/secure-connect-enterprise.zip
109
spark.target.username client-id
1110
spark.target.password client-secret
12-
spark.target.read.consistency.level LOCAL_QUORUM
1311
spark.target.keyspaceTable test.a2
1412
spark.target.autocorrect.missing false
1513
spark.target.autocorrect.mismatch false
16-
spark.target.custom.writeTime 0
1714

1815
spark.maxRetries 10
1916
spark.readRateLimit 20000
2017
spark.writeRateLimit 20000
2118
spark.splitSize 10000
2219
spark.batchSize 5
23-
spark.coveragePercent 100
24-
spark.printStatsAfter 100000
25-
spark.fieldGuardraillimitMB 10
2620

2721
spark.query.origin partition-key,clustering-key,order-date,amount
2822
spark.query.origin.partitionKey partition-key
@@ -40,6 +34,14 @@ spark.origin.writeTimeStampFilter false
4034
spark.origin.minWriteTimeStampFilter 0
4135
spark.origin.maxWriteTimeStampFilter 9223372036854775807
4236

37+
########################## ONLY CHANGE IF YOU KNOW WHAT YOU ARE DOING ###############################
38+
#spark.coveragePercent 100
39+
#spark.printStatsAfter 100000
40+
#spark.read.consistency.level LOCAL_QUORUM
41+
#spark.read.fetch.sizeInRows 1000
42+
#spark.target.custom.writeTime 0
43+
#spark.fieldGuardraillimitMB 10
44+
4345
################### ONLY USE if needing to get record count of recs greater than 10MB from Origin ######################
4446
#spark.origin.checkTableforColSize false
4547
#spark.origin.checkTableforColSize.cols partition-key,clustering-key

0 commit comments

Comments
 (0)