Copilot commented on code in PR #6716:
URL: https://github.com/apache/paimon/pull/6716#discussion_r2596832058


##########
paimon-vector/src/main/java/org/apache/paimon/vector/VectorGlobalIndexWriter.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.vector;
+
+import org.apache.paimon.globalindex.GlobalIndexWriter;
+import org.apache.paimon.globalindex.io.GlobalIndexFileWriter;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.types.ArrayType;
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.utils.Range;
+
+import org.apache.lucene.codecs.KnnVectorsFormat;
+import org.apache.lucene.codecs.lucene912.Lucene912Codec;
+import 
org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.paimon.utils.Preconditions.checkArgument;
+
+/**
+ * Vector global index writer using Apache Lucene 9.x.
+ *
+ * <p>This implementation uses Lucene's native KnnFloatVectorField with HNSW 
algorithm for efficient
+ * approximate nearest neighbor search.
+ */
+public class VectorGlobalIndexWriter implements GlobalIndexWriter {
+
+    private final GlobalIndexFileWriter fileWriter;
+    private final DataType fieldType;
+    private final VectorIndexOptions vectorOptions;
+    private final VectorSimilarityFunction similarityFunction;
+    private final int sizePerIndex;
+
+    private final List<VectorIndex> vectors;
+
+    public VectorGlobalIndexWriter(
+            GlobalIndexFileWriter fileWriter, DataType fieldType, Options 
options) {
+        checkArgument(
+                fieldType instanceof ArrayType,
+                "Vector field type must be ARRAY, but was: " + fieldType);
+        this.fileWriter = fileWriter;
+        this.fieldType = fieldType;
+        this.vectors = new ArrayList<>();
+        this.vectorOptions = new VectorIndexOptions(options);
+        this.similarityFunction = vectorOptions.metric().toLuceneFunction();
+        this.sizePerIndex = vectorOptions.sizePerIndex();
+    }
+
+    @Override
+    public void write(Object key) {
+        if (key instanceof FloatVectorIndex) {
+            FloatVectorIndex vectorKey = (FloatVectorIndex) key;
+            float[] vector = vectorKey.vector();
+
+            checkArgument(
+                    vector.length == vectorOptions.dimension(),
+                    "Vector dimension mismatch: expected "
+                            + vectorOptions.dimension()
+                            + ", but got "
+                            + vector.length);
+
+            vectors.add(vectorKey);
+        } else if (key instanceof ByteVectorIndex) {
+            ByteVectorIndex vectorKey = (ByteVectorIndex) key;
+            byte[] byteVector = vectorKey.vector();
+
+            checkArgument(
+                    byteVector.length == vectorOptions.dimension(),
+                    "Vector dimension mismatch: expected "
+                            + vectorOptions.dimension()
+                            + ", but got "
+                            + byteVector.length);
+
+            vectors.add(vectorKey);
+        } else {
+            throw new IllegalArgumentException(
+                    "Unsupported index type: " + key.getClass().getName());
+        }
+    }
+
+    @Override
+    public List<ResultEntry> finish() {
+        try {
+            if (vectors.isEmpty()) {
+                return new ArrayList<>();
+            }
+
+            List<ResultEntry> results = new ArrayList<>();
+
+            // Split vectors into batches if size exceeds sizePerIndex
+            int totalVectors = vectors.size();
+            int numBatches = (int) Math.ceil((double) totalVectors / 
sizePerIndex);
+
+            for (int batchIndex = 0; batchIndex < numBatches; batchIndex++) {
+                int startIdx = batchIndex * sizePerIndex;
+                int endIdx = Math.min(startIdx + sizePerIndex, totalVectors);
+                List<VectorIndex> batchVectors = vectors.subList(startIdx, 
endIdx);
+
+                // Build index
+                byte[] indexBytes =
+                        buildIndex(
+                                batchVectors,
+                                this.vectorOptions.m(),
+                                this.vectorOptions.efConstruction(),
+                                this.vectorOptions.writeBufferSize());
+
+                // Write to file
+                String fileName = 
fileWriter.newFileName(VectorGlobalIndexerFactory.IDENTIFIER);
+                try (OutputStream out = fileWriter.newOutputStream(fileName)) {
+                    out.write(indexBytes);
+                }
+                long minRowIdInBatch = batchVectors.get(0).rowId();
+                long maxRowIdInBatch = batchVectors.get(batchVectors.size() - 
1).rowId();
+                results.add(
+                        ResultEntry.of(
+                                fileName, null, new Range(minRowIdInBatch, 
maxRowIdInBatch)));
+            }
+
+            return results;
+        } catch (IOException e) {
+            throw new RuntimeException("Failed to write vector global index", 
e);
+        }
+    }
+
+    private byte[] buildIndex(
+            List<VectorIndex> batchVectors, int m, int efConstruction, int 
writeBufferSize)
+            throws IOException {
+
+        try (IndexMMapDirectory indexMMapDirectory = new IndexMMapDirectory()) 
{
+            IndexWriterConfig config = getIndexWriterConfig(m, efConstruction, 
writeBufferSize);
+            try (IndexWriter writer = new 
IndexWriter(indexMMapDirectory.directory(), config)) {
+                for (VectorIndex vectorIndex : batchVectors) {
+                    Document doc = new Document();
+                    doc.add(vectorIndex.indexableField(similarityFunction));
+                    doc.add(vectorIndex.rowIdStoredField());
+                    writer.addDocument(doc);
+                }
+                writer.commit();
+            }
+
+            return serializeDirectory(indexMMapDirectory.directory());
+        }
+    }
+
+    private static IndexWriterConfig getIndexWriterConfig(
+            int m, int efConstruction, int writeBufferSize) {
+        IndexWriterConfig config = new IndexWriterConfig();
+        config.setRAMBufferSizeMB(writeBufferSize);
+        config.setCodec(
+                new Lucene912Codec(Lucene912Codec.Mode.BEST_SPEED) {
+                    @Override
+                    public KnnVectorsFormat getKnnVectorsFormatForField(String 
field) {
+                        return new Lucene99HnswScalarQuantizedVectorsFormat(m, 
efConstruction);
+                    }
+                });
+        return config;
+    }
+
+    private byte[] serializeDirectory(Directory directory) throws IOException {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        String[] files = directory.listAll();
+        baos.write(intToBytes(files.length));
+        for (String fileName : files) {
+            byte[] nameBytes = fileName.getBytes(StandardCharsets.UTF_8);
+            baos.write(intToBytes(nameBytes.length));
+            baos.write(nameBytes);
+
+            long fileLength = directory.fileLength(fileName);
+            baos.write(longToBytes(fileLength));
+
+            try (org.apache.lucene.store.IndexInput input =
+                    directory.openInput(fileName, IOContext.DEFAULT)) {
+                byte[] buffer = new byte[8192];
+                long remaining = fileLength;
+                while (remaining > 0) {
+                    int toRead = (int) Math.min(buffer.length, remaining);
+                    input.readBytes(buffer, 0, toRead);
+                    baos.write(buffer, 0, toRead);
+                    remaining -= toRead;
+                }
+            }
+        }
+
+        return baos.toByteArray();
+    }

Review Comment:
   The `serializeDirectory` method loads the entire index into memory via 
`ByteArrayOutputStream`. For large indexes, this could consume significant 
memory. Consider streaming directly to the output file instead of buffering in 
memory, especially since the method already streams file content with a buffer.



##########
paimon-vector/src/main/java/org/apache/paimon/vector/VectorGlobalIndexWriter.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.vector;
+
+import org.apache.paimon.globalindex.GlobalIndexWriter;
+import org.apache.paimon.globalindex.io.GlobalIndexFileWriter;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.types.ArrayType;
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.utils.Range;
+
+import org.apache.lucene.codecs.KnnVectorsFormat;
+import org.apache.lucene.codecs.lucene912.Lucene912Codec;
+import 
org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.paimon.utils.Preconditions.checkArgument;
+
+/**
+ * Vector global index writer using Apache Lucene 9.x.

Review Comment:
   The class documentation mentions "Apache Lucene 9.x" but is imprecise. Since 
the code uses specific Lucene 9.12+ APIs (Lucene912Codec, 
Lucene99HnswScalarQuantizedVectorsFormat), consider updating to "Apache Lucene 
9.12+" for clarity.
   ```suggestion
    * Vector global index writer using Apache Lucene 9.12+.
   ```



##########
paimon-vector/src/main/java/org/apache/paimon/vector/VectorGlobalIndexReader.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.vector;
+
+import org.apache.paimon.fs.SeekableInputStream;
+import org.apache.paimon.globalindex.GlobalIndexIOMeta;
+import org.apache.paimon.globalindex.GlobalIndexReader;
+import org.apache.paimon.globalindex.GlobalIndexResult;
+import org.apache.paimon.globalindex.io.GlobalIndexFileReader;
+import org.apache.paimon.predicate.FieldRef;
+import org.apache.paimon.utils.RoaringNavigableMap64;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.StoredFields;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.KnnByteVectorQuery;
+import org.apache.lucene.search.KnnFloatVectorQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.IndexOutput;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+/**
+ * Vector global index reader using Apache Lucene 9.x.
+ *
+ * <p>This implementation uses Lucene's native KnnFloatVectorQuery with HNSW 
graph for efficient
+ * approximate nearest neighbor search.
+ */
+public class VectorGlobalIndexReader implements GlobalIndexReader {
+
+    private static final int BUFFER_SIZE = 8192; // 8KB buffer for streaming
+
+    private final List<IndexSearcher> searchers;
+    private final List<IndexMMapDirectory> directories;
+
+    public VectorGlobalIndexReader(GlobalIndexFileReader fileReader, 
List<GlobalIndexIOMeta> files)
+            throws IOException {
+        this.searchers = new ArrayList<>();
+        this.directories = new ArrayList<>();
+        loadIndices(fileReader, files);
+    }
+
+    /**
+     * Search for similar vectors using Lucene KNN search.
+     *
+     * @param query query vector
+     * @param k number of results
+     * @return global index result containing row IDs
+     */
+    public GlobalIndexResult search(float[] query, int k) {
+        KnnFloatVectorQuery knnQuery = new 
KnnFloatVectorQuery(VectorIndex.VECTOR_FIELD, query, k);
+        return search(knnQuery, k);
+    }
+
+    public GlobalIndexResult search(byte[] query, int k) {
+        KnnByteVectorQuery knnQuery = new 
KnnByteVectorQuery(VectorIndex.VECTOR_FIELD, query, k);
+        return search(knnQuery, k);
+    }
+
+    @Override
+    public void close() throws IOException {
+        // Close readers
+        for (IndexSearcher searcher : searchers) {
+            searcher.getIndexReader().close();
+        }
+        searchers.clear();
+
+        // Close directories
+        for (IndexMMapDirectory directory : directories) {
+            directory.close();
+        }
+        directories.clear();
+    }
+
+    private GlobalIndexResult search(Query query, int k) {
+        PriorityQueue<ScoredRow> topK =
+                new PriorityQueue<>(Comparator.comparingDouble(sr -> 
sr.score));
+        for (IndexSearcher searcher : searchers) {
+            try {
+                TopDocs topDocs = searcher.search(query, k);
+                StoredFields storedFields = searcher.storedFields();
+                Set<String> fieldsToLoad = Set.of(VectorIndex.ROW_ID_FIELD);
+                for (org.apache.lucene.search.ScoreDoc scoreDoc : 
topDocs.scoreDocs) {
+                    Document doc = storedFields.document(scoreDoc.doc, 
fieldsToLoad);
+                    long rowId = 
doc.getField(VectorIndex.ROW_ID_FIELD).numericValue().longValue();
+                    if (topK.size() < k) {
+                        topK.offer(new ScoredRow(rowId, scoreDoc.score));
+                    } else {
+                        if (topK.peek() != null && scoreDoc.score > 
topK.peek().score) {

Review Comment:
   Redundant null check: `topK.peek() != null` is unnecessary because 
`topK.size() < k` is already checked earlier. If `topK.size() >= k`, then 
`topK.peek()` is guaranteed to be non-null. The null check can be removed: `if 
(scoreDoc.score > topK.peek().score)`.
   ```suggestion
                           if (scoreDoc.score > topK.peek().score) {
   ```



##########
paimon-vector/src/main/java/org/apache/paimon/vector/VectorGlobalIndexReader.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.vector;
+
+import org.apache.paimon.fs.SeekableInputStream;
+import org.apache.paimon.globalindex.GlobalIndexIOMeta;
+import org.apache.paimon.globalindex.GlobalIndexReader;
+import org.apache.paimon.globalindex.GlobalIndexResult;
+import org.apache.paimon.globalindex.io.GlobalIndexFileReader;
+import org.apache.paimon.predicate.FieldRef;
+import org.apache.paimon.utils.RoaringNavigableMap64;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.StoredFields;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.KnnByteVectorQuery;
+import org.apache.lucene.search.KnnFloatVectorQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.IndexOutput;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+/**
+ * Vector global index reader using Apache Lucene 9.x.
+ *
+ * <p>This implementation uses Lucene's native KnnFloatVectorQuery with HNSW 
graph for efficient
+ * approximate nearest neighbor search.
+ */
+public class VectorGlobalIndexReader implements GlobalIndexReader {
+
+    private static final int BUFFER_SIZE = 8192; // 8KB buffer for streaming
+
+    private final List<IndexSearcher> searchers;
+    private final List<IndexMMapDirectory> directories;
+
+    public VectorGlobalIndexReader(GlobalIndexFileReader fileReader, 
List<GlobalIndexIOMeta> files)
+            throws IOException {
+        this.searchers = new ArrayList<>();
+        this.directories = new ArrayList<>();
+        loadIndices(fileReader, files);
+    }
+
+    /**
+     * Search for similar vectors using Lucene KNN search.
+     *
+     * @param query query vector
+     * @param k number of results
+     * @return global index result containing row IDs
+     */
+    public GlobalIndexResult search(float[] query, int k) {
+        KnnFloatVectorQuery knnQuery = new 
KnnFloatVectorQuery(VectorIndex.VECTOR_FIELD, query, k);
+        return search(knnQuery, k);
+    }
+
+    public GlobalIndexResult search(byte[] query, int k) {
+        KnnByteVectorQuery knnQuery = new 
KnnByteVectorQuery(VectorIndex.VECTOR_FIELD, query, k);
+        return search(knnQuery, k);
+    }
+
+    @Override
+    public void close() throws IOException {
+        // Close readers
+        for (IndexSearcher searcher : searchers) {
+            searcher.getIndexReader().close();
+        }
+        searchers.clear();
+
+        // Close directories
+        for (IndexMMapDirectory directory : directories) {
+            directory.close();
+        }
+        directories.clear();
+    }
+
+    private GlobalIndexResult search(Query query, int k) {
+        PriorityQueue<ScoredRow> topK =
+                new PriorityQueue<>(Comparator.comparingDouble(sr -> 
sr.score));
+        for (IndexSearcher searcher : searchers) {
+            try {
+                TopDocs topDocs = searcher.search(query, k);
+                StoredFields storedFields = searcher.storedFields();
+                Set<String> fieldsToLoad = Set.of(VectorIndex.ROW_ID_FIELD);
+                for (org.apache.lucene.search.ScoreDoc scoreDoc : 
topDocs.scoreDocs) {
+                    Document doc = storedFields.document(scoreDoc.doc, 
fieldsToLoad);
+                    long rowId = 
doc.getField(VectorIndex.ROW_ID_FIELD).numericValue().longValue();
+                    if (topK.size() < k) {
+                        topK.offer(new ScoredRow(rowId, scoreDoc.score));
+                    } else {
+                        if (topK.peek() != null && scoreDoc.score > 
topK.peek().score) {
+                            topK.poll();
+                            topK.offer(new ScoredRow(rowId, scoreDoc.score));
+                        }
+                    }
+                }
+            } catch (IOException e) {
+                throw new RuntimeException("Failed to search vector index", e);
+            }
+        }
+        RoaringNavigableMap64 roaringBitmap64 = new RoaringNavigableMap64();
+        for (ScoredRow scoredRow : topK) {
+            roaringBitmap64.add(scoredRow.rowId);
+        }
+        return GlobalIndexResult.create(() -> roaringBitmap64);
+    }
+
+    /** Helper class to store row ID with its score. */
+    private static class ScoredRow {
+        final long rowId;
+        final float score;
+
+        ScoredRow(long rowId, float score) {
+            this.rowId = rowId;
+            this.score = score;
+        }
+    }
+
+    private void loadIndices(GlobalIndexFileReader fileReader, 
List<GlobalIndexIOMeta> files)
+            throws IOException {
+        for (GlobalIndexIOMeta meta : files) {
+            try (SeekableInputStream in = 
fileReader.getInputStream(meta.fileName())) {
+                IndexMMapDirectory directory = deserializeDirectory(in);
+                directories.add(directory);
+                IndexReader reader = 
DirectoryReader.open(directory.directory());
+                IndexSearcher searcher = new IndexSearcher(reader);
+                searchers.add(searcher);

Review Comment:
   Resource leak potential: If `deserializeDirectory(in)` throws an exception 
after `in` is opened but before being added to the try-with-resources, the 
`SeekableInputStream` won't be closed. The same applies if 
`DirectoryReader.open()` throws an exception. Consider wrapping the entire 
block in try-catch to ensure proper cleanup, or ensure that 
`deserializeDirectory` properly closes `in` on error.
   ```suggestion
               // Open input stream for the file
               try (SeekableInputStream in = 
fileReader.getInputStream(meta.fileName())) {
                   IndexMMapDirectory directory = null;
                   IndexReader reader = null;
                   boolean success = false;
                   try {
                       directory = deserializeDirectory(in);
                       reader = DirectoryReader.open(directory.directory());
                       IndexSearcher searcher = new IndexSearcher(reader);
                       // Only add to lists after all resources are 
successfully created
                       directories.add(directory);
                       searchers.add(searcher);
                       success = true;
                   } finally {
                       if (!success) {
                           // Clean up resources if an exception occurred
                           if (reader != null) {
                               try {
                                   reader.close();
                               } catch (IOException e) {
                                   // Suppress secondary exception
                               }
                           }
                           if (directory != null) {
                               try {
                                   directory.close();
                               } catch (IOException e) {
                                   // Suppress secondary exception
                               }
                           }
                       }
                   }
   ```



##########
paimon-vector/src/main/java/org/apache/paimon/vector/VectorGlobalIndexWriter.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.vector;
+
+import org.apache.paimon.globalindex.GlobalIndexWriter;
+import org.apache.paimon.globalindex.io.GlobalIndexFileWriter;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.types.ArrayType;
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.utils.Range;
+
+import org.apache.lucene.codecs.KnnVectorsFormat;
+import org.apache.lucene.codecs.lucene912.Lucene912Codec;
+import 
org.apache.lucene.codecs.lucene99.Lucene99HnswScalarQuantizedVectorsFormat;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.paimon.utils.Preconditions.checkArgument;
+
+/**
+ * Vector global index writer using Apache Lucene 9.x.
+ *
+ * <p>This implementation uses Lucene's native KnnFloatVectorField with HNSW 
algorithm for efficient
+ * approximate nearest neighbor search.
+ */
+public class VectorGlobalIndexWriter implements GlobalIndexWriter {
+
+    private final GlobalIndexFileWriter fileWriter;
+    private final DataType fieldType;
+    private final VectorIndexOptions vectorOptions;
+    private final VectorSimilarityFunction similarityFunction;
+    private final int sizePerIndex;
+
+    private final List<VectorIndex> vectors;
+
+    public VectorGlobalIndexWriter(
+            GlobalIndexFileWriter fileWriter, DataType fieldType, Options 
options) {
+        checkArgument(
+                fieldType instanceof ArrayType,
+                "Vector field type must be ARRAY, but was: " + fieldType);
+        this.fileWriter = fileWriter;
+        this.fieldType = fieldType;
+        this.vectors = new ArrayList<>();
+        this.vectorOptions = new VectorIndexOptions(options);
+        this.similarityFunction = vectorOptions.metric().toLuceneFunction();
+        this.sizePerIndex = vectorOptions.sizePerIndex();
+    }
+
+    @Override
+    public void write(Object key) {
+        if (key instanceof FloatVectorIndex) {
+            FloatVectorIndex vectorKey = (FloatVectorIndex) key;
+            float[] vector = vectorKey.vector();
+
+            checkArgument(
+                    vector.length == vectorOptions.dimension(),
+                    "Vector dimension mismatch: expected "
+                            + vectorOptions.dimension()
+                            + ", but got "
+                            + vector.length);
+
+            vectors.add(vectorKey);
+        } else if (key instanceof ByteVectorIndex) {
+            ByteVectorIndex vectorKey = (ByteVectorIndex) key;
+            byte[] byteVector = vectorKey.vector();
+
+            checkArgument(
+                    byteVector.length == vectorOptions.dimension(),
+                    "Vector dimension mismatch: expected "
+                            + vectorOptions.dimension()
+                            + ", but got "
+                            + byteVector.length);
+
+            vectors.add(vectorKey);
+        } else {
+            throw new IllegalArgumentException(
+                    "Unsupported index type: " + key.getClass().getName());
+        }
+    }
+
+    @Override
+    public List<ResultEntry> finish() {
+        try {
+            if (vectors.isEmpty()) {
+                return new ArrayList<>();
+            }
+
+            List<ResultEntry> results = new ArrayList<>();
+
+            // Split vectors into batches if size exceeds sizePerIndex
+            int totalVectors = vectors.size();
+            int numBatches = (int) Math.ceil((double) totalVectors / 
sizePerIndex);
+
+            for (int batchIndex = 0; batchIndex < numBatches; batchIndex++) {
+                int startIdx = batchIndex * sizePerIndex;
+                int endIdx = Math.min(startIdx + sizePerIndex, totalVectors);
+                List<VectorIndex> batchVectors = vectors.subList(startIdx, 
endIdx);
+
+                // Build index
+                byte[] indexBytes =
+                        buildIndex(
+                                batchVectors,
+                                this.vectorOptions.m(),
+                                this.vectorOptions.efConstruction(),
+                                this.vectorOptions.writeBufferSize());
+
+                // Write to file
+                String fileName = 
fileWriter.newFileName(VectorGlobalIndexerFactory.IDENTIFIER);
+                try (OutputStream out = fileWriter.newOutputStream(fileName)) {
+                    out.write(indexBytes);
+                }
+                long minRowIdInBatch = batchVectors.get(0).rowId();
+                long maxRowIdInBatch = batchVectors.get(batchVectors.size() - 
1).rowId();
+                results.add(
+                        ResultEntry.of(
+                                fileName, null, new Range(minRowIdInBatch, 
maxRowIdInBatch)));
+            }
+
+            return results;
+        } catch (IOException e) {
+            throw new RuntimeException("Failed to write vector global index", 
e);
+        }
+    }
+
+    private byte[] buildIndex(
+            List<VectorIndex> batchVectors, int m, int efConstruction, int 
writeBufferSize)
+            throws IOException {
+
+        try (IndexMMapDirectory indexMMapDirectory = new IndexMMapDirectory()) 
{
+            IndexWriterConfig config = getIndexWriterConfig(m, efConstruction, 
writeBufferSize);
+            try (IndexWriter writer = new 
IndexWriter(indexMMapDirectory.directory(), config)) {
+                for (VectorIndex vectorIndex : batchVectors) {
+                    Document doc = new Document();
+                    doc.add(vectorIndex.indexableField(similarityFunction));
+                    doc.add(vectorIndex.rowIdStoredField());
+                    writer.addDocument(doc);
+                }
+                writer.commit();
+            }
+
+            return serializeDirectory(indexMMapDirectory.directory());
+        }
+    }

Review Comment:
   The `buildIndex` method creates a temporary `IndexMMapDirectory` that may 
fail to close properly if an exception occurs after directory creation but 
before the try-with-resources completes. If `getIndexWriterConfig` or 
`IndexWriter` construction throws an exception, the directory won't be closed, 
potentially leaving temporary files. Consider restructuring to ensure the 
directory is always cleaned up.



##########
paimon-vector/src/main/java/org/apache/paimon/vector/VectorGlobalIndexReader.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.vector;
+
+import org.apache.paimon.fs.SeekableInputStream;
+import org.apache.paimon.globalindex.GlobalIndexIOMeta;
+import org.apache.paimon.globalindex.GlobalIndexReader;
+import org.apache.paimon.globalindex.GlobalIndexResult;
+import org.apache.paimon.globalindex.io.GlobalIndexFileReader;
+import org.apache.paimon.predicate.FieldRef;
+import org.apache.paimon.utils.RoaringNavigableMap64;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.StoredFields;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.KnnByteVectorQuery;
+import org.apache.lucene.search.KnnFloatVectorQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.IndexOutput;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Set;
+
+/**
+ * Vector global index reader using Apache Lucene 9.x.

Review Comment:
   The class documentation mentions "Apache Lucene 9.x" but should be more 
specific. Consider updating to "Apache Lucene 9.12+" to match the actual 
dependency version and APIs used.
   ```suggestion
    * Vector global index reader using Apache Lucene 9.12+.
   ```



##########
paimon-vector/src/test/java/org/apache/paimon/vector/VectorGlobalIndexTest.java:
##########
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.vector;
+
+import org.apache.paimon.fs.FileIO;
+import org.apache.paimon.fs.Path;
+import org.apache.paimon.fs.local.LocalFileIO;
+import org.apache.paimon.globalindex.GlobalIndexIOMeta;
+import org.apache.paimon.globalindex.GlobalIndexResult;
+import org.apache.paimon.globalindex.GlobalIndexWriter;
+import org.apache.paimon.globalindex.io.GlobalIndexFileReader;
+import org.apache.paimon.globalindex.io.GlobalIndexFileWriter;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.types.ArrayType;
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.types.FloatType;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test for {@link VectorGlobalIndexWriter} and {@link 
VectorGlobalIndexReader}. */
+public class VectorGlobalIndexTest {
+
+    @TempDir java.nio.file.Path tempDir;
+
+    private FileIO fileIO;
+    private Path indexPath;
+    private DataType vectorType;
+
+    @BeforeEach
+    public void setup() {
+        fileIO = new LocalFileIO();
+        indexPath = new Path(tempDir.toString());
+        vectorType = new ArrayType(new FloatType());
+    }
+
+    @AfterEach
+    public void cleanup() throws IOException {
+        if (fileIO != null) {
+            fileIO.delete(indexPath, true);
+        }
+    }
+
+    private GlobalIndexFileWriter createFileWriter(Path path) {
+        return new GlobalIndexFileWriter() {
+            @Override
+            public String newFileName(String prefix) {
+                return prefix + "-" + UUID.randomUUID();
+            }
+
+            @Override
+            public OutputStream newOutputStream(String fileName) throws 
IOException {
+                return fileIO.newOutputStream(new Path(path, fileName), false);
+            }
+        };
+    }
+
+    private GlobalIndexFileReader createFileReader(Path path) {
+        return fileName -> fileIO.newInputStream(new Path(path, fileName));
+    }
+
+    @Test
+    public void testDifferentSimilarityFunctions() throws IOException {
+        int dimension = 32;
+        int numVectors = 20;
+
+        String[] metrics = {"COSINE", "DOT_PRODUCT", "EUCLIDEAN", 
"MAX_INNER_PRODUCT"};
+
+        for (String metric : metrics) {
+            Options options = createDefaultOptions(dimension);
+            options.setString("vector.metric", metric);
+
+            Path metricIndexPath = new Path(indexPath, metric.toLowerCase());
+            GlobalIndexFileWriter fileWriter = 
createFileWriter(metricIndexPath);
+            VectorGlobalIndexWriter writer =
+                    new VectorGlobalIndexWriter(fileWriter, vectorType, 
options);
+
+            List<float[]> testVectors = generateRandomVectors(numVectors, 
dimension);
+            for (int i = 0; i < numVectors; i++) {
+                writer.write(new FloatVectorIndex(i, testVectors.get(i)));
+            }
+
+            List<GlobalIndexWriter.ResultEntry> results = writer.finish();
+            assertThat(results).hasSize(1);
+
+            GlobalIndexWriter.ResultEntry result = results.get(0);
+            GlobalIndexFileReader fileReader = 
createFileReader(metricIndexPath);
+            List<GlobalIndexIOMeta> metas = new ArrayList<>();
+            metas.add(
+                    new GlobalIndexIOMeta(
+                            result.fileName(),
+                            fileIO.getFileSize(new Path(metricIndexPath, 
result.fileName())),
+                            result.rowRange(),
+                            result.meta()));
+
+            VectorGlobalIndexReader reader = new 
VectorGlobalIndexReader(fileReader, metas);
+
+            // Verify search works with this metric
+            GlobalIndexResult searchResult = reader.search(testVectors.get(0), 
3);
+            assertThat(searchResult).isNotNull();
+
+            reader.close();
+        }
+    }
+
+    @Test
+    public void testDifferentDimensions() throws IOException {
+        int[] dimensions = {8, 32, 128, 256};
+
+        for (int dimension : dimensions) {
+            Options options = createDefaultOptions(dimension);
+
+            Path dimIndexPath = new Path(indexPath, "dim_" + dimension);
+            GlobalIndexFileWriter fileWriter = createFileWriter(dimIndexPath);
+            VectorGlobalIndexWriter writer =
+                    new VectorGlobalIndexWriter(fileWriter, vectorType, 
options);
+
+            int numVectors = 10;
+            List<float[]> testVectors = generateRandomVectors(numVectors, 
dimension);
+            for (int i = 0; i < numVectors; i++) {
+                writer.write(new FloatVectorIndex(i, testVectors.get(i)));
+            }
+
+            List<GlobalIndexWriter.ResultEntry> results = writer.finish();
+            assertThat(results).hasSize(1);
+
+            GlobalIndexWriter.ResultEntry result = results.get(0);
+            GlobalIndexFileReader fileReader = createFileReader(dimIndexPath);
+            List<GlobalIndexIOMeta> metas = new ArrayList<>();
+            metas.add(
+                    new GlobalIndexIOMeta(
+                            result.fileName(),
+                            fileIO.getFileSize(new Path(dimIndexPath, 
result.fileName())),
+                            result.rowRange(),
+                            result.meta()));
+
+            VectorGlobalIndexReader reader = new 
VectorGlobalIndexReader(fileReader, metas);
+
+            // Verify search works with this dimension
+            GlobalIndexResult searchResult = reader.search(testVectors.get(0), 
5);
+            assertThat(searchResult).isNotNull();
+
+            reader.close();
+        }
+    }
+
+    @Test
+    public void testDimensionMismatch() throws IOException {
+        Options options = createDefaultOptions(64);
+
+        GlobalIndexFileWriter fileWriter = createFileWriter(indexPath);
+        VectorGlobalIndexWriter writer =
+                new VectorGlobalIndexWriter(fileWriter, vectorType, options);
+
+        // Try to write vector with wrong dimension
+        float[] wrongDimVector = new float[32]; // Wrong dimension
+        assertThatThrownBy(() -> writer.write(new FloatVectorIndex(0, 
wrongDimVector)))
+                .isInstanceOf(IllegalArgumentException.class)
+                .hasMessageContaining("dimension mismatch");
+    }
+
+    @Test
+    public void testFloatVectorIndexEndToEnd() throws IOException {
+        int dimension = 2;
+        Options options = createDefaultOptions(dimension);
+        options.setInteger("vector.size-per-index", 3);
+
+        float[][] vectors =
+                new float[][] {
+                    new float[] {1.0f, 0.0f}, new float[] {0.95f, 0.1f}, new 
float[] {0.1f, 0.95f},
+                    new float[] {0.98f, 0.05f}, new float[] {0.0f, 1.0f}, new 
float[] {0.05f, 0.98f}
+                };
+
+        GlobalIndexFileWriter fileWriter = createFileWriter(indexPath);
+        VectorGlobalIndexWriter writer =
+                new VectorGlobalIndexWriter(fileWriter, vectorType, options);
+        for (int i = 0; i < vectors.length; i++) {
+            writer.write(new FloatVectorIndex(i, vectors[i]));
+        }
+
+        List<GlobalIndexWriter.ResultEntry> results = writer.finish();
+        assertThat(results).hasSize(2);
+
+        GlobalIndexFileReader fileReader = createFileReader(indexPath);
+        List<GlobalIndexIOMeta> metas = new ArrayList<>();
+        for (GlobalIndexWriter.ResultEntry result : results) {
+            metas.add(
+                    new GlobalIndexIOMeta(
+                            result.fileName(),
+                            fileIO.getFileSize(new Path(indexPath, 
result.fileName())),
+                            result.rowRange(),
+                            result.meta()));
+        }
+
+        VectorGlobalIndexReader reader = new 
VectorGlobalIndexReader(fileReader, metas);
+
+        GlobalIndexResult result = reader.search(vectors[0], 1);
+        assertThat(result.results().getLongCardinality()).isEqualTo(1);
+        assertThat(containsRowId(result, 0)).isTrue();
+
+        float[] queryVector = new float[] {0.85f, 0.15f};
+        result = reader.search(queryVector, 2);
+        assertThat(result.results().getLongCardinality()).isEqualTo(2);
+        assertThat(containsRowId(result, 1)).isTrue();
+        assertThat(containsRowId(result, 3)).isTrue();
+        reader.close();
+    }
+
+    private Options createDefaultOptions(int dimension) {
+        Options options = new Options();
+        options.setInteger("vector.dim", dimension);
+        options.setString("vector.metric", "EUCLIDEAN");
+        options.setInteger("vector.m", 16);
+        options.setInteger("vector.ef-construction", 100);
+        return options;
+    }
+
+    private List<float[]> generateRandomVectors(int count, int dimension) {
+        Random random = new Random(42);
+        List<float[]> vectors = new ArrayList<>();
+        for (int i = 0; i < count; i++) {
+            float[] vector = new float[dimension];
+            for (int j = 0; j < dimension; j++) {
+                vector[j] = random.nextFloat() * 2 - 1; // Range [-1, 1]
+            }
+            // Normalize for cosine similarity
+            float norm = 0;
+            for (float v : vector) {
+                norm += v * v;
+            }
+            norm = (float) Math.sqrt(norm);
+            if (norm > 0) {
+                for (int m = 0; m < vector.length; m++) {
+                    vector[m] /= norm;
+                }
+            }
+            vectors.add(vector);
+        }
+        return vectors;
+    }
+
+    private boolean containsRowId(GlobalIndexResult result, long rowId) {
+        List<Long> resultIds = new ArrayList<>();
+        result.results().iterator().forEachRemaining(resultIds::add);
+        return resultIds.contains(rowId);
+    }
+}

Review Comment:
   The test file lacks coverage for `ByteVectorIndex` functionality. While 
`FloatVectorIndex` is extensively tested, there are no tests for byte vectors 
despite the implementation supporting both. Consider adding a test similar to 
`testFloatVectorIndexEndToEnd` but using `ByteVectorIndex` and the 
`search(byte[], int)` method.



##########
paimon-vector/src/main/java/org/apache/paimon/vector/IndexMMapDirectory.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.vector;
+
+import org.apache.lucene.store.MMapDirectory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.UUID;
+
+/** A wrapper of MMapDirectory for vector index. */
+public class IndexMMapDirectory implements AutoCloseable {
+    private final Path path;
+    private final MMapDirectory mmapDirectory;
+
+    public IndexMMapDirectory() throws IOException {
+        this.path =
+                java.nio.file.Files.createTempDirectory("paimon-vector-index" 
+ UUID.randomUUID());

Review Comment:
   The UUID is concatenated directly to the prefix string without a separator. 
This creates unclear directory names like "paimon-vector-indexabc-123-def". 
Consider adding a hyphen separator: `"paimon-vector-index-" + UUID.randomUUID()`
   ```suggestion
                   
java.nio.file.Files.createTempDirectory("paimon-vector-index-" + 
UUID.randomUUID());
   ```



##########
paimon-vector/src/main/java/org/apache/paimon/vector/IndexMMapDirectory.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.vector;
+
+import org.apache.lucene.store.MMapDirectory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.UUID;
+
+/** A wrapper of MMapDirectory for vector index. */
+public class IndexMMapDirectory implements AutoCloseable {
+    private final Path path;
+    private final MMapDirectory mmapDirectory;
+
+    public IndexMMapDirectory() throws IOException {
+        this.path =
+                java.nio.file.Files.createTempDirectory("paimon-vector-index" 
+ UUID.randomUUID());
+        this.mmapDirectory = new MMapDirectory(path);
+    }
+
+    public MMapDirectory directory() {
+        return mmapDirectory;
+    }
+
+    public void close() throws IOException {
+        mmapDirectory.close();
+        if (java.nio.file.Files.exists(path)) {
+            java.nio.file.Files.walk(path)
+                    .sorted(java.util.Comparator.reverseOrder())
+                    .forEach(
+                            p -> {
+                                try {
+                                    java.nio.file.Files.delete(p);
+                                } catch (IOException e) {
+                                    // Ignore cleanup errors
+                                }
+                            });

Review Comment:
   The `close()` method silently ignores file deletion errors in the catch 
block. While this may be intentional for cleanup, it could hide legitimate I/O 
issues and leave orphaned temporary directories. Consider logging the exception 
at least at debug level to aid troubleshooting, e.g., `// Log exception for 
debugging purposes`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to