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

ycai pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra-analytics.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 84d84fe  CASSANDRA-19791: Remove other uses of Apache Commons lang for 
hashcode, equality and random string generation (#67)
84d84fe is described below

commit 84d84fe36b0d6e250c3d221c28c40b6925e4c222
Author: jberragan <[email protected]>
AuthorDate: Mon Jul 22 13:38:28 2024 -0700

    CASSANDRA-19791: Remove other uses of Apache Commons lang for hashcode, 
equality and random string generation (#67)
    
    Patch by James Berragan; Reviewed by Francisco Guerrero, Yifan Cai for 
CASSANDRA-19791
---
 CHANGES.txt                                        |  1 +
 .../apache/cassandra/spark/utils/RandomUtils.java  | 50 +++++++++++--
 .../apache/cassandra/spark/utils/TimeProvider.java |  5 +-
 .../cassandra/spark/utils/RandomUtilsTest.java     | 85 ++++++++++++++++++++++
 .../org/apache/cassandra/spark/EndToEndTests.java  | 10 +--
 .../cassandra/spark/KryoSerializationTests.java    | 14 ++--
 .../data/partitioner/JDKSerializationTests.java    |  4 +-
 .../spark/reader/DataTypeSerializationTests.java   | 12 +--
 .../org/apache/cassandra/spark/data/CqlField.java  | 35 ++++-----
 .../org/apache/cassandra/spark/data/CqlTable.java  | 24 ++----
 .../spark/data/complex/CqlCollection.java          | 15 ++--
 .../cassandra/spark/data/complex/CqlFrozen.java    | 15 +---
 .../cassandra/spark/data/complex/CqlUdt.java       | 19 ++---
 .../cassandra/spark/data/types/StringBased.java    |  4 +-
 14 files changed, 191 insertions(+), 102 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index d9231b6..391e5fb 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 1.0.0
+ * Remove other uses of Apache Commons lang for hashcode, equality and random 
string generation (CASSANDRA-19791)
  * Split out BufferingInputStream stats into separate interface 
(CASSANDRA-19778)
  * Bump Sidecar version to 55a9efee (CASSANDRA-19774)
  * Add new module cassandra-analytics-common to store common code with minimal 
dependencies (CASSANDRA-19748)
diff --git 
a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java
 
b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java
similarity index 66%
rename from 
cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java
rename to 
cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java
index 8fd8d02..c59e11e 100644
--- 
a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java
+++ 
b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java
@@ -24,15 +24,15 @@ import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.UUID;
+import java.util.stream.IntStream;
 
 import com.google.common.net.InetAddresses;
 
-import org.apache.cassandra.spark.data.CqlField;
 import org.apache.cassandra.spark.data.partitioner.Partitioner;
 
 public final class RandomUtils
 {
-    private static final int MIN_COLLECTION_SIZE = 16;
+    public static final int MIN_COLLECTION_SIZE = 16;
 
     public static final Random RANDOM = new Random();
 
@@ -63,6 +63,20 @@ public final class RandomUtils
         return RANDOM.nextInt(bound - 1) + 1;
     }
 
+    public static int nextInt(int startInclusive, int endExclusive)
+    {
+        if (endExclusive <= startInclusive)
+        {
+            throw new IllegalArgumentException("Start value must be less than 
the end value.");
+        }
+        if (startInclusive < 0)
+        {
+            throw new IllegalArgumentException("Both range values must be 
non-negative.");
+        }
+
+        return startInclusive + RANDOM.nextInt(endExclusive - startInclusive);
+    }
+
     public static BigInteger randomBigInteger(Partitioner partitioner)
     {
         BigInteger range = 
partitioner.maxToken().subtract(partitioner.minToken());
@@ -81,7 +95,7 @@ public final class RandomUtils
 
     /**
      * Returns a random Type 1 (time-based) UUID.
-     *
+     * <p>
      * Since Java does not natively support creation of Type 1 (time-based) 
UUIDs, and in order to avoid introducing
      * a dependency on {@code org.apache.cassandra.utils.UUIDGen}, we obtain a 
Type 4 (random) UUID and "fix" it.
      *
@@ -90,7 +104,7 @@ public final class RandomUtils
     public static UUID getRandomTimeUUIDForTesting()
     {
         UUID uuid = UUID.randomUUID();
-        return new UUID(uuid.getMostSignificantBits()  ^ 0x0000000000005000L,  
 // Change UUID version from 4 to 1
+        return new UUID(uuid.getMostSignificantBits() ^ 0x0000000000005000L,   
// Change UUID version from 4 to 1
                         uuid.getLeastSignificantBits() | 0x0000010000000000L); 
 // Always set multicast bit to 1
     }
 
@@ -100,8 +114,32 @@ public final class RandomUtils
         return InetAddresses.fromInteger(RANDOM.nextInt());
     }
 
-    public static Object randomValue(CqlField.CqlType type)
+    public static String randomAlphanumeric(int minLengthInclusive, int 
maxLengthExclusive)
     {
-        return type.randomValue(MIN_COLLECTION_SIZE);
+        return randomAlphanumeric(RandomUtils.nextInt(minLengthInclusive, 
maxLengthExclusive));
+    }
+
+    public static String randomAlphanumeric(int length)
+    {
+        StringBuilder sb = new StringBuilder(length);
+        IntStream.range(0, length)
+                 .mapToObj(i -> randomAsciiAlphanumeric())
+                 .forEach(sb::append);
+        return sb.toString();
+    }
+
+    /**
+     * @return random ascii character between 0x30...0x39 for numbers and 
0x41...0x5A for uppercase letters
+     */
+    public static char randomAsciiAlphanumeric()
+    {
+        int c = RANDOM.nextInt(36);
+        if (c < 10)
+        {
+            // return ascii number
+            return (char) (c + 48);
+        }
+        // return ascii uppercase character
+        return (char) ((c - 10) + 65);
     }
 }
diff --git 
a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/TimeProvider.java
 
b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/TimeProvider.java
index 481b8f9..40a2512 100644
--- 
a/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/TimeProvider.java
+++ 
b/cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/TimeProvider.java
@@ -25,7 +25,7 @@ import com.google.common.annotations.VisibleForTesting;
 
 /**
  * Provides time
-  */
+ **/
 public interface TimeProvider
 {
     /**
@@ -57,8 +57,9 @@ public interface TimeProvider
 
     /**
      * Get the time value that is used as a reference. It should never change 
throughout the lifecycle of the provider
-     * @return a fixed epoch time in seconds
      *
+     * @return a fixed epoch time in seconds
+     * <p>
      * Note that the actual constant value returned is implementation dependent
      */
     int referenceEpochInSeconds();
diff --git 
a/cassandra-analytics-common/src/test/java/org/apache/cassandra/spark/utils/RandomUtilsTest.java
 
b/cassandra-analytics-common/src/test/java/org/apache/cassandra/spark/utils/RandomUtilsTest.java
new file mode 100644
index 0000000..4d74d8b
--- /dev/null
+++ 
b/cassandra-analytics-common/src/test/java/org/apache/cassandra/spark/utils/RandomUtilsTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.utils;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class RandomUtilsTest
+{
+    public static final Set<Character> ALPHANUMERIC_CHARS = 
"ABCDEFGHIJKLMNOPQRSTUVWXYZ1234567890".chars()
+                                                                               
                   .mapToObj(e -> (char) e)
+                                                                               
                   .collect(Collectors.toSet());
+
+    @Test
+    public void testNextInt()
+    {
+        for (int i = 0; i < 1000; i++)
+        {
+            assertEquals(4, RandomUtils.nextInt(4, 5));
+        }
+
+        for (int i = 0; i < 1000; i++)
+        {
+            int r = RandomUtils.nextInt(4, 7);
+            assertTrue(r >= 4);
+            assertTrue(r < 7);
+        }
+    }
+
+    @Test
+    public void testNextIntThrows()
+    {
+        assertThrows(IllegalArgumentException.class, () -> 
RandomUtils.nextInt(-1, 5));
+        assertThrows(IllegalArgumentException.class, () -> 
RandomUtils.nextInt(-5, -2));
+        assertThrows(IllegalArgumentException.class, () -> 
RandomUtils.nextInt(5, 5));
+        assertThrows(IllegalArgumentException.class, () -> 
RandomUtils.nextInt(10, 5));
+    }
+
+    @Test
+    public void testRandomAscii()
+    {
+        for (int i = 0; i < 1000000; i++)
+        {
+            
assertTrue(ALPHANUMERIC_CHARS.contains(RandomUtils.randomAsciiAlphanumeric()));
+        }
+    }
+
+    @Test
+    public void testRandomString()
+    {
+        for (int i = 0; i < 1000; i++)
+        {
+            int len = RandomUtils.nextInt(20, 100);
+            String str = RandomUtils.randomAlphanumeric(len);
+            assertEquals(len, str.length());
+            for (int j = 0; j < str.length(); j++)
+            {
+                assertTrue(ALPHANUMERIC_CHARS.contains(str.charAt(j)));
+            }
+        }
+    }
+}
diff --git 
a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/EndToEndTests.java
 
b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/EndToEndTests.java
index acf439e..4e23b50 100644
--- 
a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/EndToEndTests.java
+++ 
b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/EndToEndTests.java
@@ -1852,13 +1852,13 @@ public class EndToEndTests
                   for (long pk = 0; pk < Tester.DEFAULT_NUM_ROWS; pk++)
                   {
                       Map<String, Object> value = ImmutableMap.of(
-                            pk < midPoint ? "a" : "b", 
RandomUtils.randomValue(bridge.text()).toString(),
-                            "c", 
RandomUtils.randomValue(bridge.text()).toString());
+                            pk < midPoint ? "a" : "b", 
bridge.text().randomValue().toString(),
+                            "c", bridge.text().randomValue().toString());
                       values.put(pk, value);
                       writer.write(pk, bridge.toUserTypeValue(type, value),
-                                       RandomUtils.randomValue(bridge.text()),
-                                       
RandomUtils.randomValue(bridge.timestamp()),
-                                       RandomUtils.randomValue(bridge.aInt()));
+                                       bridge.text().randomValue(),
+                                       bridge.timestamp().randomValue(),
+                                       bridge.aInt().randomValue());
                   }
               })
               .withCheck(dataset -> {
diff --git 
a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java
 
b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java
index 6354428..b0ea97b 100644
--- 
a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java
+++ 
b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java
@@ -25,7 +25,6 @@ import java.util.UUID;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import org.apache.commons.lang3.RandomStringUtils;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.MethodSource;
@@ -46,6 +45,7 @@ import 
org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
 import org.apache.cassandra.spark.transports.storage.StorageCredentialPair;
 import org.apache.cassandra.spark.transports.storage.StorageCredentials;
 import 
org.apache.cassandra.spark.transports.storage.extensions.StorageTransportConfiguration;
+import org.apache.cassandra.spark.utils.RandomUtils;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -90,7 +90,7 @@ public class KryoSerializationTests
                 CqlField field = new CqlField(isPartitionKey,
                                               isClusteringKey && 
!isPartitionKey,
                                               false,
-                                              
RandomStringUtils.randomAlphanumeric(5, 20),
+                                              
RandomUtils.randomAlphanumeric(5, 20),
                                               cqlType,
                                               position);
                 Output out = serialize(field);
@@ -115,7 +115,7 @@ public class KryoSerializationTests
                 CqlField field = new CqlField(isPartitionKey,
                                               isClusteringKey && 
!isPartitionKey,
                                               false,
-                                              
RandomStringUtils.randomAlphanumeric(5, 20),
+                                              
RandomUtils.randomAlphanumeric(5, 20),
                                               setType,
                                               position);
                 Output out = serialize(field);
@@ -140,7 +140,7 @@ public class KryoSerializationTests
                 CqlField field = new CqlField(isPartitionKey,
                                               isClusteringKey && 
!isPartitionKey,
                                               false,
-                                              
RandomStringUtils.randomAlphanumeric(5, 20),
+                                              
RandomUtils.randomAlphanumeric(5, 20),
                                               listType,
                                               position);
                 Output out = serialize(field);
@@ -165,7 +165,7 @@ public class KryoSerializationTests
                 CqlField field = new CqlField(isPartitionKey,
                                               isClusteringKey && 
!isPartitionKey,
                                               false,
-                                              
RandomStringUtils.randomAlphanumeric(5, 20),
+                                              
RandomUtils.randomAlphanumeric(5, 20),
                                               mapType,
                                               2);
                 Output out = serialize(field);
@@ -190,7 +190,7 @@ public class KryoSerializationTests
                                             .withField("a", type1)
                                             .withField("b", type2)
                                             .build();
-                CqlField field = new CqlField(false, false, false, 
RandomStringUtils.randomAlphanumeric(5, 20), udt, 2);
+                CqlField field = new CqlField(false, false, false, 
RandomUtils.randomAlphanumeric(5, 20), udt, 2);
                 Output out = serialize(field);
                 CqlField deserialized = deserialize(out, CqlField.class);
                 assertEquals(field, deserialized);
@@ -215,7 +215,7 @@ public class KryoSerializationTests
                                                        
bridge.set(bridge.text()),
                                                        bridge.bigint(),
                                                        bridge.map(type2, 
bridge.timeuuid()));
-                CqlField field = new CqlField(false, false, false, 
RandomStringUtils.randomAlphanumeric(5, 20), tuple, 2);
+                CqlField field = new CqlField(false, false, false, 
RandomUtils.randomAlphanumeric(5, 20), tuple, 2);
                 Output out = serialize(field);
                 CqlField deserialized = deserialize(out, CqlField.class);
                 assertEquals(field, deserialized);
diff --git 
a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/JDKSerializationTests.java
 
b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/JDKSerializationTests.java
index 5cf6ec7..794f37b 100644
--- 
a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/JDKSerializationTests.java
+++ 
b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/JDKSerializationTests.java
@@ -32,7 +32,6 @@ import java.util.stream.Stream;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Range;
-import org.apache.commons.lang3.RandomStringUtils;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.MethodSource;
 
@@ -48,6 +47,7 @@ import org.apache.cassandra.spark.data.PartitionedDataLayer;
 import org.apache.cassandra.spark.data.ReplicationFactor;
 import org.apache.cassandra.spark.data.SSTable;
 import org.apache.cassandra.spark.data.VersionRunner;
+import org.apache.cassandra.spark.utils.RandomUtils;
 import org.apache.cassandra.spark.utils.TimeProvider;
 import org.apache.cassandra.spark.utils.test.TestSchema;
 import org.jetbrains.annotations.NotNull;
@@ -131,7 +131,7 @@ public class JDKSerializationTests extends VersionRunner
     public void testCqlFieldSet(CassandraBridge bridge)
     {
         CqlField.CqlSet setType = bridge.set(bridge.text());
-        CqlField field = new CqlField(true, false, false, 
RandomStringUtils.randomAlphanumeric(5, 20), setType, 10);
+        CqlField field = new CqlField(true, false, false, 
RandomUtils.randomAlphanumeric(5, 20), setType, 10);
         byte[] bytes = bridge.javaSerialize(field);
         CqlField deserialized = bridge.javaDeserialize(bytes, CqlField.class);
         assertEquals(field, deserialized);
diff --git 
a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/DataTypeSerializationTests.java
 
b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/DataTypeSerializationTests.java
index f7eddd4..9fa154a 100644
--- 
a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/DataTypeSerializationTests.java
+++ 
b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/DataTypeSerializationTests.java
@@ -452,7 +452,7 @@ public class DataTypeSerializationTests
             qt().forAll(TestUtils.cql3Type(bridge)).checkAssert(type -> {
                 CqlField.CqlList list = bridge.list(type);
                 List<Object> expected = IntStream.range(0, 128)
-                                                 .mapToObj(index -> 
RandomUtils.randomValue(type))
+                                                 .mapToObj(index -> 
type.randomValue())
                                                  .collect(Collectors.toList());
                 ByteBuffer buffer = list.serialize(expected);
                 List<Object> actual = Arrays.asList(((ArrayData) 
list.deserialize(buffer)).array());
@@ -471,7 +471,7 @@ public class DataTypeSerializationTests
             qt().forAll(TestUtils.cql3Type(bridge)).checkAssert(type -> {
                 CqlField.CqlSet set = bridge.set(type);
                 Set<Object> expected = IntStream.range(0, 128)
-                                                .mapToObj(integer -> 
RandomUtils.randomValue(type))
+                                                .mapToObj(integer -> 
type.randomValue())
                                                 .collect(Collectors.toSet());
                 ByteBuffer buffer = set.serialize(expected);
                 Set<Object> actual = new HashSet<>(Arrays.asList(((ArrayData) 
set.deserialize(buffer)).array()));
@@ -496,9 +496,9 @@ public class DataTypeSerializationTests
                     Object key = null;
                     while (key == null || expected.containsKey(key))
                     {
-                        key = RandomUtils.randomValue(keyType);
+                        key = keyType.randomValue();
                     }
-                    expected.put(key, RandomUtils.randomValue(valueType));
+                    expected.put(key, valueType.randomValue());
                 }
                 ByteBuffer buffer = map.serialize(expected);
                 ArrayBasedMapData mapData = ((ArrayBasedMapData) 
map.deserialize(buffer));
@@ -531,7 +531,7 @@ public class DataTypeSerializationTests
                                             .withField("b", bridge.ascii())
                                             .withField("c", secondType)
                                             .build();
-                Map<String, Object> expected = (Map<String, Object>) 
RandomUtils.randomValue(udt);
+                Map<String, Object> expected = (Map<String, Object>) 
udt.randomValue();
                 assert expected != null;
                 ByteBuffer buffer = udt.serializeUdt(expected);
                 Map<String, Object> actual = udt.deserializeUdt(buffer, false);
@@ -555,7 +555,7 @@ public class DataTypeSerializationTests
                                                        bridge.timestamp(),
                                                        bridge.uuid(),
                                                        bridge.varchar());
-                Object[] expected = (Object[]) RandomUtils.randomValue(tuple);
+                Object[] expected = (Object[]) tuple.randomValue();
                 assert expected != null;
                 ByteBuffer buffer = tuple.serializeTuple(expected);
                 Object[] actual = tuple.deserializeTuple(buffer, false);
diff --git 
a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlField.java 
b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlField.java
index 59b45bc..1488fa4 100644
--- 
a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlField.java
+++ 
b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlField.java
@@ -24,13 +24,12 @@ import java.nio.ByteBuffer;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.function.Function;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
 
 import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.io.Input;
@@ -38,12 +37,13 @@ import com.esotericsoftware.kryo.io.Output;
 import org.apache.cassandra.bridge.BigNumberConfig;
 import org.apache.cassandra.bridge.CassandraBridge;
 import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.utils.RandomUtils;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
 import org.apache.spark.sql.types.DataType;
 import org.jetbrains.annotations.NotNull;
 
-@SuppressWarnings({"WeakerAccess", "unused"})
+@SuppressWarnings({ "WeakerAccess", "unused" })
 public class CqlField implements Serializable, Comparable<CqlField>
 {
     private static final long serialVersionUID = 42L;
@@ -134,6 +134,12 @@ public class CqlField implements Serializable, 
Comparable<CqlField>
         @VisibleForTesting
         Object sparkSqlRowValue(Row row, int position);
 
+        @VisibleForTesting
+        default Object randomValue()
+        {
+            return randomValue(RandomUtils.MIN_COLLECTION_SIZE);
+        }
+
         @VisibleForTesting
         Object randomValue(int minCollectionSize);
 
@@ -354,14 +360,7 @@ public class CqlField implements Serializable, 
Comparable<CqlField>
     @Override
     public int hashCode()
     {
-        return new HashCodeBuilder()
-               .append(name)
-               .append(isPartitionKey)
-               .append(isClusteringColumn)
-               .append(isStaticColumn)
-               .append(type)
-               .append(position)
-               .toHashCode();
+        return Objects.hash(name, isPartitionKey, isClusteringColumn, 
isStaticColumn, type, position);
     }
 
     @Override
@@ -381,14 +380,12 @@ public class CqlField implements Serializable, 
Comparable<CqlField>
         }
 
         CqlField that = (CqlField) other;
-        return new EqualsBuilder()
-               .append(this.name, that.name)
-               .append(this.isPartitionKey, that.isPartitionKey)
-               .append(this.isClusteringColumn, that.isClusteringColumn)
-               .append(this.isStaticColumn, that.isStaticColumn)
-               .append(this.type, that.type)
-               .append(this.position, that.position)
-               .isEquals();
+        return Objects.equals(this.name, that.name)
+               && this.isPartitionKey == that.isPartitionKey
+               && this.isClusteringColumn == that.isClusteringColumn
+               && this.isStaticColumn == that.isStaticColumn
+               && Objects.equals(this.type, that.type)
+               && this.position == that.position;
     }
 
     public boolean equals(Object first, Object second)
diff --git 
a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlTable.java 
b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlTable.java
index 9841b2a..406c417 100644
--- 
a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlTable.java
+++ 
b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlTable.java
@@ -26,13 +26,11 @@ import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-
 import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
@@ -237,13 +235,7 @@ public class CqlTable implements Serializable
     @Override
     public int hashCode()
     {
-        return new HashCodeBuilder()
-               .append(keyspace)
-               .append(table)
-               .append(createStatement)
-               .append(fields)
-               .append(udts)
-               .toHashCode();
+        return Objects.hash(keyspace, table, createStatement, fields, udts);
     }
 
     @Override
@@ -263,13 +255,11 @@ public class CqlTable implements Serializable
         }
 
         CqlTable that = (CqlTable) other;
-        return new EqualsBuilder()
-               .append(this.keyspace, that.keyspace)
-               .append(this.table, that.table)
-               .append(this.createStatement, that.createStatement)
-               .append(this.fields, that.fields)
-               .append(this.udts, that.udts)
-               .isEquals();
+        return Objects.equals(this.keyspace, that.keyspace)
+               && Objects.equals(this.table, that.table)
+               && Objects.equals(this.createStatement, that.createStatement)
+               && Objects.equals(this.fields, that.fields)
+               && Objects.equals(this.udts, that.udts);
     }
 
     public static class Serializer extends 
com.esotericsoftware.kryo.Serializer<CqlTable>
diff --git 
a/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlCollection.java
 
b/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlCollection.java
index 385395e..99d8e43 100644
--- 
a/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlCollection.java
+++ 
b/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlCollection.java
@@ -25,12 +25,10 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-
 import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
 import org.apache.cassandra.bridge.CassandraBridge;
@@ -53,12 +51,11 @@ public abstract class CqlCollection extends CqlType 
implements CqlField.CqlColle
         this(Arrays.asList(types));
     }
 
+    @SuppressWarnings("ConfusingArgumentToVarargsMethod")
     CqlCollection(List<CqlField.CqlType> types)
     {
         this.types = new ArrayList<>(types);
-        this.hashCode = new HashCodeBuilder()
-                        .append(types.toArray(new CqlField.CqlType[0]))
-                        .hashCode();
+        this.hashCode = Objects.hash((CqlField.CqlType[]) types.toArray(new 
CqlField.CqlType[0]));
     }
 
     @Override
@@ -241,9 +238,7 @@ public abstract class CqlCollection extends CqlType 
implements CqlField.CqlColle
         }
 
         CqlCollection that = (CqlCollection) other;
-        return new EqualsBuilder()
-               .append(this.internalType(), that.internalType())
-               .append(this.types, that.types)
-               .isEquals();
+        return this.internalType() == that.internalType()
+               && Objects.equals(this.types, that.types);
     }
 }
diff --git 
a/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlFrozen.java
 
b/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlFrozen.java
index 9c364c7..ca615ac 100644
--- 
a/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlFrozen.java
+++ 
b/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlFrozen.java
@@ -20,11 +20,9 @@
 package org.apache.cassandra.spark.data.complex;
 
 import java.nio.ByteBuffer;
+import java.util.Objects;
 import java.util.Set;
 
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-
 import com.esotericsoftware.kryo.io.Output;
 import org.apache.cassandra.bridge.BigNumberConfig;
 import org.apache.cassandra.bridge.CassandraVersion;
@@ -45,10 +43,7 @@ public class CqlFrozen extends CqlType implements 
CqlField.CqlFrozen
     public CqlFrozen(CqlField.CqlType inner)
     {
         this.inner = inner;
-        this.hashCode = new HashCodeBuilder()
-                .append(internalType().ordinal())
-                .append(inner)
-                .toHashCode();
+        this.hashCode = Objects.hash(internalType().ordinal(), inner);
     }
 
     public static CqlFrozen build(CqlField.CqlType inner)
@@ -233,9 +228,7 @@ public class CqlFrozen extends CqlType implements 
CqlField.CqlFrozen
         }
 
         CqlFrozen that = (CqlFrozen) other;
-        return new EqualsBuilder()
-               .append(this.internalType(), that.internalType())
-               .append(this.inner, that.inner)
-               .isEquals();
+        return this.internalType() == that.internalType()
+               && Objects.equals(this.inner, that.inner);
     }
 }
diff --git 
a/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlUdt.java
 
b/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlUdt.java
index 6000af0..27b1b4b 100644
--- 
a/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlUdt.java
+++ 
b/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/complex/CqlUdt.java
@@ -34,8 +34,6 @@ import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
 
 import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.io.Input;
@@ -77,12 +75,7 @@ public class CqlUdt extends CqlType implements 
CqlField.CqlUdt
         this.name = name;
         this.fields = Collections.unmodifiableList(fields);
         this.fieldMap = 
this.fields.stream().collect(Collectors.toMap(CqlField::name, 
Function.identity()));
-        this.hashCode = new HashCodeBuilder()
-                .append(internalType().ordinal())
-                .append(this.keyspace)
-                .append(this.name)
-                .append(this.fields)
-                .toHashCode();
+        this.hashCode = Objects.hash(internalType().ordinal(), this.keyspace, 
this.name, this.fields);
     }
 
     @Override
@@ -481,12 +474,10 @@ public class CqlUdt extends CqlType implements 
CqlField.CqlUdt
         }
 
         CqlUdt that = (CqlUdt) other;
-        return new EqualsBuilder()
-               .append(this.internalType(), that.internalType())
-               .append(this.keyspace, that.keyspace)
-               .append(this.name, that.name)
-               .append(this.fields, that.fields)
-               .isEquals();
+        return this.internalType() == that.internalType()
+               && Objects.equals(this.keyspace, that.keyspace)
+               && Objects.equals(this.name, that.name)
+               && Objects.equals(this.fields, that.fields);
     }
 
     public static class Serializer extends 
com.esotericsoftware.kryo.Serializer<CqlUdt>
diff --git 
a/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/types/StringBased.java
 
b/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/types/StringBased.java
index 10d5e1e..5bb4d20 100644
--- 
a/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/types/StringBased.java
+++ 
b/cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/data/types/StringBased.java
@@ -21,8 +21,6 @@ package org.apache.cassandra.spark.data.types;
 
 import java.util.Comparator;
 
-import org.apache.commons.lang3.RandomStringUtils;
-
 import org.apache.cassandra.bridge.BigNumberConfig;
 import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
 import org.apache.cassandra.spark.data.NativeType;
@@ -91,7 +89,7 @@ public abstract class StringBased extends NativeType
     @Override
     public Object randomValue(int minCollectionSize)
     {
-        return 
RandomStringUtils.randomAlphanumeric(RandomUtils.randomPositiveInt(32));
+        return 
RandomUtils.randomAlphanumeric(RandomUtils.randomPositiveInt(32));
     }
 
     @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to