Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
18 changes: 18 additions & 0 deletions docs/layouts/shortcodes/generated/core_configuration.html
Original file line number Diff line number Diff line change
Expand Up @@ -1452,6 +1452,24 @@
<td>String</td>
<td>The Variant shredding schema for writing.</td>
</tr>
<tr>
<td><h5>vector-field</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Specify the vector store fields.</td>
</tr>
<tr>
<td><h5>vector.file.format</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>String</td>
<td>Specify the vector store file format.</td>
</tr>
<tr>
<td><h5>vector.target-file-size</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>MemorySize</td>
<td>Target size of a vector-store file. Default is 10 * TARGET_FILE_SIZE.</td>
</tr>
<tr>
<td><h5>visibility-callback.check-interval</h5></td>
<td style="word-wrap: break-word;">10 s</td>
Expand Down
43 changes: 43 additions & 0 deletions paimon-api/src/main/java/org/apache/paimon/CoreOptions.java
Original file line number Diff line number Diff line change
Expand Up @@ -2230,6 +2230,29 @@ public InlineElement getDescription() {
.withDescription(
"The interval for checking visibility when visibility-callback enabled.");

public static final ConfigOption<String> VECTOR_STORE_FORMAT =
key("vector.file.format")
.stringType()
.noDefaultValue()
.withDescription("Specify the vector store file format.");

public static final ConfigOption<String> VECTOR_STORE_FIELDS =
key("vector-field")
.stringType()
.noDefaultValue()
.withDescription("Specify the vector store fields.");

public static final ConfigOption<MemorySize> VECTOR_STORE_TARGET_FILE_SIZE =
key("vector.target-file-size")
.memoryType()
.noDefaultValue()
.withDescription(
Description.builder()
.text(
"Target size of a vector-store file."
+ " Default is 10 * TARGET_FILE_SIZE.")
.build());

private final Options options;

public CoreOptions(Map<String, String> options) {
Expand Down Expand Up @@ -3469,6 +3492,26 @@ public Duration visibilityCallbackCheckInterval() {
return options.get(VISIBILITY_CALLBACK_CHECK_INTERVAL);
}

public String vectorStoreFileFormatString() {
return normalizeFileFormat(options.get(VECTOR_STORE_FORMAT));
}

public List<String> vectorStoreFieldNames() {
String vectorStoreFields = options.get(CoreOptions.VECTOR_STORE_FIELDS);
if (vectorStoreFields == null || vectorStoreFields.trim().isEmpty()) {
return new ArrayList<>();
} else {
return Arrays.asList(vectorStoreFields.split(","));
}
}

public long vectorStoreTargetFileSize() {
// Since vectors are large, it would be better to set a larger target size for vectors.
return options.getOptional(VECTOR_STORE_TARGET_FILE_SIZE)
.map(MemorySize::getBytes)
.orElse(10 * targetFileSize(false));
}

/** Specifies the merge engine for table with primary key. */
public enum MergeEngine implements DescribedEnum {
DEDUPLICATE("deduplicate", "De-duplicate and keep the last row."),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,17 @@ public static FileFormat fileFormat(CoreOptions options) {
return FileFormat.fromIdentifier(options.fileFormatString(), options.toConfiguration());
}

public static FileFormat vectorStoreFileFormat(CoreOptions options) {
if (options.vectorStoreFieldNames().isEmpty()) {
return null;
}
String vectorStoreFileFormat = options.vectorStoreFileFormatString();
if (vectorStoreFileFormat == null) {
return fileFormat(options);
}
return FileFormat.fromIdentifier(vectorStoreFileFormat, options.toConfiguration());
}

public static FileFormat manifestFormat(CoreOptions options) {
return FileFormat.fromIdentifier(options.manifestFormatString(), options.toConfiguration());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
import org.apache.paimon.utils.SinkWriter.BufferedSinkWriter;
import org.apache.paimon.utils.SinkWriter.DirectSinkWriter;
import org.apache.paimon.utils.StatsCollectorFactories;
import org.apache.paimon.utils.VectorStoreUtils;

import javax.annotation.Nullable;

Expand All @@ -73,8 +74,11 @@ public class AppendOnlyWriter implements BatchRecordWriter, MemoryOwner {
private final FileIO fileIO;
private final long schemaId;
private final FileFormat fileFormat;
private final FileFormat vectorStoreFileFormat;
private final List<String> vectorStoreFieldNames;
private final long targetFileSize;
private final long blobTargetFileSize;
private final long vectorStoreTargetFileSize;
private final RowType writeSchema;
@Nullable private final List<String> writeCols;
private final DataFilePathFactory pathFactory;
Expand Down Expand Up @@ -105,8 +109,11 @@ public AppendOnlyWriter(
@Nullable IOManager ioManager,
long schemaId,
FileFormat fileFormat,
FileFormat vectorStoreFileFormat,
List<String> vectorStoreFieldNames,
long targetFileSize,
long blobTargetFileSize,
long vectorStoreTargetFileSize,
RowType writeSchema,
@Nullable List<String> writeCols,
long maxSequenceNumber,
Expand All @@ -129,8 +136,11 @@ public AppendOnlyWriter(
this.fileIO = fileIO;
this.schemaId = schemaId;
this.fileFormat = fileFormat;
this.vectorStoreFileFormat = vectorStoreFileFormat;
this.vectorStoreFieldNames = vectorStoreFieldNames;
this.targetFileSize = targetFileSize;
this.blobTargetFileSize = blobTargetFileSize;
this.vectorStoreTargetFileSize = vectorStoreTargetFileSize;
this.writeSchema = writeSchema;
this.writeCols = writeCols;
this.pathFactory = pathFactory;
Expand Down Expand Up @@ -304,13 +314,25 @@ public void toBufferedWriter() throws Exception {
}

private RollingFileWriter<InternalRow, DataFileMeta> createRollingRowWriter() {
if (writeSchema.getFieldTypes().stream().anyMatch(t -> t.is(BLOB))) {
return new RollingBlobFileWriter(
boolean hasNormal =
writeSchema.getFields().stream()
.anyMatch(
f ->
!f.type().is(BLOB)
&& !vectorStoreFieldNames.contains(f.name()));
boolean hasBlob = writeSchema.getFieldTypes().stream().anyMatch(t -> t.is(BLOB));
boolean hasSeparatedVectorStore =
VectorStoreUtils.isDifferentFormat(vectorStoreFileFormat, fileFormat);
if (hasBlob || (hasNormal && hasSeparatedVectorStore)) {
return new DataEvolutionRollingFileWriter(
fileIO,
schemaId,
fileFormat,
vectorStoreFileFormat,
vectorStoreFieldNames,
targetFileSize,
blobTargetFileSize,
vectorStoreTargetFileSize,
writeSchema,
pathFactory,
seqNumCounterProvider,
Expand All @@ -326,13 +348,20 @@ private RollingFileWriter<InternalRow, DataFileMeta> createRollingRowWriter() {
statsDenseStore,
blobConsumer);
}
FileFormat realFileFormat = hasNormal ? fileFormat : vectorStoreFileFormat;
long realTargetFileSize = hasNormal ? targetFileSize : vectorStoreTargetFileSize;
DataFilePathFactory realPathFactory =
hasNormal
? pathFactory
: pathFactory.vectorStorePathFactory(
vectorStoreFileFormat.getFormatIdentifier());
return new RowDataRollingFileWriter(
fileIO,
schemaId,
fileFormat,
targetFileSize,
realFileFormat,
realTargetFileSize,
writeSchema,
pathFactory,
realPathFactory,
seqNumCounterProvider,
fileCompression,
statsCollectorFactories.statsCollectors(writeSchema.getFieldNames()),
Expand Down
Loading