-
Notifications
You must be signed in to change notification settings - Fork 14k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-4816: Message format changes for idempotent/transactional producer #2614
Conversation
a4bd61d
to
c2a14f8
Compare
A couple initial points of discussion:
|
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
FYI: System tests run here: http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2017-03-01--001.1488358155--confluentinc--exactly-once-message-format--c2a14f8/report.html. The one failure looks like a known problem, but I haven't investigated. |
@hachikuji, good questions!
|
c2a14f8
to
76e4913
Compare
@ijuma Thanks for the suggestions.
The other thing I wanted to mention is that I've left around the old |
I'm not fond of Yes, I prefer I think it makes sense to rename |
Refer to this link for build results (access rights to CI server needed): |
So if we follow those suggestions, then we have the following hierarchy:
That seems reasonable to me. The |
@junrao, what do you think? |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
* @param value The value to write | ||
* @param out The output to write to | ||
*/ | ||
public static void writeUnsignedVarLong(long value, DataOutput out) throws IOException { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we should introduce a ByteUtils
class and move all the relevant methods there. Maybe we could do that in its own PR, which could be merged quickly independently of the message format changes?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Works for me.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@hachikuji : Thanks for the patch. Looks good overall. Made a pass of non-test files and added some comments.
* | ||
* ----------------------------------------------------------------------------------- | ||
* | Unused (5-16) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) | | ||
* ----------------------------------------------------------------------------------- |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We may add new compression codec in the future. Using the bits from 15 downwards makes adding new compression codec a bit easier in the future. Also, unused should be 5-15.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It makes sense to leave space for the compression type, but I'd hope we won't need more than 3 bits for it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My guess is we'll have another message format bump before we run out of room for compression codecs, but I don't mind adding another bit if you think it's useful.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, I may have misunderstood. Are you suggesting moving the transactional flag and timestamp type to the end of the second byte?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since we already have 3 bits for compression, so leaving this as it is will be fine.
static final int BASE_TIMESTAMP_LENGTH = 8; | ||
static final int MAX_TIMESTAMP_OFFSET = BASE_TIMESTAMP_OFFSET + BASE_TIMESTAMP_LENGTH; | ||
static final int MAX_TIMESTAMP_LENGTH = 8; | ||
static final int LAST_OFFSET_DELTA_OFFSET = MAX_TIMESTAMP_OFFSET + MAX_TIMESTAMP_LENGTH; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In the comment above the class and in the KIP wiki, LastOffsetDelta is after Attributes.
|
||
@Override | ||
public int lastSequence() { | ||
// FIXME: cast to int |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The comment is no longer valid?
while (iterator.hasNext()) { | ||
LogRecord record = iterator.next(); | ||
int offsetDelta = (int) (record.offset() - baseOffset); | ||
size += EosLogRecord.sizeInBytes(offsetDelta, record.timestamp(), record.key(), record.value()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm, should we be passing in delta timestamp?
* ValueLen => VarInt | ||
* Value => data | ||
* | ||
* The record attributes indicate whether the key and value fields are present. The current attributes |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The comment seems outdated. Attributes are no longer used to indicate the presence of key and value.
Map<String, Map<Integer, MemoryRecords>> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); | ||
struct.set(ACKS_KEY_NAME, acks); | ||
struct.set(TIMEOUT_KEY_NAME, timeout); | ||
|
||
// TODO: Include transactional id once transactions are supported |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is the TODO still valid?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I guess the TODO should really go to the other constructor.
val records = { | ||
val buffer = ByteBuffer.allocate(AbstractRecords.estimateSizeInBytes(magicValue, compressionType, | ||
Seq(new KafkaRecord(timestamp, key, value)).asJava)) | ||
val builder = MemoryRecords.builder(buffer, magicValue, compressionType, TimestampType.CREATE_TIME, 0L) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should timestampType be hardcoded to createTime or should it use the topic level config? Ditto for line 256 below.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
CREATE_TIME
is correct, see 7565dcd
@@ -579,11 +592,15 @@ class GroupMetadataManager(val brokerId: Int, | |||
case Some((magicValue, timestampType, timestamp)) => | |||
val partitionOpt = replicaManager.getPartition(appendPartition) | |||
partitionOpt.foreach { partition => | |||
val tombstones = removedOffsets.map { case (topicPartition, offsetAndMetadata) => | |||
var offset = 0L | |||
val builder = MemoryRecords.builder(ByteBuffer.allocate(1024), magicValue, compressionType, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is 1024 guaranteed to be always large enough?
print("offset: " + record.offset + " position: " + validBytes + | ||
" " + entry.timestampType + ": " + record.timestamp + " isvalid: " + record.isValid + | ||
" keysize: " + record.keySize + " valuesize: " + record.valueSize + " magic: " + entry.magic + | ||
" compresscodec: " + entry.compressionType + " crc: " + record.checksum) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we print out additional information related to EOS format (e.g. pid, epoch, sequence) etc? Should we support printing out the control message?
checkstyle/suppressions.xml
Outdated
@@ -25,6 +27,8 @@ | |||
files="Fetcher.java"/> | |||
<suppress checks="ParameterNumber" | |||
files="ConfigDef.java"/> | |||
<suppress checks="ParameterNumber" | |||
files="EosLogEntry.java"/> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Now that we have increased the param limit in checkstyle, do we still need this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We'd need to increase a bit more to capture this one too. Increasing to 12 dropped two others.
A few other high level comments.
|
@junrao, about including the message format version as a prefix or suffix. I considered that, but the issue is that more than one version is supported by each class. Unless we want to move away from that, it seems a bit awkward. |
76e4913
to
5ac42cc
Compare
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I left a few comments. I didn't review the message format changes themselves since you are about to do a bunch of renames and a trunk merge (I assume). Probably best to review after that.
/** | ||
* Maintains node api versions for access outside of NetworkClient (which is where the information is derived). | ||
* The pattern is akin to the use of {@link Metadata} for topic metadata. | ||
*/ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should add a note that this is an internal class since the package doesn't make that obvious.
usableMagic = LogEntry.MAGIC_VALUE_V2; | ||
} | ||
if (usableMagic < maxUsableMagic) | ||
maxUsableMagic = usableMagic; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: maxUsableMagic = Math.min(usableMagic, maxUsableMagic)
?
|
||
default: | ||
usableMagic = LogEntry.MAGIC_VALUE_V2; | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this switch
should be a separate method as it's the interesting logic that may need to be updated when we add more produce versions. Maybe it should live in ProduceRequest
? And we should have a test that breaks when we add a new produce version to the protocol (if we don't already) so that we don't forget to update it (since the client would still work fine, it could go unnoticed).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, that's fair. Let me see if we can move it to ProduceRequest
.
version = versionInfo.usableVersion(clientRequest.apiKey()); | ||
} else { | ||
versionInfo.ensureUsable(clientRequest.apiKey(), builder.desiredVersion()); | ||
version = builder.desiredVersion(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm, would it be better to have a NodeApiVersions.usableVersion
that takes a desiredVersion
as well? We could the collapse these two branches and there would be no need to call ensureUsable
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, that sounds good.
private static final short NODE_TOO_OLD = (short) -1; | ||
private static final short NODE_TOO_NEW = (short) -2; | ||
private final Collection<ApiVersion> nodeApiVersions; | ||
// An array of the usable versions of each API, indexed by the ApiKeys ID. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's an existing issue, but realised that this comment is out of date.
} | ||
for (UsableVersion usableVersion : usableVersions.values()) { | ||
if (usableVersion.apiVersion.apiKey == apiKey.id) | ||
return usableVersion.apiVersion; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we just simply call get
on the map? Also, maybe we should return null
if there's no element (like Map.get
).
} else { | ||
usableVersions.put(nodeApiKey, v); | ||
usableVersions.put(nodeApiKey, new UsableVersion(nodeApiVersion, v)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe we could move the logic that creates the appropriate UsableVersion
into a static factory method (or constructor) in UsableVersion
.
private static final short NODE_TOO_NEW = (short) -2; | ||
|
||
private final ApiVersion apiVersion; | ||
private final Short defaultVersion; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I found the term default
a bit confusing here. This is usableVersion
, right? I guess you were trying to avoid repeating it since the class name is UsableVersion
. Could we just call it value
?
this(null, metadata, selector, clientId, maxInFlightRequestsPerConnection, | ||
reconnectBackoffMs, socketSendBuffer, socketReceiveBuffer, requestTimeoutMs, time, discoverBrokerVersions); | ||
boolean discoverBrokerVersions, | ||
ApiVersions nodeApiVersions) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think the field name should just be apiVersion
, no?
"0.10.2" -> KAFKA_0_10_2_IV0 | ||
"0.10.2" -> KAFKA_0_10_2_IV0, | ||
// KIP-98, aka EOS, changes. | ||
"0.10.3-IV0" -> KAFKA_0_10_3_IV0, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should be "0.11.0-IV0" and the rest should be updated accordingly.
* High-level representation of a kafka record. This is useful when building record sets to | ||
* avoid depending on a specific magic version. | ||
*/ | ||
public class KafkaRecord { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would BasicRecord
or SimpleRecord
be a better name? I think the key feature is that it just captures the essence of what a record is: timestamp, key, value (and potentially headers in the future?).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
SimpleRecord
works for me. I may have actually had this in the code at one time or another.
5ac42cc
to
f07de31
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I finally have gone through the whole diff. :) LGTM to merge if the system tests pass.
I have made a few notes of follow-up changes (most are not time sensitive although a couple are) and will file JIRAs in the next few days.
While going through the changes, I put together some minor improvements and some additional tests:
https://github.com/confluentinc/kafka/pull/151/files
We can merge it to this PR or to trunk subsequently depending on your preference.
} | ||
|
||
@Test | ||
public void testIterator() { | ||
MemoryRecordsBuilder builder1 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset); | ||
MemoryRecordsBuilder builder2 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do you know why we had 2 builders?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we were verifying the auto-incrementing behavior. I will verify that we have both cases covered.
for (Header header : headers) { | ||
String headerKey = header.key(); | ||
if (headerKey == null) { | ||
size += NULL_VARINT_SIZE_BYTES; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Did we decide on this one?
buffer.position(RECORDS_OFFSET); | ||
final int totalRecords = count(); | ||
|
||
return new Iterator<Record>() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice that we avoid the overhead of the streams here.
int offsetDelta = (int) (offset - baseOffset); | ||
long timestampDelta = timestamp - baseTimestamp; | ||
long crc = DefaultRecord.writeTo(appendStream, isControlRecord, offsetDelta, timestampDelta, key, value, headers); | ||
// TODO: The crc is useless for the new message format. Maybe we should let writeTo return the written size? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, I agree that it's worth adding a brief comment explaining until we fix the issue.
* @param record the record to add | ||
*/ | ||
public void append(Record record) { | ||
appendWithOffset(record.offset(), record.isControlRecord(), record.timestamp(), record.key(), record.value(), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it OK that we don't check the magic here, but we do for the append that takes a legacy record?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, that's a good question. I'm not sure there's a good reason for the check in the LegacyRecord
case. I think initially we were doing something like copying the bytes directly, so it may have made more sense before.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, I guess we still do the memory copy in this case. The question is whether we should?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This method is only used by ByteBufferMessageSet
(i.e. Scala clients) so it seems OK to favour consistency over optimisation in this case.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@hachikuji : Thanks for the patch. LGTM. Just a few minor comments.
record.buffer(), | ||
wrapperRecordTimestamp, | ||
wrapperRecord.timestampType() | ||
); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Move this to previous line?
*/ | ||
public class MemoryRecords extends AbstractRecords { | ||
|
||
public final static MemoryRecords EMPTY = MemoryRecords.readableRecords(ByteBuffer.allocate(0)); | ||
|
||
private final ByteBuffer buffer; | ||
|
||
private final Iterable<ByteBufferLogEntry> shallowEntries = new Iterable<ByteBufferLogEntry>() { | ||
private final Iterable<MutableRecordBatch> logEntries = new Iterable<MutableRecordBatch>() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
logEntries => recordBatch?
@@ -54,7 +54,7 @@ class ConsumerFetcherThread(name: String, | |||
replicaId(Request.OrdinaryConsumerId). | |||
maxWait(config.fetchWaitMaxMs). | |||
minBytes(config.fetchMinBytes). | |||
requestVersion(kafka.api.FetchRequest.CurrentVersion) | |||
requestVersion(3) // for now, the old consumer is pinned to the old message format through the the fetch request |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
the the => the
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
a06a12f
to
343f2b4
Compare
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, fix looks good. Some minor comments.
byte magicByte) { | ||
int i = 0; | ||
for (RecordBatch batch : convertedRecords.batches()) { | ||
assertEquals("magic byte should be " + magicByte, magicByte, batch.magic()); | ||
assertTrue("Magic byte should be lower than or equal to " + magicByte, batch.magic() <= magicByte); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why doesn't assertEquals work here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We only convert messages when necessary. So if we down-convert a batch to version 1, and we have messages which are magic 0, we won't up-convert them to 1.
byte magicByte) { | ||
int i = 0; | ||
for (RecordBatch batch : convertedRecords.batches()) { | ||
assertEquals("magic byte should be " + magicByte, magicByte, batch.magic()); | ||
assertTrue("Magic byte should be lower than or equal to " + magicByte, batch.magic() <= magicByte); | ||
assertEquals("Compression type should not be affected by conversion", compressionType, batch.compressionType()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would it be worth adding some timestamp assertions as well?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ack. May as well.
|
||
// down conversion for compressed messages | ||
// down conversion for non-compressed messages |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
non-compressed
should be removed, right?
Records convertedRecords = fileRecords.downConvert(RecordBatch.MAGIC_VALUE_V0); | ||
verifyConvertedMessageSet(records, offsets, convertedRecords, RecordBatch.MAGIC_VALUE_V0); | ||
Records convertedRecords = fileRecords.downConvert(toMagic); | ||
verifyConvertedMessageSet(records, offsets, convertedRecords, compressionType, toMagic); | ||
} | ||
} | ||
|
||
private void verifyConvertedMessageSet(List<SimpleRecord> initialRecords, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
verifyConvertedBatches
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
* Note that when compression is enabled (see attributes below), the compressed record data is serialized | ||
* directly following the count of the number of records. | ||
* | ||
* The CRC covers the data from the attributes to the end of the batch. Note that the location is |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, if CRC only covers from Attributes to the end, do we really need to switch PartitionLeaderEpoch and CRC?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's true that the change is not strictly needed. The reasoning for the switch is that this way the CRC includes everything that follows it (same as for V0 and V1 formats). In V0 and V1, the offset and length are before the CRC and are not included in the computation. In V2, the offset, length, partitionLeaderEpoch and magic are before the CRC and not included in the computation.
Given that, do you think the benefit is worth the switch?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Earlier, Jason's proposal is to put CRC to the very end after records and covers everything from magic. If we do that, the switch is not ideal, but makes sense. If CRC is still at the front, then it seems leaving CRC in its current location causes less confusion.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we leave the CRC in the same position as in V0 and V1, would you still change the computation to be from attributes to the end? A couple of alternatives (each with pros/cons):
- Include magic in the CRC computation, but not partitionLeaderEpoch (con: we'd have to call
Checksum.update
twice instead of once). - Compute the CRC from magic to the end (as it was before), but with PartitionLeaderEpoch always assumed to be -1. This makes the computation simple on the producer side, but it's a bit tricky after the PartitionLeaderEpoch has been set. This is a bit similar to how TCP includes the checksum field in the checksum computation, but assumes it to be always 0 (to avoid the chicken and egg problem).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we can swap the CRC and leader epoch fields in the current version, but keep the scope of the CRC unchanged. In other words:
* RecordBatch =>
* BaseOffset => Int64
* Length => Int32
* CRC => Uint32 (covers everything beginning with Attributes)
* Magic => Int8
* PartitionLeaderEpoch => Int32
* Attributes => Int16
* LastOffsetDelta => Int32
* BaseTimestamp => Int64
* MaxTimestamp => Int64
* ProducerId => Int64
* ProducerEpoch => Int16
* BaseSequence => Int32
* Records => [Record]
It's a little odd that we change the scope of the CRC in this format, but perhaps it's less odd than changing the location of the field entirely.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a tough one. It's nice being consistent with the old format to some extent, but clients would still need to look at the magic byte first to figure out how to validate the CRC, so maybe the consistency win is marginal. The main advantage of the format in the current patch is that it's clear at a glance what is covered by the CRC, so it seems easier to explain.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Discussed offline and Jun said he was OK with leaving as is.
Ran system core and client system tests here: http://testing.confluent.io/confluent-kafka-branch-builder-system-test-results/?prefix=2017-03-23--001.1490326564--confluentinc--exactly-once-message-format--2e7cf82/. The failing zookeeper upgrade test seems to be transient. Here's a passing run: http://testing.confluent.io/confluent-kafka-branch-builder-system-test-results/?prefix=2017-03-24--001.1490323181--confluentinc--exactly-once-message-format--2e7cf82/. I investigated the failure in one case and it seems to be unrelated (apparently caused the overridden min isr setting of 2 and the fact that the test does not actually ensure that 2 brokers are always up). |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for running the system tests. Latest updates LGTM too. Will merge to trunk shortly.
This is an impressive PR, well done. :) Also, kudos to Jun for his detailed review.
I took notes of follow-up work and will file a few JIRAs in the next few days.
@hachikuji, please update the KIP page to include the updates that we did during the PR review.
LGTM! |
@@ -114,8 +143,28 @@ public FetchResponse(Struct struct) { | |||
int partition = partitionResponseHeader.getInt(PARTITION_KEY_NAME); | |||
Errors error = Errors.forCode(partitionResponseHeader.getShort(ERROR_CODE_KEY_NAME)); | |||
long highWatermark = partitionResponseHeader.getLong(HIGH_WATERMARK_KEY_NAME); | |||
long lastStableOffset = INVALID_LSO; | |||
if (partitionResponse.hasField(LAST_STABLE_OFFSET_KEY_NAME)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Bug: we should use partitionResponseHeader
instead of partitionResponse
to get the response. I guess this is not discovered by any test because the current patch always has last_stable_offset = -1L
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for catching this @lindong28. This field won't be used until the transactional code lands. I submitted a PR with tests:
No description provided.