Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,10 @@
import org.apache.fluss.security.acl.FlussPrincipal;

import java.util.List;
import java.util.Optional;

/**
* A catalog interface to modify metadata in external datalake.
* A catalog interface to modify or get metadata in external datalake.
*
* @since 0.7
*/
Expand Down Expand Up @@ -57,6 +58,17 @@ void createTable(TablePath tablePath, TableDescriptor tableDescriptor, Context c
void alterTable(TablePath tablePath, List<TableChange> tableChanges, Context context)
throws TableNotExistException;

/**
* Get the latest snapshot info for the given table.
*
* @param tablePath path of the table to get the latest snapshot info
* @param context contextual information needed for get latest snapshot info
* @return the latest snapshot information for the given table, or empty if the table does not
* exist or has no snapshots.
* @since 0.9
*/
Optional<LakeSnapshotInfo> getLatestSnapshotInfo(TablePath tablePath, Context context);

@Override
default void close() throws Exception {
// default do nothing
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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.fluss.lake.lakestorage;

import javax.annotation.Nullable;

/**
* Represents the metadata information of a snapshot in a data lake table.
*
* @see LakeCatalog#getLatestSnapshotInfo
* @since 0.9
*/
public class LakeSnapshotInfo {

private final long snapshotId;

private final long commitTimestampMillis;

/**
* The {@code fluss-offsets} property recorded in the snapshot summary.
*
* <p>This property has two different formats depending on the Fluss version that produced the
* snapshot:
*
* <ul>
* <li><b>v1 (JSON format, produced by Fluss 0.8):</b> A JSON string starting with <code>
* '&#123;'</code> that contains the serialized {@code TableBucketOffsets} data directly.
* <li><b>v2 (Path format, produced by Fluss 0.9+):</b> A file path pointing to the offsets
* file, following the pattern: {@code
* {remote.data.dir}/lake/{databaseName}/{tableName}-{tableId}/metadata/{UUID}.offsets}
* </ul>
*/
@Nullable private final String flussOffsetsProperty;

public LakeSnapshotInfo(
long snapshotId, long commitTimestampMillis, @Nullable String flussOffsetsProperty) {
this.snapshotId = snapshotId;
this.commitTimestampMillis = commitTimestampMillis;
this.flussOffsetsProperty = flussOffsetsProperty;
}

public long getSnapshotId() {
return snapshotId;
}

public long getCommitTimestampMillis() {
return commitTimestampMillis;
}

@Nullable
public String getFlussOffsetsProperty() {
return flussOffsetsProperty;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.fluss.utils.WrappingProxy;

import java.util.List;
import java.util.Optional;

/**
* A wrapper around {@link LakeStoragePlugin} that ensures the plugin classloader is used for all
Expand Down Expand Up @@ -91,6 +92,14 @@ public void alterTable(TablePath tablePath, List<TableChange> tableChanges, Cont
}
}

@Override
public Optional<LakeSnapshotInfo> getLatestSnapshotInfo(
TablePath tablePath, Context context) {
try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(loader)) {
return inner.getLatestSnapshotInfo(tablePath, context);
}
}

@Override
public void close() throws Exception {
try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(loader)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;

import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
Expand Down Expand Up @@ -153,5 +154,11 @@ public void createTable(
@Override
public void alterTable(TablePath tablePath, List<TableChange> tableChanges, Context context)
throws TableNotExistException {}

@Override
public Optional<LakeSnapshotInfo> getLatestSnapshotInfo(
TablePath tablePath, Context context) {
return Optional.empty();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,13 @@
import org.apache.fluss.exception.TableAlreadyExistException;
import org.apache.fluss.exception.TableNotExistException;
import org.apache.fluss.lake.lakestorage.LakeCatalog;
import org.apache.fluss.lake.lakestorage.LakeSnapshotInfo;
import org.apache.fluss.metadata.TableChange;
import org.apache.fluss.metadata.TableDescriptor;
import org.apache.fluss.metadata.TablePath;

import java.util.List;
import java.util.Optional;

/** Implementation of {@link LakeCatalog} for values lake. */
public class TestingValuesLakeCatalog implements LakeCatalog {
Expand All @@ -40,4 +42,9 @@ public void alterTable(TablePath tablePath, List<TableChange> tableChanges, Cont
throws TableNotExistException {
throw new RuntimeException("Not impl.");
}

@Override
public Optional<LakeSnapshotInfo> getLatestSnapshotInfo(TablePath tablePath, Context context) {
return Optional.empty();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.fluss.exception.TableNotExistException;
import org.apache.fluss.lake.iceberg.utils.IcebergCatalogUtils;
import org.apache.fluss.lake.lakestorage.LakeCatalog;
import org.apache.fluss.lake.lakestorage.LakeSnapshotInfo;
import org.apache.fluss.metadata.TableChange;
import org.apache.fluss.metadata.TableDescriptor;
import org.apache.fluss.metadata.TablePath;
Expand Down Expand Up @@ -52,6 +53,7 @@
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;

import static org.apache.fluss.metadata.TableDescriptor.BUCKET_COLUMN_NAME;
Expand Down Expand Up @@ -162,6 +164,11 @@ public void alterTable(TablePath tablePath, List<TableChange> tableChanges, Cont
}
}

@Override
public Optional<LakeSnapshotInfo> getLatestSnapshotInfo(TablePath tablePath, Context context) {
return Optional.empty();
}

private TableIdentifier toIcebergTableIdentifier(TablePath tablePath) {
return TableIdentifier.of(tablePath.getDatabaseName(), tablePath.getTableName());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,15 @@
import org.apache.fluss.exception.InvalidAlterTableException;
import org.apache.fluss.exception.TableAlreadyExistException;
import org.apache.fluss.exception.TableNotExistException;
import org.apache.fluss.lake.committer.LakeCommitter;
import org.apache.fluss.lake.lakestorage.LakeCatalog;
import org.apache.fluss.lake.lakestorage.LakeSnapshotInfo;
import org.apache.fluss.metadata.TableChange;
import org.apache.fluss.metadata.TableDescriptor;
import org.apache.fluss.metadata.TablePath;
import org.apache.fluss.utils.IOUtils;

import org.apache.paimon.Snapshot;
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.catalog.CatalogContext;
import org.apache.paimon.catalog.CatalogFactory;
Expand All @@ -44,6 +47,7 @@

import java.util.LinkedHashMap;
import java.util.List;
import java.util.Optional;

import static org.apache.fluss.lake.paimon.utils.PaimonConversions.toPaimon;
import static org.apache.fluss.lake.paimon.utils.PaimonConversions.toPaimonSchema;
Expand Down Expand Up @@ -133,6 +137,36 @@ public void alterTable(TablePath tablePath, List<TableChange> tableChanges, Cont
}
}

@Override
public Optional<LakeSnapshotInfo> getLatestSnapshotInfo(TablePath tablePath, Context context) {
Identifier identifier =
Identifier.create(tablePath.getDatabaseName(), tablePath.getTableName());
Table paimonTable;
try {
paimonTable = paimonCatalog.getTable(identifier);
} catch (Catalog.TableNotExistException e) {
return Optional.empty();
}

FileStoreTable fileStoreTable = (FileStoreTable) paimonTable;
Snapshot snapshot = fileStoreTable.snapshotManager().latestSnapshot();
if (snapshot == null) {
return Optional.empty();
}

String flussOffsets =
Optional.ofNullable(snapshot.properties())
.map(
props ->
props.get(
LakeCommitter
.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY))
.orElse(null);

return Optional.of(
new LakeSnapshotInfo(snapshot.id(), snapshot.timeMillis(), flussOffsets));
}

private boolean shouldAlterTable(TablePath tablePath, List<TableChange> tableChanges)
throws TableNotExistException {
try {
Expand Down
Loading
Loading