Skip to content

Commit

Permalink
[IGNITE-19082] Catalog. Cleanup dead code
Browse files Browse the repository at this point in the history
cleanup.
  • Loading branch information
lowka committed May 2, 2024
1 parent c38524b commit 847170c
Show file tree
Hide file tree
Showing 5 changed files with 9 additions and 39 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,9 @@ public class CatalogManagerImpl extends AbstractEventProducer<CatalogEvent, Cata
/** Versioned catalog descriptors sorted in chronological order. */
private final NavigableMap<Long, Catalog> catalogByTs = new ConcurrentSkipListMap<>();

/** A future that completes when an empty catalog is initialised. If catalog is not empty this future when this completes starts. */
private final CompletableFuture<Void> catalogInitializationFuture = new CompletableFuture<>();

private final UpdateLog updateLog;

private final PendingComparableValuesTracker<Integer, Void> versionTracker = new PendingComparableValuesTracker<>(0);
Expand All @@ -133,8 +136,6 @@ public class CatalogManagerImpl extends AbstractEventProducer<CatalogEvent, Cata
/** Busy lock to stop synchronously. */
private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();

private final CompletableFuture<Void> catalogInitializationFuture = new CompletableFuture<>();

/**
* Constructor.
*/
Expand Down Expand Up @@ -511,8 +512,6 @@ private CompletableFuture<Integer> saveUpdate(UpdateProducer updateProducer, int
return completedFuture(newVersion);
}

System.err.println("RESULT? " + result);

return saveUpdate(updateProducer, attemptNo + 1);
});
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,12 +45,12 @@
* <p>TBD: events
*/
public interface CatalogService extends EventProducer<CatalogEvent, CatalogEventParameters> {
/** Default schema name. */
String DEFAULT_SCHEMA_NAME = "PUBLIC";

/** System schema name. */
String SYSTEM_SCHEMA_NAME = "SYSTEM";

int FIRST_CATALOG_VERSION = 0;

/** Default storage profile. */
String DEFAULT_STORAGE_PROFILE = "default";

Expand Down Expand Up @@ -114,10 +114,7 @@ public interface CatalogService extends EventProducer<CatalogEvent, CatalogEvent
CompletableFuture<Void> catalogReadyFuture(int version);

/**
* Returns a future, which completes, when catalog initialization completes.
* Returns a future, which completes when empty catalog is initialised. Otherwise this future completes upon startup.
*/
CompletableFuture<Void> catalogInitializationFuture();



}
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,12 @@
import static org.apache.ignite.internal.catalog.CatalogManagerImpl.INITIAL_CAUSALITY_TOKEN;
import static org.apache.ignite.internal.catalog.storage.serialization.CatalogSerializationUtils.readList;
import static org.apache.ignite.internal.catalog.storage.serialization.CatalogSerializationUtils.writeList;
import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;

import java.io.IOException;
import java.util.ArrayList;
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.ignite.internal.catalog.descriptors.CatalogTableSchemaVersions.TableVersion;
Expand Down Expand Up @@ -126,32 +124,12 @@ private CatalogTableDescriptor(
this.pkIndexId = pkIndexId;
this.zoneId = zoneId;
this.columns = Objects.requireNonNull(columns, "No columns defined.");
primaryKeyColumns = Objects.requireNonNull(pkCols, "No primary key columns.");
colocationColumns = colocationCols == null || colocationCols.isEmpty() ? pkCols : colocationCols;

this.primaryKeyColumns = Objects.requireNonNull(pkCols, "No primary key columns.");
this.columnsMap = columns.stream().collect(Collectors.toMap(CatalogTableColumnDescriptor::name, Function.identity()));

this.colocationColumns = Objects.requireNonNullElseGet(colocationCols, List::of);
this.schemaVersions = Objects.requireNonNull(schemaVersions, "No catalog schema versions.");

this.creationToken = creationToken;
this.storageProfile = Objects.requireNonNull(storageProfile, "No storage profile.");

if (columnsMap.isEmpty()) {
throw new IllegalArgumentException("Columns are not specified");
}

if (primaryKeyColumns.stream().anyMatch(c -> Objects.requireNonNull(columnsMap.get(c), c).nullable())) {
String message = format("Primary key columns contain nullable keys. Primary keys: {}", primaryKeyColumns);
throw new IllegalArgumentException(message);
}

if (!Set.copyOf(primaryKeyColumns).containsAll(colocationColumns)) {
String message = format(
"Primary key columns must contain all colocation columns. Primary keys: {}. Colocation columns: {}",
primaryKeyColumns, colocationColumns
);
throw new IllegalArgumentException(message);
}
this.creationToken = creationToken;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import static org.apache.ignite.internal.catalog.commands.CatalogUtils.SYSTEM_SCHEMAS;
import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.assertThrowsSqlException;
import static org.apache.ignite.internal.table.TableTestUtils.getTableStrict;
import static org.apache.ignite.lang.ErrorGroups.Sql.STMT_VALIDATION_ERR;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.hasSize;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -674,9 +674,6 @@ private CompletableFuture<SchemaPlus> waitForActualSchema(String schemaName, Hyb
SchemaPlus schema = sqlSchemaManager.schema(timestamp.longValue()).getSubSchema(schemaName);

if (schema == null) {
System.err.println("sqlSchemaCurrentTimestamp: " + clockService.now());
System.err.println("sqlSchemaActivationTimestamp: " + timestamp);

throw new SchemaNotFoundException(schemaName);
}

Expand Down

0 comments on commit 847170c

Please sign in to comment.