mirror of https://github.com/apache/kafka.git
KAFKA-16518: Implement KIP-853 flags for storage-tool.sh (#16669)
As part of KIP-853, storage-tool.sh now has two new flags: --standalone, and --initial-voters. This PR implements these two flags in storage-tool.sh. There are currently two valid ways to format a cluster: The pre-KIP-853 way, where you use a statically configured controller quorum. In this case, neither --standalone nor --initial-voters may be specified, and kraft.version must be set to 0. The KIP-853 way, where one of --standalone and --initial-voters must be specified with the initial value of the dynamic controller quorum. In this case, kraft.version must be set to 1. This PR moves the formatting logic out of StorageTool.scala and into Formatter.java. The tool file was never intended to get so huge, or to implement complex logic like generating metadata records. Those things should be done by code in the metadata or raft gradle modules. This is also useful for junit tests, which often need to do formatting. (The 'info' and 'random-uuid' commands remain in StorageTool.scala, for now.) Reviewers: José Armando García Sancio <jsancio@apache.org>
This commit is contained in:
parent
39b1586f93
commit
02f541d4ea
|
@ -182,6 +182,10 @@
|
|||
<subpackage name="migration">
|
||||
<allow pkg="org.apache.kafka.controller" />
|
||||
</subpackage>
|
||||
<subpackage name="storage">
|
||||
<allow pkg="org.apache.kafka.common.internals" />
|
||||
<allow pkg="org.apache.kafka.snapshot" />
|
||||
</subpackage>
|
||||
<subpackage name="util">
|
||||
<allow class="org.apache.kafka.common.compress.Compression" exact-match="true" />
|
||||
</subpackage>
|
||||
|
|
|
@ -189,7 +189,7 @@
|
|||
|
||||
<!-- Raft -->
|
||||
<suppress checks="NPathComplexity"
|
||||
files="RecordsIterator.java"/>
|
||||
files="(DynamicVoter|RecordsIterator).java"/>
|
||||
|
||||
<!-- Streams -->
|
||||
<suppress checks="ClassFanOutComplexity"
|
||||
|
@ -325,7 +325,7 @@
|
|||
<suppress checks="(ParameterNumber|ClassDataAbstractionCoupling)"
|
||||
files="(QuorumController).java"/>
|
||||
<suppress checks="(CyclomaticComplexity|NPathComplexity)"
|
||||
files="(PartitionRegistration|PartitionChangeBuilder).java"/>
|
||||
files="(PartitionRegistration|PartitionChangeBuilder|ScramParser).java"/>
|
||||
<suppress checks="CyclomaticComplexity"
|
||||
files="(ClientQuotasImage|KafkaEventQueue|MetadataDelta|QuorumController|ReplicationControlManager|KRaftMigrationDriver|ClusterControlManager|MetaPropertiesEnsemble).java"/>
|
||||
<suppress checks="NPathComplexity"
|
||||
|
|
|
@ -24,25 +24,18 @@ import java.nio.file.{Files, Paths}
|
|||
import kafka.utils.{Exit, Logging}
|
||||
import net.sourceforge.argparse4j.ArgumentParsers
|
||||
import net.sourceforge.argparse4j.impl.Arguments.{append, store, storeTrue}
|
||||
import net.sourceforge.argparse4j.inf.Namespace
|
||||
import net.sourceforge.argparse4j.inf.{ArgumentParserException, Namespace}
|
||||
import org.apache.kafka.common.Uuid
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata}
|
||||
import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion}
|
||||
import org.apache.kafka.common.metadata.FeatureLevelRecord
|
||||
import org.apache.kafka.common.metadata.UserScramCredentialRecord
|
||||
import org.apache.kafka.common.security.scram.internals.ScramMechanism
|
||||
import org.apache.kafka.common.security.scram.internals.ScramFormatter
|
||||
import org.apache.kafka.server.config.ReplicationConfigs
|
||||
import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
|
||||
import org.apache.kafka.server.common.FeatureVersion
|
||||
import org.apache.kafka.metadata.storage.{Formatter, FormatterException}
|
||||
import org.apache.kafka.raft.DynamicVoters
|
||||
import org.apache.kafka.server.ProcessRole
|
||||
import org.apache.kafka.server.config.ReplicationConfigs
|
||||
|
||||
import java.util
|
||||
import java.util.{Base64, Collections, Optional}
|
||||
import scala.collection.mutable
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
|
||||
object StorageTool extends Logging {
|
||||
|
||||
|
@ -50,8 +43,11 @@ object StorageTool extends Logging {
|
|||
var exitCode: Integer = 0
|
||||
var message: Option[String] = None
|
||||
try {
|
||||
exitCode = execute(args)
|
||||
exitCode = execute(args, System.out)
|
||||
} catch {
|
||||
case e: FormatterException =>
|
||||
exitCode = 1
|
||||
message = Some(e.getMessage)
|
||||
case e: TerseFailure =>
|
||||
exitCode = 1
|
||||
message = Some(e.getMessage)
|
||||
|
@ -65,22 +61,31 @@ object StorageTool extends Logging {
|
|||
* @param args The command line arguments
|
||||
* @return The exit code
|
||||
*/
|
||||
def execute(args: Array[String]): Int = {
|
||||
val namespace = parseArguments(args)
|
||||
def execute(
|
||||
args: Array[String],
|
||||
printStream: PrintStream
|
||||
): Int = {
|
||||
val namespace = try {
|
||||
parseArguments(args)
|
||||
} catch {
|
||||
case e: ArgumentParserException =>
|
||||
e.printStackTrace(printStream)
|
||||
return 1
|
||||
}
|
||||
val command = namespace.getString("command")
|
||||
val config = Option(namespace.getString("config")).flatMap(
|
||||
p => Some(new KafkaConfig(Utils.loadProps(p))))
|
||||
command match {
|
||||
case "info" =>
|
||||
val directories = configToLogDirectories(config.get)
|
||||
val selfManagedMode = configToSelfManagedMode(config.get)
|
||||
infoCommand(System.out, selfManagedMode, directories)
|
||||
infoCommand(printStream, config.get.processRoles.nonEmpty, directories)
|
||||
|
||||
case "format" =>
|
||||
runFormatCommand(namespace, config.get)
|
||||
runFormatCommand(namespace, config.get, printStream)
|
||||
0
|
||||
|
||||
case "random-uuid" =>
|
||||
System.out.println(Uuid.randomUuid)
|
||||
printStream.println(Uuid.randomUuid)
|
||||
0
|
||||
case _ =>
|
||||
throw new RuntimeException(s"Unknown command $command")
|
||||
|
@ -90,106 +95,58 @@ object StorageTool extends Logging {
|
|||
/**
|
||||
* Validates arguments, configuration, prepares bootstrap metadata and delegates to {{@link formatCommand}}.
|
||||
* Visible for testing.
|
||||
*
|
||||
* @param namespace Arguments
|
||||
* @param config The server configuration
|
||||
* @return The exit code
|
||||
*/
|
||||
def runFormatCommand(namespace: Namespace, config: KafkaConfig) = {
|
||||
val directories = configToLogDirectories(config)
|
||||
val clusterId = namespace.getString("cluster_id")
|
||||
val metaProperties = new MetaProperties.Builder().
|
||||
setVersion(MetaPropertiesVersion.V1).
|
||||
setClusterId(clusterId).
|
||||
setNodeId(config.nodeId).
|
||||
build()
|
||||
val metadataRecords : ArrayBuffer[ApiMessageAndVersion] = ArrayBuffer()
|
||||
val specifiedFeatures: util.List[String] = namespace.getList("feature")
|
||||
val releaseVersionFlagSpecified = namespace.getString("release_version") != null
|
||||
if (releaseVersionFlagSpecified && specifiedFeatures != null) {
|
||||
throw new TerseFailure("Both --release-version and --feature were set. Only one of the two flags can be set.")
|
||||
}
|
||||
val featureNamesAndLevelsMap = featureNamesAndLevels(Option(specifiedFeatures).getOrElse(Collections.emptyList).asScala.toList)
|
||||
val metadataVersion = getMetadataVersion(namespace, featureNamesAndLevelsMap,
|
||||
Option(config.originals.get(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)).map(_.toString))
|
||||
validateMetadataVersion(metadataVersion, config)
|
||||
// Get all other features, validate, and create records for them
|
||||
// Use latest default for features if --release-version is not specified
|
||||
generateFeatureRecords(
|
||||
metadataRecords,
|
||||
metadataVersion,
|
||||
featureNamesAndLevelsMap,
|
||||
Features.PRODUCTION_FEATURES.asScala.toList,
|
||||
config.unstableFeatureVersionsEnabled,
|
||||
releaseVersionFlagSpecified
|
||||
)
|
||||
getUserScramCredentialRecords(namespace).foreach(userScramCredentialRecords => {
|
||||
if (!metadataVersion.isScramSupported) {
|
||||
throw new TerseFailure(s"SCRAM is only supported in metadata.version ${MetadataVersion.IBP_3_5_IV2} or later.")
|
||||
}
|
||||
for (record <- userScramCredentialRecords) {
|
||||
metadataRecords.append(new ApiMessageAndVersion(record, 0.toShort))
|
||||
}
|
||||
})
|
||||
val bootstrapMetadata = buildBootstrapMetadata(metadataVersion, Some(metadataRecords), "format command")
|
||||
val ignoreFormatted = namespace.getBoolean("ignore_formatted")
|
||||
if (!configToSelfManagedMode(config)) {
|
||||
def runFormatCommand(
|
||||
namespace: Namespace,
|
||||
config: KafkaConfig,
|
||||
printStream: PrintStream
|
||||
): Unit = {
|
||||
if (config.processRoles.isEmpty) {
|
||||
throw new TerseFailure("The kafka configuration file appears to be for " +
|
||||
"a legacy cluster. Formatting is only supported for clusters in KRaft mode.")
|
||||
}
|
||||
formatCommand(System.out, directories, metaProperties, bootstrapMetadata,
|
||||
metadataVersion,ignoreFormatted)
|
||||
val formatter = new Formatter().
|
||||
setPrintStream(printStream).
|
||||
setNodeId(config.nodeId).
|
||||
setClusterId(namespace.getString("cluster_id")).
|
||||
setUnstableFeatureVersionsEnabled(config.unstableFeatureVersionsEnabled).
|
||||
setIgnoreFormatted(namespace.getBoolean("ignore_formatted")).
|
||||
setControllerListenerName(config.controllerListenerNames.head).
|
||||
setMetadataLogDirectory(config.metadataLogDir)
|
||||
Option(namespace.getString("release_version")) match {
|
||||
case Some(releaseVersion) => formatter.setReleaseVersion(MetadataVersion.fromVersionString(releaseVersion))
|
||||
case None => Option(config.originals.get(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)).
|
||||
foreach(v => formatter.setReleaseVersion(MetadataVersion.fromVersionString(v.toString)))
|
||||
}
|
||||
Option(namespace.getString("initial_controllers")).
|
||||
foreach(v => formatter.setInitialVoters(DynamicVoters.parse(v)))
|
||||
if (namespace.getBoolean("standalone")) {
|
||||
formatter.setInitialVoters(createStandaloneDynamicVoters(config))
|
||||
}
|
||||
configToLogDirectories(config).foreach(formatter.addDirectory(_))
|
||||
formatter.run()
|
||||
}
|
||||
|
||||
private def validateMetadataVersion(metadataVersion: MetadataVersion, config: KafkaConfig): Unit = {
|
||||
if (!metadataVersion.isKRaftSupported) {
|
||||
throw new TerseFailure(s"Must specify a valid KRaft metadata.version of at least ${MetadataVersion.IBP_3_0_IV0}.")
|
||||
def createStandaloneDynamicVoters(
|
||||
config: KafkaConfig
|
||||
): DynamicVoters = {
|
||||
if (!config.processRoles.contains(ProcessRole.ControllerRole)) {
|
||||
throw new TerseFailure("You cannot use --standalone on a broker node.")
|
||||
}
|
||||
if (!metadataVersion.isProduction) {
|
||||
if (config.unstableFeatureVersionsEnabled) {
|
||||
System.out.println(s"WARNING: using pre-production metadata.version $metadataVersion.")
|
||||
} else {
|
||||
throw new TerseFailure(s"The metadata.version $metadataVersion is not ready for production use yet.")
|
||||
}
|
||||
if (config.controllerListeners.isEmpty) {
|
||||
throw new RuntimeException("No controller listeners found.")
|
||||
}
|
||||
try {
|
||||
config.validateWithMetadataVersion(metadataVersion)
|
||||
} catch {
|
||||
case e: IllegalArgumentException => throw new TerseFailure(s"Invalid configuration for metadata version: ${e.getMessage}")
|
||||
}
|
||||
}
|
||||
|
||||
private[tools] def generateFeatureRecords(metadataRecords: ArrayBuffer[ApiMessageAndVersion],
|
||||
metadataVersion: MetadataVersion,
|
||||
specifiedFeatures: Map[String, java.lang.Short],
|
||||
allFeatures: List[Features],
|
||||
unstableFeatureVersionsEnabled: Boolean,
|
||||
releaseVersionSpecified: Boolean): Unit = {
|
||||
// If we are using --release-version, the default is based on the metadata version.
|
||||
val metadataVersionForDefault = if (releaseVersionSpecified) metadataVersion else MetadataVersion.LATEST_PRODUCTION
|
||||
|
||||
val allNonZeroFeaturesAndLevels: ArrayBuffer[FeatureVersion] = mutable.ArrayBuffer[FeatureVersion]()
|
||||
|
||||
allFeatures.foreach { feature =>
|
||||
val level: java.lang.Short = specifiedFeatures.getOrElse(feature.featureName, feature.defaultValue(metadataVersionForDefault))
|
||||
// Only set feature records for levels greater than 0. 0 is assumed if there is no record. Throw an error if level < 0.
|
||||
if (level != 0) {
|
||||
allNonZeroFeaturesAndLevels.append(feature.fromFeatureLevel(level, unstableFeatureVersionsEnabled))
|
||||
}
|
||||
}
|
||||
val featuresMap = Features.featureImplsToMap(allNonZeroFeaturesAndLevels.asJava)
|
||||
featuresMap.put(MetadataVersion.FEATURE_NAME, metadataVersion.featureLevel)
|
||||
|
||||
try {
|
||||
for (feature <- allNonZeroFeaturesAndLevels) {
|
||||
// In order to validate, we need all feature versions set.
|
||||
Features.validateVersion(feature, featuresMap)
|
||||
metadataRecords.append(new ApiMessageAndVersion(new FeatureLevelRecord().
|
||||
setName(feature.featureName).
|
||||
setFeatureLevel(feature.featureLevel), 0.toShort))
|
||||
}
|
||||
} catch {
|
||||
case e: Throwable => throw new TerseFailure(e.getMessage)
|
||||
val host = if (config.controllerListeners.head.host == null) {
|
||||
"localhost"
|
||||
} else {
|
||||
config.controllerListeners.head.host
|
||||
}
|
||||
val port = config.controllerListeners.head.port
|
||||
DynamicVoters.parse(s"${config.nodeId}@${host}:${port}:${Uuid.randomUuid()}")
|
||||
}
|
||||
|
||||
def parseArguments(args: Array[String]): Namespace = {
|
||||
|
@ -223,12 +180,20 @@ object StorageTool extends Logging {
|
|||
action(storeTrue())
|
||||
formatParser.addArgument("--release-version", "-r").
|
||||
action(store()).
|
||||
help(s"A KRaft release version to use for the initial metadata.version. The minimum is ${MetadataVersion.IBP_3_0_IV0}, the default is ${MetadataVersion.LATEST_PRODUCTION}")
|
||||
help(s"The release version to use for the initial feature settings. The minimum is " +
|
||||
s"${MetadataVersion.IBP_3_0_IV0}; the default is ${MetadataVersion.LATEST_PRODUCTION}")
|
||||
formatParser.addArgument("--feature", "-f").
|
||||
help("A feature upgrade we should perform, in feature=level format. For example: `metadata.version=5`.").
|
||||
action(append());
|
||||
|
||||
parser.parseArgsOrFail(args)
|
||||
help("The setting to use for a specific feature, in feature=level format. For example: `kraft.version=1`.").
|
||||
action(append())
|
||||
val reconfigurableQuorumOptions = formatParser.addMutuallyExclusiveGroup()
|
||||
reconfigurableQuorumOptions.addArgument("--standalone", "-s").
|
||||
help("Used to initialize a single-node quorum controller quorum.").
|
||||
action(storeTrue())
|
||||
reconfigurableQuorumOptions.addArgument("--initial-controllers", "-I").
|
||||
help("A list of controller quorum voter ids, directories, and hostname:port pairs. The same values must be used to format all nodes. For example:\n" +
|
||||
"0@localhost:8082:JEXY6aqzQY-32P5TStzaFg@,1@localhost:8083:MvDxzVmcRsaTz33bUuRU6A,2@localhost:8084:07R5amHmR32VDA6jHkGbTA\n").
|
||||
action(store())
|
||||
parser.parseArgs(args)
|
||||
}
|
||||
|
||||
def configToLogDirectories(config: KafkaConfig): Seq[String] = {
|
||||
|
@ -238,148 +203,7 @@ object StorageTool extends Logging {
|
|||
directories.toSeq
|
||||
}
|
||||
|
||||
private def configToSelfManagedMode(config: KafkaConfig): Boolean = config.processRoles.nonEmpty
|
||||
|
||||
def getMetadataVersion(
|
||||
namespace: Namespace,
|
||||
featureNamesAndLevelsMap: Map[String, java.lang.Short],
|
||||
defaultVersionString: Option[String]
|
||||
): MetadataVersion = {
|
||||
val defaultValue = defaultVersionString match {
|
||||
case Some(versionString) => MetadataVersion.fromVersionString(versionString)
|
||||
case None => MetadataVersion.LATEST_PRODUCTION
|
||||
}
|
||||
|
||||
val releaseVersionTag = Option(namespace.getString("release_version"))
|
||||
val featureTag = featureNamesAndLevelsMap.get(MetadataVersion.FEATURE_NAME)
|
||||
|
||||
(releaseVersionTag, featureTag) match {
|
||||
case (Some(_), Some(_)) => // We should throw an error before we hit this case, but include for completeness
|
||||
throw new IllegalArgumentException("Both --release_version and --feature were set. Only one of the two flags can be set.")
|
||||
case (Some(version), None) =>
|
||||
MetadataVersion.fromVersionString(version)
|
||||
case (None, Some(level)) =>
|
||||
MetadataVersion.fromFeatureLevel(level)
|
||||
case (None, None) =>
|
||||
defaultValue
|
||||
}
|
||||
}
|
||||
|
||||
private def getUserScramCredentialRecord(
|
||||
mechanism: String,
|
||||
config: String
|
||||
) : UserScramCredentialRecord = {
|
||||
/*
|
||||
* Remove '[' amd ']'
|
||||
* Split K->V pairs on ',' and no K or V should contain ','
|
||||
* Split K and V on '=' but V could contain '=' if inside ""
|
||||
* Create Map of K to V and replace all " in V
|
||||
*/
|
||||
val argMap = config.substring(1, config.length - 1)
|
||||
.split(",")
|
||||
.map(_.split("=(?=(?:[^\"]*\"[^\"]*\")*[^\"]*$)"))
|
||||
.map(args => args(0) -> args(1).replaceAll("\"", "")).toMap
|
||||
|
||||
val scramMechanism = ScramMechanism.forMechanismName(mechanism)
|
||||
|
||||
def getName(argMap: Map[String,String]) : String = {
|
||||
if (!argMap.contains("name")) {
|
||||
throw new TerseFailure(s"You must supply 'name' to add-scram")
|
||||
}
|
||||
argMap("name")
|
||||
}
|
||||
|
||||
def getSalt(argMap: Map[String,String], scramMechanism : ScramMechanism) : Array[Byte] = {
|
||||
if (argMap.contains("salt")) {
|
||||
Base64.getDecoder.decode(argMap("salt"))
|
||||
} else {
|
||||
new ScramFormatter(scramMechanism).secureRandomBytes()
|
||||
}
|
||||
}
|
||||
|
||||
def getIterations(argMap: Map[String,String], scramMechanism : ScramMechanism) : Int = {
|
||||
if (argMap.contains("salt")) {
|
||||
val iterations = argMap("iterations").toInt
|
||||
if (iterations < scramMechanism.minIterations()) {
|
||||
throw new TerseFailure(s"The 'iterations' value must be >= ${scramMechanism.minIterations()} for add-scram")
|
||||
}
|
||||
if (iterations > scramMechanism.maxIterations()) {
|
||||
throw new TerseFailure(s"The 'iterations' value must be <= ${scramMechanism.maxIterations()} for add-scram")
|
||||
}
|
||||
iterations
|
||||
} else {
|
||||
4096
|
||||
}
|
||||
}
|
||||
|
||||
def getSaltedPassword(
|
||||
argMap: Map[String,String],
|
||||
scramMechanism : ScramMechanism,
|
||||
salt : Array[Byte],
|
||||
iterations: Int
|
||||
) : Array[Byte] = {
|
||||
if (argMap.contains("password")) {
|
||||
if (argMap.contains("saltedpassword")) {
|
||||
throw new TerseFailure(s"You must only supply one of 'password' or 'saltedpassword' to add-scram")
|
||||
}
|
||||
new ScramFormatter(scramMechanism).saltedPassword(argMap("password"), salt, iterations)
|
||||
} else {
|
||||
if (!argMap.contains("saltedpassword")) {
|
||||
throw new TerseFailure(s"You must supply one of 'password' or 'saltedpassword' to add-scram")
|
||||
}
|
||||
if (!argMap.contains("salt")) {
|
||||
throw new TerseFailure(s"You must supply 'salt' with 'saltedpassword' to add-scram")
|
||||
}
|
||||
Base64.getDecoder.decode(argMap("saltedpassword"))
|
||||
}
|
||||
}
|
||||
|
||||
val name = getName(argMap)
|
||||
val salt = getSalt(argMap, scramMechanism)
|
||||
val iterations = getIterations(argMap, scramMechanism)
|
||||
val saltedPassword = getSaltedPassword(argMap, scramMechanism, salt, iterations)
|
||||
|
||||
val myrecord = try {
|
||||
val formatter = new ScramFormatter(scramMechanism)
|
||||
|
||||
new UserScramCredentialRecord()
|
||||
.setName(name)
|
||||
.setMechanism(scramMechanism.`type`)
|
||||
.setSalt(salt)
|
||||
.setStoredKey(formatter.storedKey(formatter.clientKey(saltedPassword)))
|
||||
.setServerKey(formatter.serverKey(saltedPassword))
|
||||
.setIterations(iterations)
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
throw new TerseFailure(s"Error attempting to create UserScramCredentialRecord: ${e.getMessage}")
|
||||
}
|
||||
myrecord
|
||||
}
|
||||
|
||||
def getUserScramCredentialRecords(namespace: Namespace): Option[ArrayBuffer[UserScramCredentialRecord]] = {
|
||||
if (namespace.getList("add_scram") != null) {
|
||||
val listofAddConfig : List[String] = namespace.getList("add_scram").asScala.toList
|
||||
val userScramCredentialRecords : ArrayBuffer[UserScramCredentialRecord] = ArrayBuffer()
|
||||
for (singleAddConfig <- listofAddConfig) {
|
||||
val singleAddConfigList = singleAddConfig.split("\\s+")
|
||||
|
||||
// The first subarg must be of the form key=value
|
||||
val nameValueRecord = singleAddConfigList(0).split("=", 2)
|
||||
nameValueRecord(0) match {
|
||||
case "SCRAM-SHA-256" =>
|
||||
userScramCredentialRecords.append(getUserScramCredentialRecord(nameValueRecord(0), nameValueRecord(1)))
|
||||
case "SCRAM-SHA-512" =>
|
||||
userScramCredentialRecords.append(getUserScramCredentialRecord(nameValueRecord(0), nameValueRecord(1)))
|
||||
case _ => throw new TerseFailure(s"The add-scram mechanism ${nameValueRecord(0)} is not supported.")
|
||||
}
|
||||
}
|
||||
Some(userScramCredentialRecords)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
def infoCommand(stream: PrintStream, selfManagedMode: Boolean, directories: Seq[String]): Int = {
|
||||
def infoCommand(stream: PrintStream, kraftMode: Boolean, directories: Seq[String]): Int = {
|
||||
val problems = new mutable.ArrayBuffer[String]
|
||||
val foundDirectories = new mutable.ArrayBuffer[String]
|
||||
var prevMetadata: Option[MetaProperties] = None
|
||||
|
@ -419,7 +243,7 @@ object StorageTool extends Logging {
|
|||
})
|
||||
|
||||
prevMetadata.foreach { prev =>
|
||||
if (selfManagedMode) {
|
||||
if (kraftMode) {
|
||||
if (prev.version.equals(MetaPropertiesVersion.V0)) {
|
||||
problems += "The kafka configuration file appears to be for a cluster in KRaft mode, but " +
|
||||
"the directories are formatted for legacy mode."
|
||||
|
@ -465,96 +289,4 @@ object StorageTool extends Logging {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
def buildBootstrapMetadata(metadataVersion: MetadataVersion,
|
||||
metadataOptionalArguments: Option[ArrayBuffer[ApiMessageAndVersion]],
|
||||
source: String): BootstrapMetadata = {
|
||||
|
||||
val metadataRecords = new util.ArrayList[ApiMessageAndVersion]
|
||||
metadataRecords.add(new ApiMessageAndVersion(new FeatureLevelRecord().
|
||||
setName(MetadataVersion.FEATURE_NAME).
|
||||
setFeatureLevel(metadataVersion.featureLevel()), 0.toShort))
|
||||
|
||||
metadataOptionalArguments.foreach { metadataArguments =>
|
||||
for (record <- metadataArguments) metadataRecords.add(record)
|
||||
}
|
||||
|
||||
BootstrapMetadata.fromRecords(metadataRecords, source)
|
||||
}
|
||||
|
||||
def formatCommand(
|
||||
stream: PrintStream,
|
||||
directories: Seq[String],
|
||||
metaProperties: MetaProperties,
|
||||
bootstrapMetadata: BootstrapMetadata,
|
||||
metadataVersion: MetadataVersion,
|
||||
ignoreFormatted: Boolean
|
||||
): Int = {
|
||||
if (directories.isEmpty) {
|
||||
throw new TerseFailure("No log directories found in the configuration.")
|
||||
}
|
||||
val loader = new MetaPropertiesEnsemble.Loader()
|
||||
.addLogDirs(directories.asJava)
|
||||
val metaPropertiesEnsemble = loader.load()
|
||||
metaPropertiesEnsemble.verify(metaProperties.clusterId(), metaProperties.nodeId(),
|
||||
util.EnumSet.noneOf(classOf[VerificationFlag]))
|
||||
|
||||
val copier = new MetaPropertiesEnsemble.Copier(metaPropertiesEnsemble)
|
||||
if (!(ignoreFormatted || copier.logDirProps().isEmpty)) {
|
||||
val firstLogDir = copier.logDirProps().keySet().iterator().next()
|
||||
throw new TerseFailure(s"Log directory $firstLogDir is already formatted. " +
|
||||
"Use --ignore-formatted to ignore this directory and format the others.")
|
||||
}
|
||||
if (!copier.errorLogDirs().isEmpty) {
|
||||
copier.errorLogDirs().forEach(errorLogDir => {
|
||||
stream.println(s"I/O error trying to read log directory $errorLogDir. Ignoring...")
|
||||
})
|
||||
if (metaPropertiesEnsemble.emptyLogDirs().isEmpty && copier.logDirProps().isEmpty) {
|
||||
throw new TerseFailure("No available log directories to format.")
|
||||
}
|
||||
}
|
||||
if (metaPropertiesEnsemble.emptyLogDirs().isEmpty) {
|
||||
stream.println("All of the log directories are already formatted.")
|
||||
} else {
|
||||
metaPropertiesEnsemble.emptyLogDirs().forEach(logDir => {
|
||||
copier.setLogDirProps(logDir, new MetaProperties.Builder(metaProperties).
|
||||
setDirectoryId(copier.generateValidDirectoryId()).
|
||||
build())
|
||||
copier.setPreWriteHandler((logDir, _, _) => {
|
||||
stream.println(s"Formatting $logDir with metadata.version $metadataVersion.")
|
||||
Files.createDirectories(Paths.get(logDir))
|
||||
val bootstrapDirectory = new BootstrapDirectory(logDir, Optional.empty())
|
||||
bootstrapDirectory.writeBinaryFile(bootstrapMetadata)
|
||||
})
|
||||
copier.setWriteErrorHandler((logDir, e) => {
|
||||
throw new TerseFailure(s"Error while writing meta.properties file $logDir: ${e.getMessage}")
|
||||
})
|
||||
})
|
||||
copier.writeLogDirChanges()
|
||||
}
|
||||
0
|
||||
}
|
||||
|
||||
private def parseNameAndLevel(input: String): (String, java.lang.Short) = {
|
||||
val equalsIndex = input.indexOf("=")
|
||||
if (equalsIndex < 0)
|
||||
throw new RuntimeException("Can't parse feature=level string " + input + ": equals sign not found.")
|
||||
val name = input.substring(0, equalsIndex).trim
|
||||
val levelString = input.substring(equalsIndex + 1).trim
|
||||
try {
|
||||
levelString.toShort
|
||||
} catch {
|
||||
case _: Throwable =>
|
||||
throw new RuntimeException("Can't parse feature=level string " + input + ": " + "unable to parse " + levelString + " as a short.")
|
||||
}
|
||||
(name, levelString.toShort)
|
||||
}
|
||||
|
||||
def featureNamesAndLevels(features: List[String]): Map[String, java.lang.Short] = {
|
||||
features.map { (feature: String) =>
|
||||
// Ensure the feature exists
|
||||
val nameAndLevel = parseNameAndLevel(feature)
|
||||
(nameAndLevel._1, nameAndLevel._2)
|
||||
}.toMap
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,21 +18,19 @@ package kafka.api
|
|||
|
||||
import java.util.Properties
|
||||
import kafka.utils._
|
||||
import kafka.tools.StorageTool
|
||||
import kafka.zk.ConfigEntityChangeNotificationZNode
|
||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateDelegationTokenOptions, ScramCredentialInfo, UserScramCredentialAlteration, UserScramCredentialUpsertion, ScramMechanism => PublicScramMechanism}
|
||||
import org.apache.kafka.common.config.SaslConfigs
|
||||
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
|
||||
import org.apache.kafka.common.security.scram.internals.ScramMechanism
|
||||
import org.apache.kafka.common.security.token.delegation.DelegationToken
|
||||
import org.apache.kafka.metadata.storage.Formatter
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion
|
||||
import org.apache.kafka.server.config.DelegationTokenManagerConfigs
|
||||
|
||||
class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest {
|
||||
|
@ -69,17 +67,11 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest
|
|||
}
|
||||
|
||||
// Create the admin credentials for KRaft as part of controller initialization
|
||||
override def optionalMetadataRecords: Option[ArrayBuffer[ApiMessageAndVersion]] = {
|
||||
val args = Seq("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ", "-S",
|
||||
s"SCRAM-SHA-256=[name=${JaasTestUtils.KafkaScramAdmin},password=${JaasTestUtils.KafkaScramAdminPassword}]")
|
||||
val namespace = StorageTool.parseArguments(args.toArray)
|
||||
val metadataRecords : ArrayBuffer[ApiMessageAndVersion] = ArrayBuffer()
|
||||
StorageTool.getUserScramCredentialRecords(namespace).foreach {
|
||||
userScramCredentialRecords => for (record <- userScramCredentialRecords) {
|
||||
metadataRecords.append(new ApiMessageAndVersion(record, 0.toShort))
|
||||
}
|
||||
}
|
||||
Some(metadataRecords)
|
||||
|
||||
override def addFormatterSettings(formatter: Formatter): Unit = {
|
||||
formatter.setClusterId("XcZZOzUqS4yHOjhMQB6JLQ")
|
||||
formatter.setScramArguments(
|
||||
List(s"SCRAM-SHA-256=[name=${JaasTestUtils.KafkaScramAdmin},password=${JaasTestUtils.KafkaScramAdminPassword}]").asJava)
|
||||
}
|
||||
|
||||
override def createPrivilegedAdminClient(): Admin = createScramAdminClient(kafkaClientSaslMechanism, kafkaPrincipal.getName, kafkaPassword)
|
||||
|
|
|
@ -17,12 +17,11 @@
|
|||
package kafka.api
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import kafka.utils._
|
||||
import kafka.tools.StorageTool
|
||||
import kafka.zk.ConfigEntityChangeNotificationZNode
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||
import org.apache.kafka.common.security.scram.internals.ScramMechanism
|
||||
import org.apache.kafka.metadata.storage.Formatter
|
||||
import org.apache.kafka.test.TestSslUtils
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -31,9 +30,6 @@ import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
|||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion
|
||||
|
||||
class SaslScramSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTest {
|
||||
override protected def kafkaClientSaslMechanism = "SCRAM-SHA-256"
|
||||
override protected def kafkaServerSaslMechanisms = ScramMechanism.mechanismNames.asScala.toList
|
||||
|
@ -55,17 +51,10 @@ class SaslScramSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTes
|
|||
}
|
||||
|
||||
// Create the admin credentials for KRaft as part of controller initialization
|
||||
override def optionalMetadataRecords: Option[ArrayBuffer[ApiMessageAndVersion]] = {
|
||||
val args = Seq("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ", "-S",
|
||||
s"SCRAM-SHA-256=[name=${JaasTestUtils.KafkaScramAdmin},password=${JaasTestUtils.KafkaScramAdminPassword}]")
|
||||
val namespace = StorageTool.parseArguments(args.toArray)
|
||||
val metadataRecords : ArrayBuffer[ApiMessageAndVersion] = ArrayBuffer()
|
||||
StorageTool.getUserScramCredentialRecords(namespace).foreach {
|
||||
userScramCredentialRecords => for (record <- userScramCredentialRecords) {
|
||||
metadataRecords.append(new ApiMessageAndVersion(record, 0.toShort))
|
||||
}
|
||||
}
|
||||
Some(metadataRecords)
|
||||
override def addFormatterSettings(formatter: Formatter): Unit = {
|
||||
formatter.setClusterId("XcZZOzUqS4yHOjhMQB6JLQ")
|
||||
formatter.setScramArguments(List(
|
||||
s"SCRAM-SHA-256=[name=${JaasTestUtils.KafkaScramAdmin},password=${JaasTestUtils.KafkaScramAdminPassword}]").asJava)
|
||||
}
|
||||
|
||||
override def configureListeners(props: collection.Seq[Properties]): Unit = {
|
||||
|
|
|
@ -26,18 +26,17 @@ import javax.security.auth.login.Configuration
|
|||
import kafka.utils.{CoreUtils, Logging, TestInfoUtils, TestUtils}
|
||||
import kafka.zk.{AdminZkClient, EmbeddedZookeeper, KafkaZkClient}
|
||||
import org.apache.kafka.clients.consumer.GroupProtocol
|
||||
import org.apache.kafka.common.metadata.FeatureLevelRecord
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.security.JaasUtils
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.{Exit, Time}
|
||||
import org.apache.kafka.common.{DirectoryId, Uuid}
|
||||
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata
|
||||
import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.{REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR}
|
||||
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion}
|
||||
import org.apache.kafka.metadata.storage.Formatter
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ServerLogConfigs}
|
||||
import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler}
|
||||
import org.apache.zookeeper.client.ZKClientConfig
|
||||
|
@ -46,9 +45,7 @@ import org.junit.jupiter.api.Assertions._
|
|||
import org.junit.jupiter.api.{AfterAll, AfterEach, BeforeAll, BeforeEach, Tag, TestInfo}
|
||||
|
||||
import java.nio.file.{Files, Paths}
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.collection.mutable.ListBuffer
|
||||
import scala.collection.{Seq, immutable}
|
||||
import scala.collection.Seq
|
||||
import scala.compat.java8.OptionConverters._
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
|
@ -104,7 +101,7 @@ class KRaftQuorumImplementation(
|
|||
): KafkaBroker = {
|
||||
val metaPropertiesEnsemble = {
|
||||
val loader = new MetaPropertiesEnsemble.Loader()
|
||||
.addLogDirs(config.logDirs.asJava)
|
||||
loader.addLogDirs(config.logDirs.asJava)
|
||||
loader.addMetadataLogDir(config.metadataLogDir)
|
||||
val ensemble = loader.load()
|
||||
val copier = new MetaPropertiesEnsemble.Copier(ensemble)
|
||||
|
@ -186,8 +183,6 @@ abstract class QuorumTestHarness extends Logging {
|
|||
private var testInfo: TestInfo = _
|
||||
protected var implementation: QuorumImplementation = _
|
||||
|
||||
val bootstrapRecords: ListBuffer[ApiMessageAndVersion] = ListBuffer()
|
||||
|
||||
def isKRaftTest(): Boolean = {
|
||||
TestInfoUtils.isKRaft(testInfo)
|
||||
}
|
||||
|
@ -313,7 +308,7 @@ abstract class QuorumTestHarness extends Logging {
|
|||
CoreUtils.swallow(kRaftQuorumImplementation.controllerServer.shutdown(), kRaftQuorumImplementation.log)
|
||||
}
|
||||
|
||||
def optionalMetadataRecords: Option[ArrayBuffer[ApiMessageAndVersion]] = None
|
||||
def addFormatterSettings(formatter: Formatter): Unit = {}
|
||||
|
||||
private def newKRaftQuorum(testInfo: TestInfo): KRaftQuorumImplementation = {
|
||||
newKRaftQuorum(new Properties())
|
||||
|
@ -334,31 +329,6 @@ abstract class QuorumTestHarness extends Logging {
|
|||
}
|
||||
val nodeId = Integer.parseInt(props.getProperty(KRaftConfigs.NODE_ID_CONFIG))
|
||||
val metadataDir = TestUtils.tempDir()
|
||||
val metaProperties = new MetaProperties.Builder().
|
||||
setVersion(MetaPropertiesVersion.V1).
|
||||
setClusterId(Uuid.randomUuid().toString).
|
||||
setNodeId(nodeId).
|
||||
build()
|
||||
TestUtils.formatDirectories(immutable.Seq(metadataDir.getAbsolutePath), metaProperties, metadataVersion, optionalMetadataRecords)
|
||||
|
||||
val metadataRecords = new util.ArrayList[ApiMessageAndVersion]
|
||||
metadataRecords.add(new ApiMessageAndVersion(new FeatureLevelRecord().
|
||||
setName(MetadataVersion.FEATURE_NAME).
|
||||
setFeatureLevel(metadataVersion.featureLevel()), 0.toShort))
|
||||
|
||||
metadataRecords.add(new ApiMessageAndVersion(
|
||||
new FeatureLevelRecord()
|
||||
.setName(Features.TRANSACTION_VERSION.featureName)
|
||||
.setFeatureLevel(Features.TRANSACTION_VERSION.latestTesting),
|
||||
0.toShort
|
||||
))
|
||||
|
||||
optionalMetadataRecords.foreach { metadataArguments =>
|
||||
for (record <- metadataArguments) metadataRecords.add(record)
|
||||
}
|
||||
|
||||
val bootstrapMetadata = BootstrapMetadata.fromRecords(metadataRecords, "test harness")
|
||||
|
||||
props.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, metadataDir.getAbsolutePath)
|
||||
val proto = controllerListenerSecurityProtocol.toString
|
||||
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"CONTROLLER:$proto")
|
||||
|
@ -368,11 +338,24 @@ abstract class QuorumTestHarness extends Logging {
|
|||
// Setting the configuration to the same value set on the brokers via TestUtils to keep KRaft based and Zk based controller configs are consistent.
|
||||
props.setProperty(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "1000")
|
||||
val config = new KafkaConfig(props)
|
||||
|
||||
val formatter = new Formatter().
|
||||
setClusterId(Uuid.randomUuid().toString).
|
||||
setNodeId(nodeId)
|
||||
formatter.addDirectory(metadataDir.getAbsolutePath)
|
||||
formatter.setReleaseVersion(metadataVersion)
|
||||
formatter.setUnstableFeatureVersionsEnabled(true)
|
||||
formatter.setControllerListenerName(config.controllerListenerNames.head)
|
||||
formatter.setMetadataLogDirectory(config.metadataLogDir)
|
||||
addFormatterSettings(formatter)
|
||||
formatter.run()
|
||||
val bootstrapMetadata = formatter.bootstrapMetadata()
|
||||
|
||||
val controllerQuorumVotersFuture = new CompletableFuture[util.Map[Integer, InetSocketAddress]]
|
||||
val metaPropertiesEnsemble = new MetaPropertiesEnsemble.Loader().
|
||||
addMetadataLogDir(metadataDir.getAbsolutePath).
|
||||
load()
|
||||
metaPropertiesEnsemble.verify(Optional.of(metaProperties.clusterId().get()),
|
||||
metaPropertiesEnsemble.verify(Optional.of(formatter.clusterId()),
|
||||
OptionalInt.of(nodeId),
|
||||
util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR))
|
||||
val sharedServer = new SharedServer(
|
||||
|
@ -413,7 +396,7 @@ abstract class QuorumTestHarness extends Logging {
|
|||
faultHandlerFactory,
|
||||
metadataDir,
|
||||
controllerQuorumVotersFuture,
|
||||
metaProperties.clusterId.get(),
|
||||
formatter.clusterId(),
|
||||
this,
|
||||
faultHandler
|
||||
)
|
||||
|
|
|
@ -39,17 +39,17 @@ import org.apache.kafka.common.{DirectoryId, IsolationLevel, TopicIdPartition, T
|
|||
import org.apache.kafka.image.{MetadataDelta, MetadataImage}
|
||||
import org.apache.kafka.metadata.LeaderRecoveryState
|
||||
import org.apache.kafka.metadata.PartitionRegistration
|
||||
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesVersion}
|
||||
import org.apache.kafka.metadata.storage.Formatter
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
import org.apache.kafka.server.common.KRaftVersion
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs}
|
||||
import org.apache.kafka.server.common.{KRaftVersion, MetadataVersion}
|
||||
import org.apache.kafka.server.util.{MockTime, ShutdownableThread}
|
||||
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData, LogConfig, LogDirFailureChannel}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, Test}
|
||||
import org.mockito.Mockito
|
||||
|
||||
import scala.collection.{immutable, mutable}
|
||||
import scala.collection.mutable
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.util.Random
|
||||
|
||||
|
@ -159,12 +159,13 @@ class ReplicaManagerConcurrencyTest extends Logging {
|
|||
metadataCache: MetadataCache,
|
||||
): ReplicaManager = {
|
||||
val logDir = TestUtils.tempDir()
|
||||
val metaProperties = new MetaProperties.Builder().
|
||||
setVersion(MetaPropertiesVersion.V1).
|
||||
val formatter = new Formatter().
|
||||
setClusterId(Uuid.randomUuid().toString).
|
||||
setNodeId(1).
|
||||
build()
|
||||
TestUtils.formatDirectories(immutable.Seq(logDir.getAbsolutePath), metaProperties, MetadataVersion.latestTesting(), None)
|
||||
setNodeId(1)
|
||||
formatter.addDirectory(logDir.getAbsolutePath)
|
||||
formatter.setControllerListenerName("CONTROLLER")
|
||||
formatter.setMetadataLogDirectory(logDir.getAbsolutePath)
|
||||
formatter.run()
|
||||
|
||||
val props = new Properties
|
||||
props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "100@localhost:12345")
|
||||
|
|
|
@ -17,32 +17,25 @@
|
|||
|
||||
package kafka.tools
|
||||
|
||||
import java.io.{ByteArrayOutputStream, PrintStream}
|
||||
import java.io.{ByteArrayOutputStream, File, PrintStream}
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.nio.file.{Files, Paths}
|
||||
import java.nio.file.Files
|
||||
import java.util
|
||||
import java.util.{Collections, Properties}
|
||||
import org.apache.kafka.common.{DirectoryId, KafkaException}
|
||||
import java.util.Properties
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.Exit
|
||||
import kafka.utils.TestUtils
|
||||
import net.sourceforge.argparse4j.inf.Namespace
|
||||
import org.apache.commons.io.output.NullOutputStream
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, TestFeatureVersion}
|
||||
import org.apache.kafka.common.metadata.{FeatureLevelRecord, UserScramCredentialRecord}
|
||||
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
|
||||
import org.apache.kafka.server.common.Features
|
||||
import org.apache.kafka.metadata.properties.{MetaPropertiesEnsemble, PropertiesUtils}
|
||||
import org.apache.kafka.metadata.storage.FormatterException
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs}
|
||||
import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ServerLogConfigs}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue}
|
||||
import org.junit.jupiter.api.{Test, Timeout}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.{EnumSource, ValueSource}
|
||||
import org.mockito.Mockito
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
|
||||
import scala.collection.mutable
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
@Timeout(value = 40)
|
||||
class StorageToolTest {
|
||||
|
@ -146,7 +139,7 @@ Found problem:
|
|||
}
|
||||
|
||||
@Test
|
||||
def testInfoWithMismatchedSelfManagedKafkaConfig(): Unit = {
|
||||
def testInfoWithMismatchedKRaftConfig(): Unit = {
|
||||
val stream = new ByteArrayOutputStream()
|
||||
val tempDir = TestUtils.tempDir()
|
||||
try {
|
||||
|
@ -170,515 +163,282 @@ Found problem:
|
|||
} finally Utils.delete(tempDir)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatEmptyDirectory(): Unit = {
|
||||
val defaultStaticQuorumProperties = new Properties()
|
||||
defaultStaticQuorumProperties.setProperty("process.roles", "broker")
|
||||
defaultStaticQuorumProperties.setProperty("node.id", "0")
|
||||
defaultStaticQuorumProperties.setProperty("controller.listener.names", "CONTROLLER")
|
||||
defaultStaticQuorumProperties.setProperty("controller.quorum.voters", "100@localhost:9093")
|
||||
|
||||
val defaultDynamicQuorumProperties = new Properties()
|
||||
defaultDynamicQuorumProperties.setProperty("process.roles", "controller")
|
||||
defaultDynamicQuorumProperties.setProperty("node.id", "0")
|
||||
defaultDynamicQuorumProperties.setProperty("controller.listener.names", "CONTROLLER")
|
||||
defaultDynamicQuorumProperties.setProperty("controller.quorum.voters", "0@localhost:9093")
|
||||
defaultDynamicQuorumProperties.setProperty("listeners", "CONTROLLER://127.0.0.1:9093")
|
||||
defaultDynamicQuorumProperties.setProperty(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true")
|
||||
defaultDynamicQuorumProperties.setProperty(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG , "true")
|
||||
|
||||
private def runFormatCommand(
|
||||
stream: ByteArrayOutputStream,
|
||||
properties: Properties,
|
||||
extraArguments: Seq[String] = Seq(),
|
||||
ignoreFormatted: Boolean = false
|
||||
): Int = {
|
||||
val tempDir = TestUtils.tempDir()
|
||||
try {
|
||||
val metaProperties = new MetaProperties.Builder().
|
||||
setVersion(MetaPropertiesVersion.V1).
|
||||
setClusterId("XcZZOzUqS4yHOjhMQB6JLQ").
|
||||
setNodeId(2).
|
||||
build()
|
||||
val stream = new ByteArrayOutputStream()
|
||||
val bootstrapMetadata = StorageTool.buildBootstrapMetadata(MetadataVersion.latestTesting(), None, "test format command")
|
||||
assertEquals(0, StorageTool.
|
||||
formatCommand(new PrintStream(stream), Seq(tempDir.toString), metaProperties, bootstrapMetadata, MetadataVersion.latestTesting(), ignoreFormatted = false))
|
||||
assertTrue(stream.toString().startsWith("Formatting %s".format(tempDir)))
|
||||
|
||||
try assertEquals(1, StorageTool.
|
||||
formatCommand(new PrintStream(new ByteArrayOutputStream()), Seq(tempDir.toString), metaProperties, bootstrapMetadata, MetadataVersion.latestTesting(), ignoreFormatted = false)) catch {
|
||||
case e: TerseFailure => assertEquals(s"Log directory ${tempDir} is already " +
|
||||
"formatted. Use --ignore-formatted to ignore this directory and format the " +
|
||||
"others.", e.getMessage)
|
||||
val configPathString = new File(tempDir.getAbsolutePath(), "format.props").toString
|
||||
PropertiesUtils.writePropertiesFile(properties, configPathString, true)
|
||||
val arguments = ListBuffer[String]("format",
|
||||
"--cluster-id", "XcZZOzUqS4yHOjhMQB6JLQ")
|
||||
if (ignoreFormatted) {
|
||||
arguments += "--ignore-formatted"
|
||||
}
|
||||
|
||||
val stream2 = new ByteArrayOutputStream()
|
||||
assertEquals(0, StorageTool.
|
||||
formatCommand(new PrintStream(stream2), Seq(tempDir.toString), metaProperties, bootstrapMetadata, MetadataVersion.latestTesting(), ignoreFormatted = true))
|
||||
assertEquals("All of the log directories are already formatted.%n".format(), stream2.toString())
|
||||
} finally Utils.delete(tempDir)
|
||||
}
|
||||
|
||||
private def runFormatCommand(stream: ByteArrayOutputStream, directories: Seq[String], ignoreFormatted: Boolean = false): Int = {
|
||||
val metaProperties = new MetaProperties.Builder().
|
||||
setVersion(MetaPropertiesVersion.V1).
|
||||
setClusterId("XcZZOzUqS4yHOjhMQB6JLQ").
|
||||
setNodeId(2).
|
||||
build()
|
||||
val bootstrapMetadata = StorageTool.buildBootstrapMetadata(MetadataVersion.latestTesting(), None, "test format command")
|
||||
StorageTool.formatCommand(new PrintStream(stream), directories, metaProperties, bootstrapMetadata, MetadataVersion.latestTesting(), ignoreFormatted)
|
||||
arguments += "--config"
|
||||
arguments += configPathString
|
||||
extraArguments.foreach(arguments += _)
|
||||
StorageTool.execute(arguments.toArray, new PrintStream(stream))
|
||||
} finally {
|
||||
Utils.delete(tempDir)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatSucceedsIfAllDirectoriesAreAvailable(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir(), TestUtils.tempDir(), TestUtils.tempDir()).map(dir => dir.toString)
|
||||
val availableDirs = Seq(TestUtils.tempDir(), TestUtils.tempDir(), TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, availableDirs))
|
||||
val actual = stream.toString().split("\\r?\\n")
|
||||
val expect = availableDirs.map("Formatting %s".format(_))
|
||||
assertEquals(availableDirs.size, actual.size)
|
||||
expect.foreach(dir => {
|
||||
assertEquals(1, actual.count(_.startsWith(dir)))
|
||||
})
|
||||
assertEquals(0, runFormatCommand(stream, properties))
|
||||
|
||||
assertTrue(stream.toString().
|
||||
contains("Formatting metadata directory %s".format(availableDirs.head)),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
availableDirs.tail.foreach {
|
||||
dir => assertTrue(
|
||||
stream.toString().contains("Formatting data directory %s".format(dir)),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatSucceedsIfAtLeastOneDirectoryIsAvailable(): Unit = {
|
||||
val availableDir1 = TestUtils.tempDir()
|
||||
val unavailableDir1 = TestUtils.tempFile()
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", s"${availableDir1},${unavailableDir1}")
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, Seq(availableDir1.toString, unavailableDir1.toString)))
|
||||
assertTrue(stream.toString().contains("I/O error trying to read log directory %s. Ignoring...".format(unavailableDir1)))
|
||||
assertTrue(stream.toString().contains("Formatting %s".format(availableDir1)))
|
||||
assertFalse(stream.toString().contains("Formatting %s".format(unavailableDir1)))
|
||||
assertEquals(0, runFormatCommand(stream, properties))
|
||||
|
||||
assertTrue(stream.toString().contains("Formatting metadata directory %s".format(availableDir1)),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
|
||||
assertFalse(stream.toString().contains("Formatting log directory %s".format(unavailableDir1)),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
assertTrue(stream.toString().contains(
|
||||
"I/O error trying to read log directory %s. Ignoring...".format(unavailableDir1)),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatFailsOnAlreadyFormatted(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir(), TestUtils.tempDir(), TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", s"${availableDirs(0)}")
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, properties))
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream2 = new ByteArrayOutputStream()
|
||||
assertTrue(assertThrows(classOf[FormatterException],
|
||||
() => runFormatCommand(stream2, properties)).getMessage.contains(
|
||||
"already formatted. Use --ignore-formatted to ignore this directory and format the others"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testIgnoreFormatted(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir(), TestUtils.tempDir(), TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", s"${availableDirs(0)}")
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, properties))
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream2 = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream2, properties, Seq(), true))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatFailsIfAllDirectoriesAreUnavailable(): Unit = {
|
||||
val unavailableDir1 = TestUtils.tempFile()
|
||||
val unavailableDir2 = TestUtils.tempFile()
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", s"${unavailableDir1},${unavailableDir2}")
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals("No available log directories to format.", assertThrows(classOf[TerseFailure],
|
||||
() => runFormatCommand(stream, Seq(unavailableDir1.toString, unavailableDir2.toString))).getMessage)
|
||||
assertTrue(stream.toString().contains("I/O error trying to read log directory %s. Ignoring...".format(unavailableDir1)))
|
||||
assertTrue(stream.toString().contains("I/O error trying to read log directory %s. Ignoring...".format(unavailableDir2)))
|
||||
assertEquals("No available log directories to format.", assertThrows(classOf[FormatterException],
|
||||
() => runFormatCommand(stream, properties)).getMessage)
|
||||
assertTrue(stream.toString().contains(
|
||||
"I/O error trying to read log directory %s. Ignoring...".format(unavailableDir1)),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
assertTrue(stream.toString().contains(
|
||||
"I/O error trying to read log directory %s. Ignoring...".format(unavailableDir2)),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatSucceedsIfAtLeastOneFormattedDirectoryIsAvailable(): Unit = {
|
||||
val availableDir1 = TestUtils.tempDir()
|
||||
def testFormatFailsInZkMode(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
properties.setProperty("zookeeper.connect", "localhost:2181")
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, Seq(availableDir1.toString)))
|
||||
|
||||
val stream2 = new ByteArrayOutputStream()
|
||||
val unavailableDir1 = TestUtils.tempFile()
|
||||
assertEquals(0, runFormatCommand(stream2, Seq(availableDir1.toString, unavailableDir1.toString), ignoreFormatted = true))
|
||||
assertEquals("The kafka configuration file appears to be for a legacy cluster. " +
|
||||
"Formatting is only supported for clusters in KRaft mode.",
|
||||
assertThrows(classOf[TerseFailure],
|
||||
() => runFormatCommand(stream, properties)).getMessage)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testDefaultMetadataVersion(): Unit = {
|
||||
val namespace = StorageTool.parseArguments(Array("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ"))
|
||||
val mv = StorageTool.getMetadataVersion(namespace, Map.empty, defaultVersionString = None)
|
||||
assertEquals(MetadataVersion.LATEST_PRODUCTION.featureLevel(), mv.featureLevel(),
|
||||
"Expected the default metadata.version to be the latest production version")
|
||||
def testFormatWithReleaseVersion(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, properties, Seq("--release-version", "3.8-IV0")))
|
||||
assertTrue(stream.toString().contains("3.8-IV0"),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testConfiguredMetadataVersion(): Unit = {
|
||||
val namespace = StorageTool.parseArguments(Array("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ"))
|
||||
val mv = StorageTool.getMetadataVersion(namespace, Map.empty, defaultVersionString = Some(MetadataVersion.IBP_3_3_IV2.toString))
|
||||
assertEquals(MetadataVersion.IBP_3_3_IV2.featureLevel(), mv.featureLevel(),
|
||||
"Expected the default metadata.version to be 3.3-IV2")
|
||||
def testFormatWithReleaseVersionAsFeature(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, properties, Seq("--feature", "metadata.version=20")))
|
||||
assertTrue(stream.toString().contains("3.8-IV0"),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testSettingFeatureAndReleaseVersionFails(): Unit = {
|
||||
val namespace = StorageTool.parseArguments(Array("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ",
|
||||
"--release-version", "3.0-IV1", "--feature", "metadata.version=4"))
|
||||
assertThrows(classOf[IllegalArgumentException], () => StorageTool.getMetadataVersion(namespace, parseFeatures(namespace), defaultVersionString = None))
|
||||
def testFormatWithReleaseVersionAndKRaftVersion(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, properties, Seq(
|
||||
"--release-version", "3.7-IV0",
|
||||
"--feature", "kraft.version=0")))
|
||||
assertTrue(stream.toString().contains("3.7-IV0"),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testParseFeatures(): Unit = {
|
||||
def parseAddFeatures(strings: String*): Map[String, java.lang.Short] = {
|
||||
var args = mutable.Seq("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ")
|
||||
args ++= strings
|
||||
val namespace = StorageTool.parseArguments(args.toArray)
|
||||
parseFeatures(namespace)
|
||||
}
|
||||
|
||||
assertThrows(classOf[RuntimeException], () => parseAddFeatures("--feature", "blah"))
|
||||
assertThrows(classOf[RuntimeException], () => parseAddFeatures("--feature", "blah=blah"))
|
||||
|
||||
// Test with no features
|
||||
assertEquals(Map(), parseAddFeatures())
|
||||
|
||||
// Test with one feature
|
||||
val testFeatureLevel = 1
|
||||
val testArgument = TestFeatureVersion.FEATURE_NAME + "=" + testFeatureLevel.toString
|
||||
val expectedMap = Map(TestFeatureVersion.FEATURE_NAME -> testFeatureLevel.toShort)
|
||||
assertEquals(expectedMap, parseAddFeatures("--feature", testArgument))
|
||||
|
||||
// Test with two features
|
||||
val metadataFeatureLevel = 5
|
||||
val metadataArgument = MetadataVersion.FEATURE_NAME + "=" + metadataFeatureLevel.toString
|
||||
val expectedMap2 = expectedMap ++ Map (MetadataVersion.FEATURE_NAME -> metadataFeatureLevel.toShort)
|
||||
assertEquals(expectedMap2, parseAddFeatures("--feature", testArgument, "--feature", metadataArgument))
|
||||
}
|
||||
|
||||
private def parseFeatures(namespace: Namespace): Map[String, java.lang.Short] = {
|
||||
val specifiedFeatures: util.List[String] = namespace.getList("feature")
|
||||
StorageTool.featureNamesAndLevels(Option(specifiedFeatures).getOrElse(Collections.emptyList).asScala.toList)
|
||||
def testFormatWithReleaseVersionDefault(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
properties.setProperty("inter.broker.protocol.version", "3.7")
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, properties))
|
||||
assertTrue(stream.toString().contains("3.7-IV4"),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testMetadataVersionFlags(): Unit = {
|
||||
def parseMetadataVersion(strings: String*): MetadataVersion = {
|
||||
var args = mutable.Seq("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ")
|
||||
args ++= strings
|
||||
val namespace = StorageTool.parseArguments(args.toArray)
|
||||
StorageTool.getMetadataVersion(namespace, Map.empty, defaultVersionString = None)
|
||||
}
|
||||
|
||||
var mv = parseMetadataVersion("--release-version", "3.0")
|
||||
assertEquals("3.0", mv.shortVersion())
|
||||
|
||||
mv = parseMetadataVersion("--release-version", "3.0-IV1")
|
||||
assertEquals(MetadataVersion.IBP_3_0_IV1, mv)
|
||||
|
||||
assertThrows(classOf[IllegalArgumentException], () => parseMetadataVersion("--release-version", "0.0"))
|
||||
}
|
||||
|
||||
private def generateRecord(featureName: String, level: Short): ApiMessageAndVersion = {
|
||||
new ApiMessageAndVersion(new FeatureLevelRecord().
|
||||
setName(featureName).
|
||||
setFeatureLevel(level), 0.toShort)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(classOf[TestFeatureVersion])
|
||||
def testFeatureFlag(testFeatureVersion: TestFeatureVersion): Unit = {
|
||||
val featureLevel = testFeatureVersion.featureLevel
|
||||
if (featureLevel <= Features.TEST_VERSION.defaultValue(MetadataVersion.LATEST_PRODUCTION)) {
|
||||
val records = new ArrayBuffer[ApiMessageAndVersion]()
|
||||
StorageTool.generateFeatureRecords(
|
||||
records,
|
||||
MetadataVersion.LATEST_PRODUCTION,
|
||||
Map(TestFeatureVersion.FEATURE_NAME -> featureLevel),
|
||||
allFeatures,
|
||||
false,
|
||||
false
|
||||
)
|
||||
if (featureLevel > 0) {
|
||||
assertEquals(List(generateRecord(TestFeatureVersion.FEATURE_NAME, featureLevel)), records)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(classOf[MetadataVersion])
|
||||
def testVersionDefault(metadataVersion: MetadataVersion): Unit = {
|
||||
val records = new ArrayBuffer[ApiMessageAndVersion]()
|
||||
StorageTool.generateFeatureRecords(
|
||||
records,
|
||||
metadataVersion,
|
||||
Map.empty,
|
||||
allFeatures,
|
||||
true,
|
||||
true
|
||||
)
|
||||
|
||||
val expectedRecords = new ArrayBuffer[ApiMessageAndVersion]()
|
||||
|
||||
def maybeAddRecordFor(features: Features): Unit = {
|
||||
val featureLevel = features.defaultValue(metadataVersion)
|
||||
if (featureLevel > 0) {
|
||||
expectedRecords += generateRecord(features.featureName, featureLevel)
|
||||
}
|
||||
}
|
||||
|
||||
Features.FEATURES.foreach(maybeAddRecordFor)
|
||||
|
||||
assertEquals(expectedRecords, records)
|
||||
}
|
||||
@Test
|
||||
def testVersionDefaultNoArgs(): Unit = {
|
||||
val records = new ArrayBuffer[ApiMessageAndVersion]()
|
||||
StorageTool.generateFeatureRecords(
|
||||
records,
|
||||
MetadataVersion.LATEST_PRODUCTION,
|
||||
Map.empty,
|
||||
allFeatures,
|
||||
false,
|
||||
false
|
||||
)
|
||||
|
||||
assertEquals(List(generateRecord(TestFeatureVersion.FEATURE_NAME, Features.TEST_VERSION.defaultValue(MetadataVersion.LATEST_PRODUCTION))), records)
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
def testFeatureDependency(): Unit = {
|
||||
val featureLevel = 1.toShort
|
||||
assertThrows(classOf[TerseFailure], () => StorageTool.generateFeatureRecords(
|
||||
new ArrayBuffer[ApiMessageAndVersion](),
|
||||
MetadataVersion.IBP_2_8_IV1,
|
||||
Map(TestFeatureVersion.FEATURE_NAME -> featureLevel),
|
||||
allFeatures,
|
||||
false,
|
||||
false
|
||||
))
|
||||
def testFormatWithReleaseVersionDefaultAndReleaseVersion(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
properties.setProperty("inter.broker.protocol.version", "3.7")
|
||||
val stream = new ByteArrayOutputStream()
|
||||
assertEquals(0, runFormatCommand(stream, properties, Seq(
|
||||
"--release-version", "3.6-IV0",
|
||||
"--feature", "kraft.version=0")))
|
||||
assertTrue(stream.toString().contains("3.6-IV0"),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testLatestFeaturesWithOldMetadataVersion(): Unit = {
|
||||
val records = new ArrayBuffer[ApiMessageAndVersion]()
|
||||
StorageTool.generateFeatureRecords(
|
||||
records,
|
||||
MetadataVersion.IBP_3_3_IV0,
|
||||
Map.empty,
|
||||
allFeatures,
|
||||
false,
|
||||
false
|
||||
)
|
||||
|
||||
assertEquals(List(generateRecord(TestFeatureVersion.FEATURE_NAME, Features.TEST_VERSION.defaultValue(MetadataVersion.LATEST_PRODUCTION))), records)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFeatureInvalidFlag(): Unit = {
|
||||
val featureLevel = 99.toShort
|
||||
assertThrows(classOf[IllegalArgumentException], () => StorageTool.generateFeatureRecords(
|
||||
new ArrayBuffer[ApiMessageAndVersion](),
|
||||
MetadataVersion.LATEST_PRODUCTION,
|
||||
Map(TestFeatureVersion.FEATURE_NAME -> featureLevel),
|
||||
allFeatures,
|
||||
false,
|
||||
false
|
||||
))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testUnstableFeatureThrowsError(): Unit = {
|
||||
assertThrows(classOf[IllegalArgumentException], () => StorageTool.generateFeatureRecords(
|
||||
new ArrayBuffer[ApiMessageAndVersion](),
|
||||
MetadataVersion.LATEST_PRODUCTION,
|
||||
Map(TestFeatureVersion.FEATURE_NAME -> Features.TEST_VERSION.latestTesting),
|
||||
allFeatures,
|
||||
false,
|
||||
false
|
||||
))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testAddScram():Unit = {
|
||||
def parseAddScram(strings: String*): Option[ArrayBuffer[UserScramCredentialRecord]] = {
|
||||
var args = mutable.Seq("format", "-c", "config.props", "-t", "XcZZOzUqS4yHOjhMQB6JLQ")
|
||||
args ++= strings
|
||||
val namespace = StorageTool.parseArguments(args.toArray)
|
||||
StorageTool.getUserScramCredentialRecords(namespace)
|
||||
}
|
||||
|
||||
var scramRecords = parseAddScram()
|
||||
assertEquals(None, scramRecords)
|
||||
|
||||
// Validate we can add multiple SCRAM creds.
|
||||
scramRecords = parseAddScram("-S",
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\",iterations=8192]",
|
||||
"-S",
|
||||
"SCRAM-SHA-256=[name=george,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\",iterations=8192]")
|
||||
|
||||
assertEquals(2, scramRecords.get.size)
|
||||
|
||||
// Require name subfield.
|
||||
try assertEquals(1, parseAddScram("-S",
|
||||
"SCRAM-SHA-256=[salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\",iterations=8192]")) catch {
|
||||
case e: TerseFailure => assertEquals(s"You must supply 'name' to add-scram", e.getMessage)
|
||||
}
|
||||
|
||||
// Require password xor saltedpassword
|
||||
try assertEquals(1, parseAddScram("-S",
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",password=alice,saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\",iterations=8192]"))
|
||||
catch {
|
||||
case e: TerseFailure => assertEquals(s"You must only supply one of 'password' or 'saltedpassword' to add-scram", e.getMessage)
|
||||
}
|
||||
|
||||
try assertEquals(1, parseAddScram("-S",
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",iterations=8192]"))
|
||||
catch {
|
||||
case e: TerseFailure => assertEquals(s"You must supply one of 'password' or 'saltedpassword' to add-scram", e.getMessage)
|
||||
}
|
||||
|
||||
// Validate salt is required with saltedpassword
|
||||
try assertEquals(1, parseAddScram("-S",
|
||||
"SCRAM-SHA-256=[name=alice,saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\",iterations=8192]"))
|
||||
catch {
|
||||
case e: TerseFailure => assertEquals(s"You must supply 'salt' with 'saltedpassword' to add-scram", e.getMessage)
|
||||
}
|
||||
|
||||
// Validate salt is optional with password
|
||||
assertEquals(1, parseAddScram("-S", "SCRAM-SHA-256=[name=alice,password=alice,iterations=4096]").get.size)
|
||||
|
||||
// Require 4096 <= iterations <= 16384
|
||||
try assertEquals(1, parseAddScram("-S",
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",password=alice,iterations=16385]"))
|
||||
catch {
|
||||
case e: TerseFailure => assertEquals(s"The 'iterations' value must be <= 16384 for add-scram", e.getMessage)
|
||||
}
|
||||
|
||||
assertEquals(1, parseAddScram("-S",
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",password=alice,iterations=16384]")
|
||||
.get.size)
|
||||
|
||||
try assertEquals(1, parseAddScram("-S",
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",password=alice,iterations=4095]"))
|
||||
catch {
|
||||
case e: TerseFailure => assertEquals(s"The 'iterations' value must be >= 4096 for add-scram", e.getMessage)
|
||||
}
|
||||
|
||||
assertEquals(1, parseAddScram("-S",
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",password=alice,iterations=4096]")
|
||||
.get.size)
|
||||
|
||||
// Validate iterations is optional
|
||||
assertEquals(1, parseAddScram("-S", "SCRAM-SHA-256=[name=alice,password=alice]") .get.size)
|
||||
}
|
||||
|
||||
class StorageToolTestException(message: String) extends KafkaException(message) {
|
||||
}
|
||||
|
||||
@Test
|
||||
def testScramWithBadMetadataVersion(): Unit = {
|
||||
var exitString: String = ""
|
||||
def exitProcedure(exitStatus: Int, message: Option[String]) : Nothing = {
|
||||
exitString = message.getOrElse("")
|
||||
throw new StorageToolTestException(exitString)
|
||||
}
|
||||
Exit.setExitProcedure(exitProcedure)
|
||||
|
||||
val properties = newSelfManagedProperties()
|
||||
val propsFile = TestUtils.tempFile()
|
||||
val propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
properties.store(propsStream, "config.props")
|
||||
propsStream.close()
|
||||
|
||||
val args = Array("format", "-c", s"${propsFile.toPath}", "-t", "XcZZOzUqS4yHOjhMQB6JLQ", "--release-version", "3.4", "-S",
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\",password=alice,iterations=8192]")
|
||||
|
||||
try {
|
||||
assertEquals(1, StorageTool.main(args))
|
||||
} catch {
|
||||
case e: StorageToolTestException => assertEquals(s"SCRAM is only supported in metadata.version ${MetadataVersion.IBP_3_5_IV2} or later.", exitString)
|
||||
} finally {
|
||||
Exit.resetExitProcedure()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testNoScramWithMetadataVersion(): Unit = {
|
||||
var exitString: String = ""
|
||||
var exitStatus: Int = 1
|
||||
def exitProcedure(status: Int, message: Option[String]) : Nothing = {
|
||||
exitStatus = status
|
||||
exitString = message.getOrElse("")
|
||||
throw new StorageToolTestException(exitString)
|
||||
}
|
||||
Exit.setExitProcedure(exitProcedure)
|
||||
|
||||
val properties = newSelfManagedProperties()
|
||||
val propsFile = TestUtils.tempFile()
|
||||
val propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
// This test does format the directory specified so use a tempdir
|
||||
properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().toString)
|
||||
properties.store(propsStream, "config.props")
|
||||
propsStream.close()
|
||||
|
||||
val args = Array("format", "-c", s"${propsFile.toPath}", "-t", "XcZZOzUqS4yHOjhMQB6JLQ", "--release-version", "3.4")
|
||||
|
||||
try {
|
||||
StorageTool.main(args)
|
||||
} catch {
|
||||
case e: StorageToolTestException => assertEquals("", exitString)
|
||||
assertEquals(0, exitStatus)
|
||||
} finally {
|
||||
Exit.resetExitProcedure()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testDirUuidGeneration(): Unit = {
|
||||
val tempDir = TestUtils.tempDir()
|
||||
try {
|
||||
val metaProperties = new MetaProperties.Builder().
|
||||
setClusterId("XcZZOzUqS4yHOjhMQB6JLQ").
|
||||
setNodeId(2).
|
||||
build()
|
||||
val bootstrapMetadata = StorageTool.
|
||||
buildBootstrapMetadata(MetadataVersion.latestTesting(), None, "test format command")
|
||||
assertEquals(0, StorageTool.
|
||||
formatCommand(new PrintStream(NullOutputStream.NULL_OUTPUT_STREAM), Seq(tempDir.toString), metaProperties, bootstrapMetadata, MetadataVersion.latestTesting(), ignoreFormatted = false))
|
||||
|
||||
val metaPropertiesFile = Paths.get(tempDir.toURI).resolve(MetaPropertiesEnsemble.META_PROPERTIES_NAME).toFile
|
||||
assertTrue(metaPropertiesFile.exists())
|
||||
val metaProps = new MetaProperties.Builder(
|
||||
PropertiesUtils.readPropertiesFile(metaPropertiesFile.getAbsolutePath())).
|
||||
build()
|
||||
assertTrue(metaProps.directoryId().isPresent())
|
||||
assertFalse(DirectoryId.reserved(metaProps.directoryId().get()))
|
||||
} finally Utils.delete(tempDir)
|
||||
def testFormatWithStandaloneFlagOnBrokerFails(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultStaticQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
val arguments = ListBuffer[String]("--release-version", "3.9-IV0", "--standalone")
|
||||
assertEquals("You cannot use --standalone on a broker node.",
|
||||
assertThrows(classOf[TerseFailure],
|
||||
() => runFormatCommand(stream, properties, arguments.toSeq)).getMessage)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(false, true))
|
||||
def testFormattingUnstableMetadataVersionBlocked(enableUnstable: Boolean): Unit = {
|
||||
var exitString: String = ""
|
||||
var exitStatus: Int = 1
|
||||
def exitProcedure(status: Int, message: Option[String]) : Nothing = {
|
||||
exitStatus = status
|
||||
exitString = message.getOrElse("")
|
||||
throw new StorageToolTestException(exitString)
|
||||
}
|
||||
Exit.setExitProcedure(exitProcedure)
|
||||
val properties = newSelfManagedProperties()
|
||||
val propsFile = TestUtils.tempFile()
|
||||
val propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
try {
|
||||
properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().toString)
|
||||
properties.setProperty(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG, enableUnstable.toString)
|
||||
properties.store(propsStream, "config.props")
|
||||
} finally {
|
||||
propsStream.close()
|
||||
}
|
||||
val args = Array("format", "-c", s"${propsFile.toPath}",
|
||||
"-t", "XcZZOzUqS4yHOjhMQB6JLQ",
|
||||
"--release-version", MetadataVersion.latestTesting().toString)
|
||||
try {
|
||||
StorageTool.main(args)
|
||||
} catch {
|
||||
case _: StorageToolTestException =>
|
||||
} finally {
|
||||
Exit.resetExitProcedure()
|
||||
}
|
||||
if (enableUnstable) {
|
||||
assertEquals("", exitString)
|
||||
assertEquals(0, exitStatus)
|
||||
} else {
|
||||
assertEquals(s"The metadata.version ${MetadataVersion.latestTesting().toString} is not ready for " +
|
||||
"production use yet.", exitString)
|
||||
assertEquals(1, exitStatus)
|
||||
def testFormatWithStandaloneFlag(setKraftVersionFeature: Boolean): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultDynamicQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
val arguments = ListBuffer[String]("--release-version", "3.9-IV0", "--standalone")
|
||||
if (setKraftVersionFeature) {
|
||||
arguments += "--feature"
|
||||
arguments += "kraft.version=1"
|
||||
}
|
||||
assertEquals(0, runFormatCommand(stream, properties, arguments.toSeq))
|
||||
assertTrue(stream.toString().
|
||||
contains("Formatting dynamic metadata voter directory %s".format(availableDirs.head)),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testFormatValidatesConfigForMetadataVersion(): Unit = {
|
||||
val config = Mockito.spy(new KafkaConfig(TestUtils.createBrokerConfig(10, null)))
|
||||
val args = Array("format",
|
||||
"-c", "dummy.properties",
|
||||
"-t", "XcZZOzUqS4yHOjhMQB6JLQ",
|
||||
"--release-version", MetadataVersion.LATEST_PRODUCTION.toString)
|
||||
val exitCode = StorageTool.runFormatCommand(StorageTool.parseArguments(args), config)
|
||||
Mockito.verify(config, Mockito.times(1)).validateWithMetadataVersion(MetadataVersion.LATEST_PRODUCTION)
|
||||
assertEquals(0, exitCode)
|
||||
def testFormatWithStandaloneFlagAndInitialControllersFlagFails(): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultDynamicQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
val arguments = ListBuffer[String](
|
||||
"--release-version", "3.9-IV0",
|
||||
"--standalone", "--initial-controllers",
|
||||
"0@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ," +
|
||||
"1@localhost:8030:aUARLskQTCW4qCZDtS_cwA," +
|
||||
"2@localhost:8040:2ggvsS4kQb-fSJ_-zC_Ang")
|
||||
assertEquals(1, runFormatCommand(stream, properties, arguments.toSeq))
|
||||
assertTrue(stream.toString().contains("net.sourceforge.argparse4j.inf.ArgumentParserException: " +
|
||||
"argument --initial-controllers/-I: not allowed with argument --standalone/-s"),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testJbodSupportValidation(): Unit = {
|
||||
def formatWith(logDirCount: Int, metadataVersion: MetadataVersion): Integer = {
|
||||
val properties = TestUtils.createBrokerConfig(10, null, logDirCount = logDirCount)
|
||||
properties.remove(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)
|
||||
val configFile = TestUtils.tempPropertiesFile(properties.asScala.toMap).toPath.toString
|
||||
StorageTool.execute(Array("format",
|
||||
"-c", configFile,
|
||||
"-t", "XcZZOzUqS4yHOjhMQB6JLQ",
|
||||
"--release-version", metadataVersion.toString))
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(false, true))
|
||||
def testFormatWithInitialControllersFlag(setKraftVersionFeature: Boolean): Unit = {
|
||||
val availableDirs = Seq(TestUtils.tempDir())
|
||||
val properties = new Properties()
|
||||
properties.putAll(defaultDynamicQuorumProperties)
|
||||
properties.setProperty("log.dirs", availableDirs.mkString(","))
|
||||
val stream = new ByteArrayOutputStream()
|
||||
val arguments = ListBuffer[String](
|
||||
"--release-version", "3.9-IV0",
|
||||
"--initial-controllers",
|
||||
"0@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ," +
|
||||
"1@localhost:8030:aUARLskQTCW4qCZDtS_cwA," +
|
||||
"2@localhost:8040:2ggvsS4kQb-fSJ_-zC_Ang")
|
||||
if (setKraftVersionFeature) {
|
||||
arguments += "--feature"
|
||||
arguments += "kraft.version=1"
|
||||
}
|
||||
|
||||
assertEquals(0, formatWith(1, MetadataVersion.IBP_3_6_IV2))
|
||||
assertEquals("Invalid configuration for metadata version: " +
|
||||
"requirement failed: Multiple log directories (aka JBOD) are not supported in the current MetadataVersion 3.6-IV2. Need 3.7-IV2 or higher",
|
||||
assertThrows(classOf[TerseFailure], () => formatWith(2, MetadataVersion.IBP_3_6_IV2)).getMessage)
|
||||
assertEquals(0, formatWith(1, MetadataVersion.IBP_3_7_IV2))
|
||||
assertEquals(0, formatWith(2, MetadataVersion.IBP_3_7_IV2))
|
||||
assertEquals(0, runFormatCommand(stream, properties, arguments.toSeq))
|
||||
assertTrue(stream.toString().
|
||||
contains("Formatting dynamic metadata voter directory %s".format(availableDirs.head)),
|
||||
"Failed to find content in output: " + stream.toString())
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import kafka.network.RequestChannel
|
|||
import kafka.server._
|
||||
import kafka.server.checkpoints.OffsetCheckpointFile
|
||||
import kafka.server.metadata.{ConfigRepository, MockConfigRepository}
|
||||
import kafka.tools.StorageTool
|
||||
import kafka.utils.Implicits._
|
||||
import kafka.zk._
|
||||
import org.apache.kafka.clients.admin.AlterConfigOp.OpType
|
||||
|
@ -54,12 +53,11 @@ import org.apache.kafka.common.utils.Utils.formatAddress
|
|||
import org.apache.kafka.common.utils.{Time, Utils}
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
|
||||
import org.apache.kafka.metadata.properties.MetaProperties
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.queue.KafkaEventQueue
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer}
|
||||
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs}
|
||||
import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||
import org.apache.kafka.server.util.MockTime
|
||||
|
@ -86,7 +84,7 @@ import java.util.concurrent.atomic.AtomicBoolean
|
|||
import java.util.{Arrays, Collections, Optional, Properties}
|
||||
import scala.annotation.nowarn
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.collection.{Map, Seq, immutable, mutable}
|
||||
import scala.collection.{Map, Seq, mutable}
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.{Await, ExecutionContext, Future}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -1168,27 +1166,6 @@ object TestUtils extends Logging {
|
|||
assertEquals(0, threadCount, s"Found unexpected $threadCount NonDaemon threads=${nonDaemonThreads.map(t => t.getName).mkString(", ")}")
|
||||
}
|
||||
|
||||
def formatDirectories(
|
||||
directories: immutable.Seq[String],
|
||||
metaProperties: MetaProperties,
|
||||
metadataVersion: MetadataVersion,
|
||||
optionalMetadataRecords: Option[ArrayBuffer[ApiMessageAndVersion]]
|
||||
): Unit = {
|
||||
val stream = new ByteArrayOutputStream()
|
||||
var out: PrintStream = null
|
||||
try {
|
||||
out = new PrintStream(stream)
|
||||
val bootstrapMetadata = StorageTool.buildBootstrapMetadata(metadataVersion, optionalMetadataRecords, "format command")
|
||||
if (StorageTool.formatCommand(out, directories, metaProperties, bootstrapMetadata, metadataVersion, ignoreFormatted = false) != 0) {
|
||||
throw new RuntimeException(stream.toString())
|
||||
}
|
||||
debug(s"Formatted storage directory(ies) ${directories}")
|
||||
} finally {
|
||||
if (out != null) out.close()
|
||||
stream.close()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create new LogManager instance with default configuration for testing
|
||||
*/
|
||||
|
|
|
@ -20,10 +20,13 @@ package org.apache.kafka.metadata.bootstrap;
|
|||
import org.apache.kafka.common.metadata.FeatureLevelRecord;
|
||||
import org.apache.kafka.common.protocol.ApiMessage;
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
|
@ -40,6 +43,36 @@ public class BootstrapMetadata {
|
|||
private final MetadataVersion metadataVersion;
|
||||
private final String source;
|
||||
|
||||
public static BootstrapMetadata fromVersions(
|
||||
MetadataVersion metadataVersion,
|
||||
Map<String, Short> featureVersions,
|
||||
String source
|
||||
) {
|
||||
List<ApiMessageAndVersion> records = new ArrayList<>();
|
||||
records.add(new ApiMessageAndVersion(new FeatureLevelRecord().
|
||||
setName(MetadataVersion.FEATURE_NAME).
|
||||
setFeatureLevel(metadataVersion.featureLevel()), (short) 0));
|
||||
List<String> featureNames = new ArrayList<>(featureVersions.size());
|
||||
featureVersions.keySet().forEach(n -> {
|
||||
// metadata.version is handled in a special way, and kraft.version generates no
|
||||
// FeatureLevelRecord.
|
||||
if (!(n.equals(MetadataVersion.FEATURE_NAME) ||
|
||||
n.equals(KRaftVersion.FEATURE_NAME))) {
|
||||
featureNames.add(n);
|
||||
}
|
||||
});
|
||||
featureNames.sort(String::compareTo);
|
||||
for (String featureName : featureNames) {
|
||||
short level = featureVersions.get(featureName);
|
||||
if (level > 0) {
|
||||
records.add(new ApiMessageAndVersion(new FeatureLevelRecord().
|
||||
setName(featureName).
|
||||
setFeatureLevel(level), (short) 0));
|
||||
}
|
||||
}
|
||||
return new BootstrapMetadata(records, metadataVersion, source);
|
||||
}
|
||||
|
||||
public static BootstrapMetadata fromVersion(MetadataVersion metadataVersion, String source) {
|
||||
List<ApiMessageAndVersion> records = Collections.singletonList(
|
||||
new ApiMessageAndVersion(new FeatureLevelRecord().
|
||||
|
|
|
@ -0,0 +1,506 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.metadata.storage;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.metadata.MetadataRecordSerde;
|
||||
import org.apache.kafka.metadata.bootstrap.BootstrapDirectory;
|
||||
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
|
||||
import org.apache.kafka.metadata.properties.MetaProperties;
|
||||
import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble;
|
||||
import org.apache.kafka.metadata.properties.MetaPropertiesVersion;
|
||||
import org.apache.kafka.raft.DynamicVoters;
|
||||
import org.apache.kafka.raft.KafkaRaftClient;
|
||||
import org.apache.kafka.raft.internals.VoterSet;
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||
import org.apache.kafka.server.common.FeatureVersion;
|
||||
import org.apache.kafka.server.common.Features;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.apache.kafka.snapshot.FileRawSnapshotWriter;
|
||||
import org.apache.kafka.snapshot.RecordsSnapshotWriter;
|
||||
import org.apache.kafka.snapshot.Snapshots;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.PrintStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.kafka.common.internals.Topic.CLUSTER_METADATA_TOPIC_PARTITION;
|
||||
import static org.apache.kafka.server.common.KRaftVersion.KRAFT_VERSION_0;
|
||||
import static org.apache.kafka.server.common.KRaftVersion.KRAFT_VERSION_1;
|
||||
|
||||
/**
|
||||
* Formats storage directories.
|
||||
*/
|
||||
public class Formatter {
|
||||
/**
|
||||
* The stream to log to while formatting.
|
||||
*/
|
||||
private PrintStream printStream = System.out;
|
||||
|
||||
/**
|
||||
* The features that are supported.
|
||||
*/
|
||||
private List<Features> supportedFeatures = Features.PRODUCTION_FEATURES;
|
||||
|
||||
/**
|
||||
* The current node id.
|
||||
*/
|
||||
private int nodeId = -1;
|
||||
|
||||
/**
|
||||
* The cluster ID to use.
|
||||
*/
|
||||
private String clusterId = null;
|
||||
|
||||
/**
|
||||
* The directories to format.
|
||||
*/
|
||||
private final TreeSet<String> directories = new TreeSet<>();
|
||||
|
||||
/**
|
||||
* The metadata version to use.
|
||||
*/
|
||||
private MetadataVersion releaseVersion = null;
|
||||
|
||||
/**
|
||||
* Maps feature names to the level they will start off with.
|
||||
*/
|
||||
private Map<String, Short> featureLevels = new TreeMap<>();
|
||||
|
||||
/**
|
||||
* The bootstrap metadata used to format the cluster.
|
||||
*/
|
||||
private BootstrapMetadata bootstrapMetadata;
|
||||
|
||||
/**
|
||||
* True if we should enable unstable feature versions.
|
||||
*/
|
||||
private boolean unstableFeatureVersionsEnabled = false;
|
||||
|
||||
/**
|
||||
* True if we should ignore already formatted directories.
|
||||
*/
|
||||
private boolean ignoreFormatted = false;
|
||||
|
||||
/**
|
||||
* The arguments passed to --add-scram
|
||||
*/
|
||||
private List<String> scramArguments = Collections.emptyList();
|
||||
|
||||
/**
|
||||
* The name of the initial controller listener.
|
||||
*/
|
||||
private String controllerListenerName = null;
|
||||
|
||||
/**
|
||||
* The metadata log directory.
|
||||
*/
|
||||
private String metadataLogDirectory = null;
|
||||
|
||||
/**
|
||||
* The initial KIP-853 voters.
|
||||
*/
|
||||
private Optional<DynamicVoters> initialControllers = Optional.empty();
|
||||
|
||||
public Formatter setPrintStream(PrintStream printStream) {
|
||||
this.printStream = printStream;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setSupportedFeatures(List<Features> supportedFeatures) {
|
||||
this.supportedFeatures = supportedFeatures;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setNodeId(int nodeId) {
|
||||
this.nodeId = nodeId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setClusterId(String clusterId) {
|
||||
this.clusterId = clusterId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public String clusterId() {
|
||||
return clusterId;
|
||||
}
|
||||
|
||||
public Formatter setDirectories(Collection<String> directories) {
|
||||
this.directories.clear();
|
||||
this.directories.addAll(directories);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter addDirectory(String directory) {
|
||||
this.directories.add(directory);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setReleaseVersion(MetadataVersion releaseVersion) {
|
||||
this.releaseVersion = releaseVersion;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setFeatureLevel(String featureName, Short level) {
|
||||
this.featureLevels.put(featureName, level);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setUnstableFeatureVersionsEnabled(boolean unstableFeatureVersionsEnabled) {
|
||||
this.unstableFeatureVersionsEnabled = unstableFeatureVersionsEnabled;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setIgnoreFormatted(boolean ignoreFormatted) {
|
||||
this.ignoreFormatted = ignoreFormatted;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setScramArguments(List<String> scramArguments) {
|
||||
this.scramArguments = scramArguments;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setControllerListenerName(String controllerListenerName) {
|
||||
this.controllerListenerName = controllerListenerName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setMetadataLogDirectory(String metadataLogDirectory) {
|
||||
this.metadataLogDirectory = metadataLogDirectory;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Formatter setInitialVoters(DynamicVoters initialControllers) {
|
||||
this.initialControllers = Optional.of(initialControllers);
|
||||
return this;
|
||||
}
|
||||
|
||||
boolean hasDynamicQuorum() {
|
||||
if (initialControllers.isPresent()) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public BootstrapMetadata bootstrapMetadata() {
|
||||
return bootstrapMetadata;
|
||||
}
|
||||
|
||||
public void run() throws Exception {
|
||||
if (nodeId < 0) {
|
||||
throw new RuntimeException("You must specify a valid non-negative node ID.");
|
||||
}
|
||||
if (clusterId == null) {
|
||||
throw new FormatterException("You must specify the cluster id.");
|
||||
}
|
||||
if (directories.isEmpty()) {
|
||||
throw new FormatterException("You must specify at least one directory to format");
|
||||
}
|
||||
if (controllerListenerName == null) {
|
||||
throw new FormatterException("You must specify the name of the initial controller listener.");
|
||||
}
|
||||
if (metadataLogDirectory == null) {
|
||||
throw new FormatterException("You must specify the metadata log directory.");
|
||||
}
|
||||
if (!directories.contains(metadataLogDirectory)) {
|
||||
throw new FormatterException("The specified metadata log directory, " + metadataLogDirectory +
|
||||
" was not one of the given directories: " + directories);
|
||||
}
|
||||
releaseVersion = calculateEffectiveReleaseVersion();
|
||||
featureLevels = calculateEffectiveFeatureLevels();
|
||||
this.bootstrapMetadata = calculateBootstrapMetadata();
|
||||
doFormat(bootstrapMetadata);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate the effective value of release version. This will be used to set defaults
|
||||
* for the other features. We also throw an exception if something inconsistent was requested.
|
||||
*
|
||||
* @return The effective value of release version.
|
||||
*/
|
||||
MetadataVersion calculateEffectiveReleaseVersion() {
|
||||
if (featureLevels.containsKey(MetadataVersion.FEATURE_NAME)) {
|
||||
if (releaseVersion != null) {
|
||||
throw new FormatterException("Use --release-version instead of " +
|
||||
"--feature " + MetadataVersion.FEATURE_NAME + "=X to avoid ambiguity.");
|
||||
}
|
||||
return verifyReleaseVersion(MetadataVersion.fromFeatureLevel(
|
||||
featureLevels.get(MetadataVersion.FEATURE_NAME)));
|
||||
} else if (releaseVersion != null) {
|
||||
return verifyReleaseVersion(releaseVersion);
|
||||
} else if (unstableFeatureVersionsEnabled) {
|
||||
return MetadataVersion.latestTesting();
|
||||
} else {
|
||||
return MetadataVersion.latestProduction();
|
||||
}
|
||||
}
|
||||
|
||||
MetadataVersion verifyReleaseVersion(MetadataVersion metadataVersion) {
|
||||
if (!metadataVersion.isKRaftSupported()) {
|
||||
throw new FormatterException(MetadataVersion.FEATURE_NAME + " " + metadataVersion +
|
||||
" is too old to be supported.");
|
||||
}
|
||||
if (!unstableFeatureVersionsEnabled) {
|
||||
if (!metadataVersion.isProduction()) {
|
||||
throw new FormatterException(MetadataVersion.FEATURE_NAME + " " + metadataVersion +
|
||||
" is not yet stable.");
|
||||
}
|
||||
}
|
||||
return metadataVersion;
|
||||
}
|
||||
|
||||
Map<String, Short> calculateEffectiveFeatureLevels() {
|
||||
Map<String, Features> nameToSupportedFeature = new TreeMap<>();
|
||||
supportedFeatures.forEach(feature -> nameToSupportedFeature.put(feature.featureName(), feature));
|
||||
Map<String, Short> newFeatureLevels = new TreeMap<>();
|
||||
// Verify that all specified features are known to us.
|
||||
for (Map.Entry<String, Short> entry : featureLevels.entrySet()) {
|
||||
String featureName = entry.getKey();
|
||||
short level = entry.getValue();
|
||||
if (!featureName.equals(MetadataVersion.FEATURE_NAME)) {
|
||||
if (!nameToSupportedFeature.containsKey(featureName)) {
|
||||
throw new FormatterException("Unsupported feature: " + featureName +
|
||||
". Supported features are: " + nameToSupportedFeature.keySet().stream().
|
||||
collect(Collectors.joining(", ")));
|
||||
}
|
||||
}
|
||||
newFeatureLevels.put(featureName, level);
|
||||
}
|
||||
newFeatureLevels.put(MetadataVersion.FEATURE_NAME, releaseVersion.featureLevel());
|
||||
// Add default values for features that were not specified.
|
||||
supportedFeatures.forEach(supportedFeature -> {
|
||||
if (supportedFeature.featureName().equals(KRaftVersion.FEATURE_NAME)) {
|
||||
newFeatureLevels.put(KRaftVersion.FEATURE_NAME, effectiveKRaftFeatureLevel(
|
||||
Optional.ofNullable(newFeatureLevels.get(KRaftVersion.FEATURE_NAME))));
|
||||
} else if (!newFeatureLevels.containsKey(supportedFeature.featureName())) {
|
||||
newFeatureLevels.put(supportedFeature.featureName(),
|
||||
supportedFeature.defaultValue(releaseVersion));
|
||||
}
|
||||
});
|
||||
// Verify that the specified features support the given levels. This requires the full
|
||||
// features map since there may be cross-feature dependencies.
|
||||
for (Map.Entry<String, Short> entry : newFeatureLevels.entrySet()) {
|
||||
String featureName = entry.getKey();
|
||||
if (!featureName.equals(MetadataVersion.FEATURE_NAME)) {
|
||||
short level = entry.getValue();
|
||||
Features supportedFeature = nameToSupportedFeature.get(featureName);
|
||||
FeatureVersion featureVersion =
|
||||
supportedFeature.fromFeatureLevel(level, unstableFeatureVersionsEnabled);
|
||||
Features.validateVersion(featureVersion, newFeatureLevels);
|
||||
}
|
||||
}
|
||||
return newFeatureLevels;
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate the effective feature level for kraft.version. In order to keep existing
|
||||
* command-line invocations of StorageTool working, we default this to 0 if no dynamic
|
||||
* voter quorum arguments were provided. As a convenience, if dynamic voter quorum arguments
|
||||
* were passed, we set the latest kraft.version. (Currently there is only 1 non-zero version).
|
||||
*
|
||||
* @param configuredKRaftVersionLevel The configured level for kraft.version
|
||||
* @return The effective feature level.
|
||||
*/
|
||||
private short effectiveKRaftFeatureLevel(Optional<Short> configuredKRaftVersionLevel) {
|
||||
if (configuredKRaftVersionLevel.isPresent()) {
|
||||
if (configuredKRaftVersionLevel.get() == 0) {
|
||||
if (hasDynamicQuorum()) {
|
||||
throw new FormatterException("Cannot set kraft.version to " +
|
||||
configuredKRaftVersionLevel.get() + " if KIP-853 configuration is present. " +
|
||||
"Try removing the --feature flag for kraft.version.");
|
||||
}
|
||||
} else {
|
||||
if (!hasDynamicQuorum()) {
|
||||
throw new FormatterException("Cannot set kraft.version to " +
|
||||
configuredKRaftVersionLevel.get() + " unless KIP-853 configuration is present. " +
|
||||
"Try removing the --feature flag for kraft.version.");
|
||||
}
|
||||
}
|
||||
return configuredKRaftVersionLevel.get();
|
||||
} else if (hasDynamicQuorum()) {
|
||||
return KRAFT_VERSION_1.featureLevel();
|
||||
} else {
|
||||
return KRAFT_VERSION_0.featureLevel();
|
||||
}
|
||||
}
|
||||
|
||||
BootstrapMetadata calculateBootstrapMetadata() throws Exception {
|
||||
BootstrapMetadata bootstrapMetadata = BootstrapMetadata.
|
||||
fromVersions(releaseVersion, featureLevels, "format command");
|
||||
List<ApiMessageAndVersion> bootstrapRecords = new ArrayList<>(bootstrapMetadata.records());
|
||||
if (!scramArguments.isEmpty()) {
|
||||
if (!releaseVersion.isScramSupported()) {
|
||||
throw new FormatterException("SCRAM is only supported in " + MetadataVersion.FEATURE_NAME +
|
||||
" " + MetadataVersion.IBP_3_5_IV2 + " or later.");
|
||||
}
|
||||
bootstrapRecords.addAll(ScramParser.parse(scramArguments));
|
||||
}
|
||||
return BootstrapMetadata.fromRecords(bootstrapRecords, "format command");
|
||||
}
|
||||
|
||||
void doFormat(BootstrapMetadata bootstrapMetadata) throws Exception {
|
||||
MetaProperties metaProperties = new MetaProperties.Builder().
|
||||
setVersion(MetaPropertiesVersion.V1).
|
||||
setClusterId(clusterId).
|
||||
setNodeId(nodeId).
|
||||
build();
|
||||
MetaPropertiesEnsemble.Loader loader = new MetaPropertiesEnsemble.Loader();
|
||||
loader.addLogDirs(directories);
|
||||
MetaPropertiesEnsemble ensemble = loader.load();
|
||||
ensemble.verify(Optional.of(clusterId),
|
||||
OptionalInt.of(nodeId),
|
||||
EnumSet.noneOf(MetaPropertiesEnsemble.VerificationFlag.class));
|
||||
MetaPropertiesEnsemble.Copier copier = new MetaPropertiesEnsemble.Copier(ensemble);
|
||||
if (!(ignoreFormatted || copier.logDirProps().isEmpty())) {
|
||||
String firstLogDir = copier.logDirProps().keySet().iterator().next();
|
||||
throw new FormatterException("Log directory " + firstLogDir + " is already formatted. " +
|
||||
"Use --ignore-formatted to ignore this directory and format the others.");
|
||||
}
|
||||
if (!copier.errorLogDirs().isEmpty()) {
|
||||
copier.errorLogDirs().forEach(errorLogDir ->
|
||||
printStream.println("I/O error trying to read log directory " + errorLogDir + ". Ignoring..."));
|
||||
if (ensemble.emptyLogDirs().isEmpty() && copier.logDirProps().isEmpty()) {
|
||||
throw new FormatterException("No available log directories to format.");
|
||||
}
|
||||
}
|
||||
if (ensemble.emptyLogDirs().isEmpty()) {
|
||||
printStream.println("All of the log directories are already formatted.");
|
||||
} else {
|
||||
Map<String, DirectoryType> directoryTypes = new HashMap<>();
|
||||
for (String emptyLogDir : ensemble.emptyLogDirs()) {
|
||||
DirectoryType directoryType = DirectoryType.calculate(emptyLogDir,
|
||||
metadataLogDirectory,
|
||||
nodeId,
|
||||
initialControllers);
|
||||
directoryTypes.put(emptyLogDir, directoryType);
|
||||
Uuid directoryId;
|
||||
if (directoryType == DirectoryType.DYNAMIC_METADATA_VOTER_DIRECTORY) {
|
||||
directoryId = initialControllers.get().voters().get(nodeId).directoryId();
|
||||
} else {
|
||||
directoryId = copier.generateValidDirectoryId();
|
||||
}
|
||||
copier.setLogDirProps(emptyLogDir, new MetaProperties.Builder(metaProperties).
|
||||
setDirectoryId(directoryId).
|
||||
build());
|
||||
}
|
||||
copier.setPreWriteHandler((writeLogDir, __, ____) -> {
|
||||
printStream.printf("Formatting %s %s with %s %s.%n",
|
||||
directoryTypes.get(writeLogDir).description(), writeLogDir,
|
||||
MetadataVersion.FEATURE_NAME, releaseVersion);
|
||||
Files.createDirectories(Paths.get(writeLogDir));
|
||||
BootstrapDirectory bootstrapDirectory = new BootstrapDirectory(writeLogDir, Optional.empty());
|
||||
bootstrapDirectory.writeBinaryFile(bootstrapMetadata);
|
||||
if (directoryTypes.get(writeLogDir).isDynamicMetadataDirectory()) {
|
||||
writeDynamicQuorumSnapshot(writeLogDir,
|
||||
initialControllers.get(),
|
||||
featureLevels.get(KRaftVersion.FEATURE_NAME),
|
||||
controllerListenerName);
|
||||
}
|
||||
});
|
||||
copier.setWriteErrorHandler((errorLogDir, e) -> {
|
||||
throw new FormatterException("Error while writing meta.properties file " +
|
||||
errorLogDir + ": " + e);
|
||||
});
|
||||
copier.writeLogDirChanges();
|
||||
}
|
||||
}
|
||||
|
||||
enum DirectoryType {
|
||||
LOG_DIRECTORY,
|
||||
STATIC_METADATA_DIRECTORY,
|
||||
DYNAMIC_METADATA_NON_VOTER_DIRECTORY,
|
||||
DYNAMIC_METADATA_VOTER_DIRECTORY;
|
||||
|
||||
String description() {
|
||||
switch (this) {
|
||||
case LOG_DIRECTORY:
|
||||
return "data directory";
|
||||
case STATIC_METADATA_DIRECTORY:
|
||||
return "metadata directory";
|
||||
case DYNAMIC_METADATA_NON_VOTER_DIRECTORY:
|
||||
return "dynamic metadata directory";
|
||||
case DYNAMIC_METADATA_VOTER_DIRECTORY:
|
||||
return "dynamic metadata voter directory";
|
||||
}
|
||||
throw new RuntimeException("invalid enum type " + this);
|
||||
}
|
||||
|
||||
boolean isDynamicMetadataDirectory() {
|
||||
return this == DYNAMIC_METADATA_NON_VOTER_DIRECTORY ||
|
||||
this == DYNAMIC_METADATA_VOTER_DIRECTORY;
|
||||
}
|
||||
|
||||
static DirectoryType calculate(
|
||||
String logDir,
|
||||
String metadataLogDirectory,
|
||||
int nodeId,
|
||||
Optional<DynamicVoters> initialControllers
|
||||
) {
|
||||
if (!logDir.equals(metadataLogDirectory)) {
|
||||
return LOG_DIRECTORY;
|
||||
} else if (!initialControllers.isPresent()) {
|
||||
return STATIC_METADATA_DIRECTORY;
|
||||
} else if (initialControllers.get().voters().containsKey(nodeId)) {
|
||||
return DYNAMIC_METADATA_VOTER_DIRECTORY;
|
||||
} else {
|
||||
return DYNAMIC_METADATA_NON_VOTER_DIRECTORY;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static void writeDynamicQuorumSnapshot(
|
||||
String writeLogDir,
|
||||
DynamicVoters initialControllers,
|
||||
short kraftVersion,
|
||||
String controllerListenerName
|
||||
) {
|
||||
File parentDir = new File(writeLogDir);
|
||||
File clusterMetadataDirectory = new File(parentDir, String.format("%s-%d",
|
||||
CLUSTER_METADATA_TOPIC_PARTITION.topic(),
|
||||
CLUSTER_METADATA_TOPIC_PARTITION.partition()));
|
||||
VoterSet voterSet = initialControllers.toVoterSet(controllerListenerName);
|
||||
RecordsSnapshotWriter.Builder builder = new RecordsSnapshotWriter.Builder().
|
||||
setLastContainedLogTimestamp(Time.SYSTEM.milliseconds()).
|
||||
setMaxBatchSize(KafkaRaftClient.MAX_BATCH_SIZE_BYTES).
|
||||
setRawSnapshotWriter(FileRawSnapshotWriter.create(
|
||||
clusterMetadataDirectory.toPath(),
|
||||
Snapshots.BOOTSTRAP_SNAPSHOT_ID)).
|
||||
setKraftVersion(KRaftVersion.fromFeatureLevel(kraftVersion)).
|
||||
setVoterSet(Optional.of(voterSet));
|
||||
try (RecordsSnapshotWriter writer = builder.build(new MetadataRecordSerde())) {
|
||||
writer.freeze();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.metadata.storage;
|
||||
|
||||
public class FormatterException extends RuntimeException {
|
||||
public FormatterException(String what) {
|
||||
super(what);
|
||||
}
|
||||
|
||||
public FormatterException(String what, Exception cause) {
|
||||
super(what, cause);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,260 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.metadata.storage;
|
||||
|
||||
import org.apache.kafka.common.metadata.UserScramCredentialRecord;
|
||||
import org.apache.kafka.common.security.scram.internals.ScramFormatter;
|
||||
import org.apache.kafka.common.security.scram.internals.ScramMechanism;
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||
|
||||
import java.util.AbstractMap;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Base64;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class ScramParser {
|
||||
static List<ApiMessageAndVersion> parse(List<String> arguments) throws Exception {
|
||||
List<ApiMessageAndVersion> records = new ArrayList<>();
|
||||
for (String argument : arguments) {
|
||||
Entry<ScramMechanism, String> entry = parsePerMechanismArgument(argument);
|
||||
PerMechanismData data = new PerMechanismData(entry.getKey(), entry.getValue());
|
||||
records.add(new ApiMessageAndVersion(data.toRecord(), (short) 0));
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
static Entry<ScramMechanism, String> parsePerMechanismArgument(String input) {
|
||||
input = input.trim();
|
||||
int equalsIndex = input.indexOf('=');
|
||||
if (equalsIndex < 0) {
|
||||
throw new FormatterException("Failed to find equals sign in SCRAM " +
|
||||
"argument '" + input + "'");
|
||||
}
|
||||
String mechanismString = input.substring(0, equalsIndex);
|
||||
String configString = input.substring(equalsIndex + 1);
|
||||
ScramMechanism mechanism = ScramMechanism.forMechanismName(mechanismString);
|
||||
if (mechanism == null) {
|
||||
throw new FormatterException("The add-scram mechanism " + mechanismString +
|
||||
" is not supported.");
|
||||
}
|
||||
if (!configString.startsWith("[")) {
|
||||
throw new FormatterException("Expected configuration string to start with [");
|
||||
}
|
||||
if (!configString.endsWith("]")) {
|
||||
throw new FormatterException("Expected configuration string to end with ]");
|
||||
}
|
||||
return new AbstractMap.SimpleImmutableEntry<>(mechanism,
|
||||
configString.substring(1, configString.length() - 1));
|
||||
}
|
||||
|
||||
static final class PerMechanismData {
|
||||
private final ScramMechanism mechanism;
|
||||
private final String configuredName;
|
||||
private final Optional<byte[]> configuredSalt;
|
||||
private final OptionalInt configuredIterations;
|
||||
private final Optional<String> configuredPasswordString;
|
||||
private final Optional<byte[]> configuredSaltedPassword;
|
||||
|
||||
PerMechanismData(
|
||||
ScramMechanism mechanism,
|
||||
String configuredName,
|
||||
Optional<byte[]> configuredSalt,
|
||||
OptionalInt configuredIterations,
|
||||
Optional<String> configuredPasswordString,
|
||||
Optional<byte[]> configuredSaltedPassword
|
||||
) {
|
||||
this.mechanism = mechanism;
|
||||
this.configuredName = configuredName;
|
||||
this.configuredSalt = configuredSalt;
|
||||
this.configuredIterations = configuredIterations;
|
||||
this.configuredPasswordString = configuredPasswordString;
|
||||
this.configuredSaltedPassword = configuredSaltedPassword;
|
||||
}
|
||||
|
||||
PerMechanismData(
|
||||
ScramMechanism mechanism,
|
||||
String configString
|
||||
) {
|
||||
this.mechanism = mechanism;
|
||||
String[] configComponents = configString.split(",");
|
||||
Map<String, String> components = new TreeMap<>();
|
||||
for (String configComponent : configComponents) {
|
||||
Entry<String, String> entry = splitTrimmedConfigStringComponent(configComponent);
|
||||
components.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
this.configuredName = components.remove("name");
|
||||
if (this.configuredName == null) {
|
||||
throw new FormatterException("You must supply 'name' to add-scram");
|
||||
}
|
||||
|
||||
String saltString = components.remove("salt");
|
||||
if (saltString == null) {
|
||||
this.configuredSalt = Optional.empty();
|
||||
} else {
|
||||
try {
|
||||
this.configuredSalt = Optional.of(Base64.getDecoder().decode(saltString));
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new FormatterException("Failed to decode given salt: " + saltString, e);
|
||||
}
|
||||
}
|
||||
String iterationsString = components.remove("iterations");
|
||||
if (iterationsString == null) {
|
||||
this.configuredIterations = OptionalInt.empty();
|
||||
} else {
|
||||
try {
|
||||
this.configuredIterations = OptionalInt.of(Integer.parseInt(iterationsString));
|
||||
} catch (NumberFormatException e) {
|
||||
throw new FormatterException("Failed to parse iterations count: " + iterationsString, e);
|
||||
}
|
||||
}
|
||||
String passwordString = components.remove("password");
|
||||
String saltedPasswordString = components.remove("saltedpassword");
|
||||
if (passwordString == null) {
|
||||
if (saltedPasswordString == null) {
|
||||
throw new FormatterException("You must supply one of 'password' or 'saltedpassword' " +
|
||||
"to add-scram");
|
||||
} else if (!configuredSalt.isPresent()) {
|
||||
throw new FormatterException("You must supply 'salt' with 'saltedpassword' to add-scram");
|
||||
}
|
||||
try {
|
||||
this.configuredPasswordString = Optional.empty();
|
||||
this.configuredSaltedPassword = Optional.of(Base64.getDecoder().decode(saltedPasswordString));
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new FormatterException("Failed to decode given saltedPassword: " +
|
||||
saltedPasswordString, e);
|
||||
}
|
||||
} else {
|
||||
this.configuredPasswordString = Optional.of(passwordString);
|
||||
this.configuredSaltedPassword = Optional.empty();
|
||||
}
|
||||
if (!components.isEmpty()) {
|
||||
throw new FormatterException("Unknown SCRAM configurations: " +
|
||||
components.keySet().stream().collect(Collectors.joining(", ")));
|
||||
}
|
||||
}
|
||||
|
||||
byte[] salt() throws Exception {
|
||||
if (configuredSalt.isPresent()) {
|
||||
return configuredSalt.get();
|
||||
}
|
||||
return new ScramFormatter(mechanism).secureRandomBytes();
|
||||
}
|
||||
|
||||
int iterations() {
|
||||
if (configuredIterations.isPresent()) {
|
||||
return configuredIterations.getAsInt();
|
||||
}
|
||||
return 4096;
|
||||
}
|
||||
|
||||
byte[] saltedPassword(byte[] salt, int iterations) throws Exception {
|
||||
if (configuredSaltedPassword.isPresent()) {
|
||||
return configuredSaltedPassword.get();
|
||||
}
|
||||
return new ScramFormatter(mechanism).saltedPassword(
|
||||
configuredPasswordString.get(),
|
||||
salt,
|
||||
iterations);
|
||||
}
|
||||
|
||||
UserScramCredentialRecord toRecord() throws Exception {
|
||||
ScramFormatter formatter = new ScramFormatter(mechanism);
|
||||
byte[] salt = salt();
|
||||
int iterations = iterations();
|
||||
if (iterations < mechanism.minIterations()) {
|
||||
throw new FormatterException("The 'iterations' value must be >= " +
|
||||
mechanism.minIterations() + " for add-scram using " + mechanism);
|
||||
}
|
||||
if (iterations > mechanism.maxIterations()) {
|
||||
throw new FormatterException("The 'iterations' value must be <= " +
|
||||
mechanism.maxIterations() + " for add-scram using " + mechanism);
|
||||
}
|
||||
byte[] saltedPassword = saltedPassword(salt, iterations);
|
||||
return new UserScramCredentialRecord().
|
||||
setName(configuredName).
|
||||
setMechanism(mechanism.type()).
|
||||
setSalt(salt).
|
||||
setStoredKey(formatter.storedKey(formatter.clientKey(saltedPassword))).
|
||||
setServerKey(formatter.serverKey(saltedPassword)).
|
||||
setIterations(iterations);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null || (!(o.getClass().equals(PerMechanismData.class)))) return false;
|
||||
PerMechanismData other = (PerMechanismData) o;
|
||||
return mechanism.equals(other.mechanism) &&
|
||||
configuredName.equals(other.configuredName) &&
|
||||
Arrays.equals(configuredSalt.orElseGet(() -> null),
|
||||
other.configuredSalt.orElseGet(() -> null)) &&
|
||||
configuredIterations.equals(other.configuredIterations) &&
|
||||
configuredPasswordString.equals(other.configuredPasswordString) &&
|
||||
Arrays.equals(configuredSaltedPassword.orElseGet(() -> null),
|
||||
other.configuredSaltedPassword.orElseGet(() -> null));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(mechanism,
|
||||
configuredName,
|
||||
configuredSalt,
|
||||
configuredIterations,
|
||||
configuredPasswordString,
|
||||
configuredSaltedPassword);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "PerMechanismData" +
|
||||
"(mechanism=" + mechanism +
|
||||
", configuredName=" + configuredName +
|
||||
", configuredSalt=" + configuredSalt.map(v -> Arrays.toString(v)) +
|
||||
", configuredIterations=" + configuredIterations +
|
||||
", configuredPasswordString=" + configuredPasswordString +
|
||||
", configuredSaltedPassword=" + configuredSaltedPassword.map(v -> Arrays.toString(v)) +
|
||||
")";
|
||||
}
|
||||
}
|
||||
|
||||
static Entry<String, String> splitTrimmedConfigStringComponent(String input) {
|
||||
int i;
|
||||
for (i = 0; i < input.length(); i++) {
|
||||
if (input.charAt(i) == '=') {
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (i == input.length()) {
|
||||
throw new FormatterException("No equals sign found in SCRAM component: " + input);
|
||||
}
|
||||
String value = input.substring(i + 1);
|
||||
if (value.length() >= 2) {
|
||||
if (value.startsWith("\"") && value.endsWith("\"")) {
|
||||
value = value.substring(1, value.length() - 1);
|
||||
}
|
||||
}
|
||||
return new AbstractMap.SimpleImmutableEntry<>(input.substring(0, i), value);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,445 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.metadata.storage;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.metadata.FeatureLevelRecord;
|
||||
import org.apache.kafka.common.metadata.UserScramCredentialRecord;
|
||||
import org.apache.kafka.common.security.scram.internals.ScramFormatter;
|
||||
import org.apache.kafka.common.security.scram.internals.ScramMechanism;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.metadata.bootstrap.BootstrapDirectory;
|
||||
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
|
||||
import org.apache.kafka.metadata.properties.MetaProperties;
|
||||
import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble;
|
||||
import org.apache.kafka.raft.DynamicVoters;
|
||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||
import org.apache.kafka.server.common.Features;
|
||||
import org.apache.kafka.server.common.MetadataVersion;
|
||||
import org.apache.kafka.server.common.TestFeatureVersion;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.kafka.metadata.storage.ScramParserTest.TEST_SALT;
|
||||
import static org.apache.kafka.metadata.storage.ScramParserTest.TEST_SALTED_PASSWORD;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@Timeout(value = 40)
|
||||
public class FormatterTest {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(FormatterTest.class);
|
||||
|
||||
private static final int DEFAULT_NODE_ID = 1;
|
||||
|
||||
private static final Uuid DEFAULT_CLUSTER_ID = Uuid.fromString("b3dGE68sQQKzfk80C_aLZw");
|
||||
|
||||
static class TestEnv implements AutoCloseable {
|
||||
final List<String> directories;
|
||||
|
||||
TestEnv(int numDirs) {
|
||||
this.directories = new ArrayList<>(numDirs);
|
||||
for (int i = 0; i < numDirs; i++) {
|
||||
this.directories.add(TestUtils.tempDirectory().getAbsolutePath());
|
||||
}
|
||||
}
|
||||
|
||||
FormatterContext newFormatter() {
|
||||
Formatter formatter = new Formatter().
|
||||
setNodeId(DEFAULT_NODE_ID).
|
||||
setClusterId(DEFAULT_CLUSTER_ID.toString());
|
||||
directories.forEach(d -> formatter.addDirectory(d));
|
||||
formatter.setMetadataLogDirectory(directories.get(0));
|
||||
return new FormatterContext(formatter);
|
||||
}
|
||||
|
||||
String directory(int i) {
|
||||
return this.directories.get(i);
|
||||
}
|
||||
|
||||
void deleteDirectory(int i) throws IOException {
|
||||
Utils.delete(new File(directories.get(i)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
for (int i = 0; i < directories.size(); i++) {
|
||||
try {
|
||||
deleteDirectory(i);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Error deleting directory " + directories.get(i), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class FormatterContext {
|
||||
final Formatter formatter;
|
||||
final ByteArrayOutputStream stream;
|
||||
|
||||
FormatterContext(Formatter formatter) {
|
||||
this.formatter = formatter;
|
||||
this.stream = new ByteArrayOutputStream();
|
||||
this.formatter.setPrintStream(new PrintStream(stream));
|
||||
this.formatter.setControllerListenerName("CONTROLLER");
|
||||
}
|
||||
|
||||
String output() {
|
||||
return stream.toString();
|
||||
}
|
||||
|
||||
List<String> outputLines() {
|
||||
return Arrays.asList(stream.toString().trim().split("\\r*\\n"));
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(ints = {1, 2, 3})
|
||||
public void testDirectories(int numDirs) throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(numDirs)) {
|
||||
testEnv.newFormatter().formatter.run();
|
||||
MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble.Loader().
|
||||
addLogDirs(testEnv.directories).
|
||||
load();
|
||||
assertEquals(OptionalInt.of(DEFAULT_NODE_ID), ensemble.nodeId());
|
||||
assertEquals(Optional.of(DEFAULT_CLUSTER_ID.toString()), ensemble.clusterId());
|
||||
assertEquals(new HashSet<>(testEnv.directories), ensemble.logDirProps().keySet());
|
||||
BootstrapMetadata bootstrapMetadata =
|
||||
new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read();
|
||||
assertEquals(MetadataVersion.latestProduction(), bootstrapMetadata.metadataVersion());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatterFailsOnAlreadyFormatted() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
testEnv.newFormatter().formatter.run();
|
||||
assertEquals("Log directory " + testEnv.directory(0) + " is already formatted. " +
|
||||
"Use --ignore-formatted to ignore this directory and format the others.",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> testEnv.newFormatter().formatter.run()).getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatterFailsOnUnwritableDirectory() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
new File(testEnv.directory(0)).setReadOnly();
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
String expectedPrefix = "Error while writing meta.properties file";
|
||||
assertEquals(expectedPrefix,
|
||||
assertThrows(FormatterException.class,
|
||||
() -> formatter1.formatter.run()).
|
||||
getMessage().substring(0, expectedPrefix.length()));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIgnoreFormatted() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.run();
|
||||
assertEquals("Formatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.latestProduction() + ".",
|
||||
formatter1.output().trim());
|
||||
|
||||
FormatterContext formatter2 = testEnv.newFormatter();
|
||||
formatter2.formatter.setIgnoreFormatted(true);
|
||||
formatter2.formatter.run();
|
||||
assertEquals("All of the log directories are already formatted.",
|
||||
formatter2.output().trim());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneDirectoryFormattedAndOthersNotFormatted() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(2)) {
|
||||
testEnv.newFormatter().formatter.setDirectories(Arrays.asList(testEnv.directory(0))).run();
|
||||
assertEquals("Log directory " + testEnv.directory(0) + " is already formatted. " +
|
||||
"Use --ignore-formatted to ignore this directory and format the others.",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> testEnv.newFormatter().formatter.run()).getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneDirectoryFormattedAndOthersNotFormattedWithIgnoreFormatted() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(2)) {
|
||||
testEnv.newFormatter().formatter.setDirectories(Arrays.asList(testEnv.directory(0))).run();
|
||||
|
||||
FormatterContext formatter2 = testEnv.newFormatter();
|
||||
formatter2.formatter.setIgnoreFormatted(true);
|
||||
formatter2.formatter.run();
|
||||
assertEquals("Formatting data directory " + testEnv.directory(1) + " with metadata.version " +
|
||||
MetadataVersion.latestProduction() + ".",
|
||||
formatter2.output().trim());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatWithOlderReleaseVersion() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_5_IV0);
|
||||
formatter1.formatter.run();
|
||||
assertEquals("Formatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.IBP_3_5_IV0 + ".",
|
||||
formatter1.output().trim());
|
||||
BootstrapMetadata bootstrapMetadata =
|
||||
new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read();
|
||||
assertEquals(MetadataVersion.IBP_3_5_IV0, bootstrapMetadata.metadataVersion());
|
||||
assertEquals(1, bootstrapMetadata.records().size());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatWithUnstableReleaseVersionFailsWithoutEnableUnstable() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setReleaseVersion(MetadataVersion.latestTesting());
|
||||
assertEquals("metadata.version " + MetadataVersion.latestTesting() + " is not yet stable.",
|
||||
assertThrows(FormatterException.class, () -> formatter1.formatter.run()).getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatWithUnstableReleaseVersion() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setReleaseVersion(MetadataVersion.latestTesting());
|
||||
formatter1.formatter.setUnstableFeatureVersionsEnabled(true);
|
||||
formatter1.formatter.run();
|
||||
assertEquals("Formatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.latestTesting() + ".",
|
||||
formatter1.output().trim());
|
||||
BootstrapMetadata bootstrapMetadata =
|
||||
new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read();
|
||||
assertEquals(MetadataVersion.latestTesting(), bootstrapMetadata.metadataVersion());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormattingCreatesLogDirId() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.run();
|
||||
MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble.Loader().
|
||||
addLogDirs(testEnv.directories).
|
||||
load();
|
||||
MetaProperties logDirProps = ensemble.logDirProps().get(testEnv.directory(0));
|
||||
assertNotNull(logDirProps);
|
||||
assertTrue(logDirProps.directoryId().isPresent());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatWithScramFailsOnUnsupportedReleaseVersions() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_4_IV0);
|
||||
formatter1.formatter.setScramArguments(Arrays.asList(
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," +
|
||||
"saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]",
|
||||
"SCRAM-SHA-512=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," +
|
||||
"saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]"));
|
||||
assertEquals("SCRAM is only supported in metadata.version 3.5-IV2 or later.",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> formatter1.formatter.run()).getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatWithScram() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_8_IV0);
|
||||
formatter1.formatter.setScramArguments(Arrays.asList(
|
||||
"SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," +
|
||||
"saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]",
|
||||
"SCRAM-SHA-512=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," +
|
||||
"saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]"));
|
||||
formatter1.formatter.run();
|
||||
assertEquals("Formatting metadata directory " + testEnv.directory(0) +
|
||||
" with metadata.version " + MetadataVersion.IBP_3_8_IV0 + ".",
|
||||
formatter1.output().trim());
|
||||
BootstrapMetadata bootstrapMetadata =
|
||||
new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read();
|
||||
assertEquals(MetadataVersion.IBP_3_8_IV0, bootstrapMetadata.metadataVersion());
|
||||
List<ApiMessageAndVersion> scramRecords = bootstrapMetadata.records().stream().
|
||||
filter(r -> r.message() instanceof UserScramCredentialRecord).
|
||||
collect(Collectors.toList());
|
||||
ScramFormatter scram256 = new ScramFormatter(ScramMechanism.SCRAM_SHA_256);
|
||||
ScramFormatter scram512 = new ScramFormatter(ScramMechanism.SCRAM_SHA_512);
|
||||
assertEquals(Arrays.asList(
|
||||
new ApiMessageAndVersion(new UserScramCredentialRecord().
|
||||
setName("alice").
|
||||
setMechanism(ScramMechanism.SCRAM_SHA_256.type()).
|
||||
setSalt(TEST_SALT).
|
||||
setStoredKey(scram256.storedKey(scram256.clientKey(TEST_SALTED_PASSWORD))).
|
||||
setServerKey(scram256.serverKey(TEST_SALTED_PASSWORD)).
|
||||
setIterations(4096), (short) 0),
|
||||
new ApiMessageAndVersion(new UserScramCredentialRecord().
|
||||
setName("alice").
|
||||
setMechanism(ScramMechanism.SCRAM_SHA_512.type()).
|
||||
setSalt(TEST_SALT).
|
||||
setStoredKey(scram512.storedKey(scram512.clientKey(TEST_SALTED_PASSWORD))).
|
||||
setServerKey(scram512.serverKey(TEST_SALTED_PASSWORD)).
|
||||
setIterations(4096), (short) 0)),
|
||||
scramRecords);
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(shorts = {0, 1})
|
||||
public void testFeatureFlag(short version) throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(1)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setSupportedFeatures(Arrays.asList(Features.values()));
|
||||
formatter1.formatter.setFeatureLevel(TestFeatureVersion.FEATURE_NAME, version);
|
||||
formatter1.formatter.run();
|
||||
BootstrapMetadata bootstrapMetadata =
|
||||
new BootstrapDirectory(testEnv.directory(0), Optional.empty()).read();
|
||||
List<ApiMessageAndVersion> expected = new ArrayList<>();
|
||||
expected.add(new ApiMessageAndVersion(new FeatureLevelRecord().
|
||||
setName(MetadataVersion.FEATURE_NAME).
|
||||
setFeatureLevel(MetadataVersion.latestProduction().featureLevel()),
|
||||
(short) 0));
|
||||
if (version > 0) {
|
||||
expected.add(new ApiMessageAndVersion(new FeatureLevelRecord().
|
||||
setName(TestFeatureVersion.FEATURE_NAME).
|
||||
setFeatureLevel(version), (short) 0));
|
||||
}
|
||||
assertEquals(expected, bootstrapMetadata.records());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidFeatureFlag() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(2)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setSupportedFeatures(Arrays.asList(Features.values()));
|
||||
formatter1.formatter.setFeatureLevel("nonexistent.feature", (short) 1);
|
||||
assertEquals("Unsupported feature: nonexistent.feature. Supported features " +
|
||||
"are: kraft.version, test.feature.version, transaction.version",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> formatter1.formatter.run()).
|
||||
getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {false, true})
|
||||
public void testFormatWithInitialVoters(boolean specifyKRaftVersion) throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(2)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
if (specifyKRaftVersion) {
|
||||
formatter1.formatter.setFeatureLevel("kraft.version", (short) 1);
|
||||
}
|
||||
formatter1.formatter.setUnstableFeatureVersionsEnabled(true);
|
||||
formatter1.formatter.setInitialVoters(DynamicVoters.
|
||||
parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw"));
|
||||
formatter1.formatter.run();
|
||||
assertEquals(Arrays.asList(
|
||||
String.format("Formatting data directory %s with %s %s.",
|
||||
testEnv.directory(1),
|
||||
MetadataVersion.FEATURE_NAME,
|
||||
MetadataVersion.latestTesting()),
|
||||
String.format("Formatting dynamic metadata voter directory %s with %s %s.",
|
||||
testEnv.directory(0),
|
||||
MetadataVersion.FEATURE_NAME,
|
||||
MetadataVersion.latestTesting())),
|
||||
formatter1.outputLines().stream().sorted().collect(Collectors.toList()));
|
||||
MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble.Loader().
|
||||
addLogDirs(testEnv.directories).
|
||||
load();
|
||||
MetaProperties logDirProps0 = ensemble.logDirProps().get(testEnv.directory(0));
|
||||
assertNotNull(logDirProps0);
|
||||
assertEquals(Uuid.fromString("4znU-ou9Taa06bmEJxsjnw"), logDirProps0.directoryId().get());
|
||||
MetaProperties logDirProps1 = ensemble.logDirProps().get(testEnv.directory(1));
|
||||
assertNotNull(logDirProps1);
|
||||
assertNotEquals(Uuid.fromString("4znU-ou9Taa06bmEJxsjnw"), logDirProps1.directoryId().get());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatWithInitialVotersFailsWithOlderKraftVersion() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(2)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setFeatureLevel("kraft.version", (short) 0);
|
||||
formatter1.formatter.setUnstableFeatureVersionsEnabled(true);
|
||||
formatter1.formatter.setInitialVoters(DynamicVoters.
|
||||
parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw"));
|
||||
assertTrue(formatter1.formatter.hasDynamicQuorum());
|
||||
assertEquals("Cannot set kraft.version to 0 if KIP-853 configuration is present. " +
|
||||
"Try removing the --feature flag for kraft.version.",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> formatter1.formatter.run()).getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatWithoutInitialVotersFailsWithNewerKraftVersion() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(2)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setFeatureLevel("kraft.version", (short) 1);
|
||||
formatter1.formatter.setUnstableFeatureVersionsEnabled(true);
|
||||
assertFalse(formatter1.formatter.hasDynamicQuorum());
|
||||
assertEquals("Cannot set kraft.version to 1 unless KIP-853 configuration is present. " +
|
||||
"Try removing the --feature flag for kraft.version.",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> formatter1.formatter.run()).getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatWithInitialVotersFailsWithOlderMetadataVersion() throws Exception {
|
||||
try (TestEnv testEnv = new TestEnv(2)) {
|
||||
FormatterContext formatter1 = testEnv.newFormatter();
|
||||
formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_8_IV0);
|
||||
formatter1.formatter.setFeatureLevel("kraft.version", (short) 1);
|
||||
formatter1.formatter.setInitialVoters(DynamicVoters.
|
||||
parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw"));
|
||||
formatter1.formatter.setUnstableFeatureVersionsEnabled(true);
|
||||
assertEquals("kraft.version could not be set to 1 because it depends on " +
|
||||
"metadata.version level 21",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> formatter1.formatter.run()).getMessage());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,278 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.metadata.storage;
|
||||
|
||||
import org.apache.kafka.common.metadata.UserScramCredentialRecord;
|
||||
import org.apache.kafka.common.security.scram.internals.ScramFormatter;
|
||||
import org.apache.kafka.common.security.scram.internals.ScramMechanism;
|
||||
import org.apache.kafka.metadata.storage.ScramParser.PerMechanismData;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
|
||||
import java.util.AbstractMap;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
@Timeout(value = 40)
|
||||
public class ScramParserTest {
|
||||
static final byte[] TEST_SALT = new byte[] {
|
||||
49, 108, 118, 52, 112, 100, 110, 119, 52, 102, 119, 113,
|
||||
55, 110, 111, 116, 99, 120, 109, 48, 121, 121, 49, 107, 55, 113
|
||||
};
|
||||
|
||||
static final byte[] TEST_SALTED_PASSWORD = new byte[] {
|
||||
-103, 61, 50, -55, 69, 49, -98, 82, 90, 11, -33, 71, 94,
|
||||
4, 83, 73, -119, 91, -70, -90, -72, 21, 33, -83, 36,
|
||||
34, 95, 76, -53, -29, 96, 33
|
||||
};
|
||||
|
||||
@Test
|
||||
public void testSplitTrimmedConfigStringComponentOnNameEqualsFoo() {
|
||||
assertEquals(new AbstractMap.SimpleImmutableEntry<>("name", "foo"),
|
||||
ScramParser.splitTrimmedConfigStringComponent("name=foo"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitTrimmedConfigStringComponentOnNameEqualsQuotedFoo() {
|
||||
assertEquals(new AbstractMap.SimpleImmutableEntry<>("name", "foo"),
|
||||
ScramParser.splitTrimmedConfigStringComponent("name=\"foo\""));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitTrimmedConfigStringComponentOnNameEqualsEmpty() {
|
||||
assertEquals(new AbstractMap.SimpleImmutableEntry<>("name", ""),
|
||||
ScramParser.splitTrimmedConfigStringComponent("name="));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitTrimmedConfigStringComponentOnNameEqualsQuotedEmpty() {
|
||||
assertEquals(new AbstractMap.SimpleImmutableEntry<>("name", ""),
|
||||
ScramParser.splitTrimmedConfigStringComponent("name=\"\""));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitTrimmedConfigStringComponentWithNoEquals() {
|
||||
assertEquals("No equals sign found in SCRAM component: name",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> ScramParser.splitTrimmedConfigStringComponent("name")).getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRandomSalt() throws Exception {
|
||||
PerMechanismData data = new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"bob",
|
||||
Optional.empty(),
|
||||
OptionalInt.empty(),
|
||||
Optional.of("my pass"),
|
||||
Optional.empty());
|
||||
TestUtils.retryOnExceptionWithTimeout(10_000, () -> {
|
||||
assertNotEquals(data.salt().toString(), data.salt().toString());
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConfiguredSalt() throws Exception {
|
||||
assertArrayEquals(TEST_SALT, new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"bob",
|
||||
Optional.of(TEST_SALT),
|
||||
OptionalInt.empty(),
|
||||
Optional.of("my pass"),
|
||||
Optional.empty()).salt());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDefaultIterations() {
|
||||
assertEquals(4096, new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"bob",
|
||||
Optional.empty(),
|
||||
OptionalInt.empty(),
|
||||
Optional.of("my pass"),
|
||||
Optional.empty()).iterations());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConfiguredIterations() {
|
||||
assertEquals(8192, new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"bob",
|
||||
Optional.empty(),
|
||||
OptionalInt.of(8192),
|
||||
Optional.of("my pass"),
|
||||
Optional.empty()).iterations());
|
||||
}
|
||||
@Test
|
||||
public void testParsePerMechanismArgument() {
|
||||
assertEquals(new AbstractMap.SimpleImmutableEntry<>(
|
||||
ScramMechanism.SCRAM_SHA_512, "name=scram-admin,password=scram-user-secret"),
|
||||
ScramParser.parsePerMechanismArgument(
|
||||
"SCRAM-SHA-512=[name=scram-admin,password=scram-user-secret]"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismArgumentWithoutEqualsSign() {
|
||||
assertEquals("Failed to find equals sign in SCRAM argument 'SCRAM-SHA-512'",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> ScramParser.parsePerMechanismArgument(
|
||||
"SCRAM-SHA-512")).getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismArgumentWithUnsupportedScramMethod() {
|
||||
assertEquals("The add-scram mechanism SCRAM-SHA-UNSUPPORTED is not supported.",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> ScramParser.parsePerMechanismArgument(
|
||||
"SCRAM-SHA-UNSUPPORTED=[name=scram-admin,password=scram-user-secret]")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismArgumentWithConfigStringWithoutBraces() {
|
||||
assertEquals("Expected configuration string to start with [",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> ScramParser.parsePerMechanismArgument(
|
||||
"SCRAM-SHA-256=name=scram-admin,password=scram-user-secret")).getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismArgumentWithConfigStringWithoutEndBrace() {
|
||||
assertEquals("Expected configuration string to end with ]",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> ScramParser.parsePerMechanismArgument(
|
||||
"SCRAM-SHA-256=[name=scram-admin,password=scram-user-secret")).getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismData() {
|
||||
assertEquals(new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"bob",
|
||||
Optional.empty(),
|
||||
OptionalInt.empty(),
|
||||
Optional.of("mypass"),
|
||||
Optional.empty()),
|
||||
new PerMechanismData(ScramMechanism.SCRAM_SHA_256, "name=bob,password=mypass"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismDataFailsWithoutName() {
|
||||
assertEquals("You must supply 'name' to add-scram",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"password=mypass")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismDataFailsWithoutPassword() {
|
||||
assertEquals("You must supply one of 'password' or 'saltedpassword' to add-scram",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"name=bar")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismDataFailsWithExtraArguments() {
|
||||
assertEquals("Unknown SCRAM configurations: unknown, unknown2",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"name=bob,password=mypass,unknown=something,unknown2=somethingelse")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismDataWithIterations() {
|
||||
assertEquals(new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"bob",
|
||||
Optional.empty(),
|
||||
OptionalInt.of(8192),
|
||||
Optional.of("my pass"),
|
||||
Optional.empty()),
|
||||
new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"name=bob,password=my pass,iterations=8192"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismDataWithConfiguredSalt() {
|
||||
assertEquals(new PerMechanismData(ScramMechanism.SCRAM_SHA_512,
|
||||
"bob",
|
||||
Optional.of(TEST_SALT),
|
||||
OptionalInt.empty(),
|
||||
Optional.of("my pass"),
|
||||
Optional.empty()),
|
||||
new PerMechanismData(ScramMechanism.SCRAM_SHA_512,
|
||||
"name=bob,password=my pass,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\""));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismDataWithIterationsAndConfiguredSalt() {
|
||||
assertEquals(new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"bob",
|
||||
Optional.of(TEST_SALT),
|
||||
OptionalInt.of(8192),
|
||||
Optional.of("my pass"),
|
||||
Optional.empty()),
|
||||
new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"name=bob,password=my pass,iterations=8192,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\""));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismDataWithConfiguredSaltedPasswordFailsWithoutSalt() {
|
||||
assertEquals("You must supply 'salt' with 'saltedpassword' to add-scram",
|
||||
assertThrows(FormatterException.class,
|
||||
() -> new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"name=alice,saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsePerMechanismDataWithConfiguredSaltedPassword() {
|
||||
assertEquals(new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"alice",
|
||||
Optional.of(TEST_SALT),
|
||||
OptionalInt.empty(),
|
||||
Optional.empty(),
|
||||
Optional.of(TEST_SALTED_PASSWORD)),
|
||||
new PerMechanismData(ScramMechanism.SCRAM_SHA_256,
|
||||
"name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," +
|
||||
"saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\""));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPerMechanismDataToRecord() throws Exception {
|
||||
ScramFormatter formatter = new ScramFormatter(ScramMechanism.SCRAM_SHA_512);
|
||||
assertEquals(new UserScramCredentialRecord().
|
||||
setName("alice").
|
||||
setMechanism(ScramMechanism.SCRAM_SHA_512.type()).
|
||||
setSalt(TEST_SALT).
|
||||
setStoredKey(formatter.storedKey(formatter.clientKey(TEST_SALTED_PASSWORD))).
|
||||
setServerKey(formatter.serverKey(TEST_SALTED_PASSWORD)).
|
||||
setIterations(4096),
|
||||
new PerMechanismData(ScramMechanism.SCRAM_SHA_512,
|
||||
"alice",
|
||||
Optional.of(TEST_SALT),
|
||||
OptionalInt.empty(),
|
||||
Optional.empty(),
|
||||
Optional.of(TEST_SALTED_PASSWORD)).toRecord());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,188 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.raft.internals.ReplicaKey;
|
||||
import org.apache.kafka.raft.internals.VoterSet;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Collections;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* The textual representation of a KIP-853 voter.
|
||||
*
|
||||
* Since this is used in command-line tools, format changes to the parsing logic require a KIP,
|
||||
* and should be backwards compatible.
|
||||
*/
|
||||
public final class DynamicVoter {
|
||||
private final Uuid directoryId;
|
||||
private final int nodeId;
|
||||
private final String host;
|
||||
private final int port;
|
||||
|
||||
/**
|
||||
* Create a DynamicVoter object by parsing an input string.
|
||||
*
|
||||
* @param input The input string.
|
||||
*
|
||||
* @return The DynamicVoter object.
|
||||
*
|
||||
* @throws IllegalArgumentException If parsing fails.
|
||||
*/
|
||||
public static DynamicVoter parse(String input) {
|
||||
input = input.trim();
|
||||
int atIndex = input.indexOf("@");
|
||||
if (atIndex < 0) {
|
||||
throw new IllegalArgumentException("No @ found in dynamic voter string.");
|
||||
}
|
||||
if (atIndex == 0) {
|
||||
throw new IllegalArgumentException("Invalid @ at beginning of dynamic voter string.");
|
||||
}
|
||||
String idString = input.substring(0, atIndex);
|
||||
int nodeId;
|
||||
try {
|
||||
nodeId = Integer.parseInt(idString);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("Failed to parse node id in dynamic voter string.", e);
|
||||
}
|
||||
if (nodeId < 0) {
|
||||
throw new IllegalArgumentException("Invalid negative node id " + nodeId +
|
||||
" in dynamic voter string.");
|
||||
}
|
||||
input = input.substring(atIndex + 1);
|
||||
if (input.isEmpty()) {
|
||||
throw new IllegalArgumentException("No hostname found after node id.");
|
||||
}
|
||||
String host;
|
||||
if (input.startsWith("[")) {
|
||||
int endBracketIndex = input.indexOf("]");
|
||||
if (endBracketIndex < 0) {
|
||||
throw new IllegalArgumentException("Hostname began with left bracket, but no right " +
|
||||
"bracket was found.");
|
||||
}
|
||||
host = input.substring(1, endBracketIndex);
|
||||
input = input.substring(endBracketIndex + 1);
|
||||
} else {
|
||||
int endColonIndex = input.indexOf(":");
|
||||
if (endColonIndex < 0) {
|
||||
throw new IllegalArgumentException("No colon following hostname could be found.");
|
||||
}
|
||||
host = input.substring(0, endColonIndex);
|
||||
input = input.substring(endColonIndex);
|
||||
}
|
||||
if (!input.startsWith(":")) {
|
||||
throw new IllegalArgumentException("Port section must start with a colon.");
|
||||
}
|
||||
input = input.substring(1);
|
||||
int endColonIndex = input.indexOf(":");
|
||||
if (endColonIndex < 0) {
|
||||
throw new IllegalArgumentException("No colon following port could be found.");
|
||||
}
|
||||
String portString = input.substring(0, endColonIndex);
|
||||
int port;
|
||||
try {
|
||||
port = Integer.parseInt(portString);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new IllegalArgumentException("Failed to parse port in dynamic voter string.", e);
|
||||
}
|
||||
if (port < 0 || port > 65535) {
|
||||
throw new IllegalArgumentException("Invalid port " + port + " in dynamic voter string.");
|
||||
}
|
||||
String directoryIdString = input.substring(endColonIndex + 1);
|
||||
Uuid directoryId;
|
||||
try {
|
||||
directoryId = Uuid.fromString(directoryIdString);
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new IllegalArgumentException("Failed to parse directory ID in dynamic voter string.", e);
|
||||
}
|
||||
return new DynamicVoter(directoryId, nodeId, host, port);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new KIP-853 voter.
|
||||
*
|
||||
* @param directoryId The directory ID.
|
||||
* @param nodeId The voter ID.
|
||||
* @param host The voter hostname or IP address.
|
||||
* @param port The voter port.
|
||||
*/
|
||||
public DynamicVoter(
|
||||
Uuid directoryId,
|
||||
int nodeId,
|
||||
String host,
|
||||
int port
|
||||
) {
|
||||
this.directoryId = directoryId;
|
||||
this.nodeId = nodeId;
|
||||
this.host = host;
|
||||
this.port = port;
|
||||
}
|
||||
|
||||
public Uuid directoryId() {
|
||||
return directoryId;
|
||||
}
|
||||
|
||||
public int nodeId() {
|
||||
return nodeId;
|
||||
}
|
||||
|
||||
public String host() {
|
||||
return host;
|
||||
}
|
||||
|
||||
public int port() {
|
||||
return port;
|
||||
}
|
||||
|
||||
public VoterSet.VoterNode toVoterNode(String controllerListenerName) {
|
||||
ReplicaKey voterKey = ReplicaKey.of(nodeId, directoryId);
|
||||
Endpoints listeners = Endpoints.fromInetSocketAddresses(Collections.singletonMap(
|
||||
ListenerName.normalised(controllerListenerName),
|
||||
new InetSocketAddress(host, port)));
|
||||
SupportedVersionRange supportedKRaftVersion =
|
||||
new SupportedVersionRange((short) 0, (short) 1);
|
||||
return VoterSet.VoterNode.of(voterKey, listeners, supportedKRaftVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null || (!(o.getClass().equals(DynamicVoter.class)))) return false;
|
||||
DynamicVoter other = (DynamicVoter) o;
|
||||
return directoryId.equals(other.directoryId) &&
|
||||
nodeId == other.nodeId &&
|
||||
host.equals(other.host) &&
|
||||
port == other.port;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(directoryId,
|
||||
nodeId,
|
||||
host,
|
||||
port);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return nodeId + "@" + host + ":" + port + ":" + directoryId;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,117 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.raft.internals.VoterSet;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
/**
|
||||
* The textual representation of a set of KIP-853 voters.
|
||||
*
|
||||
* Since this is used in command-line tools, format changes to the parsing logic require a KIP,
|
||||
* and should be backwards compatible.
|
||||
*/
|
||||
public final class DynamicVoters {
|
||||
/**
|
||||
* Create a DynamicVoters object by parsing an input string.
|
||||
*
|
||||
* @param input The input string.
|
||||
*
|
||||
* @return The DynamicVoters object.
|
||||
*
|
||||
* @throws IllegalArgumentException If parsing fails.
|
||||
*/
|
||||
public static DynamicVoters parse(String input) {
|
||||
input = input.trim();
|
||||
List<DynamicVoter> voters = new ArrayList<>();
|
||||
for (String voterString : input.split(",")) {
|
||||
if (!voterString.isEmpty()) {
|
||||
voters.add(DynamicVoter.parse(voterString));
|
||||
}
|
||||
}
|
||||
return new DynamicVoters(voters);
|
||||
}
|
||||
|
||||
/**
|
||||
* Maps node ids to dynamic voters.
|
||||
*/
|
||||
private final NavigableMap<Integer, DynamicVoter> voters;
|
||||
|
||||
/**
|
||||
* Create a new DynamicVoters object.
|
||||
*
|
||||
* @param voters The voters.
|
||||
*/
|
||||
public DynamicVoters(Collection<DynamicVoter> voters) {
|
||||
if (voters.isEmpty()) {
|
||||
throw new IllegalArgumentException("No voters given.");
|
||||
}
|
||||
TreeMap<Integer, DynamicVoter> votersMap = new TreeMap<>();
|
||||
for (DynamicVoter voter : voters) {
|
||||
if (votersMap.put(voter.nodeId(), voter) != null) {
|
||||
throw new IllegalArgumentException("Node id " + voter.nodeId() +
|
||||
" was specified more than once.");
|
||||
}
|
||||
}
|
||||
this.voters = Collections.unmodifiableNavigableMap(votersMap);
|
||||
}
|
||||
|
||||
public NavigableMap<Integer, DynamicVoter> voters() {
|
||||
return voters;
|
||||
}
|
||||
|
||||
public VoterSet toVoterSet(String controllerListenerName) {
|
||||
Map<Integer, VoterSet.VoterNode> voterSetMap = new HashMap<>();
|
||||
for (DynamicVoter voter : voters.values()) {
|
||||
voterSetMap.put(voter.nodeId(), voter.toVoterNode(controllerListenerName));
|
||||
}
|
||||
return VoterSet.fromMap(voterSetMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null || (!(o.getClass().equals(DynamicVoters.class)))) return false;
|
||||
DynamicVoters other = (DynamicVoters) o;
|
||||
return voters.equals(other.voters);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return voters.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
String prefix = "";
|
||||
for (DynamicVoter voter : voters.values()) {
|
||||
builder.append(prefix);
|
||||
prefix = ",";
|
||||
builder.append(voter.toString());
|
||||
}
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
|
@ -426,4 +426,8 @@ public final class VoterSet {
|
|||
|
||||
return new VoterSet(voterNodes);
|
||||
}
|
||||
|
||||
public static VoterSet fromMap(Map<Integer, VoterNode> voters) {
|
||||
return new VoterSet(new HashMap<>(voters));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,185 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.raft.internals.ReplicaKey;
|
||||
import org.apache.kafka.raft.internals.VoterSet;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
public class DynamicVoterTest {
|
||||
@Test
|
||||
public void testParseDynamicVoter() {
|
||||
assertEquals(new DynamicVoter(Uuid.fromString("K90IZ-0DRNazJ49kCZ1EMQ"),
|
||||
2,
|
||||
"localhost",
|
||||
(short) 8020),
|
||||
DynamicVoter.parse("2@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseDynamicVoter2() {
|
||||
assertEquals(new DynamicVoter(Uuid.fromString("__0IZ-0DRNazJ49kCZ1EMQ"),
|
||||
100,
|
||||
"192.128.0.100",
|
||||
(short) 800),
|
||||
DynamicVoter.parse("100@192.128.0.100:800:__0IZ-0DRNazJ49kCZ1EMQ"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseDynamicVoterWithBrackets() {
|
||||
assertEquals(new DynamicVoter(Uuid.fromString("__0IZ-0DRNazJ49kCZ1EMQ"),
|
||||
5,
|
||||
"2001:4860:4860::8888",
|
||||
(short) 8020),
|
||||
DynamicVoter.parse("5@[2001:4860:4860::8888]:8020:__0IZ-0DRNazJ49kCZ1EMQ"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseDynamicVoterWithoutId() {
|
||||
assertEquals("No @ found in dynamic voter string.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseDynamicVoterWithoutId2() {
|
||||
assertEquals("Invalid @ at beginning of dynamic voter string.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseDynamicVoterWithInvalidNegativeId() {
|
||||
assertEquals("Invalid negative node id -1 in dynamic voter string.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("-1@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailedToParseNodeId() {
|
||||
assertEquals("Failed to parse node id in dynamic voter string.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("blah@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseDynamicVoterWithoutHostname() {
|
||||
assertEquals("No hostname found after node id.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("2@")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseDynamicVoterWithUnbalancedBrackets() {
|
||||
assertEquals("Hostname began with left bracket, but no right bracket was found.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("5@[2001:4860:4860::8888:8020:__0IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoColonFollowingHostname() {
|
||||
assertEquals("No colon following hostname could be found.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("2@localhost8020K90IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPortSectionMustStartWithAColon() {
|
||||
assertEquals("Port section must start with a colon.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("5@[2001:4860:4860::8888]8020:__0IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseDynamicVoterWithNoColonFollowingPort() {
|
||||
assertEquals("No colon following port could be found.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("5@[2001:4860:4860::8888]:8020__0IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailedToParsePort() {
|
||||
assertEquals("Failed to parse port in dynamic voter string.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("5@[2001:4860:4860::8888]:8020m:__0IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidNegativePort() {
|
||||
assertEquals("Invalid port -8020 in dynamic voter string.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("5@[2001:4860:4860::8888]:-8020:__0IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidPositivePort() {
|
||||
assertEquals("Invalid port 666666 in dynamic voter string.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("5@[2001:4860:4860::8888]:666666:__0IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailedToParseDirectoryId() {
|
||||
assertEquals("Failed to parse directory ID in dynamic voter string.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("5@[2001:4860:4860::8888]:8020:%_0IZ-0DRNazJ49kCZ1EMQ")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailedToParseDirectoryId2() {
|
||||
assertEquals("Failed to parse directory ID in dynamic voter string.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoter.parse("5@[2001:4860:4860::8888]:8020:")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToVoterNode() {
|
||||
ReplicaKey voterKey = ReplicaKey.of(5, Uuid.fromString("__0IZ-0DRNazJ49kCZ1EMQ"));
|
||||
Endpoints listeners = Endpoints.fromInetSocketAddresses(Collections.singletonMap(
|
||||
new ListenerName("CONTROLLER"),
|
||||
new InetSocketAddress("localhost", 8020)));
|
||||
SupportedVersionRange supportedKRaftVersion =
|
||||
new SupportedVersionRange((short) 0, (short) 1);
|
||||
assertEquals(VoterSet.VoterNode.of(voterKey, listeners, supportedKRaftVersion),
|
||||
DynamicVoter.parse("5@localhost:8020:__0IZ-0DRNazJ49kCZ1EMQ").
|
||||
toVoterNode("CONTROLLER"));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,119 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.raft.internals.VoterSet;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
public class DynamicVotersTest {
|
||||
@Test
|
||||
public void testParsingEmptyStringFails() {
|
||||
assertEquals("No voters given.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoters.parse("")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsingSingleDynamicVoter() {
|
||||
assertEquals(new DynamicVoters(Arrays.asList(
|
||||
new DynamicVoter(
|
||||
Uuid.fromString("K90IZ-0DRNazJ49kCZ1EMQ"),
|
||||
2,
|
||||
"localhost",
|
||||
(short) 8020))),
|
||||
DynamicVoters.parse("2@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsingThreeDynamicVoters() {
|
||||
assertEquals(new DynamicVoters(Arrays.asList(
|
||||
new DynamicVoter(
|
||||
Uuid.fromString("K90IZ-0DRNazJ49kCZ1EMQ"),
|
||||
0,
|
||||
"localhost",
|
||||
(short) 8020),
|
||||
new DynamicVoter(
|
||||
Uuid.fromString("aUARLskQTCW4qCZDtS_cwA"),
|
||||
1,
|
||||
"localhost",
|
||||
(short) 8030),
|
||||
new DynamicVoter(
|
||||
Uuid.fromString("2ggvsS4kQb-fSJ_-zC_Ang"),
|
||||
2,
|
||||
"localhost",
|
||||
(short) 8040))),
|
||||
DynamicVoters.parse(
|
||||
"0@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ," +
|
||||
"1@localhost:8030:aUARLskQTCW4qCZDtS_cwA," +
|
||||
"2@localhost:8040:2ggvsS4kQb-fSJ_-zC_Ang"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParsingInvalidStringWithDuplicateNodeIds() {
|
||||
assertEquals("Node id 1 was specified more than once.",
|
||||
assertThrows(IllegalArgumentException.class,
|
||||
() -> DynamicVoters.parse(
|
||||
"0@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ," +
|
||||
"1@localhost:8030:aUARLskQTCW4qCZDtS_cwA," +
|
||||
"1@localhost:8040:2ggvsS4kQb-fSJ_-zC_Ang")).
|
||||
getMessage());
|
||||
}
|
||||
|
||||
private static void testRoundTrip(String input) {
|
||||
DynamicVoters voters = DynamicVoters.parse(input);
|
||||
assertEquals(input, voters.toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRoundTripSingleVoter() {
|
||||
testRoundTrip("2@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRoundTripThreeVoters() {
|
||||
testRoundTrip(
|
||||
"0@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ," +
|
||||
"1@localhost:8030:aUARLskQTCW4qCZDtS_cwA," +
|
||||
"2@localhost:8040:2ggvsS4kQb-fSJ_-zC_Ang");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToVoterSet() {
|
||||
Map<Integer, VoterSet.VoterNode> voterMap = new HashMap<>();
|
||||
voterMap.put(0, DynamicVoter.parse(
|
||||
"0@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ").toVoterNode("CONTROLLER2"));
|
||||
voterMap.put(1, DynamicVoter.parse(
|
||||
"1@localhost:8030:aUARLskQTCW4qCZDtS_cwA").toVoterNode("CONTROLLER2"));
|
||||
voterMap.put(2, DynamicVoter.parse(
|
||||
"2@localhost:8040:2ggvsS4kQb-fSJ_-zC_Ang").toVoterNode("CONTROLLER2"));
|
||||
assertEquals(VoterSet.fromMap(voterMap),
|
||||
DynamicVoters.parse(
|
||||
"0@localhost:8020:K90IZ-0DRNazJ49kCZ1EMQ," +
|
||||
"1@localhost:8030:aUARLskQTCW4qCZDtS_cwA," +
|
||||
"2@localhost:8040:2ggvsS4kQb-fSJ_-zC_Ang").toVoterSet("CONTROLLER2"));
|
||||
}
|
||||
}
|
|
@ -71,7 +71,12 @@ public enum KRaftVersion implements FeatureVersion {
|
|||
|
||||
@Override
|
||||
public Map<String, Short> dependencies() {
|
||||
return Collections.emptyMap();
|
||||
if (this.featureLevel == 0) {
|
||||
return Collections.emptyMap();
|
||||
} else {
|
||||
return Collections.singletonMap(
|
||||
MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_9_IV0.featureLevel());
|
||||
}
|
||||
}
|
||||
|
||||
public short quorumStateVersion() {
|
||||
|
|
Loading…
Reference in New Issue