This is an automated email from the ASF dual-hosted git repository.

amoghj pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/main by this push:
     new 61241ed478 Core: Update variant class visibility (#12105)
61241ed478 is described below

commit 61241ed4789e719237d276a08fc3cc4113b312c0
Author: Ryan Blue <[email protected]>
AuthorDate: Tue Jan 28 08:46:10 2025 -0800

    Core: Update variant class visibility (#12105)
---
 .../apache/iceberg/variants/PrimitiveWrapper.java  |  15 ++-
 .../apache/iceberg/variants/SerializedArray.java   |  14 +--
 .../iceberg/variants/SerializedMetadata.java       |   5 +-
 .../apache/iceberg/variants/SerializedObject.java  |  18 +--
 .../apache/iceberg/variants/ShreddedObject.java    |  87 ++++++++++++---
 .../org/apache/iceberg/variants/VariantArray.java  |   3 +
 .../apache/iceberg/variants/VariantMetadata.java   |   3 +
 .../org/apache/iceberg/variants/VariantObject.java |   6 +
 .../java/org/apache/iceberg/variants/Variants.java |  26 +++--
 .../iceberg/variants/TestPrimitiveWrapper.java     |   2 +-
 .../iceberg/variants/TestSerializedArray.java      |   3 +-
 .../iceberg/variants/TestSerializedMetadata.java   |   2 +-
 .../iceberg/variants/TestSerializedObject.java     |  45 ++++----
 .../iceberg/variants/TestShreddedObject.java       | 124 ++++++++++-----------
 .../apache/iceberg/variants/VariantTestUtil.java   |   2 +-
 15 files changed, 216 insertions(+), 139 deletions(-)

diff --git 
a/core/src/main/java/org/apache/iceberg/variants/PrimitiveWrapper.java 
b/core/src/main/java/org/apache/iceberg/variants/PrimitiveWrapper.java
index 96d6229cbd..8eebf2f8b9 100644
--- a/core/src/main/java/org/apache/iceberg/variants/PrimitiveWrapper.java
+++ b/core/src/main/java/org/apache/iceberg/variants/PrimitiveWrapper.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.nio.charset.StandardCharsets;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.variants.Variants.PhysicalType;
 import org.apache.iceberg.variants.Variants.Primitives;
 
 class PrimitiveWrapper<T> implements VariantPrimitive<T> {
@@ -47,17 +48,23 @@ class PrimitiveWrapper<T> implements VariantPrimitive<T> {
   private static final byte BINARY_HEADER = 
VariantUtil.primitiveHeader(Primitives.TYPE_BINARY);
   private static final byte STRING_HEADER = 
VariantUtil.primitiveHeader(Primitives.TYPE_STRING);
 
-  private final Variants.PhysicalType type;
+  private final PhysicalType type;
   private final T value;
   private ByteBuffer buffer = null;
 
-  PrimitiveWrapper(Variants.PhysicalType type, T value) {
-    this.type = type;
+  PrimitiveWrapper(PhysicalType type, T value) {
+    if (value instanceof Boolean
+        && (type == PhysicalType.BOOLEAN_TRUE || type == 
PhysicalType.BOOLEAN_FALSE)) {
+      // set the boolean type from the value so that callers can use BOOLEAN_* 
interchangeably
+      this.type = ((Boolean) value) ? PhysicalType.BOOLEAN_TRUE : 
PhysicalType.BOOLEAN_FALSE;
+    } else {
+      this.type = type;
+    }
     this.value = value;
   }
 
   @Override
-  public Variants.PhysicalType type() {
+  public PhysicalType type() {
     return type;
   }
 
diff --git 
a/core/src/main/java/org/apache/iceberg/variants/SerializedArray.java 
b/core/src/main/java/org/apache/iceberg/variants/SerializedArray.java
index 774553cbb4..be6649cb0d 100644
--- a/core/src/main/java/org/apache/iceberg/variants/SerializedArray.java
+++ b/core/src/main/java/org/apache/iceberg/variants/SerializedArray.java
@@ -29,11 +29,11 @@ class SerializedArray extends Variants.SerializedValue 
implements VariantArray {
   private static final int IS_LARGE = 0b10000;
 
   @VisibleForTesting
-  static SerializedArray from(SerializedMetadata metadata, byte[] bytes) {
+  static SerializedArray from(VariantMetadata metadata, byte[] bytes) {
     return from(metadata, 
ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN), bytes[0]);
   }
 
-  static SerializedArray from(SerializedMetadata metadata, ByteBuffer value, 
int header) {
+  static SerializedArray from(VariantMetadata metadata, ByteBuffer value, int 
header) {
     Preconditions.checkArgument(
         value.order() == ByteOrder.LITTLE_ENDIAN, "Unsupported byte order: big 
endian");
     Variants.BasicType basicType = VariantUtil.basicType(header);
@@ -42,14 +42,14 @@ class SerializedArray extends Variants.SerializedValue 
implements VariantArray {
     return new SerializedArray(metadata, value, header);
   }
 
-  private final SerializedMetadata metadata;
+  private final VariantMetadata metadata;
   private final ByteBuffer value;
   private final int offsetSize;
   private final int offsetListOffset;
   private final int dataOffset;
   private final VariantValue[] array;
 
-  private SerializedArray(SerializedMetadata metadata, ByteBuffer value, int 
header) {
+  private SerializedArray(VariantMetadata metadata, ByteBuffer value, int 
header) {
     this.metadata = metadata;
     this.value = value;
     this.offsetSize = 1 + ((header & OFFSET_SIZE_MASK) >> OFFSET_SIZE_SHIFT);
@@ -61,8 +61,8 @@ class SerializedArray extends Variants.SerializedValue 
implements VariantArray {
     this.array = new VariantValue[numElements];
   }
 
-  @VisibleForTesting
-  int numElements() {
+  @Override
+  public int numElements() {
     return array.length;
   }
 
@@ -76,7 +76,7 @@ class SerializedArray extends Variants.SerializedValue 
implements VariantArray {
           VariantUtil.readLittleEndianUnsigned(
               value, offsetListOffset + (offsetSize * (1 + index)), 
offsetSize);
       array[index] =
-          Variants.from(metadata, VariantUtil.slice(value, dataOffset + 
offset, next - offset));
+          Variants.value(metadata, VariantUtil.slice(value, dataOffset + 
offset, next - offset));
     }
     return array[index];
   }
diff --git 
a/core/src/main/java/org/apache/iceberg/variants/SerializedMetadata.java 
b/core/src/main/java/org/apache/iceberg/variants/SerializedMetadata.java
index 30f4903db2..4b078e313e 100644
--- a/core/src/main/java/org/apache/iceberg/variants/SerializedMetadata.java
+++ b/core/src/main/java/org/apache/iceberg/variants/SerializedMetadata.java
@@ -32,6 +32,7 @@ class SerializedMetadata implements VariantMetadata, 
Variants.Serialized {
 
   static final ByteBuffer EMPTY_V1_BUFFER =
       ByteBuffer.wrap(new byte[] {0x01, 0x00}).order(ByteOrder.LITTLE_ENDIAN);
+  static final SerializedMetadata EMPTY_V1_METADATA = from(EMPTY_V1_BUFFER);
 
   static SerializedMetadata from(byte[] bytes) {
     return from(ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN));
@@ -63,8 +64,8 @@ class SerializedMetadata implements VariantMetadata, 
Variants.Serialized {
     this.dataOffset = offsetListOffset + ((1 + dictSize) * offsetSize);
   }
 
-  @VisibleForTesting
-  int dictionarySize() {
+  @Override
+  public int dictionarySize() {
     return dict.length;
   }
 
diff --git 
a/core/src/main/java/org/apache/iceberg/variants/SerializedObject.java 
b/core/src/main/java/org/apache/iceberg/variants/SerializedObject.java
index 6eb1039594..db0d932f8f 100644
--- a/core/src/main/java/org/apache/iceberg/variants/SerializedObject.java
+++ b/core/src/main/java/org/apache/iceberg/variants/SerializedObject.java
@@ -36,11 +36,11 @@ class SerializedObject extends Variants.SerializedValue 
implements VariantObject
   private static final int FIELD_ID_SIZE_SHIFT = 4;
   private static final int IS_LARGE = 0b1000000;
 
-  static SerializedObject from(SerializedMetadata metadata, byte[] bytes) {
+  static SerializedObject from(VariantMetadata metadata, byte[] bytes) {
     return from(metadata, 
ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN), bytes[0]);
   }
 
-  static SerializedObject from(SerializedMetadata metadata, ByteBuffer value, 
int header) {
+  static SerializedObject from(VariantMetadata metadata, ByteBuffer value, int 
header) {
     Preconditions.checkArgument(
         value.order() == ByteOrder.LITTLE_ENDIAN, "Unsupported byte order: big 
endian");
     Variants.BasicType basicType = VariantUtil.basicType(header);
@@ -49,7 +49,7 @@ class SerializedObject extends Variants.SerializedValue 
implements VariantObject
     return new SerializedObject(metadata, value, header);
   }
 
-  private final SerializedMetadata metadata;
+  private final VariantMetadata metadata;
   private final ByteBuffer value;
   private final int fieldIdSize;
   private final int fieldIdListOffset;
@@ -61,7 +61,7 @@ class SerializedObject extends Variants.SerializedValue 
implements VariantObject
   private final int dataOffset;
   private final VariantValue[] values;
 
-  private SerializedObject(SerializedMetadata metadata, ByteBuffer value, int 
header) {
+  private SerializedObject(VariantMetadata metadata, ByteBuffer value, int 
header) {
     this.metadata = metadata;
     this.value = value;
     this.offsetSize = 1 + ((header & OFFSET_SIZE_MASK) >> OFFSET_SIZE_SHIFT);
@@ -112,12 +112,13 @@ class SerializedObject extends Variants.SerializedValue 
implements VariantObject
     }
   }
 
-  @VisibleForTesting
-  int numElements() {
+  @Override
+  public int numFields() {
     return fieldIds.length;
   }
 
-  SerializedMetadata metadata() {
+  @VisibleForTesting
+  VariantMetadata metadata() {
     return metadata;
   }
 
@@ -140,6 +141,7 @@ class SerializedObject extends Variants.SerializedValue 
implements VariantObject
         };
   }
 
+  @Override
   public Iterable<String> fieldNames() {
     return () ->
         new Iterator<>() {
@@ -180,7 +182,7 @@ class SerializedObject extends Variants.SerializedValue 
implements VariantObject
 
     if (null == values[index]) {
       values[index] =
-          Variants.from(
+          Variants.value(
               metadata, VariantUtil.slice(value, dataOffset + offsets[index], 
lengths[index]));
     }
 
diff --git a/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java 
b/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java
index e9e734fce0..fc248bb2a9 100644
--- a/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java
+++ b/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java
@@ -21,9 +21,13 @@ package org.apache.iceberg.variants;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.util.Map;
+import java.util.Set;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
 import org.apache.iceberg.util.Pair;
 import org.apache.iceberg.util.SortedMerge;
 
@@ -35,22 +39,55 @@ import org.apache.iceberg.util.SortedMerge;
  * fields. This also does not allow updating or replacing the metadata for the 
unshredded object,
  * which could require recursively rewriting field IDs.
  */
-class ShreddedObject implements VariantObject {
-  private final SerializedMetadata metadata;
-  private final SerializedObject unshredded;
+public class ShreddedObject implements VariantObject {
+  private final VariantMetadata metadata;
+  private final VariantObject unshredded;
   private final Map<String, VariantValue> shreddedFields = Maps.newHashMap();
+  private final Set<String> removedFields = Sets.newHashSet();
   private SerializationState serializationState = null;
 
-  ShreddedObject(SerializedMetadata metadata) {
+  ShreddedObject(VariantMetadata metadata) {
     this.metadata = metadata;
     this.unshredded = null;
   }
 
-  ShreddedObject(SerializedObject unshredded) {
-    this.metadata = unshredded.metadata();
+  ShreddedObject(VariantMetadata metadata, VariantObject unshredded) {
+    this.metadata = metadata;
     this.unshredded = unshredded;
   }
 
+  @VisibleForTesting
+  VariantMetadata metadata() {
+    return metadata;
+  }
+
+  private Set<String> nameSet() {
+    Set<String> names = Sets.newHashSet(shreddedFields.keySet());
+
+    if (unshredded != null) {
+      Iterables.addAll(names, unshredded.fieldNames());
+    }
+
+    names.removeAll(removedFields);
+
+    return names;
+  }
+
+  @Override
+  public Iterable<String> fieldNames() {
+    return nameSet();
+  }
+
+  @Override
+  public int numFields() {
+    return nameSet().size();
+  }
+
+  public void remove(String field) {
+    shreddedFields.remove(field);
+    removedFields.add(field);
+  }
+
   public void put(String field, VariantValue value) {
     Preconditions.checkArgument(
         metadata.id(field) >= 0, "Cannot find field name in metadata: %s", 
field);
@@ -63,6 +100,10 @@ class ShreddedObject implements VariantObject {
 
   @Override
   public VariantValue get(String field) {
+    if (removedFields.contains(field)) {
+      return null;
+    }
+
     // the shredded value takes precedence if there is a conflict
     VariantValue value = shreddedFields.get(field);
     if (value != null) {
@@ -79,7 +120,8 @@ class ShreddedObject implements VariantObject {
   @Override
   public int sizeInBytes() {
     if (null == serializationState) {
-      this.serializationState = new SerializationState(metadata, unshredded, 
shreddedFields);
+      this.serializationState =
+          new SerializationState(metadata, unshredded, shreddedFields, 
removedFields);
     }
 
     return serializationState.size();
@@ -91,7 +133,8 @@ class ShreddedObject implements VariantObject {
         buffer.order() == ByteOrder.LITTLE_ENDIAN, "Invalid byte order: big 
endian");
 
     if (null == serializationState) {
-      this.serializationState = new SerializationState(metadata, unshredded, 
shreddedFields);
+      this.serializationState =
+          new SerializationState(metadata, unshredded, shreddedFields, 
removedFields);
     }
 
     return serializationState.writeTo(buffer, offset);
@@ -99,7 +142,7 @@ class ShreddedObject implements VariantObject {
 
   /** Common state for {@link #size()} and {@link #writeTo(ByteBuffer, int)} */
   private static class SerializationState {
-    private final SerializedMetadata metadata;
+    private final VariantMetadata metadata;
     private final Map<String, ByteBuffer> unshreddedFields;
     private final Map<String, VariantValue> shreddedFields;
     private final int dataSize;
@@ -109,28 +152,38 @@ class ShreddedObject implements VariantObject {
     private final int offsetSize;
 
     private SerializationState(
-        SerializedMetadata metadata,
-        SerializedObject unshredded,
-        Map<String, VariantValue> shreddedFields) {
+        VariantMetadata metadata,
+        VariantObject unshredded,
+        Map<String, VariantValue> shreddedFields,
+        Set<String> removedFields) {
       this.metadata = metadata;
       // field ID size is the size needed to store the largest field ID in the 
data
       this.fieldIdSize = VariantUtil.sizeOf(metadata.dictionarySize());
-      this.shreddedFields = shreddedFields;
+      this.shreddedFields = Maps.newHashMap(shreddedFields);
 
       int totalDataSize = 0;
       // get the unshredded field names and values as byte buffers
       ImmutableMap.Builder<String, ByteBuffer> unshreddedBuilder = 
ImmutableMap.builder();
-      if (unshredded != null) {
-        for (Pair<String, Integer> field : unshredded.fields()) {
+      if (unshredded instanceof SerializedObject) {
+        // for serialized objects, use existing buffers instead of 
materializing values
+        SerializedObject serialized = (SerializedObject) unshredded;
+        for (Pair<String, Integer> field : serialized.fields()) {
           // if the value is replaced by an unshredded field, don't include it
           String name = field.first();
-          boolean replaced = shreddedFields.containsKey(name);
+          boolean replaced = shreddedFields.containsKey(name) || 
removedFields.contains(name);
           if (!replaced) {
-            ByteBuffer value = unshredded.sliceValue(field.second());
+            ByteBuffer value = serialized.sliceValue(field.second());
             unshreddedBuilder.put(name, value);
             totalDataSize += value.remaining();
           }
         }
+      } else if (unshredded != null) {
+        for (String name : unshredded.fieldNames()) {
+          boolean replaced = shreddedFields.containsKey(name) || 
removedFields.contains(name);
+          if (!replaced) {
+            shreddedFields.put(name, unshredded.get(name));
+          }
+        }
       }
 
       this.unshreddedFields = unshreddedBuilder.build();
diff --git a/core/src/main/java/org/apache/iceberg/variants/VariantArray.java 
b/core/src/main/java/org/apache/iceberg/variants/VariantArray.java
index 55dbc071f1..dd1aa5cf4f 100644
--- a/core/src/main/java/org/apache/iceberg/variants/VariantArray.java
+++ b/core/src/main/java/org/apache/iceberg/variants/VariantArray.java
@@ -23,6 +23,9 @@ public interface VariantArray extends VariantValue {
   /** Returns the {@link VariantValue} at {@code index} in this array. */
   VariantValue get(int index);
 
+  /** Returns the number of fields stored in this array. */
+  int numElements();
+
   @Override
   default Variants.PhysicalType type() {
     return Variants.PhysicalType.ARRAY;
diff --git 
a/core/src/main/java/org/apache/iceberg/variants/VariantMetadata.java 
b/core/src/main/java/org/apache/iceberg/variants/VariantMetadata.java
index 91129e2328..e9478b249e 100644
--- a/core/src/main/java/org/apache/iceberg/variants/VariantMetadata.java
+++ b/core/src/main/java/org/apache/iceberg/variants/VariantMetadata.java
@@ -31,4 +31,7 @@ public interface VariantMetadata extends Variants.Serialized {
    * @throws NoSuchElementException if the dictionary does not contain the ID
    */
   String get(int id);
+
+  /** Returns the size of the metadata dictionary. */
+  int dictionarySize();
 }
diff --git a/core/src/main/java/org/apache/iceberg/variants/VariantObject.java 
b/core/src/main/java/org/apache/iceberg/variants/VariantObject.java
index 7bb82f94a4..33696dae41 100644
--- a/core/src/main/java/org/apache/iceberg/variants/VariantObject.java
+++ b/core/src/main/java/org/apache/iceberg/variants/VariantObject.java
@@ -23,6 +23,12 @@ public interface VariantObject extends VariantValue {
   /** Returns the {@link VariantValue} for the field named {@code name} in 
this object. */
   VariantValue get(String name);
 
+  /** Returns the names of fields stored in this object. */
+  Iterable<String> fieldNames();
+
+  /** Returns the number of fields stored in this object. */
+  int numFields();
+
   @Override
   default Variants.PhysicalType type() {
     return Variants.PhysicalType.OBJECT;
diff --git a/core/src/main/java/org/apache/iceberg/variants/Variants.java 
b/core/src/main/java/org/apache/iceberg/variants/Variants.java
index e8ea3d93ab..e10682fe54 100644
--- a/core/src/main/java/org/apache/iceberg/variants/Variants.java
+++ b/core/src/main/java/org/apache/iceberg/variants/Variants.java
@@ -172,11 +172,11 @@ public class Variants {
     ARRAY
   }
 
-  public static VariantValue from(ByteBuffer metadata, ByteBuffer value) {
-    return from(SerializedMetadata.from(metadata), value);
+  public static VariantMetadata metadata(ByteBuffer metadata) {
+    return SerializedMetadata.from(metadata);
   }
 
-  static VariantValue from(SerializedMetadata metadata, ByteBuffer value) {
+  public static VariantValue value(VariantMetadata metadata, ByteBuffer value) 
{
     int header = VariantUtil.readByte(value, 0);
     BasicType basicType = VariantUtil.basicType(header);
     switch (basicType) {
@@ -193,16 +193,24 @@ public class Variants {
     throw new UnsupportedOperationException("Unsupported basic type: " + 
basicType);
   }
 
-  static VariantPrimitive<Void> ofNull() {
+  public static ShreddedObject object(VariantMetadata metadata, VariantObject 
object) {
+    return new ShreddedObject(metadata, object);
+  }
+
+  public static ShreddedObject object(VariantMetadata metadata) {
+    return new ShreddedObject(metadata);
+  }
+
+  public static <T> VariantPrimitive<T> of(PhysicalType type, T value) {
+    return new PrimitiveWrapper<>(type, value);
+  }
+
+  public static VariantPrimitive<Void> ofNull() {
     return new PrimitiveWrapper<>(PhysicalType.NULL, null);
   }
 
   static VariantPrimitive<Boolean> of(boolean value) {
-    if (value) {
-      return new PrimitiveWrapper<>(PhysicalType.BOOLEAN_TRUE, true);
-    } else {
-      return new PrimitiveWrapper<>(PhysicalType.BOOLEAN_FALSE, false);
-    }
+    return new PrimitiveWrapper<>(PhysicalType.BOOLEAN_TRUE, value);
   }
 
   static VariantPrimitive<Byte> of(byte value) {
diff --git 
a/core/src/test/java/org/apache/iceberg/variants/TestPrimitiveWrapper.java 
b/core/src/test/java/org/apache/iceberg/variants/TestPrimitiveWrapper.java
index fd113f9cec..b8c3718ccc 100644
--- a/core/src/test/java/org/apache/iceberg/variants/TestPrimitiveWrapper.java
+++ b/core/src/test/java/org/apache/iceberg/variants/TestPrimitiveWrapper.java
@@ -74,7 +74,7 @@ public class TestPrimitiveWrapper {
     readBuffer.limit(300 + size);
 
     // read and validate the serialized bytes
-    VariantValue actual = Variants.from(SerializedMetadata.EMPTY_V1_BUFFER, 
readBuffer);
+    VariantValue actual = Variants.value(SerializedMetadata.EMPTY_V1_METADATA, 
readBuffer);
     assertThat(actual.type()).isEqualTo(primitive.type());
     assertThat(actual).isInstanceOf(VariantPrimitive.class);
     assertThat(actual.asPrimitive().get()).isEqualTo(primitive.get());
diff --git 
a/core/src/test/java/org/apache/iceberg/variants/TestSerializedArray.java 
b/core/src/test/java/org/apache/iceberg/variants/TestSerializedArray.java
index 1e052572b8..01586ecb45 100644
--- a/core/src/test/java/org/apache/iceberg/variants/TestSerializedArray.java
+++ b/core/src/test/java/org/apache/iceberg/variants/TestSerializedArray.java
@@ -28,8 +28,7 @@ import org.apache.iceberg.variants.Variants.PhysicalType;
 import org.junit.jupiter.api.Test;
 
 public class TestSerializedArray {
-  private static final SerializedMetadata EMPTY_METADATA =
-      SerializedMetadata.from(SerializedMetadata.EMPTY_V1_BUFFER);
+  private static final VariantMetadata EMPTY_METADATA = 
SerializedMetadata.EMPTY_V1_METADATA;
   private static final SerializedPrimitive NULL = SerializedPrimitive.from(new 
byte[] {0x00});
   private static final SerializedPrimitive TRUE = SerializedPrimitive.from(new 
byte[] {0b100});
   private static final SerializedPrimitive FALSE = 
SerializedPrimitive.from(new byte[] {0b1000});
diff --git 
a/core/src/test/java/org/apache/iceberg/variants/TestSerializedMetadata.java 
b/core/src/test/java/org/apache/iceberg/variants/TestSerializedMetadata.java
index 27a4dda3dc..f0b3fdab9b 100644
--- a/core/src/test/java/org/apache/iceberg/variants/TestSerializedMetadata.java
+++ b/core/src/test/java/org/apache/iceberg/variants/TestSerializedMetadata.java
@@ -35,7 +35,7 @@ public class TestSerializedMetadata {
 
   @Test
   public void testEmptyVariantMetadata() {
-    SerializedMetadata metadata = 
SerializedMetadata.from(SerializedMetadata.EMPTY_V1_BUFFER);
+    SerializedMetadata metadata = SerializedMetadata.EMPTY_V1_METADATA;
 
     assertThat(metadata.isSorted()).isFalse();
     assertThat(metadata.dictionarySize()).isEqualTo(0);
diff --git 
a/core/src/test/java/org/apache/iceberg/variants/TestSerializedObject.java 
b/core/src/test/java/org/apache/iceberg/variants/TestSerializedObject.java
index 9fe71ed0e9..b8fe75ff91 100644
--- a/core/src/test/java/org/apache/iceberg/variants/TestSerializedObject.java
+++ b/core/src/test/java/org/apache/iceberg/variants/TestSerializedObject.java
@@ -35,8 +35,7 @@ import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.ValueSource;
 
 public class TestSerializedObject {
-  private static final SerializedMetadata EMPTY_METADATA =
-      SerializedMetadata.from(SerializedMetadata.EMPTY_V1_BUFFER);
+  private static final VariantMetadata EMPTY_METADATA = 
SerializedMetadata.EMPTY_V1_METADATA;
   private static final SerializedPrimitive I1 = SerializedPrimitive.from(new 
byte[] {0b1100, 1});
   private static final SerializedPrimitive I2 = SerializedPrimitive.from(new 
byte[] {0b1100, 2});
   private static final SerializedPrimitive I3 = SerializedPrimitive.from(new 
byte[] {0b1100, 3});
@@ -70,7 +69,7 @@ public class TestSerializedObject {
     SerializedObject object = SerializedObject.from(EMPTY_METADATA, new byte[] 
{0b10, 0x00});
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(0);
+    assertThat(object.numFields()).isEqualTo(0);
   }
 
   @Test
@@ -79,7 +78,7 @@ public class TestSerializedObject {
         SerializedObject.from(EMPTY_METADATA, new byte[] {0b1000010, 0x00, 
0x00, 0x00, 0x00});
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(0);
+    assertThat(object.numFields()).isEqualTo(0);
   }
 
   @Test
@@ -88,11 +87,11 @@ public class TestSerializedObject {
     ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), true /* 
sort names */);
     ByteBuffer value = VariantTestUtil.createObject(meta, data);
 
-    SerializedMetadata metadata = SerializedMetadata.from(meta);
+    VariantMetadata metadata = Variants.metadata(meta);
     SerializedObject object = SerializedObject.from(metadata, value, 
value.get(0));
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(3);
+    assertThat(object.numFields()).isEqualTo(3);
 
     assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8);
     assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1);
@@ -109,11 +108,11 @@ public class TestSerializedObject {
     ByteBuffer meta =
         VariantTestUtil.createMetadata(Sets.newHashSet("a", "b", "c"), true /* 
sort names */);
 
-    SerializedMetadata metadata = SerializedMetadata.from(meta);
+    VariantMetadata metadata = Variants.metadata(meta);
     SerializedObject object = SerializedObject.from(metadata, UNSORTED_VALUES);
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(3);
+    assertThat(object.numFields()).isEqualTo(3);
 
     assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8);
     assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1);
@@ -131,11 +130,11 @@ public class TestSerializedObject {
     ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), false /* 
sort names */);
     ByteBuffer value = VariantTestUtil.createObject(meta, data);
 
-    SerializedMetadata metadata = SerializedMetadata.from(meta);
+    VariantMetadata metadata = Variants.metadata(meta);
     SerializedObject object = SerializedObject.from(metadata, value, 
value.get(0));
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(3);
+    assertThat(object.numFields()).isEqualTo(3);
 
     assertThat(object.get("d")).isEqualTo(null);
 
@@ -152,7 +151,7 @@ public class TestSerializedObject {
     ByteBuffer meta =
         VariantTestUtil.createMetadata(
             ImmutableList.of("a", "b", "c", "d", "e", "f"), true /* sort names 
*/);
-    SerializedMetadata metadata = SerializedMetadata.from(meta);
+    VariantMetadata metadata = Variants.metadata(meta);
 
     Map<String, VariantValue> inner = ImmutableMap.of("b", I2, "f", I3);
     ByteBuffer innerBuffer = VariantTestUtil.createObject(meta, inner);
@@ -164,7 +163,7 @@ public class TestSerializedObject {
     SerializedObject object = SerializedObject.from(metadata, value, 
value.get(0));
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(5);
+    assertThat(object.numFields()).isEqualTo(5);
 
     assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8);
     assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1);
@@ -177,7 +176,7 @@ public class TestSerializedObject {
 
     assertThat(object.get("e").type()).isEqualTo(PhysicalType.OBJECT);
     SerializedObject actualInner = (SerializedObject) 
object.get("e").asObject();
-    assertThat(actualInner.numElements()).isEqualTo(2);
+    assertThat(actualInner.numFields()).isEqualTo(2);
     assertThat(actualInner.get("b").type()).isEqualTo(PhysicalType.INT8);
     assertThat(actualInner.get("b").asPrimitive().get()).isEqualTo((byte) 2);
     assertThat(actualInner.get("f").type()).isEqualTo(PhysicalType.INT8);
@@ -195,11 +194,11 @@ public class TestSerializedObject {
     ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), true /* 
sort names */);
     ByteBuffer value = VariantTestUtil.createObject(meta, data);
 
-    SerializedMetadata metadata = SerializedMetadata.from(meta);
+    VariantMetadata metadata = Variants.metadata(meta);
     SerializedObject object = SerializedObject.from(metadata, value, 
value.get(0));
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(4);
+    assertThat(object.numFields()).isEqualTo(4);
 
     assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8);
     assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1);
@@ -223,11 +222,11 @@ public class TestSerializedObject {
     ByteBuffer meta = VariantTestUtil.createMetadata(data.keySet(), true /* 
sort names */);
     ByteBuffer value = VariantTestUtil.createObject(meta, data);
 
-    SerializedMetadata metadata = SerializedMetadata.from(meta);
+    VariantMetadata metadata = Variants.metadata(meta);
     SerializedObject object = SerializedObject.from(metadata, value, 
value.get(0));
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(4);
+    assertThat(object.numFields()).isEqualTo(4);
 
     assertThat(object.get("a").type()).isEqualTo(PhysicalType.INT8);
     assertThat(object.get("a").asPrimitive().get()).isEqualTo((byte) 1);
@@ -253,11 +252,11 @@ public class TestSerializedObject {
     ByteBuffer meta = VariantTestUtil.createMetadata(fields.keySet(), 
sortFieldNames);
     ByteBuffer value = VariantTestUtil.createObject(meta, (Map) fields);
 
-    SerializedMetadata metadata = SerializedMetadata.from(meta);
+    VariantMetadata metadata = Variants.metadata(meta);
     SerializedObject object = SerializedObject.from(metadata, value, 
value.get(0));
 
     assertThat(object.type()).isEqualTo(Variants.PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(10_000);
+    assertThat(object.numFields()).isEqualTo(10_000);
 
     for (Map.Entry<String, VariantPrimitive<String>> entry : 
fields.entrySet()) {
       VariantValue fieldValue = object.get(entry.getKey());
@@ -281,11 +280,11 @@ public class TestSerializedObject {
     ByteBuffer meta = VariantTestUtil.createMetadata(keySet, sortFieldNames);
     ByteBuffer value = VariantTestUtil.createObject(meta, data);
 
-    SerializedMetadata metadata = SerializedMetadata.from(meta);
+    VariantMetadata metadata = Variants.metadata(meta);
     SerializedObject object = SerializedObject.from(metadata, value, 
value.get(0));
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(3);
+    assertThat(object.numFields()).isEqualTo(3);
 
     assertThat(object.get("aa").type()).isEqualTo(PhysicalType.INT8);
     assertThat(object.get("aa").asPrimitive().get()).isEqualTo((byte) 1);
@@ -310,11 +309,11 @@ public class TestSerializedObject {
     ByteBuffer meta = VariantTestUtil.createMetadata(keySet, sortFieldNames);
     ByteBuffer value = VariantTestUtil.createObject(meta, data);
 
-    SerializedMetadata metadata = SerializedMetadata.from(meta);
+    VariantMetadata metadata = Variants.metadata(meta);
     SerializedObject object = SerializedObject.from(metadata, value, 
value.get(0));
 
     assertThat(object.type()).isEqualTo(PhysicalType.OBJECT);
-    assertThat(object.numElements()).isEqualTo(3);
+    assertThat(object.numFields()).isEqualTo(3);
 
     assertThat(object.get("aa").type()).isEqualTo(PhysicalType.INT8);
     assertThat(object.get("aa").asPrimitive().get()).isEqualTo((byte) 1);
diff --git 
a/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java 
b/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java
index 0a2c2b26b0..d2db9d40c5 100644
--- a/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java
+++ b/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java
@@ -31,7 +31,6 @@ import 
org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.apache.iceberg.relocated.com.google.common.collect.Maps;
 import org.apache.iceberg.relocated.com.google.common.collect.Sets;
 import org.apache.iceberg.util.DateTimeUtil;
-import org.apache.iceberg.util.Pair;
 import org.apache.iceberg.util.RandomUtil;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
@@ -51,7 +50,7 @@ public class TestShreddedObject {
 
   @Test
   public void testShreddedFields() {
-    ShreddedObject object = createShreddedObject(FIELDS).second();
+    ShreddedObject object = createShreddedObject(FIELDS);
 
     assertThat(object.get("a")).isInstanceOf(VariantPrimitive.class);
     assertThat(object.get("a").asPrimitive().get()).isEqualTo(34);
@@ -63,16 +62,15 @@ public class TestShreddedObject {
 
   @Test
   public void testShreddedSerializationMinimalBuffer() {
-    Pair<SerializedMetadata, ShreddedObject> pair = 
createShreddedObject(FIELDS);
-    SerializedMetadata metadata = pair.first();
-    ShreddedObject object = pair.second();
+    ShreddedObject object = createShreddedObject(FIELDS);
+    VariantMetadata metadata = object.metadata();
 
     VariantValue value = roundTripMinimalBuffer(object, metadata);
 
     assertThat(value).isInstanceOf(SerializedObject.class);
     SerializedObject actual = (SerializedObject) value;
 
-    assertThat(actual.numElements()).isEqualTo(3);
+    assertThat(actual.numFields()).isEqualTo(3);
     assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class);
     assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34);
     assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class);
@@ -83,16 +81,15 @@ public class TestShreddedObject {
 
   @Test
   public void testShreddedSerializationLargeBuffer() {
-    Pair<SerializedMetadata, ShreddedObject> pair = 
createShreddedObject(FIELDS);
-    SerializedMetadata metadata = pair.first();
-    ShreddedObject object = pair.second();
+    ShreddedObject object = createShreddedObject(FIELDS);
+    VariantMetadata metadata = object.metadata();
 
     VariantValue value = roundTripLargeBuffer(object, metadata);
 
     assertThat(value).isInstanceOf(SerializedObject.class);
     SerializedObject actual = (SerializedObject) value;
 
-    assertThat(actual.numElements()).isEqualTo(3);
+    assertThat(actual.numFields()).isEqualTo(3);
     assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class);
     assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34);
     assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class);
@@ -103,16 +100,15 @@ public class TestShreddedObject {
 
   @Test
   public void testUnshreddedObjectSerializationMinimalBuffer() {
-    Pair<SerializedMetadata, ShreddedObject> pair = 
createUnshreddedObject(FIELDS);
-    SerializedMetadata metadata = pair.first();
-    ShreddedObject object = pair.second();
+    ShreddedObject object = createUnshreddedObject(FIELDS);
+    VariantMetadata metadata = object.metadata();
 
     VariantValue value = roundTripMinimalBuffer(object, metadata);
 
     assertThat(value).isInstanceOf(SerializedObject.class);
     SerializedObject actual = (SerializedObject) value;
 
-    assertThat(actual.numElements()).isEqualTo(3);
+    assertThat(actual.numFields()).isEqualTo(3);
     assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class);
     assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34);
     assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class);
@@ -123,16 +119,15 @@ public class TestShreddedObject {
 
   @Test
   public void testUnshreddedObjectSerializationLargeBuffer() {
-    Pair<SerializedMetadata, ShreddedObject> pair = 
createUnshreddedObject(FIELDS);
-    SerializedMetadata metadata = pair.first();
-    ShreddedObject object = pair.second();
+    ShreddedObject object = createUnshreddedObject(FIELDS);
+    VariantMetadata metadata = object.metadata();
 
     VariantValue value = roundTripLargeBuffer(object, metadata);
 
     assertThat(value).isInstanceOf(SerializedObject.class);
     SerializedObject actual = (SerializedObject) value;
 
-    assertThat(actual.numElements()).isEqualTo(3);
+    assertThat(actual.numFields()).isEqualTo(3);
     assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class);
     assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34);
     assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class);
@@ -143,15 +138,17 @@ public class TestShreddedObject {
 
   @Test
   public void testPartiallyShreddedObjectReplacement() {
-    ShreddedObject partial = createUnshreddedObject(FIELDS).second();
+    ShreddedObject partial = createUnshreddedObject(FIELDS);
 
     // replace field c with a new value
     partial.put("c", Variants.ofIsoDate("2024-10-12"));
+    assertThat(partial.numFields()).isEqualTo(3);
+
+    partial.remove("b");
+    assertThat(partial.numFields()).isEqualTo(2);
 
     assertThat(partial.get("a")).isInstanceOf(VariantPrimitive.class);
     assertThat(partial.get("a").asPrimitive().get()).isEqualTo(34);
-    assertThat(partial.get("b")).isInstanceOf(VariantPrimitive.class);
-    assertThat(partial.get("b").asPrimitive().get()).isEqualTo("iceberg");
     assertThat(partial.get("c")).isInstanceOf(VariantPrimitive.class);
     assertThat(partial.get("c").type()).isEqualTo(Variants.PhysicalType.DATE);
     assertThat(partial.get("c").asPrimitive().get())
@@ -160,7 +157,7 @@ public class TestShreddedObject {
 
   @Test
   public void testPartiallyShreddedObjectGetMissingField() {
-    ShreddedObject partial = createUnshreddedObject(FIELDS).second();
+    ShreddedObject partial = createUnshreddedObject(FIELDS);
 
     // missing fields are returned as null
     assertThat(partial.get("d")).isNull();
@@ -168,7 +165,7 @@ public class TestShreddedObject {
 
   @Test
   public void testPartiallyShreddedObjectPutMissingFieldFailure() {
-    ShreddedObject partial = createUnshreddedObject(FIELDS).second();
+    ShreddedObject partial = createUnshreddedObject(FIELDS);
 
     // d is not defined in the variant metadata and will fail
     assertThatThrownBy(() -> partial.put("d", 
Variants.ofIsoDate("2024-10-12")))
@@ -178,12 +175,12 @@ public class TestShreddedObject {
 
   @Test
   public void testPartiallyShreddedObjectSerializationMinimalBuffer() {
-    Pair<SerializedMetadata, ShreddedObject> pair = 
createUnshreddedObject(FIELDS);
-    SerializedMetadata metadata = pair.first();
-    ShreddedObject partial = pair.second();
+    ShreddedObject partial = createUnshreddedObject(FIELDS);
+    VariantMetadata metadata = partial.metadata();
 
     // replace field c with a new value
     partial.put("c", Variants.ofIsoDate("2024-10-12"));
+    partial.remove("b");
 
     VariantValue value = roundTripMinimalBuffer(partial, metadata);
 
@@ -192,8 +189,6 @@ public class TestShreddedObject {
 
     assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class);
     assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34);
-    assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class);
-    assertThat(actual.get("b").asPrimitive().get()).isEqualTo("iceberg");
     assertThat(actual.get("c")).isInstanceOf(VariantPrimitive.class);
     assertThat(actual.get("c").type()).isEqualTo(Variants.PhysicalType.DATE);
     assertThat(actual.get("c").asPrimitive().get())
@@ -202,12 +197,12 @@ public class TestShreddedObject {
 
   @Test
   public void testPartiallyShreddedObjectSerializationLargeBuffer() {
-    Pair<SerializedMetadata, ShreddedObject> pair = 
createUnshreddedObject(FIELDS);
-    SerializedMetadata metadata = pair.first();
-    ShreddedObject partial = pair.second();
+    ShreddedObject partial = createUnshreddedObject(FIELDS);
+    VariantMetadata metadata = partial.metadata();
 
     // replace field c with a new value
     partial.put("c", Variants.ofIsoDate("2024-10-12"));
+    partial.remove("b");
 
     VariantValue value = roundTripLargeBuffer(partial, metadata);
 
@@ -216,8 +211,6 @@ public class TestShreddedObject {
 
     assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class);
     assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34);
-    assertThat(actual.get("b")).isInstanceOf(VariantPrimitive.class);
-    assertThat(actual.get("b").asPrimitive().get()).isEqualTo("iceberg");
     assertThat(actual.get("c")).isInstanceOf(VariantPrimitive.class);
     assertThat(actual.get("c").type()).isEqualTo(Variants.PhysicalType.DATE);
     assertThat(actual.get("c").asPrimitive().get())
@@ -234,12 +227,12 @@ public class TestShreddedObject {
     data.putAll(FIELDS);
     data.put("big", bigString);
 
-    Pair<SerializedMetadata, ShreddedObject> pair = createShreddedObject(data);
-    VariantValue value = roundTripLargeBuffer(pair.second(), pair.first());
+    ShreddedObject shredded = createShreddedObject(data);
+    VariantValue value = roundTripLargeBuffer(shredded, shredded.metadata());
 
     assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT);
     SerializedObject object = (SerializedObject) value;
-    assertThat(object.numElements()).isEqualTo(4);
+    assertThat(object.numFields()).isEqualTo(4);
 
     assertThat(object.get("a").type()).isEqualTo(Variants.PhysicalType.INT32);
     assertThat(object.get("a").asPrimitive().get()).isEqualTo(34);
@@ -261,12 +254,12 @@ public class TestShreddedObject {
     data.putAll(FIELDS);
     data.put("really-big", reallyBigString);
 
-    Pair<SerializedMetadata, ShreddedObject> pair = createShreddedObject(data);
-    VariantValue value = roundTripLargeBuffer(pair.second(), pair.first());
+    ShreddedObject shredded = createShreddedObject(data);
+    VariantValue value = roundTripLargeBuffer(shredded, shredded.metadata());
 
     assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT);
     SerializedObject object = (SerializedObject) value;
-    assertThat(object.numElements()).isEqualTo(4);
+    assertThat(object.numFields()).isEqualTo(4);
 
     assertThat(object.get("a").type()).isEqualTo(Variants.PhysicalType.INT32);
     assertThat(object.get("a").asPrimitive().get()).isEqualTo(34);
@@ -288,12 +281,12 @@ public class TestShreddedObject {
     data.putAll(FIELDS);
     data.put("really-big", reallyBigString);
 
-    Pair<SerializedMetadata, ShreddedObject> pair = createShreddedObject(data);
-    VariantValue value = roundTripLargeBuffer(pair.second(), pair.first());
+    ShreddedObject shredded = createShreddedObject(data);
+    VariantValue value = roundTripLargeBuffer(shredded, shredded.metadata());
 
     assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT);
     SerializedObject object = (SerializedObject) value;
-    assertThat(object.numElements()).isEqualTo(4);
+    assertThat(object.numFields()).isEqualTo(4);
 
     assertThat(object.get("a").type()).isEqualTo(Variants.PhysicalType.INT32);
     assertThat(object.get("a").asPrimitive().get()).isEqualTo(34);
@@ -316,15 +309,15 @@ public class TestShreddedObject {
           Variants.of(RandomUtil.generateString(10, random)));
     }
 
-    SerializedMetadata metadata =
-        
SerializedMetadata.from(VariantTestUtil.createMetadata(fields.keySet(), 
sortFieldNames));
+    VariantMetadata metadata =
+        Variants.metadata(VariantTestUtil.createMetadata(fields.keySet(), 
sortFieldNames));
 
     ShreddedObject shredded = createShreddedObject(metadata, (Map) fields);
     VariantValue value = roundTripLargeBuffer(shredded, metadata);
 
     assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT);
     SerializedObject object = (SerializedObject) value;
-    assertThat(object.numElements()).isEqualTo(10_000);
+    assertThat(object.numFields()).isEqualTo(10_000);
 
     for (Map.Entry<String, VariantPrimitive<String>> entry : 
fields.entrySet()) {
       VariantValue fieldValue = object.get(entry.getKey());
@@ -346,15 +339,15 @@ public class TestShreddedObject {
 
     // create metadata from the large key set and the actual keys
     keySet.addAll(data.keySet());
-    SerializedMetadata metadata =
-        SerializedMetadata.from(VariantTestUtil.createMetadata(keySet, 
sortFieldNames));
+    VariantMetadata metadata =
+        Variants.metadata(VariantTestUtil.createMetadata(keySet, 
sortFieldNames));
 
     ShreddedObject shredded = createShreddedObject(metadata, data);
     VariantValue value = roundTripLargeBuffer(shredded, metadata);
 
     assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT);
     SerializedObject object = (SerializedObject) value;
-    assertThat(object.numElements()).isEqualTo(3);
+    assertThat(object.numFields()).isEqualTo(3);
 
     assertThat(object.get("aa").type()).isEqualTo(Variants.PhysicalType.INT32);
     assertThat(object.get("aa").asPrimitive().get()).isEqualTo(34);
@@ -377,15 +370,15 @@ public class TestShreddedObject {
 
     // create metadata from the large key set and the actual keys
     keySet.addAll(data.keySet());
-    SerializedMetadata metadata =
-        SerializedMetadata.from(VariantTestUtil.createMetadata(keySet, 
sortFieldNames));
+    VariantMetadata metadata =
+        Variants.metadata(VariantTestUtil.createMetadata(keySet, 
sortFieldNames));
 
     ShreddedObject shredded = createShreddedObject(metadata, data);
     VariantValue value = roundTripLargeBuffer(shredded, metadata);
 
     assertThat(value.type()).isEqualTo(Variants.PhysicalType.OBJECT);
     SerializedObject object = (SerializedObject) value;
-    assertThat(object.numElements()).isEqualTo(3);
+    assertThat(object.numFields()).isEqualTo(3);
 
     assertThat(object.get("aa").type()).isEqualTo(Variants.PhysicalType.INT32);
     assertThat(object.get("aa").asPrimitive().get()).isEqualTo(34);
@@ -396,16 +389,16 @@ public class TestShreddedObject {
   }
 
   private static VariantValue roundTripMinimalBuffer(
-      ShreddedObject object, SerializedMetadata metadata) {
+      ShreddedObject object, VariantMetadata metadata) {
     ByteBuffer serialized =
         
ByteBuffer.allocate(object.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN);
     object.writeTo(serialized, 0);
 
-    return Variants.from(metadata, serialized);
+    return Variants.value(metadata, serialized);
   }
 
   private static VariantValue roundTripLargeBuffer(
-      ShreddedObject object, SerializedMetadata metadata) {
+      ShreddedObject object, VariantMetadata metadata) {
     ByteBuffer serialized =
         ByteBuffer.allocate(1000 + 
object.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN);
     object.writeTo(serialized, 300);
@@ -414,11 +407,12 @@ public class TestShreddedObject {
     slice.position(300);
     slice.limit(300 + object.sizeInBytes());
 
-    return Variants.from(metadata, slice);
+    return Variants.value(metadata, slice);
   }
 
+  /** Creates a ShreddedObject with fields in its shredded map, using the 
given metadata */
   private static ShreddedObject createShreddedObject(
-      SerializedMetadata metadata, Map<String, VariantValue> fields) {
+      VariantMetadata metadata, Map<String, VariantValue> fields) {
     ShreddedObject object = new ShreddedObject(metadata);
     for (Map.Entry<String, VariantValue> field : fields.entrySet()) {
       object.put(field.getKey(), field.getValue());
@@ -427,22 +421,24 @@ public class TestShreddedObject {
     return object;
   }
 
-  private static Pair<SerializedMetadata, ShreddedObject> createShreddedObject(
-      Map<String, VariantValue> fields) {
+  /** Creates a ShreddedObject with fields in its shredded map */
+  private static ShreddedObject createShreddedObject(Map<String, VariantValue> 
fields) {
     ByteBuffer metadataBuffer = 
VariantTestUtil.createMetadata(fields.keySet(), false);
-    SerializedMetadata metadata = SerializedMetadata.from(metadataBuffer);
-    return Pair.of(metadata, createShreddedObject(metadata, fields));
+    VariantMetadata metadata = Variants.metadata(metadataBuffer);
+    return createShreddedObject(metadata, fields);
   }
 
-  private static Pair<SerializedMetadata, ShreddedObject> 
createUnshreddedObject(
-      Map<String, VariantValue> fields) {
+  /** Creates a ShreddedObject with fields in its unshredded base object */
+  private static ShreddedObject createUnshreddedObject(Map<String, 
VariantValue> fields) {
     SerializedObject serialized = createSerializedObject(fields);
-    return Pair.of(serialized.metadata(), new ShreddedObject(serialized));
+    return new ShreddedObject(serialized.metadata(), serialized);
   }
 
   private static SerializedObject createSerializedObject(Map<String, 
VariantValue> fields) {
     ByteBuffer metadataBuffer = 
VariantTestUtil.createMetadata(fields.keySet(), false);
     return (SerializedObject)
-        Variants.from(metadataBuffer, 
VariantTestUtil.createObject(metadataBuffer, fields));
+        Variants.value(
+            Variants.metadata(metadataBuffer),
+            VariantTestUtil.createObject(metadataBuffer, fields));
   }
 }
diff --git 
a/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java 
b/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java
index b6caec6375..576e06a9d1 100644
--- a/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java
+++ b/core/src/test/java/org/apache/iceberg/variants/VariantTestUtil.java
@@ -110,7 +110,7 @@ public class VariantTestUtil {
 
   static ByteBuffer createObject(ByteBuffer metadataBuffer, Map<String, 
VariantValue> data) {
     // create the metadata to look up field names
-    SerializedMetadata metadata = SerializedMetadata.from(metadataBuffer);
+    VariantMetadata metadata = Variants.metadata(metadataBuffer);
 
     int numElements = data.size();
     boolean isLarge = numElements > 0xFF;

Reply via email to