Skip to content

Conversation

@XuQianJin-Stars
Copy link
Contributor

@XuQianJin-Stars XuQianJin-Stars commented Jan 11, 2026

Purpose

Linked issue: close #1569

Refactor fluss-lake-lance module to eliminate code duplication by reusing fluss-common's ArrowFieldWriter implementations.
Issue: The fluss-lake-lance module previously maintained a separate LanceArrowWriter with duplicate FieldWriter implementations because fluss-common uses shaded Arrow API while Lance library requires non-shaded Arrow API.
Solution: Implement ArrowDataConverter to bridge shaded and non-shaded Arrow via zero-copy off-heap memory sharing. This allows lance module to reuse fluss-common's ArrowFieldWriter implementations, eliminating the need for duplicate writer code.

Brief change log

1. Eliminated Code Duplication (~400 lines removed)

  • Removed LanceArrowWriter class and all its inner FieldWriter classes from fluss-lake-lance
  • No longer need to maintain separate writer implementations for lance module
  • All Arrow writing logic now unified through fluss-common's ArrowFieldWriter

2. Created ArrowDataConverter (Zero-Copy Bridge)

  • Implements efficient conversion between shaded and non-shaded Arrow VectorSchemaRoot
  • Uses direct off-heap memory sharing via ByteBuffer transfer (zero serialization overhead)
  • Key insight: Both shaded and non-shaded Arrow use identical off-heap memory layout
  • Extracts ByteBuffer from shaded ArrowBuf and directly copies to non-shaded ArrowBuf

3. Created ShadedArrowBatchWriter (Reuse Adapter)

  • Wraps shaded Arrow VectorSchemaRoot and reuses ArrowUtils.createArrowFieldWriter()
  • Provides simple batch writing interface: writeRow(), finish(), reset()
  • Enables lance module to leverage all existing ArrowFieldWriter implementations from fluss-common

4. Refactored LanceLakeWriter (Unified Writer Path)

  • Changed from non-shaded Arrow writer to shaded Arrow writer + converter
  • Write path: InternalRowShadedArrowBatchWriterArrowDataConverterLance Fragment.create()
  • Uses ArrayList buffer to collect rows, writes in batches
  • Maintains both shaded and non-shaded allocators for conversion process

5. Architecture Benefits

  • Single source of truth: Only fluss-common maintains ArrowFieldWriter implementations
  • Zero maintenance overhead: New data type support only needs changes in fluss-common
  • Performance: Zero-copy memory sharing eliminates serialization overhead
  • Consistency: Lance and other modules use identical Arrow writing logic

Tests

Unit Tests

  • LanceTieringTest.testTieringWriteTable (with/without partitions)
  • LakeEnabledTableCreateITCase (table creation with various data types)

Verification

  • Compilation successful
  • No behavioral changes to external APIs
  • Off-heap memory sharing verified through reflection-based buffer access

API and Format

API Changes: None - This is an internal refactoring

  • External APIs (LakeWriter, LanceLakeTieringFactory) remain unchanged
  • Lance dataset format and compatibility are preserved

Internal Changes:

  • LanceLakeWriter now uses ShadedArrowBatchWriter + ArrowDataConverter
  • New ArrowDataConverter class for zero-copy shaded/non-shaded conversion
  • New ShadedArrowBatchWriter class to reuse fluss-common's ArrowFieldWriter
  • Removed LanceArrowWriter and its duplicate FieldWriter implementations

Documentation

Documentation Updates: Not required

  • This is an internal code refactoring
  • No new features exposed to users
  • Existing Lance documentation remains valid

Code Quality Improvements:

  • Eliminated code duplication (~400 lines of duplicate FieldWriter code removed)
  • Unified Arrow writing logic across all modules
  • Better maintainability (changes to ArrowFieldWriter only need to be made once)
  • Performance optimization through zero-copy memory sharing
  • Reduced future maintenance burden for new data type support

@XuQianJin-Stars XuQianJin-Stars force-pushed the refactor/lance-arrow-writer branch 2 times, most recently from 8630ee7 to a1ed91b Compare January 11, 2026 11:58
@XuQianJin-Stars XuQianJin-Stars changed the title [lake/lance] refactor LanceArrowWriter [lake/lance] Refactor LanceArrowWriter Jan 11, 2026
@wuchong
Copy link
Member

wuchong commented Jan 11, 2026

Just to double-check, doesn’t this still retain the LanceArrowWriters?

The original goal of the issue was to reuse the ArrowWriter implementations from fluss-common to avoid maintaining two separate copies. Could we consider removing LanceArrowWriters entirely and relying on the shared utilities instead?

Copy link
Member

@wuchong wuchong left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @XuQianJin-Stars , I left some comments.

<groupId>org.apache.fluss</groupId>
<artifactId>fluss-common</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why need to include fluss-common into shaded jar? If this is needed for testing, we can add a test scope.

@luoyuxia could you also help to check this ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, we don't need to include fluss-common

Comment on lines 103 to 137
private static List<org.apache.fluss.shaded.arrow.org.apache.arrow.memory.ArrowBuf>
getFieldBuffers(
org.apache.fluss.shaded.arrow.org.apache.arrow.vector.FieldVector vector) {
try {
Method method = vector.getClass().getMethod("getFieldBuffers");
return (List<org.apache.fluss.shaded.arrow.org.apache.arrow.memory.ArrowBuf>)
method.invoke(vector);
} catch (Exception e) {
throw new RuntimeException("Failed to get field buffers from shaded vector", e);
}
}

private static int getValueCount(
org.apache.fluss.shaded.arrow.org.apache.arrow.vector.FieldVector vector) {
try {
Method method = vector.getClass().getMethod("getValueCount");
return (int) method.invoke(vector);
} catch (Exception e) {
throw new RuntimeException("Failed to get value count from shaded vector", e);
}
}

private static ByteBuffer getByteBuffer(
org.apache.fluss.shaded.arrow.org.apache.arrow.memory.ArrowBuf buf) {
try {
Method method = buf.getClass().getMethod("nioBuffer", long.class, int.class);
return (ByteBuffer) method.invoke(buf, 0L, (int) buf.capacity());
} catch (Exception e) {
try {
Field field = buf.getClass().getDeclaredField("memoryAddress");
field.setAccessible(true);
long address = (long) field.get(buf);
return null;
} catch (Exception ex) {
throw new RuntimeException("Failed to get ByteBuffer from ArrowBuf", ex);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why introduce these reflections? It seems they provide these methods and can be directly invoked.

Comment on lines 46 to 47
FieldVector fieldVector = shadedRoot.getVector(i);
fieldWriters[i] = ArrowUtils.createArrowFieldWriter(fieldVector, rowType.getTypeAt(i));
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we direclty use ArrowWriter instead of ArrowFieldWriter? It seems here missed to call initFieldVector which has been done in ArrowWriter.

shadedRoot.setRowCount(recordsCount);
}

public void reset() {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is never called, should we call it in LanceLakeWriter#complete?

@Override
public void write(LogRecord record) throws IOException {
arrowWriter.write(record);
buffer.add(record.getRow());
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why buffer it first instead of writing to arrow directly? This introduce doubled memory overhead.

List<FragmentMetadata> fragments =
Fragment.create(datasetUri, nonShadedAllocator, nonShadedRoot, writeParams);

allFragments.addAll(fragments);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No need to use a memory shared variable allFragments, this can be a local variable and as a return value of the flush() method? I can't find the reset of allFragments.

@XuQianJin-Stars XuQianJin-Stars force-pushed the refactor/lance-arrow-writer branch from 1baa6a1 to 458473b Compare January 16, 2026 06:32
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[lake/lance] refactor LanceArrowWriter

3 participants