Skip to content

Commit 4e04f3c

Browse files
authored
[core] Pass file ser version to LookupSerializerFactory to let it deserialize from old version (#6677)
1 parent ca2e8cf commit 4e04f3c

11 files changed

+122
-56
lines changed

paimon-core/src/main/java/org/apache/paimon/mergetree/LookupFile.java

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,7 @@ public class LookupFile {
5050
private final File localFile;
5151
private final int level;
5252
private final long schemaId;
53+
private final String serVersion;
5354
private final LookupStoreReader reader;
5455
private final Runnable callback;
5556

@@ -58,10 +59,16 @@ public class LookupFile {
5859
private boolean isClosed = false;
5960

6061
public LookupFile(
61-
File localFile, int level, long schemaId, LookupStoreReader reader, Runnable callback) {
62+
File localFile,
63+
int level,
64+
long schemaId,
65+
String serVersion,
66+
LookupStoreReader reader,
67+
Runnable callback) {
6268
this.localFile = localFile;
6369
this.level = level;
6470
this.schemaId = schemaId;
71+
this.serVersion = serVersion;
6572
this.reader = reader;
6673
this.callback = callback;
6774
}
@@ -74,6 +81,10 @@ public long schemaId() {
7481
return schemaId;
7582
}
7683

84+
public String serVersion() {
85+
return serVersion;
86+
}
87+
7788
@Nullable
7889
public byte[] get(byte[] key) throws IOException {
7990
checkArgument(!isClosed);

paimon-core/src/main/java/org/apache/paimon/mergetree/LookupLevels.java

Lines changed: 79 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import org.apache.paimon.utils.BloomFilter;
3535
import org.apache.paimon.utils.FileIOUtils;
3636
import org.apache.paimon.utils.IOFunction;
37+
import org.apache.paimon.utils.Pair;
3738

3839
import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache;
3940

@@ -45,6 +46,7 @@
4546
import java.util.Comparator;
4647
import java.util.HashSet;
4748
import java.util.Map;
49+
import java.util.Optional;
4850
import java.util.Set;
4951
import java.util.TreeSet;
5052
import java.util.concurrent.ConcurrentHashMap;
@@ -68,8 +70,7 @@ public class LookupLevels<T> implements Levels.DropFileCallback, Closeable {
6870
private final Function<Long, BloomFilter.Builder> bfGenerator;
6971
private final Cache<String, LookupFile> lookupFileCache;
7072
private final Set<String> ownCachedFiles;
71-
private final String remoteSstSuffix;
72-
private final Map<Long, PersistProcessor<T>> schemaIdToProcessors;
73+
private final Map<Pair<Long, String>, PersistProcessor<T>> schemaIdAndSerVersionToProcessors;
7374

7475
@Nullable private RemoteFileDownloader remoteFileDownloader;
7576

@@ -99,13 +100,7 @@ public LookupLevels(
99100
this.bfGenerator = bfGenerator;
100101
this.lookupFileCache = lookupFileCache;
101102
this.ownCachedFiles = new HashSet<>();
102-
this.remoteSstSuffix =
103-
"."
104-
+ processorFactory.identifier()
105-
+ "."
106-
+ serializerFactory.identifier()
107-
+ REMOTE_LOOKUP_FILE_SUFFIX;
108-
this.schemaIdToProcessors = new ConcurrentHashMap<>();
103+
this.schemaIdAndSerVersionToProcessors = new ConcurrentHashMap<>();
109104
levels.addDropFileCallback(this);
110105
}
111106

@@ -170,17 +165,17 @@ private T lookup(InternalRow key, DataFileMeta file) throws IOException {
170165
return null;
171166
}
172167

173-
return getOrCreateProcessor(lookupFile.schemaId())
168+
return getOrCreateProcessor(lookupFile.schemaId(), lookupFile.serVersion())
174169
.readFromDisk(key, lookupFile.level(), valueBytes, file.fileName());
175170
}
176171

177-
private PersistProcessor<T> getOrCreateProcessor(long schemaId) {
178-
return schemaIdToProcessors.computeIfAbsent(
179-
schemaId,
172+
private PersistProcessor<T> getOrCreateProcessor(long schemaId, String serVersion) {
173+
return schemaIdAndSerVersionToProcessors.computeIfAbsent(
174+
Pair.of(schemaId, serVersion),
180175
id -> {
181176
RowType fileSchema =
182177
schemaId == currentSchemaId ? null : schemaFunction.apply(schemaId);
183-
return processorFactory.create(serializerFactory, fileSchema);
178+
return processorFactory.create(serVersion, serializerFactory, fileSchema);
184179
});
185180
}
186181

@@ -191,9 +186,12 @@ public LookupFile createLookupFile(DataFileMeta file) throws IOException {
191186
}
192187

193188
long schemaId = this.currentSchemaId;
194-
if (tryToDownloadRemoteSst(file, localFile)) {
189+
String fileSerVersion = serializerFactory.version();
190+
Optional<String> downloadSerVersion = tryToDownloadRemoteSst(file, localFile);
191+
if (downloadSerVersion.isPresent()) {
195192
// use schema id from remote file
196193
schemaId = file.schemaId();
194+
fileSerVersion = downloadSerVersion.get();
197195
} else {
198196
createSstFileFromDataFile(file, localFile);
199197
}
@@ -203,21 +201,35 @@ public LookupFile createLookupFile(DataFileMeta file) throws IOException {
203201
localFile,
204202
file.level(),
205203
schemaId,
204+
fileSerVersion,
206205
lookupStoreFactory.createReader(localFile),
207206
() -> ownCachedFiles.remove(file.fileName()));
208207
}
209208

210-
private boolean tryToDownloadRemoteSst(DataFileMeta file, File localFile) {
209+
private Optional<String> tryToDownloadRemoteSst(DataFileMeta file, File localFile) {
211210
if (remoteFileDownloader == null) {
212-
return false;
211+
return Optional.empty();
212+
}
213+
Optional<RemoteSstFile> remoteSstFile = remoteSst(file);
214+
if (!remoteSstFile.isPresent()) {
215+
return Optional.empty();
213216
}
217+
218+
RemoteSstFile remoteSst = remoteSstFile.get();
219+
214220
// validate schema matched, no exception here
215221
try {
216-
getOrCreateProcessor(file.schemaId());
222+
getOrCreateProcessor(file.schemaId(), remoteSst.serVersion);
217223
} catch (UnsupportedOperationException e) {
218-
return false;
224+
return Optional.empty();
225+
}
226+
boolean success =
227+
remoteFileDownloader.tryToDownload(file, remoteSst.sstFileName, localFile);
228+
if (!success) {
229+
return Optional.empty();
219230
}
220-
return remoteFileDownloader.tryToDownload(file, localFile);
231+
232+
return Optional.of(remoteSst.serVersion);
221233
}
222234

223235
public void addLocalFile(DataFileMeta file, LookupFile lookupFile) {
@@ -229,7 +241,8 @@ private void createSstFileFromDataFile(DataFileMeta file, File localFile) throws
229241
lookupStoreFactory.createWriter(
230242
localFile, bfGenerator.apply(file.rowCount()));
231243
RecordReader<KeyValue> reader = fileReaderFactory.apply(file)) {
232-
PersistProcessor<T> processor = getOrCreateProcessor(currentSchemaId);
244+
PersistProcessor<T> processor =
245+
getOrCreateProcessor(currentSchemaId, serializerFactory.version());
233246
KeyValue kv;
234247
if (processor.withPosition()) {
235248
FileRecordIterator<KeyValue> batch;
@@ -258,8 +271,39 @@ private void createSstFileFromDataFile(DataFileMeta file, File localFile) throws
258271
}
259272
}
260273

261-
public String remoteSstSuffix() {
262-
return remoteSstSuffix;
274+
public Optional<RemoteSstFile> remoteSst(DataFileMeta file) {
275+
Optional<String> sstFile =
276+
file.extraFiles().stream()
277+
.filter(f -> f.endsWith(REMOTE_LOOKUP_FILE_SUFFIX))
278+
.findFirst();
279+
if (!sstFile.isPresent()) {
280+
return Optional.empty();
281+
}
282+
283+
String sstFileName = sstFile.get();
284+
String[] split = sstFileName.split("\\.");
285+
if (split.length < 3) {
286+
return Optional.empty();
287+
}
288+
289+
String processorId = split[split.length - 3];
290+
if (!processorFactory.identifier().equals(processorId)) {
291+
return Optional.empty();
292+
}
293+
294+
String serVersion = split[split.length - 2];
295+
return Optional.of(new RemoteSstFile(sstFileName, serVersion));
296+
}
297+
298+
public String newRemoteSst(DataFileMeta file, long length) {
299+
return file.fileName()
300+
+ "."
301+
+ length
302+
+ "."
303+
+ processorFactory.identifier()
304+
+ "."
305+
+ serializerFactory.version()
306+
+ REMOTE_LOOKUP_FILE_SUFFIX;
263307
}
264308

265309
@Override
@@ -269,4 +313,16 @@ public void close() throws IOException {
269313
lookupFileCache.invalidate(cachedFile);
270314
}
271315
}
316+
317+
/** Remote sst file with serVersion. */
318+
public static class RemoteSstFile {
319+
320+
private final String sstFileName;
321+
private final String serVersion;
322+
323+
private RemoteSstFile(String sstFileName, String serVersion) {
324+
this.sstFileName = sstFileName;
325+
this.serVersion = serVersion;
326+
}
327+
}
272328
}

paimon-core/src/main/java/org/apache/paimon/mergetree/lookup/DefaultLookupSerializerFactory.java

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@
3030
public class DefaultLookupSerializerFactory implements LookupSerializerFactory {
3131

3232
@Override
33-
public String identifier() {
33+
public String version() {
3434
return "v1";
3535
}
3636

@@ -42,7 +42,10 @@ public Function<InternalRow, byte[]> createSerializer(RowType currentSchema) {
4242

4343
@Override
4444
public Function<byte[], InternalRow> createDeserializer(
45-
RowType currentSchema, @Nullable RowType fileSchema) {
45+
String fileSerVersion, RowType currentSchema, @Nullable RowType fileSchema) {
46+
if (!version().equals(fileSerVersion)) {
47+
throw new UnsupportedOperationException();
48+
}
4649
if (fileSchema != null && !fileSchema.equalsIgnoreNullable(currentSchema)) {
4750
throw new UnsupportedOperationException();
4851
}

paimon-core/src/main/java/org/apache/paimon/mergetree/lookup/LookupSerializerFactory.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -32,12 +32,12 @@
3232
/** Factory to create serializer for lookup. */
3333
public interface LookupSerializerFactory {
3434

35-
String identifier();
35+
String version();
3636

3737
Function<InternalRow, byte[]> createSerializer(RowType currentSchema);
3838

3939
Function<byte[], InternalRow> createDeserializer(
40-
RowType currentSchema, @Nullable RowType fileSchema);
40+
String fileSerVersion, RowType currentSchema, @Nullable RowType fileSchema);
4141

4242
Supplier<LookupSerializerFactory> INSTANCE =
4343
Suppliers.memoize(

paimon-core/src/main/java/org/apache/paimon/mergetree/lookup/PersistEmptyProcessor.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,9 @@ public String identifier() {
5353

5454
@Override
5555
public PersistProcessor<Boolean> create(
56-
LookupSerializerFactory serializerFactory, @Nullable RowType fileSchema) {
56+
String fileSerVersion,
57+
LookupSerializerFactory serializerFactory,
58+
@Nullable RowType fileSchema) {
5759
return new PersistEmptyProcessor();
5860
}
5961
};

paimon-core/src/main/java/org/apache/paimon/mergetree/lookup/PersistPositionProcessor.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,9 @@ public String identifier() {
6565

6666
@Override
6767
public PersistProcessor<FilePosition> create(
68-
LookupSerializerFactory serializerFactory, @Nullable RowType fileSchema) {
68+
String fileSerVersion,
69+
LookupSerializerFactory serializerFactory,
70+
@Nullable RowType fileSchema) {
6971
return new PersistPositionProcessor();
7072
}
7173
};

paimon-core/src/main/java/org/apache/paimon/mergetree/lookup/PersistProcessor.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,8 @@ interface Factory<T> {
4343
String identifier();
4444

4545
PersistProcessor<T> create(
46-
LookupSerializerFactory serializerFactory, @Nullable RowType fileSchema);
46+
String fileSerVersion,
47+
LookupSerializerFactory serializerFactory,
48+
@Nullable RowType fileSchema);
4749
}
4850
}

paimon-core/src/main/java/org/apache/paimon/mergetree/lookup/PersistValueAndPosProcessor.java

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -85,10 +85,13 @@ public String identifier() {
8585

8686
@Override
8787
public PersistProcessor<PositionedKeyValue> create(
88-
LookupSerializerFactory serializerFactory, @Nullable RowType fileSchema) {
88+
String fileSerVersion,
89+
LookupSerializerFactory serializerFactory,
90+
@Nullable RowType fileSchema) {
8991
return new PersistValueAndPosProcessor(
9092
serializerFactory.createSerializer(valueType),
91-
serializerFactory.createDeserializer(valueType, fileSchema));
93+
serializerFactory.createDeserializer(
94+
fileSerVersion, valueType, fileSchema));
9295
}
9396
};
9497
}

paimon-core/src/main/java/org/apache/paimon/mergetree/lookup/PersistValueProcessor.java

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -73,10 +73,13 @@ public String identifier() {
7373

7474
@Override
7575
public PersistProcessor<KeyValue> create(
76-
LookupSerializerFactory serializerFactory, @Nullable RowType fileSchema) {
76+
String fileSerVersion,
77+
LookupSerializerFactory serializerFactory,
78+
@Nullable RowType fileSchema) {
7779
return new PersistValueProcessor(
7880
serializerFactory.createSerializer(valueType),
79-
serializerFactory.createDeserializer(valueType, fileSchema));
81+
serializerFactory.createDeserializer(
82+
fileSerVersion, valueType, fileSchema));
8083
}
8184
};
8285
}

paimon-core/src/main/java/org/apache/paimon/mergetree/lookup/RemoteFileDownloader.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,5 +25,5 @@
2525
/** Downloader to try to download remote lookup file to local. */
2626
public interface RemoteFileDownloader {
2727

28-
boolean tryToDownload(DataFileMeta dataFile, File localFile);
28+
boolean tryToDownload(DataFileMeta dataFile, String remoteSstFile, File localFile);
2929
}

0 commit comments

Comments
 (0)