KAFKA-13599: Upgrade RocksDB to 6.27.3 (#11690)

RocksDB v6.27.3 has been released and it is the first release to support s390x. RocksDB is currently the only dependency in gradle/dependencies.gradle without s390x support.

RocksDB v6.27.3 has added some new options that require an update to streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java but no other changes are needed to upgrade.

I have run the unit/integration tests locally on s390x and also the :streams tests on x86_64 and they pass.

Reviewers: Luke Chen <showuon@gmail.com>, Bruno Cadonna <cadonna@apache.org>
This commit is contained in:
Jonathan Albrecht 2022-02-02 04:56:14 -05:00 committed by GitHub
parent 67cf187603
commit ec05f90a3d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 99 additions and 1 deletions

View File

@ -107,7 +107,7 @@ versions += [
netty: "4.1.73.Final",
powermock: "2.0.9",
reflections: "0.9.12",
rocksDB: "6.22.1.1",
rocksDB: "6.27.3",
scalaCollectionCompat: "2.4.4",
scalafmt: "2.7.5",
scalaJava8Compat : "1.0.0",

View File

@ -1552,6 +1552,17 @@ public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter extends
return columnFamilyOptions.ttl();
}
@Override
public Options setPeriodicCompactionSeconds(final long periodicCompactionSeconds) {
columnFamilyOptions.setPeriodicCompactionSeconds(periodicCompactionSeconds);
return this;
}
@Override
public long periodicCompactionSeconds() {
return columnFamilyOptions.periodicCompactionSeconds();
}
@Override
public Options setAtomicFlush(final boolean atomicFlush) {
dbOptions.setAtomicFlush(atomicFlush);
@ -1662,16 +1673,103 @@ public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter extends
return columnFamilyOptions.compactionThreadLimiter();
}
@Override
public Options setCompactionFilter(final AbstractCompactionFilter<? extends AbstractSlice<?>> compactionFilter) {
columnFamilyOptions.setCompactionFilter(compactionFilter);
return this;
}
@Override
public Options setCompactionFilterFactory(final AbstractCompactionFilterFactory<? extends AbstractCompactionFilter<?>> compactionFilterFactory) {
columnFamilyOptions.setCompactionFilterFactory(compactionFilterFactory);
return this;
}
//
// BEGIN options for blobs (integrated BlobDB)
//
@Override
public Options setEnableBlobFiles(final boolean enableBlobFiles) {
columnFamilyOptions.setEnableBlobFiles(enableBlobFiles);
return this;
}
@Override
public boolean enableBlobFiles() {
return columnFamilyOptions.enableBlobFiles();
}
@Override
public Options setMinBlobSize(final long minBlobSize) {
columnFamilyOptions.setMinBlobSize(minBlobSize);
return this;
}
@Override
public long minBlobSize() {
return columnFamilyOptions.minBlobSize();
}
@Override
public Options setBlobFileSize(final long blobFileSize) {
columnFamilyOptions.setBlobFileSize(blobFileSize);
return this;
}
@Override
public long blobFileSize() {
return columnFamilyOptions.blobFileSize();
}
@Override
public Options setBlobCompressionType(final CompressionType compressionType) {
columnFamilyOptions.setBlobCompressionType(compressionType);
return this;
}
@Override
public CompressionType blobCompressionType() {
return columnFamilyOptions.blobCompressionType();
}
@Override
public Options setEnableBlobGarbageCollection(final boolean enableBlobGarbageCollection) {
columnFamilyOptions.setEnableBlobGarbageCollection(enableBlobGarbageCollection);
return this;
}
@Override
public boolean enableBlobGarbageCollection() {
return columnFamilyOptions.enableBlobGarbageCollection();
}
@Override
public Options setBlobGarbageCollectionAgeCutoff(final double blobGarbageCollectionAgeCutoff) {
columnFamilyOptions.setBlobGarbageCollectionAgeCutoff(blobGarbageCollectionAgeCutoff);
return this;
}
@Override
public double blobGarbageCollectionAgeCutoff() {
return columnFamilyOptions.blobGarbageCollectionAgeCutoff();
}
@Override
public Options setBlobGarbageCollectionForceThreshold(final double blobGarbageCollectionForceThreshold) {
columnFamilyOptions.setBlobGarbageCollectionForceThreshold(blobGarbageCollectionForceThreshold);
return this;
}
@Override
public double blobGarbageCollectionForceThreshold() {
return columnFamilyOptions.blobGarbageCollectionForceThreshold();
}
//
// END options for blobs (integrated BlobDB)
//
@Override
public void close() {
// ColumnFamilyOptions should be closed after DBOptions