private ProduceRequest build(short version, boolean validate) { if (validate) { // Validate the given records first for (MemoryRecords records : partitionRecords.values()) { ProduceRequest.validateRecords(version, records); } } return new ProduceRequest(version, acks, timeout, partitionRecords, transactionalId); }
@Override public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { /* In case the producer doesn't actually want any response */ if (acks == 0) return null; Errors error = Errors.forException(e); Map<TopicPartition, ProduceResponse.PartitionResponse> responseMap = new HashMap<>(); ProduceResponse.PartitionResponse partitionResponse = new ProduceResponse.PartitionResponse(error); for (TopicPartition tp : partitions()) responseMap.put(tp, partitionResponse); short versionId = version(); switch (versionId) { case 0: case 1: case 2: case 3: case 4: case 5: case 6: case 7: return new ProduceResponse(responseMap, throttleTimeMs); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ApiKeys.PRODUCE.latestVersion())); } }
private ProduceRequest(short version, short acks, int timeout, Map<TopicPartition, MemoryRecords> partitionRecords, String transactionalId) { super(ApiKeys.PRODUCE, version); this.acks = acks; this.timeout = timeout; this.transactionalId = transactionalId; this.partitionRecords = partitionRecords; this.partitionSizes = createPartitionSizes(partitionRecords); for (MemoryRecords records : partitionRecords.values()) { setFlags(records); } }
@Override public boolean matches(AbstractRequest body) { ProduceRequest request = (ProduceRequest) body; if (request.version() != 2) return false; MemoryRecords records = request.partitionRecordsOrFail().get(tp0); return records != null && records.sizeInBytes() > 0 && records.hasMatchingMagic(RecordBatch.MAGIC_VALUE_V1); } }, produceResponse(tp0, offset, Errors.NONE, 0));
@Test public void produceRequestToStringTest() { ProduceRequest request = createProduceRequest(ApiKeys.PRODUCE.latestVersion()); assertEquals(1, request.partitionRecordsOrFail().size()); assertFalse(request.toString(false).contains("partitionSizes")); assertTrue(request.toString(false).contains("numPartitions=1")); assertTrue(request.toString(true).contains("partitionSizes")); assertFalse(request.toString(true).contains("numPartitions")); request.clearPartitionRecords(); try { request.partitionRecordsOrFail(); fail("partitionRecordsOrFail should fail after clearPartitionRecords()"); } catch (IllegalStateException e) { // OK } // `toString` should behave the same after `clearPartitionRecords` assertFalse(request.toString(false).contains("partitionSizes")); assertTrue(request.toString(false).contains("numPartitions=1")); assertTrue(request.toString(true).contains("partitionSizes")); assertFalse(request.toString(true).contains("numPartitions")); }
@Test public void produceRequestGetErrorResponseTest() { ProduceRequest request = createProduceRequest(ApiKeys.PRODUCE.latestVersion()); Set<TopicPartition> partitions = new HashSet<>(request.partitionRecordsOrFail().keySet()); ProduceResponse errorResponse = (ProduceResponse) request.getErrorResponse(new NotEnoughReplicasException()); assertEquals(partitions, errorResponse.responses().keySet()); ProduceResponse.PartitionResponse partitionResponse = errorResponse.responses().values().iterator().next(); assertEquals(Errors.NOT_ENOUGH_REPLICAS, partitionResponse.error); assertEquals(ProduceResponse.INVALID_OFFSET, partitionResponse.baseOffset); assertEquals(RecordBatch.NO_TIMESTAMP, partitionResponse.logAppendTime); request.clearPartitionRecords(); // `getErrorResponse` should behave the same after `clearPartitionRecords` errorResponse = (ProduceResponse) request.getErrorResponse(new NotEnoughReplicasException()); assertEquals(partitions, errorResponse.responses().keySet()); partitionResponse = errorResponse.responses().values().iterator().next(); assertEquals(Errors.NOT_ENOUGH_REPLICAS, partitionResponse.error); assertEquals(ProduceResponse.INVALID_OFFSET, partitionResponse.baseOffset); assertEquals(RecordBatch.NO_TIMESTAMP, partitionResponse.logAppendTime); }
/** * Create a produce request from the given record batches */ private ClientRequest produceRequest(long now, int destination, short acks, int timeout, List<RecordBatch> batches) { Map<TopicPartition, ByteBuffer> produceRecordsByPartition = new HashMap<TopicPartition, ByteBuffer>(batches.size()); Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<TopicPartition, RecordBatch>(batches.size()); for (RecordBatch batch : batches) { TopicPartition tp = batch.topicPartition; ByteBuffer recordsBuffer = batch.records.buffer(); recordsBuffer.flip(); produceRecordsByPartition.put(tp, recordsBuffer); recordsByPartition.put(tp, batch); } ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition); RequestSend send = new RequestSend(destination, this.client.nextRequestHeader(ApiKeys.PRODUCE), request.toStruct()); return new ClientRequest(now, acks != 0, send, recordsByPartition); }
public static ProduceRequest parse(ByteBuffer buffer, short version) { return new ProduceRequest(ApiKeys.PRODUCE.parseRequest(version, buffer), version); }
@Override public boolean matches(AbstractRequest body) { ProduceRequest produceRequest = (ProduceRequest) body; assertTrue(produceRequest.hasIdempotentRecords()); MemoryRecords records = produceRequest.partitionRecordsOrFail().get(tp0); Iterator<MutableRecordBatch> batchIterator = records.batches().iterator(); RecordBatch firstBatch = batchIterator.next(); assertFalse(batchIterator.hasNext()); assertEquals(expectedSequence, firstBatch.baseSequence()); return true; } }, produceResponse(tp, responseOffset, responseError, 0, logStartOffset));
private MockClient.RequestMatcher produceRequestMatcher(final long pid, final short epoch) { return body -> { ProduceRequest produceRequest = (ProduceRequest) body; MemoryRecords records = produceRequest.partitionRecordsOrFail().get(tp0); assertNotNull(records); Iterator<MutableRecordBatch> batchIterator = records.batches().iterator(); assertTrue(batchIterator.hasNext()); MutableRecordBatch batch = batchIterator.next(); assertFalse(batchIterator.hasNext()); assertTrue(batch.isTransactional()); assertEquals(pid, batch.producerId()); assertEquals(epoch, batch.producerEpoch()); assertEquals(transactionalId, produceRequest.transactionalId()); return true; }; }
@Override public boolean matches(AbstractRequest body) { if (!(body instanceof ProduceRequest)) return false; ProduceRequest request = (ProduceRequest) body; Map<TopicPartition, MemoryRecords> recordsMap = request.partitionRecordsOrFail(); MemoryRecords records = recordsMap.get(tp); if (records == null) return false; List<MutableRecordBatch> batches = TestUtils.toList(records.batches()); if (batches.isEmpty() || batches.size() > 1) return false; MutableRecordBatch batch = batches.get(0); return batch.baseOffset() == 0L && batch.baseSequence() == sequence && batch.producerId() == producerIdAndEpoch.producerId && batch.producerEpoch() == producerIdAndEpoch.epoch && batch.isTransactional() == isTransactional; } };
@Test public void testMixedIdempotentData() { final long producerId = 15L; final short producerEpoch = 5; final int sequence = 10; final MemoryRecords nonTxnRecords = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("foo".getBytes())); final MemoryRecords txnRecords = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("bar".getBytes())); final Map<TopicPartition, MemoryRecords> recordsByPartition = new LinkedHashMap<>(); recordsByPartition.put(new TopicPartition("foo", 0), txnRecords); recordsByPartition.put(new TopicPartition("foo", 1), nonTxnRecords); final ProduceRequest.Builder builder = ProduceRequest.Builder.forMagic(RecordVersion.current().value, (short) -1, 5000, recordsByPartition, null); final ProduceRequest request = builder.build(); assertFalse(request.hasTransactionalRecords()); assertTrue(request.hasIdempotentRecords()); }
@Override public boolean matches(AbstractRequest body) { return body instanceof ProduceRequest && ((ProduceRequest) body).hasIdempotentRecords(); } });
private void assertThrowsInvalidRecordException(ProduceRequest.Builder builder, short version) { try { builder.build(version).toStruct(); fail("Builder did not raise " + InvalidRecordException.class.getName() + " as expected"); } catch (RuntimeException e) { assertTrue("Unexpected exception type " + e.getClass().getName(), InvalidRecordException.class.isAssignableFrom(e.getClass())); } }
@Test public void shouldBeFlaggedAsTransactionalWhenTransactionalRecords() throws Exception { final MemoryRecords memoryRecords = MemoryRecords.withTransactionalRecords(0, CompressionType.NONE, 1L, (short) 1, 1, 1, simpleRecord); final ProduceRequest request = ProduceRequest.Builder.forCurrentMagic((short) -1, 10, Collections.singletonMap(new TopicPartition("topic", 1), memoryRecords)).build(); assertTrue(request.hasTransactionalRecords()); }
@Override public Map<Errors, Integer> errorCounts(Throwable e) { Errors error = Errors.forException(e); return Collections.singletonMap(error, partitions().size()); }
@Override public boolean matches(AbstractRequest body) { ProduceRequest request = (ProduceRequest) body; if (request.version() != 2) return false; Map<TopicPartition, MemoryRecords> recordsMap = request.partitionRecordsOrFail(); if (recordsMap.size() != 2) return false; for (MemoryRecords records : recordsMap.values()) { if (records == null || records.sizeInBytes() == 0 || !records.hasMatchingMagic(RecordBatch.MAGIC_VALUE_V1)) return false; } return true; } }, produceResponse);
switch (apiKey) { case PRODUCE: return new ProduceRequest(struct, apiVersion); case FETCH: return new FetchRequest(struct, apiVersion);
@Override public boolean matches(AbstractRequest body) { if (body instanceof ProduceRequest) { ProduceRequest request = (ProduceRequest) body; MemoryRecords records = request.partitionRecordsOrFail().get(tp0); Iterator<MutableRecordBatch> batchIterator = records.batches().iterator(); assertTrue(batchIterator.hasNext()); RecordBatch batch = batchIterator.next(); assertFalse(batchIterator.hasNext()); assertEquals(0, batch.baseSequence()); assertEquals(producerId, batch.producerId()); assertEquals(0, batch.producerEpoch()); return true; } return false; } }, produceResponse(tp0, 0, Errors.NONE, 0));
@Test public void testMixedTransactionalData() { final long producerId = 15L; final short producerEpoch = 5; final int sequence = 10; final String transactionalId = "txnlId"; final MemoryRecords nonTxnRecords = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("foo".getBytes())); final MemoryRecords txnRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, new SimpleRecord("bar".getBytes())); final Map<TopicPartition, MemoryRecords> recordsByPartition = new LinkedHashMap<>(); recordsByPartition.put(new TopicPartition("foo", 0), txnRecords); recordsByPartition.put(new TopicPartition("foo", 1), nonTxnRecords); final ProduceRequest.Builder builder = ProduceRequest.Builder.forMagic(RecordVersion.current().value, (short) -1, 5000, recordsByPartition, transactionalId); final ProduceRequest request = builder.build(); assertTrue(request.hasTransactionalRecords()); assertTrue(request.hasIdempotentRecords()); }