mirror of https://github.com/apache/kafka.git
MINOR: Add and use new method TestUtils.tempPropertiesFile() (#12976)
Reviewers: Divij Vaidya <diviv@amazon.com>
This commit is contained in:
parent
9b7f7e0fa0
commit
546b912b83
|
|
@ -12,7 +12,6 @@
|
|||
*/
|
||||
package kafka.api
|
||||
|
||||
import java.nio.file.Files
|
||||
import java.time.Duration
|
||||
import java.util.{Collections, Properties}
|
||||
import java.util.concurrent.{ExecutionException, TimeUnit}
|
||||
|
|
@ -195,13 +194,7 @@ class SaslClientsWithInvalidCredentialsTest extends IntegrationTestHarness with
|
|||
}
|
||||
|
||||
private def prepareConsumerGroupService = {
|
||||
val propsFile = TestUtils.tempFile()
|
||||
val propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
try {
|
||||
propsStream.write("security.protocol=SASL_PLAINTEXT\n".getBytes())
|
||||
propsStream.write(s"sasl.mechanism=$kafkaClientSaslMechanism".getBytes())
|
||||
}
|
||||
finally propsStream.close()
|
||||
val propsFile = TestUtils.tempPropertiesFile(Map("security.protocol" -> "SASL_PLAINTEXT", "sasl.mechanism" -> kafkaClientSaslMechanism))
|
||||
|
||||
val cgcArgs = Array("--bootstrap-server", bootstrapServers(),
|
||||
"--describe",
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@
|
|||
|
||||
package kafka.server
|
||||
|
||||
import java.io.{Closeable, File, FileWriter, IOException, Reader, StringReader}
|
||||
import java.io.{Closeable, File, IOException, Reader, StringReader}
|
||||
import java.nio.file.{Files, Paths, StandardCopyOption}
|
||||
import java.lang.management.ManagementFactory
|
||||
import java.security.KeyStore
|
||||
|
|
@ -1712,15 +1712,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
|
|||
}
|
||||
|
||||
private def alterConfigsUsingConfigCommand(props: Properties): Unit = {
|
||||
val propsFile = TestUtils.tempFile()
|
||||
val propsWriter = new FileWriter(propsFile)
|
||||
try {
|
||||
clientProps(SecurityProtocol.SSL).forEach {
|
||||
case (k, v) => propsWriter.write(s"$k=$v\n")
|
||||
}
|
||||
} finally {
|
||||
propsWriter.close()
|
||||
}
|
||||
val propsFile = TestUtils.tempPropertiesFile(clientProps(SecurityProtocol.SSL))
|
||||
|
||||
servers.foreach { server =>
|
||||
val args = Array("--bootstrap-server", TestUtils.bootstrapServers(servers, new ListenerName(SecureInternal)),
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@
|
|||
*/
|
||||
package kafka.admin
|
||||
|
||||
import java.io.{File, PrintWriter}
|
||||
import java.io.File
|
||||
import java.util.Properties
|
||||
import javax.management.InstanceAlreadyExistsException
|
||||
import kafka.admin.AclCommand.AclCommandOptions
|
||||
|
|
@ -195,10 +195,7 @@ class AclCommandTest extends QuorumTestHarness with Logging {
|
|||
|
||||
@Test
|
||||
def testAclCliWithClientId(): Unit = {
|
||||
val adminClientConfig = TestUtils.tempFile()
|
||||
val pw = new PrintWriter(adminClientConfig)
|
||||
pw.println("client.id=my-client")
|
||||
pw.close()
|
||||
val adminClientConfig = TestUtils.tempFile("client.id=my-client")
|
||||
|
||||
createServer(Some(adminClientConfig))
|
||||
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ import java.util.Properties
|
|||
import kafka.admin.ConfigCommand.ConfigCommandOptions
|
||||
import kafka.cluster.Broker
|
||||
import kafka.server.{ConfigEntityName, ConfigType}
|
||||
import kafka.utils.{Exit, Logging}
|
||||
import kafka.utils.{Exit, Logging, TestUtils}
|
||||
import kafka.zk.{AdminZkClient, KafkaZkClient}
|
||||
import org.apache.kafka.clients.admin._
|
||||
import org.apache.kafka.common.Node
|
||||
|
|
@ -31,7 +31,6 @@ import org.apache.kafka.common.internals.KafkaFutureImpl
|
|||
import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent}
|
||||
import org.apache.kafka.common.security.scram.internals.ScramCredentialUtils
|
||||
import org.apache.kafka.common.utils.Sanitizer
|
||||
import org.apache.kafka.test.TestUtils
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.mockito.ArgumentMatchers.anyString
|
||||
|
|
@ -787,9 +786,9 @@ class ConfigCommandTest extends Logging {
|
|||
|
||||
def doShouldAlterTopicConfig(file: Boolean): Unit = {
|
||||
var filePath = ""
|
||||
val addedConfigs = Seq("delete.retention.ms=1000000", "min.insync.replicas=2")
|
||||
val addedConfigs = Map("delete.retention.ms" -> "1000000", "min.insync.replicas" -> "2")
|
||||
if (file) {
|
||||
val file = TestUtils.tempFile(addedConfigs.mkString("\n"))
|
||||
val file = TestUtils.tempPropertiesFile(addedConfigs)
|
||||
filePath = file.getPath
|
||||
}
|
||||
|
||||
|
|
@ -799,7 +798,7 @@ class ConfigCommandTest extends Logging {
|
|||
"--entity-type", "topics",
|
||||
"--alter",
|
||||
if (file) "--add-config-file" else "--add-config",
|
||||
if (file) filePath else addedConfigs.mkString(","),
|
||||
if (file) filePath else addedConfigs.map{case (k, v) => k + "=" + v}.mkString(","),
|
||||
"--delete-config", "unclean.leader.election.enable"))
|
||||
var alteredConfigs = false
|
||||
|
||||
|
|
|
|||
|
|
@ -18,7 +18,6 @@
|
|||
package kafka.tools
|
||||
|
||||
import java.io.{ByteArrayOutputStream, PrintStream}
|
||||
import java.nio.file.Files
|
||||
import java.util.{HashMap, Optional, Map => JMap}
|
||||
import java.time.Duration
|
||||
import kafka.tools.ConsoleConsumer.ConsumerWrapper
|
||||
|
|
@ -366,11 +365,7 @@ class ConsoleConsumerTest {
|
|||
|
||||
@Test
|
||||
def shouldParseConfigsFromFile(): Unit = {
|
||||
val propsFile = TestUtils.tempFile()
|
||||
val propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
propsStream.write("request.timeout.ms=1000\n".getBytes())
|
||||
propsStream.write("group.id=group1".getBytes())
|
||||
propsStream.close()
|
||||
val propsFile = TestUtils.tempPropertiesFile(Map("request.timeout.ms" -> "1000", "group.id" -> "group1"))
|
||||
val args: Array[String] = Array(
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
|
@ -388,10 +383,7 @@ class ConsoleConsumerTest {
|
|||
Exit.setExitProcedure((_, message) => throw new IllegalArgumentException(message.orNull))
|
||||
|
||||
// different in all three places
|
||||
var propsFile = TestUtils.tempFile()
|
||||
var propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
propsStream.write("group.id=group-from-file".getBytes())
|
||||
propsStream.close()
|
||||
var propsFile = TestUtils.tempPropertiesFile(Map("group.id" -> "group-from-file"))
|
||||
var args: Array[String] = Array(
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
|
@ -403,10 +395,7 @@ class ConsoleConsumerTest {
|
|||
assertThrows(classOf[IllegalArgumentException], () => new ConsoleConsumer.ConsumerConfig(args))
|
||||
|
||||
// the same in all three places
|
||||
propsFile = TestUtils.tempFile()
|
||||
propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
propsStream.write("group.id=test-group".getBytes())
|
||||
propsStream.close()
|
||||
propsFile = TestUtils.tempPropertiesFile(Map("group.id" -> "test-group"))
|
||||
args = Array(
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
|
@ -420,10 +409,7 @@ class ConsoleConsumerTest {
|
|||
assertEquals("test-group", props.getProperty("group.id"))
|
||||
|
||||
// different via --consumer-property and --consumer.config
|
||||
propsFile = TestUtils.tempFile()
|
||||
propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
propsStream.write("group.id=group-from-file".getBytes())
|
||||
propsStream.close()
|
||||
propsFile = TestUtils.tempPropertiesFile(Map("group.id" -> "group-from-file"))
|
||||
args = Array(
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
|
@ -444,10 +430,7 @@ class ConsoleConsumerTest {
|
|||
assertThrows(classOf[IllegalArgumentException], () => new ConsoleConsumer.ConsumerConfig(args))
|
||||
|
||||
// different via --group and --consumer.config
|
||||
propsFile = TestUtils.tempFile()
|
||||
propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
propsStream.write("group.id=group-from-file".getBytes())
|
||||
propsStream.close()
|
||||
propsFile = TestUtils.tempPropertiesFile(Map("group.id" -> "group-from-file"))
|
||||
args = Array(
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
|
@ -492,12 +475,7 @@ class ConsoleConsumerTest {
|
|||
|
||||
@Test
|
||||
def testCustomConfigShouldBePassedToConfigureMethod(): Unit = {
|
||||
val propsFile = TestUtils.tempFile()
|
||||
val propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
propsStream.write("key.deserializer.my-props=abc\n".getBytes())
|
||||
propsStream.write("print.key=false".getBytes())
|
||||
propsStream.close()
|
||||
|
||||
val propsFile = TestUtils.tempPropertiesFile(Map("key.deserializer.my-props" -> "abc", "print.key" -> "false"))
|
||||
val args = Array(
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
|
|
|||
|
|
@ -19,7 +19,6 @@ package kafka.tools
|
|||
|
||||
import kafka.common.MessageReader
|
||||
|
||||
import java.nio.file.Files
|
||||
import kafka.tools.ConsoleProducer.LineMessageReader
|
||||
import kafka.utils.{Exit, TestUtils}
|
||||
import org.apache.kafka.clients.producer.{Producer, ProducerConfig, ProducerRecord}
|
||||
|
|
@ -150,11 +149,7 @@ class ConsoleProducerTest {
|
|||
|
||||
@Test
|
||||
def testParseReaderConfigFile(): Unit = {
|
||||
val propsFile = TestUtils.tempFile()
|
||||
val propsStream = Files.newOutputStream(propsFile.toPath)
|
||||
propsStream.write("parse.key=true\n".getBytes())
|
||||
propsStream.write("key.separator=|".getBytes())
|
||||
propsStream.close()
|
||||
val propsFile = TestUtils.tempPropertiesFile(Map("parse.key" -> "true", "key.separator" -> "|"))
|
||||
|
||||
val args = Array(
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
|
|
|
|||
|
|
@ -156,6 +156,21 @@ object TestUtils extends Logging {
|
|||
*/
|
||||
def tempFile(prefix: String, suffix: String): File = JTestUtils.tempFile(prefix, suffix)
|
||||
|
||||
/**
|
||||
* Create a file with the given contents in the default temporary-file directory,
|
||||
* using `kafka` as the prefix and `tmp` as the suffix to generate its name.
|
||||
*/
|
||||
def tempFile(contents: String): File = JTestUtils.tempFile(contents)
|
||||
|
||||
def tempPropertiesFile(properties: Properties): File = {
|
||||
return tempPropertiesFile(properties.asScala)
|
||||
}
|
||||
|
||||
def tempPropertiesFile(properties: Map[String, String]): File = {
|
||||
val content = properties.map{case (k, v) => k + "=" + v}.mkString(System.lineSeparator())
|
||||
return tempFile(content)
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a temporary file and return an open file channel for this file
|
||||
*/
|
||||
|
|
|
|||
Loading…
Reference in New Issue