/** * A {@link ProducerBatch} configured using a timestamp preceding its create time is interpreted correctly * * as not expired by {@link ProducerBatch#hasReachedDeliveryTimeout(long, long)}. */ @Test public void testBatchExpirationAfterReenqueue() { ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); // Set batch.retry = true batch.reenqueued(now); // Set `now` to 2ms before the create time. assertFalse(batch.hasReachedDeliveryTimeout(10240, now - 2L)); }
/** * Abort any batches which have not been drained */ void abortUndrainedBatches(RuntimeException reason) { for (ProducerBatch batch : incomplete.copyAll()) { Deque<ProducerBatch> dq = getDeque(batch.topicPartition); boolean aborted = false; synchronized (dq) { if ((transactionManager != null && !batch.hasSequence()) || (transactionManager == null && !batch.isClosed())) { aborted = true; batch.abortRecordAppends(); dq.remove(batch); } } if (aborted) { batch.abort(reason); deallocate(batch); } } }
private synchronized void startSequencesAtBeginning(TopicPartition topicPartition) { int sequence = 0; for (ProducerBatch inFlightBatch : inflightBatchesBySequence.get(topicPartition)) { log.info("Resetting sequence number of batch with current sequence {} for partition {} to {}", inFlightBatch.baseSequence(), inFlightBatch.topicPartition, sequence); inFlightBatch.resetProducerState(new ProducerIdAndEpoch(inFlightBatch.producerId(), inFlightBatch.producerEpoch()), sequence, inFlightBatch.isTransactional()); sequence += inFlightBatch.recordCount; } setNextSequence(topicPartition, sequence); lastAckedSequence.remove(topicPartition); }
/** * Deallocate the record batch */ public void deallocate(ProducerBatch batch) { incomplete.remove(batch); // 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()); }
/** * We can retry a send if the error is transient and the number of attempts taken is fewer than the maximum allowed. * We can also retry OutOfOrderSequence exceptions for future batches, since if the first batch has failed, the * future batches are certain to fail with an OutOfOrderSequence exception. */ private boolean canRetry(ProducerBatch batch, ProduceResponse.PartitionResponse response, long now) { return !batch.hasReachedDeliveryTimeout(accumulator.getDeliveryTimeoutMs(), now) && batch.attempts() < this.retries && !batch.isDone() && ((response.error.exception() instanceof RetriableException) || (transactionManager != null && transactionManager.canRetry(response, batch))); }
boolean backoff = first.attempts() > 0 && first.waitedTimeMs(now) < retryBackoffMs; if (size + first.estimatedSizeInBytes() > maxSize && !ready.isEmpty()) { transactionManager != null ? transactionManager.producerIdAndEpoch() : null; ProducerBatch batch = deque.pollFirst(); if (producerIdAndEpoch != null && !batch.hasSequence()) { batch.setProducerState(producerIdAndEpoch, transactionManager.sequenceNumber(batch.topicPartition), isTransactional); transactionManager.incrementSequenceNumber(batch.topicPartition, batch.recordCount); log.debug("Assigned producerId {} and producerEpoch {} to batch with base sequence " + "{} being sent to partition {}", producerIdAndEpoch.producerId, producerIdAndEpoch.epoch, batch.baseSequence(), tp); batch.close(); size += batch.records().sizeInBytes(); ready.add(batch); batch.drained(now);
compressionType, TimestampType.CREATE_TIME, 0L); ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), builder, now); while (true) { FutureRecordMetadata future = batch.tryAppend(now, "hi".getBytes(), "there".getBytes(), Record.EMPTY_HEADERS, null, now); if (future == null) Deque<ProducerBatch> batches = batch.split(512); assertTrue(batches.size() >= 2); assertEquals(magic, splitProducerBatch.magic()); assertTrue(splitProducerBatch.isSplitBatch()); for (RecordBatch splitBatch : splitProducerBatch.records().batches()) { assertEquals(magic, splitBatch.magic()); assertEquals(0L, splitBatch.baseOffset());
Thunk thunk = thunkIter.next(); if (batch == null) batch = createBatchOffAccumulatorForRecord(record, splitBatchSize); if (!batch.tryAppendForSplit(record.timestamp(), record.key(), record.value(), record.headers(), thunk)) { batches.add(batch); batch = createBatchOffAccumulatorForRecord(record, splitBatchSize); batch.tryAppendForSplit(record.timestamp(), record.key(), record.value(), record.headers(), thunk); produceFuture.done(); if (hasSequence()) { int sequence = baseSequence(); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(producerId(), producerEpoch()); for (ProducerBatch newBatch : batches) { newBatch.setProducerState(producerIdAndEpoch, sequence, isTransactional()); sequence += newBatch.recordCount;
Errors error = response.error; if (error == Errors.MESSAGE_TOO_LARGE && batch.recordCount > 1 && !batch.isDone() && (batch.magic() >= RecordBatch.MAGIC_VALUE_V2 || batch.isCompressed())) { correlationId, batch.topicPartition, this.retries - batch.attempts(), error); if (transactionManager != null) correlationId, batch.topicPartition, this.retries - batch.attempts() - 1, error); if (transactionManager == null) { reenqueueBatch(batch, now); } else if (transactionManager.hasProducerIdAndEpoch(batch.producerId(), batch.producerEpoch())) { batch.topicPartition, batch.producerId(), batch.baseSequence()); reenqueueBatch(batch, now); } else { failBatch(batch, response, new OutOfOrderSequenceException("Attempted to retry sending a " + "batch but the producer id changed from " + batch.producerId() + " to " + transactionManager.producerIdAndEpoch().producerId + " in the mean time. This batch will be dropped."), false); failBatch(batch, response, exception, batch.attempts() < this.retries);
TimestampType.CREATE_TIME, 0L); ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), builder, now); Header header = new RecordHeader("header-key", "header-value".getBytes()); FutureRecordMetadata future = batch.tryAppend( now, "hi".getBytes(), "there".getBytes(), new Header[]{header}, null, now); Deque<ProducerBatch> batches = batch.split(200); assertTrue("This batch should be split to multiple small batches.", batches.size() >= 2); for (RecordBatch splitBatch : splitProducerBatch.records().batches()) { for (Record record : splitBatch) { assertTrue("Header size should be 1.", record.headers().length == 1);
@Test public void testBatchAbort() throws Exception { ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); MockCallback callback = new MockCallback(); FutureRecordMetadata future = batch.tryAppend(now, null, new byte[10], Record.EMPTY_HEADERS, callback, now); KafkaException exception = new KafkaException(); batch.abort(exception); assertTrue(future.isDone()); assertEquals(1, callback.invocations); assertEquals(exception, callback.exception); assertNull(callback.metadata); // subsequent completion should be ignored assertFalse(batch.done(500L, 2342342341L, null)); assertFalse(batch.done(-1, -1, new KafkaException())); assertEquals(1, callback.invocations); assertTrue(future.isDone()); try { future.get(); fail("Future should have thrown"); } catch (ExecutionException e) { assertEquals(exception, e.getCause()); } }
ProducerBatch batch = new ProducerBatch(tp1, builder, now, true); Future<RecordMetadata> future1 = batch.tryAppend(now, null, value, Record.EMPTY_HEADERS, cb, now); Future<RecordMetadata> future2 = batch.tryAppend(now, null, value, Record.EMPTY_HEADERS, cb, now); assertNotNull(future1); assertNotNull(future2); batch.close(); assertFalse(drained.isEmpty()); assertFalse(drained.get(node1.id()).isEmpty()); drained.get(node1.id()).get(0).done(acked.get(), 100L, null); assertEquals("The first message should have been acked.", 1, acked.get()); assertTrue(future1.isDone()); assertFalse(drained.isEmpty()); assertFalse(drained.get(node1.id()).isEmpty()); drained.get(node1.id()).get(0).done(acked.get(), 100L, null); assertEquals("Both message should have been acked.", 2, acked.get()); assertTrue(future2.isDone());
private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionResponse response) { if (transactionManager != null) { if (transactionManager.hasProducerIdAndEpoch(batch.producerId(), batch.producerEpoch())) { transactionManager .maybeUpdateLastAckedSequence(batch.topicPartition, batch.baseSequence() + batch.recordCount - 1); log.debug("ProducerId: {}; Set last ack'd sequence number for topic-partition {} to {}", batch.producerId(), batch.topicPartition, transactionManager.lastAckedSequence(batch.topicPartition)); } transactionManager.updateLastAckedOffset(response, batch); transactionManager.removeInFlightBatch(batch); } if (batch.done(response.baseOffset, response.logAppendTime, null)) { maybeRemoveFromInflightBatches(batch); this.accumulator.deallocate(batch); } }
@Test public void testChecksumNullForMagicV2() { ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); FutureRecordMetadata future = batch.tryAppend(now, null, new byte[10], Record.EMPTY_HEADERS, null, now); assertNotNull(future); assertNull(future.checksumOrNull()); }
@Test public void testBatchCannotCompleteTwice() throws Exception { ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); MockCallback callback = new MockCallback(); FutureRecordMetadata future = batch.tryAppend(now, null, new byte[10], Record.EMPTY_HEADERS, callback, now); batch.done(500L, 10L, null); assertEquals(1, callback.invocations); assertNull(callback.exception); assertNotNull(callback.metadata); try { batch.done(1000L, 20L, null); fail("Expected exception from done"); } catch (IllegalStateException e) { // expected } RecordMetadata recordMetadata = future.get(); assertEquals(500L, recordMetadata.offset()); assertEquals(10L, recordMetadata.timestamp()); }
@Test public void testBatchCannotAbortTwice() throws Exception { ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); MockCallback callback = new MockCallback(); FutureRecordMetadata future = batch.tryAppend(now, null, new byte[10], Record.EMPTY_HEADERS, callback, now); KafkaException exception = new KafkaException(); batch.abort(exception); assertEquals(1, callback.invocations); assertEquals(exception, callback.exception); assertNull(callback.metadata); try { batch.abort(new KafkaException()); fail("Expected exception from abort"); } catch (IllegalStateException e) { // expected } assertEquals(1, callback.invocations); assertTrue(future.isDone()); try { future.get(); fail("Future should have thrown"); } catch (ExecutionException e) { assertEquals(exception, e.getCause()); } }
private void insertInSequenceOrder(Deque<ProducerBatch> deque, ProducerBatch batch) { if (batch.baseSequence() == RecordBatch.NO_SEQUENCE) throw new IllegalStateException("Trying to re-enqueue a batch which doesn't have a sequence even " + "though idempotency is enabled."); "requests. batch.topicPartition: " + batch.topicPartition + "; batch.baseSequence: " + batch.baseSequence()); if (firstBatchInQueue != null && firstBatchInQueue.hasSequence() && firstBatchInQueue.baseSequence() < batch.baseSequence()) { while (deque.peekFirst() != null && deque.peekFirst().hasSequence() && deque.peekFirst().baseSequence() < batch.baseSequence()) orderedBatches.add(deque.pollFirst()); "position {}", batch.baseSequence(), batch.topicPartition, orderedBatches.size());
ProducerBatch batch = new ProducerBatch(tp, recordsBuilder, time.milliseconds()); FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, headers, callback, time.milliseconds())); return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true);
while (iter.hasNext()) { ProducerBatch batch = iter.next(); if (batch.hasReachedDeliveryTimeout(accumulator.getDeliveryTimeoutMs(), now)) { iter.remove(); if (!batch.isDone()) { expiredBatches.add(batch); } else { throw new IllegalStateException(batch.topicPartition + " batch created at " + batch.createdMs + " gets unexpected final state " + batch.finalState());
private ProducerBatch createBatchOffAccumulatorForRecord(Record record, int batchSize) { int initialSize = Math.max(AbstractRecords.estimateSizeInBytesUpperBound(magic(), recordsBuilder.compressionType(), record.key(), record.value(), record.headers()), batchSize); ByteBuffer buffer = ByteBuffer.allocate(initialSize); // Note that we intentionally do not set producer state (producerId, epoch, sequence, and isTransactional) // for the newly created batch. This will be set when the batch is dequeued for sending (which is consistent // with how normal batches are handled). MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic(), recordsBuilder.compressionType(), TimestampType.CREATE_TIME, 0L); return new ProducerBatch(topicPartition, builder, this.createdMs, true); }