-
Notifications
You must be signed in to change notification settings - Fork 886
JAVA-3143: Extend driver vector support to arbitrary subtypes and fix handling of variable length types (OSS C* 5.0) #1952
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
JAVA-3143: Extend driver vector support to arbitrary subtypes and fix handling of variable length types (OSS C* 5.0) #1952
Conversation
This branch is on top of #1931 |
@@ -196,7 +233,8 @@ public int hashCode() { | |||
|
|||
@Override | |||
public String toString() { | |||
return Iterables.toString(this.list); | |||
TypeCodec<T> subcodec = CodecRegistry.DEFAULT.codecFor(list.get(0)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Previously we promised that CqlVector.from
should mirror CqlVector.toString
. CqlVector.from
has access to a subtype codec and expects some values to be single-quoted, e.g. texts. Therefore, without this change to toString
, a round-trip test like CqlVector.from(vector1.toString(), codec);
will fail. However, fetching the default codec for the subtype looks like too much effort for something simple like toString
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We could store the subtype codec as a field of the vector but then it will only be available if the vector was created with from()
so it's a bit weird.
Using the default codec could lead to unexpected behavior if the user has custom codecs so I don't think it's a good solution either.
The C# CqlVector
class has methods to easily convert between native arrays and CqlVector
objects so that the user can use existing code that is compatible with arrays like debugging, printing, etc. There's no way to convert a C# CqlVector
object to string
directly but there's also no way to create a CqlVector
object from a string representation. I think this was a fine idea when CqlVector
was only for float
values but it becomes very complex to support when we extend the subtype to any cql supported type including collections, udts, etc.
Personally I'd try to find a way to keep supporting this for subtypes that were already supported before this change but just throw an exception if the subtype would require access to the TypeCodec
to make it work. Alternatively if we don't really mind breaking this string conversion support for existing apps that use the current CqlVector
class then that would also work but I don't think that's a good idea.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In the past we only supported the from
and toString
of number elements. I updated so that toString
only works for number elements and will throw otherwise.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
On no, if toString
throws an exception, it will cause too much inconvenience, e.g. the data provider annotation will even break. I changed it to
return this.list.stream()
.map(ele -> ele.toString())
.collect(Collectors.joining(", ", "[", "]"));
The only problem is that strings and date times are not properly quoted. Everything else works fine.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍 looks fine to me, @absurdfarce can I get your feedback here before I close this thread?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I mean, I... guess this is fine. But it seems to me like the current implementation pretty much does... exactly what Iterables.toString() does, right? Maybe I'm missing something there... but I don't think so. This passes without issue:
$ git diff
diff --git a/core/src/main/java/com/datastax/oss/driver/api/core/data/CqlVector.java b/core/src/main/java/com/datastax/oss/driver/api/core/data/CqlVector.java
index 4e2fc77cf..69da04a9b 100644
--- a/core/src/main/java/com/datastax/oss/driver/api/core/data/CqlVector.java
+++ b/core/src/main/java/com/datastax/oss/driver/api/core/data/CqlVector.java
@@ -241,6 +241,8 @@ public class CqlVector<T> implements Iterable<T>, Serializable {
.collect(Collectors.joining(", ", "[", "]"));
}
+ public String oldToString() { return Iterables.toString(this.list); }
+
/**
* Serialization proxy for CqlVector. Allows serialization regardless of implementation of list
* field.
diff --git a/core/src/test/java/com/datastax/oss/driver/api/core/data/CqlVectorTest.java b/core/src/test/java/com/datastax/oss/driver/api/core/data/CqlVectorTest.java
index 3e0872cb9..208440d30 100644
--- a/core/src/test/java/com/datastax/oss/driver/api/core/data/CqlVectorTest.java
+++ b/core/src/test/java/com/datastax/oss/driver/api/core/data/CqlVectorTest.java
@@ -29,6 +29,7 @@ import com.tngtech.java.junit.dataprovider.UseDataProvider;
import java.io.ByteArrayInputStream;
import java.io.ObjectInputStream;
import java.io.ObjectStreamException;
+import java.time.LocalDate;
import java.time.LocalTime;
import java.util.AbstractList;
import java.util.ArrayList;
@@ -256,4 +257,15 @@ public class CqlVectorTest {
assertThat(e).isInstanceOf(ObjectStreamException.class);
}
}
+
+ @Test
+ public void compare_iterables_to_custom() {
+
+ CqlVector<String> stringVector = CqlVector.newInstance("abc", "def", "ghi");
+ assertThat(stringVector.toString()).isEqualTo(stringVector.oldToString());
+ CqlVector<LocalTime> timeVector = CqlVector.newInstance(LocalTime.now(), LocalTime.now(), LocalTime.now());
+ assertThat(timeVector.toString()).isEqualTo(timeVector.oldToString());
+ CqlVector<LocalDate> dateVector = CqlVector.newInstance(LocalDate.now(), LocalDate.now(), LocalDate.now());
+ assertThat(dateVector.toString()).isEqualTo(dateVector.oldToString());
+ }
}
I guess I'd argue that unless our custom implementation gives us something we don't get from the well-proven third-party lib I'd just as soon continue using Iterables.toString()
@@ -49,6 +50,7 @@ public class DefaultSelect implements SelectFrom, Select { | |||
private final ImmutableList<Relation> relations; | |||
private final ImmutableList<Selector> groupByClauses; | |||
private final ImmutableMap<CqlIdentifier, ClusteringOrder> orderings; | |||
private final Ann ann; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shall we expose a getter like for other fields?
core/src/main/java/com/datastax/oss/driver/internal/core/type/codec/VectorCodec.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good! I left some comments but they're almost all NITs
core/src/main/java/com/datastax/oss/driver/api/core/data/CqlVector.java
Outdated
Show resolved
Hide resolved
core/src/main/java/com/datastax/oss/driver/api/core/data/CqlVector.java
Outdated
Show resolved
Hide resolved
@@ -196,7 +233,8 @@ public int hashCode() { | |||
|
|||
@Override | |||
public String toString() { | |||
return Iterables.toString(this.list); | |||
TypeCodec<T> subcodec = CodecRegistry.DEFAULT.codecFor(list.get(0)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We could store the subtype codec as a field of the vector but then it will only be available if the vector was created with from()
so it's a bit weird.
Using the default codec could lead to unexpected behavior if the user has custom codecs so I don't think it's a good solution either.
The C# CqlVector
class has methods to easily convert between native arrays and CqlVector
objects so that the user can use existing code that is compatible with arrays like debugging, printing, etc. There's no way to convert a C# CqlVector
object to string
directly but there's also no way to create a CqlVector
object from a string representation. I think this was a fine idea when CqlVector
was only for float
values but it becomes very complex to support when we extend the subtype to any cql supported type including collections, udts, etc.
Personally I'd try to find a way to keep supporting this for subtypes that were already supported before this change but just throw an exception if the subtype would require access to the TypeCodec
to make it work. Alternatively if we don't really mind breaking this string conversion support for existing apps that use the current CqlVector
class then that would also work but I don't think that's a good idea.
DefaultVectorType.VECTOR_CLASS_NAME.length() + 1, className.length() - 1)); | ||
DataType subType = classNameParser.parse(params.get(0), AttachmentPoint.NONE); | ||
int dimensions = Integer.parseInt(params.get(1)); | ||
String paramsString = |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why did you decide to stop using paramSplitter
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
paramSplitter
does not work for cases like vector<vector<float,2>,2>
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does it make sense to improve paramSplitter to work for these cases instead? Maybe Bret's feedback here is needed as well @absurdfarce
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I agree with @SiyaoIsHiding that the original impl doesn't handle nested vector types. For example, with the current code the following fails on "test_type_string_nested" but passes everywhere else:
package com.datastax.oss.driver.api.core.type;
import com.datastax.oss.driver.internal.core.type.DefaultVectorType;
import org.junit.Test;
import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
public class DataTypesTest {
private DataType float3 = DataTypes.vectorOf(DataTypes.FLOAT,3);
@Test
public void test_type_string_basic() {
String typeString = "org.apache.cassandra.db.marshal.VectorType(" +
"org.apache.cassandra.db.marshal.FloatType,3)";
DataType dataType = DataTypes.custom(typeString);
assertThat(dataType).isInstanceOf(DefaultVectorType.class);
DefaultVectorType result = (DefaultVectorType)dataType;
assertThat(result.getElementType()).isEqualTo(DataTypes.FLOAT);
assertThat(result.getDimensions()).isEqualTo(3);
}
@Test
public void test_type_string_nested() {
String typeString = "org.apache.cassandra.db.marshal.VectorType(" +
"org.apache.cassandra.db.marshal.VectorType(" +
"org.apache.cassandra.db.marshal.FloatType,3), 3)";
DataType dataType = DataTypes.custom(typeString);
assertThat(dataType).isInstanceOf(DefaultVectorType.class);
DefaultVectorType result = (DefaultVectorType)dataType;
assertThat(result.getElementType()).isEqualTo(float3);
assertThat(result.getDimensions()).isEqualTo(3);
}
@Test
public void test_vector_data_type_basic() {
assertThat(float3).isInstanceOf(DefaultVectorType.class);
DefaultVectorType result = (DefaultVectorType)float3;
assertThat(result.getElementType()).isEqualTo(DataTypes.FLOAT);
assertThat(result.getDimensions()).isEqualTo(3);
}
@Test
public void test_vector_data_type_nested() {
DataType float3 = DataTypes.vectorOf(DataTypes.FLOAT,3);
DataType float3Nested = DataTypes.vectorOf(float3, 3);
assertThat(float3Nested).isInstanceOf(DefaultVectorType.class);
DefaultVectorType result = (DefaultVectorType)float3Nested;
assertThat(result.getElementType()).isEqualTo(float3);
assertThat(result.getDimensions()).isEqualTo(3);
}
}
The changes in this PR make that test pass in it's entirety.
I will argue that the implementation in this PR does split the logic for handling these argument lists into two places. Note that DataTypeClassNameParser already has logic for handling type parameters for type strings. This PR then re-implements something similar in DataTypes.custom(). This has the effect of actually bouncing back and forth between the two classes when dealing with nested types: DataTypes.custom() calls DTCNP.parse() which calls DataTypes.custom() if it doesn't find a customized handler for a type and so on.
This back-and-forth behaviour seems to be the key to the whole thing. To me it's indicating that DataTypeClassNameParser hasn't been properly updated to handle vector types yet. A quick review of the source code confirms that that's the case. I'll note that I can also make the DataTypesTest class above pass via the following change:
$ git diff [29/29]
diff --git a/core/src/main/java/com/datastax/oss/driver/api/core/type/DataTypes.java b/core/src/main/java/com/datastax/oss/driver/api/core/type/DataTypes.java
index 4447e2830..492fc121c 100644
--- a/core/src/main/java/com/datastax/oss/driver/api/core/type/DataTypes.java
+++ b/core/src/main/java/com/datastax/oss/driver/api/core/type/DataTypes.java
@@ -65,29 +65,8 @@ public class DataTypes {
if (className.equals("org.apache.cassandra.db.marshal.DurationType")) return DURATION;
/* Vector support is currently implemented as a custom type but is also parameterized */ - if (className.startsWith(DefaultVectorType.VECTOR_CLASS_NAME)) {
- String paramsString =
- className.substring(
- DefaultVectorType.VECTOR_CLASS_NAME.length() + 1, className.length() - 1);
- int lastCommaIndex = paramsString.lastIndexOf(',');
- if (lastCommaIndex == -1) {
- throw new IllegalArgumentException(
- String.format(
- "Invalid vector type %s, expected format is %s<subtype, dimensions>",
- className, DefaultVectorType.VECTOR_CLASS_NAME));
- }
- String subTypeString = paramsString.substring(0, lastCommaIndex).trim();
- String dimensionsString = paramsString.substring(lastCommaIndex + 1).trim();
-
- DataType subType = classNameParser.parse(subTypeString, AttachmentPoint.NONE);
- int dimensions = Integer.parseInt(dimensionsString);
- if (dimensions <= 0) {
- throw new IllegalArgumentException(
- String.format(
- "Request to create vector of size %d, size must be positive", dimensions));
- }
- return new DefaultVectorType(subType, dimensions);
- }
+ if (className.startsWith(DefaultVectorType.VECTOR_CLASS_NAME))
+ return classNameParser.parse(className, AttachmentPoint.NONE);
return new DefaultCustomType(className);
}
diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/metadata/schema/parsing/DataTypeClassNameParser.java b/core/src/main/java/com/datastax/oss/driver/internal/core/metadata/schema/parsing/Da
taTypeClassNameParser.java
index fd6f1a4bd..4cf77d467 100644
--- a/core/src/main/java/com/datastax/oss/driver/internal/core/metadata/schema/parsing/DataTypeClassNameParser.java
+++ b/core/src/main/java/com/datastax/oss/driver/internal/core/metadata/schema/parsing/DataTypeClassNameParser.java
@@ -26,17 +26,21 @@ import com.datastax.oss.driver.api.core.type.UserDefinedType;
import com.datastax.oss.driver.api.core.type.codec.TypeCodecs;
import com.datastax.oss.driver.internal.core.context.InternalDriverContext;
import com.datastax.oss.driver.internal.core.type.DefaultTupleType;
+import com.datastax.oss.driver.internal.core.type.DefaultVectorType;
import com.datastax.oss.driver.internal.core.type.UserDefinedTypeBuilder;
import com.datastax.oss.driver.internal.core.type.codec.ParseUtils;
import com.datastax.oss.driver.shaded.guava.common.annotations.VisibleForTesting;
import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableList;
import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableMap;
import com.datastax.oss.protocol.internal.util.Bytes;
+
import java.util.ArrayList;
import java.util.Collections;
+import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
+
import net.jcip.annotations.ThreadSafe;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -164,6 +168,13 @@ public class DataTypeClassNameParser implements DataTypeParser {
return new DefaultTupleType(componentTypesBuilder.build(), attachmentPoint);
}
+ if (next.startsWith("org.apache.cassandra.db.marshal.VectorType")) {
+ Iterator<String> rawTypes = parser.getTypeParameters().iterator();
+ DataType subtype = parse(rawTypes.next(), userTypes, attachmentPoint, logPrefix);
+ int dimensions = Integer.parseInt(rawTypes.next());
+ return DataTypes.vectorOf(subtype, dimensions);
+ }
+
DataType type = NATIVE_TYPES_BY_CLASS_NAME.get(next);
return type == null ? DataTypes.custom(toParse) : type;
}
I'd argue this fix (or something very much like it) is much preferred because (a) it keeps the arg parsing logic consolidated in DataTypeClassNameParser (which seems like the ideal place to put it) and (b) it improves DTCNP so that it can be used for additional vector-related functionality in the future.
core/src/main/java/com/datastax/oss/driver/internal/core/type/codec/VectorCodec.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Very much agree with @joao-r-reis; this is really good work @SiyaoIsHiding! There are a few changes I think need to happen here but I don't think any of them are huge.
core/src/main/java/com/datastax/oss/driver/api/core/type/codec/TypeCodec.java
Show resolved
Hide resolved
core/src/main/java/com/datastax/oss/driver/internal/core/type/codec/BlobCodec.java
Outdated
Show resolved
Hide resolved
core/src/test/java/com/datastax/oss/driver/api/core/data/CqlVectorTest.java
Outdated
Show resolved
Hide resolved
core/src/test/java/com/datastax/oss/driver/internal/core/type/codec/VectorCodecTest.java
Outdated
Show resolved
Hide resolved
core/src/main/java/com/datastax/oss/driver/internal/core/type/codec/VectorCodec.java
Outdated
Show resolved
Hide resolved
core/src/main/java/com/datastax/oss/driver/internal/core/type/codec/VectorCodec.java
Outdated
Show resolved
Hide resolved
core/src/main/java/com/datastax/oss/driver/internal/core/type/util/VIntCoding.java
Show resolved
Hide resolved
2c8d9a9
to
8817dd9
Compare
List<SubtypeT> rv = new ArrayList<SubtypeT>(cqlType.getDimensions()); | ||
for (int i = 0; i < cqlType.getDimensions(); ++i) { | ||
int size = VIntCoding.getUnsignedVInt32(input, input.position()); | ||
input.position(input.position() + VIntCoding.computeUnsignedVIntSize(size)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can't we put the if (isVarSized)
here instead? Otherwise there's still a bit of duplication in my mind. Same for encode
Pseudocode:
for (int i = 0; i < cqlType.getDimensions(); ++i) {
if (isVarSized) {
// decode elementSize and move input position forward
}
if (elementSize > 0) { // this will always be true if is not varSized so it works for both cases
// decode element and add it to collection
} else {
// add null to collection (maybe call codec with null? Idk what's correct in java driver, looks like in the current version of this PR the codec is being called with "null"
}
}
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agree with @joao-r-reis, we should be able to simplify this a bit. I was able to get VectorCodecTest passing with the following:
@Nullable
@Override
public CqlVector<SubtypeT> decode(
@Nullable ByteBuffer bytes, @NonNull ProtocolVersion protocolVersion) {
if (bytes == null || bytes.remaining() == 0) {
return null;
}
// Upfront check for fixed-size types only
subtypeCodec.serializedSize().ifPresent((fixed_size) -> {
if (bytes.remaining() != cqlType.getDimensions() * fixed_size) {
throw new IllegalArgumentException(
String.format(
"Expected elements of uniform size, observed %d elements with total bytes %d",
cqlType.getDimensions(), bytes.remaining()));
}
});
;
ByteBuffer slice = bytes.slice();
List<SubtypeT> rv = new ArrayList<SubtypeT>(cqlType.getDimensions());
for (int i = 0; i < cqlType.getDimensions(); ++i) {
int size = subtypeCodec.serializedSize().orElseGet(() -> {
int vint_size = VIntCoding.getUnsignedVInt32(slice, slice.position());
// Side effects! Skip over the bytes that vint encode the size of this element
slice.position(slice.position() + VIntCoding.computeUnsignedVIntSize(vint_size));
return vint_size;
});
int originalPosition = slice.position();
slice.limit(originalPosition + size);
rv.add(this.subtypeCodec.decode(slice, protocolVersion));
// Move to the start of the next element
slice.position(originalPosition + size);
// Reset the limit to the end of the buffer
slice.limit(slice.capacity());
}
// if too many elements, throw
if (slice.hasRemaining()) {
throw new IllegalArgumentException(
String.format(
"Too many elements; must provide elements for %d dimensions",
cqlType.getDimensions()));
}
return CqlVector.newInstance(rv);
}
I might be missing an error case or two there but I think that covers everything.
I don't love the side effects in the orElseGet() Supplier implementation... but at least they're constrained.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Okay, I can make it at least a bit clearer with this cleaned up (and already formatted!) version:
@Nullable
@Override
public CqlVector<SubtypeT> decode(
@Nullable ByteBuffer bytes, @NonNull ProtocolVersion protocolVersion) {
if (bytes == null || bytes.remaining() == 0) {
return null;
}
// Upfront check for fixed-size types only
subtypeCodec
.serializedSize()
.ifPresent(
(fixed_size) -> {
if (bytes.remaining() != cqlType.getDimensions() * fixed_size) {
throw new IllegalArgumentException(
String.format(
"Expected elements of uniform size, observed %d elements with total bytes %d",
cqlType.getDimensions(), bytes.remaining()));
}
});
;
ByteBuffer slice = bytes.slice();
List<SubtypeT> rv = new ArrayList<SubtypeT>(cqlType.getDimensions());
for (int i = 0; i < cqlType.getDimensions(); ++i) {
int size =
subtypeCodec
.serializedSize()
.orElseGet(() -> VIntCoding.getUnsignedVInt32(slice, slice.position()));
// If we aren't dealing with a fixed-size type we need to move the current slice position
// beyond the vint-encoded size of the current element. Ideally this would be
// serializedSize().ifNotPresent(Consumer) but the Optional API isn't doing us any favors
// there.
if (!subtypeCodec.serializedSize().isPresent())
slice.position(slice.position() + VIntCoding.computeUnsignedVIntSize(size));
int originalPosition = slice.position();
slice.limit(originalPosition + size);
rv.add(this.subtypeCodec.decode(slice, protocolVersion));
// Move to the start of the next element
slice.position(originalPosition + size);
// Reset the limit to the end of the buffer
slice.limit(slice.capacity());
}
// if too many elements, throw
if (slice.hasRemaining()) {
throw new IllegalArgumentException(
String.format(
"Too many elements; must provide elements for %d dimensions",
cqlType.getDimensions()));
}
return CqlVector.newInstance(rv);
}
Updated |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 but CI failure needs to be looked at
That CI failure appears to be from a run that went down on January 14 '25. There's been a lot of changes since then so I kicked off another run just now to make sure we're good (or at least approaching good). |
Yup, confirmed; re-run of the Jenkins build for this PR is now green. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One (relatively) minor change on a comment that I don't think applies anymore... and then I think we're there.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Many, many thanks to @SiyaoIsHiding for her excellent work and her persistence in getting this done!
All righty @SiyaoIsHiding, this ticket is finally wrapped up and we're ready to merge it in! Can you squash all the commits down to a single commit and add the "patch by" line? Really great work on this one! |
… handling of variable length types (OSS C* 5.0) patch by Jane He; reviewed by Bret McGuire and João Reis for JAVA-3143
4d81347
to
4658c33
Compare
Squashed! @absurdfarce |
Thanks @SiyaoIsHiding! |
… handling of variable length types (OSS C* 5.0) patch by Jane He; reviewed by Bret McGuire and João Reis reference: #1952
* CASSANDRA-19635: Run integration tests with C* 5.x patch by Lukasz Antoniak; reviewed by Andy Tolbert, and Bret McGuire for CASSANDRA-19635 * CASSANDRA-19635: Configure Jenkins to run integration tests with C* 5.x patch by Lukasz Antoniak; reviewed by Bret McGuire for CASSANDRA-19635 * update badge URL to org.apache.cassandra/java-driver-core * Limit calls to Conversions.resolveExecutionProfile Those repeated calls account for a non-negligible portion of my application CPU (0.6%) and can definitly be a final field so that it gets resolved only once per CqlRequestHandler. patch by Benoit Tellier; reviewed by Andy Tolbert, and Bret McGuire reference: apache#1623 * autolink JIRA tickets in commit messages patch by Stefan Miklosovic; reviewed by Michael Semb Wever for CASSANDRA-19854 * Don't return empty routing key when partition key is unbound DefaultBoundStatement#getRoutingKey has logic to infer the routing key when no one has explicitly called setRoutingKey or otherwise set the routing key on the statement. It however doesn't check for cases where nothing has been bound yet on the statement. This causes more problems if the user decides to get a BoundStatementBuilder from the PreparedStatement, set some fields on it, and then copy it by constructing new BoundStatementBuilder objects with the BoundStatement as a parameter, since the empty ByteBuffer gets copied to all bound statements, resulting in all requests being targeted to the same Cassandra node in a token-aware load balancing policy. patch by Ammar Khaku; reviewed by Andy Tolbert, and Bret McGuire reference: apache#1620 * JAVA-3167: CompletableFutures.allSuccessful() may return never completed future patch by Lukasz Antoniak; reviewed by Andy Tolbert, and Bret McGuire for JAVA-3167 * ninja-fix Various test fixes * Run integration tests with DSE 6.9.0 patch by Lukasz Antoniak; reviewed by Bret McGuire reference: apache#1955 * JAVA-3117: Call CcmCustomRule#after if CcmCustomRule#before fails to allow subsequent tests to run patch by Henry Hughes; reviewed by Alexandre Dutra and Andy Tolbert for JAVA-3117 * JAVA-3149: Support request cancellation in request throttler patch by Lukasz Antoniak; reviewed by Andy Tolbert and Chris Lohfink for JAVA-3149 * Fix C* 3.0 tests failing on Jenkins patch by Lukasz Antoniak; reviewed by Bret McGuire reference: apache#1939 * Reduce lock held duration in ConcurrencyLimitingRequestThrottler It might take some (small) time for callback handling when the throttler request proceeds to submission. Before this change, the throttler proceed request will happen while holding the lock, preventing other tasks from proceeding when there is spare capacity and even preventing tasks from enqueuing until the callback completes. By tracking the expected outcome, we can perform the callback outside of the lock. This means that request registration and submission can proceed even when a long callback is being processed. patch by Jason Koch; Reviewed by Andy Tolbert and Chris Lohfink for CASSANDRA-19922 * Annotate BatchStatement, Statement, SimpleStatement methods with CheckReturnValue Since the driver's default implementation is for BatchStatement and SimpleStatement methods to be immutable, we should annotate those methods with @CheckReturnValue. Statement#setNowInSeconds implementations are immutable so annotate that too. patch by Ammar Khaku; reviewed by Andy Tolbert and Bret McGuire reference: apache#1607 * Remove "beta" support for Java17 from docs patch by Bret McGuire; reviewed by Andy Tolbert and Alexandre Dutra reference: apache#1962 * Fix uncaught exception during graceful channel shutdown after exceeding max orphan ids patch by Christian Aistleitner; reviewed by Andy Tolbert, and Bret McGuire for apache#1938 * Build a public CI for Apache Cassandra Java Driver patch by Siyao (Jane) He; reviewed by Mick Semb Wever for CASSANDRA-19832 * CASSANDRA-19932: Allow to define extensions while creating table patch by Lukasz Antoniak; reviewed by Bret McGuire and Chris Lohfink * Fix DefaultSslEngineFactory missing null check on close patch by Abe Ratnofsky; reviewed by Andy Tolbert and Chris Lohfink for CASSANDRA-20001 * Query builder support for NOT CQL syntax patch by Bret McGuire; reviewed by Bret McGuire and Andy Tolbert for CASSANDRA-19930 * Fix CustomCcmRule to drop `CURRENT` flag no matter what If super.after() throws an Exception `CURRENT` flag is never dropped which leads next tests to fail with IllegalStateException("Attempting to use a Ccm rule while another is in use. This is disallowed") Patch by Dmitry Kropachev; reviewed by Andy Tolbert and Bret McGuire for JAVA-3117 * JAVA-3051: Memory leak patch by Jane He; reviewed by Alexandre Dutra and Bret McGuire for JAVA-3051 * Automate latest Cassandra versions when running CI patch by Siyao (Jane) He; reviewed by Mick Semb Wever for CASSJAVA-25 * Refactor integration tests to support multiple C* distributions. Test with DataStax HCD 1.0.0 patch by Lukasz Antoniak; reviewed by Bret McGuire reference: apache#1958 * Fix TableMetadata.describe() when containing a vector column patch by Stefan Miklosovic; reviewed by Bret McGuire for CASSJAVA-2 * Move Apache Cassandra 5.x off of beta1 and remove some older Apache Cassandra versions. patch by Bret McGuire; reviewed by Bret McGuire for CASSJAVA-54 * Update link to Jira to be CASSJAVA Updating the link to Jira. Previously we had a component in the CASSANDRA Jira project but now we have a project for each driver - in the case of Java, it's CASSJAVA. Added CASSJAVA to .asf.yaml patch by Jeremy Hanna; reviewed by Bret McGuire for CASSJAVA-61 * Move DataStax shaded Guava module into Java driver patch by Lukasz Antoniak; reviewed by Alexandre Dutra and Bret McGuire for CASSJAVA-52 * JAVA-3057 Allow decoding a UDT that has more fields than expected patch by Ammar Khaku; reviewed by Andy Tolbert and Bret McGuire reference: apache#1635 * CASSJAVA-55 Remove setting "Host" header for metadata requests. With some sysprops enabled this will actually be respected which completely borks Astra routing. patch by Bret McGuire; reviewed by Alexandre Dutra and Bret McGuire for CASSJAVA-55 * JAVA-3118: Add support for vector data type in Schema Builder, QueryBuilder patch by Jane He; reviewed by Mick Semb Wever and Bret McGuire for JAVA-3118 reference: apache#1931 * Upgrade Guava to 33.3.1-jre patch by Lukasz Antoniak; reviewed by Alexandre Dutra and Bret McGuire for CASSJAVA-53 * Do not always cleanup Guava shaded module before packaging * Revert "Do not always cleanup Guava shaded module before packaging" This reverts commit 5be52ec. * Conditionally compile shaded Guava module * JAVA-3143: Extend driver vector support to arbitrary subtypes and fix handling of variable length types (OSS C* 5.0) patch by Jane He; reviewed by Bret McGuire and João Reis reference: apache#1952 * JAVA-3168 Copy node info for contact points on initial node refresh only from first match by endpoint patch by Alex Sasnouskikh; reviewed by Andy Tolbert and Alexandre Dura for JAVA-3168 * JAVA-3055: Prevent PreparedStatement cache to be polluted if a request is cancelled. There was a critical issue when the external code cancels a request, indeed the cached CompletableFuture will then always throw a CancellationException. This may happens, for example, when used by reactive like Mono.zip or Mono.firstWithValue. patch by Luc Boutier; reviewed by Alexandre Dutra and Bret McGuire reference: apache#1757 * Expose a decorator for CqlPrepareAsyncProcessor cache rather than the ability to specify an arbitrary cache from scratch. Also bringing tests from apache#2003 forward with a few minor changes due to this implementation patch by Bret McGuire; reviewed by Bret McGuire and Andy Tolbert reference: apache#2008 * ninja-fix Using shaded Guava classes for import in order to make OSGi class paths happy. Major hat tip to Dmitry Konstantinov for the find here! * Changelog updates for 4.19.0 * [maven-release-plugin] prepare release 4.19.0 * [maven-release-plugin] prepare for next development iteration * Specify maven-clean-plugin version Sets the version to 3.4.1 in parent pom. Having it unspecified causes the following warning: ``` [WARNING] Some problems were encountered while building the effective model for com.scylladb:java-driver-guava-shaded:jar:4.19.0.0-SNAPSHOT [WARNING] 'build.plugins.plugin.version' for org.apache.maven.plugins:maven-clean-plugin is missing. @ line 97, column 15 [WARNING] [WARNING] It is highly recommended to fix these problems because they threaten the stability of your build. [WARNING] [WARNING] For this reason, future Maven versions might no longer support building such malformed projects. [WARNING] ``` * Install guava-shaded before running core's compile in CI Without this module available "Build" and "Unit tests" job fail with "package <x> does not exist" or "cannot find symbol" pointing to `[...].shaded.guava.[...]` packages. * Remove exception catch in `prepared_stmt_metadata_update_loopholes_test` Since incorporating "JAVA-3057 Allow decoding a UDT that has more fields than expected" the asuumptions of removed check are no longer valid. * Switch shaded guava's groupId in osgi-tests Switches from `org.apache.cassandra` to `com.scylladb` in `BundleOptions#commonBundles()`. * Typo: increase line's loglevel in CcmBridge --------- Co-authored-by: Lukasz Antoniak <[email protected]> Co-authored-by: Brad Schoening <[email protected]> Co-authored-by: Benoit Tellier <[email protected]> Co-authored-by: Stefan Miklosovic <[email protected]> Co-authored-by: Ammar Khaku <[email protected]> Co-authored-by: absurdfarce <[email protected]> Co-authored-by: Henry Hughes <[email protected]> Co-authored-by: Jason Koch <[email protected]> Co-authored-by: Christian Aistleitner <[email protected]> Co-authored-by: janehe <[email protected]> Co-authored-by: Abe Ratnofsky <[email protected]> Co-authored-by: absurdfarce <[email protected]> Co-authored-by: Dmitry Kropachev <[email protected]> Co-authored-by: janehe <[email protected]> Co-authored-by: Jeremy Hanna <[email protected]> Co-authored-by: SiyaoIsHiding <[email protected]> Co-authored-by: Alex Sasnouskikh <[email protected]> Co-authored-by: Luc Boutier <[email protected]>
No description provided.