Skip to content

Commit

Permalink
IGNITE-24305 Fix broken Public API compatibility with 3.0 (#5115)
Browse files Browse the repository at this point in the history
  • Loading branch information
AMashenkov authored Jan 24, 2025
1 parent 9b5b7d6 commit 32069fe
Show file tree
Hide file tree
Showing 32 changed files with 78 additions and 65 deletions.
30 changes: 21 additions & 9 deletions modules/api/src/main/java/org/apache/ignite/table/Table.java
Original file line number Diff line number Diff line change
Expand Up @@ -36,11 +36,23 @@
*/
public interface Table {
/**
* Gets the name of the table.
* Gets the canonical name of the table ([schema_name].[table_name]) with SQL-parser style quotation.
*
* @return Table name.
* <p>E.g. "PUBLIC.TBL0" - for TBL0 table in PUBLIC schema (both names are case insensitive),
* "\"MySchema\".\"Tbl0\"" - for Tbl0 table in MySchema schema (both names are case sensitive), etc.
*
* @return Canonical table name.
*/
default String name() {
return qualifiedName().toCanonicalForm();
}

/**
* Gets the qualified name of the table.
*
* @return Qualified name of the table.
*/
QualifiedName name();
QualifiedName qualifiedName();

/**
* Gets the partition manager.
Expand All @@ -53,7 +65,7 @@ public interface Table {
* Gets a record view of the table using the specified record class mapper.
*
* @param recMapper Record class mapper.
* @param <R> Record type.
* @param <R> Record type.
* @return Table record view.
*/
<R> RecordView<R> recordView(Mapper<R> recMapper);
Expand All @@ -69,7 +81,7 @@ public interface Table {
* Gets a record view of the table using the default mapper for the specified record class.
*
* @param recCls Record class.
* @param <R> Record type.
* @param <R> Record type.
* @return Table record view.
*/
default <R> RecordView<R> recordView(Class<R> recCls) {
Expand All @@ -81,8 +93,8 @@ default <R> RecordView<R> recordView(Class<R> recCls) {
*
* @param keyMapper Key class mapper.
* @param valMapper Value class mapper.
* @param <K> Key type.
* @param <V> Value type.
* @param <K> Key type.
* @param <V> Value type.
* @return Table key-value view.
*/
<K, V> KeyValueView<K, V> keyValueView(Mapper<K> keyMapper, Mapper<V> valMapper);
Expand All @@ -99,8 +111,8 @@ default <R> RecordView<R> recordView(Class<R> recCls) {
*
* @param keyCls Key class.
* @param valCls Value class.
* @param <K> Key type.
* @param <V> Value type.
* @param <K> Key type.
* @param <V> Value type.
* @return Table key-value view.
*/
default <K, V> KeyValueView<K, V> keyValueView(Class<K> keyCls, Class<V> valCls) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -405,9 +405,9 @@ public void createAndGetDefinitionTest() {
@Test
public void createAllColumnTypesFromPojo() {
Table table = catalog().createTable(AllColumnTypesPojo.class);
assertEquals("ALLCOLUMNTYPESPOJO", table.name().objectName());
assertEquals("ALLCOLUMNTYPESPOJO", table.qualifiedName().objectName());

TableDefinition tableDef = catalog().tableDefinition(table.name());
TableDefinition tableDef = catalog().tableDefinition(table.qualifiedName());
assertEquals(tableDef.tableName(), tableDef.tableName());

List<ColumnDefinition> columns = tableDef.columns();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public static CompletableFuture<Void> process(
out.packNil();
} else {
out.packInt(((TableViewInternal) table).tableId());
out.packString(quoteTableNameIfNotAllUpper(table.name().objectName()));
out.packString(quoteTableNameIfNotAllUpper(table.qualifiedName().objectName()));
}
});
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ public static CompletableFuture<Void> process(
var tableImpl = (TableViewInternal) table;

out.packInt(tableImpl.tableId());
out.packString(quoteTableNameIfNotAllUpper(table.name().objectName()));
out.packString(quoteTableNameIfNotAllUpper(table.qualifiedName().objectName()));
}
});
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -466,7 +466,7 @@ private CompletableFuture<ClientTable> getTable(String tableName) {
}

ClientTable clientTable = (ClientTable) t;
tableCache.put(t.name(), clientTable);
tableCache.put(t.qualifiedName(), clientTable);

return clientTable;
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ public CompletableFuture<AsyncCursor<T>> queryAsync(
.thenCompose((schema) -> {
SqlSerializer ser = new SqlSerializer.Builder()
.columns(Arrays.asList(columnNames(schema.columns())))
.tableName(tbl.name())
.tableName(tbl.qualifiedName())
.indexName(indexName != null ? QualifiedName.parse(indexName).objectName() : null)
.where(criteria)
.build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ ReliableChannel channel() {

/** {@inheritDoc} */
@Override
public QualifiedName name() {
public QualifiedName qualifiedName() {
return name;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ public void beforeEach() throws InterruptedException {

protected void dropTables(Ignite ignite) {
for (var t : ignite.tables().tables()) {
((FakeIgniteTables) ignite.tables()).dropTable(t.name());
((FakeIgniteTables) ignite.tables()).dropTable(t.qualifiedName());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,8 @@ public void loggersSetToDifferentClientsNotInterfereWithEachOther() {
var client1 = createClient(loggerFactory1, 10901, 10902);
var client2 = createClient(loggerFactory2, 10901, 10902);

assertEquals("T", client1.tables().tables().get(0).name().objectName());
assertEquals("T", client2.tables().tables().get(0).name().objectName());
assertEquals("T", client1.tables().tables().get(0).qualifiedName().objectName());
assertEquals("T", client2.tables().tables().get(0).qualifiedName().objectName());

server.close();

Expand All @@ -71,8 +71,8 @@ public void loggersSetToDifferentClientsNotInterfereWithEachOther() {

server2 = startServer(ignite2, 10902);

assertEquals("T2", client1.tables().tables().get(0).name().objectName());
assertEquals("T2", client2.tables().tables().get(0).name().objectName());
assertEquals("T2", client1.tables().tables().get(0).qualifiedName().objectName());
assertEquals("T2", client2.tables().tables().get(0).qualifiedName().objectName());

assertThat(loggerFactory1.logger.entries(), not(empty()));
assertThat(loggerFactory2.logger.entries(), not(empty()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -479,7 +479,7 @@ public void testGetFromDroppedTableThrowsException() {

private void checkSchemaUpdate(Consumer<RecordView<Tuple>> consumer) throws Exception {
try (var client2 = startClient()) {
var table = client2.tables().table(defaultTable().name());
var table = client2.tables().table(defaultTable().qualifiedName());
Map<Integer, Object> schemas = IgniteTestUtils.getFieldValue(table, "schemas");
var recView = table.recordView();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,12 @@

package org.apache.ignite.client;

import static java.util.stream.Collectors.toList;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;

import java.util.stream.Collectors;
import org.apache.ignite.client.fakes.FakeIgniteTables;
import org.apache.ignite.table.Table;
import org.junit.jupiter.api.Test;
Expand All @@ -39,7 +39,7 @@ public void testTablesWhenTablesExist() {
var tables = client.tables().tables();
assertEquals(2, tables.size());

assertThat(tables.stream().map(t -> t.name().objectName()).collect(Collectors.toList()), containsInAnyOrder(DEFAULT_TABLE, "T"));
assertThat(tables.stream().map(t -> t.qualifiedName().objectName()).collect(toList()), containsInAnyOrder(DEFAULT_TABLE, "T"));
}

@Test
Expand All @@ -54,7 +54,7 @@ public void testTableReturnsInstanceWhenExists() {
((FakeIgniteTables) server.tables()).createTable(DEFAULT_TABLE);
Table table = client.tables().table(DEFAULT_TABLE);

assertEquals(DEFAULT_TABLE, table.name().objectName());
assertEquals(DEFAULT_TABLE, table.qualifiedName().objectName());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -449,8 +449,8 @@ public void testExecuteColocatedTupleKeyRoutesRequestToPrimaryNode() {

JobDescriptor<Object, String> job = JobDescriptor.<Object, String>builder("job").build();

assertThat(compute().executeAsync(JobTarget.colocated(table.name(), t1), job, null), willBe(nodeKey1));
assertThat(compute().executeAsync(JobTarget.colocated(table.name(), t2), job, null), willBe(nodeKey2));
assertThat(compute().executeAsync(JobTarget.colocated(table.qualifiedName(), t1), job, null), willBe(nodeKey1));
assertThat(compute().executeAsync(JobTarget.colocated(table.qualifiedName(), t2), job, null), willBe(nodeKey2));
}

@Test
Expand All @@ -459,8 +459,8 @@ public void testExecuteColocatedObjectKeyRoutesRequestToPrimaryNode() {
Table table = defaultTable();
JobDescriptor<Object, String> job = JobDescriptor.<Object, String>builder("job").build();

assertThat(compute().executeAsync(JobTarget.colocated(table.name(), 1L, mapper), job, null), willBe(nodeKey1));
assertThat(compute().executeAsync(JobTarget.colocated(table.name(), 2L, mapper), job, null), willBe(nodeKey2));
assertThat(compute().executeAsync(JobTarget.colocated(table.qualifiedName(), 1L, mapper), job, null), willBe(nodeKey1));
assertThat(compute().executeAsync(JobTarget.colocated(table.qualifiedName(), 2L, mapper), job, null), willBe(nodeKey2));
}

@ParameterizedTest
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ public void clientReconnectsToAnotherAddressOnNodeFail() {
.retryPolicy(new RetryLimitPolicy().retryLimit(100))
.build();

assertEquals("T", client.tables().tables().get(0).name().objectName());
assertEquals("T", client.tables().tables().get(0).qualifiedName().objectName());

server.close();

Expand All @@ -70,7 +70,7 @@ public void clientReconnectsToAnotherAddressOnNodeFail() {

server2 = new TestServer(0, ignite2, null, null, null, AbstractClientTest.clusterId, null, 10950);

assertEquals("T2", client.tables().tables().get(0).name().objectName());
assertEquals("T2", client.tables().tables().get(0).qualifiedName().objectName());
}

@Test
Expand All @@ -85,7 +85,7 @@ public void testOperationFailsWhenAllServersFail() {
.addresses("127.0.0.1:" + server.port(), "127.0.0.1:10960")
.build();

assertEquals("T", client.tables().tables().get(0).name().objectName());
assertEquals("T", client.tables().tables().get(0).qualifiedName().objectName());

server.close();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ public void testNoRetryPolicySecondRequestFails() {
initServer(reqId -> reqId % 3 == 0);

try (var client = getClient(null)) {
assertEquals("T", client.tables().tables().get(0).name().objectName());
assertEquals("T", client.tables().tables().get(0).qualifiedName().objectName());
assertThrows(IgniteException.class, () -> client.tables().tables().get(0).name());
}
}
Expand All @@ -79,7 +79,7 @@ public void testRetryPolicyCompletesOperationWithoutException() {

try (var client = getClient(plc)) {
for (int i = 0; i < ITER; i++) {
assertEquals("T", client.tables().tables().get(0).name().objectName());
assertEquals("T", client.tables().tables().get(0).qualifiedName().objectName());
}

assertEquals(ITER / 2 - 1, plc.invocations.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -470,10 +470,9 @@ private CompletableFuture<ClusterNode> primaryReplicaForPartition(TableViewInter
return topologyService.getById(replicaMeta.getLeaseholderId());
}

String tableName = table.name().toCanonicalForm();
throw new ComputeException(
Compute.PRIMARY_REPLICA_RESOLVE_ERR,
"Can not find primary replica for [table=" + tableName + ", partition=" + partitionIndex + "]."
"Can not find primary replica for [table=" + table.name() + ", partition=" + partitionIndex + "]."
);
});
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ enum SyncApiOperation {
TABLES_TABLE(refs -> refs.tables.table(TEST_TABLE_NAME)),

TABLE_NAME(refs -> refs.table.name()),
TABLE_QUALIFIED_NAME(refs -> refs.table.qualifiedName()),
TABLE_KV_VIEW(refs -> refs.table.keyValueView()),
TABLE_TYPED_KV_VIEW(refs -> refs.table.keyValueView(Integer.class, String.class)),
TABLE_MAPPED_KV_VIEW(refs -> refs.table.keyValueView(Mapper.of(Integer.class), Mapper.of(String.class))),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1291,7 +1291,7 @@ private void assertTablePresent(TableManager tableManager, String tableName) {
boolean isPresent = false;

for (TableImpl table : tables) {
if (table.name().objectName().equals(tableName)) {
if (table.qualifiedName().objectName().equals(tableName)) {
isPresent = true;

break;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -276,7 +276,7 @@ void colocated() {
column("v", ColumnType.INT32)
)
.build()
).name();
).qualifiedName();

// When run job with custom marshaller for string argument.
var tup = Tuple.create().set("key", 1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ void testThinClientConnectsToServerNodesAndExecutesBasicTableOperations() {
assertEquals(1, tables.size());

Table table = tables.get(0);
assertEquals(TABLE_NAME, table.name().objectName());
assertEquals(TABLE_NAME, table.qualifiedName().objectName());

var tuple = Tuple.create().set(COLUMN_KEY, 1).set(COLUMN_VAL, "Hello");
var keyTuple = Tuple.create().set(COLUMN_KEY, 1);
Expand Down Expand Up @@ -130,14 +130,14 @@ void testExceptionHasHint() {
void testServerReturnsActualTableName() {
// Quoting is not necessary.
Table table = client().tables().table("tbl1");
assertEquals("TBL1", table.name().objectName());
assertEquals("TBL1", table.qualifiedName().objectName());

// Quoting is necessary.
client().sql().execute(null, "CREATE TABLE IF NOT EXISTS \"tbl-2\" (key INTEGER PRIMARY KEY)");

try {
Table table2 = client().tables().table("\"tbl-2\"");
assertEquals("tbl-2", table2.name().objectName());
assertEquals("tbl-2", table2.qualifiedName().objectName());
} finally {
client().sql().execute(null, "DROP TABLE \"tbl-2\"");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ void forwardIncompatibleSchemaChangesDoNotAllowSyncCommit(ForwardIncompatibleDdl
containsString(String.format(
"Commit failed because schema is not forward-compatible [fromSchemaVersion=1, toSchemaVersion=2, table=%s, "
+ "details=%s]",
table.name().objectName(),
table.qualifiedName().objectName(),
ddl.expectedDetails
))
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ void readWriteOperationInTxAfterAlteringSchemaOnTargetTableIsRejected(Operation
ex.getMessage(),
containsString(String.format(
"Table schema was updated after the transaction was started [table=%s, startSchema=1, operationSchema=2]",
table.name().objectName()
table.qualifiedName().objectName()
))
);
} else {
Expand All @@ -118,7 +118,7 @@ void readWriteOperationInTxAfterAlteringSchemaOnTargetTableIsRejected(Operation
ex.getMessage(),
is(String.format(
"Table schema was updated after the transaction was started [table=%s, startSchema=1, operationSchema=2]",
table.name().objectName()
table.qualifiedName().objectName()
))
);
}
Expand Down Expand Up @@ -146,7 +146,7 @@ private void enlistTableInTransaction(Table table, Transaction tx) {

private static void executeRwReadOn(Table table, Transaction tx, int key, Cluster cluster) {
cluster.doInSession(0, session -> {
executeUpdate("SELECT * FROM " + table.name().toCanonicalForm() + " WHERE id = " + key, session, tx);
executeUpdate("SELECT * FROM " + table.name() + " WHERE id = " + key, session, tx);
});
}

Expand Down Expand Up @@ -177,7 +177,7 @@ boolean sql() {
@Override
void execute(Table table, Transaction tx, Cluster cluster) {
cluster.doInSession(0, session -> {
executeUpdate("UPDATE " + table.name().toCanonicalForm() + " SET val = 'new value' WHERE id = " + KEY, session, tx);
executeUpdate("UPDATE " + table.name() + " SET val = 'new value' WHERE id = " + KEY, session, tx);
});
}

Expand Down Expand Up @@ -297,7 +297,8 @@ void commitAfterDroppingTargetTableIsRejected(CommitOperation operation) {
assertThat(ex, is(instanceOf(IncompatibleSchemaException.class)));
assertThat(
ex.getMessage(),
containsString(String.format("Commit failed because a table was already dropped [table=%s]", table.name().objectName()))
containsString(String.format("Commit failed because a table was already dropped [table=%s]",
table.qualifiedName().objectName()))
);

assertThat(((IncompatibleSchemaException) ex).code(), is(Transactions.TX_INCOMPATIBLE_SCHEMA_ERR));
Expand Down Expand Up @@ -357,7 +358,7 @@ void readingDataInFutureVersionsFails(boolean scan) {
containsString(String.format(
"Operation failed because it tried to access a row with newer schema version than transaction's [table=%s, "
+ "txSchemaVersion=1, rowSchemaVersion=2]",
table.name().objectName()
table.qualifiedName().objectName()
))
);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public void createTables() {
@BeforeEach
public void clearTables() {
for (Table t : CLUSTER.aliveNode().tables().tables()) {
sql("DELETE FROM " + t.name().toCanonicalForm());
sql("DELETE FROM " + t.name());
}
}

Expand Down
Loading

0 comments on commit 32069fe

Please sign in to comment.