Skip to content

Commit d54a204

Browse files
committed
Use the LuceneSerializer to encode store fields, too.
Since existing fields will be straight protobuf without the serializer's prefix byte, recognize that, too.
1 parent f0bc0f2 commit d54a204

File tree

3 files changed

+70
-5
lines changed

3 files changed

+70
-5
lines changed

fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectory.java

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -444,10 +444,11 @@ public int writeData(final long id, final int block, @Nonnull final byte[] value
444444
* Write stored fields document to the DB.
445445
* @param segmentName the segment name writing to
446446
* @param docID the document ID to write
447-
* @param value the bytes value of the stored fields
447+
* @param rawBytes the bytes value of the stored fields
448448
*/
449-
public void writeStoredFields(@Nonnull String segmentName, int docID, @Nonnull final byte[] value) {
449+
public void writeStoredFields(@Nonnull String segmentName, int docID, @Nonnull final byte[] rawBytes) {
450450
byte[] key = storedFieldsSubspace.pack(Tuple.from(segmentName, docID));
451+
byte[] value = serializer.encode(rawBytes);
451452
agilityContext.recordSize(LuceneEvents.SizeEvents.LUCENE_WRITE_STORED_FIELDS, key.length + value.length);
452453
if (LOGGER.isTraceEnabled()) {
453454
LOGGER.trace(getLogMessage("Write lucene stored fields data",
@@ -542,7 +543,7 @@ private CompletableFuture<byte[]> readData(long id, int block) {
542543
}
543544

544545
@Nonnull
545-
public byte[] readStoredFields(String segmentName, int docId) throws IOException {
546+
public byte[] readStoredFields(String segmentName, int docId) {
546547
final byte[] key = storedFieldsSubspace.pack(Tuple.from(segmentName, docId));
547548
final byte[] rawBytes = asyncToSync(LuceneEvents.Waits.WAIT_LUCENE_GET_STORED_FIELDS,
548549
agilityContext.instrument(LuceneEvents.Events.LUCENE_READ_STORED_FIELDS,
@@ -553,7 +554,7 @@ public byte[] readStoredFields(String segmentName, int docId) throws IOException
553554
.addLogInfo(LuceneLogMessageKeys.DOC_ID, docId)
554555
.addLogInfo(LogMessageKeys.KEY, ByteArrayUtil2.loggable(key));
555556
}
556-
return rawBytes;
557+
return Objects.requireNonNull(serializer.decodePossiblyWithoutPrefix(rawBytes));
557558
}
558559

559560
@Nonnull

fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/LuceneSerializer.java

Lines changed: 33 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -131,7 +131,7 @@ public byte[] decode(@Nullable byte[] data) {
131131
return null;
132132
}
133133

134-
if (data.length < 2) {
134+
if (data.length < 1) {
135135
throw new RecordCoreException("Invalid data")
136136
.addLogInfo(LuceneLogMessageKeys.DATA_VALUE, data);
137137
}
@@ -270,4 +270,36 @@ private void decryptIfNeeded(@Nonnull CompressedAndEncryptedSerializerState stat
270270
}
271271
encodedDataInput.reset(decrypted);
272272
}
273+
274+
@Nullable
275+
public byte[] decodePossiblyWithoutPrefix(@Nullable byte[] bytes) {
276+
if (bytes == null) {
277+
return null;
278+
}
279+
280+
if (isProtobufMessageWithoutPrefix(bytes)) {
281+
return bytes;
282+
}
283+
return decode(bytes);
284+
}
285+
286+
// This can be removed once it is guaranteed that all indexes are using the encoded format.
287+
// Only works for Protobuf messages all of whose fields are themselves length-delimited.
288+
private boolean isProtobufMessageWithoutPrefix(@Nonnull byte[] bytes) {
289+
if (bytes.length < 1) {
290+
return true; // No room for prefix; empty message.
291+
}
292+
final int byte0 = bytes[0];
293+
final int fieldTypeOrPrefixFlags = byte0 & 7;
294+
// Either Protobuf LEN or ENCODING_COMPRESSED.
295+
if (fieldTypeOrPrefixFlags != 2) {
296+
return false;
297+
}
298+
final int fieldNumberOrKeyNumber = byte0 >> 3;
299+
// ENCODING_COMPRESSED will never have a key; 0 is not a valid field number.
300+
if (fieldNumberOrKeyNumber == 0) {
301+
return false;
302+
}
303+
return true;
304+
}
273305
}

fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/LuceneSerializerTest.java

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,10 +32,13 @@
3232
import org.junit.jupiter.api.Tag;
3333
import org.junit.jupiter.api.Test;
3434
import org.junit.jupiter.params.ParameterizedTest;
35+
import org.junit.jupiter.params.provider.Arguments;
36+
import org.junit.jupiter.params.provider.MethodSource;
3537

3638
import javax.crypto.KeyGenerator;
3739
import javax.crypto.SecretKey;
3840
import java.util.concurrent.ThreadLocalRandom;
41+
import java.util.stream.Stream;
3942

4043
/**
4144
* Test for Lucene data compression/decompression and encryption/decryption validation.
@@ -122,4 +125,33 @@ void testEncodingWithEncryption(boolean compressToo) throws Exception {
122125
final LuceneFileSystemProto.LuceneFileReference decryptedReference = LuceneFileSystemProto.LuceneFileReference.parseFrom(decodedValue);
123126
Assertions.assertEquals(content, decryptedReference.getContent());
124127
}
128+
129+
@ParameterizedTest
130+
@MethodSource
131+
void testProtobufMessageWithoutPrefix(boolean encode, boolean encrypt, boolean compress) throws Exception {
132+
KeyGenerator keyGen = KeyGenerator.getInstance("AES");
133+
keyGen.init(128);
134+
SecretKey key = keyGen.generateKey();
135+
final SerializationKeyManager keyManager = new FixedZeroKeyManager(key, null, null);
136+
final LuceneSerializer serializer = new LuceneSerializer(compress, encrypt, keyManager);
137+
final String storedField = RandomUtil.randomAlphanumericString(ThreadLocalRandom.current(), 20);
138+
final LuceneStoredFieldsProto.LuceneStoredFields.Builder builder = LuceneStoredFieldsProto.LuceneStoredFields.newBuilder();
139+
builder.addStoredFieldsBuilder().setFieldNumber(5).setStringValue(storedField);
140+
final byte[] value = builder.build().toByteArray();
141+
final byte[] encodedValue = encode ? serializer.encode(value) : value;
142+
final byte[] decodedValue = serializer.decodePossiblyWithoutPrefix(encodedValue);
143+
Assertions.assertArrayEquals(value, decodedValue);
144+
final LuceneStoredFieldsProto.LuceneStoredFields storedFields = LuceneStoredFieldsProto.LuceneStoredFields.parseFrom(decodedValue);
145+
Assertions.assertEquals(storedField, storedFields.getStoredFields(0).getStringValue());
146+
}
147+
148+
static Stream<Arguments> testProtobufMessageWithoutPrefix() {
149+
return Stream.of(
150+
Arguments.of(false, false, false),
151+
Arguments.of(true, false, false),
152+
Arguments.of(true, false, true),
153+
Arguments.of(true, true, false),
154+
Arguments.of(true, true, true)
155+
);
156+
}
125157
}

0 commit comments

Comments
 (0)