You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by em...@apache.org on 2019/07/03 04:16:28 UTC

[arrow] branch master updated: ARROW-5778: [Java] Extract the logic for vector data copying to the super classes

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

emkornfield pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new e28d366  ARROW-5778: [Java] Extract the logic for vector data copying to the super classes
e28d366 is described below

commit e28d3662a008b7a4a2ebbc5b3a6265c13970e742
Author: liyafan82 <fa...@foxmail.com>
AuthorDate: Tue Jul 2 21:15:37 2019 -0700

    ARROW-5778: [Java] Extract the logic for vector data copying to the super classes
    
    Currently, each vector has its own copyFrom method. The implementations for fixed-width vectors are similar, whereas the implementations for the variable-width vectors are similar.
    
    This issue extract such implementations to the base classes, with the following benefits:
    
    1. Less code makes it easier to maintain
    
    2. Move the method to the base class makes the method more convenient to use.
    
    Benchmark evaluations show that there are minor performance improvements, but the overall performance remain almost identical:
    
    Fixed width data type (float8)
    before:
    Float8Benchmarks.copyFromBenchmark  avgt    4  17.809 ± 0.022  us/op
    after:
    Float8Benchmarks.copyFromBenchmark  avgt    4  17.712 ± 0.050  us/op
    
    Variable width data type (varchar)
    before:
    VarCharBenchmarks.copyFromBenchmark  avgt    5  36.455 ± 0.125  us/op
    after:
    VarCharBenchmarks.copyFromBenchmark  avgt    5  33.015 ± 0.065  us/op
    
    Author: liyafan82 <fa...@foxmail.com>
    
    Closes #4764 from liyafan82/fly_5778 and squashes the following commits:
    
    54193917e <liyafan82> Merge branch 'master' into fly_5778
    1612c9e8f <liyafan82>  Extract the logic for vector data copying to the super classes
---
 .../org/apache/arrow/vector/Float8Benchmarks.java  | 24 +++++++++
 ...loat8Benchmarks.java => VarCharBenchmarks.java} | 42 ++++++++--------
 .../apache/arrow/vector/BaseFixedWidthVector.java  | 33 +++++++++++++
 .../arrow/vector/BaseVariableWidthVector.java      | 57 ++++++++++++++++++++++
 .../java/org/apache/arrow/vector/BigIntVector.java | 29 -----------
 .../java/org/apache/arrow/vector/BitVector.java    | 20 ++------
 .../org/apache/arrow/vector/DateDayVector.java     | 29 -----------
 .../org/apache/arrow/vector/DateMilliVector.java   | 29 -----------
 .../org/apache/arrow/vector/DecimalVector.java     | 28 -----------
 .../org/apache/arrow/vector/DurationVector.java    | 30 ------------
 .../apache/arrow/vector/FixedSizeBinaryVector.java | 28 -----------
 .../java/org/apache/arrow/vector/Float4Vector.java | 29 -----------
 .../java/org/apache/arrow/vector/Float8Vector.java | 28 -----------
 .../java/org/apache/arrow/vector/IntVector.java    | 29 -----------
 .../org/apache/arrow/vector/IntervalDayVector.java | 29 -----------
 .../apache/arrow/vector/IntervalYearVector.java    | 29 -----------
 .../org/apache/arrow/vector/SmallIntVector.java    | 29 -----------
 .../org/apache/arrow/vector/TimeMicroVector.java   | 29 -----------
 .../org/apache/arrow/vector/TimeMilliVector.java   | 28 -----------
 .../org/apache/arrow/vector/TimeNanoVector.java    | 28 -----------
 .../org/apache/arrow/vector/TimeSecVector.java     | 28 -----------
 .../org/apache/arrow/vector/TimeStampVector.java   | 28 -----------
 .../org/apache/arrow/vector/TinyIntVector.java     | 28 -----------
 .../java/org/apache/arrow/vector/UInt1Vector.java  | 18 -------
 .../java/org/apache/arrow/vector/UInt2Vector.java  | 16 ------
 .../java/org/apache/arrow/vector/UInt4Vector.java  | 19 --------
 .../java/org/apache/arrow/vector/UInt8Vector.java  | 19 --------
 .../org/apache/arrow/vector/VarBinaryVector.java   | 42 ----------------
 .../org/apache/arrow/vector/VarCharVector.java     | 42 ----------------
 29 files changed, 140 insertions(+), 707 deletions(-)

diff --git a/java/performance/src/test/java/org/apache/arrow/vector/Float8Benchmarks.java b/java/performance/src/test/java/org/apache/arrow/vector/Float8Benchmarks.java
index 9ab6e37..4617f5b 100644
--- a/java/performance/src/test/java/org/apache/arrow/vector/Float8Benchmarks.java
+++ b/java/performance/src/test/java/org/apache/arrow/vector/Float8Benchmarks.java
@@ -50,6 +50,8 @@ public class Float8Benchmarks {
 
   private Float8Vector vector;
 
+  private Float8Vector fromVector;
+
   /**
    * Setup benchmarks.
    */
@@ -58,6 +60,18 @@ public class Float8Benchmarks {
     allocator = new RootAllocator(ALLOCATOR_CAPACITY);
     vector = new Float8Vector("vector", allocator);
     vector.allocateNew(VECTOR_LENGTH);
+
+    fromVector = new Float8Vector("vector", allocator);
+    fromVector.allocateNew(VECTOR_LENGTH);
+
+    for (int i = 0;i < VECTOR_LENGTH; i++) {
+      if (i % 3 == 0) {
+        fromVector.setNull(i);
+      } else {
+        fromVector.set(i, i * i);
+      }
+    }
+    fromVector.setValueCount(VECTOR_LENGTH);
   }
 
   /**
@@ -66,6 +80,7 @@ public class Float8Benchmarks {
   @TearDown
   public void tearDown() {
     vector.close();
+    fromVector.close();
     allocator.close();
   }
 
@@ -88,6 +103,15 @@ public class Float8Benchmarks {
     return sum;
   }
 
+  @Benchmark
+  @BenchmarkMode(Mode.AverageTime)
+  @OutputTimeUnit(TimeUnit.MICROSECONDS)
+  public void copyFromBenchmark() {
+    for (int i = 0; i < VECTOR_LENGTH; i++) {
+      vector.copyFrom(i, i, (Float8Vector) fromVector);
+    }
+  }
+
   @Test
   public void evaluate() throws RunnerException {
     Options opt = new OptionsBuilder()
diff --git a/java/performance/src/test/java/org/apache/arrow/vector/Float8Benchmarks.java b/java/performance/src/test/java/org/apache/arrow/vector/VarCharBenchmarks.java
similarity index 73%
copy from java/performance/src/test/java/org/apache/arrow/vector/Float8Benchmarks.java
copy to java/performance/src/test/java/org/apache/arrow/vector/VarCharBenchmarks.java
index 9ab6e37..39ff9c0 100644
--- a/java/performance/src/test/java/org/apache/arrow/vector/Float8Benchmarks.java
+++ b/java/performance/src/test/java/org/apache/arrow/vector/VarCharBenchmarks.java
@@ -19,7 +19,6 @@ package org.apache.arrow.vector;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.arrow.memory.BoundsChecking;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
 import org.junit.Test;
@@ -37,10 +36,10 @@ import org.openjdk.jmh.runner.options.Options;
 import org.openjdk.jmh.runner.options.OptionsBuilder;
 
 /**
- * Benchmarks for {@link Float8Vector}.
+ * Benchmarks for {@link VarCharVector}.
  */
 @State(Scope.Benchmark)
-public class Float8Benchmarks {
+public class VarCharBenchmarks {
 
   private static final int VECTOR_LENGTH = 1024;
 
@@ -48,7 +47,9 @@ public class Float8Benchmarks {
 
   private BufferAllocator allocator;
 
-  private Float8Vector vector;
+  private VarCharVector vector;
+
+  private VarCharVector fromVector;
 
   /**
    * Setup benchmarks.
@@ -56,8 +57,20 @@ public class Float8Benchmarks {
   @Setup
   public void prepare() {
     allocator = new RootAllocator(ALLOCATOR_CAPACITY);
-    vector = new Float8Vector("vector", allocator);
-    vector.allocateNew(VECTOR_LENGTH);
+    vector = new VarCharVector("vector", allocator);
+    vector.allocateNew(ALLOCATOR_CAPACITY / 4, VECTOR_LENGTH);
+
+    fromVector = new VarCharVector("vector", allocator);
+    fromVector.allocateNew(ALLOCATOR_CAPACITY / 4, VECTOR_LENGTH);
+
+    for (int i = 0;i < VECTOR_LENGTH; i++) {
+      if (i % 3 == 0) {
+        fromVector.setNull(i);
+      } else {
+        fromVector.set(i, String.valueOf(i * 1000).getBytes());
+      }
+    }
+    fromVector.setValueCount(VECTOR_LENGTH);
   }
 
   /**
@@ -66,32 +79,23 @@ public class Float8Benchmarks {
   @TearDown
   public void tearDown() {
     vector.close();
+    fromVector.close();
     allocator.close();
   }
 
-  /**
-   * Test reading/writing on {@link Float8Vector}.
-   * The performance of this benchmark is influenced by the states of two flags:
-   * 1. The flag for boundary checking. For details, please see {@link BoundsChecking}.
-   * 2. The flag for null checking in get methods. For details, please see {@link NullCheckingForGet}.
-   * @return useless. To avoid DCE by JIT.
-   */
   @Benchmark
   @BenchmarkMode(Mode.AverageTime)
   @OutputTimeUnit(TimeUnit.MICROSECONDS)
-  public double readWriteBenchmark() {
-    double sum = 0;
+  public void copyFromBenchmark() {
     for (int i = 0; i < VECTOR_LENGTH; i++) {
-      vector.set(i, i + 10.0);
-      sum += vector.get(i);
+      vector.copyFrom(i, i, fromVector);
     }
-    return sum;
   }
 
   @Test
   public void evaluate() throws RunnerException {
     Options opt = new OptionsBuilder()
-            .include(Float8Benchmarks.class.getSimpleName())
+            .include(VarCharBenchmarks.class.getSimpleName())
             .forks(1)
             .build();
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java
index e4bed23..8feca75 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java
@@ -30,6 +30,7 @@ import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
 
 import io.netty.buffer.ArrowBuf;
+import io.netty.util.internal.PlatformDependent;
 
 /**
  * BaseFixedWidthVector provides an abstract interface for
@@ -804,4 +805,36 @@ public abstract class BaseFixedWidthVector extends BaseValueVector
       reAlloc();
     }
   }
+
+  /**
+   * Copy a cell value from a particular index in source vector to a particular
+   * position in this vector.
+   *
+   * @param fromIndex position to copy from in source vector
+   * @param thisIndex position to copy to in this vector
+   * @param from      source vector
+   */
+  public void copyFrom(int fromIndex, int thisIndex, BaseFixedWidthVector from) {
+    if (from.isNull(fromIndex)) {
+      BitVectorHelper.setValidityBit(this.getValidityBuffer(), thisIndex, 0);
+    } else {
+      BitVectorHelper.setValidityBit(this.getValidityBuffer(), thisIndex, 1);
+      PlatformDependent.copyMemory(from.getDataBuffer().memoryAddress() + fromIndex * typeWidth,
+              this.getDataBuffer().memoryAddress() + thisIndex * typeWidth, typeWidth);
+    }
+  }
+
+  /**
+   * Same as {@link #copyFrom(int, int, BaseFixedWidthVector)} except that
+   * it handles the case when the capacity of the vector needs to be expanded
+   * before copy.
+   *
+   * @param fromIndex position to copy from in source vector
+   * @param thisIndex position to copy to in this vector
+   * @param from      source vector
+   */
+  public void copyFromSafe(int fromIndex, int thisIndex, BaseFixedWidthVector from) {
+    handleSafe(thisIndex);
+    copyFrom(fromIndex, thisIndex, from);
+  }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java
index e7fa289..5262f33 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java
@@ -1277,4 +1277,61 @@ public abstract class BaseVariableWidthVector extends BaseValueVector
 
     return buffer;
   }
+
+  /**
+   * Copy a cell value from a particular index in source vector to a particular
+   * position in this vector.
+   *
+   * @param fromIndex position to copy from in source vector
+   * @param thisIndex position to copy to in this vector
+   * @param from source vector
+   */
+  public void copyFrom(int fromIndex, int thisIndex, BaseVariableWidthVector from) {
+    if (from.isNull(fromIndex)) {
+      fillHoles(thisIndex);
+      BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, 0);
+      final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
+      offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart);
+    } else {
+      final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
+      final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
+      final int length = end - start;
+      fillHoles(thisIndex);
+      BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, 1);
+      final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
+      from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
+      offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
+    }
+    lastSet = thisIndex;
+  }
+
+  /**
+   * Same as {@link #copyFrom(int, int, BaseVariableWidthVector)} except that
+   * it handles the case when the capacity of the vector needs to be expanded
+   * before copy.
+   *
+   * @param fromIndex position to copy from in source vector
+   * @param thisIndex position to copy to in this vector
+   * @param from source vector
+   */
+  public void copyFromSafe(int fromIndex, int thisIndex, BaseVariableWidthVector from) {
+    if (from.isNull(fromIndex)) {
+      handleSafe(thisIndex, 0);
+      fillHoles(thisIndex);
+      BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, 0);
+      final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
+      offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart);
+    } else {
+      final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
+      final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
+      final int length = end - start;
+      handleSafe(thisIndex, length);
+      fillHoles(thisIndex);
+      BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, 1);
+      final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
+      from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
+      offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
+    }
+    lastSet = thisIndex;
+  }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BigIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BigIntVector.java
index 6d235dd..1001cec 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BigIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BigIntVector.java
@@ -145,35 +145,6 @@ public class BigIntVector extends BaseFixedWidthVector implements BaseIntVector
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, BigIntVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
-    valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, BigIntVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, BigIntVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
index f75ccdc..ff4504f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
@@ -296,26 +296,12 @@ public class BitVector extends BaseFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from      source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, BitVector from) {
+  @Override
+  public void copyFrom(int fromIndex, int thisIndex, BaseFixedWidthVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    BitVectorHelper.setValidityBit(valueBuffer, thisIndex, from.getBit(fromIndex));
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, BitVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from      source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, BitVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
+    BitVectorHelper.setValidityBit(valueBuffer, thisIndex, ((BitVector) from).getBit(fromIndex));
   }
 
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/DateDayVector.java b/java/vector/src/main/java/org/apache/arrow/vector/DateDayVector.java
index e634e7e..72af5de 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/DateDayVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/DateDayVector.java
@@ -146,35 +146,6 @@ public class DateDayVector extends BaseFixedWidthVector {
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, DateDayVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
-    valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, DateDayVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, DateDayVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/DateMilliVector.java b/java/vector/src/main/java/org/apache/arrow/vector/DateMilliVector.java
index 7ea427d..be4fcbe 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/DateMilliVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/DateMilliVector.java
@@ -150,35 +150,6 @@ public class DateMilliVector extends BaseFixedWidthVector {
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, DateMilliVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
-    valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, DateMilliVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, DateMilliVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/DecimalVector.java b/java/vector/src/main/java/org/apache/arrow/vector/DecimalVector.java
index 4fc35a3..1db83a1 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/DecimalVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/DecimalVector.java
@@ -163,34 +163,6 @@ public class DecimalVector extends BaseFixedWidthVector {
   }
 
   /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, DecimalVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    from.valueBuffer.getBytes(fromIndex * TYPE_WIDTH, valueBuffer,
-            thisIndex * TYPE_WIDTH, TYPE_WIDTH);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, DecimalVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, DecimalVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-  /**
    * Return scale for the decimal value.
    */
   public int getScale() {
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/DurationVector.java b/java/vector/src/main/java/org/apache/arrow/vector/DurationVector.java
index 76572bc..92a9e70 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/DurationVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/DurationVector.java
@@ -193,42 +193,12 @@ public class DurationVector extends BaseFixedWidthVector {
     return new StringBuilder(getObject(index).toString());
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, DurationVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    from.valueBuffer.getBytes(fromIndex * TYPE_WIDTH, this.valueBuffer,
-              thisIndex * TYPE_WIDTH, TYPE_WIDTH);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, DurationVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, DurationVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
    |                                                                |
    *----------------------------------------------------------------*/
 
-
   /**
    * Set the element at the given index to the given value.
    *
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/FixedSizeBinaryVector.java b/java/vector/src/main/java/org/apache/arrow/vector/FixedSizeBinaryVector.java
index 50179de..61bd57c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/FixedSizeBinaryVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/FixedSizeBinaryVector.java
@@ -158,34 +158,6 @@ public class FixedSizeBinaryVector extends BaseFixedWidthVector {
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from      source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, FixedSizeBinaryVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    from.valueBuffer.getBytes(fromIndex * byteWidth, valueBuffer,
-        thisIndex * byteWidth, byteWidth);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, FixedSizeBinaryVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from      source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, FixedSizeBinaryVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
   public int getByteWidth() {
     return byteWidth;
   }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/Float4Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/Float4Vector.java
index 7976b02..96b5625 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/Float4Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/Float4Vector.java
@@ -147,35 +147,6 @@ public class Float4Vector extends BaseFixedWidthVector {
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, Float4Vector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final float value = from.valueBuffer.getFloat(fromIndex * TYPE_WIDTH);
-    valueBuffer.setFloat(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, Float4Vector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, Float4Vector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/Float8Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/Float8Vector.java
index 109bf5d..24128cd 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/Float8Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/Float8Vector.java
@@ -147,34 +147,6 @@ public class Float8Vector extends BaseFixedWidthVector {
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, Float8Vector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final double value = from.valueBuffer.getDouble(fromIndex * TYPE_WIDTH);
-    valueBuffer.setDouble(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, Float8Vector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, Float8Vector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/IntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/IntVector.java
index 43e5f2c..4ce5e1b 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/IntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/IntVector.java
@@ -147,35 +147,6 @@ public class IntVector extends BaseFixedWidthVector implements BaseIntVector {
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from      source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, IntVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
-    valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, IntVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from      source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, IntVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/IntervalDayVector.java b/java/vector/src/main/java/org/apache/arrow/vector/IntervalDayVector.java
index 1bdbd48..0d7125b 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/IntervalDayVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/IntervalDayVector.java
@@ -223,35 +223,6 @@ public class IntervalDayVector extends BaseFixedWidthVector {
             .append(millis));
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, IntervalDayVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    from.valueBuffer.getBytes(fromIndex * TYPE_WIDTH, this.valueBuffer,
-              thisIndex * TYPE_WIDTH, TYPE_WIDTH);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, IntervalDayVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, IntervalDayVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/IntervalYearVector.java b/java/vector/src/main/java/org/apache/arrow/vector/IntervalYearVector.java
index 65e3b59..2b73d02 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/IntervalYearVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/IntervalYearVector.java
@@ -196,35 +196,6 @@ public class IntervalYearVector extends BaseFixedWidthVector {
         .append(monthString));
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, IntervalYearVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
-    valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, IntervalYearVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, IntervalYearVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/SmallIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/SmallIntVector.java
index 1a06bcd..165b774 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/SmallIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/SmallIntVector.java
@@ -148,35 +148,6 @@ public class SmallIntVector extends BaseFixedWidthVector implements BaseIntVecto
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, SmallIntVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final short value = from.valueBuffer.getShort(fromIndex * TYPE_WIDTH);
-    valueBuffer.setShort(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, SmallIntVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, SmallIntVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/TimeMicroVector.java b/java/vector/src/main/java/org/apache/arrow/vector/TimeMicroVector.java
index 0f58bab..089164c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/TimeMicroVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeMicroVector.java
@@ -147,35 +147,6 @@ public class TimeMicroVector extends BaseFixedWidthVector {
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, TimeMicroVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
-    valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, TimeMicroVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, TimeMicroVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
   /*----------------------------------------------------------------*
    |                                                                |
    |          vector value setter methods                           |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/TimeMilliVector.java b/java/vector/src/main/java/org/apache/arrow/vector/TimeMilliVector.java
index 5552353..9f41c84 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/TimeMilliVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeMilliVector.java
@@ -150,34 +150,6 @@ public class TimeMilliVector extends BaseFixedWidthVector {
     return DateUtility.getLocalDateTimeFromEpochMilli(millis);
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, TimeMilliVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
-    valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, TimeMilliVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, TimeMilliVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/TimeNanoVector.java b/java/vector/src/main/java/org/apache/arrow/vector/TimeNanoVector.java
index 2c05952..053a722 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/TimeNanoVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeNanoVector.java
@@ -147,34 +147,6 @@ public class TimeNanoVector extends BaseFixedWidthVector {
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, TimeNanoVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
-    valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, TimeNanoVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, TimeNanoVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/TimeSecVector.java b/java/vector/src/main/java/org/apache/arrow/vector/TimeSecVector.java
index 02e9dcb..15992af 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/TimeSecVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeSecVector.java
@@ -147,34 +147,6 @@ public class TimeSecVector extends BaseFixedWidthVector {
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, TimeSecVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
-    valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, TimeSecVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, TimeSecVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/TimeStampVector.java b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampVector.java
index dea0510..53bcbc0 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/TimeStampVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampVector.java
@@ -77,34 +77,6 @@ public abstract class TimeStampVector extends BaseFixedWidthVector {
     return valueBuffer.getLong(index * TYPE_WIDTH);
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, TimeStampVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
-    valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFromSafe(int, int, TimeStampVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, TimeStampVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/TinyIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/TinyIntVector.java
index 673740d..97a0ea1 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/TinyIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TinyIntVector.java
@@ -148,34 +148,6 @@ public class TinyIntVector extends BaseFixedWidthVector implements BaseIntVector
     }
   }
 
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, TinyIntVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final byte value = from.valueBuffer.getByte(fromIndex * TYPE_WIDTH);
-    valueBuffer.setByte(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, TinyIntVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, TinyIntVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java
index a57e1ef..5bf843c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java
@@ -146,24 +146,6 @@ public class UInt1Vector extends BaseFixedWidthVector implements BaseIntVector {
     }
   }
 
-  /**
-   * Copies the value at fromIndex to thisIndex (including validity).
-   */
-  public void copyFrom(int fromIndex, int thisIndex, UInt1Vector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final byte value = from.valueBuffer.getByte(fromIndex * TYPE_WIDTH);
-    valueBuffer.setByte(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Identical to {@link #copyFrom(int, int, UInt1Vector)} but reallocates buffer if index is larger
-   * than capacity.
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, UInt1Vector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java
index c29adf6..7e7c544 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java
@@ -127,22 +127,6 @@ public class UInt2Vector extends BaseFixedWidthVector implements BaseIntVector {
     }
   }
 
-  /** Copies a value and validity bit from the given vector to this one. */
-  public void copyFrom(int fromIndex, int thisIndex, UInt2Vector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final char value = from.valueBuffer.getChar(fromIndex * TYPE_WIDTH);
-    valueBuffer.setChar(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, UInt2Vector)} but reallocate buffer if
-   * index is larger than capacity.
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, UInt2Vector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java
index 65d0e32..525d82c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java
@@ -144,25 +144,6 @@ public class UInt4Vector extends BaseFixedWidthVector implements BaseIntVector {
     }
   }
 
-  /**
-   * Copies a value and validity setting to the thisIndex position from the given vector
-   * at fromIndex.
-   */
-  public void copyFrom(int fromIndex, int thisIndex, UInt4Vector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
-    valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, UInt4Vector)} but will allocate additional space
-   * if fromIndex is larger than current capacity.
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, UInt4Vector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java
index e756132..5bd54c0 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java
@@ -149,25 +149,6 @@ public class UInt8Vector extends BaseFixedWidthVector implements BaseIntVector {
     }
   }
 
-  /**
-   * Copy a value and validity setting from fromIndex in <code>from</code> to this
-   * Vector at thisIndex.
-   */
-  public void copyFrom(int fromIndex, int thisIndex, UInt8Vector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
-    valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, UInt8Vector)} but reallocates if thisIndex is
-   * larger then current capacity.
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, UInt8Vector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
 
   /*----------------------------------------------------------------*
    |                                                                |
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/VarBinaryVector.java b/java/vector/src/main/java/org/apache/arrow/vector/VarBinaryVector.java
index ac6df0d..bd76f3c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/VarBinaryVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/VarBinaryVector.java
@@ -164,48 +164,6 @@ public class VarBinaryVector extends BaseVariableWidthVector {
 
 
   /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, VarBinaryVector from) {
-    final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
-    final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
-    final int length = end - start;
-    fillHoles(thisIndex);
-    BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
-    from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
-    offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
-    lastSet = thisIndex;
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, VarBinaryVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, VarBinaryVector from) {
-    final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
-    final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
-    final int length = end - start;
-    handleSafe(thisIndex, length);
-    fillHoles(thisIndex);
-    BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
-    from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
-    offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
-    lastSet = thisIndex;
-  }
-
-  /**
    * Set the variable length element at the specified index to the data
    * buffer supplied in the holder.
    *
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/VarCharVector.java b/java/vector/src/main/java/org/apache/arrow/vector/VarCharVector.java
index 3e035fa..c012ce3 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/VarCharVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/VarCharVector.java
@@ -163,48 +163,6 @@ public class VarCharVector extends BaseVariableWidthVector {
 
 
   /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, VarCharVector from) {
-    final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
-    final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
-    final int length = end - start;
-    fillHoles(thisIndex);
-    BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
-    from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
-    offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
-    lastSet = thisIndex;
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, VarCharVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, VarCharVector from) {
-    final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
-    final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
-    final int length = end - start;
-    handleSafe(thisIndex, length);
-    fillHoles(thisIndex);
-    BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, from.isSet(fromIndex));
-    final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
-    from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
-    offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
-    lastSet = thisIndex;
-  }
-
-  /**
    * Set the variable length element at the specified index to the data
    * buffer supplied in the holder.
    *