[3/6] kafka git commit: KAFKA-5121; Implement transaction index for KIP-98
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/ProducerIdMapping.scala -- diff --git a/core/src/main/scala/kafka/log/ProducerIdMapping.scala b/core/src/main/scala/kafka/log/ProducerIdMapping.scala deleted file mode 100644 index bcadce5..000 --- a/core/src/main/scala/kafka/log/ProducerIdMapping.scala +++ /dev/null @@ -1,384 +0,0 @@ -/* - * 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 kafka.log - -import java.io._ -import java.nio.ByteBuffer -import java.nio.file.Files - -import kafka.common.KafkaException -import kafka.utils.{Logging, nonthreadsafe} -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.{DuplicateSequenceNumberException, OutOfOrderSequenceException, ProducerFencedException} -import org.apache.kafka.common.protocol.types._ -import org.apache.kafka.common.record.RecordBatch -import org.apache.kafka.common.utils.{ByteUtils, Crc32C} - -import scala.collection.{immutable, mutable} - -private[log] object ProducerIdEntry { - val Empty = ProducerIdEntry(RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, --1, 0, RecordBatch.NO_TIMESTAMP) -} - -private[log] case class ProducerIdEntry(epoch: Short, lastSeq: Int, lastOffset: Long, offsetDelta: Int, timestamp: Long) { - def firstSeq: Int = lastSeq - offsetDelta - def firstOffset: Long = lastOffset - offsetDelta - - def isDuplicate(batch: RecordBatch): Boolean = { -batch.producerEpoch == epoch && - batch.baseSequence == firstSeq && - batch.lastSequence == lastSeq - } -} - -private[log] class ProducerAppendInfo(val pid: Long, initialEntry: ProducerIdEntry) { - // the initialEntry here is the last successful appended batch. we validate incoming entries transitively, starting - // with the last appended entry. - private var epoch = initialEntry.epoch - private var firstSeq = initialEntry.firstSeq - private var lastSeq = initialEntry.lastSeq - private var lastOffset = initialEntry.lastOffset - private var maxTimestamp = initialEntry.timestamp - - private def validateAppend(epoch: Short, firstSeq: Int, lastSeq: Int) = { -if (this.epoch > epoch) { - throw new ProducerFencedException(s"Producer's epoch is no longer valid. There is probably another producer with a newer epoch. $epoch (request epoch), ${this.epoch} (server epoch)") -} else if (this.epoch == RecordBatch.NO_PRODUCER_EPOCH || this.epoch < epoch) { - if (firstSeq != 0) -throw new OutOfOrderSequenceException(s"Invalid sequence number for new epoch: $epoch " + - s"(request epoch), $firstSeq (seq. number)") -} else if (firstSeq == this.firstSeq && lastSeq == this.lastSeq) { - throw new DuplicateSequenceNumberException(s"Duplicate sequence number: $pid (pid), $firstSeq " + -s"(seq. number), ${this.firstSeq} (expected seq. number)") -} else if (firstSeq != this.lastSeq + 1L) { - throw new OutOfOrderSequenceException(s"Invalid sequence number: $pid (pid), $firstSeq " + -s"(seq. number), ${this.lastSeq} (expected seq. number)") -} - } - - def assignLastOffsetAndTimestamp(lastOffset: Long, lastTimestamp: Long): Unit = { -this.lastOffset = lastOffset -this.maxTimestamp = lastTimestamp - } - - private def append(epoch: Short, firstSeq: Int, lastSeq: Int, lastTimestamp: Long, lastOffset: Long) { -validateAppend(epoch, firstSeq, lastSeq) -this.epoch = epoch -this.firstSeq = firstSeq -this.lastSeq = lastSeq -this.maxTimestamp = lastTimestamp -this.lastOffset = lastOffset - } - - def append(batch: RecordBatch): Unit = -append(batch.producerEpoch, batch.baseSequence, batch.lastSequence, batch.maxTimestamp, batch.lastOffset) - - def append(entry: ProducerIdEntry): Unit = -append(entry.epoch, entry.firstSeq, entry.lastSeq, entry.timestamp, entry.lastOffset) - - def lastEntry: ProducerIdEntry = -ProducerIdEntry(epoch, lastSeq, lastOffset, lastSeq - firstSeq, maxTimestamp) -} - -class CorruptSnapshotException(msg: String) extends KafkaException(msg) - -object ProducerIdMapping { - private val PidSnapshotVersion: Short = 1 - private val VersionF
[4/6] kafka git commit: KAFKA-5121; Implement transaction index for KIP-98
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/Log.scala -- diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index b7f340f..5722a43 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -30,9 +30,10 @@ import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata} import kafka.utils._ import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException} import org.apache.kafka.common.record._ -import org.apache.kafka.common.requests.ListOffsetRequest +import org.apache.kafka.common.requests.{IsolationLevel, ListOffsetRequest} import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer import scala.collection.{Seq, mutable} import com.yammer.metrics.core.Gauge import org.apache.kafka.common.utils.{Time, Utils} @@ -40,10 +41,13 @@ import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCod import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile} import kafka.server.epoch.{LeaderEpochCache, LeaderEpochFileCache} import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction +import java.util.Map.{Entry => JEntry} +import java.lang.{Long => JLong} object LogAppendInfo { val UnknownLogAppendInfo = LogAppendInfo(-1, -1, RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP, -NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false, Map.empty[Long, ProducerAppendInfo], false) +NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false) } /** @@ -59,9 +63,6 @@ object LogAppendInfo { * @param shallowCount The number of shallow messages * @param validBytes The number of valid bytes * @param offsetsMonotonic Are the offsets in this message set monotonically increasing - * @param producerAppendInfos A map from a Pid to a ProducerAppendInfo, which is used to validate each Record in a - *RecordBatch and keep track of metadata across Records in a RecordBatch. - * @param isDuplicate Indicates whether the message set is a duplicate of a message at the tail of the log. */ case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, @@ -72,9 +73,19 @@ case class LogAppendInfo(var firstOffset: Long, targetCodec: CompressionCodec, shallowCount: Int, validBytes: Int, - offsetsMonotonic: Boolean, - producerAppendInfos: Map[Long, ProducerAppendInfo], - isDuplicate: Boolean = false) + offsetsMonotonic: Boolean) + +/** + * A class used to hold useful metadata about a completed transaction. This is used to build + * the transaction index after appending to the log. + * + * @param producerId The ID of the producer + * @param firstOffset The first offset (inclusive) of the transaction + * @param lastOffset The last offset (inclusive) of the transaction. This is always the offset of the + * COMMIT/ABORT control record which indicates the transaction's completion. + * @param isAborted Whether or not the transaction was aborted + */ +case class CompletedTxn(producerId: Long, firstOffset: Long, lastOffset: Long, isAborted: Boolean) /** * An append-only log for storing messages. @@ -111,8 +122,7 @@ class Log(@volatile var dir: File, scheduler: Scheduler, time: Time = Time.SYSTEM, val maxPidExpirationMs: Int = 60 * 60 * 1000, - val pidExpirationCheckIntervalMs: Int = 10 * 60 * 1000, - val pidSnapshotIntervalMs: Int = 60 * 1000) extends Logging with KafkaMetricsGroup { + val pidExpirationCheckIntervalMs: Int = 10 * 60 * 1000) extends Logging with KafkaMetricsGroup { import kafka.log.Log._ @@ -133,8 +143,10 @@ class Log(@volatile var dir: File, @volatile private var nextOffsetMetadata: LogOffsetMetadata = _ - /* Construct and load PID map */ - private val pidMap = new ProducerIdMapping(config, topicPartition, dir, maxPidExpirationMs) + /* The earliest offset which is part of an incomplete transaction. This is used to compute the LSO. */ + @volatile var firstUnstableOffset: Option[LogOffsetMetadata] = None + + private val producerStateManager = new ProducerStateManager(topicPartition, dir, maxPidExpirationMs) /* the actual segments of the log */ private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment] @@ -147,7 +159,7 @@ class Log(@volatile var dir: File, loadSegments() /* Calculate the of
[5/6] kafka git commit: KAFKA-5121; Implement transaction index for KIP-98
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java -- diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index e38e583..0a0f3d9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -44,6 +44,7 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.ControlRecordType; import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.record.EndTransactionMarker; import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; @@ -181,16 +182,18 @@ public class FetcherTest { assertEquals(1, fetcher.sendFetches()); assertFalse(fetcher.hasCompletedFetches()); +long producerId = 1; +short epoch = 0; +int baseSequence = 0; + ByteBuffer buffer = ByteBuffer.allocate(1024); -MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L); -builder.append(0L, "key".getBytes(), null); -builder.appendControlRecord(0L, ControlRecordType.COMMIT, null); +MemoryRecordsBuilder builder = MemoryRecords.idempotentBuilder(buffer, CompressionType.NONE, 0L, producerId, +epoch, baseSequence); builder.append(0L, "key".getBytes(), null); builder.close(); -builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 3L); -builder.appendControlRecord(0L, ControlRecordType.ABORT, null); -builder.close(); +MemoryRecords.writeEndTransactionalMarker(buffer, 1L, producerId, epoch, new EndTransactionMarker(ControlRecordType.ABORT, 0) +); buffer.flip(); @@ -202,10 +205,11 @@ public class FetcherTest { assertTrue(partitionRecords.containsKey(tp1)); List> records = partitionRecords.get(tp1); -assertEquals(2, records.size()); -assertEquals(4L, subscriptions.position(tp1).longValue()); -for (ConsumerRecord record : records) -assertArrayEquals("key".getBytes(), record.key()); +assertEquals(1, records.size()); +assertEquals(2L, subscriptions.position(tp1).longValue()); + +ConsumerRecord record = records.get(0); +assertArrayEquals("key".getBytes(), record.key()); } @Test @@ -814,6 +818,29 @@ public class FetcherTest { } @Test +public void testListOffsetsSendsIsolationLevel() { +for (final IsolationLevel isolationLevel : IsolationLevel.values()) { +Fetcher fetcher = createFetcher(subscriptions, new Metrics(), new ByteArrayDeserializer(), +new ByteArrayDeserializer(), Integer.MAX_VALUE, isolationLevel); + +subscriptions.assignFromUser(singleton(tp1)); +subscriptions.needOffsetReset(tp1, OffsetResetStrategy.LATEST); + +client.prepareResponse(new MockClient.RequestMatcher() { +@Override +public boolean matches(AbstractRequest body) { +ListOffsetRequest request = (ListOffsetRequest) body; +return request.isolationLevel() == isolationLevel; +} +}, listOffsetResponse(Errors.NONE, 1L, 5L)); +fetcher.updateFetchPositions(singleton(tp1)); +assertFalse(subscriptions.isOffsetResetNeeded(tp1)); +assertTrue(subscriptions.isFetchable(tp1)); +assertEquals(5, subscriptions.position(tp1).longValue()); +} +} + +@Test public void testUpdateFetchPositionResetToEarliestOffset() { subscriptions.assignFromUser(singleton(tp1)); subscriptions.needOffsetReset(tp1, OffsetResetStrategy.EARLIEST); @@ -1206,7 +1233,7 @@ public class FetcherTest { new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()), new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes())); -currentOffset += abortTransaction(buffer, 1L, currentOffset, time.milliseconds()); +currentOffset += abortTransaction(buffer, 1L, currentOffset); buffer.flip(); @@ -1240,7 +1267,7 @@ public class FetcherTest { new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()), new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes())); -currentOffset += commitTransaction(buffer, 1L
[1/6] kafka git commit: KAFKA-5121; Implement transaction index for KIP-98
Repository: kafka Updated Branches: refs/heads/trunk 29994dd10 -> e71dce89c http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala -- diff --git a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala new file mode 100644 index 000..e8c918d --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala @@ -0,0 +1,562 @@ +/** + * 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 kafka.log + +import java.io.File + +import kafka.server.LogOffsetMetadata +import kafka.utils.TestUtils +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors._ +import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, RecordBatch} +import org.apache.kafka.common.utils.{MockTime, Utils} +import org.junit.Assert._ +import org.junit.{After, Before, Test} +import org.scalatest.junit.JUnitSuite + +class ProducerStateManagerTest extends JUnitSuite { + var idMappingDir: File = null + var idMapping: ProducerStateManager = null + val partition = new TopicPartition("test", 0) + val pid = 1L + val maxPidExpirationMs = 60 * 1000 + val time = new MockTime + + @Before + def setUp(): Unit = { +idMappingDir = TestUtils.tempDir() +idMapping = new ProducerStateManager(partition, idMappingDir, maxPidExpirationMs) + } + + @After + def tearDown(): Unit = { +Utils.delete(idMappingDir) + } + + @Test + def testBasicIdMapping(): Unit = { +val epoch = 0.toShort + +// First entry for id 0 added +append(idMapping, pid, 0, epoch, 0L, 0L) + +// Second entry for id 0 added +append(idMapping, pid, 1, epoch, 0L, 1L) + +// Duplicate sequence number (matches previous sequence number) +assertThrows[DuplicateSequenceNumberException] { + append(idMapping, pid, 1, epoch, 0L, 1L) +} + +// Invalid sequence number (greater than next expected sequence number) +assertThrows[OutOfOrderSequenceException] { + append(idMapping, pid, 5, epoch, 0L, 2L) +} + +// Change epoch +append(idMapping, pid, 0, (epoch + 1).toShort, 0L, 3L) + +// Incorrect epoch +assertThrows[ProducerFencedException] { + append(idMapping, pid, 0, epoch, 0L, 4L) +} + } + + @Test + def testNoValidationOnFirstEntryWhenLoadingLog(): Unit = { +val epoch = 5.toShort +val sequence = 16 +val offset = 735L +append(idMapping, pid, sequence, epoch, offset, isLoadingFromLog = true) + +val maybeLastEntry = idMapping.lastEntry(pid) +assertTrue(maybeLastEntry.isDefined) + +val lastEntry = maybeLastEntry.get +assertEquals(epoch, lastEntry.producerEpoch) +assertEquals(sequence, lastEntry.firstSeq) +assertEquals(sequence, lastEntry.lastSeq) +assertEquals(offset, lastEntry.lastOffset) +assertEquals(offset, lastEntry.firstOffset) + } + + @Test + def testControlRecordBumpsEpoch(): Unit = { +val epoch = 0.toShort +append(idMapping, pid, 0, epoch, 0L) + +val bumpedEpoch = 1.toShort +val (completedTxn, lastStableOffset) = appendEndTxnMarker(idMapping, pid, bumpedEpoch, ControlRecordType.ABORT, 1L) +assertEquals(1L, completedTxn.firstOffset) +assertEquals(1L, completedTxn.lastOffset) +assertEquals(2L, lastStableOffset) +assertTrue(completedTxn.isAborted) +assertEquals(pid, completedTxn.producerId) + +val maybeLastEntry = idMapping.lastEntry(pid) +assertTrue(maybeLastEntry.isDefined) + +val lastEntry = maybeLastEntry.get +assertEquals(bumpedEpoch, lastEntry.producerEpoch) +assertEquals(None, lastEntry.currentTxnFirstOffset) +assertEquals(RecordBatch.NO_SEQUENCE, lastEntry.firstSeq) +assertEquals(RecordBatch.NO_SEQUENCE, lastEntry.lastSeq) + +// should be able to append with the new epoch if we start at sequence 0 +append(idMapping, pid, 0, bumpedEpoch, 2L) +assertEquals(Some(0), idMapping.lastEntry(pid).map(_.firstSeq)) + } + + @Test + def testTxnFirstOffsetMetadataCached(): Unit = { +val producerEpoch = 0.toShort +val off
[2/6] kafka git commit: KAFKA-5121; Implement transaction index for KIP-98
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala -- diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala index eb3f50c..bf634d7 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala @@ -109,7 +109,8 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Abstrac private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long): Seq[(Int, Int)] = { for (_ <- 0 until numDups; key <- 0 until numKeys) yield { val count = counter - log.appendAsLeader(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) + log.appendAsLeader(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec, + key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0) counter += 1 (key, count) } http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala -- diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 44d47c9..fe07fdd 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -821,7 +821,7 @@ class LogCleanerTest extends JUnitSuite { def record(key: Int, value: Int, pid: Long = RecordBatch.NO_PRODUCER_ID, epoch: Short = RecordBatch.NO_PRODUCER_EPOCH, sequence: Int = RecordBatch.NO_SEQUENCE, partitionLeaderEpoch: Int = RecordBatch.NO_PARTITION_LEADER_EPOCH): MemoryRecords = { -MemoryRecords.withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, CompressionType.NONE, pid, epoch, sequence, +MemoryRecords.withIdempotentRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, CompressionType.NONE, pid, epoch, sequence, partitionLeaderEpoch, new SimpleRecord(key.toString.getBytes, value.toString.getBytes)) } http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogManagerTest.scala -- diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 2f9396f..a6fe2e4 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -151,8 +151,9 @@ class LogManagerTest { assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments) time.sleep(log.config.fileDeleteDelayMs + 1) -//There should be a log file, two indexes, the leader epoch checkpoint and the pid snapshot dir -assertEquals("Files should have been deleted", log.numberOfSegments * 3 + 2, log.dir.list.length) +// there should be a log file, two indexes (the txn index is created lazily), +// the leader epoch checkpoint and two pid mapping files (one for the active and previous segments) +assertEquals("Files should have been deleted", log.numberOfSegments * 3 + 3, log.dir.list.length) assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).records.sizeInBytes) try { log.read(0, 1024) http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala -- diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 3f531d9..4709b77 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -16,31 +16,40 @@ */ package kafka.log +import java.io.File + import kafka.utils.TestUtils import kafka.utils.TestUtils.checkEquals +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.record.MemoryRecords.withEndTransactionMarker import org.apache.kafka.common.record.{RecordBatch, _} -import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.utils.{Time, Utils} import org.junit.Assert._ -import org.junit.{After, Test} +import org.junit.{After, Before, Test} import scala.collection.JavaConverters._ import scala.collection._ class LogSegmentTest { + val topicPartition = new TopicPartition("topic", 0) val segments = mutable.ArrayBuffer[LogSegment]() - + var logDir: File = _ + /* create a segment with the given base offset */ def c
[6/6] kafka git commit: KAFKA-5121; Implement transaction index for KIP-98
KAFKA-5121; Implement transaction index for KIP-98 Author: Jason Gustafson Reviewers: Ismael Juma , Jun Rao Closes #2910 from hachikuji/eos-txn-index Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/e71dce89 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/e71dce89 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/e71dce89 Branch: refs/heads/trunk Commit: e71dce89c0da50f3eccc47d0fc050c92d5a99b88 Parents: 29994dd Author: Jason Gustafson Authored: Sat May 6 11:49:35 2017 -0700 Committer: Jason Gustafson Committed: Sat May 6 11:49:35 2017 -0700 -- .../clients/consumer/internals/Fetcher.java | 79 +-- .../TransactionCoordinatorFencedException.java | 30 + .../apache/kafka/common/protocol/Errors.java| 17 +- .../apache/kafka/common/protocol/Protocol.java | 17 +- .../record/AbstractLegacyRecordBatch.java | 2 +- .../kafka/common/record/AbstractRecords.java| 9 +- .../kafka/common/record/ControlRecordType.java | 24 +- .../kafka/common/record/DefaultRecord.java | 50 +- .../kafka/common/record/DefaultRecordBatch.java | 22 +- .../common/record/EndTransactionMarker.java | 124 .../kafka/common/record/FileLogInputStream.java | 6 + .../apache/kafka/common/record/FileRecords.java | 13 +- .../kafka/common/record/MemoryRecords.java | 109 +++- .../common/record/MemoryRecordsBuilder.java | 66 +- .../org/apache/kafka/common/record/Record.java | 9 - .../apache/kafka/common/record/RecordBatch.java | 8 + .../common/requests/ListOffsetRequest.java | 34 +- .../org/apache/kafka/common/utils/Utils.java| 5 + .../clients/consumer/internals/FetcherTest.java | 97 +-- .../common/record/DefaultRecordBatchTest.java | 31 +- .../kafka/common/record/DefaultRecordTest.java | 45 +- .../common/record/EndTransactionMarkerTest.java | 70 +++ .../kafka/common/record/FileRecordsTest.java| 11 +- .../common/record/MemoryRecordsBuilderTest.java | 93 ++- .../kafka/common/record/MemoryRecordsTest.java | 80 ++- .../common/requests/RequestResponseTest.java| 22 +- .../main/scala/kafka/cluster/Partition.scala| 4 +- core/src/main/scala/kafka/cluster/Replica.scala | 23 +- .../group/GroupMetadataManager.scala| 27 +- .../transaction/TransactionStateManager.scala | 19 +- .../main/scala/kafka/log/AbstractIndex.scala| 23 +- core/src/main/scala/kafka/log/Log.scala | 611 --- core/src/main/scala/kafka/log/LogCleaner.scala | 34 +- core/src/main/scala/kafka/log/LogManager.scala | 6 +- core/src/main/scala/kafka/log/LogSegment.scala | 132 ++-- .../src/main/scala/kafka/log/LogValidator.scala | 83 +-- core/src/main/scala/kafka/log/OffsetIndex.scala | 20 +- .../scala/kafka/log/ProducerIdMapping.scala | 384 .../scala/kafka/log/ProducerStateManager.scala | 590 ++ core/src/main/scala/kafka/log/TimeIndex.scala | 7 +- .../main/scala/kafka/log/TransactionIndex.scala | 243 .../main/scala/kafka/server/DelayedFetch.scala | 14 +- .../main/scala/kafka/server/FetchDataInfo.scala | 4 +- .../src/main/scala/kafka/server/KafkaApis.scala | 67 +- .../scala/kafka/server/LogOffsetMetadata.scala | 8 +- .../scala/kafka/server/ReplicaManager.scala | 31 +- .../scala/kafka/tools/DumpLogSegments.scala | 29 +- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- .../group/GroupCoordinatorResponseTest.scala| 6 +- .../group/GroupMetadataManagerTest.scala| 22 +- .../TransactionStateManagerTest.scala | 7 +- .../unit/kafka/log/BrokerCompressionTest.scala | 2 +- .../kafka/log/LogCleanerIntegrationTest.scala | 2 +- .../log/LogCleanerLagIntegrationTest.scala | 3 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 2 +- .../scala/unit/kafka/log/LogManagerTest.scala | 5 +- .../scala/unit/kafka/log/LogSegmentTest.scala | 104 +++- .../src/test/scala/unit/kafka/log/LogTest.scala | 555 +++-- .../scala/unit/kafka/log/LogValidatorTest.scala | 209 +-- .../scala/unit/kafka/log/OffsetIndexTest.scala | 24 +- .../unit/kafka/log/ProducerIdMappingTest.scala | 291 - .../kafka/log/ProducerStateManagerTest.scala| 562 + .../unit/kafka/log/TransactionIndexTest.scala | 173 ++ .../scala/unit/kafka/server/LogOffsetTest.scala | 12 +- .../kafka/server/ReplicaManagerQuotasTest.scala | 7 +- .../unit/kafka/server/ReplicaManagerTest.scala | 151 - .../unit/kafka/server/RequestQuotaTest.scala| 7 +- .../unit/kafka/server/SimpleFetchTest.scala | 7 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 2 +- 69 files changed, 4099 insertions(+), 1488 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/bl
kafka git commit: KAFKA-4839; Throw NoOffsetForPartitionException from poll once for all TopicPartitions affected
Repository: kafka Updated Branches: refs/heads/trunk 257ad524d -> 2554a8dd4 KAFKA-4839; Throw NoOffsetForPartitionException from poll once for all TopicPartitions affected Signed-off-by: radai-rosenblatt Author: radai-rosenblatt Reviewers: Apurva Mehta , Vahid Hashemian , Ismael Juma , Jason Gustafson Closes #2637 from radai-rosenblatt/KAFKA-4839 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/2554a8dd Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/2554a8dd Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/2554a8dd Branch: refs/heads/trunk Commit: 2554a8dd4dd07b0ac844839b51533bd1e67eed85 Parents: 257ad52 Author: radai-rosenblatt Authored: Mon May 8 09:42:44 2017 -0700 Committer: Jason Gustafson Committed: Mon May 8 09:42:44 2017 -0700 -- .../consumer/NoOffsetForPartitionException.java | 25 .../clients/consumer/internals/Fetcher.java | 22 ++--- .../clients/consumer/internals/FetcherTest.java | 15 3 files changed, 50 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/2554a8dd/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java index 14bb710..375cda2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java @@ -18,7 +18,9 @@ package org.apache.kafka.clients.consumer; import org.apache.kafka.common.TopicPartition; +import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.Set; /** @@ -29,19 +31,34 @@ public class NoOffsetForPartitionException extends InvalidOffsetException { private static final long serialVersionUID = 1L; -private final TopicPartition partition; +private final Set partitions; public NoOffsetForPartitionException(TopicPartition partition) { super("Undefined offset with no reset policy for partition: " + partition); -this.partition = partition; +this.partitions = Collections.singleton(partition); } +public NoOffsetForPartitionException(Collection partitions) { +super("Undefined offset with no reset policy for partitions: " + partitions); +this.partitions = Collections.unmodifiableSet(new HashSet<>(partitions)); +} + +/** + * returns the first partition (out of {@link #partitions}) for which no offset is defined. + * @deprecated please use {@link #partitions} + * @return a partition with no offset + */ +@Deprecated public TopicPartition partition() { -return partition; +return partitions.isEmpty() ? null : partitions.iterator().next(); } +/** + * returns all partitions for which no offests are defined. + * @return all partitions without offsets + */ public Set partitions() { -return Collections.singleton(partition); +return partitions; } } http://git-wip-us.apache.org/repos/asf/kafka/blob/2554a8dd/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index dc6c338..bf5df95 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -382,16 +382,17 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { return client.send(node, request); } -private long offsetResetStrategyTimestamp(final TopicPartition partition) { +private void offsetResetStrategyTimestamp( +final TopicPartition partition, +final Map output, +final Set partitionsWithNoOffsets) { OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); -final long timestamp; if (strategy == OffsetResetStrategy.EARLIEST) -timestamp = ListOffsetRequest.EARLIEST_TIMESTAMP; +output.put(partition, ListOffsetRequest.EARLIEST_TIMESTAMP); else if (strategy == OffsetResetStrategy.LATEST) -timestamp = endTimestamp(); +output.put(partition, endTimestamp()); else -thro
kafka git commit: KAFKA-3353; Remove deprecated producer configs
Repository: kafka Updated Branches: refs/heads/trunk 2554a8dd4 -> 3bcadbfb4 KAFKA-3353; Remove deprecated producer configs These configs have been deprecated since 0.9.0.0: block.on.buffer.full, metadata.fetch.timeout.ms and timeout.ms Author: Ismael Juma Reviewers: Jason Gustafson Closes #2987 from ijuma/kafka-3353-remove-deprecated-producer-configs Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/3bcadbfb Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/3bcadbfb Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/3bcadbfb Branch: refs/heads/trunk Commit: 3bcadbfb474f6caccc939fb3775a6f969d136af7 Parents: 2554a8d Author: Ismael Juma Authored: Mon May 8 10:00:04 2017 -0700 Committer: Jason Gustafson Committed: Mon May 8 10:00:04 2017 -0700 -- .../kafka/clients/producer/KafkaProducer.java | 45 +--- .../kafka/clients/producer/ProducerConfig.java | 43 --- docs/upgrade.html | 2 + 3 files changed, 4 insertions(+), 86 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/3bcadbfb/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 286387b..f812389 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -270,8 +270,8 @@ public class KafkaProducer implements Producer { this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG); this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG)); -this.maxBlockTimeMs = configureMaxBlockTime(config, userProvidedConfigs); -this.requestTimeoutMs = configureRequestTimeout(config, userProvidedConfigs); +this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG); +this.requestTimeoutMs = config.getInt(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG); this.transactionManager = configureTransactionState(config); int retries = configureRetries(config, transactionManager != null); int maxInflightRequests = configureInflightRequests(config, transactionManager != null); @@ -335,47 +335,6 @@ public class KafkaProducer implements Producer { return serializer instanceof ExtendedSerializer ? (ExtendedSerializer) serializer : new ExtendedSerializer.Wrapper<>(serializer); } -private static long configureMaxBlockTime(ProducerConfig config, Map userProvidedConfigs) { -/* check for user defined settings. - * If the BLOCK_ON_BUFFER_FULL is set to true,we do not honor METADATA_FETCH_TIMEOUT_CONFIG. - * This should be removed with release 0.9 when the deprecated configs are removed. - */ -if (userProvidedConfigs.containsKey(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG)) { -log.warn(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG + " config is deprecated and will be removed soon. " + -"Please use " + ProducerConfig.MAX_BLOCK_MS_CONFIG); -boolean blockOnBufferFull = config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG); -if (blockOnBufferFull) { -return Long.MAX_VALUE; -} else if (userProvidedConfigs.containsKey(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG)) { -log.warn(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG + " config is deprecated and will be removed soon. " + -"Please use " + ProducerConfig.MAX_BLOCK_MS_CONFIG); -return config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); -} else { -return config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG); -} -} else if (userProvidedConfigs.containsKey(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG)) { -log.warn(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG + " config is deprecated and will be removed soon. " + -"Please use " + ProducerConfig.MAX_BLOCK_MS_CONFIG); -return config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); -} else { -return config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG); -} -} - -private static int configureRequestTimeout(ProducerConfig config, Map userProvidedConfigs) { -/* check for user defined settings. - * If the TIME_OUT config is set use that for request timeout. - * This shou
kafka git commit: KAFKA-3763; Remove deprecated APIs for 0.11.0.0
Repository: kafka Updated Branches: refs/heads/trunk 3bcadbfb4 -> 59b918ec2 KAFKA-3763; Remove deprecated APIs for 0.11.0.0 This only removes deprecated methods, fields and constructors in a small number of classes. Deprecated producer configs is tracked via KAFKA-3353 and the old clients and related (tools, etc.) won't be removed in 0.11.0.0. Author: Ismael Juma Reviewers: Jason Gustafson Closes #2995 from ijuma/kafka-3763-remove-deprecated-0.11 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/59b918ec Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/59b918ec Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/59b918ec Branch: refs/heads/trunk Commit: 59b918ec2be5ec69fd5b62613e7665510f5217c0 Parents: 3bcadbf Author: Ismael Juma Authored: Mon May 8 10:29:31 2017 -0700 Committer: Jason Gustafson Committed: Mon May 8 10:29:31 2017 -0700 -- .../kafka/clients/CommonClientConfigs.java | 5 -- .../java/org/apache/kafka/common/Cluster.java | 14 - .../org/apache/kafka/common/MetricName.java | 64 docs/upgrade.html | 1 + 4 files changed, 1 insertion(+), 83 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/59b918ec/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index b2c8937..e06900c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -36,11 +36,6 @@ public class CommonClientConfigs { + "host1:port1,host2:port2, Since these servers are just used for the initial connection to " + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of " + "servers (you may want more than one, though, in case a server is down)."; -/** - * @deprecated This will be removed in a future release. Please use {@link #BOOTSTRAP_SERVERS_DOC} - */ -@Deprecated -public static final String BOOSTRAP_SERVERS_DOC = BOOTSTRAP_SERVERS_DOC; public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions."; http://git-wip-us.apache.org/repos/asf/kafka/blob/59b918ec/clients/src/main/java/org/apache/kafka/common/Cluster.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 6619b4c..0c59f33 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -46,20 +46,6 @@ public final class Cluster { private final ClusterResource clusterResource; /** - * Create a new cluster with the given nodes and partitions - * @param nodes The nodes in the cluster - * @param partitions Information about a subset of the topic-partitions this cluster hosts - * @deprecated Use the Cluster constructor with 5 parameters - */ -@Deprecated -public Cluster(Collection nodes, - Collection partitions, - Set unauthorizedTopics) { -this(null, false, nodes, partitions, unauthorizedTopics, Collections.emptySet(), null); -} - - -/** * Create a new cluster with the given id, nodes and partitions * @param nodes The nodes in the cluster * @param partitions Information about a subset of the topic-partitions this cluster hosts http://git-wip-us.apache.org/repos/asf/kafka/blob/59b918ec/clients/src/main/java/org/apache/kafka/common/MetricName.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/MetricName.java b/clients/src/main/java/org/apache/kafka/common/MetricName.java index cd1ae4e..2136a72 100644 --- a/clients/src/main/java/org/apache/kafka/common/MetricName.java +++ b/clients/src/main/java/org/apache/kafka/common/MetricName.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.common; -import java.util.HashMap; import java.util.Map; import org.apache.kafka.commo
kafka git commit: HOTFIX: Increase kafkatest startup wait time on ConnectDistributed service
Repository: kafka Updated Branches: refs/heads/trunk c69842336 -> 8ace736f7 HOTFIX: Increase kafkatest startup wait time on ConnectDistributed service Author: Konstantine Karantasis Reviewers: Magesh Nandakumar , Jason Gustafson Closes #3006 from kkonstantine/HOTFIX-Align-startup-wait-time-for-ConnectDistributed-service-with-ConnectStandalone-in-kafkatests Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8ace736f Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8ace736f Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8ace736f Branch: refs/heads/trunk Commit: 8ace736f7121161518953b4ce2a03f36a8306e18 Parents: c698423 Author: Konstantine Karantasis Authored: Tue May 9 14:02:30 2017 -0700 Committer: Jason Gustafson Committed: Tue May 9 14:02:30 2017 -0700 -- tests/kafkatest/services/connect.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/8ace736f/tests/kafkatest/services/connect.py -- diff --git a/tests/kafkatest/services/connect.py b/tests/kafkatest/services/connect.py index 5a8581a..a27acd7 100644 --- a/tests/kafkatest/services/connect.py +++ b/tests/kafkatest/services/connect.py @@ -272,7 +272,7 @@ class ConnectDistributedService(ConnectServiceBase): cmd = self.start_cmd(node) self.logger.debug("Connect distributed command: %s", cmd) node.account.ssh(cmd) -monitor.wait_until('Kafka Connect started', timeout_sec=15, err_msg="Never saw message indicating Kafka Connect finished startup on " + str(node.account)) +monitor.wait_until('Kafka Connect started', timeout_sec=30, err_msg="Never saw message indicating Kafka Connect finished startup on " + str(node.account)) if len(self.pids(node)) == 0: raise RuntimeError("No process ids recorded")
kafka git commit: HOTFIX: Increase kafkatest startup wait time on ConnectDistributed service in 0.10.2
Repository: kafka Updated Branches: refs/heads/0.10.2 0e8b08477 -> bce189fa4 HOTFIX: Increase kafkatest startup wait time on ConnectDistributed service in 0.10.2 Author: Konstantine Karantasis Reviewers: Jason Gustafson Closes #3007 from kkonstantine/HOTFIX-Align-startup-wait-time-for-ConnectDistributed-service-with-ConnectStandalone-in-kafkatests-0.10.2 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/bce189fa Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/bce189fa Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/bce189fa Branch: refs/heads/0.10.2 Commit: bce189fa451141df9f5b1c86a307a321caeaf807 Parents: 0e8b084 Author: Konstantine Karantasis Authored: Tue May 9 14:20:49 2017 -0700 Committer: Jason Gustafson Committed: Tue May 9 14:20:49 2017 -0700 -- tests/kafkatest/services/connect.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/bce189fa/tests/kafkatest/services/connect.py -- diff --git a/tests/kafkatest/services/connect.py b/tests/kafkatest/services/connect.py index 5a8581a..a27acd7 100644 --- a/tests/kafkatest/services/connect.py +++ b/tests/kafkatest/services/connect.py @@ -272,7 +272,7 @@ class ConnectDistributedService(ConnectServiceBase): cmd = self.start_cmd(node) self.logger.debug("Connect distributed command: %s", cmd) node.account.ssh(cmd) -monitor.wait_until('Kafka Connect started', timeout_sec=15, err_msg="Never saw message indicating Kafka Connect finished startup on " + str(node.account)) +monitor.wait_until('Kafka Connect started', timeout_sec=30, err_msg="Never saw message indicating Kafka Connect finished startup on " + str(node.account)) if len(self.pids(node)) == 0: raise RuntimeError("No process ids recorded")
kafka git commit: KAFKA-5213; Mark a MemoryRecordsBuilder as full as soon as the append stream is closed
Repository: kafka Updated Branches: refs/heads/trunk 0bede30ad -> 970c00eab KAFKA-5213; Mark a MemoryRecordsBuilder as full as soon as the append stream is closed Author: Apurva Mehta Reviewers: Ismael Juma , Jason Gustafson Closes #3015 from apurvam/KAFKA-5213-illegalstateexception-in-ensureOpenForAppend Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/970c00ea Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/970c00ea Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/970c00ea Branch: refs/heads/trunk Commit: 970c00eab80d82b97456e276ee0f5615cb1ccfa1 Parents: 0bede30 Author: Apurva Mehta Authored: Wed May 10 18:15:54 2017 -0700 Committer: Jason Gustafson Committed: Wed May 10 18:15:54 2017 -0700 -- .../kafka/common/record/MemoryRecordsBuilder.java| 2 +- .../producer/internals/ProducerBatchTest.java| 15 +++ 2 files changed, 16 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/970c00ea/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index f7451cf..025b402 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -664,7 +664,7 @@ public class MemoryRecordsBuilder { public boolean isFull() { // note that the write limit is respected only after the first record is added which ensures we can always // create non-empty batches (this is used to disable batching when the producer's batch size is set to 0). -return isClosed() || (this.numRecords > 0 && this.writeLimit <= estimatedBytesWritten()); +return appendStreamIsClosed || (this.numRecords > 0 && this.writeLimit <= estimatedBytesWritten()); } public int sizeInBytes() { http://git-wip-us.apache.org/repos/asf/kafka/blob/970c00ea/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java -- diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java index 6895fce..fede528 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java @@ -20,12 +20,16 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; +import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.TimestampType; import org.junit.Test; import java.nio.ByteBuffer; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; public class ProducerBatchTest { @@ -70,4 +74,15 @@ public class ProducerBatchTest { // Set `now` to 2ms before the create time. assertFalse(batch.maybeExpire(10240, 10240L, now - 2L, 10240L, true)); } + +@Test +public void testShouldNotAttemptAppendOnceRecordsBuilderIsClosedForAppends() { +ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); +FutureRecordMetadata result0 = batch.tryAppend(now, null, new byte[10], Record.EMPTY_HEADERS, null, now); +assertNotNull(result0); +assertTrue(memoryRecordsBuilder.hasRoomFor(now, null, new byte[10])); +memoryRecordsBuilder.closeForRecordAppends(); +assertFalse(memoryRecordsBuilder.hasRoomFor(now, null, new byte[10])); +assertEquals(null, batch.tryAppend(now + 1, null, new byte[10], Record.EMPTY_HEADERS, null, now + 1)); +} }
kafka git commit: MINOR: Fix consumer and producer to actually support metrics recording level
Repository: kafka Updated Branches: refs/heads/trunk 495184916 -> 1c2bbaa50 MINOR: Fix consumer and producer to actually support metrics recording level Also add tests and a few clean-ups. Author: Ismael Juma Reviewers: Eno Thereska , Jason Gustafson Closes #2937 from ijuma/metrics-recording-level-producer Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1c2bbaa5 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1c2bbaa5 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1c2bbaa5 Branch: refs/heads/trunk Commit: 1c2bbaa501c2d0fd4db9c2dacacc3ff7f5236d3d Parents: 4951849 Author: Ismael Juma Authored: Fri May 12 10:36:44 2017 -0700 Committer: Jason Gustafson Committed: Fri May 12 10:36:44 2017 -0700 -- .../kafka/clients/consumer/KafkaConsumer.java | 10 +++-- .../kafka/clients/producer/KafkaProducer.java | 8 ++-- .../kafka/clients/producer/ProducerConfig.java | 18 +++- .../apache/kafka/common/network/Selector.java | 20 + .../clients/consumer/KafkaConsumerTest.java | 15 +++ .../clients/producer/KafkaProducerTest.java | 46 +--- 6 files changed, 84 insertions(+), 33 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/1c2bbaa5/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 9df674d..aad4453 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -41,6 +41,7 @@ import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.ChannelBuilder; import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.requests.IsolationLevel; @@ -58,7 +59,6 @@ import java.util.Collections; import java.util.ConcurrentModificationException; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -519,6 +519,9 @@ public class KafkaConsumer implements Consumer { private static final String JMX_PREFIX = "kafka.consumer"; static final long DEFAULT_CLOSE_TIMEOUT_MS = 30 * 1000; +// Visible for testing +final Metrics metrics; + private final String clientId; private final ConsumerCoordinator coordinator; private final Deserializer keyDeserializer; @@ -528,7 +531,6 @@ public class KafkaConsumer implements Consumer { private final Time time; private final ConsumerNetworkClient client; -private final Metrics metrics; private final SubscriptionState subscriptions; private final Metadata metadata; private final long retryBackoffMs; @@ -622,10 +624,10 @@ public class KafkaConsumer implements Consumer { if (clientId.length() <= 0) clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement(); this.clientId = clientId; -Map metricsTags = new LinkedHashMap<>(); -metricsTags.put("client-id", clientId); +Map metricsTags = Collections.singletonMap("client-id", clientId); MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS) + .recordLevel(Sensor.RecordingLevel.forName(config.getString(ConsumerConfig.METRICS_RECORDING_LEVEL_CONFIG))) .tags(metricsTags); List reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); http://git-wip-us.apache.org/repos/asf/kafka/blob/1c2bbaa5/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index f812389..b1f405a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -64,7 +64,6 @@ import org.slf4j.LoggerFactory; import java.net.InetSock
[1/2] kafka git commit: KAFKA-5196; Make LogCleaner transaction-aware
Repository: kafka Updated Branches: refs/heads/trunk 1c2bbaa50 -> 7baa58d79 http://git-wip-us.apache.org/repos/asf/kafka/blob/7baa58d7/core/src/test/scala/unit/kafka/log/LogTest.scala -- diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index b4fe9fb..2283077 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -375,9 +375,10 @@ class LogTest { producerId: Long, epoch: Short, offset: Long = 0L, -coordinatorEpoch: Int = 0): MemoryRecords = { +coordinatorEpoch: Int = 0, +partitionLeaderEpoch: Int = 0): MemoryRecords = { val marker = new EndTransactionMarker(controlRecordType, coordinatorEpoch) -MemoryRecords.withEndTransactionMarker(offset, producerId, epoch, marker) +MemoryRecords.withEndTransactionMarker(offset, time.milliseconds(), partitionLeaderEpoch, producerId, epoch, marker) } @Test @@ -2382,22 +2383,22 @@ class LogTest { private def allAbortedTransactions(log: Log) = log.logSegments.flatMap(_.txnIndex.allAbortedTxns) - private def appendTransactionalAsLeader(log: Log, pid: Long, producerEpoch: Short): Int => Unit = { + private def appendTransactionalAsLeader(log: Log, producerId: Long, producerEpoch: Short): Int => Unit = { var sequence = 0 numRecords: Int => { val simpleRecords = (sequence until sequence + numRecords).map { seq => new SimpleRecord(s"$seq".getBytes) } - val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, + val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, simpleRecords: _*) log.appendAsLeader(records, leaderEpoch = 0) sequence += numRecords } } - private def appendEndTxnMarkerAsLeader(log: Log, pid: Long, producerEpoch: Short, + private def appendEndTxnMarkerAsLeader(log: Log, producerId: Long, producerEpoch: Short, controlType: ControlRecordType, coordinatorEpoch: Int = 0): Unit = { -val records = endTxnRecords(controlType, pid, producerEpoch, coordinatorEpoch = coordinatorEpoch) +val records = endTxnRecords(controlType, producerId, producerEpoch, coordinatorEpoch = coordinatorEpoch) log.appendAsLeader(records, isFromClient = false, leaderEpoch = 0) } @@ -2409,10 +2410,10 @@ class LogTest { log.appendAsLeader(records, leaderEpoch = 0) } - private def appendTransactionalToBuffer(buffer: ByteBuffer, pid: Long, epoch: Short): (Long, Int) => Unit = { + private def appendTransactionalToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short): (Long, Int) => Unit = { var sequence = 0 (offset: Long, numRecords: Int) => { - val builder = MemoryRecords.builder(buffer, CompressionType.NONE, offset, pid, epoch, sequence, true) + val builder = MemoryRecords.builder(buffer, CompressionType.NONE, offset, producerId, producerEpoch, sequence, true) for (seq <- sequence until sequence + numRecords) { val record = new SimpleRecord(s"$seq".getBytes) builder.append(record) @@ -2424,9 +2425,9 @@ class LogTest { } private def appendEndTxnMarkerToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, offset: Long, -controlType: ControlRecordType, coordinatorEpoch: Int = 0): Unit = { + controlType: ControlRecordType, coordinatorEpoch: Int = 0): Unit = { val marker = new EndTransactionMarker(controlType, coordinatorEpoch) -MemoryRecords.writeEndTransactionalMarker(buffer, offset, producerId, producerEpoch, marker) +MemoryRecords.writeEndTransactionalMarker(buffer, offset, time.milliseconds(), 0, producerId, producerEpoch, marker) } private def appendNonTransactionalToBuffer(buffer: ByteBuffer, offset: Long, numRecords: Int): Unit = { http://git-wip-us.apache.org/repos/asf/kafka/blob/7baa58d7/core/src/test/scala/unit/kafka/log/TransactionIndexTest.scala -- diff --git a/core/src/test/scala/unit/kafka/log/TransactionIndexTest.scala b/core/src/test/scala/unit/kafka/log/TransactionIndexTest.scala index 4546818..16173eb 100644 --- a/core/src/test/scala/unit/kafka/log/TransactionIndexTest.scala +++ b/core/src/test/scala/unit/kafka/log/TransactionIndexTest.scala @@ -85,15 +85,13 @@ class TransactionIndexTest extends JUnitSuite { @Test def testCollectAbortedTransactions(): Unit = { -val abortedTxns = List( +val abortedTransactions = List( new AbortedTxn(producerId = 0L, firstOffset = 0, lastOffset = 10,
[2/2] kafka git commit: KAFKA-5196; Make LogCleaner transaction-aware
KAFKA-5196; Make LogCleaner transaction-aware Author: Jason Gustafson Reviewers: Jun Rao Closes #3008 from hachikuji/KAFKA-5196 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/7baa58d7 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/7baa58d7 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/7baa58d7 Branch: refs/heads/trunk Commit: 7baa58d797126b6fb2b1de30e72428895d2bcb40 Parents: 1c2bbaa Author: Jason Gustafson Authored: Fri May 12 12:07:22 2017 -0700 Committer: Jason Gustafson Committed: Fri May 12 12:07:22 2017 -0700 -- .../clients/consumer/internals/Fetcher.java | 38 ++- .../kafka/common/record/MemoryRecords.java | 51 +++- .../clients/consumer/internals/FetcherTest.java | 189 ++--- .../common/record/MemoryRecordsBuilderTest.java | 4 +- .../kafka/common/record/MemoryRecordsTest.java | 57 +++- core/src/main/scala/kafka/log/Log.scala | 30 +- core/src/main/scala/kafka/log/LogCleaner.scala | 257 + .../scala/kafka/log/LogCleanerManager.scala | 29 +- core/src/main/scala/kafka/log/OffsetMap.scala | 5 + .../scala/kafka/log/ProducerStateManager.scala | 2 +- .../main/scala/kafka/log/TransactionIndex.scala | 13 +- .../unit/kafka/log/LogCleanerManagerTest.scala | 48 .../scala/unit/kafka/log/LogCleanerTest.scala | 280 --- .../scala/unit/kafka/log/LogSegmentTest.scala | 26 +- .../src/test/scala/unit/kafka/log/LogTest.scala | 21 +- .../unit/kafka/log/TransactionIndexTest.scala | 11 +- 16 files changed, 840 insertions(+), 221 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/7baa58d7/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 66221c0..e4365da 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -1015,6 +1015,11 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { maybeEnsureValid(currentBatch); if (isolationLevel == IsolationLevel.READ_COMMITTED && currentBatch.hasProducerId()) { +// remove from the aborted transaction queue all aborted transactions which have begun +// before the current batch's last offset and add the associated producerIds to the +// aborted producer set + consumeAbortedTransactionsUpTo(currentBatch.lastOffset()); + long producerId = currentBatch.producerId(); if (containsAbortMarker(currentBatch)) { abortedProducerIds.remove(producerId); @@ -1072,29 +1077,18 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { return records; } -private boolean isBatchAborted(RecordBatch batch) { -/* When in READ_COMMITTED mode, we need to do the following for each incoming entry: -* 0. Check whether the pid is in the 'abortedProducerIds' set && the entry does not include an abort marker. -* If so, skip the entry. -* 1. If the pid is in aborted pids and the entry contains an abort marker, remove the pid from -* aborted pids and skip the entry. -* 2. Check lowest offset entry in the abort index. If the PID of the current entry matches the -* pid of the abort index entry, and the incoming offset is no smaller than the abort index offset, -* this means that the entry has been aborted. Add the pid to the aborted pids set, and remove -* the entry from the abort index. -*/ -long producerId = batch.producerId(); -if (abortedProducerIds.contains(producerId)) { -return true; -} else if (abortedTransactions != null && !abortedTransactions.isEmpty()) { -FetchResponse.AbortedTransaction nextAbortedTransaction = abortedTransactions.peek(); -if (nextAbortedTransaction.producerId == producerId && nextAbortedTransaction.firstOffset <= batch.baseOffset()) { -abortedProducerIds.add(producerId); -abortedTransactions.poll(); -return true; -} +private void consumeAbortedTransactionsUpTo(long offset) { +if (abortedTransactions == null) +
kafka git commit: KAFKA-5160; KIP-98 Broker side support for TxnOffsetCommitRequest
Repository: kafka Updated Branches: refs/heads/trunk da0b5b859 -> 7258a5fdd KAFKA-5160; KIP-98 Broker side support for TxnOffsetCommitRequest This patch adds support for the `TxnOffsetCommitRequest` added in KIP-98. Desired handling for this request is [described here](https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8/edit#bookmark=id.55yzhvkppi6m) . The functionality includes handling the stable state of receiving `TxnOffsetCommitRequests` and materializing results only when the commit marker for the transaction is received. It also handles partition emigration and immigration and rebuilds the required data structures on these events. Tests are included for all the functionality. Author: Apurva Mehta Reviewers: Jason Gustafson Closes #2970 from apurvam/KAFKA-5160-broker-side-support-for-txnoffsetcommitrequest Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/7258a5fd Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/7258a5fd Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/7258a5fd Branch: refs/heads/trunk Commit: 7258a5fddf3fb77480cd414819cdbfbd96b709e5 Parents: da0b5b8 Author: Apurva Mehta Authored: Fri May 12 16:27:11 2017 -0700 Committer: Jason Gustafson Committed: Fri May 12 16:27:11 2017 -0700 -- .../kafka/common/record/MemoryRecords.java | 18 +- .../coordinator/group/GroupCoordinator.scala| 28 +- .../kafka/coordinator/group/GroupMetadata.scala | 63 - .../group/GroupMetadataManager.scala| 209 +++--- core/src/main/scala/kafka/log/Log.scala | 3 +- .../scala/kafka/log/ProducerStateManager.scala | 43 +-- .../src/main/scala/kafka/server/KafkaApis.scala | 94 ++- .../group/GroupCoordinatorResponseTest.scala| 249 - .../group/GroupMetadataManagerTest.scala| 276 +++ .../kafka/log/ProducerStateManagerTest.scala| 14 +- 10 files changed, 906 insertions(+), 91 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/7258a5fd/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index a222cc3..cec309e 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -398,8 +398,24 @@ public class MemoryRecords extends AbstractRecords { int baseSequence, boolean isTransactional, int partitionLeaderEpoch) { +return builder(buffer, magic, compressionType, timestampType, baseOffset, +logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, false, partitionLeaderEpoch); +} + +public static MemoryRecordsBuilder builder(ByteBuffer buffer, + byte magic, + CompressionType compressionType, + TimestampType timestampType, + long baseOffset, + long logAppendTime, + long producerId, + short producerEpoch, + int baseSequence, + boolean isTransactional, + boolean isControlBatch, + int partitionLeaderEpoch) { return new MemoryRecordsBuilder(buffer, magic, compressionType, timestampType, baseOffset, -logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, false, partitionLeaderEpoch, +logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, buffer.remaining()); } http://git-wip-us.apache.org/repos/asf/kafka/blob/7258a5fd/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala -- diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index f5b1a29..a57b6be 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala
[2/2] kafka git commit: MINOR: Rename InitPidRequest/InitPidResponse to InitProducerIdRequest/InitProducerIdResponse
MINOR: Rename InitPidRequest/InitPidResponse to InitProducerIdRequest/InitProducerIdResponse Author: Jason Gustafson Reviewers: Guozhang Wang Closes #2997 from hachikuji/minor-rename-initpid Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/a1c8e7d9 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/a1c8e7d9 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/a1c8e7d9 Branch: refs/heads/trunk Commit: a1c8e7d941ad9c765dac232435a297f905d5 Parents: 1cb39f7 Author: Jason Gustafson Authored: Fri May 12 19:59:34 2017 -0700 Committer: Jason Gustafson Committed: Fri May 12 19:59:34 2017 -0700 -- .../kafka/clients/producer/KafkaProducer.java | 2 +- .../clients/producer/internals/PidAndEpoch.java | 36 -- .../producer/internals/ProducerBatch.java | 4 +- .../producer/internals/ProducerIdAndEpoch.java | 36 ++ .../producer/internals/RecordAccumulator.java | 12 +- .../clients/producer/internals/Sender.java | 24 ++-- .../producer/internals/TransactionManager.java | 64 +- .../errors/InvalidTxnTimeoutException.java | 2 +- .../apache/kafka/common/protocol/Errors.java| 3 +- .../kafka/common/requests/AbstractRequest.java | 2 +- .../kafka/common/requests/AbstractResponse.java | 2 +- .../kafka/common/requests/InitPidRequest.java | 104 .../kafka/common/requests/InitPidResponse.java | 89 -- .../common/requests/InitProducerIdRequest.java | 104 .../common/requests/InitProducerIdResponse.java | 89 ++ .../clients/producer/internals/SenderTest.java | 20 ++-- .../internals/TransactionManagerTest.java | 26 ++-- .../common/requests/RequestResponseTest.java| 8 +- .../transaction/ProducerIdManager.scala | 119 ++- .../transaction/TransactionCoordinator.scala| 90 +++--- .../TransactionMarkerChannelManager.scala | 7 +- .../scala/kafka/log/ProducerStateManager.scala | 84 ++--- .../src/main/scala/kafka/server/KafkaApis.scala | 29 +++-- .../main/scala/kafka/server/KafkaConfig.scala | 2 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 6 +- .../transaction/ProducerIdManagerTest.scala | 14 +-- .../TransactionCoordinatorIntegrationTest.scala | 16 +-- .../TransactionCoordinatorTest.scala| 50 .../unit/kafka/server/RequestQuotaTest.scala| 4 +- 29 files changed, 525 insertions(+), 523 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/a1c8e7d9/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index b1f405a..05edf65 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -688,7 +688,7 @@ public class KafkaProducer implements Producer { if (transactionManager == null) return; -if (transactionManager.isTransactional() && !transactionManager.hasPid()) +if (transactionManager.isTransactional() && !transactionManager.hasProducerId()) throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions when transactions are enabled."); if (transactionManager.isFenced()) http://git-wip-us.apache.org/repos/asf/kafka/blob/a1c8e7d9/clients/src/main/java/org/apache/kafka/clients/producer/internals/PidAndEpoch.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/PidAndEpoch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/PidAndEpoch.java deleted file mode 100644 index 8647a7b..000 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/PidAndEpoch.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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 KI
[1/2] kafka git commit: MINOR: Rename InitPidRequest/InitPidResponse to InitProducerIdRequest/InitProducerIdResponse
Repository: kafka Updated Branches: refs/heads/trunk 1cb39f757 -> a1c8e7d94 http://git-wip-us.apache.org/repos/asf/kafka/blob/a1c8e7d9/core/src/main/scala/kafka/log/ProducerStateManager.scala -- diff --git a/core/src/main/scala/kafka/log/ProducerStateManager.scala b/core/src/main/scala/kafka/log/ProducerStateManager.scala index 4b2cedb..d7b1c33 100644 --- a/core/src/main/scala/kafka/log/ProducerStateManager.scala +++ b/core/src/main/scala/kafka/log/ProducerStateManager.scala @@ -73,28 +73,28 @@ private[log] class ProducerAppendInfo(val producerId: Long, initialEntry: Produc private var coordinatorEpoch = initialEntry.coordinatorEpoch private val transactions = ListBuffer.empty[TxnMetadata] - def this(pid: Long, initialEntry: Option[ProducerIdEntry], loadingFromLog: Boolean) = -this(pid, initialEntry.getOrElse(ProducerIdEntry.Empty), loadingFromLog) + def this(producerId: Long, initialEntry: Option[ProducerIdEntry], loadingFromLog: Boolean) = +this(producerId, initialEntry.getOrElse(ProducerIdEntry.Empty), loadingFromLog) - private def validateAppend(epoch: Short, firstSeq: Int, lastSeq: Int, shouldValidateSequenceNumbers: Boolean) = { -if (this.producerEpoch > epoch) { + private def validateAppend(producerEpoch: Short, firstSeq: Int, lastSeq: Int, shouldValidateSequenceNumbers: Boolean) = { +if (this.producerEpoch > producerEpoch) { throw new ProducerFencedException(s"Producer's epoch is no longer valid. There is probably another producer " + -s"with a newer epoch. $epoch (request epoch), ${this.producerEpoch} (server epoch)") +s"with a newer epoch. $producerEpoch (request epoch), ${this.producerEpoch} (server epoch)") } else if (shouldValidateSequenceNumbers) { - if (this.producerEpoch == RecordBatch.NO_PRODUCER_EPOCH || this.producerEpoch < epoch) { + if (this.producerEpoch == RecordBatch.NO_PRODUCER_EPOCH || this.producerEpoch < producerEpoch) { if (firstSeq != 0) - throw new OutOfOrderSequenceException(s"Invalid sequence number for new epoch: $epoch " + + throw new OutOfOrderSequenceException(s"Invalid sequence number for new epoch: $producerEpoch " + s"(request epoch), $firstSeq (seq. number)") } else if (this.firstSeq == RecordBatch.NO_SEQUENCE && firstSeq != 0) { // the epoch was bumped by a control record, so we expect the sequence number to be reset -throw new OutOfOrderSequenceException(s"Out of order sequence number: $producerId (pid), found $firstSeq " + +throw new OutOfOrderSequenceException(s"Out of order sequence number for producerId $producerId: found $firstSeq " + s"(incoming seq. number), but expected 0") } else if (firstSeq == this.firstSeq && lastSeq == this.lastSeq) { -throw new DuplicateSequenceNumberException(s"Duplicate sequence number: pid: $producerId, (incomingBatch.firstSeq, " + +throw new DuplicateSequenceNumberException(s"Duplicate sequence number for producerId $producerId: (incomingBatch.firstSeq, " + s"incomingBatch.lastSeq): ($firstSeq, $lastSeq), (lastEntry.firstSeq, lastEntry.lastSeq): " + s"(${this.firstSeq}, ${this.lastSeq}).") } else if (firstSeq != this.lastSeq + 1L) { -throw new OutOfOrderSequenceException(s"Out of order sequence number: $producerId (pid), $firstSeq " + +throw new OutOfOrderSequenceException(s"Out of order sequence number for producerId $producerId: $firstSeq " + s"(incoming seq. number), ${this.lastSeq} (current end sequence number)") } } @@ -202,25 +202,25 @@ private[log] class ProducerAppendInfo(val producerId: Long, initialEntry: Produc } object ProducerStateManager { - private val PidSnapshotVersion: Short = 1 + private val ProducerSnapshotVersion: Short = 1 private val VersionField = "version" private val CrcField = "crc" - private val PidField = "pid" + private val ProducerIdField = "producer_id" private val LastSequenceField = "last_sequence" private val ProducerEpochField = "epoch" private val LastOffsetField = "last_offset" private val OffsetDeltaField = "offset_delta" private val TimestampField = "timestamp" - private val PidEntriesField = "pid_entries" + private val ProducerEntriesField = "producer_entries" private val CoordinatorEpochField = "coordinator_epoch" private val CurrentTxnFirstOffsetField = "current_txn_first_offset" private val VersionOffset = 0 private val CrcOffset = VersionOffset + 2 - private val PidEntriesOffset = CrcOffset + 4 + private val ProducerEntriesOffset = CrcOffset + 4 - val PidSnapshotEntrySchema = new Schema( -new Field(PidField, Type.INT64, "The producer ID"), + val ProducerSnapshotEntrySchema = new Schema( +new Field(ProducerIdField, Type.INT64, "The producer ID"), new Field(ProducerEpochField, Type.INT16, "Cur
kafka git commit: HOTFIX: AddOffsetsToTxnResponse using incorrect schema in parse
Repository: kafka Updated Branches: refs/heads/trunk a511a4702 -> 82e84fabf HOTFIX: AddOffsetsToTxnResponse using incorrect schema in parse The parse method was incorrectly referring to `ApiKeys.ADD_PARTITIONS_TO_TXN` Author: Damian Guy Reviewers: Ismael Juma , Jason Gustafson Closes #3056 from dguy/hotfix-add-offsets-to-txn-response Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/82e84fab Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/82e84fab Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/82e84fab Branch: refs/heads/trunk Commit: 82e84fabf87fa5b5e6b4b9fd464edafaa9c5a744 Parents: a511a47 Author: Damian Guy Authored: Mon May 15 11:15:22 2017 -0700 Committer: Jason Gustafson Committed: Mon May 15 11:15:22 2017 -0700 -- .../org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/82e84fab/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java index 8c41ae4..8b3a589 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java @@ -65,6 +65,6 @@ public class AddOffsetsToTxnResponse extends AbstractResponse { } public static AddOffsetsToTxnResponse parse(ByteBuffer buffer, short version) { -return new AddOffsetsToTxnResponse(ApiKeys.ADD_PARTITIONS_TO_TXN.parseResponse(version, buffer)); +return new AddOffsetsToTxnResponse(ApiKeys.ADD_OFFSETS_TO_TXN.parseResponse(version, buffer)); } }
kafka git commit: MINOR: Eliminate PID terminology from non test code
Repository: kafka Updated Branches: refs/heads/trunk e40e27b4e -> 3e6669000 MINOR: Eliminate PID terminology from non test code Producer id is used instead. Also refactored TransactionLog schema code to follow our naming convention and to have better structure. Author: Ismael Juma Reviewers: Guozhang Wang , Jason Gustafson Closes #3041 from ijuma/eliminate-pid-terminology Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/3e666900 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/3e666900 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/3e666900 Branch: refs/heads/trunk Commit: 3e6669000f082808999a7216b00c4b0f5a94e1da Parents: e40e27b Author: Ismael Juma Authored: Mon May 15 11:26:08 2017 -0700 Committer: Jason Gustafson Committed: Mon May 15 11:26:08 2017 -0700 -- .../producer/internals/RecordAccumulator.java | 10 +- .../clients/producer/internals/Sender.java | 27 +-- .../producer/internals/TransactionManager.java | 10 +- .../apache/kafka/common/protocol/Errors.java| 4 +- .../kafka/common/record/DefaultRecordBatch.java | 4 +- .../common/record/MemoryRecordsBuilder.java | 10 +- .../apache/kafka/common/record/RecordBatch.java | 4 +- .../common/requests/AddOffsetsToTxnRequest.java | 6 +- .../requests/AddPartitionsToTxnRequest.java | 12 +- .../kafka/common/requests/EndTxnRequest.java| 12 +- .../kafka/common/requests/FetchResponse.java| 6 +- .../common/requests/TxnOffsetCommitRequest.java | 12 +- .../common/requests/WriteTxnMarkersRequest.java | 12 +- .../requests/WriteTxnMarkersResponse.java | 6 +- .../clients/producer/internals/SenderTest.java | 4 +- .../internals/TransactionManagerTest.java | 4 +- .../kafka/coordinator/group/GroupMetadata.scala | 2 +- .../transaction/TransactionLog.scala| 193 ++- ...nsactionMarkerRequestCompletionHandler.scala | 2 +- .../transaction/TransactionMetadata.scala | 6 +- .../transaction/TransactionStateManager.scala | 2 +- core/src/main/scala/kafka/log/Log.scala | 22 +-- core/src/main/scala/kafka/log/LogManager.scala | 4 +- .../scala/kafka/log/ProducerStateManager.scala | 22 +-- .../main/scala/kafka/server/KafkaConfig.scala | 8 +- .../kafka/tools/ConsumerOffsetChecker.scala | 20 +- .../scala/kafka/tools/DumpLogSegments.scala | 2 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 2 +- .../src/test/scala/unit/kafka/log/LogTest.scala | 6 +- 29 files changed, 221 insertions(+), 213 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/3e666900/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index cf3736c..d53c19d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -446,19 +446,19 @@ public final class RecordAccumulator { } else { ProducerIdAndEpoch producerIdAndEpoch = null; if (transactionManager != null) { -producerIdAndEpoch = transactionManager.pidAndEpoch(); +producerIdAndEpoch = transactionManager.producerIdAndEpoch(); if (!producerIdAndEpoch.isValid()) -// we cannot send the batch until we have refreshed the PID +// we cannot send the batch until we have refreshed the producer id break; } ProducerBatch batch = deque.pollFirst(); if (producerIdAndEpoch != null && !batch.inRetry()) { -// If the batch is in retry, then we should not change the pid and +// If the batch is in retry, then we should not change the producer id and // sequence number, since this may introduce duplicates. In particular, // the previous attempt may actually have been accepted, and if we change -// th
kafka git commit: KAFKA-5248; Remove unused/unneeded retention time in TxnOffsetCommitRequest
Repository: kafka Updated Branches: refs/heads/trunk c26545ea5 -> 018679410 KAFKA-5248; Remove unused/unneeded retention time in TxnOffsetCommitRequest Author: Jason Gustafson Reviewers: Guozhang Wang , Apurva Mehta , Ismael Juma Closes #3058 from hachikuji/KAFKA-5248 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/01867941 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/01867941 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/01867941 Branch: refs/heads/trunk Commit: 0186794104b7106fe426024d65730fec79ad999a Parents: c26545e Author: Jason Gustafson Authored: Mon May 15 12:58:36 2017 -0700 Committer: Jason Gustafson Committed: Mon May 15 12:58:50 2017 -0700 -- .../producer/internals/TransactionManager.java | 3 +-- .../org/apache/kafka/common/protocol/Protocol.java | 3 --- .../common/requests/TxnOffsetCommitRequest.java| 17 +++-- .../kafka/common/requests/RequestResponseTest.java | 2 +- .../scala/unit/kafka/server/RequestQuotaTest.scala | 2 +- 5 files changed, 6 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/01867941/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 7e2f813..f3ed252 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -37,7 +37,6 @@ import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.InitProducerIdRequest; import org.apache.kafka.common.requests.InitProducerIdResponse; -import org.apache.kafka.common.requests.OffsetCommitRequest; import org.apache.kafka.common.requests.TransactionResult; import org.apache.kafka.common.requests.TxnOffsetCommitRequest; import org.apache.kafka.common.requests.TxnOffsetCommitRequest.CommittedOffset; @@ -461,7 +460,7 @@ public class TransactionManager { pendingTxnOffsetCommits.put(entry.getKey(), committedOffset); } TxnOffsetCommitRequest.Builder builder = new TxnOffsetCommitRequest.Builder(consumerGroupId, -producerIdAndEpoch.producerId, producerIdAndEpoch.epoch, OffsetCommitRequest.DEFAULT_RETENTION_TIME, +producerIdAndEpoch.producerId, producerIdAndEpoch.epoch, pendingTxnOffsetCommits); return new TxnOffsetCommitHandler(result, builder); } http://git-wip-us.apache.org/repos/asf/kafka/blob/01867941/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index fb3c8c9..5e05738 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -1521,9 +1521,6 @@ public class Protocol { new Field("producer_epoch", INT16, "Current epoch associated with the producer id."), -new Field("retention_time", -INT64, -"The time in ms to retain the offset."), new Field("topics", new ArrayOf(new Schema( new Field("topic", STRING), http://git-wip-us.apache.org/repos/asf/kafka/blob/01867941/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java index 3f3024f..f5334f2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java @@ -30,7 +30,6 @@ public class TxnOffsetCommitRequest extends AbstractRequest { private static final String CONSUMER_GROUP_ID_KEY_NAME = "consumer_group_id"; private static final String PRODUCER_ID_KEY_NAME = "producer_id"; private static final String PRODUCER_EPOCH_KEY_NAME = "producer_epoch"; -private static final String RETENTION_TIME_KEY_NAME = "retention_time";
kafka git commit: KAFKA-5252; Fix transient failures LogCleanerTest testCommitMarkerRemoval and testAbortMarkerRemoval
Repository: kafka Updated Branches: refs/heads/trunk dd9f43140 -> cacbba651 KAFKA-5252; Fix transient failures LogCleanerTest testCommitMarkerRemoval and testAbortMarkerRemoval Author: Jason Gustafson Reviewers: Apurva Mehta , Guozhang Wang Closes #3064 from hachikuji/KAFKA-5252 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/cacbba65 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/cacbba65 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/cacbba65 Branch: refs/heads/trunk Commit: cacbba6519a150ed25780889e8fa9b03c27832ee Parents: dd9f431 Author: Jason Gustafson Authored: Mon May 15 22:56:20 2017 -0700 Committer: Jason Gustafson Committed: Mon May 15 22:56:20 2017 -0700 -- core/src/test/scala/unit/kafka/log/LogCleanerTest.scala | 12 ++-- 1 file changed, 6 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/cacbba65/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala -- diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 704aa73..6eb65ca 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -187,7 +187,7 @@ class LogCleanerTest extends JUnitSuite { log.roll() // cannot remove the marker in this pass because there are still valid records -var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = time.milliseconds())._1 +var dirtyOffset = cleaner.doClean(LogToClean(tp, log, 0L, 100L), deleteHorizonMs = Long.MaxValue)._1 assertEquals(List(1, 3, 2), keysInLog(log)) assertEquals(List(0, 2, 3, 4, 5), offsetsInLog(log)) @@ -196,7 +196,7 @@ class LogCleanerTest extends JUnitSuite { log.roll() // the first cleaning preserves the commit marker (at offset 3) since there were still records for the transaction -dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = time.milliseconds())._1 +dirtyOffset = cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue)._1 assertEquals(List(2, 1, 3), keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) @@ -205,8 +205,8 @@ class LogCleanerTest extends JUnitSuite { assertEquals(List(2, 1, 3), keysInLog(log)) assertEquals(List(3, 4, 5, 6, 7, 8), offsetsInLog(log)) -// clean again with the delete horizon set back to the current time and verify the marker is removed -cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = time.milliseconds()) +// clean again with large delete horizon and verify the marker is removed +cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue) assertEquals(List(2, 1, 3), keysInLog(log)) assertEquals(List(4, 5, 6, 7, 8), offsetsInLog(log)) } @@ -235,8 +235,8 @@ class LogCleanerTest extends JUnitSuite { assertEquals(List(3), keysInLog(log)) assertEquals(List(3, 4, 5), offsetsInLog(log)) -// clean again with the delete horizon set back to the current time and verify the marker is removed -cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = time.milliseconds()) +// clean again with large delete horizon and verify the marker is removed +cleaner.doClean(LogToClean(tp, log, dirtyOffset, 100L), deleteHorizonMs = Long.MaxValue) assertEquals(List(3), keysInLog(log)) assertEquals(List(4, 5), offsetsInLog(log)) }
kafka git commit: KAFKA-5129; Add ACL checks for Transactional APIs
Repository: kafka Updated Branches: refs/heads/trunk a281fe17f -> d66e7af65 KAFKA-5129; Add ACL checks for Transactional APIs Add ACL checks for Transactional APIs Author: Damian Guy Reviewers: Apurva Mehta , Ismael Juma , Jason Gustafson Closes #2979 from dguy/kafka-5129 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/d66e7af6 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/d66e7af6 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/d66e7af6 Branch: refs/heads/trunk Commit: d66e7af6526f208900a5d6cb588cf47058800804 Parents: a281fe1 Author: Damian Guy Authored: Tue May 16 09:57:15 2017 -0700 Committer: Jason Gustafson Committed: Tue May 16 09:57:15 2017 -0700 -- .../kafka/clients/producer/KafkaProducer.java | 25 +- .../producer/internals/ProducerBatch.java | 4 + .../producer/internals/RecordAccumulator.java | 8 +- .../clients/producer/internals/Sender.java | 16 +- .../producer/internals/TransactionManager.java | 72 +++-- .../ProducerIdAuthorizationException.java | 23 ++ .../TransactionalIdAuthorizationException.java | 23 ++ .../apache/kafka/common/protocol/Errors.java| 21 +- .../apache/kafka/common/protocol/Protocol.java | 10 +- .../common/record/MemoryRecordsBuilder.java | 17 +- .../requests/AddPartitionsToTxnRequest.java | 7 +- .../requests/AddPartitionsToTxnResponse.java| 56 +++- .../requests/FindCoordinatorResponse.java | 10 + .../kafka/common/requests/ProduceRequest.java | 12 + .../internals/TransactionManagerTest.java | 51 +++- .../common/record/MemoryRecordsBuilderTest.java | 66 + .../common/requests/ProduceRequestTest.java | 95 ++ .../common/requests/RequestResponseTest.java| 2 +- .../scala/kafka/security/auth/Resource.scala| 2 + .../kafka/security/auth/ResourceType.scala | 11 +- .../src/main/scala/kafka/server/KafkaApis.scala | 290 +++ .../kafka/api/AuthorizerIntegrationTest.scala | 113 +++- 22 files changed, 752 insertions(+), 182 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/d66e7af6/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 05edf65..aeef92f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -694,19 +694,18 @@ public class KafkaProducer implements Producer { if (transactionManager.isFenced()) throw new ProducerFencedException("The current producer has been fenced off by a another producer using the same transactional id."); -if (transactionManager.isInTransaction()) { -if (transactionManager.isInErrorState()) { -String errorMessage = -"Cannot perform a transactional send because at least one previous transactional request has failed with errors."; -Exception lastError = transactionManager.lastError(); -if (lastError != null) -throw new KafkaException(errorMessage, lastError); -else -throw new KafkaException(errorMessage); -} -if (transactionManager.isCompletingTransaction()) -throw new IllegalStateException("Cannot call send while a commit or abort is in progress."); +if (transactionManager.isInErrorState()) { +String errorMessage = +"Cannot perform send because at least one previous transactional or idempotent request has failed with errors."; +Exception lastError = transactionManager.lastError(); +if (lastError != null) +throw new KafkaException(errorMessage, lastError); +else +throw new KafkaException(errorMessage); } +if (transactionManager.isCompletingTransaction()) +throw new IllegalStateException("Cannot call send while a commit or abort is in progress."); + } private void setReadOnly(Headers headers) { @@ -1032,7 +1031,7 @@ public class KafkaProducer implements Producer { this.userCallback.onCompletion(metadata, exception); if (exception != null && transactionManager != null) -transactionManager.maybeSetError(exception); +transactionManager.setError(exception); } } } http://git-wip-us.apache.org/repos/asf/kafka/blob/d66e7af6/clients/src/main/java/org/apache/kafka/c
kafka git commit: MINOR: Some minor improvements to TxnOffsetCommit handling
Repository: kafka Updated Branches: refs/heads/trunk d66e7af65 -> 4e3092d27 MINOR: Some minor improvements to TxnOffsetCommit handling Author: Jason Gustafson Reviewers: Ismael Juma Closes #3040 from hachikuji/txn-offset-commit-cleanups Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/4e3092d2 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/4e3092d2 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/4e3092d2 Branch: refs/heads/trunk Commit: 4e3092d27648b9db18a496d16cd44f600429dcfc Parents: d66e7af Author: Jason Gustafson Authored: Tue May 16 13:20:38 2017 -0700 Committer: Jason Gustafson Committed: Tue May 16 13:22:15 2017 -0700 -- .../coordinator/group/GroupCoordinator.scala| 84 ++-- core/src/main/scala/kafka/log/Log.scala | 14 +- core/src/main/scala/kafka/log/LogSegment.scala | 3 +- .../scala/kafka/log/ProducerStateManager.scala | 40 +- .../src/main/scala/kafka/server/KafkaApis.scala | 74 ++-- .../group/GroupCoordinatorResponseTest.scala| 14 +- .../group/GroupCoordinatorTest.scala| 36 -- .../src/test/scala/unit/kafka/log/LogTest.scala | 7 +- .../kafka/log/ProducerStateManagerTest.scala| 409 ++- 9 files changed, 350 insertions(+), 331 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/4e3092d2/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala -- diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index 031a9c1..36e3c63 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -27,8 +27,8 @@ import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.record.RecordBatch -import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetFetchResponse, TransactionResult} +import org.apache.kafka.common.record.RecordBatch.{NO_PRODUCER_EPOCH, NO_PRODUCER_ID} +import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.Time import scala.collection.{Map, Seq, immutable} @@ -48,6 +48,8 @@ class GroupCoordinator(val brokerId: Int, val heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat], val joinPurgatory: DelayedOperationPurgatory[DelayedJoin], time: Time) extends Logging { + import GroupCoordinator._ + type JoinCallback = JoinGroupResult => Unit type SyncCallback = (Array[Byte], Errors) => Unit @@ -395,42 +397,52 @@ class GroupCoordinator(val brokerId: Int, } } + def handleTxnCommitOffsets(groupId: String, + producerId: Long, + producerEpoch: Short, + offsetMetadata: immutable.Map[TopicPartition, OffsetAndMetadata], + responseCallback: immutable.Map[TopicPartition, Errors] => Unit): Unit = { +validateGroup(groupId) match { + case Some(error) => responseCallback(offsetMetadata.mapValues(_ => error)) + case None => +val group = groupManager.getGroup(groupId).getOrElse(groupManager.addGroup(new GroupMetadata(groupId))) +doCommitOffsets(group, NoMemberId, NoGeneration, producerId, producerEpoch, offsetMetadata, responseCallback) +} + } + def handleCommitOffsets(groupId: String, memberId: String, generationId: Int, offsetMetadata: immutable.Map[TopicPartition, OffsetAndMetadata], - responseCallback: immutable.Map[TopicPartition, Errors] => Unit, - producerId: Long = RecordBatch.NO_PRODUCER_ID, - producerEpoch: Short = RecordBatch.NO_PRODUCER_EPOCH) { -if (!isActive.get) { - responseCallback(offsetMetadata.mapValues(_ => Errors.COORDINATOR_NOT_AVAILABLE)) -} else if (!isCoordinatorForGroup(groupId)) { - responseCallback(offsetMetadata.mapValues(_ => Errors.NOT_COORDINATOR)) -} else if (isCoordinatorLoadInProgress(groupId)) { - responseCallback(offsetMetadata.mapValues(_ => Errors.COORDINATOR_LOAD_IN_PROGRESS)) -} else { - groupManager.getGroup(groupId) match { -case None => - if (generationId < 0) { -// the group is not relying on Kafka for group management, so allow the commit -val group = groupManager.addGroup(new GroupMetadata(groupId)) -doCommitOffsets(group, member
kafka git commit: KAFKA-4714; Flatten and Cast single message transforms (KIP-66)
Repository: kafka Updated Branches: refs/heads/trunk ebc7f7caa -> 1cea4d8f5 KAFKA-4714; Flatten and Cast single message transforms (KIP-66) Author: Ewen Cheslack-Postava Reviewers: Konstantine Karantasis , Shikhar Bhushan , Jason Gustafson Closes #2458 from ewencp/kafka-3209-even-more-transforms Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1cea4d8f Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1cea4d8f Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1cea4d8f Branch: refs/heads/trunk Commit: 1cea4d8f5a51cc5795ddd3af2ea015b9e14d937d Parents: ebc7f7c Author: Ewen Cheslack-Postava Authored: Tue May 16 23:05:35 2017 -0700 Committer: Jason Gustafson Committed: Tue May 16 23:05:35 2017 -0700 -- .../kafka/connect/data/SchemaBuilder.java | 2 +- .../kafka/connect/tools/TransformationDoc.java | 6 +- .../apache/kafka/connect/transforms/Cast.java | 417 +++ .../kafka/connect/transforms/Flatten.java | 281 + .../connect/transforms/util/Requirements.java | 1 + .../connect/transforms/util/SchemaUtil.java | 4 + .../kafka/connect/transforms/CastTest.java | 384 + .../kafka/connect/transforms/FlattenTest.java | 257 8 files changed, 1350 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java -- diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java index f0c4586..058660e 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java @@ -75,7 +75,7 @@ public class SchemaBuilder implements Schema { // Additional parameters for logical types. private Map parameters; -private SchemaBuilder(Type type) { +public SchemaBuilder(Type type) { this.type = type; if (type == Type.STRUCT) { fields = new LinkedHashMap<>(); http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java -- diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java index 5616613..1a8f0a8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -17,7 +17,9 @@ package org.apache.kafka.connect.tools; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.transforms.Cast; import org.apache.kafka.connect.transforms.ExtractField; +import org.apache.kafka.connect.transforms.Flatten; import org.apache.kafka.connect.transforms.HoistField; import org.apache.kafka.connect.transforms.InsertField; import org.apache.kafka.connect.transforms.MaskField; @@ -54,7 +56,9 @@ public class TransformationDoc { new DocInfo(ExtractField.class.getName(), ExtractField.OVERVIEW_DOC, ExtractField.CONFIG_DEF), new DocInfo(SetSchemaMetadata.class.getName(), SetSchemaMetadata.OVERVIEW_DOC, SetSchemaMetadata.CONFIG_DEF), new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF), -new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF) +new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF), +new DocInfo(Flatten.class.getName(), Flatten.OVERVIEW_DOC, Flatten.CONFIG_DEF), +new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF) ); private static void printTransformationHtml(PrintStream out, DocInfo docInfo) { http://git-wip-us.apache.org/repos/asf/kafka/blob/1cea4d8f/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java -- diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java new file mode 100644 index 000..17be48c --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java @@ -0,0 +1,417 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license ag
kafka git commit: KAFKA-4743; [KIP-122] Add Reset Consumer Group Offsets tooling
Repository: kafka Updated Branches: refs/heads/trunk 6910baf54 -> 2181ae768 KAFKA-4743; [KIP-122] Add Reset Consumer Group Offsets tooling Author: Jorge Quilcate Reviewers: Ismael Juma , Jason Gustafson Closes #2624 from jeqo/feature/rewind-consumer-group-offset Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/2181ae76 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/2181ae76 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/2181ae76 Branch: refs/heads/trunk Commit: 2181ae768719a9ae3a929ba875faa89c67edf643 Parents: 6910baf Author: Jorge Quilcate Authored: Wed May 17 14:24:27 2017 -0700 Committer: Jason Gustafson Committed: Wed May 17 14:24:40 2017 -0700 -- .../kafka/admin/ConsumerGroupCommand.scala | 302 +- .../admin/ResetConsumerGroupOffsetTest.scala| 601 +++ 2 files changed, 874 insertions(+), 29 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/2181ae76/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala -- diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index dd7a477..69f0d8a 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -17,7 +17,9 @@ package kafka.admin -import java.util.Properties +import java.text.SimpleDateFormat +import java.util.{Date, Properties} +import javax.xml.datatype.DatatypeFactory import joptsimple.{OptionParser, OptionSpec} import kafka.api.{OffsetFetchRequest, OffsetFetchResponse, OffsetRequest, PartitionOffsetRequestInfo} @@ -27,7 +29,7 @@ import kafka.consumer.SimpleConsumer import kafka.utils._ import org.I0Itec.zkclient.exception.ZkNoNodeException import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata} import org.apache.kafka.common.errors.BrokerNotAvailableException import org.apache.kafka.common.{KafkaException, Node, TopicPartition} import org.apache.kafka.common.internals.Topic @@ -38,7 +40,7 @@ import org.apache.kafka.common.serialization.StringDeserializer import org.apache.kafka.common.utils.Utils import scala.collection.JavaConverters._ -import scala.collection.{Set, mutable} +import scala.collection.{Seq, Set, mutable} object ConsumerGroupCommand extends Logging { @@ -46,12 +48,12 @@ object ConsumerGroupCommand extends Logging { val opts = new ConsumerGroupCommandOptions(args) if (args.length == 0) - CommandLineUtils.printUsageAndDie(opts.parser, "List all consumer groups, describe a consumer group, or delete consumer group info.") + CommandLineUtils.printUsageAndDie(opts.parser, "List all consumer groups, describe a consumer group, delete consumer group info, or reset consumer group offsets.") // should have exactly one action -val actions = Seq(opts.listOpt, opts.describeOpt, opts.deleteOpt).count(opts.options.has _) +val actions = Seq(opts.listOpt, opts.describeOpt, opts.deleteOpt, opts.resetOffsetsOpt).count(opts.options.has _) if (actions != 1) - CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: --list, --describe, --delete") + CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: --list, --describe, --delete, --reset-offset") opts.checkArgs() @@ -102,6 +104,15 @@ object ConsumerGroupCommand extends Logging { case _ => throw new IllegalStateException(s"delete is not supported for $consumerGroupService.") } } + else if (opts.options.has(opts.resetOffsetsOpt)) { +val offsetsToReset = consumerGroupService.resetOffsets() +val export = opts.options.has(opts.exportOpt) +if (export) { + val exported = consumerGroupService.exportOffsetsToReset(offsetsToReset) + println(exported) +} else + printOffsetsToReset(offsetsToReset) + } } catch { case e: Throwable => printError(s"Executing consumer group command failed due to ${e.getMessage}", Some(e)) @@ -134,6 +145,20 @@ object ConsumerGroupCommand extends Logging { } } + def printOffsetsToReset(groupAssignmentsToReset: Map[TopicPartition, OffsetAndMetadata]): Unit = { +print("\n%-30s %-10s %-15s".format("TOPIC", "PARTITION", "NEW-OFFSET")) +println() + +groupAssignmentsToReset.foreach { + case (consumerAssignment, offsetAndMetadata) => +print("%-30s %-10s %-15s".format( + consumerAssignment.topic(), +
kafka git commit: KAFKA-5188; Integration tests for transactions
Repository: kafka Updated Branches: refs/heads/trunk 2181ae768 -> b3a33ce4b KAFKA-5188; Integration tests for transactions Author: Apurva Mehta Author: Jason Gustafson Reviewers: Jason Gustafson Closes #2994 from apurvam/KAFKA-5188-exactly-once-integration-tests Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b3a33ce4 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b3a33ce4 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b3a33ce4 Branch: refs/heads/trunk Commit: b3a33ce4b81a20ae5635cf28490fd2e1f9d86141 Parents: 2181ae7 Author: Apurva Mehta Authored: Wed May 17 16:20:33 2017 -0700 Committer: Jason Gustafson Committed: Wed May 17 16:20:33 2017 -0700 -- .../clients/producer/internals/Sender.java | 59 ++- .../producer/internals/TransactionManager.java | 25 +- .../kafka/common/requests/EndTxnRequest.java| 4 + .../internals/TransactionManagerTest.java | 47 ++- .../transaction/TransactionCoordinator.scala| 24 +- .../transaction/TransactionMetadata.scala | 2 +- .../transaction/TransactionStateManager.scala | 1 - .../kafka/api/TransactionsBounceTest.scala | 185 + .../kafka/api/TransactionsTest.scala| 400 +++ .../test/scala/unit/kafka/utils/TestUtils.scala | 95 - 10 files changed, 806 insertions(+), 36 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/b3a33ce4/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 209a979..7180171 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.NetworkClientUtils; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; @@ -39,7 +40,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; -import org.apache.kafka.clients.NetworkClientUtils; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.InitProducerIdRequest; @@ -187,10 +187,12 @@ public class Sender implements Runnable { * @param now The current POSIX time in milliseconds */ void run(long now) { -long pollTimeout = 0; -if (!maybeSendTransactionalRequest(now)) +long pollTimeout = retryBackoffMs; +if (!maybeSendTransactionalRequest(now)) { pollTimeout = sendProducerData(now); +} +log.trace("waiting {}ms in poll", pollTimeout); this.client.poll(pollTimeout, now); } @@ -203,6 +205,7 @@ public class Sender implements Runnable { final KafkaException exception = transactionManager.lastError() instanceof KafkaException ? (KafkaException) transactionManager.lastError() : new KafkaException(transactionManager.lastError()); +log.error("aborting producer batches because the transaction manager is in an error state.", exception); this.accumulator.abortBatches(exception); return Long.MAX_VALUE; } @@ -281,25 +284,35 @@ public class Sender implements Runnable { } private boolean maybeSendTransactionalRequest(long now) { -if (transactionManager != null && transactionManager.hasInflightRequest()) +if (transactionManager == null || !transactionManager.isTransactional()) +return false; + +if (transactionManager.hasInflightRequest()) { +log.trace("TransactionalId: {} -- There is already an inflight transactional request. Going to wait for the response.", +transactionManager.transactionalId()); return true; +} -if (transactionManager == null || !transactionManager.hasPendingTransactionalRequests()) +if (!transactionManager.hasPendingTransactionalRequests()) { +log.trace("TransactionalId: {} -- There are no pending transactional requests to send", transactionManager.transactionalId()); return false; +}
[1/2] kafka git commit: KAFKA-2273; Sticky partition assignment strategy (KIP-54)
Repository: kafka Updated Branches: refs/heads/trunk 9815e18fe -> e1abf1770 http://git-wip-us.apache.org/repos/asf/kafka/blob/e1abf177/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala -- diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 4a49833..e565ce2 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -399,7 +399,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(0, consumer0.assignment().size) -val pattern1 = Pattern.compile(".*o.*") // only 'topic' and 'foo' match this +val pattern1 = Pattern.compile(".*o.*") // only 'topic' and 'foo' match this consumer0.subscribe(pattern1, new TestConsumerReassignmentListener) consumer0.poll(50) @@ -883,6 +883,58 @@ class PlaintextConsumerTest extends BaseConsumerTest { } } + def reverse(m: Map[Long, Set[TopicPartition]]) = +m.values.toSet.flatten.map(v => (v, m.keys.filter(m(_).contains(v)).head)).toMap + + /** + * This test runs the following scenario to verify sticky assignor behavior. + * Topics: single-topic, with random number of partitions, where #par is 10, 20, 30, 40, 50, 60, 70, 80, 90, or 100 + * Consumers: 9 consumers subscribed to the single topic + * Expected initial assignment: partitions are assigned to consumers in a round robin fashion. + * - (#par mod 9) consumers will get (#par / 9 + 1) partitions, and the rest get (#par / 9) partitions + * Then consumer #10 is added to the list (subscribing to the same single topic) + * Expected new assignment: + * - (#par / 10) partition per consumer, where one partition from each of the early (#par mod 9) consumers + *will move to consumer #10, leading to a total of (#par mod 9) partition movement + */ + @Test + def testMultiConsumerStickyAssignment() { +this.consumers.clear() +this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "sticky-group") + this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[StickyAssignor].getName) + +// create one new topic +val topic = "single-topic" +val rand = 1 + scala.util.Random.nextInt(10) +val partitions = createTopicAndSendRecords(topic, rand * 10, 100) + +// create a group of consumers, subscribe the consumers to the single topic and start polling +// for the topic partition assignment +val (_, consumerPollers) = createConsumerGroupAndWaitForAssignment(9, List(topic), partitions) +validateGroupAssignment(consumerPollers, partitions, s"Did not get valid initial assignment for partitions ${partitions.asJava}") +val prePartition2PollerId = reverse(consumerPollers.map(poller => (poller.getId, poller.consumerAssignment())).toMap) + +// add one more consumer and validate re-assignment +addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic), partitions) + +val postPartition2PollerId = reverse(consumerPollers.map(poller => (poller.getId, poller.consumerAssignment())).toMap) +val keys = prePartition2PollerId.keySet.union(postPartition2PollerId.keySet) +var changes = 0 +keys.foreach { key => + val preVal = prePartition2PollerId.get(key) + val postVal = postPartition2PollerId.get(key) + if (preVal.nonEmpty && postVal.nonEmpty) { +if (preVal.get != postVal.get) + changes += 1 + } else +changes += 1 +} + +consumerPollers.foreach(_.shutdown()) + +assertEquals("Expected only two topic partitions that have switched to other consumers.", rand, changes) + } + /** * This test re-uses BaseConsumerTest's consumers. * As a result, it is testing the default assignment strategy set by BaseConsumerTest @@ -1477,8 +1529,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { * Subscribes consumer 'consumer' to a given list of topics 'topicsToSubscribe', creates * consumer poller and starts polling. * Assumes that the consumer is not subscribed to any topics yet -* -* @param consumer consumer + * + * @param consumer consumer * @param topicsToSubscribe topics that this consumer will subscribe to * @return consumer poller for the given consumer */
[2/2] kafka git commit: KAFKA-2273; Sticky partition assignment strategy (KIP-54)
KAFKA-2273; Sticky partition assignment strategy (KIP-54) This PR implements a new partition assignment strategy called "sticky", and it's purpose is to balance partitions across consumers in a way that minimizes moving partitions around, or, in other words, preserves existing partition assignments as much as possible. This patch is co-authored with rajinisivaram and edoardocomar. Author: Vahid Hashemian Reviewers: Jason Gustafson Closes #1020 from vahidhashemian/KAFKA-2273 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/e1abf177 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/e1abf177 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/e1abf177 Branch: refs/heads/trunk Commit: e1abf17708918b82d3974ea028a4d74e3892fa0f Parents: 9815e18 Author: Vahid Hashemian Authored: Wed May 17 20:13:19 2017 -0700 Committer: Jason Gustafson Committed: Wed May 17 20:15:17 2017 -0700 -- .../kafka/clients/consumer/StickyAssignor.java | 933 +++ .../internals/AbstractPartitionAssignor.java| 8 +- .../consumer/internals/ConsumerProtocol.java| 1 - .../consumer/internals/PartitionAssignor.java | 2 +- .../org/apache/kafka/common/TopicPartition.java | 1 - .../clients/consumer/StickyAssignorTest.java| 689 ++ .../kafka/api/PlaintextConsumerTest.scala | 58 +- 7 files changed, 1685 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/e1abf177/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java new file mode 100644 index 000..58e5915 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java @@ -0,0 +1,933 @@ +/* + * 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.clients.consumer; + +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.types.ArrayOf; +import org.apache.kafka.common.protocol.types.Field; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.protocol.types.Type; +import org.apache.kafka.common.utils.CollectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The sticky assignor serves two purposes. First, it guarantees an assignment that is as balanced as possible, meaning either: + * - the numbers of topic partitions assigned to consumers differ by at most one; or + * - each consumer that has 2+ fewer topic partitions than some other consumer cannot get any of those topic partitions transferred to it. + * Second, it preserved as many existing assignment as possible when a reassignment occurs. This helps in saving some of the + * overhead processing when topic partitions move from one consumer to another. + * + * Starting fresh it would work by distributing the partitions over consumers as evenly as possible. Even though this may sound similar to + * how round robin assignor works, the second example below shows that it is not. + * During a reassignment it would perform the reassignment in such a way that in the new assignment + * 1. topic partitions are still distributed as evenly as possible, and + * 2. topic partitions stay with their previously assigned consumers as much as possible. + * Of course, the first
kafka git commit: KAFKA-5268; Fix bounce test transient failure by clearing partitions before writing Complete state to transaction log
Repository: kafka Updated Branches: refs/heads/trunk 45f226176 -> d662b09c9 KAFKA-5268; Fix bounce test transient failure by clearing partitions before writing Complete state to transaction log Author: Jason Gustafson Reviewers: Guozhang Wang Closes #3089 from hachikuji/KAFKA-5268 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/d662b09c Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/d662b09c Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/d662b09c Branch: refs/heads/trunk Commit: d662b09c9f32d7d4dcfc18522a4d2789b43d319c Parents: 45f2261 Author: Jason Gustafson Authored: Thu May 18 11:17:30 2017 -0700 Committer: Jason Gustafson Committed: Thu May 18 11:17:30 2017 -0700 -- .../coordinator/transaction/TransactionCoordinator.scala | 2 +- .../kafka/coordinator/transaction/TransactionMetadata.scala | 2 +- .../coordinator/transaction/TransactionCoordinatorTest.scala | 8 +++- 3 files changed, 9 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/d662b09c/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala -- diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index ebfbde5..8de6dbd 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -243,7 +243,7 @@ class TransactionCoordinator(brokerId: Int, Left(Errors.CONCURRENT_TRANSACTIONS) } else if (txnMetadata.state == PrepareCommit || txnMetadata.state == PrepareAbort) { Left(Errors.CONCURRENT_TRANSACTIONS) -} else if (partitions.subsetOf(txnMetadata.topicPartitions)) { +} else if (txnMetadata.state == Ongoing && partitions.subsetOf(txnMetadata.topicPartitions)) { // this is an optimization: if the partitions are already in the metadata reply OK immediately Left(Errors.NONE) } else { http://git-wip-us.apache.org/repos/asf/kafka/blob/d662b09c/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala -- diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala index d739b9a..6e29308 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala @@ -185,7 +185,7 @@ private[transaction] class TransactionMetadata(val producerId: Long, def prepareComplete(updateTimestamp: Long): TransactionMetadataTransition = { val newState = if (state == PrepareCommit) CompleteCommit else CompleteAbort -prepareTransitionTo(newState, producerEpoch, txnTimeoutMs, topicPartitions.toSet, txnStartTimestamp, updateTimestamp) +prepareTransitionTo(newState, producerEpoch, txnTimeoutMs, Set.empty[TopicPartition], txnStartTimestamp, updateTimestamp) } // visible for testing only http://git-wip-us.apache.org/repos/asf/kafka/blob/d662b09c/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala -- diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index 7271edd..04f76bd 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -672,7 +672,13 @@ class TransactionCoordinatorTest { .andReturn(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, prepareMetadata))) .once() -val newMetadata = prepareMetadata.copy().prepareComplete(now) +val newMetadata = TransactionMetadataTransition(producerId = pid, + producerEpoch = epoch, + txnTimeoutMs = txnTimeoutMs, + txnState = finalState, + topicPartitions = Set.empty[TopicPartition], + txnStartTimestamp = prepareMetadata.txnStartTimestamp, + txnLastUpdateTimestamp = now) EasyMock.expect(transactionMarkerChannelManager.addTxnMarkersToSend( EasyMock.eq(transactionalId), EasyMock.eq(coordinatorEpoch),
kafka git commit: KAFKA-5171; TC should not accept empty string transactional id
Repository: kafka Updated Branches: refs/heads/trunk d662b09c9 -> 34e379f10 KAFKA-5171; TC should not accept empty string transactional id This is a rebase version of [PR#2973](https://github.com/apache/kafka/pull/2973). guozhangwang , please review this updated PR. Author: umesh chaudhary Reviewers: Guozhang Wang , Jason Gustafson Closes #3086 from umesh9794/mylocal Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/34e379f1 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/34e379f1 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/34e379f1 Branch: refs/heads/trunk Commit: 34e379f10779bfb17fb399fde0357d17dc34ab62 Parents: d662b09 Author: umesh chaudhary Authored: Thu May 18 11:38:44 2017 -0700 Committer: Jason Gustafson Committed: Thu May 18 11:38:44 2017 -0700 -- .../coordinator/transaction/TransactionCoordinator.scala| 9 +++-- .../transaction/TransactionCoordinatorTest.scala| 6 +++--- 2 files changed, 10 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/34e379f1/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala -- diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 8de6dbd..491e16a 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -93,11 +93,16 @@ class TransactionCoordinator(brokerId: Int, def handleInitProducerId(transactionalId: String, transactionTimeoutMs: Int, responseCallback: InitProducerIdCallback): Unit = { -if (transactionalId == null || transactionalId.isEmpty) { - // if the transactional id is not specified, then always blindly accept the request + +if (transactionalId == null) { + // if the transactional id is null, then always blindly accept the request // and return a new producerId from the producerId manager val producerId = producerIdManager.generateProducerId() responseCallback(InitProducerIdResult(producerId, producerEpoch = 0, Errors.NONE)) +} else if (transactionalId.isEmpty) { + //If transactional id is empty then return error as invalid request. This is + // to make TransactionCoordinator's behavior consistent with producer client + responseCallback(initTransactionError(Errors.INVALID_REQUEST)) } else if (!txnManager.isCoordinatorFor(transactionalId)) { // check if it is the assigned coordinator for the transactional id responseCallback(initTransactionError(Errors.NOT_COORDINATOR)) http://git-wip-us.apache.org/repos/asf/kafka/blob/34e379f1/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala -- diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index 04f76bd..43ad7a7 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -86,14 +86,14 @@ class TransactionCoordinatorTest { } @Test - def shouldAcceptInitPidAndReturnNextPidWhenTransactionalIdIsEmpty(): Unit = { + def shouldReturnInvalidRequestWhenTransactionalIdIsEmpty(): Unit = { mockPidManager() EasyMock.replay(pidManager) coordinator.handleInitProducerId("", txnTimeoutMs, initProducerIdMockCallback) -assertEquals(InitProducerIdResult(0L, 0, Errors.NONE), result) +assertEquals(InitProducerIdResult(-1L, -1, Errors.INVALID_REQUEST), result) coordinator.handleInitProducerId("", txnTimeoutMs, initProducerIdMockCallback) -assertEquals(InitProducerIdResult(1L, 0, Errors.NONE), result) +assertEquals(InitProducerIdResult(-1L, -1, Errors.INVALID_REQUEST), result) } @Test
kafka git commit: KAFKA-5033; Set default retries for the idempotent producer to be infinite
Repository: kafka Updated Branches: refs/heads/trunk 34e379f10 -> c1fdf575d KAFKA-5033; Set default retries for the idempotent producer to be infinite Author: Apurva Mehta Reviewers: Jason Gustafson Closes #3091 from apurvam/KAFKA-5033-bump-retries-for-idempotent-producer Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/c1fdf575 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/c1fdf575 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/c1fdf575 Branch: refs/heads/trunk Commit: c1fdf575deed0398e1692be16cacf24308afd5d5 Parents: 34e379f Author: Apurva Mehta Authored: Thu May 18 13:06:56 2017 -0700 Committer: Jason Gustafson Committed: Thu May 18 13:07:06 2017 -0700 -- .../apache/kafka/clients/producer/KafkaProducer.java | 13 - 1 file changed, 8 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/c1fdf575/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index aeef92f..71fb077 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -374,8 +374,11 @@ public class KafkaProducer implements Producer { userConfiguredRetries = true; } if (idempotenceEnabled && !userConfiguredRetries) { -log.info("Overriding the default retries config to " + 3 + " since the idempotent producer is enabled."); -return 3; +// We recommend setting infinite retries when the idempotent producer is enabled, so it makes sense to make +// this the default. +log.info("Overriding the default retries config to the recommended value of {} since the idempotent " + +"producer is enabled.", Integer.MAX_VALUE); +return Integer.MAX_VALUE; } if (idempotenceEnabled && config.getInt(ProducerConfig.RETRIES_CONFIG) == 0) { throw new ConfigException("Must set " + ProducerConfig.RETRIES_CONFIG + " to non-zero when using the idempotent producer."); @@ -389,7 +392,7 @@ public class KafkaProducer implements Producer { userConfiguredInflights = true; } if (idempotenceEnabled && !userConfiguredInflights) { -log.info("Overriding the default " + ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION + " to 1 since idempontence is enabled."); +log.info("Overriding the default {} to 1 since idempontence is enabled.", ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION); return 1; } if (idempotenceEnabled && config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION) != 1) { @@ -407,13 +410,13 @@ public class KafkaProducer implements Producer { } if (idempotenceEnabled && !userConfiguredAcks) { -log.info("Overriding the default " + ProducerConfig.ACKS_CONFIG + " to all since idempotence is enabled"); +log.info("Overriding the default {} to all since idempotence is enabled.", ProducerConfig.ACKS_CONFIG); return -1; } if (idempotenceEnabled && acks != -1) { throw new ConfigException("Must set " + ProducerConfig.ACKS_CONFIG + " to all in order to use the idempotent " + -"producer. Otherwise we cannot guarantee idempotence"); +"producer. Otherwise we cannot guarantee idempotence."); } return acks; }
kafka git commit: HOTFIX: Close transactional producers in all new tests
Repository: kafka Updated Branches: refs/heads/trunk 96959bc56 -> 30736da65 HOTFIX: Close transactional producers in all new tests Author: Apurva Mehta Reviewers: Jason Gustafson Closes #3093 from apurvam/HOTFIX-close-leaked-producers-in-transactions-test Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/30736da6 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/30736da6 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/30736da6 Branch: refs/heads/trunk Commit: 30736da65d6e47eb68ab7563357a063caf753964 Parents: 96959bc Author: Apurva Mehta Authored: Thu May 18 15:21:15 2017 -0700 Committer: Jason Gustafson Committed: Thu May 18 15:21:15 2017 -0700 -- .../integration/kafka/api/AuthorizerIntegrationTest.scala| 1 + .../test/scala/integration/kafka/api/TransactionsTest.scala | 8 2 files changed, 5 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/30736da6/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala -- diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index a362577..0198d38 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -204,6 +204,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { producers.foreach(_.close()) consumers.foreach(_.wakeup()) consumers.foreach(_.close()) +transactionalProducer.close() removeAllAcls() super.tearDown() } http://git-wip-us.apache.org/repos/asf/kafka/blob/30736da6/core/src/test/scala/integration/kafka/api/TransactionsTest.scala -- diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index e8669e9..1832dc2 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -194,7 +194,6 @@ class TransactionsTest extends KafkaServerTestHarness { } catch { case e : ProducerFencedException => // good! - producer1.close() case e : Exception => fail("Got an unexpected exception from a fenced producer.", e) } @@ -207,6 +206,7 @@ class TransactionsTest extends KafkaServerTestHarness { } } finally { consumer.close() + producer1.close() producer2.close() } } @@ -237,7 +237,6 @@ class TransactionsTest extends KafkaServerTestHarness { } catch { case e : ProducerFencedException => // good! - producer1.close() case e : Exception => fail("Got an unexpected exception from a fenced producer.", e) } @@ -250,6 +249,7 @@ class TransactionsTest extends KafkaServerTestHarness { } } finally { consumer.close() + producer1.close() producer2.close() } } @@ -287,7 +287,6 @@ class TransactionsTest extends KafkaServerTestHarness { producer1.close() case e : ExecutionException => assertTrue(e.getCause.isInstanceOf[ProducerFencedException]) - producer1.close() case e : Exception => fail("Got an unexpected exception from a fenced producer.", e) } @@ -300,6 +299,7 @@ class TransactionsTest extends KafkaServerTestHarness { } } finally { consumer.close() + producer1.close() producer2.close() } } @@ -339,7 +339,6 @@ class TransactionsTest extends KafkaServerTestHarness { producer1.close() case e : ExecutionException => assertTrue(e.getCause.isInstanceOf[ProducerFencedException]) - producer1.close() case e : Exception => fail("Got an unexpected exception from a fenced producer.", e) } @@ -352,6 +351,7 @@ class TransactionsTest extends KafkaServerTestHarness { } } finally { consumer.close() + producer1.close() producer2.close() } }
kafka git commit: KAFKA-4714; TimestampConverter transformation (KIP-66)
Repository: kafka Updated Branches: refs/heads/trunk abe699176 -> 61bab2d87 KAFKA-4714; TimestampConverter transformation (KIP-66) Author: Ewen Cheslack-Postava Reviewers: Konstantine Karantasis , Jason Gustafson Closes #3065 from ewencp/kafka-3209-timestamp-converter Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/61bab2d8 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/61bab2d8 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/61bab2d8 Branch: refs/heads/trunk Commit: 61bab2d875ab5e03d0df4f62217346549a4c64c3 Parents: abe6991 Author: Ewen Cheslack-Postava Authored: Fri May 19 11:26:59 2017 -0700 Committer: Jason Gustafson Committed: Fri May 19 11:26:59 2017 -0700 -- .../kafka/connect/tools/TransformationDoc.java | 4 +- .../connect/transforms/TimestampConverter.java | 452 +++ .../connect/transforms/util/Requirements.java | 2 +- .../transforms/TimestampConverterTest.java | 370 +++ 4 files changed, 826 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/61bab2d8/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java -- diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java index 1a8f0a8..b76e7d4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -26,6 +26,7 @@ import org.apache.kafka.connect.transforms.MaskField; import org.apache.kafka.connect.transforms.RegexRouter; import org.apache.kafka.connect.transforms.ReplaceField; import org.apache.kafka.connect.transforms.SetSchemaMetadata; +import org.apache.kafka.connect.transforms.TimestampConverter; import org.apache.kafka.connect.transforms.TimestampRouter; import org.apache.kafka.connect.transforms.ValueToKey; @@ -58,7 +59,8 @@ public class TransformationDoc { new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF), new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF), new DocInfo(Flatten.class.getName(), Flatten.OVERVIEW_DOC, Flatten.CONFIG_DEF), -new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF) +new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF), +new DocInfo(TimestampConverter.class.getName(), TimestampConverter.OVERVIEW_DOC, TimestampConverter.CONFIG_DEF) ); private static void printTransformationHtml(PrintStream out, DocInfo docInfo) { http://git-wip-us.apache.org/repos/asf/kafka/blob/61bab2d8/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java -- diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java new file mode 100644 index 000..ce7d002 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java @@ -0,0 +1,452 @@ +/* + * 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.connect.transforms; + +import org.apache.kafka.common.cache.Cache; +import org.apache.kafka.common.cache.LRUCache; +import org.apache.kafka.common.cache.SynchronizedCache; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Schem
kafka git commit: KAFKA-4714; TimestampConverter transformation (KIP-66)
Repository: kafka Updated Branches: refs/heads/0.11.0 b31df613f -> 64e12d3c6 KAFKA-4714; TimestampConverter transformation (KIP-66) Author: Ewen Cheslack-Postava Reviewers: Konstantine Karantasis , Jason Gustafson Closes #3065 from ewencp/kafka-3209-timestamp-converter (cherry picked from commit 61bab2d875ab5e03d0df4f62217346549a4c64c3) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/64e12d3c Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/64e12d3c Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/64e12d3c Branch: refs/heads/0.11.0 Commit: 64e12d3c641baeb6a4a624b971ba33721d656054 Parents: b31df61 Author: Ewen Cheslack-Postava Authored: Fri May 19 11:26:59 2017 -0700 Committer: Jason Gustafson Committed: Fri May 19 11:28:20 2017 -0700 -- .../kafka/connect/tools/TransformationDoc.java | 4 +- .../connect/transforms/TimestampConverter.java | 452 +++ .../connect/transforms/util/Requirements.java | 2 +- .../transforms/TimestampConverterTest.java | 370 +++ 4 files changed, 826 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/64e12d3c/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java -- diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java index 1a8f0a8..b76e7d4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -26,6 +26,7 @@ import org.apache.kafka.connect.transforms.MaskField; import org.apache.kafka.connect.transforms.RegexRouter; import org.apache.kafka.connect.transforms.ReplaceField; import org.apache.kafka.connect.transforms.SetSchemaMetadata; +import org.apache.kafka.connect.transforms.TimestampConverter; import org.apache.kafka.connect.transforms.TimestampRouter; import org.apache.kafka.connect.transforms.ValueToKey; @@ -58,7 +59,8 @@ public class TransformationDoc { new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF), new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF), new DocInfo(Flatten.class.getName(), Flatten.OVERVIEW_DOC, Flatten.CONFIG_DEF), -new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF) +new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF), +new DocInfo(TimestampConverter.class.getName(), TimestampConverter.OVERVIEW_DOC, TimestampConverter.CONFIG_DEF) ); private static void printTransformationHtml(PrintStream out, DocInfo docInfo) { http://git-wip-us.apache.org/repos/asf/kafka/blob/64e12d3c/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java -- diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java new file mode 100644 index 000..ce7d002 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampConverter.java @@ -0,0 +1,452 @@ +/* + * 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.connect.transforms; + +import org.apache.kafka.common.cache.Cache; +import org.apache.kafka.common.cache.LRUCache; +import org.apache.kafka.common.cache.SynchronizedCache; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.conne
kafka git commit: KAFKA-5269; Retry on unknown topic/partition error in transactional requests
Repository: kafka Updated Branches: refs/heads/trunk 338857e1c -> 8b04d8ba0 KAFKA-5269; Retry on unknown topic/partition error in transactional requests We should retry AddPartitionsToTxnRequest and TxnOffsetCommitRequest when receiving an UNKNOWN_TOPIC_OR_PARTITION error. As described in the JIRA: It turns out that the `UNKNOWN_TOPIC_OR_PARTITION` is returned from the request handler in KafkaAPis for the AddPartitionsToTxn and the TxnOffsetCommitRequest when the broker's metadata doesn't contain one or more partitions in the request. This can happen for instance when the broker is bounced and has not received the cluster metadata yet. We should retry in these cases, as this is the model followed by the consumer when committing offsets, and by the producer with a ProduceRequest. Author: Apurva Mehta Reviewers: Guozhang Wang , Jason Gustafson Closes #3094 from apurvam/KAFKA-5269-handle-unknown-topic-partition-in-transaction-manager Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8b04d8ba Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8b04d8ba Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8b04d8ba Branch: refs/heads/trunk Commit: 8b04d8ba071fbcafc72a17d4cbfe6f00613e59b3 Parents: 338857e Author: Apurva Mehta Authored: Fri May 19 18:51:37 2017 -0700 Committer: Jason Gustafson Committed: Fri May 19 18:51:42 2017 -0700 -- .../kafka/clients/producer/KafkaProducer.java | 3 +- .../producer/internals/TransactionManager.java | 10 +- .../apache/kafka/common/protocol/Errors.java| 3 +- .../internals/TransactionManagerTest.java | 154 +++ .../kafka/api/TransactionsBounceTest.scala | 4 +- 5 files changed, 140 insertions(+), 34 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/8b04d8ba/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 1ba13b2..ac0169a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -51,6 +51,7 @@ import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.ChannelBuilder; import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.RecordBatch; @@ -696,7 +697,7 @@ public class KafkaProducer implements Producer { throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions when transactions are enabled."); if (transactionManager.isFenced()) -throw new ProducerFencedException("The current producer has been fenced off by a another producer using the same transactional id."); +throw Errors.INVALID_PRODUCER_EPOCH.exception(); if (transactionManager.isInErrorState()) { String errorMessage = http://git-wip-us.apache.org/repos/asf/kafka/blob/8b04d8ba/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 55c1782..c6787f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -179,7 +179,7 @@ public class TransactionManager { public synchronized TransactionalRequestResult beginAbortingTransaction() { ensureTransactional(); if (isFenced()) -throw new ProducerFencedException("There is a newer producer using the same transactional.id."); +throw Errors.INVALID_PRODUCER_EPOCH.exception(); transitionTo(State.ABORTING_TRANSACTION); return beginCompletingTransaction(false); } @@ -424,7 +424,7 @@ public class TransactionManager { private void maybeFailWithError() { if (isFenced()) -throw new ProducerFencedException("There is a newer producer instance using the same transactional id."); +throw Errors.INVALID_PRODUCER_EPOCH.exception(); if (isInErrorState()) { String e
kafka git commit: KAFKA-5269; Retry on unknown topic/partition error in transactional requests
Repository: kafka Updated Branches: refs/heads/0.11.0 037f63882 -> 55330cc29 KAFKA-5269; Retry on unknown topic/partition error in transactional requests We should retry AddPartitionsToTxnRequest and TxnOffsetCommitRequest when receiving an UNKNOWN_TOPIC_OR_PARTITION error. As described in the JIRA: It turns out that the `UNKNOWN_TOPIC_OR_PARTITION` is returned from the request handler in KafkaAPis for the AddPartitionsToTxn and the TxnOffsetCommitRequest when the broker's metadata doesn't contain one or more partitions in the request. This can happen for instance when the broker is bounced and has not received the cluster metadata yet. We should retry in these cases, as this is the model followed by the consumer when committing offsets, and by the producer with a ProduceRequest. Author: Apurva Mehta Reviewers: Guozhang Wang , Jason Gustafson Closes #3094 from apurvam/KAFKA-5269-handle-unknown-topic-partition-in-transaction-manager Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/55330cc2 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/55330cc2 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/55330cc2 Branch: refs/heads/0.11.0 Commit: 55330cc2931cc05cb1172cc207ada60154a6250a Parents: 037f638 Author: Apurva Mehta Authored: Fri May 19 18:51:37 2017 -0700 Committer: Jason Gustafson Committed: Sat May 20 10:03:39 2017 -0700 -- .../kafka/clients/producer/KafkaProducer.java | 3 +- .../producer/internals/TransactionManager.java | 10 +- .../apache/kafka/common/protocol/Errors.java| 3 +- .../internals/TransactionManagerTest.java | 154 +++ .../kafka/api/TransactionsBounceTest.scala | 4 +- 5 files changed, 140 insertions(+), 34 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/55330cc2/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 1ba13b2..ac0169a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -51,6 +51,7 @@ import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.ChannelBuilder; import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.RecordBatch; @@ -696,7 +697,7 @@ public class KafkaProducer implements Producer { throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions when transactions are enabled."); if (transactionManager.isFenced()) -throw new ProducerFencedException("The current producer has been fenced off by a another producer using the same transactional id."); +throw Errors.INVALID_PRODUCER_EPOCH.exception(); if (transactionManager.isInErrorState()) { String errorMessage = http://git-wip-us.apache.org/repos/asf/kafka/blob/55330cc2/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 55c1782..c6787f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -179,7 +179,7 @@ public class TransactionManager { public synchronized TransactionalRequestResult beginAbortingTransaction() { ensureTransactional(); if (isFenced()) -throw new ProducerFencedException("There is a newer producer using the same transactional.id."); +throw Errors.INVALID_PRODUCER_EPOCH.exception(); transitionTo(State.ABORTING_TRANSACTION); return beginCompletingTransaction(false); } @@ -424,7 +424,7 @@ public class TransactionManager { private void maybeFailWithError() { if (isFenced()) -throw new ProducerFencedException("There is a newer producer instance using the same transactional id."); +throw Errors.INVALID_PRODUCER_EPOCH.exception(); if (isInErrorState()) { String
kafka git commit: KAFKA-5277; Sticky Assignor should not cache previous assignment (KIP-54 follow-up)
Repository: kafka Updated Branches: refs/heads/trunk c2ced5fb5 -> 65861a712 KAFKA-5277; Sticky Assignor should not cache previous assignment (KIP-54 follow-up) ... plus some minor cleanup Author: Vahid Hashemian Reviewers: Jason Gustafson Closes #3092 from vahidhashemian/KAFKA-5277 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/65861a71 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/65861a71 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/65861a71 Branch: refs/heads/trunk Commit: 65861a712ddf67eb071a00218730926fdeef7084 Parents: c2ced5f Author: Vahid Hashemian Authored: Mon May 22 10:59:38 2017 -0700 Committer: Jason Gustafson Committed: Mon May 22 10:59:38 2017 -0700 -- .../kafka/clients/consumer/RangeAssignor.java | 8 +- .../clients/consumer/RoundRobinAssignor.java| 10 +- .../kafka/clients/consumer/StickyAssignor.java | 153 +++-- .../internals/AbstractPartitionAssignor.java| 18 +- .../clients/consumer/RangeAssignorTest.java | 90 .../consumer/RoundRobinAssignorTest.java| 88 .../clients/consumer/StickyAssignorTest.java| 216 +++ .../internals/MockPartitionAssignor.java| 2 +- 8 files changed, 307 insertions(+), 278 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/65861a71/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java index ec6c62f..d8d72ee 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java @@ -45,11 +45,11 @@ public class RangeAssignor extends AbstractPartitionAssignor { return "range"; } -private Map> consumersPerTopic(Map> consumerMetadata) { +private Map> consumersPerTopic(Map consumerMetadata) { Map> res = new HashMap<>(); -for (Map.Entry> subscriptionEntry : consumerMetadata.entrySet()) { +for (Map.Entry subscriptionEntry : consumerMetadata.entrySet()) { String consumerId = subscriptionEntry.getKey(); -for (String topic : subscriptionEntry.getValue()) +for (String topic : subscriptionEntry.getValue().topics()) put(res, topic, consumerId); } return res; @@ -57,7 +57,7 @@ public class RangeAssignor extends AbstractPartitionAssignor { @Override public Map> assign(Map partitionsPerTopic, -Map> subscriptions) { +Map subscriptions) { Map> consumersPerTopic = consumersPerTopic(subscriptions); Map> assignment = new HashMap<>(); for (String memberId : subscriptions.keySet()) http://git-wip-us.apache.org/repos/asf/kafka/blob/65861a71/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java index 8e38b84..7e8d6f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java @@ -57,7 +57,7 @@ public class RoundRobinAssignor extends AbstractPartitionAssignor { @Override public Map> assign(Map partitionsPerTopic, -Map> subscriptions) { +Map subscriptions) { Map> assignment = new HashMap<>(); for (String memberId : subscriptions.keySet()) assignment.put(memberId, new ArrayList()); @@ -65,7 +65,7 @@ public class RoundRobinAssignor extends AbstractPartitionAssignor { CircularIterator assigner = new CircularIterator<>(Utils.sorted(subscriptions.keySet())); for (TopicPartition partition : allPartitionsSorted(partitionsPerTopic, subscriptions)) { final String topic = partition.topic(); -while (!subscriptions.get(assigner.peek()).contains(topic)) +while (!subscriptions.get(assigner.peek()).topics().contains(topic)) assigner.next(); assignment.get(assigner.next()).add(partition); } @@ -74,10 +74,10 @@ public class RoundRobinAssignor extends AbstractPartitionAssignor { public List allPartitionsSo
kafka git commit: KAFKA-5277; Sticky Assignor should not cache previous assignment (KIP-54 follow-up)
Repository: kafka Updated Branches: refs/heads/0.11.0 4c29b1544 -> 67f691bbd KAFKA-5277; Sticky Assignor should not cache previous assignment (KIP-54 follow-up) ... plus some minor cleanup Author: Vahid Hashemian Reviewers: Jason Gustafson Closes #3092 from vahidhashemian/KAFKA-5277 (cherry picked from commit 65861a712ddf67eb071a00218730926fdeef7084) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/67f691bb Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/67f691bb Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/67f691bb Branch: refs/heads/0.11.0 Commit: 67f691bbd93b06efe58f2ba28f1b85efedf99651 Parents: 4c29b15 Author: Vahid Hashemian Authored: Mon May 22 10:59:38 2017 -0700 Committer: Jason Gustafson Committed: Mon May 22 11:01:15 2017 -0700 -- .../kafka/clients/consumer/RangeAssignor.java | 8 +- .../clients/consumer/RoundRobinAssignor.java| 10 +- .../kafka/clients/consumer/StickyAssignor.java | 153 +++-- .../internals/AbstractPartitionAssignor.java| 18 +- .../clients/consumer/RangeAssignorTest.java | 90 .../consumer/RoundRobinAssignorTest.java| 88 .../clients/consumer/StickyAssignorTest.java| 216 +++ .../internals/MockPartitionAssignor.java| 2 +- 8 files changed, 307 insertions(+), 278 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/67f691bb/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java index ec6c62f..d8d72ee 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java @@ -45,11 +45,11 @@ public class RangeAssignor extends AbstractPartitionAssignor { return "range"; } -private Map> consumersPerTopic(Map> consumerMetadata) { +private Map> consumersPerTopic(Map consumerMetadata) { Map> res = new HashMap<>(); -for (Map.Entry> subscriptionEntry : consumerMetadata.entrySet()) { +for (Map.Entry subscriptionEntry : consumerMetadata.entrySet()) { String consumerId = subscriptionEntry.getKey(); -for (String topic : subscriptionEntry.getValue()) +for (String topic : subscriptionEntry.getValue().topics()) put(res, topic, consumerId); } return res; @@ -57,7 +57,7 @@ public class RangeAssignor extends AbstractPartitionAssignor { @Override public Map> assign(Map partitionsPerTopic, -Map> subscriptions) { +Map subscriptions) { Map> consumersPerTopic = consumersPerTopic(subscriptions); Map> assignment = new HashMap<>(); for (String memberId : subscriptions.keySet()) http://git-wip-us.apache.org/repos/asf/kafka/blob/67f691bb/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java index 8e38b84..7e8d6f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java @@ -57,7 +57,7 @@ public class RoundRobinAssignor extends AbstractPartitionAssignor { @Override public Map> assign(Map partitionsPerTopic, -Map> subscriptions) { +Map subscriptions) { Map> assignment = new HashMap<>(); for (String memberId : subscriptions.keySet()) assignment.put(memberId, new ArrayList()); @@ -65,7 +65,7 @@ public class RoundRobinAssignor extends AbstractPartitionAssignor { CircularIterator assigner = new CircularIterator<>(Utils.sorted(subscriptions.keySet())); for (TopicPartition partition : allPartitionsSorted(partitionsPerTopic, subscriptions)) { final String topic = partition.topic(); -while (!subscriptions.get(assigner.peek()).contains(topic)) +while (!subscriptions.get(assigner.peek()).topics().contains(topic)) assigner.next(); assignment.get(assigner.next()).add(partition); } @@ -74,10 +74,10 @@ p
kafka git commit: KAFKA-5186; Avoid expensive log scan to build producer state when upgrading
Repository: kafka Updated Branches: refs/heads/trunk 65861a712 -> fcdbb7195 KAFKA-5186; Avoid expensive log scan to build producer state when upgrading Author: Jason Gustafson Reviewers: Jun Rao Closes #3113 from hachikuji/KAFKA-5186 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/fcdbb719 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/fcdbb719 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/fcdbb719 Branch: refs/heads/trunk Commit: fcdbb71953fc4c92559a9c7adb4cb8ad4a74acd6 Parents: 65861a7 Author: Jason Gustafson Authored: Mon May 22 15:41:26 2017 -0700 Committer: Jason Gustafson Committed: Mon May 22 15:41:26 2017 -0700 -- core/src/main/scala/kafka/log/Log.scala | 50 +--- .../src/test/scala/unit/kafka/log/LogTest.scala | 27 +++ 2 files changed, 60 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/fcdbb719/core/src/main/scala/kafka/log/Log.scala -- diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 7a47657..55eb46a 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -421,25 +421,41 @@ class Log(@volatile var dir: File, private def loadProducerState(lastOffset: Long): Unit = lock synchronized { info(s"Loading producer state from offset $lastOffset for partition $topicPartition") -val currentTimeMs = time.milliseconds -producerStateManager.truncateAndReload(logStartOffset, lastOffset, currentTimeMs) - -// only do the potentially expensive reloading of the last snapshot offset is lower than the -// log end offset (which would be the case on first startup) and there are active producers. -// if there are no active producers, then truncating shouldn't change that fact (although it -// could cause a producerId to expire earlier than expected), so we can skip the loading. -// This is an optimization for users which are not yet using idempotent/transactional features yet. -if (lastOffset > producerStateManager.mapEndOffset || !producerStateManager.isEmpty) { - logSegments(producerStateManager.mapEndOffset, lastOffset).foreach { segment => -val startOffset = math.max(segment.baseOffset, producerStateManager.mapEndOffset) -val fetchDataInfo = segment.read(startOffset, Some(lastOffset), Int.MaxValue) -if (fetchDataInfo != null) - loadProducersFromLog(producerStateManager, fetchDataInfo.records) + +if (producerStateManager.latestSnapshotOffset.isEmpty) { + // if there are no snapshots to load producer state from, we assume that the brokers are + // being upgraded, which means there would be no previous idempotent/transactional producers + // to load state for. To avoid an expensive scan through all of the segments, we take + // empty snapshots from the start of the last two segments and the last offset. The purpose + // of taking the segment snapshots is to avoid the full scan in the case that the log needs + // truncation. + val nextLatestSegmentBaseOffset = Option(segments.lowerEntry(activeSegment.baseOffset)).map(_.getValue.baseOffset) + val offsetsToSnapshot = Seq(nextLatestSegmentBaseOffset, Some(activeSegment.baseOffset), Some(lastOffset)) + offsetsToSnapshot.flatten.foreach { offset => +producerStateManager.updateMapEndOffset(offset) +producerStateManager.takeSnapshot() + } +} else { + val currentTimeMs = time.milliseconds + producerStateManager.truncateAndReload(logStartOffset, lastOffset, currentTimeMs) + + // only do the potentially expensive reloading of the last snapshot offset is lower than the + // log end offset (which would be the case on first startup) and there are active producers. + // if there are no active producers, then truncating shouldn't change that fact (although it + // could cause a producerId to expire earlier than expected), so we can skip the loading. + // This is an optimization for users which are not yet using idempotent/transactional features yet. + if (lastOffset > producerStateManager.mapEndOffset || !producerStateManager.isEmpty) { +logSegments(producerStateManager.mapEndOffset, lastOffset).foreach { segment => + val startOffset = math.max(segment.baseOffset, producerStateManager.mapEndOffset) + val fetchDataInfo = segment.read(startOffset, Some(lastOffset), Int.MaxValue) + if (fetchDataInfo != null) +loadProducersFromLog(producerStateManager, fetchDataInfo.records) +} } -} -producerStateManager.updateMapEndOffset(lastOf
kafka git commit: KAFKA-5186; Avoid expensive log scan to build producer state when upgrading
Repository: kafka Updated Branches: refs/heads/0.11.0 67f691bbd -> a9c9dbf58 KAFKA-5186; Avoid expensive log scan to build producer state when upgrading Author: Jason Gustafson Reviewers: Jun Rao Closes #3113 from hachikuji/KAFKA-5186 (cherry picked from commit fcdbb71953fc4c92559a9c7adb4cb8ad4a74acd6) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/a9c9dbf5 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/a9c9dbf5 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/a9c9dbf5 Branch: refs/heads/0.11.0 Commit: a9c9dbf58f0e9ccf873e100b62f0f811ad5e4659 Parents: 67f691b Author: Jason Gustafson Authored: Mon May 22 15:41:26 2017 -0700 Committer: Jason Gustafson Committed: Mon May 22 15:41:39 2017 -0700 -- core/src/main/scala/kafka/log/Log.scala | 50 +--- .../src/test/scala/unit/kafka/log/LogTest.scala | 27 +++ 2 files changed, 60 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/a9c9dbf5/core/src/main/scala/kafka/log/Log.scala -- diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 7a47657..55eb46a 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -421,25 +421,41 @@ class Log(@volatile var dir: File, private def loadProducerState(lastOffset: Long): Unit = lock synchronized { info(s"Loading producer state from offset $lastOffset for partition $topicPartition") -val currentTimeMs = time.milliseconds -producerStateManager.truncateAndReload(logStartOffset, lastOffset, currentTimeMs) - -// only do the potentially expensive reloading of the last snapshot offset is lower than the -// log end offset (which would be the case on first startup) and there are active producers. -// if there are no active producers, then truncating shouldn't change that fact (although it -// could cause a producerId to expire earlier than expected), so we can skip the loading. -// This is an optimization for users which are not yet using idempotent/transactional features yet. -if (lastOffset > producerStateManager.mapEndOffset || !producerStateManager.isEmpty) { - logSegments(producerStateManager.mapEndOffset, lastOffset).foreach { segment => -val startOffset = math.max(segment.baseOffset, producerStateManager.mapEndOffset) -val fetchDataInfo = segment.read(startOffset, Some(lastOffset), Int.MaxValue) -if (fetchDataInfo != null) - loadProducersFromLog(producerStateManager, fetchDataInfo.records) + +if (producerStateManager.latestSnapshotOffset.isEmpty) { + // if there are no snapshots to load producer state from, we assume that the brokers are + // being upgraded, which means there would be no previous idempotent/transactional producers + // to load state for. To avoid an expensive scan through all of the segments, we take + // empty snapshots from the start of the last two segments and the last offset. The purpose + // of taking the segment snapshots is to avoid the full scan in the case that the log needs + // truncation. + val nextLatestSegmentBaseOffset = Option(segments.lowerEntry(activeSegment.baseOffset)).map(_.getValue.baseOffset) + val offsetsToSnapshot = Seq(nextLatestSegmentBaseOffset, Some(activeSegment.baseOffset), Some(lastOffset)) + offsetsToSnapshot.flatten.foreach { offset => +producerStateManager.updateMapEndOffset(offset) +producerStateManager.takeSnapshot() + } +} else { + val currentTimeMs = time.milliseconds + producerStateManager.truncateAndReload(logStartOffset, lastOffset, currentTimeMs) + + // only do the potentially expensive reloading of the last snapshot offset is lower than the + // log end offset (which would be the case on first startup) and there are active producers. + // if there are no active producers, then truncating shouldn't change that fact (although it + // could cause a producerId to expire earlier than expected), so we can skip the loading. + // This is an optimization for users which are not yet using idempotent/transactional features yet. + if (lastOffset > producerStateManager.mapEndOffset || !producerStateManager.isEmpty) { +logSegments(producerStateManager.mapEndOffset, lastOffset).foreach { segment => + val startOffset = math.max(segment.baseOffset, producerStateManager.mapEndOffset) + val fetchDataInfo = segment.read(startOffset, Some(lastOffset), Int.MaxValue) + if (fetchDataInfo != null) +loadProducersFromLog(producerStateManager,
kafka git commit: MINOR: Broker should disallow downconversion of transactional/idempotent records
Repository: kafka Updated Branches: refs/heads/trunk cca9ad424 -> e3e2f1d22 MINOR: Broker should disallow downconversion of transactional/idempotent records Author: Jason Gustafson Reviewers: Guozhang Wang Closes #3118 from hachikuji/disallow-transactional-idempotent-downconversion Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/e3e2f1d2 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/e3e2f1d2 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/e3e2f1d2 Branch: refs/heads/trunk Commit: e3e2f1d22d17a20f67218c600e7e1647e1ca Parents: cca9ad4 Author: Jason Gustafson Authored: Mon May 22 20:00:07 2017 -0700 Committer: Jason Gustafson Committed: Mon May 22 20:00:07 2017 -0700 -- .../kafka/common/record/MemoryRecords.java | 6 +++ .../kafka/common/requests/ProduceResponse.java | 1 + .../src/main/scala/kafka/log/LogValidator.scala | 40 ++ .../scala/unit/kafka/log/LogValidatorTest.scala | 44 +++- 4 files changed, 72 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/e3e2f1d2/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index cec309e..7391e7e 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -443,6 +443,12 @@ public class MemoryRecords extends AbstractRecords { RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, partitionLeaderEpoch, false, records); } +public static MemoryRecords withIdempotentRecords(CompressionType compressionType, long producerId, + short producerEpoch, int baseSequence, SimpleRecord... records) { +return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compressionType, TimestampType.CREATE_TIME, producerId, producerEpoch, +baseSequence, RecordBatch.NO_PARTITION_LEADER_EPOCH, false, records); +} + public static MemoryRecords withIdempotentRecords(byte magic, long initialOffset, CompressionType compressionType, long producerId, short producerEpoch, int baseSequence, int partitionLeaderEpoch, SimpleRecord... records) { http://git-wip-us.apache.org/repos/asf/kafka/blob/e3e2f1d2/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 06c1f6e..42ae434 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -60,6 +60,7 @@ public class ProduceResponse extends AbstractResponse { * NOT_ENOUGH_REPLICAS_AFTER_APPEND (20) * INVALID_REQUIRED_ACKS (21) * TOPIC_AUTHORIZATION_FAILED (29) + * UNSUPPORTED_FOR_MESSAGE_FORMAT (43) */ private static final String BASE_OFFSET_KEY_NAME = "base_offset"; http://git-wip-us.apache.org/repos/asf/kafka/blob/e3e2f1d2/core/src/main/scala/kafka/log/LogValidator.scala -- diff --git a/core/src/main/scala/kafka/log/LogValidator.scala b/core/src/main/scala/kafka/log/LogValidator.scala index 33257fd..ee5cb58 100644 --- a/core/src/main/scala/kafka/log/LogValidator.scala +++ b/core/src/main/scala/kafka/log/LogValidator.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import kafka.common.LongRef import kafka.message.{CompressionCodec, NoCompressionCodec} import kafka.utils.Logging -import org.apache.kafka.common.errors.InvalidTimestampException +import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedForMessageFormatException} import org.apache.kafka.common.record._ import scala.collection.mutable @@ -62,14 +62,14 @@ private[kafka] object LogValidator extends Logging { else // Do in-place validation, offset assignment and maybe set timestamp assignOffsetsNonCompressed(records, offsetCounter, now, compactedTopic, timestampType, timestampDiffMaxMs, - partitionLeaderEpoch, isFromClient) + partitionLeaderEpoch, isFromClient, magic) } else { validateMessagesAndAssignOffsetsCompressed(records, offsetCounter, now,
kafka git commit: MINOR: Broker should disallow downconversion of transactional/idempotent records
Repository: kafka Updated Branches: refs/heads/0.11.0 9ae4deaa5 -> 33216846f MINOR: Broker should disallow downconversion of transactional/idempotent records Author: Jason Gustafson Reviewers: Guozhang Wang Closes #3118 from hachikuji/disallow-transactional-idempotent-downconversion (cherry picked from commit e3e2f1d22d17a20f67218c600e7e1647e1ca) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/33216846 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/33216846 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/33216846 Branch: refs/heads/0.11.0 Commit: 33216846fe070dfb5dd09eab9f53490bae92c357 Parents: 9ae4dea Author: Jason Gustafson Authored: Mon May 22 20:00:07 2017 -0700 Committer: Jason Gustafson Committed: Mon May 22 20:01:44 2017 -0700 -- .../kafka/common/record/MemoryRecords.java | 6 +++ .../kafka/common/requests/ProduceResponse.java | 1 + .../src/main/scala/kafka/log/LogValidator.scala | 40 ++ .../scala/unit/kafka/log/LogValidatorTest.scala | 44 +++- 4 files changed, 72 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/33216846/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index cec309e..7391e7e 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -443,6 +443,12 @@ public class MemoryRecords extends AbstractRecords { RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, partitionLeaderEpoch, false, records); } +public static MemoryRecords withIdempotentRecords(CompressionType compressionType, long producerId, + short producerEpoch, int baseSequence, SimpleRecord... records) { +return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compressionType, TimestampType.CREATE_TIME, producerId, producerEpoch, +baseSequence, RecordBatch.NO_PARTITION_LEADER_EPOCH, false, records); +} + public static MemoryRecords withIdempotentRecords(byte magic, long initialOffset, CompressionType compressionType, long producerId, short producerEpoch, int baseSequence, int partitionLeaderEpoch, SimpleRecord... records) { http://git-wip-us.apache.org/repos/asf/kafka/blob/33216846/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 06c1f6e..42ae434 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -60,6 +60,7 @@ public class ProduceResponse extends AbstractResponse { * NOT_ENOUGH_REPLICAS_AFTER_APPEND (20) * INVALID_REQUIRED_ACKS (21) * TOPIC_AUTHORIZATION_FAILED (29) + * UNSUPPORTED_FOR_MESSAGE_FORMAT (43) */ private static final String BASE_OFFSET_KEY_NAME = "base_offset"; http://git-wip-us.apache.org/repos/asf/kafka/blob/33216846/core/src/main/scala/kafka/log/LogValidator.scala -- diff --git a/core/src/main/scala/kafka/log/LogValidator.scala b/core/src/main/scala/kafka/log/LogValidator.scala index 33257fd..ee5cb58 100644 --- a/core/src/main/scala/kafka/log/LogValidator.scala +++ b/core/src/main/scala/kafka/log/LogValidator.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import kafka.common.LongRef import kafka.message.{CompressionCodec, NoCompressionCodec} import kafka.utils.Logging -import org.apache.kafka.common.errors.InvalidTimestampException +import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedForMessageFormatException} import org.apache.kafka.common.record._ import scala.collection.mutable @@ -62,14 +62,14 @@ private[kafka] object LogValidator extends Logging { else // Do in-place validation, offset assignment and maybe set timestamp assignOffsetsNonCompressed(records, offsetCounter, now, compactedTopic, timestampType, timestampDiffMaxMs, - partitionLeaderEpoch, isFromClient) + partitionLeaderEpoch, isFromClie
kafka git commit: KAFKA-5247; Materialize committed offsets in offset order
Repository: kafka Updated Branches: refs/heads/trunk e3e2f1d22 -> d1853f791 KAFKA-5247; Materialize committed offsets in offset order With this patch, offset commits are always materialized according to the order of the commit records in the offsets topic. Before this patch, transactional offset commits were materialized in transaction order. However, the log cleaner will always preserve the record with the greatest offset. This meant that if there was a mix of offset commits from a consumer and a transactional producer, then it we would switch from transactional order to offset order after cleaning, resulting in an inconsistent state. Author: Apurva Mehta Reviewers: Guozhang Wang , Jason Gustafson Closes #3108 from apurvam/KAFKA-5247-materialize-committed-offsets-in-offset-order Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/d1853f79 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/d1853f79 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/d1853f79 Branch: refs/heads/trunk Commit: d1853f791147d66303c2700a8e58af762518188c Parents: e3e2f1d Author: Apurva Mehta Authored: Tue May 23 09:45:31 2017 -0700 Committer: Jason Gustafson Committed: Tue May 23 09:45:31 2017 -0700 -- .../kafka/coordinator/group/GroupMetadata.scala | 96 .../group/GroupMetadataManager.scala| 50 + .../group/GroupMetadataManagerTest.scala| 102 - .../coordinator/group/GroupMetadataTest.scala | 109 ++- 4 files changed, 310 insertions(+), 47 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/d1853f79/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala -- diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala index 2f76d63..302fcb5 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala @@ -127,6 +127,16 @@ case class GroupSummary(state: String, members: List[MemberSummary]) /** + * We cache offset commits along with their commit record offset. This enables us to ensure that the latest offset + * commit is always materialized when we have a mix of transactional and regular offset commits. Without preserving + * information of the commit record offset, compaction of the offsets topic it self may result in the wrong offset commit + * being materialized. + */ +case class CommitRecordMetadataAndOffset(appendedBatchOffset: Option[Long], offsetAndMetadata: OffsetAndMetadata) { + def olderThan(that: CommitRecordMetadataAndOffset) : Boolean = appendedBatchOffset.get < that.appendedBatchOffset.get +} + +/** * Group contains the following metadata: * * Membership metadata: @@ -143,12 +153,17 @@ case class GroupSummary(state: String, private[group] class GroupMetadata(val groupId: String, initialState: GroupState = Empty) extends Logging { private var state: GroupState = initialState + private val members = new mutable.HashMap[String, MemberMetadata] - private val offsets = new mutable.HashMap[TopicPartition, OffsetAndMetadata] + + private val offsets = new mutable.HashMap[TopicPartition, CommitRecordMetadataAndOffset] + private val pendingOffsetCommits = new mutable.HashMap[TopicPartition, OffsetAndMetadata] - // A map from a producer id to the open offset commits for that producer id. - private val pendingTransactionalOffsetCommits = new mutable.HashMap[Long, mutable.Map[TopicPartition, OffsetAndMetadata]]() + + private val pendingTransactionalOffsetCommits = new mutable.HashMap[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]]() + private var receivedTransactionalOffsetCommits = false + private var receivedConsumerOffsetCommits = false var protocolType: Option[String] = None @@ -271,19 +286,27 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState GroupOverview(groupId, protocolType.getOrElse("")) } - def initializeOffsets(offsets: collection.Map[TopicPartition, OffsetAndMetadata], -pendingTxnOffsets: Map[Long, mutable.Map[TopicPartition, OffsetAndMetadata]]) { + def initializeOffsets(offsets: collection.Map[TopicPartition, CommitRecordMetadataAndOffset], +pendingTxnOffsets: Map[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]]) { this.offsets ++= offsets this.pendingTransactionalOffsetCommits ++= pendingTxnOffsets } - def completePendingOffsetWrite(topicPartition: TopicPartition, offset: OffsetAndMetadata) { -if (pendingOffsetCommit
kafka git commit: KAFKA-5247; Materialize committed offsets in offset order
Repository: kafka Updated Branches: refs/heads/0.11.0 33216846f -> 2b326faf5 KAFKA-5247; Materialize committed offsets in offset order With this patch, offset commits are always materialized according to the order of the commit records in the offsets topic. Before this patch, transactional offset commits were materialized in transaction order. However, the log cleaner will always preserve the record with the greatest offset. This meant that if there was a mix of offset commits from a consumer and a transactional producer, then it we would switch from transactional order to offset order after cleaning, resulting in an inconsistent state. Author: Apurva Mehta Reviewers: Guozhang Wang , Jason Gustafson Closes #3108 from apurvam/KAFKA-5247-materialize-committed-offsets-in-offset-order (cherry picked from commit d1853f791147d66303c2700a8e58af762518188c) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/2b326faf Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/2b326faf Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/2b326faf Branch: refs/heads/0.11.0 Commit: 2b326faf5f5a2bc632b842c26d2f48a07fa8a15e Parents: 3321684 Author: Apurva Mehta Authored: Tue May 23 09:45:31 2017 -0700 Committer: Jason Gustafson Committed: Tue May 23 09:47:03 2017 -0700 -- .../kafka/coordinator/group/GroupMetadata.scala | 96 .../group/GroupMetadataManager.scala| 50 + .../group/GroupMetadataManagerTest.scala| 102 - .../coordinator/group/GroupMetadataTest.scala | 109 ++- 4 files changed, 310 insertions(+), 47 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/2b326faf/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala -- diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala index 2f76d63..302fcb5 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala @@ -127,6 +127,16 @@ case class GroupSummary(state: String, members: List[MemberSummary]) /** + * We cache offset commits along with their commit record offset. This enables us to ensure that the latest offset + * commit is always materialized when we have a mix of transactional and regular offset commits. Without preserving + * information of the commit record offset, compaction of the offsets topic it self may result in the wrong offset commit + * being materialized. + */ +case class CommitRecordMetadataAndOffset(appendedBatchOffset: Option[Long], offsetAndMetadata: OffsetAndMetadata) { + def olderThan(that: CommitRecordMetadataAndOffset) : Boolean = appendedBatchOffset.get < that.appendedBatchOffset.get +} + +/** * Group contains the following metadata: * * Membership metadata: @@ -143,12 +153,17 @@ case class GroupSummary(state: String, private[group] class GroupMetadata(val groupId: String, initialState: GroupState = Empty) extends Logging { private var state: GroupState = initialState + private val members = new mutable.HashMap[String, MemberMetadata] - private val offsets = new mutable.HashMap[TopicPartition, OffsetAndMetadata] + + private val offsets = new mutable.HashMap[TopicPartition, CommitRecordMetadataAndOffset] + private val pendingOffsetCommits = new mutable.HashMap[TopicPartition, OffsetAndMetadata] - // A map from a producer id to the open offset commits for that producer id. - private val pendingTransactionalOffsetCommits = new mutable.HashMap[Long, mutable.Map[TopicPartition, OffsetAndMetadata]]() + + private val pendingTransactionalOffsetCommits = new mutable.HashMap[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]]() + private var receivedTransactionalOffsetCommits = false + private var receivedConsumerOffsetCommits = false var protocolType: Option[String] = None @@ -271,19 +286,27 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState GroupOverview(groupId, protocolType.getOrElse("")) } - def initializeOffsets(offsets: collection.Map[TopicPartition, OffsetAndMetadata], -pendingTxnOffsets: Map[Long, mutable.Map[TopicPartition, OffsetAndMetadata]]) { + def initializeOffsets(offsets: collection.Map[TopicPartition, CommitRecordMetadataAndOffset], +pendingTxnOffsets: Map[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]]) { this.offsets ++= offsets this.pendingTransactionalOffsetCommits ++= pendingTxnOffsets } - def completePendi
kafka git commit: MINOR: Log transaction metadata state transitions plus a few cleanups
Repository: kafka Updated Branches: refs/heads/trunk 5a6676bfc -> 70ec4b1d9 MINOR: Log transaction metadata state transitions plus a few cleanups Author: Jason Gustafson Reviewers: Guozhang Wang Closes #3081 from hachikuji/minor-add-txn-transition-logging Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/70ec4b1d Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/70ec4b1d Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/70ec4b1d Branch: refs/heads/trunk Commit: 70ec4b1d927cad7373eda2c54ef44bfc4275832f Parents: 5a6676b Author: Jason Gustafson Authored: Tue May 23 09:53:18 2017 -0700 Committer: Jason Gustafson Committed: Tue May 23 09:53:18 2017 -0700 -- .../producer/internals/TransactionManager.java | 4 +- .../transaction/TransactionCoordinator.scala| 22 +-- .../transaction/TransactionLog.scala| 41 ++--- .../TransactionMarkerChannelManager.scala | 5 +- .../transaction/TransactionMetadata.scala | 160 +++ .../transaction/TransactionStateManager.scala | 41 ++--- .../TransactionCoordinatorTest.scala| 119 -- .../transaction/TransactionLogTest.scala| 42 +++-- .../TransactionMarkerChannelManagerTest.scala | 15 +- ...tionMarkerRequestCompletionHandlerTest.scala | 3 +- .../TransactionStateManagerTest.scala | 133 +++ 11 files changed, 306 insertions(+), 279 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/70ec4b1d/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index c6787f2..d84a88e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -407,10 +407,10 @@ public class TransactionManager { } private synchronized void transitionTo(State target, Exception error) { -if (target == State.ERROR && error != null) -lastError = error; if (currentState.isTransitionValid(currentState, target)) { currentState = target; +if (target == State.ERROR && error != null) +lastError = error; } else { throw new KafkaException("Invalid transition attempted from state " + currentState.name() + " to state " + target.name()); http://git-wip-us.apache.org/repos/asf/kafka/blob/70ec4b1d/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala -- diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 491e16a..b58c710 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -59,7 +59,7 @@ object TransactionCoordinator { InitProducerIdResult(RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, error) } - private def initTransactionMetadata(txnMetadata: TransactionMetadataTransition): InitProducerIdResult = { + private def initTransactionMetadata(txnMetadata: TxnTransitMetadata): InitProducerIdResult = { InitProducerIdResult(txnMetadata.producerId, txnMetadata.producerEpoch, Errors.NONE) } } @@ -113,18 +113,20 @@ class TransactionCoordinator(brokerId: Int, responseCallback(initTransactionError(Errors.INVALID_TRANSACTION_TIMEOUT)) } else { // only try to get a new producerId and update the cache if the transactional id is unknown - val result: Either[InitProducerIdResult, (Int, TransactionMetadataTransition)] = txnManager.getTransactionState(transactionalId) match { + val result: Either[InitProducerIdResult, (Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId) match { case None => val producerId = producerIdManager.generateProducerId() val now = time.milliseconds() - val createdMetadata = new TransactionMetadata(producerId = producerId, + val createdMetadata = new TransactionMetadata( +transactionalId = transactionalId, +producerId = producerId, producerEpoch = 0, txnTimeoutMs = transactionTimeoutMs, state = Empty, topicPartitions = collection.mutable.Set.emp
kafka git commit: MINOR: Log transaction metadata state transitions plus a few cleanups
Repository: kafka Updated Branches: refs/heads/0.11.0 9e0581df5 -> fdbf2e5f7 MINOR: Log transaction metadata state transitions plus a few cleanups Author: Jason Gustafson Reviewers: Guozhang Wang Closes #3081 from hachikuji/minor-add-txn-transition-logging (cherry picked from commit 70ec4b1d927cad7373eda2c54ef44bfc4275832f) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/fdbf2e5f Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/fdbf2e5f Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/fdbf2e5f Branch: refs/heads/0.11.0 Commit: fdbf2e5f7ad99b25feff2122cbcf4c51697250e3 Parents: 9e0581d Author: Jason Gustafson Authored: Tue May 23 09:53:18 2017 -0700 Committer: Jason Gustafson Committed: Tue May 23 09:57:08 2017 -0700 -- .../producer/internals/TransactionManager.java | 4 +- .../transaction/TransactionCoordinator.scala| 22 +-- .../transaction/TransactionLog.scala| 41 ++--- .../TransactionMarkerChannelManager.scala | 5 +- .../transaction/TransactionMetadata.scala | 160 +++ .../transaction/TransactionStateManager.scala | 41 ++--- .../TransactionCoordinatorTest.scala| 119 -- .../transaction/TransactionLogTest.scala| 42 +++-- .../TransactionMarkerChannelManagerTest.scala | 15 +- ...tionMarkerRequestCompletionHandlerTest.scala | 3 +- .../TransactionStateManagerTest.scala | 133 +++ 11 files changed, 306 insertions(+), 279 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/fdbf2e5f/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index c6787f2..d84a88e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -407,10 +407,10 @@ public class TransactionManager { } private synchronized void transitionTo(State target, Exception error) { -if (target == State.ERROR && error != null) -lastError = error; if (currentState.isTransitionValid(currentState, target)) { currentState = target; +if (target == State.ERROR && error != null) +lastError = error; } else { throw new KafkaException("Invalid transition attempted from state " + currentState.name() + " to state " + target.name()); http://git-wip-us.apache.org/repos/asf/kafka/blob/fdbf2e5f/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala -- diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 491e16a..b58c710 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -59,7 +59,7 @@ object TransactionCoordinator { InitProducerIdResult(RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, error) } - private def initTransactionMetadata(txnMetadata: TransactionMetadataTransition): InitProducerIdResult = { + private def initTransactionMetadata(txnMetadata: TxnTransitMetadata): InitProducerIdResult = { InitProducerIdResult(txnMetadata.producerId, txnMetadata.producerEpoch, Errors.NONE) } } @@ -113,18 +113,20 @@ class TransactionCoordinator(brokerId: Int, responseCallback(initTransactionError(Errors.INVALID_TRANSACTION_TIMEOUT)) } else { // only try to get a new producerId and update the cache if the transactional id is unknown - val result: Either[InitProducerIdResult, (Int, TransactionMetadataTransition)] = txnManager.getTransactionState(transactionalId) match { + val result: Either[InitProducerIdResult, (Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId) match { case None => val producerId = producerIdManager.generateProducerId() val now = time.milliseconds() - val createdMetadata = new TransactionMetadata(producerId = producerId, + val createdMetadata = new TransactionMetadata( +transactionalId = transactionalId, +producerId = producerId, producerEpoch = 0, txnTimeoutMs = tra
kafka git commit: MINOR: Improve the help doc of consumer group command
Repository: kafka Updated Branches: refs/heads/trunk 4d89db968 -> 88200938f MINOR: Improve the help doc of consumer group command Clarify the consumer group command help message around `zookeeper`, `bootstrap-server`, and `new-consumer` options. Author: Vahid Hashemian Reviewers: Jason Gustafson Closes #2046 from vahidhashemian/minor/improve_consumer_group_command_doc Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/88200938 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/88200938 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/88200938 Branch: refs/heads/trunk Commit: 88200938f0c3ace5ec0bdb65a315ebd9b6adb4d0 Parents: 4d89db9 Author: Vahid Hashemian Authored: Wed May 24 09:14:10 2017 -0700 Committer: Jason Gustafson Committed: Wed May 24 09:14:10 2017 -0700 -- core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/88200938/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala -- diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 69f0d8a..7715a95 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -676,9 +676,9 @@ object ConsumerGroupCommand extends Logging { } class ConsumerGroupCommandOptions(args: Array[String]) { -val ZkConnectDoc = "REQUIRED (only when using old consumer): The connection string for the zookeeper connection in the form host:port. " + +val ZkConnectDoc = "REQUIRED (for consumer groups based on the old consumer): The connection string for the zookeeper connection in the form host:port. " + "Multiple URLS can be given to allow fail-over." -val BootstrapServerDoc = "REQUIRED (unless old consumer is used): The server to connect to." +val BootstrapServerDoc = "REQUIRED (for consumer groups based on the new consumer): The server to connect to." val GroupDoc = "The consumer group we wish to act on." val TopicDoc = "The topic whose consumer group information should be deleted or topic whose should be included in the reset offset process. " + "In `reset-offsets` case, partitions can be specified using this format: `topic1:0,1,2`, where 0,1,2 are the partition to be included in the process. " + @@ -694,7 +694,7 @@ object ConsumerGroupCommand extends Logging { "Pass in just a topic to delete the given topic's partition offsets and ownership information " + "for every consumer group. For instance --topic t1" + nl + "WARNING: Group deletion only works for old ZK-based consumer groups, and one has to use it carefully to only delete groups that are not active." -val NewConsumerDoc = "Use new consumer. This is the default." +val NewConsumerDoc = "Use new consumer. This option requires that the 'bootstrap-server' option is used." val TimeoutMsDoc = "The timeout that can be set for some use cases. For example, it can be used when describing the group " + "to specify the maximum amount of time in milliseconds to wait before the group stabilizes (when the group is just created, " + "or is going through some changes)."
kafka git commit: MINOR: Improve the help doc of consumer group command
Repository: kafka Updated Branches: refs/heads/0.11.0 23de4f4cd -> 308c7e47c MINOR: Improve the help doc of consumer group command Clarify the consumer group command help message around `zookeeper`, `bootstrap-server`, and `new-consumer` options. Author: Vahid Hashemian Reviewers: Jason Gustafson Closes #2046 from vahidhashemian/minor/improve_consumer_group_command_doc (cherry picked from commit 88200938f0c3ace5ec0bdb65a315ebd9b6adb4d0) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/308c7e47 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/308c7e47 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/308c7e47 Branch: refs/heads/0.11.0 Commit: 308c7e47c3b8c4fa2835534b9d28dd9c2e2dc1a3 Parents: 23de4f4 Author: Vahid Hashemian Authored: Wed May 24 09:14:10 2017 -0700 Committer: Jason Gustafson Committed: Wed May 24 09:14:28 2017 -0700 -- core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/308c7e47/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala -- diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 69f0d8a..7715a95 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -676,9 +676,9 @@ object ConsumerGroupCommand extends Logging { } class ConsumerGroupCommandOptions(args: Array[String]) { -val ZkConnectDoc = "REQUIRED (only when using old consumer): The connection string for the zookeeper connection in the form host:port. " + +val ZkConnectDoc = "REQUIRED (for consumer groups based on the old consumer): The connection string for the zookeeper connection in the form host:port. " + "Multiple URLS can be given to allow fail-over." -val BootstrapServerDoc = "REQUIRED (unless old consumer is used): The server to connect to." +val BootstrapServerDoc = "REQUIRED (for consumer groups based on the new consumer): The server to connect to." val GroupDoc = "The consumer group we wish to act on." val TopicDoc = "The topic whose consumer group information should be deleted or topic whose should be included in the reset offset process. " + "In `reset-offsets` case, partitions can be specified using this format: `topic1:0,1,2`, where 0,1,2 are the partition to be included in the process. " + @@ -694,7 +694,7 @@ object ConsumerGroupCommand extends Logging { "Pass in just a topic to delete the given topic's partition offsets and ownership information " + "for every consumer group. For instance --topic t1" + nl + "WARNING: Group deletion only works for old ZK-based consumer groups, and one has to use it carefully to only delete groups that are not active." -val NewConsumerDoc = "Use new consumer. This is the default." +val NewConsumerDoc = "Use new consumer. This option requires that the 'bootstrap-server' option is used." val TimeoutMsDoc = "The timeout that can be set for some use cases. For example, it can be used when describing the group " + "to specify the maximum amount of time in milliseconds to wait before the group stabilizes (when the group is just created, " + "or is going through some changes)."
[5/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
KAFKA-5259; TransactionalId auth implies ProducerId auth Author: Jason Gustafson Reviewers: Apurva Mehta , Jun Rao Closes #3075 from hachikuji/KAFKA-5259-FIXED Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/38f6cae9 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/38f6cae9 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/38f6cae9 Branch: refs/heads/trunk Commit: 38f6cae9e879baa35c5dbc5829bf09ecd59930c2 Parents: 8820093 Author: Jason Gustafson Authored: Wed May 24 15:26:46 2017 -0700 Committer: Jason Gustafson Committed: Wed May 24 15:26:46 2017 -0700 -- .../apache/kafka/clients/ClientResponse.java|7 +- .../kafka/clients/admin/AclOperation.java |7 +- .../clients/consumer/internals/Fetcher.java |2 +- .../kafka/clients/producer/KafkaProducer.java | 35 +- .../clients/producer/internals/Sender.java | 169 +- .../producer/internals/TransactionManager.java | 268 ++-- .../ProducerIdAuthorizationException.java | 23 - .../TransactionalIdAuthorizationException.java |2 +- .../apache/kafka/common/protocol/Errors.java| 13 +- .../apache/kafka/common/protocol/Protocol.java |3 + .../common/requests/AddOffsetsToTxnRequest.java | 11 + .../requests/AddOffsetsToTxnResponse.java |5 +- .../requests/AddPartitionsToTxnRequest.java | 11 + .../requests/AddPartitionsToTxnResponse.java|3 +- .../kafka/common/requests/EndTxnRequest.java| 11 + .../kafka/common/requests/EndTxnResponse.java |3 +- .../common/requests/InitProducerIdResponse.java | 12 +- .../kafka/common/requests/ProduceRequest.java |5 +- .../kafka/common/requests/ProduceResponse.java |3 + .../common/requests/TxnOffsetCommitRequest.java | 31 +- .../requests/TxnOffsetCommitResponse.java |1 + .../requests/WriteTxnMarkersResponse.java |1 + .../kafka/clients/admin/AclOperationTest.java |3 +- .../clients/producer/internals/SenderTest.java | 183 +-- .../internals/TransactionManagerTest.java | 445 -- .../common/requests/RequestResponseTest.java| 10 +- .../src/main/scala/kafka/admin/AclCommand.scala | 54 +- .../kafka/coordinator/group/GroupMetadata.scala | 43 +- .../group/GroupMetadataManager.scala|5 +- .../coordinator/group/MemberMetadata.scala | 14 +- .../scala/kafka/security/auth/Operation.scala |6 +- .../scala/kafka/security/auth/Resource.scala|3 +- .../kafka/security/auth/ResourceType.scala | 16 +- .../src/main/scala/kafka/server/KafkaApis.scala | 316 ++-- .../kafka/api/AuthorizerIntegrationTest.scala | 292 +++- .../kafka/api/TransactionsBounceTest.scala | 31 +- .../kafka/api/TransactionsTest.scala|7 +- .../scala/unit/kafka/admin/AclCommandTest.scala | 36 +- .../group/GroupCoordinatorResponseTest.scala| 1492 -- .../group/GroupCoordinatorTest.scala| 1492 ++ .../coordinator/group/GroupMetadataTest.scala |2 +- .../unit/kafka/server/RequestQuotaTest.scala|9 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 10 +- 43 files changed, 2771 insertions(+), 2324 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java index 715eae7..0ff30e9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.RequestHeader; @@ -31,7 +32,7 @@ public class ClientResponse { private final long receivedTimeMs; private final long latencyMs; private final boolean disconnected; -private final RuntimeException versionMismatch; +private final UnsupportedVersionException versionMismatch; private final AbstractResponse responseBody; /** @@ -51,7 +52,7 @@ public class ClientResponse { long createdTimeMs, long receivedTimeMs, boolean disconnected, - RuntimeException versionMismatch, + UnsupportedVersionException versionMismatch, AbstractResponse responseBody) { this.requestHeader = requestHeader; this.callback = callbac
[4/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java -- diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index cc30f4d..719efe9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.Cluster; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.MetricConfig; @@ -231,20 +232,8 @@ public class SenderTest { int maxRetries = 1; Metrics m = new Metrics(); try { -Sender sender = new Sender(client, -metadata, -this.accumulator, -false, -MAX_REQUEST_SIZE, -ACKS_ALL, -maxRetries, -m, -time, -REQUEST_TIMEOUT, -50, -null, -apiVersions -); +Sender sender = new Sender(client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, +maxRetries, m, time, REQUEST_TIMEOUT, 50, null, apiVersions); // do a successful retry Future future = accumulator.append(tp0, 0L, "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); // connect @@ -290,20 +279,8 @@ public class SenderTest { int maxRetries = 1; Metrics m = new Metrics(); try { -Sender sender = new Sender(client, -metadata, -this.accumulator, -true, -MAX_REQUEST_SIZE, -ACKS_ALL, -maxRetries, -m, -time, -REQUEST_TIMEOUT, -50, -null, -apiVersions -); +Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, +m, time, REQUEST_TIMEOUT, 50, null, apiVersions); // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 Cluster cluster1 = TestUtils.clusterWith(2, "test", 2); metadata.update(cluster1, Collections.emptySet(), time.milliseconds()); @@ -375,21 +352,63 @@ public class SenderTest { } @Test -public void testInitPidRequest() throws Exception { +public void testInitProducerIdRequest() throws Exception { final long producerId = 343434L; TransactionManager transactionManager = new TransactionManager(); setupWithTransactionState(transactionManager); client.setNode(new Node(1, "localhost", 33343)); +prepareAndReceiveInitProducerId(producerId, Errors.NONE); +assertTrue(transactionManager.hasProducerId()); +assertEquals(producerId, transactionManager.producerIdAndEpoch().producerId); +assertEquals((short) 0, transactionManager.producerIdAndEpoch().epoch); +} + +@Test +public void testClusterAuthorizationExceptionInInitProducerIdRequest() throws Exception { +final long producerId = 343434L; +TransactionManager transactionManager = new TransactionManager(); +setupWithTransactionState(transactionManager); +client.setNode(new Node(1, "localhost", 33343)); +prepareAndReceiveInitProducerId(producerId, Errors.CLUSTER_AUTHORIZATION_FAILED); +assertFalse(transactionManager.hasProducerId()); +assertTrue(transactionManager.isInErrorState()); +assertTrue(transactionManager.lastError() instanceof ClusterAuthorizationException); + +// cluster authorization is a fatal error for the producer +assertSendFailure(ClusterAuthorizationException.class); +} + +@Test +public void testClusterAuthorizationExceptionInProduceRequest() throws Exception { +final long producerId = 343434L; +TransactionManager transactionManager = new TransactionManager(); +setupWithTransactionState(transactionManager); + +client.setNode(new Node(1, "localhost", 33343)); +prepareAndReceiveInitProducerId(producerId, Errors.NONE); +assertTrue(transactionManager.hasProduc
[1/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
Repository: kafka Updated Branches: refs/heads/trunk 88200938f -> 38f6cae9e http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala -- diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala new file mode 100644 index 000..358e12c --- /dev/null +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -0,0 +1,1492 @@ +/** + * 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 kafka.coordinator.group + +import kafka.common.OffsetAndMetadata +import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager} +import kafka.utils._ +import kafka.utils.timer.MockTimer +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} +import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse +import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, OffsetFetchResponse, TransactionResult} +import org.easymock.{Capture, EasyMock, IAnswer} +import java.util.concurrent.TimeUnit + +import org.apache.kafka.common.internals.Topic +import org.junit.Assert._ +import org.junit.{After, Assert, Before, Test} +import org.scalatest.junit.JUnitSuite + +import scala.collection._ +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, Future, Promise, TimeoutException} + +class GroupCoordinatorTest extends JUnitSuite { + type JoinGroupCallback = JoinGroupResult => Unit + type SyncGroupCallbackParams = (Array[Byte], Errors) + type SyncGroupCallback = (Array[Byte], Errors) => Unit + type HeartbeatCallbackParams = Errors + type HeartbeatCallback = Errors => Unit + type CommitOffsetCallbackParams = Map[TopicPartition, Errors] + type CommitOffsetCallback = Map[TopicPartition, Errors] => Unit + type LeaveGroupCallbackParams = Errors + type LeaveGroupCallback = Errors => Unit + + val ClientId = "consumer-test" + val ClientHost = "localhost" + val ConsumerMinSessionTimeout = 10 + val ConsumerMaxSessionTimeout = 1000 + val DefaultRebalanceTimeout = 500 + val DefaultSessionTimeout = 500 + val GroupInitialRebalanceDelay = 50 + var timer: MockTimer = null + var groupCoordinator: GroupCoordinator = null + var replicaManager: ReplicaManager = null + var scheduler: KafkaScheduler = null + var zkUtils: ZkUtils = null + + private val groupId = "groupId" + private val protocolType = "consumer" + private val memberId = "memberId" + private val metadata = Array[Byte]() + private val protocols = List(("range", metadata)) + private var groupPartitionId: Int = -1 + + // we use this string value since its hashcode % #.partitions is different + private val otherGroupId = "otherGroup" + + @Before + def setUp() { +val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") +props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) +props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) +props.setProperty(KafkaConfig.GroupInitialRebalanceDelayMsProp, GroupInitialRebalanceDelay.toString) +// make two partitions of the group topic to make sure some partitions are not owned by the coordinator +val ret = mutable.Map[String, Map[Int, Seq[Int]]]() +ret += (Topic.GROUP_METADATA_TOPIC_NAME -> Map(0 -> Seq(1), 1 -> Seq(1))) + +replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager]) + +zkUtils = EasyMock.createNiceMock(classOf[ZkUtils]) +// make two partitions of the group topic to make sure some partitions are not owned by the coordinator + EasyMock.expect(zkUtils.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME)).andReturn(Some(2)) +EasyMock.replay(zkUtils) + +timer = new MockTimer + +val config = KafkaConfig.fromProps(props) + +val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false) +
[2/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala -- diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala deleted file mode 100644 index 0a8209e..000 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala +++ /dev/null @@ -1,1492 +0,0 @@ -/** - * 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 kafka.coordinator.group - -import kafka.common.OffsetAndMetadata -import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager} -import kafka.utils._ -import kafka.utils.timer.MockTimer -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} -import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse -import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, OffsetFetchResponse, TransactionResult} -import org.easymock.{Capture, EasyMock, IAnswer} -import java.util.concurrent.TimeUnit - -import org.apache.kafka.common.internals.Topic -import org.junit.Assert._ -import org.junit.{After, Assert, Before, Test} -import org.scalatest.junit.JUnitSuite - -import scala.collection._ -import scala.concurrent.duration.Duration -import scala.concurrent.{Await, Future, Promise, TimeoutException} - -class GroupCoordinatorResponseTest extends JUnitSuite { - type JoinGroupCallback = JoinGroupResult => Unit - type SyncGroupCallbackParams = (Array[Byte], Errors) - type SyncGroupCallback = (Array[Byte], Errors) => Unit - type HeartbeatCallbackParams = Errors - type HeartbeatCallback = Errors => Unit - type CommitOffsetCallbackParams = Map[TopicPartition, Errors] - type CommitOffsetCallback = Map[TopicPartition, Errors] => Unit - type LeaveGroupCallbackParams = Errors - type LeaveGroupCallback = Errors => Unit - - val ClientId = "consumer-test" - val ClientHost = "localhost" - val ConsumerMinSessionTimeout = 10 - val ConsumerMaxSessionTimeout = 1000 - val DefaultRebalanceTimeout = 500 - val DefaultSessionTimeout = 500 - val GroupInitialRebalanceDelay = 50 - var timer: MockTimer = null - var groupCoordinator: GroupCoordinator = null - var replicaManager: ReplicaManager = null - var scheduler: KafkaScheduler = null - var zkUtils: ZkUtils = null - - private val groupId = "groupId" - private val protocolType = "consumer" - private val memberId = "memberId" - private val metadata = Array[Byte]() - private val protocols = List(("range", metadata)) - private var groupPartitionId: Int = -1 - - // we use this string value since its hashcode % #.partitions is different - private val otherGroupId = "otherGroup" - - @Before - def setUp() { -val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") -props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) -props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) -props.setProperty(KafkaConfig.GroupInitialRebalanceDelayMsProp, GroupInitialRebalanceDelay.toString) -// make two partitions of the group topic to make sure some partitions are not owned by the coordinator -val ret = mutable.Map[String, Map[Int, Seq[Int]]]() -ret += (Topic.GROUP_METADATA_TOPIC_NAME -> Map(0 -> Seq(1), 1 -> Seq(1))) - -replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager]) - -zkUtils = EasyMock.createNiceMock(classOf[ZkUtils]) -// make two partitions of the group topic to make sure some partitions are not owned by the coordinator - EasyMock.expect(zkUtils.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME)).andReturn(Some(2)) -EasyMock.replay(zkUtils) - -timer = new MockTimer - -val config = KafkaConfig.fromProps(props) - -val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false) -val joinPurgatory = new Delaye
[3/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
http://git-wip-us.apache.org/repos/asf/kafka/blob/9a21bf20/core/src/main/scala/kafka/server/KafkaApis.scala -- diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index b0d354b..380685f 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -93,7 +93,7 @@ class KafkaApis(val requestChannel: RequestChannel, trace("Handling request:%s from connection %s;securityProtocol:%s,principal:%s". format(request.requestDesc(true), request.connectionId, request.securityProtocol, request.session.principal)) ApiKeys.forId(request.requestId) match { -case ApiKeys.PRODUCE => handleProducerRequest(request) +case ApiKeys.PRODUCE => handleProduceRequest(request) case ApiKeys.FETCH => handleFetchRequest(request) case ApiKeys.LIST_OFFSETS => handleListOffsetRequest(request) case ApiKeys.METADATA => handleTopicMetadataRequest(request) @@ -368,97 +368,107 @@ class KafkaApis(val requestChannel: RequestChannel, /** * Handle a produce request */ - def handleProducerRequest(request: RequestChannel.Request) { + def handleProduceRequest(request: RequestChannel.Request) { val produceRequest = request.body[ProduceRequest] val numBytesAppended = request.header.toStruct.sizeOf + request.bodyAndSize.size -if (produceRequest.isTransactional && !authorize(request.session, Write, new Resource(ProducerTransactionalId, produceRequest.transactionalId( - sendResponseMaybeThrottle(request, (throttleMs: Int) => produceRequest.getErrorResponse(throttleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception)) -else if (produceRequest.isIdempotent && !authorize(request.session, Write, Resource.ProducerIdResource)) - sendResponseMaybeThrottle(request, (throttleMs: Int) => produceRequest.getErrorResponse(throttleMs, Errors.PRODUCER_ID_AUTHORIZATION_FAILED.exception)) -else { - val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = -produceRequest.partitionRecordsOrFail.asScala.partition { case (tp, _) => - authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic) -} +def sendErrorResponse(error: Errors): Unit = { + sendResponseMaybeThrottle(request, requestThrottleMs => +produceRequest.getErrorResponse(requestThrottleMs, error.exception)) +} - val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition { -case (tp, _) => authorize(request.session, Write, new Resource(Topic, tp.topic)) +if (produceRequest.isTransactional) { + if (!authorize(request.session, Write, new Resource(TransactionalId, produceRequest.transactionalId))) { +sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) +return } + // Note that authorization to a transactionalId implies ProducerId authorization - // the callback for sending a produce response - def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { +} else if (produceRequest.isIdempotent && !authorize(request.session, IdempotentWrite, Resource.ClusterResource)) { + sendErrorResponse(Errors.CLUSTER_AUTHORIZATION_FAILED) + return +} + +val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = + produceRequest.partitionRecordsOrFail.asScala.partition { case (tp, _) => +authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic) + } -val mergedResponseStatus = responseStatus ++ -unauthorizedForWriteRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED)) ++ -nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION)) +val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition { + case (tp, _) => authorize(request.session, Write, new Resource(Topic, tp.topic)) +} -var errorInResponse = false +// the callback for sending a produce response +def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { -mergedResponseStatus.foreach { case (topicPartition, status) => - if (status.error != Errors.NONE) { -errorInResponse = true -debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - request.header.correlationId, - request.header.clientId, - topicPartition, - status.error.exceptionName)) - } + val mergedResponseStatus = responseSt
[5/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
KAFKA-5259; TransactionalId auth implies ProducerId auth Author: Jason Gustafson Reviewers: Apurva Mehta , Jun Rao Closes #3075 from hachikuji/KAFKA-5259-FIXED (cherry picked from commit 38f6cae9e879baa35c5dbc5829bf09ecd59930c2) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/9a21bf20 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/9a21bf20 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/9a21bf20 Branch: refs/heads/0.11.0 Commit: 9a21bf20b623b790c9910813e373d2d61fe84c2c Parents: 7cb57dc Author: Jason Gustafson Authored: Wed May 24 15:26:46 2017 -0700 Committer: Jason Gustafson Committed: Wed May 24 15:28:55 2017 -0700 -- .../apache/kafka/clients/ClientResponse.java|7 +- .../kafka/clients/admin/AclOperation.java |7 +- .../clients/consumer/internals/Fetcher.java |2 +- .../kafka/clients/producer/KafkaProducer.java | 35 +- .../clients/producer/internals/Sender.java | 169 +- .../producer/internals/TransactionManager.java | 268 ++-- .../ProducerIdAuthorizationException.java | 23 - .../TransactionalIdAuthorizationException.java |2 +- .../apache/kafka/common/protocol/Errors.java| 13 +- .../apache/kafka/common/protocol/Protocol.java |3 + .../common/requests/AddOffsetsToTxnRequest.java | 11 + .../requests/AddOffsetsToTxnResponse.java |5 +- .../requests/AddPartitionsToTxnRequest.java | 11 + .../requests/AddPartitionsToTxnResponse.java|3 +- .../kafka/common/requests/EndTxnRequest.java| 11 + .../kafka/common/requests/EndTxnResponse.java |3 +- .../common/requests/InitProducerIdResponse.java | 12 +- .../kafka/common/requests/ProduceRequest.java |5 +- .../kafka/common/requests/ProduceResponse.java |3 + .../common/requests/TxnOffsetCommitRequest.java | 31 +- .../requests/TxnOffsetCommitResponse.java |1 + .../requests/WriteTxnMarkersResponse.java |1 + .../kafka/clients/admin/AclOperationTest.java |3 +- .../clients/producer/internals/SenderTest.java | 183 +-- .../internals/TransactionManagerTest.java | 445 -- .../common/requests/RequestResponseTest.java| 10 +- .../src/main/scala/kafka/admin/AclCommand.scala | 54 +- .../kafka/coordinator/group/GroupMetadata.scala | 43 +- .../group/GroupMetadataManager.scala|5 +- .../coordinator/group/MemberMetadata.scala | 14 +- .../scala/kafka/security/auth/Operation.scala |6 +- .../scala/kafka/security/auth/Resource.scala|3 +- .../kafka/security/auth/ResourceType.scala | 16 +- .../src/main/scala/kafka/server/KafkaApis.scala | 316 ++-- .../kafka/api/AuthorizerIntegrationTest.scala | 292 +++- .../kafka/api/TransactionsBounceTest.scala | 31 +- .../kafka/api/TransactionsTest.scala|7 +- .../scala/unit/kafka/admin/AclCommandTest.scala | 36 +- .../group/GroupCoordinatorResponseTest.scala| 1492 -- .../group/GroupCoordinatorTest.scala| 1492 ++ .../coordinator/group/GroupMetadataTest.scala |2 +- .../unit/kafka/server/RequestQuotaTest.scala|9 +- .../test/scala/unit/kafka/utils/TestUtils.scala | 10 +- 43 files changed, 2771 insertions(+), 2324 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/9a21bf20/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java index 715eae7..0ff30e9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.RequestHeader; @@ -31,7 +32,7 @@ public class ClientResponse { private final long receivedTimeMs; private final long latencyMs; private final boolean disconnected; -private final RuntimeException versionMismatch; +private final UnsupportedVersionException versionMismatch; private final AbstractResponse responseBody; /** @@ -51,7 +52,7 @@ public class ClientResponse { long createdTimeMs, long receivedTimeMs, boolean disconnected, - RuntimeException versionMismatch, + UnsupportedVersionException versionMismatch, Abstrac
[1/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
Repository: kafka Updated Branches: refs/heads/0.11.0 7cb57dc99 -> 9a21bf20b http://git-wip-us.apache.org/repos/asf/kafka/blob/9a21bf20/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala -- diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala new file mode 100644 index 000..358e12c --- /dev/null +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -0,0 +1,1492 @@ +/** + * 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 kafka.coordinator.group + +import kafka.common.OffsetAndMetadata +import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager} +import kafka.utils._ +import kafka.utils.timer.MockTimer +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} +import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse +import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, OffsetFetchResponse, TransactionResult} +import org.easymock.{Capture, EasyMock, IAnswer} +import java.util.concurrent.TimeUnit + +import org.apache.kafka.common.internals.Topic +import org.junit.Assert._ +import org.junit.{After, Assert, Before, Test} +import org.scalatest.junit.JUnitSuite + +import scala.collection._ +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, Future, Promise, TimeoutException} + +class GroupCoordinatorTest extends JUnitSuite { + type JoinGroupCallback = JoinGroupResult => Unit + type SyncGroupCallbackParams = (Array[Byte], Errors) + type SyncGroupCallback = (Array[Byte], Errors) => Unit + type HeartbeatCallbackParams = Errors + type HeartbeatCallback = Errors => Unit + type CommitOffsetCallbackParams = Map[TopicPartition, Errors] + type CommitOffsetCallback = Map[TopicPartition, Errors] => Unit + type LeaveGroupCallbackParams = Errors + type LeaveGroupCallback = Errors => Unit + + val ClientId = "consumer-test" + val ClientHost = "localhost" + val ConsumerMinSessionTimeout = 10 + val ConsumerMaxSessionTimeout = 1000 + val DefaultRebalanceTimeout = 500 + val DefaultSessionTimeout = 500 + val GroupInitialRebalanceDelay = 50 + var timer: MockTimer = null + var groupCoordinator: GroupCoordinator = null + var replicaManager: ReplicaManager = null + var scheduler: KafkaScheduler = null + var zkUtils: ZkUtils = null + + private val groupId = "groupId" + private val protocolType = "consumer" + private val memberId = "memberId" + private val metadata = Array[Byte]() + private val protocols = List(("range", metadata)) + private var groupPartitionId: Int = -1 + + // we use this string value since its hashcode % #.partitions is different + private val otherGroupId = "otherGroup" + + @Before + def setUp() { +val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") +props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) +props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) +props.setProperty(KafkaConfig.GroupInitialRebalanceDelayMsProp, GroupInitialRebalanceDelay.toString) +// make two partitions of the group topic to make sure some partitions are not owned by the coordinator +val ret = mutable.Map[String, Map[Int, Seq[Int]]]() +ret += (Topic.GROUP_METADATA_TOPIC_NAME -> Map(0 -> Seq(1), 1 -> Seq(1))) + +replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager]) + +zkUtils = EasyMock.createNiceMock(classOf[ZkUtils]) +// make two partitions of the group topic to make sure some partitions are not owned by the coordinator + EasyMock.expect(zkUtils.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME)).andReturn(Some(2)) +EasyMock.replay(zkUtils) + +timer = new MockTimer + +val config = KafkaConfig.fromProps(props) + +val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false)
[2/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
http://git-wip-us.apache.org/repos/asf/kafka/blob/9a21bf20/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala -- diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala deleted file mode 100644 index 0a8209e..000 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala +++ /dev/null @@ -1,1492 +0,0 @@ -/** - * 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 kafka.coordinator.group - -import kafka.common.OffsetAndMetadata -import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager} -import kafka.utils._ -import kafka.utils.timer.MockTimer -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.record.{MemoryRecords, RecordBatch} -import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse -import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, OffsetFetchResponse, TransactionResult} -import org.easymock.{Capture, EasyMock, IAnswer} -import java.util.concurrent.TimeUnit - -import org.apache.kafka.common.internals.Topic -import org.junit.Assert._ -import org.junit.{After, Assert, Before, Test} -import org.scalatest.junit.JUnitSuite - -import scala.collection._ -import scala.concurrent.duration.Duration -import scala.concurrent.{Await, Future, Promise, TimeoutException} - -class GroupCoordinatorResponseTest extends JUnitSuite { - type JoinGroupCallback = JoinGroupResult => Unit - type SyncGroupCallbackParams = (Array[Byte], Errors) - type SyncGroupCallback = (Array[Byte], Errors) => Unit - type HeartbeatCallbackParams = Errors - type HeartbeatCallback = Errors => Unit - type CommitOffsetCallbackParams = Map[TopicPartition, Errors] - type CommitOffsetCallback = Map[TopicPartition, Errors] => Unit - type LeaveGroupCallbackParams = Errors - type LeaveGroupCallback = Errors => Unit - - val ClientId = "consumer-test" - val ClientHost = "localhost" - val ConsumerMinSessionTimeout = 10 - val ConsumerMaxSessionTimeout = 1000 - val DefaultRebalanceTimeout = 500 - val DefaultSessionTimeout = 500 - val GroupInitialRebalanceDelay = 50 - var timer: MockTimer = null - var groupCoordinator: GroupCoordinator = null - var replicaManager: ReplicaManager = null - var scheduler: KafkaScheduler = null - var zkUtils: ZkUtils = null - - private val groupId = "groupId" - private val protocolType = "consumer" - private val memberId = "memberId" - private val metadata = Array[Byte]() - private val protocols = List(("range", metadata)) - private var groupPartitionId: Int = -1 - - // we use this string value since its hashcode % #.partitions is different - private val otherGroupId = "otherGroup" - - @Before - def setUp() { -val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "") -props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString) -props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString) -props.setProperty(KafkaConfig.GroupInitialRebalanceDelayMsProp, GroupInitialRebalanceDelay.toString) -// make two partitions of the group topic to make sure some partitions are not owned by the coordinator -val ret = mutable.Map[String, Map[Int, Seq[Int]]]() -ret += (Topic.GROUP_METADATA_TOPIC_NAME -> Map(0 -> Seq(1), 1 -> Seq(1))) - -replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager]) - -zkUtils = EasyMock.createNiceMock(classOf[ZkUtils]) -// make two partitions of the group topic to make sure some partitions are not owned by the coordinator - EasyMock.expect(zkUtils.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME)).andReturn(Some(2)) -EasyMock.replay(zkUtils) - -timer = new MockTimer - -val config = KafkaConfig.fromProps(props) - -val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false) -val joinPurgatory = new Delaye
[3/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/main/scala/kafka/server/KafkaApis.scala -- diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index b0d354b..380685f 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -93,7 +93,7 @@ class KafkaApis(val requestChannel: RequestChannel, trace("Handling request:%s from connection %s;securityProtocol:%s,principal:%s". format(request.requestDesc(true), request.connectionId, request.securityProtocol, request.session.principal)) ApiKeys.forId(request.requestId) match { -case ApiKeys.PRODUCE => handleProducerRequest(request) +case ApiKeys.PRODUCE => handleProduceRequest(request) case ApiKeys.FETCH => handleFetchRequest(request) case ApiKeys.LIST_OFFSETS => handleListOffsetRequest(request) case ApiKeys.METADATA => handleTopicMetadataRequest(request) @@ -368,97 +368,107 @@ class KafkaApis(val requestChannel: RequestChannel, /** * Handle a produce request */ - def handleProducerRequest(request: RequestChannel.Request) { + def handleProduceRequest(request: RequestChannel.Request) { val produceRequest = request.body[ProduceRequest] val numBytesAppended = request.header.toStruct.sizeOf + request.bodyAndSize.size -if (produceRequest.isTransactional && !authorize(request.session, Write, new Resource(ProducerTransactionalId, produceRequest.transactionalId( - sendResponseMaybeThrottle(request, (throttleMs: Int) => produceRequest.getErrorResponse(throttleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception)) -else if (produceRequest.isIdempotent && !authorize(request.session, Write, Resource.ProducerIdResource)) - sendResponseMaybeThrottle(request, (throttleMs: Int) => produceRequest.getErrorResponse(throttleMs, Errors.PRODUCER_ID_AUTHORIZATION_FAILED.exception)) -else { - val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = -produceRequest.partitionRecordsOrFail.asScala.partition { case (tp, _) => - authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic) -} +def sendErrorResponse(error: Errors): Unit = { + sendResponseMaybeThrottle(request, requestThrottleMs => +produceRequest.getErrorResponse(requestThrottleMs, error.exception)) +} - val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition { -case (tp, _) => authorize(request.session, Write, new Resource(Topic, tp.topic)) +if (produceRequest.isTransactional) { + if (!authorize(request.session, Write, new Resource(TransactionalId, produceRequest.transactionalId))) { +sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) +return } + // Note that authorization to a transactionalId implies ProducerId authorization - // the callback for sending a produce response - def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { +} else if (produceRequest.isIdempotent && !authorize(request.session, IdempotentWrite, Resource.ClusterResource)) { + sendErrorResponse(Errors.CLUSTER_AUTHORIZATION_FAILED) + return +} + +val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = + produceRequest.partitionRecordsOrFail.asScala.partition { case (tp, _) => +authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic) + } -val mergedResponseStatus = responseStatus ++ -unauthorizedForWriteRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED)) ++ -nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION)) +val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition { + case (tp, _) => authorize(request.session, Write, new Resource(Topic, tp.topic)) +} -var errorInResponse = false +// the callback for sending a produce response +def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { -mergedResponseStatus.foreach { case (topicPartition, status) => - if (status.error != Errors.NONE) { -errorInResponse = true -debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - request.header.correlationId, - request.header.clientId, - topicPartition, - status.error.exceptionName)) - } + val mergedResponseStatus = responseSt
[4/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth
http://git-wip-us.apache.org/repos/asf/kafka/blob/9a21bf20/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java -- diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index cc30f4d..719efe9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.Cluster; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.MetricConfig; @@ -231,20 +232,8 @@ public class SenderTest { int maxRetries = 1; Metrics m = new Metrics(); try { -Sender sender = new Sender(client, -metadata, -this.accumulator, -false, -MAX_REQUEST_SIZE, -ACKS_ALL, -maxRetries, -m, -time, -REQUEST_TIMEOUT, -50, -null, -apiVersions -); +Sender sender = new Sender(client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, +maxRetries, m, time, REQUEST_TIMEOUT, 50, null, apiVersions); // do a successful retry Future future = accumulator.append(tp0, 0L, "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); // connect @@ -290,20 +279,8 @@ public class SenderTest { int maxRetries = 1; Metrics m = new Metrics(); try { -Sender sender = new Sender(client, -metadata, -this.accumulator, -true, -MAX_REQUEST_SIZE, -ACKS_ALL, -maxRetries, -m, -time, -REQUEST_TIMEOUT, -50, -null, -apiVersions -); +Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, +m, time, REQUEST_TIMEOUT, 50, null, apiVersions); // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 Cluster cluster1 = TestUtils.clusterWith(2, "test", 2); metadata.update(cluster1, Collections.emptySet(), time.milliseconds()); @@ -375,21 +352,63 @@ public class SenderTest { } @Test -public void testInitPidRequest() throws Exception { +public void testInitProducerIdRequest() throws Exception { final long producerId = 343434L; TransactionManager transactionManager = new TransactionManager(); setupWithTransactionState(transactionManager); client.setNode(new Node(1, "localhost", 33343)); +prepareAndReceiveInitProducerId(producerId, Errors.NONE); +assertTrue(transactionManager.hasProducerId()); +assertEquals(producerId, transactionManager.producerIdAndEpoch().producerId); +assertEquals((short) 0, transactionManager.producerIdAndEpoch().epoch); +} + +@Test +public void testClusterAuthorizationExceptionInInitProducerIdRequest() throws Exception { +final long producerId = 343434L; +TransactionManager transactionManager = new TransactionManager(); +setupWithTransactionState(transactionManager); +client.setNode(new Node(1, "localhost", 33343)); +prepareAndReceiveInitProducerId(producerId, Errors.CLUSTER_AUTHORIZATION_FAILED); +assertFalse(transactionManager.hasProducerId()); +assertTrue(transactionManager.isInErrorState()); +assertTrue(transactionManager.lastError() instanceof ClusterAuthorizationException); + +// cluster authorization is a fatal error for the producer +assertSendFailure(ClusterAuthorizationException.class); +} + +@Test +public void testClusterAuthorizationExceptionInProduceRequest() throws Exception { +final long producerId = 343434L; +TransactionManager transactionManager = new TransactionManager(); +setupWithTransactionState(transactionManager); + +client.setNode(new Node(1, "localhost", 33343)); +prepareAndReceiveInitProducerId(producerId, Errors.NONE); +assertTrue(transactionManager.hasProduc
kafka git commit: MINOR: GroupCoordinator can append with group lock
Repository: kafka Updated Branches: refs/heads/trunk c5d44af77 -> fdcee8b8b MINOR: GroupCoordinator can append with group lock Author: Jason Gustafson Reviewers: Guozhang Wang Closes #3133 from hachikuji/minor-replica-manager-append-refactor Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/fdcee8b8 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/fdcee8b8 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/fdcee8b8 Branch: refs/heads/trunk Commit: fdcee8b8b3c027cdc1b13031fa19fcfc7de5609f Parents: c5d44af Author: Jason Gustafson Authored: Wed May 24 21:00:44 2017 -0700 Committer: Jason Gustafson Committed: Wed May 24 21:00:44 2017 -0700 -- .../coordinator/group/GroupCoordinator.scala| 25 +-- .../kafka/coordinator/group/GroupMetadata.scala | 2 +- .../group/GroupMetadataManager.scala| 39 ++-- .../transaction/TransactionStateManager.scala | 2 +- .../scala/kafka/server/DelayedProduce.scala | 28 +-- .../scala/kafka/server/ReplicaManager.scala | 10 +- .../group/GroupCoordinatorTest.scala| 7 +- .../group/GroupMetadataManagerTest.scala| 222 +++ 8 files changed, 218 insertions(+), 117 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/fdcee8b8/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala -- diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index 36e3c63..7c1e002 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -245,8 +245,6 @@ class GroupCoordinator(val brokerId: Int, memberId: String, groupAssignment: Map[String, Array[Byte]], responseCallback: SyncCallback) { -var delayedGroupStore: Option[DelayedStore] = None - group synchronized { if (!group.has(memberId)) { responseCallback(Array.empty, Errors.UNKNOWN_MEMBER_ID) @@ -271,7 +269,7 @@ class GroupCoordinator(val brokerId: Int, val missing = group.allMembers -- groupAssignment.keySet val assignment = groupAssignment ++ missing.map(_ -> Array.empty[Byte]).toMap - delayedGroupStore = groupManager.prepareStoreGroup(group, assignment, (error: Errors) => { + groupManager.storeGroup(group, assignment, (error: Errors) => { group synchronized { // another member may have joined the group while we were awaiting this callback, // so we must ensure we are still in the AwaitingSync state and the same generation @@ -297,11 +295,6 @@ class GroupCoordinator(val brokerId: Int, } } } - -// store the group metadata without holding the group lock to avoid the potential -// for deadlock if the callback is invoked holding other locks (e.g. the replica -// state change lock) -delayedGroupStore.foreach(groupManager.store) } def handleLeaveGroup(groupId: String, memberId: String, responseCallback: Errors => Unit) { @@ -452,15 +445,13 @@ class GroupCoordinator(val brokerId: Int, producerEpoch: Short, offsetMetadata: immutable.Map[TopicPartition, OffsetAndMetadata], responseCallback: immutable.Map[TopicPartition, Errors] => Unit) { -var delayedOffsetStore: Option[DelayedStore] = None group synchronized { if (group.is(Dead)) { responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_MEMBER_ID)) } else if ((generationId < 0 && group.is(Empty)) || (producerId != NO_PRODUCER_ID)) { // the group is only using Kafka to store offsets // Also, for transactional offset commits we don't need to validate group membership and the generation. -delayedOffsetStore = groupManager.prepareStoreOffsets(group, memberId, generationId, - offsetMetadata, responseCallback, producerId, producerEpoch) +groupManager.storeOffsets(group, memberId, offsetMetadata, responseCallback, producerId, producerEpoch) } else if (group.is(AwaitingSync)) { responseCallback(offsetMetadata.mapValues(_ => Errors.REBALANCE_IN_PROGRESS)) } else if (!group.has(memberId)) { @@ -470,13 +461,9 @@ class GroupCoordinator(val brokerId: Int, } else { val member = group.get(memberId) completeAndScheduleNextHeartbeatExpiration(group, member) -delayedOffsetStore = groupManager.prepareStoreOffsets(group, memberId, generationId, -
kafka git commit: MINOR: GroupCoordinator can append with group lock
Repository: kafka Updated Branches: refs/heads/0.11.0 a8dbce47f -> 8dfc098ee MINOR: GroupCoordinator can append with group lock Author: Jason Gustafson Reviewers: Guozhang Wang Closes #3133 from hachikuji/minor-replica-manager-append-refactor (cherry picked from commit fdcee8b8b3c027cdc1b13031fa19fcfc7de5609f) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8dfc098e Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8dfc098e Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8dfc098e Branch: refs/heads/0.11.0 Commit: 8dfc098ee5c5024a580dab50281e19fec2dd6cbf Parents: a8dbce4 Author: Jason Gustafson Authored: Wed May 24 21:00:44 2017 -0700 Committer: Jason Gustafson Committed: Wed May 24 21:02:42 2017 -0700 -- .../coordinator/group/GroupCoordinator.scala| 25 +-- .../kafka/coordinator/group/GroupMetadata.scala | 2 +- .../group/GroupMetadataManager.scala| 39 ++-- .../transaction/TransactionStateManager.scala | 2 +- .../scala/kafka/server/DelayedProduce.scala | 28 +-- .../scala/kafka/server/ReplicaManager.scala | 10 +- .../group/GroupCoordinatorTest.scala| 7 +- .../group/GroupMetadataManagerTest.scala| 222 +++ 8 files changed, 218 insertions(+), 117 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/8dfc098e/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala -- diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index 36e3c63..7c1e002 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -245,8 +245,6 @@ class GroupCoordinator(val brokerId: Int, memberId: String, groupAssignment: Map[String, Array[Byte]], responseCallback: SyncCallback) { -var delayedGroupStore: Option[DelayedStore] = None - group synchronized { if (!group.has(memberId)) { responseCallback(Array.empty, Errors.UNKNOWN_MEMBER_ID) @@ -271,7 +269,7 @@ class GroupCoordinator(val brokerId: Int, val missing = group.allMembers -- groupAssignment.keySet val assignment = groupAssignment ++ missing.map(_ -> Array.empty[Byte]).toMap - delayedGroupStore = groupManager.prepareStoreGroup(group, assignment, (error: Errors) => { + groupManager.storeGroup(group, assignment, (error: Errors) => { group synchronized { // another member may have joined the group while we were awaiting this callback, // so we must ensure we are still in the AwaitingSync state and the same generation @@ -297,11 +295,6 @@ class GroupCoordinator(val brokerId: Int, } } } - -// store the group metadata without holding the group lock to avoid the potential -// for deadlock if the callback is invoked holding other locks (e.g. the replica -// state change lock) -delayedGroupStore.foreach(groupManager.store) } def handleLeaveGroup(groupId: String, memberId: String, responseCallback: Errors => Unit) { @@ -452,15 +445,13 @@ class GroupCoordinator(val brokerId: Int, producerEpoch: Short, offsetMetadata: immutable.Map[TopicPartition, OffsetAndMetadata], responseCallback: immutable.Map[TopicPartition, Errors] => Unit) { -var delayedOffsetStore: Option[DelayedStore] = None group synchronized { if (group.is(Dead)) { responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_MEMBER_ID)) } else if ((generationId < 0 && group.is(Empty)) || (producerId != NO_PRODUCER_ID)) { // the group is only using Kafka to store offsets // Also, for transactional offset commits we don't need to validate group membership and the generation. -delayedOffsetStore = groupManager.prepareStoreOffsets(group, memberId, generationId, - offsetMetadata, responseCallback, producerId, producerEpoch) +groupManager.storeOffsets(group, memberId, offsetMetadata, responseCallback, producerId, producerEpoch) } else if (group.is(AwaitingSync)) { responseCallback(offsetMetadata.mapValues(_ => Errors.REBALANCE_IN_PROGRESS)) } else if (!group.has(memberId)) { @@ -470,13 +461,9 @@ class GroupCoordinator(val brokerId: Int, } else { val member = group.get(memberId) completeAndScheduleNextHeartbeatExpiration(group, member
kafka git commit: KAFKA-5147; Add missing synchronization to TransactionManager
Repository: kafka Updated Branches: refs/heads/trunk 73ca0d215 -> 02c0c3b01 KAFKA-5147; Add missing synchronization to TransactionManager The basic idea is that exactly three collections, ie. `pendingRequests`, `newPartitionsToBeAddedToTransaction`, and `partitionsInTransaction` are accessed from the context of application threads. The first two are modified from the application threads, and the last is read from those threads. So to make the `TransactionManager` truly thread safe, we have to ensure that all accesses to these three members are done in a synchronized block. I inspected the code, and I believe this patch puts the synchronization in all the correct places. Author: Apurva Mehta Reviewers: Jason Gustafson Closes #3132 from apurvam/KAFKA-5147-transaction-manager-synchronization-fixes Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/02c0c3b0 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/02c0c3b0 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/02c0c3b0 Branch: refs/heads/trunk Commit: 02c0c3b01730bdbff8a09d355c1b017715c7ce10 Parents: 73ca0d2 Author: Apurva Mehta Authored: Thu May 25 16:17:18 2017 -0700 Committer: Jason Gustafson Committed: Thu May 25 16:21:04 2017 -0700 -- .../producer/internals/TransactionManager.java | 34 +++- .../internals/TransactionalRequestResult.java | 2 +- 2 files changed, 20 insertions(+), 16 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/02c0c3b0/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index d674697..e5c6ec2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -240,23 +240,23 @@ public class TransactionManager { return transactionalId != null; } -public boolean isCompletingTransaction() { +public synchronized boolean isCompletingTransaction() { return currentState == State.COMMITTING_TRANSACTION || currentState == State.ABORTING_TRANSACTION; } -public boolean isInTransaction() { -return currentState == State.IN_TRANSACTION || isCompletingTransaction(); +public synchronized boolean isInErrorState() { +return currentState == State.ABORTABLE_ERROR || currentState == State.FATAL_ERROR; } -public boolean isInErrorState() { -return currentState == State.ABORTABLE_ERROR || currentState == State.FATAL_ERROR; +synchronized boolean isInTransaction() { +return currentState == State.IN_TRANSACTION || isCompletingTransaction(); } -public synchronized void transitionToAbortableError(RuntimeException exception) { +synchronized void transitionToAbortableError(RuntimeException exception) { transitionTo(State.ABORTABLE_ERROR, exception); } -public synchronized void transitionToFatalError(RuntimeException exception) { +synchronized void transitionToFatalError(RuntimeException exception) { transitionTo(State.FATAL_ERROR, exception); } @@ -383,17 +383,17 @@ public class TransactionManager { } // visible for testing -boolean transactionContainsPartition(TopicPartition topicPartition) { +synchronized boolean transactionContainsPartition(TopicPartition topicPartition) { return isInTransaction() && partitionsInTransaction.contains(topicPartition); } // visible for testing -boolean hasPendingOffsetCommits() { +synchronized boolean hasPendingOffsetCommits() { return isInTransaction() && !pendingTxnOffsetCommits.isEmpty(); } // visible for testing -boolean isReadyForTransaction() { +synchronized boolean isReadyForTransaction() { return isTransactional() && currentState == State.READY; } @@ -443,7 +443,7 @@ public class TransactionManager { return false; } -private void lookupCoordinator(FindCoordinatorRequest.CoordinatorType type, String coordinatorKey) { +private synchronized void lookupCoordinator(FindCoordinatorRequest.CoordinatorType type, String coordinatorKey) { switch (type) { case GROUP: consumerGroupCoordinator = null; @@ -459,7 +459,7 @@ public class TransactionManager { pendingRequests.add(new FindCoordinatorHandler(builder)); } -private void completeTransaction() { +private synchronized void completeTran
kafka git commit: KAFKA-5147; Add missing synchronization to TransactionManager
Repository: kafka Updated Branches: refs/heads/0.11.0 7dde914aa -> c62793eab KAFKA-5147; Add missing synchronization to TransactionManager The basic idea is that exactly three collections, ie. `pendingRequests`, `newPartitionsToBeAddedToTransaction`, and `partitionsInTransaction` are accessed from the context of application threads. The first two are modified from the application threads, and the last is read from those threads. So to make the `TransactionManager` truly thread safe, we have to ensure that all accesses to these three members are done in a synchronized block. I inspected the code, and I believe this patch puts the synchronization in all the correct places. Author: Apurva Mehta Reviewers: Jason Gustafson Closes #3132 from apurvam/KAFKA-5147-transaction-manager-synchronization-fixes (cherry picked from commit 02c0c3b01730bdbff8a09d355c1b017715c7ce10) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/c62793ea Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/c62793ea Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/c62793ea Branch: refs/heads/0.11.0 Commit: c62793eab35d166c33c018d931fa4a40024a2524 Parents: 7dde914 Author: Apurva Mehta Authored: Thu May 25 16:17:18 2017 -0700 Committer: Jason Gustafson Committed: Thu May 25 16:25:46 2017 -0700 -- .../producer/internals/TransactionManager.java | 34 +++- .../internals/TransactionalRequestResult.java | 2 +- 2 files changed, 20 insertions(+), 16 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/c62793ea/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index d674697..e5c6ec2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -240,23 +240,23 @@ public class TransactionManager { return transactionalId != null; } -public boolean isCompletingTransaction() { +public synchronized boolean isCompletingTransaction() { return currentState == State.COMMITTING_TRANSACTION || currentState == State.ABORTING_TRANSACTION; } -public boolean isInTransaction() { -return currentState == State.IN_TRANSACTION || isCompletingTransaction(); +public synchronized boolean isInErrorState() { +return currentState == State.ABORTABLE_ERROR || currentState == State.FATAL_ERROR; } -public boolean isInErrorState() { -return currentState == State.ABORTABLE_ERROR || currentState == State.FATAL_ERROR; +synchronized boolean isInTransaction() { +return currentState == State.IN_TRANSACTION || isCompletingTransaction(); } -public synchronized void transitionToAbortableError(RuntimeException exception) { +synchronized void transitionToAbortableError(RuntimeException exception) { transitionTo(State.ABORTABLE_ERROR, exception); } -public synchronized void transitionToFatalError(RuntimeException exception) { +synchronized void transitionToFatalError(RuntimeException exception) { transitionTo(State.FATAL_ERROR, exception); } @@ -383,17 +383,17 @@ public class TransactionManager { } // visible for testing -boolean transactionContainsPartition(TopicPartition topicPartition) { +synchronized boolean transactionContainsPartition(TopicPartition topicPartition) { return isInTransaction() && partitionsInTransaction.contains(topicPartition); } // visible for testing -boolean hasPendingOffsetCommits() { +synchronized boolean hasPendingOffsetCommits() { return isInTransaction() && !pendingTxnOffsetCommits.isEmpty(); } // visible for testing -boolean isReadyForTransaction() { +synchronized boolean isReadyForTransaction() { return isTransactional() && currentState == State.READY; } @@ -443,7 +443,7 @@ public class TransactionManager { return false; } -private void lookupCoordinator(FindCoordinatorRequest.CoordinatorType type, String coordinatorKey) { +private synchronized void lookupCoordinator(FindCoordinatorRequest.CoordinatorType type, String coordinatorKey) { switch (type) { case GROUP: consumerGroupCoordinator = null; @@ -459,7 +459,7 @@ public class TransactionManager { pendingRequests.add(new FindCoordinatorHandler(b
kafka git commit: KAFKA-5128; Check inter broker version in transactional methods
Repository: kafka Updated Branches: refs/heads/trunk faa1803aa -> 7892b4e6c KAFKA-5128; Check inter broker version in transactional methods Add check in `KafkaApis` that the inter broker protocol version is at least `KAFKA_0_11_0_IV0`, i.e., supporting transactions Author: Damian Guy Reviewers: Ismael Juma , Jason Gustafson Closes #3103 from dguy/kafka-5128 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/7892b4e6 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/7892b4e6 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/7892b4e6 Branch: refs/heads/trunk Commit: 7892b4e6c7c32be09d78a8bbbeeaa823d3197aaa Parents: faa1803 Author: Damian Guy Authored: Fri May 26 09:52:47 2017 -0700 Committer: Jason Gustafson Committed: Fri May 26 09:52:47 2017 -0700 -- .../apache/kafka/common/protocol/ApiKeys.java | 71 +++- .../common/requests/ApiVersionsResponse.java| 18 +-- .../clients/consumer/internals/FetcherTest.java | 2 +- .../clients/producer/internals/SenderTest.java | 2 +- .../requests/ApiVersionsResponseTest.java | 68 .../transaction/TransactionStateManager.scala | 1 - .../src/main/scala/kafka/server/KafkaApis.scala | 14 ++- .../scala/unit/kafka/server/KafkaApisTest.scala | 111 +++ 8 files changed, 246 insertions(+), 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/7892b4e6/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 36f6403..721a610 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -19,6 +19,7 @@ package org.apache.kafka.common.protocol; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.SchemaException; import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.RecordBatch; import java.nio.ByteBuffer; @@ -26,25 +27,25 @@ import java.nio.ByteBuffer; * Identifiers for all the Kafka APIs */ public enum ApiKeys { -PRODUCE(0, "Produce", false), -FETCH(1, "Fetch", false), -LIST_OFFSETS(2, "Offsets", false), -METADATA(3, "Metadata", false), +PRODUCE(0, "Produce"), +FETCH(1, "Fetch"), +LIST_OFFSETS(2, "Offsets"), +METADATA(3, "Metadata"), LEADER_AND_ISR(4, "LeaderAndIsr", true), STOP_REPLICA(5, "StopReplica", true), UPDATE_METADATA_KEY(6, "UpdateMetadata", true), CONTROLLED_SHUTDOWN_KEY(7, "ControlledShutdown", true), -OFFSET_COMMIT(8, "OffsetCommit", false), -OFFSET_FETCH(9, "OffsetFetch", false), -FIND_COORDINATOR(10, "FindCoordinator", false), -JOIN_GROUP(11, "JoinGroup", false), -HEARTBEAT(12, "Heartbeat", false), -LEAVE_GROUP(13, "LeaveGroup", false), -SYNC_GROUP(14, "SyncGroup", false), -DESCRIBE_GROUPS(15, "DescribeGroups", false), -LIST_GROUPS(16, "ListGroups", false), -SASL_HANDSHAKE(17, "SaslHandshake", false), -API_VERSIONS(18, "ApiVersions", false) { +OFFSET_COMMIT(8, "OffsetCommit"), +OFFSET_FETCH(9, "OffsetFetch"), +FIND_COORDINATOR(10, "FindCoordinator"), +JOIN_GROUP(11, "JoinGroup"), +HEARTBEAT(12, "Heartbeat"), +LEAVE_GROUP(13, "LeaveGroup"), +SYNC_GROUP(14, "SyncGroup"), +DESCRIBE_GROUPS(15, "DescribeGroups"), +LIST_GROUPS(16, "ListGroups"), +SASL_HANDSHAKE(17, "SaslHandshake"), +API_VERSIONS(18, "ApiVersions") { @Override public Struct parseResponse(short version, ByteBuffer buffer) { // Fallback to version 0 for ApiVersions response. If a client sends an ApiVersionsRequest @@ -53,21 +54,21 @@ public enum ApiKeys { return parseResponse(version, buffer, (short) 0); } }, -CREATE_TOPICS(19, "CreateTopics", false), -DELETE_TOPICS(20, "DeleteTopics", false), -DELETE_RECORDS(21, "DeleteRecords", false), -INIT_PRODUCER_ID(22, "InitProducerId", false), +CREATE_TOPICS(19, "CreateTopics"), +DELETE_TOPICS(20, "DeleteTopics"), +DELETE_RECORDS(21, "DeleteRecords"), +INIT_PRODUCER_ID(22, "InitProducerId"), OFFSET_FOR_LEADER_EPOCH(23, "OffsetForLeaderEpoch", true), -ADD_PARTITIONS_TO_TXN(24, "AddPartitionsToTxn", false), -ADD_OFFSETS_TO_TXN(25, "AddOffsetsToTxn", false), -END_TXN(26, "EndTxn", false), -WRITE_TXN_MARKERS(27, "WriteTxnMarkers", true), -TXN_OFFSET_COMMIT(28, "TxnOffsetCommit", false), -DESCRIBE_ACLS(29, "DescribeAcls", false), -CREATE_ACLS(30, "CreateAcls", false), -DELE
kafka git commit: KAFKA-5128; Check inter broker version in transactional methods
Repository: kafka Updated Branches: refs/heads/0.11.0 1b15adde1 -> 012332042 KAFKA-5128; Check inter broker version in transactional methods Add check in `KafkaApis` that the inter broker protocol version is at least `KAFKA_0_11_0_IV0`, i.e., supporting transactions Author: Damian Guy Reviewers: Ismael Juma , Jason Gustafson Closes #3103 from dguy/kafka-5128 (cherry picked from commit 7892b4e6c7c32be09d78a8bbbeeaa823d3197aaa) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/01233204 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/01233204 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/01233204 Branch: refs/heads/0.11.0 Commit: 0123320426341fe50d4124f0ef398d7f5aaee909 Parents: 1b15add Author: Damian Guy Authored: Fri May 26 09:52:47 2017 -0700 Committer: Jason Gustafson Committed: Fri May 26 09:54:21 2017 -0700 -- .../apache/kafka/common/protocol/ApiKeys.java | 71 +++- .../common/requests/ApiVersionsResponse.java| 18 +-- .../clients/consumer/internals/FetcherTest.java | 2 +- .../clients/producer/internals/SenderTest.java | 2 +- .../requests/ApiVersionsResponseTest.java | 68 .../transaction/TransactionStateManager.scala | 1 - .../src/main/scala/kafka/server/KafkaApis.scala | 14 ++- .../scala/unit/kafka/server/KafkaApisTest.scala | 111 +++ 8 files changed, 246 insertions(+), 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/01233204/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 36f6403..721a610 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -19,6 +19,7 @@ package org.apache.kafka.common.protocol; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.SchemaException; import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.RecordBatch; import java.nio.ByteBuffer; @@ -26,25 +27,25 @@ import java.nio.ByteBuffer; * Identifiers for all the Kafka APIs */ public enum ApiKeys { -PRODUCE(0, "Produce", false), -FETCH(1, "Fetch", false), -LIST_OFFSETS(2, "Offsets", false), -METADATA(3, "Metadata", false), +PRODUCE(0, "Produce"), +FETCH(1, "Fetch"), +LIST_OFFSETS(2, "Offsets"), +METADATA(3, "Metadata"), LEADER_AND_ISR(4, "LeaderAndIsr", true), STOP_REPLICA(5, "StopReplica", true), UPDATE_METADATA_KEY(6, "UpdateMetadata", true), CONTROLLED_SHUTDOWN_KEY(7, "ControlledShutdown", true), -OFFSET_COMMIT(8, "OffsetCommit", false), -OFFSET_FETCH(9, "OffsetFetch", false), -FIND_COORDINATOR(10, "FindCoordinator", false), -JOIN_GROUP(11, "JoinGroup", false), -HEARTBEAT(12, "Heartbeat", false), -LEAVE_GROUP(13, "LeaveGroup", false), -SYNC_GROUP(14, "SyncGroup", false), -DESCRIBE_GROUPS(15, "DescribeGroups", false), -LIST_GROUPS(16, "ListGroups", false), -SASL_HANDSHAKE(17, "SaslHandshake", false), -API_VERSIONS(18, "ApiVersions", false) { +OFFSET_COMMIT(8, "OffsetCommit"), +OFFSET_FETCH(9, "OffsetFetch"), +FIND_COORDINATOR(10, "FindCoordinator"), +JOIN_GROUP(11, "JoinGroup"), +HEARTBEAT(12, "Heartbeat"), +LEAVE_GROUP(13, "LeaveGroup"), +SYNC_GROUP(14, "SyncGroup"), +DESCRIBE_GROUPS(15, "DescribeGroups"), +LIST_GROUPS(16, "ListGroups"), +SASL_HANDSHAKE(17, "SaslHandshake"), +API_VERSIONS(18, "ApiVersions") { @Override public Struct parseResponse(short version, ByteBuffer buffer) { // Fallback to version 0 for ApiVersions response. If a client sends an ApiVersionsRequest @@ -53,21 +54,21 @@ public enum ApiKeys { return parseResponse(version, buffer, (short) 0); } }, -CREATE_TOPICS(19, "CreateTopics", false), -DELETE_TOPICS(20, "DeleteTopics", false), -DELETE_RECORDS(21, "DeleteRecords", false), -INIT_PRODUCER_ID(22, "InitProducerId", false), +CREATE_TOPICS(19, "CreateTopics"), +DELETE_TOPICS(20, "DeleteTopics"), +DELETE_RECORDS(21, "DeleteRecords"), +INIT_PRODUCER_ID(22, "InitProducerId"), OFFSET_FOR_LEADER_EPOCH(23, "OffsetForLeaderEpoch", true), -ADD_PARTITIONS_TO_TXN(24, "AddPartitionsToTxn", false), -ADD_OFFSETS_TO_TXN(25, "AddOffsetsToTxn", false), -END_TXN(26, "EndTxn", false), -WRITE_TXN_MARKERS(27, "WriteTxnMarkers", true), -TXN_OFFSET_COMMIT(28, "TxnOffsetCommit", fal
kafka git commit: KAFKA-5316; LogCleaner should account for larger record sets after cleaning
Repository: kafka Updated Branches: refs/heads/trunk b50387eb7 -> dfa3c8a92 KAFKA-5316; LogCleaner should account for larger record sets after cleaning Author: Jason Gustafson Reviewers: Ismael Juma , Jun Rao Closes #3142 from hachikuji/KAFKA-5316 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/dfa3c8a9 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/dfa3c8a9 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/dfa3c8a9 Branch: refs/heads/trunk Commit: dfa3c8a9258cab95e12c72669f250bb99683 Parents: b50387e Author: Jason Gustafson Authored: Sun May 28 09:57:59 2017 -0700 Committer: Jason Gustafson Committed: Sun May 28 09:57:59 2017 -0700 -- checkstyle/checkstyle.xml | 2 +- .../producer/internals/ProducerBatch.java | 57 +++-- .../record/AbstractLegacyRecordBatch.java | 6 + .../kafka/common/record/DefaultRecordBatch.java | 14 +-- .../kafka/common/record/FileLogInputStream.java | 7 -- .../apache/kafka/common/record/FileRecords.java | 21 +--- .../kafka/common/record/MemoryRecords.java | 110 ++-- .../common/record/MemoryRecordsBuilder.java | 124 --- .../kafka/common/record/MutableRecordBatch.java | 9 ++ .../common/utils/ByteBufferOutputStream.java| 45 ++- .../org/apache/kafka/clients/MockClient.java| 15 ++- .../internals/RecordAccumulatorTest.java| 8 +- .../clients/producer/internals/SenderTest.java | 61 ++--- .../common/record/FileLogInputStreamTest.java | 2 +- .../kafka/common/record/MemoryRecordsTest.java | 70 +-- .../utils/ByteBufferOutputStreamTest.java | 101 +++ core/src/main/scala/kafka/log/Log.scala | 2 +- core/src/main/scala/kafka/log/LogCleaner.scala | 11 +- core/src/main/scala/kafka/log/LogSegment.scala | 14 +-- .../scala/kafka/tools/DumpLogSegments.scala | 4 +- .../scala/unit/kafka/log/LogSegmentTest.scala | 10 +- .../src/test/scala/unit/kafka/log/LogTest.scala | 9 +- 22 files changed, 490 insertions(+), 212 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/dfa3c8a9/checkstyle/checkstyle.xml -- diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index 743c68d..ccab85c 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -105,7 +105,7 @@ - + http://git-wip-us.apache.org/repos/asf/kafka/blob/dfa3c8a9/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java index df79707..974e230 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java @@ -16,9 +16,6 @@ */ package org.apache.kafka.clients.producer.internals; -import java.util.ArrayDeque; -import java.util.Deque; -import java.util.Iterator; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; @@ -27,23 +24,24 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.CompressionRatioEstimator; -import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MutableRecordBatch; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; -import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.MemoryRecordsBuilder; -import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.requests.ProduceResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Deque; +import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import static org.apache.kafka.common.record.RecordBatch.NO_TIMESTAMP; @@ -119,9 +117,9 @@ public final class ProducerBatch { }
kafka git commit: KAFKA-5316; LogCleaner should account for larger record sets after cleaning
Repository: kafka Updated Branches: refs/heads/0.11.0 d90899bb9 -> 569248f6f KAFKA-5316; LogCleaner should account for larger record sets after cleaning Author: Jason Gustafson Reviewers: Ismael Juma , Jun Rao Closes #3142 from hachikuji/KAFKA-5316 (cherry picked from commit dfa3c8a9258cab95e12c72669f250bb99683) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/569248f6 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/569248f6 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/569248f6 Branch: refs/heads/0.11.0 Commit: 569248f6fb7515609a2bace98d5dc35721f060f8 Parents: d90899b Author: Jason Gustafson Authored: Sun May 28 09:57:59 2017 -0700 Committer: Jason Gustafson Committed: Sun May 28 10:00:07 2017 -0700 -- checkstyle/checkstyle.xml | 2 +- .../producer/internals/ProducerBatch.java | 57 +++-- .../record/AbstractLegacyRecordBatch.java | 6 + .../kafka/common/record/DefaultRecordBatch.java | 14 +-- .../kafka/common/record/FileLogInputStream.java | 7 -- .../apache/kafka/common/record/FileRecords.java | 21 +--- .../kafka/common/record/MemoryRecords.java | 110 ++-- .../common/record/MemoryRecordsBuilder.java | 124 --- .../kafka/common/record/MutableRecordBatch.java | 9 ++ .../common/utils/ByteBufferOutputStream.java| 45 ++- .../org/apache/kafka/clients/MockClient.java| 15 ++- .../internals/RecordAccumulatorTest.java| 8 +- .../clients/producer/internals/SenderTest.java | 61 ++--- .../common/record/FileLogInputStreamTest.java | 2 +- .../kafka/common/record/MemoryRecordsTest.java | 70 +-- .../utils/ByteBufferOutputStreamTest.java | 101 +++ core/src/main/scala/kafka/log/Log.scala | 2 +- core/src/main/scala/kafka/log/LogCleaner.scala | 11 +- core/src/main/scala/kafka/log/LogSegment.scala | 14 +-- .../scala/kafka/tools/DumpLogSegments.scala | 4 +- .../scala/unit/kafka/log/LogSegmentTest.scala | 10 +- .../src/test/scala/unit/kafka/log/LogTest.scala | 9 +- 22 files changed, 490 insertions(+), 212 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/569248f6/checkstyle/checkstyle.xml -- diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index 743c68d..ccab85c 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -105,7 +105,7 @@ - + http://git-wip-us.apache.org/repos/asf/kafka/blob/569248f6/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java index df79707..974e230 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java @@ -16,9 +16,6 @@ */ package org.apache.kafka.clients.producer.internals; -import java.util.ArrayDeque; -import java.util.Deque; -import java.util.Iterator; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; @@ -27,23 +24,24 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.CompressionRatioEstimator; -import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MutableRecordBatch; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; -import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.MemoryRecordsBuilder; -import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.requests.ProduceResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Deque; +import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import static org.apache.kafka.commo
kafka git commit: KAFKA-5260; Producer should not send AbortTxn unless transaction has actually begun
Repository: kafka Updated Branches: refs/heads/trunk dc520d275 -> a8794d8a5 KAFKA-5260; Producer should not send AbortTxn unless transaction has actually begun Keep track of when a transaction has begun by setting a flag, `transactionStarted` when a successfull `AddPartitionsToTxnResponse` or `AddOffsetsToTxnResponse` had been received. If an `AbortTxnRequest` about to be sent and `transactionStarted` is false, don't send the request and transition the state to `READY` Author: Damian Guy Reviewers: Apurva Mehta , Guozhang Wang , Jason Gustafson Closes #3126 from dguy/kafka-5260 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/a8794d8a Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/a8794d8a Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/a8794d8a Branch: refs/heads/trunk Commit: a8794d8a5d18bb4eaafceac1ef675243af945862 Parents: dc520d2 Author: Damian Guy Authored: Mon May 29 22:52:59 2017 -0700 Committer: Jason Gustafson Committed: Mon May 29 22:52:59 2017 -0700 -- .../producer/internals/TransactionManager.java | 13 .../internals/TransactionManagerTest.java | 64 .../src/main/scala/kafka/server/KafkaApis.scala | 1 - .../kafka/api/AuthorizerIntegrationTest.scala | 25 +++- 4 files changed, 99 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/a8794d8a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index e5c6ec2..ec7ced2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -79,6 +79,7 @@ public class TransactionManager { private volatile State currentState = State.UNINITIALIZED; private volatile RuntimeException lastError = null; private volatile ProducerIdAndEpoch producerIdAndEpoch; +private volatile boolean transactionStarted = false; private enum State { UNINITIALIZED, @@ -343,7 +344,16 @@ public class TransactionManager { return null; } +if (nextRequestHandler != null && nextRequestHandler.isEndTxn() && !transactionStarted) { +((EndTxnHandler) nextRequestHandler).result.done(); +if (currentState != State.FATAL_ERROR) { +completeTransaction(); +} +return pendingRequests.poll(); +} + return nextRequestHandler; + } synchronized void retry(TxnRequestHandler request) { @@ -462,6 +472,7 @@ public class TransactionManager { private synchronized void completeTransaction() { transitionTo(State.READY); lastError = null; +transactionStarted = false; partitionsInTransaction.clear(); } @@ -686,6 +697,7 @@ public class TransactionManager { } else { partitionsInTransaction.addAll(pendingPartitionsToBeAddedToTransaction); pendingPartitionsToBeAddedToTransaction.clear(); +transactionStarted = true; result.done(); } } @@ -831,6 +843,7 @@ public class TransactionManager { if (error == Errors.NONE) { // note the result is not completed until the TxnOffsetCommit returns pendingRequests.add(txnOffsetCommitHandler(result, offsets, builder.consumerGroupId())); +transactionStarted = true; } else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR) { lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); reenqueue(); http://git-wip-us.apache.org/repos/asf/kafka/blob/a8794d8a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java -- diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index e9363d0..a1bd970 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -737,6 +737,70 @@ public class TransactionManagerTest { verifyAddPartitionsFailsWithPartitionLevelError
kafka git commit: KAFKA-5260; Producer should not send AbortTxn unless transaction has actually begun
Repository: kafka Updated Branches: refs/heads/0.11.0 a1da4c534 -> 9dd7db2f5 KAFKA-5260; Producer should not send AbortTxn unless transaction has actually begun Keep track of when a transaction has begun by setting a flag, `transactionStarted` when a successfull `AddPartitionsToTxnResponse` or `AddOffsetsToTxnResponse` had been received. If an `AbortTxnRequest` about to be sent and `transactionStarted` is false, don't send the request and transition the state to `READY` Author: Damian Guy Reviewers: Apurva Mehta , Guozhang Wang , Jason Gustafson Closes #3126 from dguy/kafka-5260 (cherry picked from commit a8794d8a5d18bb4eaafceac1ef675243af945862) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/9dd7db2f Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/9dd7db2f Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/9dd7db2f Branch: refs/heads/0.11.0 Commit: 9dd7db2f5fb4b6c14ce72d2e87eea917016bf02b Parents: a1da4c5 Author: Damian Guy Authored: Mon May 29 22:52:59 2017 -0700 Committer: Jason Gustafson Committed: Mon May 29 22:55:50 2017 -0700 -- .../producer/internals/TransactionManager.java | 13 .../internals/TransactionManagerTest.java | 64 .../src/main/scala/kafka/server/KafkaApis.scala | 1 - .../kafka/api/AuthorizerIntegrationTest.scala | 25 +++- 4 files changed, 99 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/9dd7db2f/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index e5c6ec2..ec7ced2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -79,6 +79,7 @@ public class TransactionManager { private volatile State currentState = State.UNINITIALIZED; private volatile RuntimeException lastError = null; private volatile ProducerIdAndEpoch producerIdAndEpoch; +private volatile boolean transactionStarted = false; private enum State { UNINITIALIZED, @@ -343,7 +344,16 @@ public class TransactionManager { return null; } +if (nextRequestHandler != null && nextRequestHandler.isEndTxn() && !transactionStarted) { +((EndTxnHandler) nextRequestHandler).result.done(); +if (currentState != State.FATAL_ERROR) { +completeTransaction(); +} +return pendingRequests.poll(); +} + return nextRequestHandler; + } synchronized void retry(TxnRequestHandler request) { @@ -462,6 +472,7 @@ public class TransactionManager { private synchronized void completeTransaction() { transitionTo(State.READY); lastError = null; +transactionStarted = false; partitionsInTransaction.clear(); } @@ -686,6 +697,7 @@ public class TransactionManager { } else { partitionsInTransaction.addAll(pendingPartitionsToBeAddedToTransaction); pendingPartitionsToBeAddedToTransaction.clear(); +transactionStarted = true; result.done(); } } @@ -831,6 +843,7 @@ public class TransactionManager { if (error == Errors.NONE) { // note the result is not completed until the TxnOffsetCommit returns pendingRequests.add(txnOffsetCommitHandler(result, offsets, builder.consumerGroupId())); +transactionStarted = true; } else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR) { lookupCoordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); reenqueue(); http://git-wip-us.apache.org/repos/asf/kafka/blob/9dd7db2f/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java -- diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index e9363d0..a1bd970 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -737,6 +7
kafka git commit: KAFKA-5316; Follow-up with ByteBufferOutputStream and other misc improvements
Repository: kafka Updated Branches: refs/heads/trunk 6b0349791 -> b3788d8dc KAFKA-5316; Follow-up with ByteBufferOutputStream and other misc improvements ByteBufferOutputStream improvements: * Document pitfalls * Improve efficiency when dealing with direct byte buffers * Improve handling of buffer expansion * Be consistent about using `limit` instead of `capacity` * Add constructors that allocate the internal buffer Other minor changes: * Fix log warning to specify correct Kafka version * Clean-ups Author: Ismael Juma Reviewers: Jason Gustafson Closes #3166 from ijuma/minor-kafka-5316-follow-ups Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b3788d8d Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b3788d8d Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b3788d8d Branch: refs/heads/trunk Commit: b3788d8dcbeee7a20f562e878c187a75bac11ff0 Parents: 6b03497 Author: Ismael Juma Authored: Tue May 30 12:53:32 2017 -0700 Committer: Jason Gustafson Committed: Tue May 30 12:53:32 2017 -0700 -- .../producer/internals/RecordAccumulator.java | 2 +- .../kafka/common/record/DefaultRecord.java | 19 - .../kafka/common/record/MemoryRecords.java | 8 +- .../common/record/MemoryRecordsBuilder.java | 7 +- .../common/utils/ByteBufferOutputStream.java| 85 +--- .../kafka/common/record/DefaultRecordTest.java | 18 +++-- .../utils/ByteBufferOutputStreamTest.java | 1 + 7 files changed, 77 insertions(+), 63 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/b3788d8d/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index e1f04a8..d3d1b82 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -531,7 +531,7 @@ public final class RecordAccumulator { */ public void deallocate(ProducerBatch batch) { incomplete.remove(batch); -// Only deallocate the batch if it is not a split batch because split batch are allocated aside the +// Only deallocate the batch if it is not a split batch because split batch are allocated outside the // buffer pool. if (!batch.isSplitBatch()) free.deallocate(batch.buffer(), batch.initialCapacity()); http://git-wip-us.apache.org/repos/asf/kafka/blob/b3788d8d/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java index 05b5bb2..e61bbc9 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java @@ -18,7 +18,6 @@ package org.apache.kafka.common.record; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.internals.RecordHeader; -import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.utils.ByteUtils; import org.apache.kafka.common.utils.Checksums; import org.apache.kafka.common.utils.Crc32C; @@ -230,24 +229,6 @@ public class DefaultRecord implements Record { return ByteUtils.sizeOfVarint(sizeInBytes) + sizeInBytes; } -/** - * Write the record to `out` and return its size. - */ -public static int writeTo(ByteBuffer out, - int offsetDelta, - long timestampDelta, - ByteBuffer key, - ByteBuffer value, - Header[] headers) { -try { -return writeTo(new DataOutputStream(new ByteBufferOutputStream(out)), offsetDelta, timestampDelta, -key, value, headers); -} catch (IOException e) { -// cannot actually be raised by ByteBufferOutputStream -throw new IllegalStateException("Unexpected exception raised from ByteBufferOutputStream", e); -} -} - @Override public boolean hasMagic(byte magic) { return magic >= MAGIC_VALUE_V2; http://git-wip-us.apache.org/repos/asf/kafka/blob/b3788d8d/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java --
kafka git commit: KAFKA-5316; Follow-up with ByteBufferOutputStream and other misc improvements
Repository: kafka Updated Branches: refs/heads/0.11.0 396f17a94 -> 8b63fc81d KAFKA-5316; Follow-up with ByteBufferOutputStream and other misc improvements ByteBufferOutputStream improvements: * Document pitfalls * Improve efficiency when dealing with direct byte buffers * Improve handling of buffer expansion * Be consistent about using `limit` instead of `capacity` * Add constructors that allocate the internal buffer Other minor changes: * Fix log warning to specify correct Kafka version * Clean-ups Author: Ismael Juma Reviewers: Jason Gustafson Closes #3166 from ijuma/minor-kafka-5316-follow-ups (cherry picked from commit b3788d8dcbeee7a20f562e878c187a75bac11ff0) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8b63fc81 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8b63fc81 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8b63fc81 Branch: refs/heads/0.11.0 Commit: 8b63fc81ddc193d90f6a0e0694f2735f7466969c Parents: 396f17a Author: Ismael Juma Authored: Tue May 30 12:53:32 2017 -0700 Committer: Jason Gustafson Committed: Tue May 30 12:54:04 2017 -0700 -- .../producer/internals/RecordAccumulator.java | 2 +- .../kafka/common/record/DefaultRecord.java | 19 - .../kafka/common/record/MemoryRecords.java | 8 +- .../common/record/MemoryRecordsBuilder.java | 7 +- .../common/utils/ByteBufferOutputStream.java| 85 +--- .../kafka/common/record/DefaultRecordTest.java | 18 +++-- .../utils/ByteBufferOutputStreamTest.java | 1 + 7 files changed, 77 insertions(+), 63 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/8b63fc81/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index e1f04a8..d3d1b82 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -531,7 +531,7 @@ public final class RecordAccumulator { */ public void deallocate(ProducerBatch batch) { incomplete.remove(batch); -// Only deallocate the batch if it is not a split batch because split batch are allocated aside the +// Only deallocate the batch if it is not a split batch because split batch are allocated outside the // buffer pool. if (!batch.isSplitBatch()) free.deallocate(batch.buffer(), batch.initialCapacity()); http://git-wip-us.apache.org/repos/asf/kafka/blob/8b63fc81/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java index 05b5bb2..e61bbc9 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java @@ -18,7 +18,6 @@ package org.apache.kafka.common.record; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.internals.RecordHeader; -import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.utils.ByteUtils; import org.apache.kafka.common.utils.Checksums; import org.apache.kafka.common.utils.Crc32C; @@ -230,24 +229,6 @@ public class DefaultRecord implements Record { return ByteUtils.sizeOfVarint(sizeInBytes) + sizeInBytes; } -/** - * Write the record to `out` and return its size. - */ -public static int writeTo(ByteBuffer out, - int offsetDelta, - long timestampDelta, - ByteBuffer key, - ByteBuffer value, - Header[] headers) { -try { -return writeTo(new DataOutputStream(new ByteBufferOutputStream(out)), offsetDelta, timestampDelta, -key, value, headers); -} catch (IOException e) { -// cannot actually be raised by ByteBufferOutputStream -throw new IllegalStateException("Unexpected exception raised from ByteBufferOutputStream", e); -} -} - @Override public boolean hasMagic(byte magic) { return magic >= MAGIC_VALUE_V2; http://git-wip-us.apache.org/repos/asf/kafka/blob/8b63fc81/clients/src/main/java/org/apache/k
kafka git commit: KAFKA-5202: Handle topic deletion while trying to send txn markers
Repository: kafka Updated Branches: refs/heads/trunk f0745cd51 -> 80223b14e KAFKA-5202: Handle topic deletion while trying to send txn markers Here is the sketch of this proposal: 1. When it is time to send the txn markers, only look for the leader node of the partition once instead of retrying, and if that information is not available, it means the partition is highly likely been removed since it was in the cache before. In this case, we just remove the partition from the metadata object and skip putting into the corresponding queue, and if all partitions' leader broker are non-available, complete this delayed operation to proceed to write the complete txn log entry. 2. If the leader id is unknown from the cache but the corresponding node object with the listener name is not available, it means that the leader is likely unavailable right now. Put it into a separate queue and let sender thread retry fetching its metadata again each time upon draining the queue. One caveat of this approach is the delete-and-recreate case, and the argument is that since all the messages are deleted anyways when deleting the topic-partition, it does not matter whether the markers are on the log partitions or not. Author: Guozhang Wang Reviewers: Apurva Mehta , Damian Guy , Jason Gustafson Closes #3130 from guozhangwang/K5202-handle-topic-deletion Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/80223b14 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/80223b14 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/80223b14 Branch: refs/heads/trunk Commit: 80223b14ee092e95d05b40b12631df2d6db7ef53 Parents: f0745cd Author: Guozhang Wang Authored: Tue May 30 14:35:51 2017 -0700 Committer: Jason Gustafson Committed: Tue May 30 14:35:51 2017 -0700 -- .../TransactionMarkerChannelManager.scala | 90 +--- .../src/main/scala/kafka/server/KafkaApis.scala | 2 +- .../main/scala/kafka/server/MetadataCache.scala | 26 ++-- .../scala/kafka/server/ReplicaManager.scala | 1 - .../TransactionMarkerChannelManagerTest.scala | 146 --- 5 files changed, 178 insertions(+), 87 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/80223b14/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala -- diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 461867d..344863f 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -128,10 +128,9 @@ class TransactionMarkerChannelManager(config: KafkaConfig, private val markersQueuePerBroker: concurrent.Map[Int, TxnMarkerQueue] = concurrent.TrieMap.empty[Int, TxnMarkerQueue] - private val interBrokerListenerName: ListenerName = config.interBrokerListenerName + private val markersQueueForUnknownBroker = new TxnMarkerQueue(Node.noNode) - // TODO: What is reasonable for this - private val brokerNotAliveBackoffMs = 10 + private val interBrokerListenerName: ListenerName = config.interBrokerListenerName private val txnMarkerSendThread: InterBrokerSendThread = { new InterBrokerSendThread("TxnMarkerSenderThread-" + config.brokerId, networkClient, drainQueuedTransactionMarkers, time) @@ -156,6 +155,9 @@ class TransactionMarkerChannelManager(config: KafkaConfig, } // visible for testing + private[transaction] def queueForUnknownBroker = markersQueueForUnknownBroker + + // visible for testing private[transaction] def senderThread = txnMarkerSendThread private[transaction] def addMarkersForBroker(broker: Node, txnTopicPartition: Int, txnIdAndMarker: TxnIdAndMarkerEntry) { @@ -171,6 +173,22 @@ class TransactionMarkerChannelManager(config: KafkaConfig, } private[transaction] def drainQueuedTransactionMarkers(): Iterable[RequestAndCompletionHandler] = { +val txnIdAndMarkerEntries: java.util.List[TxnIdAndMarkerEntry] = new util.ArrayList[TxnIdAndMarkerEntry]() +markersQueueForUnknownBroker.forEachTxnTopicPartition { case (_, queue) => + queue.drainTo(txnIdAndMarkerEntries) +} + +for (txnIdAndMarker: TxnIdAndMarkerEntry <- txnIdAndMarkerEntries.asScala) { + val transactionalId = txnIdAndMarker.txnId + val producerId = txnIdAndMarker.txnMarkerEntry.producerId + val producerEpoch = txnIdAndMarker.txnMarkerEntry.producerEpoch + val txnResult = txnIdAndMarker.txnMarkerEntry.transactionResult + val coordinatorEpoch = txnIdAndMarker.txnMarkerEntry.coor
kafka git commit: KAFKA-5202: Handle topic deletion while trying to send txn markers
Repository: kafka Updated Branches: refs/heads/0.11.0 1e963d04d -> d834e2aa9 KAFKA-5202: Handle topic deletion while trying to send txn markers Here is the sketch of this proposal: 1. When it is time to send the txn markers, only look for the leader node of the partition once instead of retrying, and if that information is not available, it means the partition is highly likely been removed since it was in the cache before. In this case, we just remove the partition from the metadata object and skip putting into the corresponding queue, and if all partitions' leader broker are non-available, complete this delayed operation to proceed to write the complete txn log entry. 2. If the leader id is unknown from the cache but the corresponding node object with the listener name is not available, it means that the leader is likely unavailable right now. Put it into a separate queue and let sender thread retry fetching its metadata again each time upon draining the queue. One caveat of this approach is the delete-and-recreate case, and the argument is that since all the messages are deleted anyways when deleting the topic-partition, it does not matter whether the markers are on the log partitions or not. Author: Guozhang Wang Reviewers: Apurva Mehta , Damian Guy , Jason Gustafson Closes #3130 from guozhangwang/K5202-handle-topic-deletion (cherry picked from commit 80223b14ee092e95d05b40b12631df2d6db7ef53) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/d834e2aa Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/d834e2aa Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/d834e2aa Branch: refs/heads/0.11.0 Commit: d834e2aa9712461bf6a3c304ec2dee62217e3d19 Parents: 1e963d0 Author: Guozhang Wang Authored: Tue May 30 14:35:51 2017 -0700 Committer: Jason Gustafson Committed: Tue May 30 14:57:39 2017 -0700 -- .../TransactionMarkerChannelManager.scala | 90 +--- .../src/main/scala/kafka/server/KafkaApis.scala | 2 +- .../main/scala/kafka/server/MetadataCache.scala | 26 ++-- .../scala/kafka/server/ReplicaManager.scala | 1 - .../TransactionMarkerChannelManagerTest.scala | 146 --- 5 files changed, 178 insertions(+), 87 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/d834e2aa/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala -- diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 461867d..344863f 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -128,10 +128,9 @@ class TransactionMarkerChannelManager(config: KafkaConfig, private val markersQueuePerBroker: concurrent.Map[Int, TxnMarkerQueue] = concurrent.TrieMap.empty[Int, TxnMarkerQueue] - private val interBrokerListenerName: ListenerName = config.interBrokerListenerName + private val markersQueueForUnknownBroker = new TxnMarkerQueue(Node.noNode) - // TODO: What is reasonable for this - private val brokerNotAliveBackoffMs = 10 + private val interBrokerListenerName: ListenerName = config.interBrokerListenerName private val txnMarkerSendThread: InterBrokerSendThread = { new InterBrokerSendThread("TxnMarkerSenderThread-" + config.brokerId, networkClient, drainQueuedTransactionMarkers, time) @@ -156,6 +155,9 @@ class TransactionMarkerChannelManager(config: KafkaConfig, } // visible for testing + private[transaction] def queueForUnknownBroker = markersQueueForUnknownBroker + + // visible for testing private[transaction] def senderThread = txnMarkerSendThread private[transaction] def addMarkersForBroker(broker: Node, txnTopicPartition: Int, txnIdAndMarker: TxnIdAndMarkerEntry) { @@ -171,6 +173,22 @@ class TransactionMarkerChannelManager(config: KafkaConfig, } private[transaction] def drainQueuedTransactionMarkers(): Iterable[RequestAndCompletionHandler] = { +val txnIdAndMarkerEntries: java.util.List[TxnIdAndMarkerEntry] = new util.ArrayList[TxnIdAndMarkerEntry]() +markersQueueForUnknownBroker.forEachTxnTopicPartition { case (_, queue) => + queue.drainTo(txnIdAndMarkerEntries) +} + +for (txnIdAndMarker: TxnIdAndMarkerEntry <- txnIdAndMarkerEntries.asScala) { + val transactionalId = txnIdAndMarker.txnId + val producerId = txnIdAndMarker.txnMarkerEntry.producerId + val producerEpoch = txnIdAndMarker.txnMarkerEntry.producerEpoch + val txnResult = txnIdAnd
kafka git commit: KAFKA-5251; Producer should cancel unsent AddPartitions and Produce requests on abort
Repository: kafka Updated Branches: refs/heads/trunk 3250cc767 -> d41cf1b77 KAFKA-5251; Producer should cancel unsent AddPartitions and Produce requests on abort Author: Jason Gustafson Reviewers: Apurva Mehta , Guozhang Wang Closes #3161 from hachikuji/KAFKA-5251 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/d41cf1b7 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/d41cf1b7 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/d41cf1b7 Branch: refs/heads/trunk Commit: d41cf1b77819ede5716b31683d0137eb60cb7bfb Parents: 3250cc7 Author: Jason Gustafson Authored: Tue May 30 20:32:51 2017 -0700 Committer: Jason Gustafson Committed: Tue May 30 20:32:51 2017 -0700 -- .../kafka/clients/producer/KafkaProducer.java | 2 +- .../producer/internals/RecordAccumulator.java | 19 +- .../clients/producer/internals/Sender.java | 66 +++--- .../producer/internals/TransactionManager.java | 62 +++--- .../common/requests/AddOffsetsToTxnRequest.java | 3 +- .../requests/AddPartitionsToTxnRequest.java | 7 +- .../kafka/common/requests/EndTxnRequest.java| 3 +- .../kafka/common/requests/FetchRequest.java | 7 +- .../common/requests/TxnOffsetCommitRequest.java | 3 +- .../clients/producer/internals/SenderTest.java | 2 +- .../internals/TransactionManagerTest.java | 209 +-- .../kafka/api/TransactionsTest.scala| 1 + 12 files changed, 259 insertions(+), 125 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/d41cf1b7/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 4fcbcc8..dc6b911 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -697,7 +697,7 @@ public class KafkaProducer implements Producer { throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions " + "when transactions are enabled."); -if (transactionManager.isInErrorState()) { +if (transactionManager.hasError()) { Exception lastError = transactionManager.lastError(); throw new KafkaException("Cannot perform send because at least one previous transactional or " + "idempotent request has failed with errors.", lastError); http://git-wip-us.apache.org/repos/asf/kafka/blob/d41cf1b7/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index d3d1b82..330c244 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -621,7 +621,6 @@ public final class RecordAccumulator { void abortBatches(final RuntimeException reason) { for (ProducerBatch batch : incomplete.all()) { Deque dq = getDeque(batch.topicPartition); -// Close the batch before aborting synchronized (dq) { batch.abort(); dq.remove(batch); @@ -631,6 +630,24 @@ public final class RecordAccumulator { } } +void abortUnclosedBatches(RuntimeException reason) { +for (ProducerBatch batch : incomplete.all()) { +Deque dq = getDeque(batch.topicPartition); +boolean aborted = false; +synchronized (dq) { +if (!batch.isClosed()) { +aborted = true; +batch.abort(); +dq.remove(batch); +} +} +if (aborted) { +batch.done(-1L, RecordBatch.NO_TIMESTAMP, reason); +deallocate(batch); +} +} +} + public void mutePartition(TopicPartition tp) { muted.add(tp); } http://git-wip-us.apache.org/repos/asf/kafka/blob/d41cf1b7/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.
kafka git commit: KAFKA-5251; Producer should cancel unsent AddPartitions and Produce requests on abort
Repository: kafka Updated Branches: refs/heads/0.11.0 d00e1bdeb -> 651d9a538 KAFKA-5251; Producer should cancel unsent AddPartitions and Produce requests on abort Author: Jason Gustafson Reviewers: Apurva Mehta , Guozhang Wang Closes #3161 from hachikuji/KAFKA-5251 (cherry picked from commit d41cf1b77819ede5716b31683d0137eb60cb7bfb) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/651d9a53 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/651d9a53 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/651d9a53 Branch: refs/heads/0.11.0 Commit: 651d9a5385c103aafd9f5d44ae016af7879f694f Parents: d00e1bd Author: Jason Gustafson Authored: Tue May 30 20:32:51 2017 -0700 Committer: Jason Gustafson Committed: Tue May 30 20:54:13 2017 -0700 -- .../kafka/clients/producer/KafkaProducer.java | 2 +- .../producer/internals/RecordAccumulator.java | 19 +- .../clients/producer/internals/Sender.java | 66 +++--- .../producer/internals/TransactionManager.java | 62 +++--- .../common/requests/AddOffsetsToTxnRequest.java | 3 +- .../requests/AddPartitionsToTxnRequest.java | 7 +- .../kafka/common/requests/EndTxnRequest.java| 3 +- .../kafka/common/requests/FetchRequest.java | 7 +- .../common/requests/TxnOffsetCommitRequest.java | 3 +- .../clients/producer/internals/SenderTest.java | 2 +- .../internals/TransactionManagerTest.java | 209 +-- .../kafka/api/TransactionsTest.scala| 1 + 12 files changed, 259 insertions(+), 125 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/651d9a53/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 4fcbcc8..dc6b911 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -697,7 +697,7 @@ public class KafkaProducer implements Producer { throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions " + "when transactions are enabled."); -if (transactionManager.isInErrorState()) { +if (transactionManager.hasError()) { Exception lastError = transactionManager.lastError(); throw new KafkaException("Cannot perform send because at least one previous transactional or " + "idempotent request has failed with errors.", lastError); http://git-wip-us.apache.org/repos/asf/kafka/blob/651d9a53/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index d3d1b82..330c244 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -621,7 +621,6 @@ public final class RecordAccumulator { void abortBatches(final RuntimeException reason) { for (ProducerBatch batch : incomplete.all()) { Deque dq = getDeque(batch.topicPartition); -// Close the batch before aborting synchronized (dq) { batch.abort(); dq.remove(batch); @@ -631,6 +630,24 @@ public final class RecordAccumulator { } } +void abortUnclosedBatches(RuntimeException reason) { +for (ProducerBatch batch : incomplete.all()) { +Deque dq = getDeque(batch.topicPartition); +boolean aborted = false; +synchronized (dq) { +if (!batch.isClosed()) { +aborted = true; +batch.abort(); +dq.remove(batch); +} +} +if (aborted) { +batch.done(-1L, RecordBatch.NO_TIMESTAMP, reason); +deallocate(batch); +} +} +} + public void mutePartition(TopicPartition tp) { muted.add(tp); } http://git-wip-us.apache.org/repos/asf/kafka/blob/651d9a53/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/pro
kafka git commit: KAFKA-5211; Do not skip a corrupted record in consumer
Repository: kafka Updated Branches: refs/heads/0.11.0 651d9a538 -> a1a487945 KAFKA-5211; Do not skip a corrupted record in consumer Author: Jiangjie Qin Reviewers: Jason Gustafson Closes #3114 from becketqin/KAFKA-5211 (cherry picked from commit d082563907103ea79eed681305df7093053f52ec) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/a1a48794 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/a1a48794 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/a1a48794 Branch: refs/heads/0.11.0 Commit: a1a48794548c546f45d55162f55928420fec0c8f Parents: 651d9a5 Author: Jiangjie Qin Authored: Tue May 30 22:09:53 2017 -0700 Committer: Jason Gustafson Committed: Tue May 30 22:59:09 2017 -0700 -- .../clients/consumer/internals/Fetcher.java | 84 +--- .../clients/consumer/internals/FetcherTest.java | 127 +-- 2 files changed, 123 insertions(+), 88 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/a1a48794/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 5287b4e..e3f2355 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -113,7 +113,6 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { private final IsolationLevel isolationLevel; private PartitionRecords nextInLineRecords = null; -private ExceptionMetadata nextInLineExceptionMetadata = null; public Fetcher(ConsumerNetworkClient client, int minBytes, @@ -154,7 +153,7 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { private ExtendedDeserializer ensureExtended(Deserializer deserializer) { return deserializer instanceof ExtendedDeserializer ? (ExtendedDeserializer) deserializer : new ExtendedDeserializer.Wrapper<>(deserializer); } - + /** * Represents data about an offset returned by a broker. */ @@ -513,31 +512,18 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { * the defaultResetPolicy is NONE */ public Map>> fetchedRecords() { -if (nextInLineExceptionMetadata != null) { -ExceptionMetadata exceptionMetadata = nextInLineExceptionMetadata; -nextInLineExceptionMetadata = null; -TopicPartition tp = exceptionMetadata.partition; -if (subscriptions.isFetchable(tp) && subscriptions.position(tp) == exceptionMetadata.fetchedOffset) -throw exceptionMetadata.exception; -} Map>> fetched = new HashMap<>(); int recordsRemaining = maxPollRecords; -// Needed to construct ExceptionMetadata if any exception is found when processing completedFetch -TopicPartition fetchedPartition = null; -long fetchedOffset = -1; try { while (recordsRemaining > 0) { if (nextInLineRecords == null || nextInLineRecords.isFetched) { -CompletedFetch completedFetch = completedFetches.poll(); +CompletedFetch completedFetch = completedFetches.peek(); if (completedFetch == null) break; -fetchedPartition = completedFetch.partition; -fetchedOffset = completedFetch.fetchedOffset; nextInLineRecords = parseCompletedFetch(completedFetch); +completedFetches.poll(); } else { -fetchedPartition = nextInLineRecords.partition; -fetchedOffset = nextInLineRecords.nextFetchOffset; List> records = fetchRecords(nextInLineRecords, recordsRemaining); TopicPartition partition = nextInLineRecords.partition; if (!records.isEmpty()) { @@ -560,8 +546,6 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { } catch (KafkaException e) { if (fetched.isEmpty()) throw e; -// To be thrown in the next call of this method -nextInLineExceptionMetadata = new ExceptionMetadata(fetchedPartition, fetchedOffset, e); } return fetched; } @@ -952,10 +936,11 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { private int recordsRead; private int bytesRead; private RecordBatch current
kafka git commit: KAFKA-5211; Do not skip a corrupted record in consumer
Repository: kafka Updated Branches: refs/heads/trunk d41cf1b77 -> d08256390 KAFKA-5211; Do not skip a corrupted record in consumer Author: Jiangjie Qin Reviewers: Jason Gustafson Closes #3114 from becketqin/KAFKA-5211 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/d0825639 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/d0825639 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/d0825639 Branch: refs/heads/trunk Commit: d082563907103ea79eed681305df7093053f52ec Parents: d41cf1b Author: Jiangjie Qin Authored: Tue May 30 22:09:53 2017 -0700 Committer: Jason Gustafson Committed: Tue May 30 22:41:21 2017 -0700 -- .../clients/consumer/internals/Fetcher.java | 84 +--- .../clients/consumer/internals/FetcherTest.java | 127 +-- 2 files changed, 123 insertions(+), 88 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/d0825639/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 5287b4e..e3f2355 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -113,7 +113,6 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { private final IsolationLevel isolationLevel; private PartitionRecords nextInLineRecords = null; -private ExceptionMetadata nextInLineExceptionMetadata = null; public Fetcher(ConsumerNetworkClient client, int minBytes, @@ -154,7 +153,7 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { private ExtendedDeserializer ensureExtended(Deserializer deserializer) { return deserializer instanceof ExtendedDeserializer ? (ExtendedDeserializer) deserializer : new ExtendedDeserializer.Wrapper<>(deserializer); } - + /** * Represents data about an offset returned by a broker. */ @@ -513,31 +512,18 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { * the defaultResetPolicy is NONE */ public Map>> fetchedRecords() { -if (nextInLineExceptionMetadata != null) { -ExceptionMetadata exceptionMetadata = nextInLineExceptionMetadata; -nextInLineExceptionMetadata = null; -TopicPartition tp = exceptionMetadata.partition; -if (subscriptions.isFetchable(tp) && subscriptions.position(tp) == exceptionMetadata.fetchedOffset) -throw exceptionMetadata.exception; -} Map>> fetched = new HashMap<>(); int recordsRemaining = maxPollRecords; -// Needed to construct ExceptionMetadata if any exception is found when processing completedFetch -TopicPartition fetchedPartition = null; -long fetchedOffset = -1; try { while (recordsRemaining > 0) { if (nextInLineRecords == null || nextInLineRecords.isFetched) { -CompletedFetch completedFetch = completedFetches.poll(); +CompletedFetch completedFetch = completedFetches.peek(); if (completedFetch == null) break; -fetchedPartition = completedFetch.partition; -fetchedOffset = completedFetch.fetchedOffset; nextInLineRecords = parseCompletedFetch(completedFetch); +completedFetches.poll(); } else { -fetchedPartition = nextInLineRecords.partition; -fetchedOffset = nextInLineRecords.nextFetchOffset; List> records = fetchRecords(nextInLineRecords, recordsRemaining); TopicPartition partition = nextInLineRecords.partition; if (!records.isEmpty()) { @@ -560,8 +546,6 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { } catch (KafkaException e) { if (fetched.isEmpty()) throw e; -// To be thrown in the next call of this method -nextInLineExceptionMetadata = new ExceptionMetadata(fetchedPartition, fetchedOffset, e); } return fetched; } @@ -952,10 +936,11 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { private int recordsRead; private int bytesRead; private RecordBatch currentBatch; +private Record lastRecord; private CloseableIterator records; private
kafka git commit: KAFKA-5266; Follow-up improvements for consumer offset reset tool (KIP-122)
Repository: kafka Updated Branches: refs/heads/trunk 2cc8f48ae -> ef9551297 KAFKA-5266; Follow-up improvements for consumer offset reset tool (KIP-122) Implement improvements defined here: https://issues.apache.org/jira/browse/KAFKA-5266 Author: Jorge Quilcate Otoya Reviewers: Jason Gustafson Closes #3102 from jeqo/feature/KAFKA-5266 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/ef955129 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/ef955129 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/ef955129 Branch: refs/heads/trunk Commit: ef9551297c815a0ac3065a65a0831863090714f0 Parents: 2cc8f48 Author: Jorge Quilcate Otoya Authored: Wed May 31 00:42:43 2017 -0700 Committer: Jason Gustafson Committed: Wed May 31 00:50:48 2017 -0700 -- .../kafka/admin/ConsumerGroupCommand.scala | 88 .../admin/ResetConsumerGroupOffsetTest.scala| 33 ++-- 2 files changed, 80 insertions(+), 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/ef955129/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala -- diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index fb589a2..2f26f57 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -106,8 +106,7 @@ object ConsumerGroupCommand extends Logging { } else if (opts.options.has(opts.resetOffsetsOpt)) { val offsetsToReset = consumerGroupService.resetOffsets() -val export = opts.options.has(opts.exportOpt) -if (export) { +if (opts.options.has(opts.exportOpt)) { val exported = consumerGroupService.exportOffsetsToReset(offsetsToReset) println(exported) } else @@ -523,7 +522,7 @@ object ConsumerGroupCommand extends Logging { val groupId = opts.options.valueOf(opts.groupOpt) val consumerGroupSummary = adminClient.describeConsumerGroup(groupId, opts.options.valueOf(opts.timeoutMsOpt)) consumerGroupSummary.state match { -case "Empty" => +case "Empty" | "Dead" => val partitionsToReset = getPartitionsToReset(groupId) val preparedOffsets = prepareOffsetsToReset(groupId, partitionsToReset) val execute = opts.options.has(opts.executeOpt) @@ -536,7 +535,7 @@ object ConsumerGroupCommand extends Logging { } } -private def parseTopicPartitionsToReset(topicArgs: Seq[String]): Iterable[TopicPartition] = topicArgs.flatMap { +private def parseTopicPartitionsToReset(topicArgs: Seq[String]): Seq[TopicPartition] = topicArgs.flatMap { case topicArg if topicArg.contains(":") => val topicAndPartitions = topicArg.split(":") val topic = topicAndPartitions(0) @@ -545,14 +544,18 @@ object ConsumerGroupCommand extends Logging { .map(partitionInfo => new TopicPartition(topic, partitionInfo.partition)) } -private def getPartitionsToReset(groupId: String): Iterable[TopicPartition] = { +private def getPartitionsToReset(groupId: String): Seq[TopicPartition] = { if (opts.options.has(opts.allTopicsOpt)) { -adminClient.listGroupOffsets(groupId).keys +val allTopicPartitions = adminClient.listGroupOffsets(groupId).keys.toSeq +allTopicPartitions } else if (opts.options.has(opts.topicOpt)) { val topics = opts.options.valuesOf(opts.topicOpt).asScala parseTopicPartitionsToReset(topics) } else { -CommandLineUtils.printUsageAndDie(opts.parser, "One of the reset scopes should be defined: --all-topics, --topic.") +if (opts.options.has(opts.resetFromFileOpt)) + Nil +else + CommandLineUtils.printUsageAndDie(opts.parser, "One of the reset scopes should be defined: --all-topics, --topic.") } } @@ -570,20 +573,22 @@ object ConsumerGroupCommand extends Logging { if (opts.options.has(opts.resetToOffsetOpt)) { val offset = opts.options.valueOf(opts.resetToOffsetOpt) partitionsToReset.map { - topicPartition => (topicPartition, new OffsetAndMetadata(offset)) + topicPartition => +val newOffset: Long = checkOffsetRange(topicPartition, offset) +(topicPartition, new OffsetAndMetadata(newOffset)) }.toMap } else if (opts.options.has(opts.resetToEarliestOpt)) { partitionsToReset.map { topicPartition => getLogStartOffset(topicPartition) match { case LogOffsetResult.LogOffset(offset) => (topicPartition, new OffsetAndMetadata(offset)) -case _ => null +
kafka git commit: KAFKA-5266; Follow-up improvements for consumer offset reset tool (KIP-122)
Repository: kafka Updated Branches: refs/heads/0.11.0 85d2ce23c -> 08c80c6d4 KAFKA-5266; Follow-up improvements for consumer offset reset tool (KIP-122) Implement improvements defined here: https://issues.apache.org/jira/browse/KAFKA-5266 Author: Jorge Quilcate Otoya Reviewers: Jason Gustafson Closes #3102 from jeqo/feature/KAFKA-5266 (cherry picked from commit ef9551297c815a0ac3065a65a0831863090714f0) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/08c80c6d Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/08c80c6d Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/08c80c6d Branch: refs/heads/0.11.0 Commit: 08c80c6d4859a25be3c72f1dbefb4bf2734fcd17 Parents: 85d2ce2 Author: Jorge Quilcate Otoya Authored: Wed May 31 00:42:43 2017 -0700 Committer: Jason Gustafson Committed: Wed May 31 00:52:47 2017 -0700 -- .../kafka/admin/ConsumerGroupCommand.scala | 88 .../admin/ResetConsumerGroupOffsetTest.scala| 33 ++-- 2 files changed, 80 insertions(+), 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/08c80c6d/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala -- diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index fb589a2..2f26f57 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -106,8 +106,7 @@ object ConsumerGroupCommand extends Logging { } else if (opts.options.has(opts.resetOffsetsOpt)) { val offsetsToReset = consumerGroupService.resetOffsets() -val export = opts.options.has(opts.exportOpt) -if (export) { +if (opts.options.has(opts.exportOpt)) { val exported = consumerGroupService.exportOffsetsToReset(offsetsToReset) println(exported) } else @@ -523,7 +522,7 @@ object ConsumerGroupCommand extends Logging { val groupId = opts.options.valueOf(opts.groupOpt) val consumerGroupSummary = adminClient.describeConsumerGroup(groupId, opts.options.valueOf(opts.timeoutMsOpt)) consumerGroupSummary.state match { -case "Empty" => +case "Empty" | "Dead" => val partitionsToReset = getPartitionsToReset(groupId) val preparedOffsets = prepareOffsetsToReset(groupId, partitionsToReset) val execute = opts.options.has(opts.executeOpt) @@ -536,7 +535,7 @@ object ConsumerGroupCommand extends Logging { } } -private def parseTopicPartitionsToReset(topicArgs: Seq[String]): Iterable[TopicPartition] = topicArgs.flatMap { +private def parseTopicPartitionsToReset(topicArgs: Seq[String]): Seq[TopicPartition] = topicArgs.flatMap { case topicArg if topicArg.contains(":") => val topicAndPartitions = topicArg.split(":") val topic = topicAndPartitions(0) @@ -545,14 +544,18 @@ object ConsumerGroupCommand extends Logging { .map(partitionInfo => new TopicPartition(topic, partitionInfo.partition)) } -private def getPartitionsToReset(groupId: String): Iterable[TopicPartition] = { +private def getPartitionsToReset(groupId: String): Seq[TopicPartition] = { if (opts.options.has(opts.allTopicsOpt)) { -adminClient.listGroupOffsets(groupId).keys +val allTopicPartitions = adminClient.listGroupOffsets(groupId).keys.toSeq +allTopicPartitions } else if (opts.options.has(opts.topicOpt)) { val topics = opts.options.valuesOf(opts.topicOpt).asScala parseTopicPartitionsToReset(topics) } else { -CommandLineUtils.printUsageAndDie(opts.parser, "One of the reset scopes should be defined: --all-topics, --topic.") +if (opts.options.has(opts.resetFromFileOpt)) + Nil +else + CommandLineUtils.printUsageAndDie(opts.parser, "One of the reset scopes should be defined: --all-topics, --topic.") } } @@ -570,20 +573,22 @@ object ConsumerGroupCommand extends Logging { if (opts.options.has(opts.resetToOffsetOpt)) { val offset = opts.options.valueOf(opts.resetToOffsetOpt) partitionsToReset.map { - topicPartition => (topicPartition, new OffsetAndMetadata(offset)) + topicPartition => +val newOffset: Long = checkOffsetRange(topicPartition, offset) +(topicPartition, new OffsetAndMetadata(newOffset)) }.toMap } else if (opts.options.has(opts.resetToEarliestOpt)) { partitionsToReset.map { topicPartition => getLogStartOffset(topicPartition) match { case LogOffsetResult.LogOff
kafka git commit: KAFKA-5093; Avoid loading full batch data when possible when iterating FileRecords
Repository: kafka Updated Branches: refs/heads/trunk da9a171c9 -> 81f0c1e8f KAFKA-5093; Avoid loading full batch data when possible when iterating FileRecords Author: Jason Gustafson Reviewers: Ismael Juma Closes #3160 from hachikuji/KAFKA-5093 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/81f0c1e8 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/81f0c1e8 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/81f0c1e8 Branch: refs/heads/trunk Commit: 81f0c1e8f2ba2d86f061361b5ee33bb8e6f640c5 Parents: da9a171 Author: Jason Gustafson Authored: Wed May 31 14:11:47 2017 -0700 Committer: Jason Gustafson Committed: Wed May 31 14:11:47 2017 -0700 -- .../record/AbstractLegacyRecordBatch.java | 73 ++ .../kafka/common/record/DefaultRecordBatch.java | 77 +++ .../kafka/common/record/FileLogInputStream.java | 216 +++--- .../apache/kafka/common/record/FileRecords.java | 4 +- .../kafka/common/record/LegacyRecord.java | 43 ++-- .../kafka/common/record/MemoryRecords.java | 11 +- .../org/apache/kafka/common/record/Records.java | 2 + .../clients/consumer/internals/FetcherTest.java | 2 +- .../common/record/FileLogInputStreamTest.java | 223 ++- .../kafka/common/record/FileRecordsTest.java| 6 +- .../common/record/MemoryRecordsBuilderTest.java | 2 +- .../kafka/message/ByteBufferMessageSet.scala| 4 +- .../scala/kafka/message/MessageAndOffset.scala | 24 +- 13 files changed, 504 insertions(+), 183 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/81f0c1e8/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java index e4938be..9b74d06 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java @@ -29,6 +29,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; import java.util.ArrayDeque; import java.util.NoSuchElementException; @@ -510,4 +511,76 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl } } +static class LegacyFileChannelRecordBatch extends FileLogInputStream.FileChannelRecordBatch { + +LegacyFileChannelRecordBatch(long offset, + byte magic, + FileChannel channel, + int position, + int batchSize) { +super(offset, magic, channel, position, batchSize); +} + +@Override +protected RecordBatch toMemoryRecordBatch(ByteBuffer buffer) { +return new ByteBufferLegacyRecordBatch(buffer); +} + +@Override +public long baseOffset() { +return loadFullBatch().baseOffset(); +} + +@Override +public long lastOffset() { +return offset; +} + +@Override +public long producerId() { +return RecordBatch.NO_PRODUCER_ID; +} + +@Override +public short producerEpoch() { +return RecordBatch.NO_PRODUCER_EPOCH; +} + +@Override +public int baseSequence() { +return RecordBatch.NO_SEQUENCE; +} + +@Override +public int lastSequence() { +return RecordBatch.NO_SEQUENCE; +} + +@Override +public Integer countOrNull() { +return null; +} + +@Override +public boolean isTransactional() { +return false; +} + +@Override +public boolean isControlBatch() { +return false; +} + +@Override +public int partitionLeaderEpoch() { +return RecordBatch.NO_PARTITION_LEADER_EPOCH; +} + +@Override +protected int headerSize() { +return LOG_OVERHEAD + LegacyRecord.headerSize(magic); +} + +} + } http://git-wip-us.apache.org/repos/asf/kafka/blob/81f0c1e8/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index 2bf889f
kafka git commit: KAFKA-5093; Avoid loading full batch data when possible when iterating FileRecords
Repository: kafka Updated Branches: refs/heads/0.11.0 1b64a4e63 -> 1aad6b468 KAFKA-5093; Avoid loading full batch data when possible when iterating FileRecords Author: Jason Gustafson Reviewers: Ismael Juma Closes #3160 from hachikuji/KAFKA-5093 (cherry picked from commit 81f0c1e8f2ba2d86f061361b5ee33bb8e6f640c5) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1aad6b46 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1aad6b46 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1aad6b46 Branch: refs/heads/0.11.0 Commit: 1aad6b468dec2b51849e7906d793970595f7a997 Parents: 1b64a4e Author: Jason Gustafson Authored: Wed May 31 14:11:47 2017 -0700 Committer: Jason Gustafson Committed: Wed May 31 14:14:24 2017 -0700 -- .../record/AbstractLegacyRecordBatch.java | 73 ++ .../kafka/common/record/DefaultRecordBatch.java | 77 +++ .../kafka/common/record/FileLogInputStream.java | 216 +++--- .../apache/kafka/common/record/FileRecords.java | 4 +- .../kafka/common/record/LegacyRecord.java | 43 ++-- .../kafka/common/record/MemoryRecords.java | 11 +- .../org/apache/kafka/common/record/Records.java | 2 + .../clients/consumer/internals/FetcherTest.java | 2 +- .../common/record/FileLogInputStreamTest.java | 223 ++- .../kafka/common/record/FileRecordsTest.java| 6 +- .../common/record/MemoryRecordsBuilderTest.java | 2 +- .../kafka/message/ByteBufferMessageSet.scala| 4 +- .../scala/kafka/message/MessageAndOffset.scala | 24 +- 13 files changed, 504 insertions(+), 183 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/1aad6b46/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java index e4938be..9b74d06 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java @@ -29,6 +29,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; import java.util.ArrayDeque; import java.util.NoSuchElementException; @@ -510,4 +511,76 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl } } +static class LegacyFileChannelRecordBatch extends FileLogInputStream.FileChannelRecordBatch { + +LegacyFileChannelRecordBatch(long offset, + byte magic, + FileChannel channel, + int position, + int batchSize) { +super(offset, magic, channel, position, batchSize); +} + +@Override +protected RecordBatch toMemoryRecordBatch(ByteBuffer buffer) { +return new ByteBufferLegacyRecordBatch(buffer); +} + +@Override +public long baseOffset() { +return loadFullBatch().baseOffset(); +} + +@Override +public long lastOffset() { +return offset; +} + +@Override +public long producerId() { +return RecordBatch.NO_PRODUCER_ID; +} + +@Override +public short producerEpoch() { +return RecordBatch.NO_PRODUCER_EPOCH; +} + +@Override +public int baseSequence() { +return RecordBatch.NO_SEQUENCE; +} + +@Override +public int lastSequence() { +return RecordBatch.NO_SEQUENCE; +} + +@Override +public Integer countOrNull() { +return null; +} + +@Override +public boolean isTransactional() { +return false; +} + +@Override +public boolean isControlBatch() { +return false; +} + +@Override +public int partitionLeaderEpoch() { +return RecordBatch.NO_PARTITION_LEADER_EPOCH; +} + +@Override +protected int headerSize() { +return LOG_OVERHEAD + LegacyRecord.headerSize(magic); +} + +} + } http://git-wip-us.apache.org/repos/asf/kafka/blob/1aad6b46/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord
kafka git commit: KAFKA-5340; Batch splitting should preserve magic and transactional flag
Repository: kafka Updated Branches: refs/heads/trunk 6360e04e7 -> e4a6b50de KAFKA-5340; Batch splitting should preserve magic and transactional flag Author: Jason Gustafson Reviewers: Apurva Mehta , Jiangjie Qin , Guozhang Wang Closes #3162 from hachikuji/KAFKA-5340 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/e4a6b50d Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/e4a6b50d Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/e4a6b50d Branch: refs/heads/trunk Commit: e4a6b50deca8fabc9880c6764334bfaa830a6d5e Parents: 6360e04 Author: Jason Gustafson Authored: Wed May 31 21:31:52 2017 -0700 Committer: Jason Gustafson Committed: Wed May 31 21:31:52 2017 -0700 -- .../producer/internals/ProducerBatch.java | 30 +-- .../producer/internals/RecordAccumulator.java | 13 +-- .../clients/producer/internals/Sender.java | 4 +- .../producer/internals/TransactionManager.java | 9 +++ .../kafka/common/record/MemoryRecords.java | 13 +-- .../common/record/MemoryRecordsBuilder.java | 7 +- .../producer/internals/ProducerBatchTest.java | 43 +- .../internals/RecordAccumulatorTest.java| 2 +- .../clients/producer/internals/SenderTest.java | 85 +++- .../common/record/MemoryRecordsBuilderTest.java | 9 +-- 10 files changed, 162 insertions(+), 53 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/e4a6b50d/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java index 974e230..c7253a9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.CompressionRatioEstimator; +import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MutableRecordBatch; @@ -43,6 +44,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V2; import static org.apache.kafka.common.record.RecordBatch.NO_TIMESTAMP; @@ -179,23 +181,29 @@ public final class ProducerBatch { public Deque split(int splitBatchSize) { Deque batches = new ArrayDeque<>(); MemoryRecords memoryRecords = recordsBuilder.build(); + Iterator recordBatchIter = memoryRecords.batches().iterator(); if (!recordBatchIter.hasNext()) throw new IllegalStateException("Cannot split an empty producer batch."); + RecordBatch recordBatch = recordBatchIter.next(); +if (recordBatch.magic() < MAGIC_VALUE_V2 && !recordBatch.isCompressed()) +throw new IllegalArgumentException("Batch splitting cannot be used with non-compressed messages " + +"with version v0 and v1"); + if (recordBatchIter.hasNext()) -throw new IllegalStateException("A producer batch should only have one record batch."); +throw new IllegalArgumentException("A producer batch should only have one record batch."); Iterator thunkIter = thunks.iterator(); // We always allocate batch size because we are already splitting a big batch. // And we also Retain the create time of the original batch. ProducerBatch batch = null; + for (Record record : recordBatch) { assert thunkIter.hasNext(); Thunk thunk = thunkIter.next(); -if (batch == null) { +if (batch == null) batch = createBatchOffAccumulatorForRecord(record, splitBatchSize); -} // A newly created batch can always host the first message. if (!batch.tryAppendForSplit(record.timestamp(), record.key(), record.value(), record.headers(), thunk)) { @@ -204,6 +212,7 @@ public final class ProducerBatch { batch.tryAppendForSplit(record.timestamp(), record.key(), record.value(), record.headers(), thunk); } } + // Close the last batch and add it to the batch list after split. if (batch != null) batches.add(bat
kafka git commit: KAFKA-5340; Batch splitting should preserve magic and transactional flag
Repository: kafka Updated Branches: refs/heads/0.11.0 458a9a4c5 -> 3c3edc9db KAFKA-5340; Batch splitting should preserve magic and transactional flag Author: Jason Gustafson Reviewers: Apurva Mehta , Jiangjie Qin , Guozhang Wang Closes #3162 from hachikuji/KAFKA-5340 (cherry picked from commit e4a6b50deca8fabc9880c6764334bfaa830a6d5e) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/3c3edc9d Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/3c3edc9d Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/3c3edc9d Branch: refs/heads/0.11.0 Commit: 3c3edc9dbaa5d22e0d5c466eb6015d7dea1e0c01 Parents: 458a9a4 Author: Jason Gustafson Authored: Wed May 31 21:31:52 2017 -0700 Committer: Jason Gustafson Committed: Wed May 31 21:34:11 2017 -0700 -- .../producer/internals/ProducerBatch.java | 30 +-- .../producer/internals/RecordAccumulator.java | 13 +-- .../clients/producer/internals/Sender.java | 4 +- .../producer/internals/TransactionManager.java | 9 +++ .../kafka/common/record/MemoryRecords.java | 13 +-- .../common/record/MemoryRecordsBuilder.java | 7 +- .../producer/internals/ProducerBatchTest.java | 43 +- .../internals/RecordAccumulatorTest.java| 2 +- .../clients/producer/internals/SenderTest.java | 85 +++- .../common/record/MemoryRecordsBuilderTest.java | 9 +-- 10 files changed, 162 insertions(+), 53 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/3c3edc9d/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java index 974e230..c7253a9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.CompressionRatioEstimator; +import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MutableRecordBatch; @@ -43,6 +44,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V2; import static org.apache.kafka.common.record.RecordBatch.NO_TIMESTAMP; @@ -179,23 +181,29 @@ public final class ProducerBatch { public Deque split(int splitBatchSize) { Deque batches = new ArrayDeque<>(); MemoryRecords memoryRecords = recordsBuilder.build(); + Iterator recordBatchIter = memoryRecords.batches().iterator(); if (!recordBatchIter.hasNext()) throw new IllegalStateException("Cannot split an empty producer batch."); + RecordBatch recordBatch = recordBatchIter.next(); +if (recordBatch.magic() < MAGIC_VALUE_V2 && !recordBatch.isCompressed()) +throw new IllegalArgumentException("Batch splitting cannot be used with non-compressed messages " + +"with version v0 and v1"); + if (recordBatchIter.hasNext()) -throw new IllegalStateException("A producer batch should only have one record batch."); +throw new IllegalArgumentException("A producer batch should only have one record batch."); Iterator thunkIter = thunks.iterator(); // We always allocate batch size because we are already splitting a big batch. // And we also Retain the create time of the original batch. ProducerBatch batch = null; + for (Record record : recordBatch) { assert thunkIter.hasNext(); Thunk thunk = thunkIter.next(); -if (batch == null) { +if (batch == null) batch = createBatchOffAccumulatorForRecord(record, splitBatchSize); -} // A newly created batch can always host the first message. if (!batch.tryAppendForSplit(record.timestamp(), record.key(), record.value(), record.headers(), thunk)) { @@ -204,6 +212,7 @@ public final class ProducerBatch { batch.tryAppendForSplit(record.timestamp(), record.key(), record.value(), record.headers(), thunk); } } + // Close the last
kafka git commit: KAFKA-5360; Down-converted uncompressed batches should respect fetch offset
Repository: kafka Updated Branches: refs/heads/0.11.0 64855109d -> b42cefe2f KAFKA-5360; Down-converted uncompressed batches should respect fetch offset More specifically, V2 messages are always batched (whether compressed or not) while V0/V1 are only batched if they are compressed. Clients like librdkafka expect to receive messages from the fetch offset when dealing with uncompressed V0/V1 messages. When converting from V2 to V0/1, we were returning all the messages in the V2 batch. Author: Ismael Juma Reviewers: Jason Gustafson Closes #3191 from ijuma/kafka-5360-down-converted-uncompressed-respect-offset (cherry picked from commit 8e8b3c56572a825d3c1beb6ad77ce88571354f51) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b42cefe2 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b42cefe2 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b42cefe2 Branch: refs/heads/0.11.0 Commit: b42cefe2f977ced5cd3c600a97f9672be5862373 Parents: 6485510 Author: Ismael Juma Authored: Thu Jun 1 10:17:03 2017 -0700 Committer: Jason Gustafson Committed: Thu Jun 1 10:19:29 2017 -0700 -- .../clients/producer/internals/Sender.java | 2 +- .../kafka/common/record/AbstractRecords.java| 22 - .../apache/kafka/common/record/FileRecords.java | 4 +- .../kafka/common/record/MemoryRecords.java | 4 +- .../org/apache/kafka/common/record/Records.java | 4 +- .../kafka/common/record/FileRecordsTest.java| 54 +--- .../common/record/MemoryRecordsBuilderTest.java | 43 -- .../src/main/scala/kafka/server/KafkaApis.scala | 29 --- .../unit/kafka/server/FetchRequestTest.scala| 89 +--- 9 files changed, 201 insertions(+), 50 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/b42cefe2/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 01ff91a..4f1c7d4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -648,7 +648,7 @@ public class Sender implements Runnable { // not all support the same message format version. For example, if a partition migrates from a broker // which is supporting the new magic version to one which doesn't, then we will need to convert. if (!records.hasMatchingMagic(minUsedMagic)) -records = batch.records().downConvert(minUsedMagic); +records = batch.records().downConvert(minUsedMagic, 0); produceRecordsByPartition.put(tp, records); recordsByPartition.put(tp, batch); } http://git-wip-us.apache.org/repos/asf/kafka/blob/b42cefe2/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java index 2771ab7..04d7071 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java @@ -50,7 +50,16 @@ public abstract class AbstractRecords implements Records { return true; } -protected MemoryRecords downConvert(Iterable batches, byte toMagic) { +/** + * Down convert batches to the provided message format version. The first offset parameter is only relevant in the + * conversion from uncompressed v2 or higher to v1 or lower. The reason is that uncompressed records in v0 and v1 + * are not batched (put another way, each batch always has 1 record). + * + * If a client requests records in v1 format starting from the middle of an uncompressed batch in v2 format, we + * need to drop records from the batch during the conversion. Some versions of librdkafka rely on this for + * correctness. + */ +protected MemoryRecords downConvert(Iterable batches, byte toMagic, long firstOffset) { // maintain the batch along with the decompressed records to avoid the need to decompress again List recordBatchAndRecordsList = new ArrayList<>(); int totalSizeEstimate = 0; @@ -63,9 +72,16 @@ public abstract class AbstractRecords implements Records { totalSizeEstimate += batch.sizeInBytes(); recordBatchAndRecordsList.add(new RecordBatchAndReco
kafka git commit: KAFKA-5360; Down-converted uncompressed batches should respect fetch offset
Repository: kafka Updated Branches: refs/heads/trunk a08634642 -> 8e8b3c565 KAFKA-5360; Down-converted uncompressed batches should respect fetch offset More specifically, V2 messages are always batched (whether compressed or not) while V0/V1 are only batched if they are compressed. Clients like librdkafka expect to receive messages from the fetch offset when dealing with uncompressed V0/V1 messages. When converting from V2 to V0/1, we were returning all the messages in the V2 batch. Author: Ismael Juma Reviewers: Jason Gustafson Closes #3191 from ijuma/kafka-5360-down-converted-uncompressed-respect-offset Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8e8b3c56 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8e8b3c56 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8e8b3c56 Branch: refs/heads/trunk Commit: 8e8b3c56572a825d3c1beb6ad77ce88571354f51 Parents: a086346 Author: Ismael Juma Authored: Thu Jun 1 10:17:03 2017 -0700 Committer: Jason Gustafson Committed: Thu Jun 1 10:17:03 2017 -0700 -- .../clients/producer/internals/Sender.java | 2 +- .../kafka/common/record/AbstractRecords.java| 22 - .../apache/kafka/common/record/FileRecords.java | 4 +- .../kafka/common/record/MemoryRecords.java | 4 +- .../org/apache/kafka/common/record/Records.java | 4 +- .../kafka/common/record/FileRecordsTest.java| 54 +--- .../common/record/MemoryRecordsBuilderTest.java | 43 -- .../src/main/scala/kafka/server/KafkaApis.scala | 29 --- .../unit/kafka/server/FetchRequestTest.scala| 89 +--- 9 files changed, 201 insertions(+), 50 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/8e8b3c56/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 01ff91a..4f1c7d4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -648,7 +648,7 @@ public class Sender implements Runnable { // not all support the same message format version. For example, if a partition migrates from a broker // which is supporting the new magic version to one which doesn't, then we will need to convert. if (!records.hasMatchingMagic(minUsedMagic)) -records = batch.records().downConvert(minUsedMagic); +records = batch.records().downConvert(minUsedMagic, 0); produceRecordsByPartition.put(tp, records); recordsByPartition.put(tp, batch); } http://git-wip-us.apache.org/repos/asf/kafka/blob/8e8b3c56/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java index 2771ab7..04d7071 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java @@ -50,7 +50,16 @@ public abstract class AbstractRecords implements Records { return true; } -protected MemoryRecords downConvert(Iterable batches, byte toMagic) { +/** + * Down convert batches to the provided message format version. The first offset parameter is only relevant in the + * conversion from uncompressed v2 or higher to v1 or lower. The reason is that uncompressed records in v0 and v1 + * are not batched (put another way, each batch always has 1 record). + * + * If a client requests records in v1 format starting from the middle of an uncompressed batch in v2 format, we + * need to drop records from the batch during the conversion. Some versions of librdkafka rely on this for + * correctness. + */ +protected MemoryRecords downConvert(Iterable batches, byte toMagic, long firstOffset) { // maintain the batch along with the decompressed records to avoid the need to decompress again List recordBatchAndRecordsList = new ArrayList<>(); int totalSizeEstimate = 0; @@ -63,9 +72,16 @@ public abstract class AbstractRecords implements Records { totalSizeEstimate += batch.sizeInBytes(); recordBatchAndRecordsList.add(new RecordBatchAndRecords(batch, null, null)); } else { -List records = Utils.toList(batch.itera
kafka git commit: KAFKA-5281; System tests for transactions
Repository: kafka Updated Branches: refs/heads/trunk 8e8b3c565 -> 1959835d9 KAFKA-5281; System tests for transactions Author: Apurva Mehta Reviewers: Jason Gustafson Closes #3149 from apurvam/KAFKA-5281-transactions-system-tests Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1959835d Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1959835d Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1959835d Branch: refs/heads/trunk Commit: 1959835d9e148f0eb6407b36ff96b334d5e785cb Parents: 8e8b3c5 Author: Apurva Mehta Authored: Thu Jun 1 10:25:29 2017 -0700 Committer: Jason Gustafson Committed: Thu Jun 1 10:25:29 2017 -0700 -- .../producer/internals/TransactionManager.java | 4 +- .../common/requests/OffsetCommitResponse.java | 1 + .../requests/TxnOffsetCommitResponse.java | 1 + ...nsactionMarkerRequestCompletionHandler.scala | 3 +- .../scala/kafka/tools/ConsoleConsumer.scala | 12 +- tests/kafkatest/services/console_consumer.py| 9 +- .../services/transactional_message_copier.py| 183 tests/kafkatest/tests/core/transactions_test.py | 207 + tests/kafkatest/version.py | 5 + .../kafka/tools/TransactionalMessageCopier.java | 287 +++ 10 files changed, 706 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/1959835d/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 221816c..11068a7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -928,7 +928,9 @@ public class TransactionManager { Errors error = entry.getValue(); if (error == Errors.NONE) { pendingTxnOffsetCommits.remove(topicPartition); -} else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR) { +} else if (error == Errors.COORDINATOR_NOT_AVAILABLE +|| error == Errors.NOT_COORDINATOR +|| error == Errors.REQUEST_TIMED_OUT) { hadFailure = true; if (!coordinatorReloaded) { coordinatorReloaded = true; http://git-wip-us.apache.org/repos/asf/kafka/blob/1959835d/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 06e5608..782ffa5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -44,6 +44,7 @@ public class OffsetCommitResponse extends AbstractResponse { * Possible error codes: * * UNKNOWN_TOPIC_OR_PARTITION (3) + * REQUEST_TIMED_OUT (7) * OFFSET_METADATA_TOO_LARGE (12) * COORDINATOR_LOAD_IN_PROGRESS (14) * GROUP_COORDINATOR_NOT_AVAILABLE (15) http://git-wip-us.apache.org/repos/asf/kafka/blob/1959835d/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java index e7b349c..4c0f010 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java @@ -42,6 +42,7 @@ public class TxnOffsetCommitResponse extends AbstractResponse { // GroupAuthorizationFailed // InvalidCommitOffsetSize // TransactionalIdAuthorizationFailed +// RequestTimedOut private final Map errors; private final int throttleTimeMs; http://git-wip-us.apache.org/repos/asf/kafka/blob/1959835d/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala -- diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala
kafka git commit: KAFKA-5281; System tests for transactions
Repository: kafka Updated Branches: refs/heads/0.11.0 b42cefe2f -> dc15062ea KAFKA-5281; System tests for transactions Author: Apurva Mehta Reviewers: Jason Gustafson Closes #3149 from apurvam/KAFKA-5281-transactions-system-tests (cherry picked from commit 1959835d9e148f0eb6407b36ff96b334d5e785cb) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/dc15062e Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/dc15062e Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/dc15062e Branch: refs/heads/0.11.0 Commit: dc15062ea9c8f9db864e6114819966daa40f6a45 Parents: b42cefe Author: Apurva Mehta Authored: Thu Jun 1 10:25:29 2017 -0700 Committer: Jason Gustafson Committed: Thu Jun 1 10:27:01 2017 -0700 -- .../producer/internals/TransactionManager.java | 4 +- .../common/requests/OffsetCommitResponse.java | 1 + .../requests/TxnOffsetCommitResponse.java | 1 + ...nsactionMarkerRequestCompletionHandler.scala | 3 +- .../scala/kafka/tools/ConsoleConsumer.scala | 12 +- tests/kafkatest/services/console_consumer.py| 9 +- .../services/transactional_message_copier.py| 183 tests/kafkatest/tests/core/transactions_test.py | 207 + tests/kafkatest/version.py | 5 + .../kafka/tools/TransactionalMessageCopier.java | 287 +++ 10 files changed, 706 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/dc15062e/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 221816c..11068a7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -928,7 +928,9 @@ public class TransactionManager { Errors error = entry.getValue(); if (error == Errors.NONE) { pendingTxnOffsetCommits.remove(topicPartition); -} else if (error == Errors.COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR) { +} else if (error == Errors.COORDINATOR_NOT_AVAILABLE +|| error == Errors.NOT_COORDINATOR +|| error == Errors.REQUEST_TIMED_OUT) { hadFailure = true; if (!coordinatorReloaded) { coordinatorReloaded = true; http://git-wip-us.apache.org/repos/asf/kafka/blob/dc15062e/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 06e5608..782ffa5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -44,6 +44,7 @@ public class OffsetCommitResponse extends AbstractResponse { * Possible error codes: * * UNKNOWN_TOPIC_OR_PARTITION (3) + * REQUEST_TIMED_OUT (7) * OFFSET_METADATA_TOO_LARGE (12) * COORDINATOR_LOAD_IN_PROGRESS (14) * GROUP_COORDINATOR_NOT_AVAILABLE (15) http://git-wip-us.apache.org/repos/asf/kafka/blob/dc15062e/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java index e7b349c..4c0f010 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java @@ -42,6 +42,7 @@ public class TxnOffsetCommitResponse extends AbstractResponse { // GroupAuthorizationFailed // InvalidCommitOffsetSize // TransactionalIdAuthorizationFailed +// RequestTimedOut private final Map errors; private final int throttleTimeMs; http://git-wip-us.apache.org/repos/asf/kafka/blob/dc15062e/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandler.scala -- diff --g
kafka git commit: MINOR: Logging/debugging improvements for transactions
Repository: kafka Updated Branches: refs/heads/trunk 4c8ec795a -> 0c3e466eb MINOR: Logging/debugging improvements for transactions Author: Jason Gustafson Author: Apurva Mehta Reviewers: Apurva Mehta , Ismael Juma Closes #3185 from hachikuji/minor-transaction-logging-improvements Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/0c3e466e Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/0c3e466e Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/0c3e466e Branch: refs/heads/trunk Commit: 0c3e466eb035859659ce41404f3b71b577467dca Parents: 4c8ec79 Author: Jason Gustafson Authored: Thu Jun 1 19:08:32 2017 -0700 Committer: Jason Gustafson Committed: Thu Jun 1 19:08:32 2017 -0700 -- .../clients/consumer/internals/Fetcher.java | 11 ++- .../producer/internals/ProducerIdAndEpoch.java | 5 ++ .../producer/internals/RecordAccumulator.java | 6 +- .../clients/producer/internals/Sender.java | 35 - .../producer/internals/TransactionManager.java | 74 +--- .../requests/AddOffsetsToTxnResponse.java | 9 +++ .../requests/AddPartitionsToTxnResponse.java| 9 +++ .../kafka/common/requests/EndTxnResponse.java | 8 +++ .../kafka/common/requests/FetchResponse.java| 7 +- .../requests/FindCoordinatorResponse.java | 4 +- .../common/requests/InitProducerIdResponse.java | 9 +++ .../requests/TxnOffsetCommitResponse.java | 8 +++ .../transaction/TransactionCoordinator.scala| 13 ++-- core/src/main/scala/kafka/log/Log.scala | 7 +- .../scala/kafka/tools/DumpLogSegments.scala | 36 -- 15 files changed, 157 insertions(+), 84 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/0c3e466e/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index c2beff8..defbbb7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -223,6 +223,10 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { TopicPartition partition = entry.getKey(); long fetchOffset = request.fetchData().get(partition).fetchOffset; FetchResponse.PartitionData fetchData = entry.getValue(); + +log.debug("Fetch at offset {} for partition {} returned fetch data {}", fetchOffset, +partition, fetchData); + completedFetches.add(new CompletedFetch(partition, fetchOffset, fetchData, metricAggregator, resp.requestHeader().apiVersion())); } @@ -232,7 +236,7 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { @Override public void onFailure(RuntimeException e) { -log.debug("Fetch request to {} for partitions {} failed", fetchTarget, request.fetchData().keySet(), e); +log.debug("Fetch request {} to {} failed", request.fetchData(), fetchTarget, e); } }); } @@ -792,8 +796,9 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { Map requests = new HashMap<>(); for (Map.Entry> entry : fetchable.entrySet()) { Node node = entry.getKey(); -FetchRequest.Builder fetch = FetchRequest.Builder.forConsumer(this.maxWaitMs, this.minBytes, entry.getValue(), isolationLevel). -setMaxBytes(this.maxBytes); +FetchRequest.Builder fetch = FetchRequest.Builder.forConsumer(this.maxWaitMs, this.minBytes, +entry.getValue(), isolationLevel) +.setMaxBytes(this.maxBytes); requests.put(node, fetch); } return requests; http://git-wip-us.apache.org/repos/asf/kafka/blob/0c3e466e/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerIdAndEpoch.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerIdAndEpoch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerIdAndEpoch.java index 01d5e86..293bb51 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerIdA
kafka git commit: MINOR: Logging/debugging improvements for transactions
Repository: kafka Updated Branches: refs/heads/0.11.0 0be4d1af0 -> 4424534e9 MINOR: Logging/debugging improvements for transactions Author: Jason Gustafson Author: Apurva Mehta Reviewers: Apurva Mehta , Ismael Juma Closes #3185 from hachikuji/minor-transaction-logging-improvements (cherry picked from commit 0c3e466eb035859659ce41404f3b71b577467dca) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/4424534e Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/4424534e Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/4424534e Branch: refs/heads/0.11.0 Commit: 4424534e99a92f54a873771dc3f05f23f3487669 Parents: 0be4d1a Author: Jason Gustafson Authored: Thu Jun 1 19:08:32 2017 -0700 Committer: Jason Gustafson Committed: Thu Jun 1 19:08:52 2017 -0700 -- .../clients/consumer/internals/Fetcher.java | 11 ++- .../producer/internals/ProducerIdAndEpoch.java | 5 ++ .../producer/internals/RecordAccumulator.java | 6 +- .../clients/producer/internals/Sender.java | 35 - .../producer/internals/TransactionManager.java | 74 +--- .../requests/AddOffsetsToTxnResponse.java | 9 +++ .../requests/AddPartitionsToTxnResponse.java| 9 +++ .../kafka/common/requests/EndTxnResponse.java | 8 +++ .../kafka/common/requests/FetchResponse.java| 7 +- .../requests/FindCoordinatorResponse.java | 4 +- .../common/requests/InitProducerIdResponse.java | 9 +++ .../requests/TxnOffsetCommitResponse.java | 8 +++ .../transaction/TransactionCoordinator.scala| 13 ++-- core/src/main/scala/kafka/log/Log.scala | 7 +- .../scala/kafka/tools/DumpLogSegments.scala | 36 -- 15 files changed, 157 insertions(+), 84 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/4424534e/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index c2beff8..defbbb7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -223,6 +223,10 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { TopicPartition partition = entry.getKey(); long fetchOffset = request.fetchData().get(partition).fetchOffset; FetchResponse.PartitionData fetchData = entry.getValue(); + +log.debug("Fetch at offset {} for partition {} returned fetch data {}", fetchOffset, +partition, fetchData); + completedFetches.add(new CompletedFetch(partition, fetchOffset, fetchData, metricAggregator, resp.requestHeader().apiVersion())); } @@ -232,7 +236,7 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { @Override public void onFailure(RuntimeException e) { -log.debug("Fetch request to {} for partitions {} failed", fetchTarget, request.fetchData().keySet(), e); +log.debug("Fetch request {} to {} failed", request.fetchData(), fetchTarget, e); } }); } @@ -792,8 +796,9 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { Map requests = new HashMap<>(); for (Map.Entry> entry : fetchable.entrySet()) { Node node = entry.getKey(); -FetchRequest.Builder fetch = FetchRequest.Builder.forConsumer(this.maxWaitMs, this.minBytes, entry.getValue(), isolationLevel). -setMaxBytes(this.maxBytes); +FetchRequest.Builder fetch = FetchRequest.Builder.forConsumer(this.maxWaitMs, this.minBytes, +entry.getValue(), isolationLevel) +.setMaxBytes(this.maxBytes); requests.put(node, fetch); } return requests; http://git-wip-us.apache.org/repos/asf/kafka/blob/4424534e/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerIdAndEpoch.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerIdAndEpoch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerIdAndEpoch.java index 01d5
[2/2] kafka git commit: KAFKA-5283; Handle producer epoch/sequence overflow
KAFKA-5283; Handle producer epoch/sequence overflow - Producer sequence numbers should wrap around - Generate a new producerId if the producer epoch would overflow Author: Jason Gustafson Reviewers: Ismael Juma , Apurva Mehta , Guozhang Wang Closes #3183 from hachikuji/KAFKA-5283 Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1c882ee5 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1c882ee5 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1c882ee5 Branch: refs/heads/trunk Commit: 1c882ee5fb4ef2d256c914bd69239d58d9706108 Parents: 0c3e466 Author: Jason Gustafson Authored: Thu Jun 1 23:37:31 2017 -0700 Committer: Jason Gustafson Committed: Thu Jun 1 23:37:36 2017 -0700 -- .../kafka/common/record/DefaultRecord.java | 4 +- .../kafka/common/record/DefaultRecordBatch.java | 10 +- .../common/record/DefaultRecordBatchTest.java | 36 .../transaction/TransactionCoordinator.scala| 70 +++ .../TransactionMarkerChannelManager.scala | 4 +- ...nsactionMarkerRequestCompletionHandler.scala | 4 +- .../transaction/TransactionMetadata.scala | 131 - .../transaction/TransactionStateManager.scala | 78 .../scala/kafka/log/ProducerStateManager.scala | 18 +- .../kafka/api/TransactionsTest.scala| 10 +- .../TransactionCoordinatorTest.scala| 146 ++ .../TransactionMarkerChannelManagerTest.scala | 4 +- ...tionMarkerRequestCompletionHandlerTest.scala | 8 +- .../transaction/TransactionMetadataTest.scala | 188 +++ .../TransactionStateManagerTest.scala | 66 +++ .../kafka/log/ProducerStateManagerTest.scala| 27 +++ 16 files changed, 605 insertions(+), 199 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/1c882ee5/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java index 5972b42..8910b30 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java @@ -323,7 +323,9 @@ public class DefaultRecord implements Record { int offsetDelta = ByteUtils.readVarint(buffer); long offset = baseOffset + offsetDelta; -int sequence = baseSequence >= 0 ? baseSequence + offsetDelta : RecordBatch.NO_SEQUENCE; +int sequence = baseSequence >= 0 ? +DefaultRecordBatch.incrementSequence(baseSequence, offsetDelta) : +RecordBatch.NO_SEQUENCE; ByteBuffer key = null; int keySize = ByteUtils.readVarint(buffer); http://git-wip-us.apache.org/repos/asf/kafka/blob/1c882ee5/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index 7a0e530..c05cab8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java @@ -185,7 +185,9 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe int baseSequence = baseSequence(); if (baseSequence == RecordBatch.NO_SEQUENCE) return RecordBatch.NO_SEQUENCE; -return baseSequence() + lastOffsetDelta(); + +int delta = lastOffsetDelta(); +return incrementSequence(baseSequence, delta); } @Override @@ -462,6 +464,12 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe return RECORD_BATCH_OVERHEAD + DefaultRecord.recordSizeUpperBound(key, value, headers); } +static int incrementSequence(int baseSequence, int increment) { +if (baseSequence > Integer.MAX_VALUE - increment) +return increment - (Integer.MAX_VALUE - baseSequence) - 1; +return baseSequence + increment; +} + private abstract class RecordIterator implements CloseableIterator { private final Long logAppendTime; private final long baseOffset; http://git-wip-us.apache.org/repos/asf/kafka/blob/1c882ee5/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java -- diff --git a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java b/clients/src/test/java/org/apache
[2/2] kafka git commit: KAFKA-5283; Handle producer epoch/sequence overflow
KAFKA-5283; Handle producer epoch/sequence overflow - Producer sequence numbers should wrap around - Generate a new producerId if the producer epoch would overflow Author: Jason Gustafson Reviewers: Ismael Juma , Apurva Mehta , Guozhang Wang Closes #3183 from hachikuji/KAFKA-5283 (cherry picked from commit 1c882ee5fb4ef2d256c914bd69239d58d9706108) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/07cfcc53 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/07cfcc53 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/07cfcc53 Branch: refs/heads/0.11.0 Commit: 07cfcc53bcb16a015903e9045e3c04c908e86e52 Parents: 4424534 Author: Jason Gustafson Authored: Thu Jun 1 23:37:31 2017 -0700 Committer: Jason Gustafson Committed: Thu Jun 1 23:41:46 2017 -0700 -- .../kafka/common/record/DefaultRecord.java | 4 +- .../kafka/common/record/DefaultRecordBatch.java | 10 +- .../common/record/DefaultRecordBatchTest.java | 36 .../transaction/TransactionCoordinator.scala| 70 +++ .../TransactionMarkerChannelManager.scala | 4 +- ...nsactionMarkerRequestCompletionHandler.scala | 4 +- .../transaction/TransactionMetadata.scala | 131 - .../transaction/TransactionStateManager.scala | 78 .../scala/kafka/log/ProducerStateManager.scala | 18 +- .../kafka/api/TransactionsTest.scala| 10 +- .../TransactionCoordinatorTest.scala| 146 ++ .../TransactionMarkerChannelManagerTest.scala | 4 +- ...tionMarkerRequestCompletionHandlerTest.scala | 8 +- .../transaction/TransactionMetadataTest.scala | 188 +++ .../TransactionStateManagerTest.scala | 66 +++ .../kafka/log/ProducerStateManagerTest.scala| 27 +++ 16 files changed, 605 insertions(+), 199 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/07cfcc53/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java index 5972b42..8910b30 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java @@ -323,7 +323,9 @@ public class DefaultRecord implements Record { int offsetDelta = ByteUtils.readVarint(buffer); long offset = baseOffset + offsetDelta; -int sequence = baseSequence >= 0 ? baseSequence + offsetDelta : RecordBatch.NO_SEQUENCE; +int sequence = baseSequence >= 0 ? +DefaultRecordBatch.incrementSequence(baseSequence, offsetDelta) : +RecordBatch.NO_SEQUENCE; ByteBuffer key = null; int keySize = ByteUtils.readVarint(buffer); http://git-wip-us.apache.org/repos/asf/kafka/blob/07cfcc53/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java index 7a0e530..c05cab8 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java @@ -185,7 +185,9 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe int baseSequence = baseSequence(); if (baseSequence == RecordBatch.NO_SEQUENCE) return RecordBatch.NO_SEQUENCE; -return baseSequence() + lastOffsetDelta(); + +int delta = lastOffsetDelta(); +return incrementSequence(baseSequence, delta); } @Override @@ -462,6 +464,12 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe return RECORD_BATCH_OVERHEAD + DefaultRecord.recordSizeUpperBound(key, value, headers); } +static int incrementSequence(int baseSequence, int increment) { +if (baseSequence > Integer.MAX_VALUE - increment) +return increment - (Integer.MAX_VALUE - baseSequence) - 1; +return baseSequence + increment; +} + private abstract class RecordIterator implements CloseableIterator { private final Long logAppendTime; private final long baseOffset; http://git-wip-us.apache.org/repos/asf/kafka/blob/07cfcc53/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java -- diff --git a/clients/src/te
[1/2] kafka git commit: KAFKA-5283; Handle producer epoch/sequence overflow
Repository: kafka Updated Branches: refs/heads/0.11.0 4424534e9 -> 07cfcc53b http://git-wip-us.apache.org/repos/asf/kafka/blob/07cfcc53/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala -- diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 2094528..54246c4 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -98,13 +98,13 @@ class TransactionStateManagerTest { def testAddGetPids() { transactionManager.addLoadedTransactionsToCache(partitionId, coordinatorEpoch, new Pool[String, TransactionMetadata]()) -assertEquals(Right(None), transactionManager.getAndMaybeAddTransactionState(transactionalId1)) +assertEquals(Right(None), transactionManager.getTransactionState(transactionalId1)) assertEquals(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata1))), - transactionManager.getAndMaybeAddTransactionState(transactionalId1, Some(txnMetadata1))) + transactionManager.putTransactionStateIfNotExists(transactionalId1, txnMetadata1)) assertEquals(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata1))), - transactionManager.getAndMaybeAddTransactionState(transactionalId1)) + transactionManager.getTransactionState(transactionalId1)) assertEquals(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata1))), - transactionManager.getAndMaybeAddTransactionState(transactionalId1, Some(txnMetadata2))) + transactionManager.putTransactionStateIfNotExists(transactionalId1, txnMetadata2)) } @Test @@ -160,11 +160,11 @@ class TransactionStateManagerTest { prepareTxnLog(topicPartition, startOffset, records) // this partition should not be part of the owned partitions -transactionManager.getAndMaybeAddTransactionState(transactionalId1).fold( +transactionManager.getTransactionState(transactionalId1).fold( err => assertEquals(Errors.NOT_COORDINATOR, err), _ => fail(transactionalId1 + "'s transaction state is already in the cache") ) -transactionManager.getAndMaybeAddTransactionState(transactionalId2).fold( +transactionManager.getTransactionState(transactionalId2).fold( err => assertEquals(Errors.NOT_COORDINATOR, err), _ => fail(transactionalId2 + "'s transaction state is already in the cache") ) @@ -174,16 +174,16 @@ class TransactionStateManagerTest { // let the time advance to trigger the background thread loading scheduler.tick() -transactionManager.getAndMaybeAddTransactionState(transactionalId1).fold( +transactionManager.getTransactionState(transactionalId1).fold( err => fail(transactionalId1 + "'s transaction state access returns error " + err), entry => entry.getOrElse(fail(transactionalId1 + "'s transaction state was not loaded into the cache")) ) -val cachedPidMetadata1 = transactionManager.getAndMaybeAddTransactionState(transactionalId1).fold( +val cachedPidMetadata1 = transactionManager.getTransactionState(transactionalId1).fold( err => fail(transactionalId1 + "'s transaction state access returns error " + err), entry => entry.getOrElse(fail(transactionalId1 + "'s transaction state was not loaded into the cache")) ) -val cachedPidMetadata2 = transactionManager.getAndMaybeAddTransactionState(transactionalId2).fold( +val cachedPidMetadata2 = transactionManager.getTransactionState(transactionalId2).fold( err => fail(transactionalId2 + "'s transaction state access returns error " + err), entry => entry.getOrElse(fail(transactionalId2 + "'s transaction state was not loaded into the cache")) ) @@ -197,11 +197,11 @@ class TransactionStateManagerTest { // let the time advance to trigger the background thread removing scheduler.tick() -transactionManager.getAndMaybeAddTransactionState(transactionalId1).fold( +transactionManager.getTransactionState(transactionalId1).fold( err => assertEquals(Errors.NOT_COORDINATOR, err), _ => fail(transactionalId1 + "'s transaction state is still in the cache") ) -transactionManager.getAndMaybeAddTransactionState(transactionalId2).fold( +transactionManager.getTransactionState(transactionalId2).fold( err => assertEquals(Errors.NOT_COORDINATOR, err), _ => fail(transactionalId2 + "'s transaction state is still in the cache") ) @@ -212,7 +212,7 @@ class TransactionStateManagerTest { transactionManager.addLoadedTransactionsToCache(partitionId, coordinatorEpoch, new Pool[String, TransactionMetadata]()) // first insert the in
[1/2] kafka git commit: KAFKA-5283; Handle producer epoch/sequence overflow
Repository: kafka Updated Branches: refs/heads/trunk 0c3e466eb -> 1c882ee5f http://git-wip-us.apache.org/repos/asf/kafka/blob/1c882ee5/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala -- diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 2094528..54246c4 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -98,13 +98,13 @@ class TransactionStateManagerTest { def testAddGetPids() { transactionManager.addLoadedTransactionsToCache(partitionId, coordinatorEpoch, new Pool[String, TransactionMetadata]()) -assertEquals(Right(None), transactionManager.getAndMaybeAddTransactionState(transactionalId1)) +assertEquals(Right(None), transactionManager.getTransactionState(transactionalId1)) assertEquals(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata1))), - transactionManager.getAndMaybeAddTransactionState(transactionalId1, Some(txnMetadata1))) + transactionManager.putTransactionStateIfNotExists(transactionalId1, txnMetadata1)) assertEquals(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata1))), - transactionManager.getAndMaybeAddTransactionState(transactionalId1)) + transactionManager.getTransactionState(transactionalId1)) assertEquals(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata1))), - transactionManager.getAndMaybeAddTransactionState(transactionalId1, Some(txnMetadata2))) + transactionManager.putTransactionStateIfNotExists(transactionalId1, txnMetadata2)) } @Test @@ -160,11 +160,11 @@ class TransactionStateManagerTest { prepareTxnLog(topicPartition, startOffset, records) // this partition should not be part of the owned partitions -transactionManager.getAndMaybeAddTransactionState(transactionalId1).fold( +transactionManager.getTransactionState(transactionalId1).fold( err => assertEquals(Errors.NOT_COORDINATOR, err), _ => fail(transactionalId1 + "'s transaction state is already in the cache") ) -transactionManager.getAndMaybeAddTransactionState(transactionalId2).fold( +transactionManager.getTransactionState(transactionalId2).fold( err => assertEquals(Errors.NOT_COORDINATOR, err), _ => fail(transactionalId2 + "'s transaction state is already in the cache") ) @@ -174,16 +174,16 @@ class TransactionStateManagerTest { // let the time advance to trigger the background thread loading scheduler.tick() -transactionManager.getAndMaybeAddTransactionState(transactionalId1).fold( +transactionManager.getTransactionState(transactionalId1).fold( err => fail(transactionalId1 + "'s transaction state access returns error " + err), entry => entry.getOrElse(fail(transactionalId1 + "'s transaction state was not loaded into the cache")) ) -val cachedPidMetadata1 = transactionManager.getAndMaybeAddTransactionState(transactionalId1).fold( +val cachedPidMetadata1 = transactionManager.getTransactionState(transactionalId1).fold( err => fail(transactionalId1 + "'s transaction state access returns error " + err), entry => entry.getOrElse(fail(transactionalId1 + "'s transaction state was not loaded into the cache")) ) -val cachedPidMetadata2 = transactionManager.getAndMaybeAddTransactionState(transactionalId2).fold( +val cachedPidMetadata2 = transactionManager.getTransactionState(transactionalId2).fold( err => fail(transactionalId2 + "'s transaction state access returns error " + err), entry => entry.getOrElse(fail(transactionalId2 + "'s transaction state was not loaded into the cache")) ) @@ -197,11 +197,11 @@ class TransactionStateManagerTest { // let the time advance to trigger the background thread removing scheduler.tick() -transactionManager.getAndMaybeAddTransactionState(transactionalId1).fold( +transactionManager.getTransactionState(transactionalId1).fold( err => assertEquals(Errors.NOT_COORDINATOR, err), _ => fail(transactionalId1 + "'s transaction state is still in the cache") ) -transactionManager.getAndMaybeAddTransactionState(transactionalId2).fold( +transactionManager.getTransactionState(transactionalId2).fold( err => assertEquals(Errors.NOT_COORDINATOR, err), _ => fail(transactionalId2 + "'s transaction state is still in the cache") ) @@ -212,7 +212,7 @@ class TransactionStateManagerTest { transactionManager.addLoadedTransactionsToCache(partitionId, coordinatorEpoch, new Pool[String, TransactionMetadata]()) // first insert the ini
kafka git commit: KAFKA-5364; Don't fail producer if drained partition is not yet in transaction
Repository: kafka Updated Branches: refs/heads/0.11.0 07cfcc53b -> da6da5356 KAFKA-5364; Don't fail producer if drained partition is not yet in transaction Due to the async nature of the producer, it is possible to attempt to drain a messages whose partition hasn't been added to the transaction yet. Before this patch, we considered this a fatal error. However, it is only in error if the partition isn't in the queue to be sent to the coordinator. This patch updates the logic so that we only fail the producer if the partition would never be added to the transaction. If the partition of the batch is yet to be added, we will simply wait for the partition to be added to the transaction before sending the batch to the broker. Author: Apurva Mehta Reviewers: Guozhang Wang , Jason Gustafson Closes #3202 from apurvam/KAFKA-5364-ensure-partitions-added-to-txn-before-send (cherry picked from commit 673ab671e6d72f48fcc98de0b73564983c34e752) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/da6da535 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/da6da535 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/da6da535 Branch: refs/heads/0.11.0 Commit: da6da53563066519c587c751c1b16b65760fe096 Parents: 07cfcc5 Author: Apurva Mehta Authored: Fri Jun 2 00:53:21 2017 -0700 Committer: Jason Gustafson Committed: Fri Jun 2 00:53:39 2017 -0700 -- .../producer/internals/TransactionManager.java | 34 +- .../internals/TransactionManagerTest.java | 111 ++- 2 files changed, 139 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/da6da535/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 9d9deac..0a69e02 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -234,10 +234,17 @@ public class TransactionManager { } public synchronized boolean ensurePartitionAdded(TopicPartition tp) { -if (isInTransaction() && !partitionsInTransaction.contains(tp)) { -transitionToFatalError(new IllegalStateException("Attempted to dequeue a record batch to send " + -"for partition " + tp + ", which hasn't been added to the transaction yet")); +if (hasFatalError()) return false; +if (isInTransaction() || hasAbortableError()) { +// We should enter this branch in an error state because if this partition is already in the transaction, +// there is a chance that the corresponding batch is in retry. So we must let it completely flush. +if (!(partitionsInTransaction.contains(tp) || isPartitionPending(tp))) { +transitionToFatalError(new IllegalStateException("Attempted to dequeue a record batch to send " + +"for partition " + tp + ", which would never be added to the transaction.")); +return false; +} +return partitionsInTransaction.contains(tp); } return true; } @@ -416,6 +423,16 @@ public class TransactionManager { return inFlightRequestCorrelationId != NO_INFLIGHT_REQUEST_CORRELATION_ID; } +// visible for testing. +boolean hasFatalError() { +return currentState == State.FATAL_ERROR; +} + +// visible for testing. +boolean hasAbortableError() { +return currentState == State.ABORTABLE_ERROR; +} + // visible for testing synchronized boolean transactionContainsPartition(TopicPartition topicPartition) { return isInTransaction() && partitionsInTransaction.contains(topicPartition); @@ -431,6 +448,10 @@ public class TransactionManager { return isTransactional() && currentState == State.READY; } +private synchronized boolean isPartitionPending(TopicPartition tp) { +return isInTransaction() && (pendingPartitionsInTransaction.contains(tp) || newPartitionsInTransaction.contains(tp)); +} + private void transitionTo(State target) { transitionTo(target, null); } @@ -448,7 +469,12 @@ public class TransactionManager { lastError = null; } -log.debug("{}Transition from state {} to {}", logPrefix, currentState, target); +if (lastError != null) +log.error("{}Transition from state {} to error stat
kafka git commit: KAFKA-5364; Don't fail producer if drained partition is not yet in transaction
Repository: kafka Updated Branches: refs/heads/trunk 1c882ee5f -> 673ab671e KAFKA-5364; Don't fail producer if drained partition is not yet in transaction Due to the async nature of the producer, it is possible to attempt to drain a messages whose partition hasn't been added to the transaction yet. Before this patch, we considered this a fatal error. However, it is only in error if the partition isn't in the queue to be sent to the coordinator. This patch updates the logic so that we only fail the producer if the partition would never be added to the transaction. If the partition of the batch is yet to be added, we will simply wait for the partition to be added to the transaction before sending the batch to the broker. Author: Apurva Mehta Reviewers: Guozhang Wang , Jason Gustafson Closes #3202 from apurvam/KAFKA-5364-ensure-partitions-added-to-txn-before-send Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/673ab671 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/673ab671 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/673ab671 Branch: refs/heads/trunk Commit: 673ab671e6d72f48fcc98de0b73564983c34e752 Parents: 1c882ee Author: Apurva Mehta Authored: Fri Jun 2 00:53:21 2017 -0700 Committer: Jason Gustafson Committed: Fri Jun 2 00:53:21 2017 -0700 -- .../producer/internals/TransactionManager.java | 34 +- .../internals/TransactionManagerTest.java | 111 ++- 2 files changed, 139 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/673ab671/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 9d9deac..0a69e02 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -234,10 +234,17 @@ public class TransactionManager { } public synchronized boolean ensurePartitionAdded(TopicPartition tp) { -if (isInTransaction() && !partitionsInTransaction.contains(tp)) { -transitionToFatalError(new IllegalStateException("Attempted to dequeue a record batch to send " + -"for partition " + tp + ", which hasn't been added to the transaction yet")); +if (hasFatalError()) return false; +if (isInTransaction() || hasAbortableError()) { +// We should enter this branch in an error state because if this partition is already in the transaction, +// there is a chance that the corresponding batch is in retry. So we must let it completely flush. +if (!(partitionsInTransaction.contains(tp) || isPartitionPending(tp))) { +transitionToFatalError(new IllegalStateException("Attempted to dequeue a record batch to send " + +"for partition " + tp + ", which would never be added to the transaction.")); +return false; +} +return partitionsInTransaction.contains(tp); } return true; } @@ -416,6 +423,16 @@ public class TransactionManager { return inFlightRequestCorrelationId != NO_INFLIGHT_REQUEST_CORRELATION_ID; } +// visible for testing. +boolean hasFatalError() { +return currentState == State.FATAL_ERROR; +} + +// visible for testing. +boolean hasAbortableError() { +return currentState == State.ABORTABLE_ERROR; +} + // visible for testing synchronized boolean transactionContainsPartition(TopicPartition topicPartition) { return isInTransaction() && partitionsInTransaction.contains(topicPartition); @@ -431,6 +448,10 @@ public class TransactionManager { return isTransactional() && currentState == State.READY; } +private synchronized boolean isPartitionPending(TopicPartition tp) { +return isInTransaction() && (pendingPartitionsInTransaction.contains(tp) || newPartitionsInTransaction.contains(tp)); +} + private void transitionTo(State target) { transitionTo(target, null); } @@ -448,7 +469,12 @@ public class TransactionManager { lastError = null; } -log.debug("{}Transition from state {} to {}", logPrefix, currentState, target); +if (lastError != null) +log.error("{}Transition from state {} to error state {}", logPrefix, currentState, +target, lastError); +else +log
kafka git commit: KAFKA-5322; Add `OPERATION_NOT_ATTEMPTED` error code to resolve AddPartitionsToTxn inconsistency
Repository: kafka Updated Branches: refs/heads/trunk 4959444af -> a318b1512 KAFKA-5322; Add `OPERATION_NOT_ATTEMPTED` error code to resolve AddPartitionsToTxn inconsistency In the `AddPartitionsToTxn` request handling, if even one partition fails authorization checks, the entire request is essentially failed. However, the `AddPartitionsToTxnResponse` today will only contain the error codes for the topics which failed authorization. It will have no error code for the topics which succeeded, making it inconsistent with other APIs. This patch adds a new error code `OPERATION_NOT_ATTEMPTED` which is returned for the successful partitions to indicate that they were not added to the transaction. Author: Apurva Mehta Reviewers: Ismael Juma , Jason Gustafson Closes #3204 from apurvam/KAFKA-5322-add-operation-not-attempted-for-add-partitions Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/a318b151 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/a318b151 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/a318b151 Branch: refs/heads/trunk Commit: a318b1512900d29dc9ff0082f52f467999ba69b8 Parents: 4959444 Author: Apurva Mehta Authored: Fri Jun 2 14:18:54 2017 -0700 Committer: Jason Gustafson Committed: Fri Jun 2 14:21:50 2017 -0700 -- .../producer/internals/TransactionManager.java | 3 + .../errors/OperationNotAttemptedException.java | 27 +++ .../apache/kafka/common/protocol/Errors.java| 13 +++- .../internals/TransactionManagerTest.java | 26 --- .../src/main/scala/kafka/server/KafkaApis.scala | 11 +-- .../server/AddPartitionsToTxnRequestTest.scala | 76 6 files changed, 141 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/a318b151/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 0a69e02..aade83e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -734,6 +734,9 @@ public class TransactionManager { return; } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { unauthorizedTopics.add(topicPartition.topic()); +} else if (error == Errors.OPERATION_NOT_ATTEMPTED) { +log.debug("{}Did not attempt to add partition {} to transaction because other partitions in the batch had errors.", logPrefix, topicPartition); +hasPartitionErrors = true; } else { log.error("{}Could not add partition {} due to unexpected error {}", logPrefix, topicPartition, error); hasPartitionErrors = true; http://git-wip-us.apache.org/repos/asf/kafka/blob/a318b151/clients/src/main/java/org/apache/kafka/common/errors/OperationNotAttemptedException.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OperationNotAttemptedException.java b/clients/src/main/java/org/apache/kafka/common/errors/OperationNotAttemptedException.java new file mode 100644 index 000..96df321 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/OperationNotAttemptedException.java @@ -0,0 +1,27 @@ +/* + * 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.common.errors; + +/** + * Indicates that the broker did not attempt to execute this operation. This may happen for batched RPCs where some + * operations in the batch failed, causing the broker to respond without trying the rest. + */ +public class OperationNotAttemptedException extends ApiExcepti
kafka git commit: KAFKA-5322; Add `OPERATION_NOT_ATTEMPTED` error code to resolve AddPartitionsToTxn inconsistency
Repository: kafka Updated Branches: refs/heads/0.11.0 bf8378a84 -> 27a8b4fe5 KAFKA-5322; Add `OPERATION_NOT_ATTEMPTED` error code to resolve AddPartitionsToTxn inconsistency In the `AddPartitionsToTxn` request handling, if even one partition fails authorization checks, the entire request is essentially failed. However, the `AddPartitionsToTxnResponse` today will only contain the error codes for the topics which failed authorization. It will have no error code for the topics which succeeded, making it inconsistent with other APIs. This patch adds a new error code `OPERATION_NOT_ATTEMPTED` which is returned for the successful partitions to indicate that they were not added to the transaction. Author: Apurva Mehta Reviewers: Ismael Juma , Jason Gustafson Closes #3204 from apurvam/KAFKA-5322-add-operation-not-attempted-for-add-partitions (cherry picked from commit a318b1512900d29dc9ff0082f52f467999ba69b8) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/27a8b4fe Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/27a8b4fe Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/27a8b4fe Branch: refs/heads/0.11.0 Commit: 27a8b4fe540ac755986f8ea5fff9f496b5e6254e Parents: bf8378a Author: Apurva Mehta Authored: Fri Jun 2 14:18:54 2017 -0700 Committer: Jason Gustafson Committed: Fri Jun 2 14:23:34 2017 -0700 -- .../producer/internals/TransactionManager.java | 3 + .../errors/OperationNotAttemptedException.java | 27 +++ .../apache/kafka/common/protocol/Errors.java| 13 +++- .../internals/TransactionManagerTest.java | 26 --- .../src/main/scala/kafka/server/KafkaApis.scala | 11 +-- .../server/AddPartitionsToTxnRequestTest.scala | 76 6 files changed, 141 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/27a8b4fe/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index 0a69e02..aade83e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -734,6 +734,9 @@ public class TransactionManager { return; } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { unauthorizedTopics.add(topicPartition.topic()); +} else if (error == Errors.OPERATION_NOT_ATTEMPTED) { +log.debug("{}Did not attempt to add partition {} to transaction because other partitions in the batch had errors.", logPrefix, topicPartition); +hasPartitionErrors = true; } else { log.error("{}Could not add partition {} due to unexpected error {}", logPrefix, topicPartition, error); hasPartitionErrors = true; http://git-wip-us.apache.org/repos/asf/kafka/blob/27a8b4fe/clients/src/main/java/org/apache/kafka/common/errors/OperationNotAttemptedException.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OperationNotAttemptedException.java b/clients/src/main/java/org/apache/kafka/common/errors/OperationNotAttemptedException.java new file mode 100644 index 000..96df321 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/OperationNotAttemptedException.java @@ -0,0 +1,27 @@ +/* + * 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.common.errors; + +/** + * Indicates that the broker did not attempt to execute this operation. This may happen for batched RPCs where some + * operations in the batch failed, causing the broker t
kafka git commit: HOTFIX: Reinstate the placeholder for logPrefix in TransactionManager
Repository: kafka Updated Branches: refs/heads/0.11.0 27a8b4fe5 -> f2673eb37 HOTFIX: Reinstate the placeholder for logPrefix in TransactionManager Author: Apurva Mehta Reviewers: Jason Gustafson Closes #3216 from apurvam/HOTFIX-logging-bug-in-transaction-manager (cherry picked from commit 0b107e5f752f4f7d1d6dc97891cb9569c3db5b3d) Signed-off-by: Jason Gustafson Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/f2673eb3 Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/f2673eb3 Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/f2673eb3 Branch: refs/heads/0.11.0 Commit: f2673eb37818f5e74fdbec90b5277822ce247bd3 Parents: 27a8b4f Author: Apurva Mehta Authored: Fri Jun 2 15:29:09 2017 -0700 Committer: Jason Gustafson Committed: Fri Jun 2 15:29:21 2017 -0700 -- .../kafka/clients/producer/internals/TransactionManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/f2673eb3/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index aade83e..dad6b5d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -473,7 +473,7 @@ public class TransactionManager { log.error("{}Transition from state {} to error state {}", logPrefix, currentState, target, lastError); else -log.debug("Transition from state {} to {}", logPrefix, currentState, target); +log.debug("{}Transition from state {} to {}", logPrefix, currentState, target); currentState = target; }
kafka git commit: HOTFIX: Reinstate the placeholder for logPrefix in TransactionManager
Repository: kafka Updated Branches: refs/heads/trunk a318b1512 -> 0b107e5f7 HOTFIX: Reinstate the placeholder for logPrefix in TransactionManager Author: Apurva Mehta Reviewers: Jason Gustafson Closes #3216 from apurvam/HOTFIX-logging-bug-in-transaction-manager Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/0b107e5f Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/0b107e5f Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/0b107e5f Branch: refs/heads/trunk Commit: 0b107e5f752f4f7d1d6dc97891cb9569c3db5b3d Parents: a318b15 Author: Apurva Mehta Authored: Fri Jun 2 15:29:09 2017 -0700 Committer: Jason Gustafson Committed: Fri Jun 2 15:29:09 2017 -0700 -- .../kafka/clients/producer/internals/TransactionManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/0b107e5f/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java -- diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index aade83e..dad6b5d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -473,7 +473,7 @@ public class TransactionManager { log.error("{}Transition from state {} to error state {}", logPrefix, currentState, target, lastError); else -log.debug("Transition from state {} to {}", logPrefix, currentState, target); +log.debug("{}Transition from state {} to {}", logPrefix, currentState, target); currentState = target; }
kafka git commit: MINOR: Remove redundant volatile write in RecordHeaders
Repository: kafka Updated Branches: refs/heads/trunk d7ec9feaf -> c7bc8f7d8 MINOR: Remove redundant volatile write in RecordHeaders The JMH benchmark included shows that the redundant volatile write causes the constructor of `ProducerRecord` to take more than 50% longer: ProducerRecordBenchmark.constructorBenchmark avgt 15 24.136 ± 1.458 ns/op (before) ProducerRecordBenchmark.constructorBenchmark avgt 15 14.904 ± 0.231 ns/op (after) Author: Ismael Juma Reviewers: Jason Gustafson Closes #3233 from ijuma/remove-volatile-write-in-records-header-constructor Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/c7bc8f7d Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/c7bc8f7d Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/c7bc8f7d Branch: refs/heads/trunk Commit: c7bc8f7d8c0101103853b5f6778edb04d1f41b76 Parents: d7ec9fe Author: Ismael Juma Authored: Sun Jun 4 10:48:34 2017 -0700 Committer: Jason Gustafson Committed: Sun Jun 4 10:48:34 2017 -0700 -- checkstyle/import-control.xml | 1 + .../common/header/internals/RecordHeaders.java | 2 +- gradle/findbugs-exclude.xml | 1 + .../jmh/producer/ProducerRecordBenchmark.java | 47 4 files changed, 50 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/kafka/blob/c7bc8f7d/checkstyle/import-control.xml -- diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index f83f6e7..26f4a77 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -216,6 +216,7 @@ + http://git-wip-us.apache.org/repos/asf/kafka/blob/c7bc8f7d/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java -- diff --git a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java index afd991f..141c972 100644 --- a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java +++ b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java @@ -30,7 +30,7 @@ import org.apache.kafka.common.utils.AbstractIterator; public class RecordHeaders implements Headers { private final List headers; -private volatile boolean isReadOnly = false; +private volatile boolean isReadOnly; public RecordHeaders() { this((Iterable) null); http://git-wip-us.apache.org/repos/asf/kafka/blob/c7bc8f7d/gradle/findbugs-exclude.xml -- diff --git a/gradle/findbugs-exclude.xml b/gradle/findbugs-exclude.xml index eec9268..9262a78 100644 --- a/gradle/findbugs-exclude.xml +++ b/gradle/findbugs-exclude.xml @@ -165,6 +165,7 @@ For a detailed description of findbugs bug categories, see http://findbugs.sourc + http://git-wip-us.apache.org/repos/asf/kafka/blob/c7bc8f7d/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java -- diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java new file mode 100644 index 000..e7b3982 --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java @@ -0,0 +1,47 @@ +/* + * 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.jmh.producer; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.