You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2023/11/21 12:55:13 UTC

(ignite-3) branch main updated: IGNITE-20807 Java thin: Implement nullable operations in ClientKeyValueView (#2855)

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

ptupitsyn pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new e1c9b1c4cf IGNITE-20807 Java thin: Implement nullable operations in ClientKeyValueView (#2855)
e1c9b1c4cf is described below

commit e1c9b1c4cf589c71aecc4815a3c8a14ae8fbf2f3
Author: Pavel Tupitsyn <pt...@apache.org>
AuthorDate: Tue Nov 21 14:55:07 2023 +0200

    IGNITE-20807 Java thin: Implement nullable operations in ClientKeyValueView (#2855)
    
    * Implement nullable operations in client KV view: `getNullableAsync`, `getOrDefaultAsync`, `getNullableAndPutAsync`, `getNullableAndRemoveAsync`, `getNullableAndReplaceAsync`
    * Throw `UnexpectedNullValueException` from all other methods (same way as we do in embedded mode)
---
 .../java/org/apache/ignite/table/KeyValueView.java |   5 +
 .../internal/client/table/ClientKeyValueView.java  |  69 +++++++++---
 .../ignite/client/ClientKeyValueViewTest.java      | 123 ++++++++++++++++++++-
 3 files changed, 182 insertions(+), 15 deletions(-)

diff --git a/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java b/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java
index 2ce7d764e8..1d3137bedd 100644
--- a/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java
@@ -69,6 +69,10 @@ public interface KeyValueView<K, V> extends DataStreamerTarget<Entry<K, V>> {
     /**
      * Gets a nullable value associated with a given key.
      *
+     * <p>Examples:
+     *     {@code getNullable(tx, key)} returns {@code null} after {@code remove(tx, key)}.
+     *     {@code getNullable(tx, key)} returns {@code Nullable.of(null)} after {@code put(tx, key, null)}.
+     *
      * @param tx Transaction or {@code null} to auto commit.
      * @param key Key whose value is to be returned. The key cannot be {@code null}.
      * @return Wrapped nullable value or {@code null} if it does not exist.
@@ -82,6 +86,7 @@ public interface KeyValueView<K, V> extends DataStreamerTarget<Entry<K, V>> {
      * @param tx Transaction or {@code null} to auto-commit.
      * @param key Key whose value is to be returned. The key cannot be {@code null}.
      * @return Future that represents the pending completion of the operation.
+     *     The future returns wrapped nullable value or {@code null} if the row with the given key does not exist.
      * @throws MarshallerException if the key doesn't match the schema.
      * @see #getNullable(Transaction, Object)
      */
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueView.java b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueView.java
index 2bccc85173..f44b242ceb 100644
--- a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueView.java
+++ b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueView.java
@@ -45,6 +45,7 @@ import org.apache.ignite.internal.marshaller.MarshallerException;
 import org.apache.ignite.internal.streamer.StreamerBatchSender;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.NullableValue;
+import org.apache.ignite.lang.UnexpectedNullValueException;
 import org.apache.ignite.table.DataStreamerOptions;
 import org.apache.ignite.table.KeyValueView;
 import org.apache.ignite.table.mapper.Mapper;
@@ -96,7 +97,7 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
         return tbl.doSchemaOutInOpAsync(
                 ClientOp.TUPLE_GET,
                 (s, w) -> keySer.writeRec(tx, key, s, w, TuplePart.KEY),
-                (s, r) -> valSer.readRec(s, r, TuplePart.VAL),
+                (s, r) -> throwIfNull(valSer.readRec(s, r, TuplePart.VAL)),
                 null,
                 ClientTupleSerializer.getPartitionAwarenessProvider(tx, keySer.mapper(), key));
     }
@@ -110,8 +111,15 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<NullableValue<V>> getNullableAsync(@Nullable Transaction tx, K key) {
-        // TODO IGNITE-20807
-        throw new UnsupportedOperationException("Not implemented yet.");
+        Objects.requireNonNull(key);
+
+        // Null means row does not exist, NullableValue.NULL means row exists, but mapped value column is null.
+        return tbl.doSchemaOutInOpAsync(
+                ClientOp.TUPLE_GET,
+                (s, w) -> keySer.writeRec(tx, key, s, w, TuplePart.KEY),
+                (s, r) -> NullableValue.of(valSer.readRec(s, r, TuplePart.VAL)),
+                null,
+                ClientTupleSerializer.getPartitionAwarenessProvider(tx, keySer.mapper(), key));
     }
 
     /** {@inheritDoc} */
@@ -123,8 +131,14 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<V> getOrDefaultAsync(@Nullable Transaction tx, K key, V defaultValue) {
-        // TODO IGNITE-20807
-        throw new UnsupportedOperationException("Not implemented yet.");
+        Objects.requireNonNull(key);
+
+        return tbl.doSchemaOutInOpAsync(
+                ClientOp.TUPLE_GET,
+                (s, w) -> keySer.writeRec(tx, key, s, w, TuplePart.KEY),
+                (s, r) -> valSer.readRec(s, r, TuplePart.VAL),
+                defaultValue,
+                ClientTupleSerializer.getPartitionAwarenessProvider(tx, keySer.mapper(), key));
     }
 
     /** {@inheritDoc} */
@@ -229,7 +243,7 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
         return tbl.doSchemaOutInOpAsync(
                 ClientOp.TUPLE_GET_AND_UPSERT,
                 (s, w) -> writeKeyValue(s, w, tx, key, val),
-                (s, r) -> valSer.readRec(s, r, TuplePart.VAL),
+                (s, r) -> throwIfNull(valSer.readRec(s, r, TuplePart.VAL)),
                 null,
                 ClientTupleSerializer.getPartitionAwarenessProvider(tx, keySer.mapper(), key));
     }
@@ -243,8 +257,14 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<NullableValue<V>> getNullableAndPutAsync(@Nullable Transaction tx, K key, V val) {
-        // TODO IGNITE-20807
-        throw new UnsupportedOperationException("Not implemented yet.");
+        Objects.requireNonNull(key);
+
+        return tbl.doSchemaOutInOpAsync(
+                ClientOp.TUPLE_GET_AND_UPSERT,
+                (s, w) -> writeKeyValue(s, w, tx, key, val),
+                (s, r) -> NullableValue.of(valSer.readRec(s, r, TuplePart.VAL)),
+                null,
+                ClientTupleSerializer.getPartitionAwarenessProvider(tx, keySer.mapper(), key));
     }
 
     /** {@inheritDoc} */
@@ -338,7 +358,7 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
         return tbl.doSchemaOutInOpAsync(
                 ClientOp.TUPLE_GET_AND_DELETE,
                 (s, w) -> keySer.writeRec(tx, key, s, w, TuplePart.KEY),
-                (s, r) -> valSer.readRec(s, r, TuplePart.VAL),
+                (s, r) -> throwIfNull(valSer.readRec(s, r, TuplePart.VAL)),
                 null,
                 ClientTupleSerializer.getPartitionAwarenessProvider(tx, keySer.mapper(), key));
     }
@@ -352,8 +372,14 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<NullableValue<V>> getNullableAndRemoveAsync(@Nullable Transaction tx, K key) {
-        // TODO IGNITE-20807
-        throw new UnsupportedOperationException("Not implemented yet.");
+        Objects.requireNonNull(key);
+
+        return tbl.doSchemaOutInOpAsync(
+                ClientOp.TUPLE_GET_AND_DELETE,
+                (s, w) -> keySer.writeRec(tx, key, s, w, TuplePart.KEY),
+                (s, r) -> NullableValue.of(valSer.readRec(s, r, TuplePart.VAL)),
+                null,
+                ClientTupleSerializer.getPartitionAwarenessProvider(tx, keySer.mapper(), key));
     }
 
     /** {@inheritDoc} */
@@ -413,7 +439,7 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
         return tbl.doSchemaOutInOpAsync(
                 ClientOp.TUPLE_GET_AND_REPLACE,
                 (s, w) -> writeKeyValue(s, w, tx, key, val),
-                (s, r) -> valSer.readRec(s, r, TuplePart.VAL),
+                (s, r) -> throwIfNull(valSer.readRec(s, r, TuplePart.VAL)),
                 null,
                 ClientTupleSerializer.getPartitionAwarenessProvider(tx, keySer.mapper(), key));
     }
@@ -427,8 +453,15 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
     /** {@inheritDoc} */
     @Override
     public CompletableFuture<NullableValue<V>> getNullableAndReplaceAsync(@Nullable Transaction tx, K key, V val) {
-        // TODO IGNITE-20807
-        throw new UnsupportedOperationException("Not implemented yet.");
+        Objects.requireNonNull(key);
+        Objects.requireNonNull(val);
+
+        return tbl.doSchemaOutInOpAsync(
+                ClientOp.TUPLE_GET_AND_REPLACE,
+                (s, w) -> writeKeyValue(s, w, tx, key, val),
+                (s, r) -> NullableValue.of(valSer.readRec(s, r, TuplePart.VAL)),
+                null,
+                ClientTupleSerializer.getPartitionAwarenessProvider(tx, keySer.mapper(), key));
     }
 
     private void writeKeyValue(ClientSchema s, PayloadOutputChannel w, @Nullable Transaction tx, K key, @Nullable V val) {
@@ -507,4 +540,12 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
 
         return ClientDataStreamer.streamData(publisher, opts, batchSender, provider, tbl);
     }
+
+    private static <T> T throwIfNull(T obj) {
+        if (obj == null) {
+            throw new UnexpectedNullValueException("Got unexpected null value: use `getNullable` sibling method instead.");
+        }
+
+        return obj;
+    }
 }
diff --git a/modules/client/src/test/java/org/apache/ignite/client/ClientKeyValueViewTest.java b/modules/client/src/test/java/org/apache/ignite/client/ClientKeyValueViewTest.java
index b0fa0cd1bb..3af37ae9d8 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/ClientKeyValueViewTest.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/ClientKeyValueViewTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.client;
 
 import static java.time.temporal.ChronoField.NANO_OF_SECOND;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.anyOf;
 import static org.hamcrest.Matchers.containsString;
@@ -38,7 +39,10 @@ import java.util.BitSet;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Consumer;
 import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.NullableValue;
+import org.apache.ignite.lang.UnexpectedNullValueException;
 import org.apache.ignite.table.KeyValueView;
 import org.apache.ignite.table.RecordView;
 import org.apache.ignite.table.Table;
@@ -288,6 +292,24 @@ public class ClientKeyValueViewTest extends AbstractClientTableTest {
         assertEquals("100", res[1]);
     }
 
+    @Test
+    public void testGetAllNullAndMissingValue() {
+        KeyValueView<Long, String> primitiveView = defaultTable().keyValueView(Mapper.of(Long.class), Mapper.of(String.class));
+
+        primitiveView.put(null, DEFAULT_ID, DEFAULT_NAME);
+        primitiveView.put(null, -1L, null);
+        primitiveView.remove(null, -2L);
+
+        var res = primitiveView.getAll(null, List.of(DEFAULT_ID, -1L, -2L));
+
+        assertEquals(2, res.size());
+        assertEquals(DEFAULT_NAME, res.get(DEFAULT_ID));
+        assertNull(res.get(-1L));
+
+        assertTrue(res.containsKey(-1L));
+        assertFalse(res.containsKey(-2L));
+    }
+
     @Test
     public void testPutAll() {
         KeyValueView<Long, String> pojoView = defaultTable().keyValueView(Mapper.of(Long.class), Mapper.of(String.class));
@@ -327,7 +349,7 @@ public class ClientKeyValueViewTest extends AbstractClientTableTest {
         pojoView.put(null, DEFAULT_ID, DEFAULT_NAME);
         pojoView.put(null, DEFAULT_ID, null);
 
-        assertNull(pojoView.get(null, DEFAULT_ID));
+        assertNull(pojoView.getNullable(null, DEFAULT_ID).get());
     }
 
     @Test
@@ -493,4 +515,103 @@ public class ClientKeyValueViewTest extends AbstractClientTableTest {
         assertTrue(ex.getMessage().contains("null was passed, but column is not nullable"), ex.getMessage());
         assertThat(Arrays.asList(ex.getStackTrace()), anyOf(hasToString(containsString("ClientKeyValueView"))));
     }
+
+    @Test
+    public void testGetNullValueThrows() {
+        testNullValueThrows(view -> view.get(null, DEFAULT_ID));
+    }
+
+    @Test
+    public void testGetAndPutNullValueThrows() {
+        testNullValueThrows(view -> view.getAndPut(null, DEFAULT_ID, DEFAULT_NAME));
+    }
+
+    @Test
+    public void testGetAndRemoveNullValueThrows() {
+        testNullValueThrows(view -> view.getAndRemove(null, DEFAULT_ID));
+    }
+
+    @Test
+    public void testGetAndReplaceNullValueThrows() {
+        testNullValueThrows(view -> view.getAndReplace(null, DEFAULT_ID, DEFAULT_NAME));
+    }
+
+    private void testNullValueThrows(Consumer<KeyValueView<Long, String>> run) {
+        KeyValueView<Long, String> primitiveView = defaultTable().keyValueView(Mapper.of(Long.class), Mapper.of(String.class));
+        primitiveView.put(null, DEFAULT_ID, null);
+
+        var ex = assertThrowsWithCause(() -> run.accept(primitiveView), UnexpectedNullValueException.class);
+        assertEquals(
+                "Failed to deserialize server response: Got unexpected null value: use `getNullable` sibling method instead.",
+                ex.getMessage());
+    }
+
+    @Test
+    public void testGetNullable() {
+        KeyValueView<Long, String> primitiveView = defaultTable().keyValueView(Mapper.of(Long.class), Mapper.of(String.class));
+
+        primitiveView.put(null, DEFAULT_ID, null);
+        primitiveView.remove(null, -1L);
+
+        NullableValue<String> nullVal = primitiveView.getNullable(null, DEFAULT_ID);
+        NullableValue<String> missingVal = primitiveView.getNullable(null, -1L);
+
+        assertNull(nullVal.get());
+        assertNull(missingVal);
+    }
+
+    @Test
+    public void testGetNullableAndPut() {
+        KeyValueView<Long, String> primitiveView = defaultTable().keyValueView(Mapper.of(Long.class), Mapper.of(String.class));
+
+        primitiveView.put(null, DEFAULT_ID, null);
+        primitiveView.remove(null, -1L);
+
+        NullableValue<String> nullVal = primitiveView.getNullableAndPut(null, DEFAULT_ID, DEFAULT_NAME);
+        NullableValue<String> missingVal = primitiveView.getNullableAndPut(null, -1L, DEFAULT_NAME);
+
+        assertNull(nullVal.get());
+        assertNull(missingVal);
+    }
+
+    @Test
+    public void testGetNullableAndRemove() {
+        KeyValueView<Long, String> primitiveView = defaultTable().keyValueView(Mapper.of(Long.class), Mapper.of(String.class));
+
+        primitiveView.put(null, DEFAULT_ID, null);
+        primitiveView.remove(null, -1L);
+
+        NullableValue<String> nullVal = primitiveView.getNullableAndRemove(null, DEFAULT_ID);
+        NullableValue<String> missingVal = primitiveView.getNullableAndRemove(null, -1L);
+
+        assertNull(nullVal.get());
+        assertNull(missingVal);
+    }
+
+    @Test
+    public void testGetNullableAndReplace() {
+        KeyValueView<Long, String> primitiveView = defaultTable().keyValueView(Mapper.of(Long.class), Mapper.of(String.class));
+
+        primitiveView.put(null, DEFAULT_ID, null);
+        primitiveView.remove(null, -1L);
+
+        NullableValue<String> nullVal = primitiveView.getNullableAndReplace(null, DEFAULT_ID, DEFAULT_NAME);
+        NullableValue<String> missingVal = primitiveView.getNullableAndReplace(null, -1L, DEFAULT_NAME);
+
+        assertNull(nullVal.get());
+        assertNull(missingVal);
+    }
+
+    @Test
+    public void testGetOrDefault() {
+        KeyValueView<Long, String> primitiveView = defaultTable().keyValueView(Mapper.of(Long.class), Mapper.of(String.class));
+
+        primitiveView.put(null, DEFAULT_ID, DEFAULT_NAME);
+        primitiveView.put(null, -1L, null);
+        primitiveView.remove(null, -2L);
+
+        assertNull(primitiveView.getOrDefault(null, -1L, "default"));
+        assertEquals(DEFAULT_NAME, primitiveView.getOrDefault(null, DEFAULT_ID, "default"));
+        assertEquals("default", primitiveView.getOrDefault(null, -2L, "default"));
+    }
 }