All Downloads are FREE. Search and download functionalities are using the official Maven repository.

io.trino.plugin.deltalake.DeltaLakeMetadata Maven / Gradle / Ivy

/*
 * Licensed 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 io.trino.plugin.deltalake;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.VerifyException;
import com.google.common.collect.Comparators;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.ImmutableTable;
import com.google.common.collect.Sets;
import io.airlift.json.JsonCodec;
import io.airlift.log.Logger;
import io.airlift.slice.Slice;
import io.airlift.stats.cardinality.HyperLogLog;
import io.airlift.units.DataSize;
import io.trino.filesystem.FileIterator;
import io.trino.filesystem.Location;
import io.trino.filesystem.TrinoFileSystem;
import io.trino.filesystem.TrinoFileSystemFactory;
import io.trino.plugin.base.classloader.ClassLoaderSafeSystemTable;
import io.trino.plugin.base.filter.UtcConstraintExtractor;
import io.trino.plugin.base.projection.ApplyProjectionUtil;
import io.trino.plugin.deltalake.DeltaLakeAnalyzeProperties.AnalyzeMode;
import io.trino.plugin.deltalake.expression.ParsingException;
import io.trino.plugin.deltalake.expression.SparkExpressionParser;
import io.trino.plugin.deltalake.metastore.DeltaLakeMetastore;
import io.trino.plugin.deltalake.metastore.DeltaMetastoreTable;
import io.trino.plugin.deltalake.metastore.NotADeltaLakeTableException;
import io.trino.plugin.deltalake.procedure.DeltaLakeTableExecuteHandle;
import io.trino.plugin.deltalake.procedure.DeltaLakeTableProcedureId;
import io.trino.plugin.deltalake.procedure.DeltaTableOptimizeHandle;
import io.trino.plugin.deltalake.statistics.CachingExtendedStatisticsAccess;
import io.trino.plugin.deltalake.statistics.DeltaLakeColumnStatistics;
import io.trino.plugin.deltalake.statistics.DeltaLakeTableStatisticsProvider;
import io.trino.plugin.deltalake.statistics.ExtendedStatistics;
import io.trino.plugin.deltalake.transactionlog.AddFileEntry;
import io.trino.plugin.deltalake.transactionlog.CdcEntry;
import io.trino.plugin.deltalake.transactionlog.CommitInfoEntry;
import io.trino.plugin.deltalake.transactionlog.DeltaLakeComputedStatistics;
import io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport;
import io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.ColumnMappingMode;
import io.trino.plugin.deltalake.transactionlog.MetadataEntry;
import io.trino.plugin.deltalake.transactionlog.MetadataEntry.Format;
import io.trino.plugin.deltalake.transactionlog.ProtocolEntry;
import io.trino.plugin.deltalake.transactionlog.RemoveFileEntry;
import io.trino.plugin.deltalake.transactionlog.TableSnapshot;
import io.trino.plugin.deltalake.transactionlog.TransactionLogAccess;
import io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointWriterManager;
import io.trino.plugin.deltalake.transactionlog.statistics.DeltaLakeFileStatistics;
import io.trino.plugin.deltalake.transactionlog.statistics.DeltaLakeJsonFileStatistics;
import io.trino.plugin.deltalake.transactionlog.writer.TransactionConflictException;
import io.trino.plugin.deltalake.transactionlog.writer.TransactionLogWriter;
import io.trino.plugin.deltalake.transactionlog.writer.TransactionLogWriterFactory;
import io.trino.plugin.hive.HiveType;
import io.trino.plugin.hive.TrinoViewHiveMetastore;
import io.trino.plugin.hive.metastore.Column;
import io.trino.plugin.hive.metastore.Database;
import io.trino.plugin.hive.metastore.HivePrincipal;
import io.trino.plugin.hive.metastore.PrincipalPrivileges;
import io.trino.plugin.hive.metastore.StorageFormat;
import io.trino.plugin.hive.metastore.Table;
import io.trino.plugin.hive.security.AccessControlMetadata;
import io.trino.spi.NodeManager;
import io.trino.spi.TrinoException;
import io.trino.spi.block.Block;
import io.trino.spi.connector.Assignment;
import io.trino.spi.connector.BeginTableExecuteResult;
import io.trino.spi.connector.CatalogSchemaTableName;
import io.trino.spi.connector.ColumnHandle;
import io.trino.spi.connector.ColumnMetadata;
import io.trino.spi.connector.ColumnNotFoundException;
import io.trino.spi.connector.ConnectorAnalyzeMetadata;
import io.trino.spi.connector.ConnectorInsertTableHandle;
import io.trino.spi.connector.ConnectorMergeTableHandle;
import io.trino.spi.connector.ConnectorMetadata;
import io.trino.spi.connector.ConnectorOutputMetadata;
import io.trino.spi.connector.ConnectorOutputTableHandle;
import io.trino.spi.connector.ConnectorPartitioningHandle;
import io.trino.spi.connector.ConnectorSession;
import io.trino.spi.connector.ConnectorTableExecuteHandle;
import io.trino.spi.connector.ConnectorTableHandle;
import io.trino.spi.connector.ConnectorTableLayout;
import io.trino.spi.connector.ConnectorTableMetadata;
import io.trino.spi.connector.ConnectorTableProperties;
import io.trino.spi.connector.ConnectorViewDefinition;
import io.trino.spi.connector.Constraint;
import io.trino.spi.connector.ConstraintApplicationResult;
import io.trino.spi.connector.ProjectionApplicationResult;
import io.trino.spi.connector.RetryMode;
import io.trino.spi.connector.RowChangeParadigm;
import io.trino.spi.connector.SchemaNotFoundException;
import io.trino.spi.connector.SchemaTableName;
import io.trino.spi.connector.SchemaTablePrefix;
import io.trino.spi.connector.SystemTable;
import io.trino.spi.connector.TableColumnsMetadata;
import io.trino.spi.connector.TableNotFoundException;
import io.trino.spi.connector.TableScanRedirectApplicationResult;
import io.trino.spi.connector.ViewNotFoundException;
import io.trino.spi.connector.WriterScalingOptions;
import io.trino.spi.expression.ConnectorExpression;
import io.trino.spi.expression.Variable;
import io.trino.spi.predicate.Domain;
import io.trino.spi.predicate.TupleDomain;
import io.trino.spi.security.GrantInfo;
import io.trino.spi.security.Privilege;
import io.trino.spi.security.RoleGrant;
import io.trino.spi.security.TrinoPrincipal;
import io.trino.spi.statistics.ColumnStatisticMetadata;
import io.trino.spi.statistics.ColumnStatisticType;
import io.trino.spi.statistics.ComputedStatistics;
import io.trino.spi.statistics.TableStatisticType;
import io.trino.spi.statistics.TableStatistics;
import io.trino.spi.statistics.TableStatisticsMetadata;
import io.trino.spi.type.ArrayType;
import io.trino.spi.type.DecimalType;
import io.trino.spi.type.FixedWidthType;
import io.trino.spi.type.HyperLogLogType;
import io.trino.spi.type.MapType;
import io.trino.spi.type.RowType;
import io.trino.spi.type.TimestampType;
import io.trino.spi.type.TimestampWithTimeZoneType;
import io.trino.spi.type.Type;
import io.trino.spi.type.TypeManager;
import io.trino.spi.type.VarcharType;
import jakarta.annotation.Nullable;

import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.time.Instant;
import java.util.ArrayDeque;
import java.util.Collection;
import java.util.Collections;
import java.util.Deque;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.OptionalLong;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import static com.google.common.base.MoreObjects.firstNonNull;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Verify.verify;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
import static com.google.common.collect.Maps.filterKeys;
import static com.google.common.collect.Sets.difference;
import static com.google.common.primitives.Ints.max;
import static io.trino.filesystem.Locations.appendPath;
import static io.trino.plugin.base.filter.UtcConstraintExtractor.extractTupleDomain;
import static io.trino.plugin.base.projection.ApplyProjectionUtil.ProjectedColumnRepresentation;
import static io.trino.plugin.base.projection.ApplyProjectionUtil.extractSupportedProjectedColumns;
import static io.trino.plugin.base.projection.ApplyProjectionUtil.replaceWithNewVariables;
import static io.trino.plugin.deltalake.DataFileInfo.DataFileType.DATA;
import static io.trino.plugin.deltalake.DeltaLakeAnalyzeProperties.AnalyzeMode.FULL_REFRESH;
import static io.trino.plugin.deltalake.DeltaLakeAnalyzeProperties.AnalyzeMode.INCREMENTAL;
import static io.trino.plugin.deltalake.DeltaLakeAnalyzeProperties.getColumnNames;
import static io.trino.plugin.deltalake.DeltaLakeAnalyzeProperties.getFilesModifiedAfterProperty;
import static io.trino.plugin.deltalake.DeltaLakeAnalyzeProperties.getRefreshMode;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.FILE_MODIFIED_TIME_COLUMN_NAME;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.PATH_COLUMN_NAME;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.fileModifiedTimeColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.fileSizeColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.mergeRowIdColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeColumnHandle.pathColumnHandle;
import static io.trino.plugin.deltalake.DeltaLakeColumnType.PARTITION_KEY;
import static io.trino.plugin.deltalake.DeltaLakeColumnType.REGULAR;
import static io.trino.plugin.deltalake.DeltaLakeColumnType.SYNTHESIZED;
import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_BAD_WRITE;
import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_FILESYSTEM_ERROR;
import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_INVALID_SCHEMA;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getHiveCatalogName;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isCollectExtendedStatisticsColumnStatisticsOnWrite;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isExtendedStatisticsEnabled;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isProjectionPushdownEnabled;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isQueryPartitionFilterRequired;
import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isTableStatisticsEnabled;
import static io.trino.plugin.deltalake.DeltaLakeSplitManager.partitionMatchesPredicate;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.CHANGE_DATA_FEED_ENABLED_PROPERTY;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.CHECKPOINT_INTERVAL_PROPERTY;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.COLUMN_MAPPING_MODE_PROPERTY;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.LOCATION_PROPERTY;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.PARTITIONED_BY_PROPERTY;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.getChangeDataFeedEnabled;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.getCheckpointInterval;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.getLocation;
import static io.trino.plugin.deltalake.DeltaLakeTableProperties.getPartitionedBy;
import static io.trino.plugin.deltalake.metastore.HiveMetastoreBackedDeltaLakeMetastore.TABLE_PROVIDER_PROPERTY;
import static io.trino.plugin.deltalake.metastore.HiveMetastoreBackedDeltaLakeMetastore.TABLE_PROVIDER_VALUE;
import static io.trino.plugin.deltalake.procedure.DeltaLakeTableProcedureId.OPTIMIZE;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.APPEND_ONLY_CONFIGURATION_KEY;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.COLUMN_MAPPING_PHYSICAL_NAME_CONFIGURATION_KEY;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.ColumnMappingMode.ID;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.ColumnMappingMode.NAME;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.ColumnMappingMode.NONE;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.MAX_COLUMN_ID_CONFIGURATION_KEY;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.TIMESTAMP_NTZ_FEATURE_NAME;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.changeDataFeedEnabled;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.deserializeType;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.extractPartitionColumns;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.extractSchema;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.generateColumnMetadata;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getCheckConstraints;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getColumnComments;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getColumnIdentities;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getColumnInvariants;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getColumnMappingMode;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getColumnTypes;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getColumnsMetadata;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getColumnsNullability;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getExactColumnNames;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getGeneratedColumnExpressions;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getMaxColumnId;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.isAppendOnly;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.serializeColumnType;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.serializeSchemaAsJson;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.serializeStatsAsJson;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.unsupportedReaderFeatures;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.unsupportedWriterFeatures;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.validateType;
import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.verifySupportedColumnMapping;
import static io.trino.plugin.deltalake.transactionlog.MetadataEntry.DELTA_CHANGE_DATA_FEED_ENABLED_PROPERTY;
import static io.trino.plugin.deltalake.transactionlog.MetadataEntry.configurationForNewTable;
import static io.trino.plugin.deltalake.transactionlog.TransactionLogParser.getMandatoryCurrentVersion;
import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogDir;
import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.METADATA;
import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.PROTOCOL;
import static io.trino.plugin.hive.HiveMetadata.TRINO_QUERY_ID_NAME;
import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE;
import static io.trino.plugin.hive.TableType.MANAGED_TABLE;
import static io.trino.plugin.hive.TableType.VIRTUAL_VIEW;
import static io.trino.plugin.hive.metastore.MetastoreUtil.buildInitialPrivilegeSet;
import static io.trino.plugin.hive.metastore.StorageFormat.create;
import static io.trino.plugin.hive.util.HiveClassNames.HIVE_SEQUENCEFILE_OUTPUT_FORMAT_CLASS;
import static io.trino.plugin.hive.util.HiveClassNames.LAZY_SIMPLE_SERDE_CLASS;
import static io.trino.plugin.hive.util.HiveClassNames.SEQUENCEFILE_INPUT_FORMAT_CLASS;
import static io.trino.plugin.hive.util.HiveUtil.escapeTableName;
import static io.trino.plugin.hive.util.HiveUtil.isDeltaLakeTable;
import static io.trino.plugin.hive.util.HiveUtil.isHiveSystemSchema;
import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static io.trino.spi.StandardErrorCode.INVALID_ANALYZE_PROPERTY;
import static io.trino.spi.StandardErrorCode.INVALID_SCHEMA_PROPERTY;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static io.trino.spi.StandardErrorCode.QUERY_REJECTED;
import static io.trino.spi.connector.RetryMode.NO_RETRIES;
import static io.trino.spi.connector.RowChangeParadigm.DELETE_ROW_AND_INSERT_ROW;
import static io.trino.spi.connector.SchemaTableName.schemaTableName;
import static io.trino.spi.predicate.Range.greaterThanOrEqual;
import static io.trino.spi.predicate.Range.lessThanOrEqual;
import static io.trino.spi.predicate.Range.range;
import static io.trino.spi.predicate.TupleDomain.withColumnDomains;
import static io.trino.spi.predicate.Utils.blockToNativeValue;
import static io.trino.spi.predicate.ValueSet.ofRanges;
import static io.trino.spi.statistics.ColumnStatisticType.MAX_VALUE;
import static io.trino.spi.statistics.ColumnStatisticType.MIN_VALUE;
import static io.trino.spi.statistics.ColumnStatisticType.NUMBER_OF_DISTINCT_VALUES_SUMMARY;
import static io.trino.spi.statistics.ColumnStatisticType.NUMBER_OF_NON_NULL_VALUES;
import static io.trino.spi.statistics.ColumnStatisticType.TOTAL_SIZE_IN_BYTES;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.BooleanType.BOOLEAN;
import static io.trino.spi.type.DateTimeEncoding.unpackMillisUtc;
import static io.trino.spi.type.DateType.DATE;
import static io.trino.spi.type.DoubleType.DOUBLE;
import static io.trino.spi.type.IntegerType.INTEGER;
import static io.trino.spi.type.RealType.REAL;
import static io.trino.spi.type.SmallintType.SMALLINT;
import static io.trino.spi.type.TinyintType.TINYINT;
import static io.trino.spi.type.TypeUtils.isFloatingPointNaN;
import static io.trino.spi.type.VarbinaryType.VARBINARY;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.time.Instant.EPOCH;
import static java.util.Collections.singletonList;
import static java.util.Collections.unmodifiableMap;
import static java.util.Comparator.naturalOrder;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
import static java.util.UUID.randomUUID;
import static java.util.function.Function.identity;
import static java.util.function.Predicate.not;
import static java.util.stream.Collectors.partitioningBy;

public class DeltaLakeMetadata
        implements ConnectorMetadata
{
    public static final Logger LOG = Logger.get(DeltaLakeMetadata.class);

    public static final String PATH_PROPERTY = "path";
    public static final StorageFormat DELTA_STORAGE_FORMAT = create(
            LAZY_SIMPLE_SERDE_CLASS,
            SEQUENCEFILE_INPUT_FORMAT_CLASS,
            HIVE_SEQUENCEFILE_OUTPUT_FORMAT_CLASS);
    public static final String CREATE_TABLE_AS_OPERATION = "CREATE TABLE AS SELECT";
    public static final String CREATE_TABLE_OPERATION = "CREATE TABLE";
    public static final String ADD_COLUMN_OPERATION = "ADD COLUMNS";
    public static final String DROP_COLUMN_OPERATION = "DROP COLUMNS";
    public static final String RENAME_COLUMN_OPERATION = "RENAME COLUMN";
    public static final String INSERT_OPERATION = "WRITE";
    public static final String MERGE_OPERATION = "MERGE";
    public static final String UPDATE_OPERATION = "UPDATE"; // used by old Trino versions and Spark
    public static final String DELETE_OPERATION = "DELETE"; // used Trino for whole table/partition deletes as well as Spark
    public static final String TRUNCATE_OPERATION = "TRUNCATE";
    public static final String OPTIMIZE_OPERATION = "OPTIMIZE";
    public static final String SET_TBLPROPERTIES_OPERATION = "SET TBLPROPERTIES";
    public static final String CHANGE_COLUMN_OPERATION = "CHANGE COLUMN";
    public static final String ISOLATION_LEVEL = "WriteSerializable";

    public static final int DEFAULT_READER_VERSION = 1;
    public static final int DEFAULT_WRITER_VERSION = 2;
    // The highest reader and writer versions Trino supports
    private static final int MAX_READER_VERSION = 3;
    public static final int MAX_WRITER_VERSION = 7;
    private static final int CDF_SUPPORTED_WRITER_VERSION = 4;
    private static final int COLUMN_MAPPING_MODE_SUPPORTED_READER_VERSION = 2;
    private static final int COLUMN_MAPPING_MODE_SUPPORTED_WRITER_VERSION = 5;
    private static final int TIMESTAMP_NTZ_SUPPORTED_READER_VERSION = 3;
    private static final int TIMESTAMP_NTZ_SUPPORTED_WRITER_VERSION = 7;

    // Matches the dummy column Databricks stores in the metastore
    private static final List DUMMY_DATA_COLUMNS = ImmutableList.of(
            new Column("col", HiveType.toHiveType(new ArrayType(VarcharType.createUnboundedVarcharType())), Optional.empty(), Map.of()));
    private static final Set SUPPORTED_STATISTICS_TYPE = ImmutableSet.builder()
            .add(TOTAL_SIZE_IN_BYTES)
            .add(NUMBER_OF_DISTINCT_VALUES_SUMMARY)
            .add(MAX_VALUE)
            .add(MIN_VALUE)
            .add(NUMBER_OF_NON_NULL_VALUES)
            .build();
    private static final String ENABLE_NON_CONCURRENT_WRITES_CONFIGURATION_KEY = "delta.enable-non-concurrent-writes";
    public static final Set UPDATABLE_TABLE_PROPERTIES = ImmutableSet.of(CHANGE_DATA_FEED_ENABLED_PROPERTY);

    public static final Set CHANGE_DATA_FEED_COLUMN_NAMES = ImmutableSet.builder()
            .add("_change_type")
            .add("_commit_version")
            .add("_commit_timestamp")
            .build();

    private static final String CHECK_CONSTRAINT_CONVERT_FAIL_EXPRESSION = "CAST(fail('Failed to convert Delta check constraints to Trino expression') AS boolean)";

    private final DeltaLakeMetastore metastore;
    private final TransactionLogAccess transactionLogAccess;
    private final DeltaLakeTableStatisticsProvider tableStatisticsProvider;
    private final TrinoFileSystemFactory fileSystemFactory;
    private final TypeManager typeManager;
    private final AccessControlMetadata accessControlMetadata;
    private final TrinoViewHiveMetastore trinoViewHiveMetastore;
    private final CheckpointWriterManager checkpointWriterManager;
    private final long defaultCheckpointInterval;
    private final int domainCompactionThreshold;
    private final boolean unsafeWritesEnabled;
    private final JsonCodec dataFileInfoCodec;
    private final JsonCodec mergeResultJsonCodec;
    private final TransactionLogWriterFactory transactionLogWriterFactory;
    private final String nodeVersion;
    private final String nodeId;
    private final AtomicReference rollbackAction = new AtomicReference<>();
    private final DeltaLakeRedirectionsProvider deltaLakeRedirectionsProvider;
    private final CachingExtendedStatisticsAccess statisticsAccess;
    private final boolean deleteSchemaLocationsFallback;
    private final boolean useUniqueTableLocation;
    private final boolean allowManagedTableRename;
    private final Map queriedVersions = new ConcurrentHashMap<>();
    private final Map queriedSnapshots = new ConcurrentHashMap<>();

    private record QueriedTable(SchemaTableName schemaTableName, long version)
    {
        QueriedTable
        {
            requireNonNull(schemaTableName, "schemaTableName is null");
        }
    }

    public DeltaLakeMetadata(
            DeltaLakeMetastore metastore,
            TransactionLogAccess transactionLogAccess,
            DeltaLakeTableStatisticsProvider tableStatisticsProvider,
            TrinoFileSystemFactory fileSystemFactory,
            TypeManager typeManager,
            AccessControlMetadata accessControlMetadata,
            TrinoViewHiveMetastore trinoViewHiveMetastore,
            int domainCompactionThreshold,
            boolean unsafeWritesEnabled,
            JsonCodec dataFileInfoCodec,
            JsonCodec mergeResultJsonCodec,
            TransactionLogWriterFactory transactionLogWriterFactory,
            NodeManager nodeManager,
            CheckpointWriterManager checkpointWriterManager,
            long defaultCheckpointInterval,
            boolean deleteSchemaLocationsFallback,
            DeltaLakeRedirectionsProvider deltaLakeRedirectionsProvider,
            CachingExtendedStatisticsAccess statisticsAccess,
            boolean useUniqueTableLocation,
            boolean allowManagedTableRename)
    {
        this.metastore = requireNonNull(metastore, "metastore is null");
        this.transactionLogAccess = requireNonNull(transactionLogAccess, "transactionLogAccess is null");
        this.tableStatisticsProvider = requireNonNull(tableStatisticsProvider, "tableStatisticsProvider is null");
        this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null");
        this.typeManager = requireNonNull(typeManager, "typeManager is null");
        this.accessControlMetadata = requireNonNull(accessControlMetadata, "accessControlMetadata is null");
        this.trinoViewHiveMetastore = requireNonNull(trinoViewHiveMetastore, "trinoViewHiveMetastore is null");
        this.domainCompactionThreshold = domainCompactionThreshold;
        this.unsafeWritesEnabled = unsafeWritesEnabled;
        this.dataFileInfoCodec = requireNonNull(dataFileInfoCodec, "dataFileInfoCodec is null");
        this.mergeResultJsonCodec = requireNonNull(mergeResultJsonCodec, "mergeResultJsonCodec is null");
        this.transactionLogWriterFactory = requireNonNull(transactionLogWriterFactory, "transactionLogWriterFactory is null");
        this.nodeVersion = nodeManager.getCurrentNode().getVersion();
        this.nodeId = nodeManager.getCurrentNode().getNodeIdentifier();
        this.checkpointWriterManager = requireNonNull(checkpointWriterManager, "checkpointWriterManager is null");
        this.defaultCheckpointInterval = defaultCheckpointInterval;
        this.deltaLakeRedirectionsProvider = requireNonNull(deltaLakeRedirectionsProvider, "deltaLakeRedirectionsProvider is null");
        this.statisticsAccess = requireNonNull(statisticsAccess, "statisticsAccess is null");
        this.deleteSchemaLocationsFallback = deleteSchemaLocationsFallback;
        this.useUniqueTableLocation = useUniqueTableLocation;
        this.allowManagedTableRename = allowManagedTableRename;
    }

    public TableSnapshot getSnapshot(ConnectorSession session, SchemaTableName table, String tableLocation, long atVersion)
    {
        return getSnapshot(session, table, tableLocation, Optional.of(atVersion));
    }

    @VisibleForTesting
    protected TableSnapshot getSnapshot(ConnectorSession session, SchemaTableName table, String tableLocation, Optional atVersion)
    {
        try {
            if (atVersion.isEmpty()) {
                atVersion = Optional.ofNullable(queriedVersions.get(table));
            }
            if (atVersion.isPresent()) {
                long version = atVersion.get();
                TableSnapshot snapshot = queriedSnapshots.get(new QueriedTable(table, version));
                checkState(snapshot != null, "No previously loaded snapshot found for query %s, table %s [%s] at version %s", session.getQueryId(), table, tableLocation, version);
                return snapshot;
            }

            TableSnapshot snapshot = transactionLogAccess.loadSnapshot(session, table, tableLocation);
            // Lack of concurrency for given query is currently guaranteed by DeltaLakeMetadata
            checkState(queriedVersions.put(table, snapshot.getVersion()) == null, "queriedLocations changed concurrently for %s", table);
            queriedSnapshots.put(new QueriedTable(table, snapshot.getVersion()), snapshot);
            return snapshot;
        }
        catch (IOException | RuntimeException e) {
            throw new TrinoException(DELTA_LAKE_INVALID_SCHEMA, "Error getting snapshot for " + table, e);
        }
    }

    @Override
    public List listSchemaNames(ConnectorSession session)
    {
        return metastore.getAllDatabases().stream()
                .filter(schema -> !schema.equalsIgnoreCase("sys"))
                .collect(toImmutableList());
    }

    private static boolean isHiveTable(Table table)
    {
        return !isDeltaLakeTable(table);
    }

    @Override
    public Optional redirectTable(ConnectorSession session, SchemaTableName tableName)
    {
        requireNonNull(session, "session is null");
        requireNonNull(tableName, "tableName is null");
        Optional targetCatalogName = getHiveCatalogName(session);
        if (targetCatalogName.isEmpty()) {
            return Optional.empty();
        }
        if (isHiveSystemSchema(tableName.getSchemaName())) {
            return Optional.empty();
        }

        // we need to chop off any "$partitions" and similar suffixes from table name while querying the metastore for the Table object
        int metadataMarkerIndex = tableName.getTableName().lastIndexOf('$');
        SchemaTableName tableNameBase = (metadataMarkerIndex == -1) ? tableName : schemaTableName(
                tableName.getSchemaName(),
                tableName.getTableName().substring(0, metadataMarkerIndex));

        Optional table = metastore.getRawMetastoreTable(tableNameBase.getSchemaName(), tableNameBase.getTableName());
        if (table.isEmpty() || VIRTUAL_VIEW.name().equals(table.get().getTableType())) {
            return Optional.empty();
        }
        if (isHiveTable(table.get())) {
            // After redirecting, use the original table name, with "$partitions" and similar suffixes
            return targetCatalogName.map(catalog -> new CatalogSchemaTableName(catalog, tableName));
        }
        return Optional.empty();
    }

    @Override
    public LocatedTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName)
    {
        requireNonNull(tableName, "tableName is null");
        if (!DeltaLakeTableName.isDataTable(tableName.getTableName())) {
            // Pretend the table does not exist to produce better error message in case of table redirects to Hive
            return null;
        }
        Optional table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName());
        if (table.isEmpty()) {
            return null;
        }
        boolean managed = table.get().managed();

        String tableLocation = table.get().location();
        TableSnapshot tableSnapshot = getSnapshot(session, tableName, tableLocation, Optional.empty());
        Map, Object> logEntries;
        try {
            logEntries = transactionLogAccess.getTransactionLogEntries(
                    session,
                    tableSnapshot,
                    ImmutableSet.of(METADATA, PROTOCOL),
                    entryStream -> entryStream
                            .filter(entry -> entry.getMetaData() != null || entry.getProtocol() != null)
                            .map(entry -> firstNonNull(entry.getMetaData(), entry.getProtocol())));
        }
        catch (TrinoException e) {
            if (e.getErrorCode().equals(DELTA_LAKE_INVALID_SCHEMA.toErrorCode())) {
                return new CorruptedDeltaLakeTableHandle(tableName, managed, tableLocation, e);
            }
            throw e;
        }
        MetadataEntry metadataEntry = (MetadataEntry) logEntries.get(MetadataEntry.class);
        if (metadataEntry == null) {
            return new CorruptedDeltaLakeTableHandle(tableName, managed, tableLocation, new TrinoException(DELTA_LAKE_INVALID_SCHEMA, "Metadata not found in transaction log for " + tableSnapshot.getTable()));
        }
        ProtocolEntry protocolEntry = (ProtocolEntry) logEntries.get(ProtocolEntry.class);
        if (protocolEntry == null) {
            return new CorruptedDeltaLakeTableHandle(tableName, managed, tableLocation, new TrinoException(DELTA_LAKE_INVALID_SCHEMA, "Protocol not found in transaction log for " + tableSnapshot.getTable()));
        }
        if (protocolEntry.getMinReaderVersion() > MAX_READER_VERSION) {
            LOG.debug("Skip %s because the reader version is unsupported: %d", tableName, protocolEntry.getMinReaderVersion());
            return null;
        }
        Set unsupportedReaderFeatures = unsupportedReaderFeatures(protocolEntry.getReaderFeatures().orElse(ImmutableSet.of()));
        if (!unsupportedReaderFeatures.isEmpty()) {
            LOG.debug("Skip %s because the table contains unsupported reader features: %s", tableName, unsupportedReaderFeatures);
            return null;
        }
        verifySupportedColumnMapping(getColumnMappingMode(metadataEntry, protocolEntry));
        return new DeltaLakeTableHandle(
                tableName.getSchemaName(),
                tableName.getTableName(),
                managed,
                tableLocation,
                metadataEntry,
                protocolEntry,
                TupleDomain.all(),
                TupleDomain.all(),
                Optional.empty(),
                Optional.empty(),
                Optional.empty(),
                Optional.empty(),
                Optional.empty(),
                tableSnapshot.getVersion());
    }

    @Override
    public ConnectorTableProperties getTableProperties(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        return new ConnectorTableProperties(
                ((DeltaLakeTableHandle) tableHandle).getEnforcedPartitionConstraint()
                        .transformKeys(ColumnHandle.class::cast),
                Optional.empty(),
                Optional.empty(),
                ImmutableList.of());
    }

    @Override
    public SchemaTableName getTableName(ConnectorSession session, ConnectorTableHandle table)
    {
        if (table instanceof CorruptedDeltaLakeTableHandle corruptedTableHandle) {
            return corruptedTableHandle.schemaTableName();
        }
        return ((DeltaLakeTableHandle) table).getSchemaTableName();
    }

    @Override
    public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table)
    {
        DeltaLakeTableHandle tableHandle = checkValidTableHandle(table);
        // This method does not calculate column metadata for the projected columns
        checkArgument(tableHandle.getProjectedColumns().isEmpty(), "Unexpected projected columns");
        MetadataEntry metadataEntry = tableHandle.getMetadataEntry();
        ProtocolEntry protocolEntry = tableHandle.getProtocolEntry();

        List constraints = ImmutableList.builder()
                .addAll(getCheckConstraints(metadataEntry, protocolEntry).values())
                .addAll(getColumnInvariants(metadataEntry, protocolEntry).values()) // The internal logic for column invariants in Delta Lake is same as check constraints
                .build();
        List columns = getTableColumnMetadata(metadataEntry, protocolEntry);

        ImmutableMap.Builder properties = ImmutableMap.builder()
                .put(LOCATION_PROPERTY, tableHandle.getLocation());
        List partitionColumnNames = metadataEntry.getLowercasePartitionColumns();
        if (!partitionColumnNames.isEmpty()) {
            properties.put(PARTITIONED_BY_PROPERTY, partitionColumnNames);
        }

        Optional checkpointInterval = metadataEntry.getCheckpointInterval();
        checkpointInterval.ifPresent(value -> properties.put(CHECKPOINT_INTERVAL_PROPERTY, value));

        changeDataFeedEnabled(metadataEntry, protocolEntry)
                .ifPresent(value -> properties.put(CHANGE_DATA_FEED_ENABLED_PROPERTY, value));

        ColumnMappingMode columnMappingMode = DeltaLakeSchemaSupport.getColumnMappingMode(metadataEntry, protocolEntry);
        if (columnMappingMode != NONE) {
            properties.put(COLUMN_MAPPING_MODE_PROPERTY, columnMappingMode.name());
        }

        return new ConnectorTableMetadata(
                tableHandle.getSchemaTableName(),
                columns,
                properties.buildOrThrow(),
                Optional.ofNullable(metadataEntry.getDescription()),
                constraints.stream()
                        .map(constraint -> {
                            try {
                                return SparkExpressionParser.toTrinoExpression(constraint);
                            }
                            catch (ParsingException e) {
                                return CHECK_CONSTRAINT_CONVERT_FAIL_EXPRESSION;
                            }
                        })
                        .collect(toImmutableList()));
    }

    private List getTableColumnMetadata(MetadataEntry metadataEntry, ProtocolEntry protocolEntry)
    {
        Map columnComments = getColumnComments(metadataEntry);
        Map columnsNullability = getColumnsNullability(metadataEntry);
        Map columnGenerations = getGeneratedColumnExpressions(metadataEntry);
        List columns = getColumns(metadataEntry, protocolEntry).stream()
                .map(column -> getColumnMetadata(column, columnComments.get(column.getBaseColumnName()), columnsNullability.getOrDefault(column.getBaseColumnName(), true), columnGenerations.get(column.getBaseColumnName())))
                .collect(toImmutableList());
        return columns;
    }

    @Override
    public List listTables(ConnectorSession session, Optional schemaName)
    {
        return schemaName.map(Collections::singletonList)
                .orElseGet(() -> listSchemaNames(session))
                .stream()
                .flatMap(schema -> metastore.getAllTables(schema).stream()
                        .map(table -> new SchemaTableName(schema, table)))
                .collect(toImmutableList());
    }

    @Override
    public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        DeltaLakeTableHandle table = checkValidTableHandle(tableHandle);
        return table.getProjectedColumns()
                .map(projectColumns -> (Collection) projectColumns)
                .orElseGet(() -> getColumns(table.getMetadataEntry(), table.getProtocolEntry())).stream()
                // This method does not calculate column name for the projected columns
                .peek(handle -> checkArgument(handle.isBaseColumn(), "Unsupported projected column: %s", handle))
                .collect(toImmutableMap(DeltaLakeColumnHandle::getBaseColumnName, identity()));
    }

    @Override
    public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle)
    {
        DeltaLakeTableHandle table = (DeltaLakeTableHandle) tableHandle;
        DeltaLakeColumnHandle column = (DeltaLakeColumnHandle) columnHandle;
        if (column.getProjectionInfo().isPresent()) {
            return getColumnMetadata(column, null, true, null);
        }
        return getColumnMetadata(
                column,
                getColumnComments(table.getMetadataEntry()).get(column.getBaseColumnName()),
                getColumnsNullability(table.getMetadataEntry()).getOrDefault(column.getBaseColumnName(), true),
                getGeneratedColumnExpressions(table.getMetadataEntry()).get(column.getBaseColumnName()));
    }

    /**
     * Provides partitioning scheme of table for query planner to decide how to
     * write to multiple partitions.
     */
    @Override
    public Optional getNewTableLayout(ConnectorSession session, ConnectorTableMetadata tableMetadata)
    {
        validateTableColumns(tableMetadata);

        List partitionColumnNames = getPartitionedBy(tableMetadata.getProperties());

        if (partitionColumnNames.isEmpty()) {
            return Optional.empty();
        }

        return Optional.of(new ConnectorTableLayout(partitionColumnNames));
    }

    @Override
    public Optional getInsertLayout(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        DeltaLakeTableHandle deltaLakeTableHandle = (DeltaLakeTableHandle) tableHandle;
        List partitionColumnNames = deltaLakeTableHandle.getMetadataEntry().getLowercasePartitionColumns();

        if (partitionColumnNames.isEmpty()) {
            return Optional.empty();
        }

        return Optional.of(new ConnectorTableLayout(partitionColumnNames));
    }

    @Override
    public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix)
    {
        throw new UnsupportedOperationException("The deprecated listTableColumns is not supported because streamTableColumns is implemented instead");
    }

    @Override
    public Iterator streamTableColumns(ConnectorSession session, SchemaTablePrefix prefix)
    {
        List tables = prefix.getTable()
                .map(ignored -> singletonList(prefix.toSchemaTableName()))
                .orElseGet(() -> listTables(session, prefix.getSchema()));

        return tables.stream()
                .flatMap(table -> {
                    try {
                        if (redirectTable(session, table).isPresent()) {
                            // put "redirect marker" for current table
                            return Stream.of(TableColumnsMetadata.forRedirectedTable(table));
                        }

                        Optional metastoreTable = metastore.getTable(table.getSchemaName(), table.getTableName());
                        if (metastoreTable.isEmpty()) {
                            // this may happen when table is being deleted concurrently,
                            return Stream.of();
                        }
                        String tableLocation = metastoreTable.get().location();
                        TableSnapshot snapshot = transactionLogAccess.loadSnapshot(session, table, tableLocation);
                        MetadataEntry metadata = transactionLogAccess.getMetadataEntry(snapshot, session);
                        ProtocolEntry protocol = transactionLogAccess.getProtocolEntry(session, snapshot);
                        Map columnComments = getColumnComments(metadata);
                        Map columnsNullability = getColumnsNullability(metadata);
                        Map columnGenerations = getGeneratedColumnExpressions(metadata);
                        List columnMetadata = getColumns(metadata, protocol).stream()
                                .map(column -> getColumnMetadata(column, columnComments.get(column.getColumnName()), columnsNullability.getOrDefault(column.getBaseColumnName(), true), columnGenerations.get(column.getBaseColumnName())))
                                .collect(toImmutableList());
                        return Stream.of(TableColumnsMetadata.forTable(table, columnMetadata));
                    }
                    catch (NotADeltaLakeTableException | IOException e) {
                        return Stream.empty();
                    }
                    catch (RuntimeException e) {
                        // this may happen when table is being deleted concurrently, it still exists in metastore but TL is no longer present
                        // there can be several different exceptions thrown this is why all RTE are caught and ignored here
                        LOG.debug(e, "Ignored exception when trying to list columns from %s", table);
                        return Stream.empty();
                    }
                })
                .iterator();
    }

    private List getColumns(MetadataEntry deltaMetadata, ProtocolEntry protocolEntry)
    {
        ImmutableList.Builder columns = ImmutableList.builder();
        extractSchema(deltaMetadata, protocolEntry, typeManager).stream()
                .map(column -> toColumnHandle(column.getName(), column.getType(), column.getFieldId(), column.getPhysicalName(), column.getPhysicalColumnType(), deltaMetadata.getLowercasePartitionColumns()))
                .forEach(columns::add);
        columns.add(pathColumnHandle());
        columns.add(fileSizeColumnHandle());
        columns.add(fileModifiedTimeColumnHandle());
        return columns.build();
    }

    @Override
    public TableStatistics getTableStatistics(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        DeltaLakeTableHandle handle = checkValidTableHandle(tableHandle);
        if (!isTableStatisticsEnabled(session)) {
            return TableStatistics.empty();
        }
        return tableStatisticsProvider.getTableStatistics(session, handle, getSnapshot(session, handle));
    }

    @Override
    public void createSchema(ConnectorSession session, String schemaName, Map properties, TrinoPrincipal owner)
    {
        Optional location = DeltaLakeSchemaProperties.getLocation(properties).map(locationUri -> {
            try {
                fileSystemFactory.create(session).directoryExists(Location.of(locationUri));
            }
            catch (IOException | IllegalArgumentException e) {
                throw new TrinoException(INVALID_SCHEMA_PROPERTY, "Invalid location URI: " + locationUri, e);
            }
            return locationUri;
        });

        String queryId = session.getQueryId();

        Database database = Database.builder()
                .setDatabaseName(schemaName)
                .setLocation(location)
                .setOwnerType(Optional.of(owner.getType()))
                .setOwnerName(Optional.of(owner.getName()))
                .setParameters(ImmutableMap.of(TRINO_QUERY_ID_NAME, queryId))
                .build();

        // Ensure the database has queryId set. This is relied on for exception handling
        verify(
                getQueryId(database).orElseThrow(() -> new IllegalArgumentException("Query id is not present")).equals(queryId),
                "Database '%s' does not have correct query id set",
                database.getDatabaseName());

        metastore.createDatabase(database);
    }

    @Override
    public void dropSchema(ConnectorSession session, String schemaName, boolean cascade)
    {
        if (cascade) {
            for (SchemaTableName viewName : listViews(session, Optional.of(schemaName))) {
                try {
                    dropView(session, viewName);
                }
                catch (ViewNotFoundException e) {
                    LOG.debug("View disappeared during DROP SCHEMA CASCADE: %s", viewName);
                }
            }
            for (SchemaTableName tableName : listTables(session, Optional.of(schemaName))) {
                ConnectorTableHandle table = getTableHandle(session, tableName, Optional.empty(), Optional.empty());
                if (table == null) {
                    LOG.debug("Table disappeared during DROP SCHEMA CASCADE: %s", tableName);
                    continue;
                }
                try {
                    dropTable(session, table);
                }
                catch (TableNotFoundException e) {
                    LOG.debug("Table disappeared during DROP SCHEMA CASCADE: %s", tableName);
                }
            }
        }

        Optional location = metastore.getDatabase(schemaName)
                .orElseThrow(() -> new SchemaNotFoundException(schemaName))
                .getLocation();

        // If we see files in the schema location, don't delete it.
        // If we see no files or can't see the location at all, use fallback.
        boolean deleteData = location.map(path -> {
            try {
                return !fileSystemFactory.create(session).listFiles(Location.of(path)).hasNext();
            }
            catch (IOException | RuntimeException e) {
                LOG.warn(e, "Could not check schema directory '%s'", path);
                return deleteSchemaLocationsFallback;
            }
        }).orElse(deleteSchemaLocationsFallback);

        metastore.dropDatabase(schemaName, deleteData);
    }

    @Override
    public void createTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, boolean ignoreExisting)
    {
        SchemaTableName schemaTableName = tableMetadata.getTable();
        String schemaName = schemaTableName.getSchemaName();
        String tableName = schemaTableName.getTableName();

        Database schema = metastore.getDatabase(schemaName).orElseThrow(() -> new SchemaNotFoundException(schemaName));

        boolean external = true;
        String location = getLocation(tableMetadata.getProperties());
        if (location == null) {
            location = getTableLocation(schema, tableName);
            checkPathContainsNoFiles(session, Location.of(location));
            external = false;
        }
        Location deltaLogDirectory = Location.of(getTransactionLogDir(location));
        Optional checkpointInterval = getCheckpointInterval(tableMetadata.getProperties());
        Optional changeDataFeedEnabled = getChangeDataFeedEnabled(tableMetadata.getProperties());
        ColumnMappingMode columnMappingMode = DeltaLakeTableProperties.getColumnMappingMode(tableMetadata.getProperties());
        AtomicInteger fieldId = new AtomicInteger();

        try {
            TrinoFileSystem fileSystem = fileSystemFactory.create(session);
            if (!fileSystem.listFiles(deltaLogDirectory).hasNext()) {
                validateTableColumns(tableMetadata);

                List partitionColumns = getPartitionedBy(tableMetadata.getProperties());
                ImmutableList.Builder columnNames = ImmutableList.builderWithExpectedSize(tableMetadata.getColumns().size());
                ImmutableMap.Builder columnTypes = ImmutableMap.builderWithExpectedSize(tableMetadata.getColumns().size());
                ImmutableMap.Builder> columnsMetadata = ImmutableMap.builderWithExpectedSize(tableMetadata.getColumns().size());
                boolean containsTimestampType = false;
                for (ColumnMetadata column : tableMetadata.getColumns()) {
                    columnNames.add(column.getName());
                    columnTypes.put(column.getName(), serializeColumnType(columnMappingMode, fieldId, column.getType()));
                    columnsMetadata.put(column.getName(), generateColumnMetadata(columnMappingMode, fieldId));
                    if (!containsTimestampType) {
                        containsTimestampType = containsTimestampType(column.getType());
                    }
                }
                Map columnComments = tableMetadata.getColumns().stream()
                        .filter(column -> column.getComment() != null)
                        .collect(toImmutableMap(ColumnMetadata::getName, ColumnMetadata::getComment));
                Map columnsNullability = tableMetadata.getColumns().stream()
                        .collect(toImmutableMap(ColumnMetadata::getName, ColumnMetadata::isNullable));
                OptionalInt maxFieldId = OptionalInt.empty();
                if (columnMappingMode == ID || columnMappingMode == NAME) {
                    maxFieldId = OptionalInt.of(fieldId.get());
                }

                TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriterWithoutTransactionIsolation(session, location);
                appendTableEntries(
                        0,
                        transactionLogWriter,
                        randomUUID().toString(),
                        columnNames.build(),
                        partitionColumns,
                        columnTypes.buildOrThrow(),
                        columnComments,
                        columnsNullability,
                        columnsMetadata.buildOrThrow(),
                        configurationForNewTable(checkpointInterval, changeDataFeedEnabled, columnMappingMode, maxFieldId),
                        CREATE_TABLE_OPERATION,
                        session,
                        tableMetadata.getComment(),
                        protocolEntryForNewTable(containsTimestampType, tableMetadata.getProperties()));

                setRollback(() -> deleteRecursivelyIfExists(fileSystem, deltaLogDirectory));
                transactionLogWriter.flush();
            }
            else {
                throw new TrinoException(
                        NOT_SUPPORTED,
                        "Using CREATE TABLE with an existing table content is disallowed, instead use the system.register_table() procedure.");
            }
        }
        catch (IOException e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, "Unable to access file system for: " + location, e);
        }

        Table table = buildTable(session, schemaTableName, location, external);

        // Ensure the table has queryId set. This is relied on for exception handling
        String queryId = session.getQueryId();
        verify(
                getQueryId(table).orElseThrow(() -> new IllegalArgumentException("Query id is not present")).equals(queryId),
                "Table '%s' does not have correct query id set",
                table);

        PrincipalPrivileges principalPrivileges = buildInitialPrivilegeSet(table.getOwner().orElseThrow());
        // As a precaution, clear the caches
        statisticsAccess.invalidateCache(schemaTableName, Optional.of(location));
        transactionLogAccess.invalidateCache(schemaTableName, Optional.of(location));
        metastore.createTable(session, table, principalPrivileges);
    }

    public static Table buildTable(ConnectorSession session, SchemaTableName schemaTableName, String location, boolean isExternal)
    {
        Table.Builder tableBuilder = Table.builder()
                .setDatabaseName(schemaTableName.getSchemaName())
                .setTableName(schemaTableName.getTableName())
                .setOwner(Optional.of(session.getUser()))
                .setTableType(isExternal ? EXTERNAL_TABLE.name() : MANAGED_TABLE.name())
                .setDataColumns(DUMMY_DATA_COLUMNS)
                .setParameters(deltaTableProperties(session, location, isExternal));

        setDeltaStorageFormat(tableBuilder, location);
        return tableBuilder.build();
    }

    private static Map deltaTableProperties(ConnectorSession session, String location, boolean external)
    {
        ImmutableMap.Builder properties = ImmutableMap.builder()
                .put(TRINO_QUERY_ID_NAME, session.getQueryId())
                .put(LOCATION_PROPERTY, location)
                .put(TABLE_PROVIDER_PROPERTY, TABLE_PROVIDER_VALUE)
                // Set bogus table stats to prevent Hive 3.x from gathering these stats at table creation.
                // These stats are not useful by themselves and can take a long time to collect when creating a
                // table over a large data set.
                .put("numFiles", "-1")
                .put("totalSize", "-1");

        if (external) {
            // Mimicking the behavior of the Hive connector which sets both `Table#setTableType` and the "EXTERNAL" table property
            properties.put("EXTERNAL", "TRUE");
        }
        return properties.buildOrThrow();
    }

    private static void setDeltaStorageFormat(Table.Builder tableBuilder, String location)
    {
        tableBuilder.getStorageBuilder()
                // this mimics what Databricks is doing when creating a Delta table in the Hive metastore
                .setStorageFormat(DELTA_STORAGE_FORMAT)
                .setSerdeParameters(ImmutableMap.of(PATH_PROPERTY, location))
                .setLocation(location);
    }

    @Override
    public DeltaLakeOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, Optional layout, RetryMode retryMode)
    {
        validateTableColumns(tableMetadata);

        SchemaTableName schemaTableName = tableMetadata.getTable();
        String schemaName = schemaTableName.getSchemaName();
        String tableName = schemaTableName.getTableName();

        Database schema = metastore.getDatabase(schemaName).orElseThrow(() -> new SchemaNotFoundException(schemaName));
        List partitionedBy = getPartitionedBy(tableMetadata.getProperties());

        boolean external = true;
        String location = getLocation(tableMetadata.getProperties());
        if (location == null) {
            location = getTableLocation(schema, tableName);
            external = false;
        }

        ColumnMappingMode columnMappingMode = DeltaLakeTableProperties.getColumnMappingMode(tableMetadata.getProperties());
        AtomicInteger fieldId = new AtomicInteger();

        Location finalLocation = Location.of(location);
        checkPathContainsNoFiles(session, finalLocation);
        setRollback(() -> deleteRecursivelyIfExists(fileSystemFactory.create(session), finalLocation));

        boolean usePhysicalName = columnMappingMode == ID || columnMappingMode == NAME;
        boolean containsTimestampType = false;
        int columnSize = tableMetadata.getColumns().size();
        ImmutableList.Builder columnNames = ImmutableList.builderWithExpectedSize(columnSize);
        ImmutableMap.Builder columnTypes = ImmutableMap.builderWithExpectedSize(columnSize);
        ImmutableMap.Builder columnNullabilities = ImmutableMap.builderWithExpectedSize(columnSize);
        ImmutableMap.Builder> columnsMetadata = ImmutableMap.builderWithExpectedSize(columnSize);
        ImmutableList.Builder columnHandles = ImmutableList.builderWithExpectedSize(columnSize);
        for (ColumnMetadata column : tableMetadata.getColumns()) {
            columnNames.add(column.getName());
            columnNullabilities.put(column.getName(), column.isNullable());
            containsTimestampType |= containsTimestampType(column.getType());

            Object serializedType = serializeColumnType(columnMappingMode, fieldId, column.getType());
            Type physicalType = deserializeType(typeManager, serializedType, usePhysicalName);
            columnTypes.put(column.getName(), serializedType);

            OptionalInt id;
            String physicalName;
            Map columnMetadata;
            switch (columnMappingMode) {
                case NONE -> {
                    id = OptionalInt.empty();
                    physicalName = column.getName();
                    columnMetadata = ImmutableMap.of();
                }
                case ID, NAME -> {
                    columnMetadata = generateColumnMetadata(columnMappingMode, fieldId);
                    id = OptionalInt.of(fieldId.get());
                    physicalName = (String) columnMetadata.get(COLUMN_MAPPING_PHYSICAL_NAME_CONFIGURATION_KEY);
                }
                default -> throw new IllegalArgumentException("Unexpected column mapping mode: " + columnMappingMode);
            }
            columnHandles.add(toColumnHandle(column.getName(), column.getType(), id, physicalName, physicalType, partitionedBy));
            columnsMetadata.put(column.getName(), columnMetadata);
        }

        String schemaString = serializeSchemaAsJson(
                columnNames.build(),
                columnTypes.buildOrThrow(),
                ImmutableMap.of(),
                columnNullabilities.buildOrThrow(),
                columnsMetadata.buildOrThrow());

        OptionalInt maxFieldId = OptionalInt.empty();
        if (columnMappingMode == ID || columnMappingMode == NAME) {
            maxFieldId = OptionalInt.of(fieldId.get());
        }

        return new DeltaLakeOutputTableHandle(
                schemaName,
                tableName,
                columnHandles.build(),
                location,
                getCheckpointInterval(tableMetadata.getProperties()),
                external,
                tableMetadata.getComment(),
                getChangeDataFeedEnabled(tableMetadata.getProperties()),
                schemaString,
                columnMappingMode,
                maxFieldId,
                protocolEntryForNewTable(containsTimestampType, tableMetadata.getProperties()));
    }

    private Optional getSchemaLocation(Database database)
    {
        Optional schemaLocation = database.getLocation();
        if (schemaLocation.isEmpty() || schemaLocation.get().isEmpty()) {
            return Optional.empty();
        }

        return schemaLocation;
    }

    private String getTableLocation(Database schema, String tableName)
    {
        String schemaLocation = getSchemaLocation(schema)
                .orElseThrow(() -> new TrinoException(NOT_SUPPORTED, "The 'location' property must be specified either for the table or the schema"));
        String tableNameLocationComponent = escapeTableName(tableName);
        if (useUniqueTableLocation) {
            tableNameLocationComponent += "-" + randomUUID().toString().replace("-", "");
        }
        return appendPath(schemaLocation, tableNameLocationComponent);
    }

    private void checkPathContainsNoFiles(ConnectorSession session, Location targetPath)
    {
        try {
            TrinoFileSystem fileSystem = fileSystemFactory.create(session);
            if (fileSystem.listFiles(targetPath).hasNext()) {
                throw new TrinoException(NOT_SUPPORTED, "Target location cannot contain any files: " + targetPath);
            }
        }
        catch (IOException e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, "Unable to access file system for: " + targetPath, e);
        }
    }

    private void validateTableColumns(ConnectorTableMetadata tableMetadata)
    {
        checkPartitionColumns(tableMetadata.getColumns(), getPartitionedBy(tableMetadata.getProperties()));
        checkColumnTypes(tableMetadata.getColumns());
        if (getChangeDataFeedEnabled(tableMetadata.getProperties()).orElse(false)) {
            Set conflicts = Sets.intersection(tableMetadata.getColumns().stream().map(ColumnMetadata::getName).collect(toImmutableSet()), CHANGE_DATA_FEED_COLUMN_NAMES);
            if (!conflicts.isEmpty()) {
                throw new TrinoException(NOT_SUPPORTED, "Unable to use %s when change data feed is enabled".formatted(conflicts));
            }
        }
    }

    private static void checkPartitionColumns(List columns, List partitionColumnNames)
    {
        Set columnNames = columns.stream()
                .map(ColumnMetadata::getName)
                .collect(toImmutableSet());
        List invalidPartitionNames = partitionColumnNames.stream()
                .filter(partitionColumnName -> !columnNames.contains(partitionColumnName))
                .collect(toImmutableList());

        if (columns.stream().filter(column -> partitionColumnNames.contains(column.getName()))
                .anyMatch(column -> column.getType() instanceof ArrayType || column.getType() instanceof MapType || column.getType() instanceof RowType)) {
            throw new TrinoException(DELTA_LAKE_INVALID_SCHEMA, "Using array, map or row type on partitioned columns is unsupported");
        }

        if (!invalidPartitionNames.isEmpty()) {
            throw new TrinoException(DELTA_LAKE_INVALID_SCHEMA, "Table property 'partition_by' contained column names which do not exist: " + invalidPartitionNames);
        }
        if (columns.size() == partitionColumnNames.size()) {
            throw new TrinoException(DELTA_LAKE_INVALID_SCHEMA, "Using all columns for partition columns is unsupported");
        }
    }

    private void checkColumnTypes(List columnMetadata)
    {
        for (ColumnMetadata column : columnMetadata) {
            Type type = column.getType();
            validateType(type);
        }
    }

    private static void deleteRecursivelyIfExists(TrinoFileSystem fileSystem, Location path)
    {
        try {
            fileSystem.deleteDirectory(path);
        }
        catch (IOException e) {
            LOG.warn(e, "IOException while trying to delete '%s'", path);
        }
    }

    private static boolean containsTimestampType(Type type)
    {
        if (type instanceof ArrayType arrayType) {
            return containsTimestampType(arrayType.getElementType());
        }
        if (type instanceof MapType mapType) {
            return containsTimestampType(mapType.getKeyType()) || containsTimestampType(mapType.getValueType());
        }
        if (type instanceof RowType rowType) {
            return rowType.getFields().stream().anyMatch(field -> containsTimestampType(field.getType()));
        }
        checkArgument(type.getTypeParameters().isEmpty(), "Unexpected type parameters for type %s", type);
        return type instanceof TimestampType;
    }

    @Override
    public Optional finishCreateTable(
            ConnectorSession session,
            ConnectorOutputTableHandle tableHandle,
            Collection fragments,
            Collection computedStatistics)
    {
        DeltaLakeOutputTableHandle handle = (DeltaLakeOutputTableHandle) tableHandle;

        String schemaName = handle.getSchemaName();
        String tableName = handle.getTableName();
        String location = handle.getLocation();

        List dataFileInfos = fragments.stream()
                .map(Slice::getBytes)
                .map(dataFileInfoCodec::fromJson)
                .collect(toImmutableList());

        SchemaTableName schemaTableName = schemaTableName(schemaName, tableName);
        Table table = buildTable(session, schemaTableName, location, handle.isExternal());
        // Ensure the table has queryId set. This is relied on for exception handling
        String queryId = session.getQueryId();
        verify(
                getQueryId(table).orElseThrow(() -> new IllegalArgumentException("Query id is not present")).equals(queryId),
                "Table '%s' does not have correct query id set",
                table);

        ColumnMappingMode columnMappingMode = handle.getColumnMappingMode();
        String schemaString = handle.getSchemaString();
        List columnNames = handle.getInputColumns().stream().map(DeltaLakeColumnHandle::getBaseColumnName).collect(toImmutableList());
        List physicalPartitionNames = handle.getInputColumns().stream()
                .filter(column -> column.getColumnType() == PARTITION_KEY)
                .map(DeltaLakeColumnHandle::getBasePhysicalColumnName)
                .collect(toImmutableList());
        try {
            // For CTAS there is no risk of multiple writers racing. Using writer without transaction isolation so we are not limiting support for CTAS to
            // filesystems for which we have proper implementations of TransactionLogSynchronizers.
            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriterWithoutTransactionIsolation(session, handle.getLocation());

            appendTableEntries(
                    0,
                    transactionLogWriter,
                    randomUUID().toString(),
                    schemaString,
                    handle.getPartitionedBy(),
                    configurationForNewTable(handle.getCheckpointInterval(), handle.getChangeDataFeedEnabled(), columnMappingMode, handle.getMaxColumnId()),
                    CREATE_TABLE_AS_OPERATION,
                    session,
                    handle.getComment(),
                    handle.getProtocolEntry());
            appendAddFileEntries(transactionLogWriter, dataFileInfos, physicalPartitionNames, columnNames, true);
            transactionLogWriter.flush();

            if (isCollectExtendedStatisticsColumnStatisticsOnWrite(session) && !computedStatistics.isEmpty()) {
                Optional maxFileModificationTime = dataFileInfos.stream()
                        .map(DataFileInfo::getCreationTime)
                        .max(Long::compare)
                        .map(Instant::ofEpochMilli);
                Map physicalColumnMapping = DeltaLakeSchemaSupport.getColumnMetadata(schemaString, typeManager, columnMappingMode).stream()
                        .map(e -> Map.entry(e.getName(), e.getPhysicalName()))
                        .collect(toImmutableMap(Entry::getKey, Entry::getValue));

                updateTableStatistics(
                        session,
                        Optional.empty(),
                        schemaTableName,
                        location,
                        maxFileModificationTime,
                        computedStatistics,
                        columnNames,
                        Optional.of(physicalColumnMapping));
            }

            PrincipalPrivileges principalPrivileges = buildInitialPrivilegeSet(table.getOwner().orElseThrow());

            // As a precaution, clear the caches
            statisticsAccess.invalidateCache(schemaTableName, Optional.of(location));
            transactionLogAccess.invalidateCache(schemaTableName, Optional.of(location));
            metastore.createTable(session, table, principalPrivileges);
        }
        catch (Exception e) {
            // Remove the transaction log entry if the table creation fails
            try {
                Location transactionLogDir = Location.of(getTransactionLogDir(location));
                fileSystemFactory.create(session).deleteDirectory(transactionLogDir);
            }
            catch (IOException ioException) {
                // Nothing to do, the IOException is probably the same reason why the initial write failed
                LOG.error(ioException, "Transaction log cleanup failed during CREATE TABLE rollback");
            }
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, "Failed to write Delta Lake transaction log entry", e);
        }

        return Optional.empty();
    }

    @Override
    public void setTableComment(ConnectorSession session, ConnectorTableHandle tableHandle, Optional comment)
    {
        DeltaLakeTableHandle handle = checkValidTableHandle(tableHandle);
        checkSupportedWriterVersion(handle);
        ColumnMappingMode columnMappingMode = getColumnMappingMode(handle.getMetadataEntry(), handle.getProtocolEntry());
        if (columnMappingMode != ID && columnMappingMode != NAME && columnMappingMode != NONE) {
            throw new TrinoException(NOT_SUPPORTED, "Setting a table comment with column mapping %s is not supported".formatted(columnMappingMode));
        }
        ProtocolEntry protocolEntry = handle.getProtocolEntry();
        checkUnsupportedWriterFeatures(protocolEntry);

        ConnectorTableMetadata tableMetadata = getTableMetadata(session, handle);

        try {
            long commitVersion = handle.getReadVersion() + 1;

            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, handle.getLocation());
            appendTableEntries(
                    commitVersion,
                    transactionLogWriter,
                    handle.getMetadataEntry().getId(),
                    handle.getMetadataEntry().getSchemaString(),
                    getPartitionedBy(tableMetadata.getProperties()),
                    handle.getMetadataEntry().getConfiguration(),
                    SET_TBLPROPERTIES_OPERATION,
                    session,
                    comment,
                    protocolEntry);
            transactionLogWriter.flush();
        }
        catch (Exception e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, format("Unable to comment on table: %s.%s", handle.getSchemaName(), handle.getTableName()), e);
        }
    }

    @Override
    public void setColumnComment(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle column, Optional comment)
    {
        DeltaLakeTableHandle deltaLakeTableHandle = (DeltaLakeTableHandle) tableHandle;
        DeltaLakeColumnHandle deltaLakeColumnHandle = (DeltaLakeColumnHandle) column;
        verify(deltaLakeColumnHandle.isBaseColumn(), "Unexpected dereference: %s", column);
        checkSupportedWriterVersion(deltaLakeTableHandle);
        ColumnMappingMode columnMappingMode = getColumnMappingMode(deltaLakeTableHandle.getMetadataEntry(), deltaLakeTableHandle.getProtocolEntry());
        if (columnMappingMode != ID && columnMappingMode != NAME && columnMappingMode != NONE) {
            throw new TrinoException(NOT_SUPPORTED, "Setting a column comment with column mapping %s is not supported".formatted(columnMappingMode));
        }
        ProtocolEntry protocolEntry = deltaLakeTableHandle.getProtocolEntry();
        checkUnsupportedWriterFeatures(protocolEntry);

        try {
            long commitVersion = deltaLakeTableHandle.getReadVersion() + 1;

            ImmutableMap.Builder columnComments = ImmutableMap.builder();
            columnComments.putAll(getColumnComments(deltaLakeTableHandle.getMetadataEntry()).entrySet().stream()
                    .filter(e -> !e.getKey().equals(deltaLakeColumnHandle.getBaseColumnName()))
                    .collect(Collectors.toMap(Entry::getKey, Entry::getValue)));
            comment.ifPresent(s -> columnComments.put(deltaLakeColumnHandle.getBaseColumnName(), s));

            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, deltaLakeTableHandle.getLocation());
            appendTableEntries(
                    commitVersion,
                    transactionLogWriter,
                    deltaLakeTableHandle.getMetadataEntry().getId(),
                    getExactColumnNames(deltaLakeTableHandle.getMetadataEntry()),
                    deltaLakeTableHandle.getMetadataEntry().getOriginalPartitionColumns(),
                    getColumnTypes(deltaLakeTableHandle.getMetadataEntry()),
                    columnComments.buildOrThrow(),
                    getColumnsNullability(deltaLakeTableHandle.getMetadataEntry()),
                    getColumnsMetadata(deltaLakeTableHandle.getMetadataEntry()),
                    deltaLakeTableHandle.getMetadataEntry().getConfiguration(),
                    CHANGE_COLUMN_OPERATION,
                    session,
                    Optional.ofNullable(deltaLakeTableHandle.getMetadataEntry().getDescription()),
                    protocolEntry);
            transactionLogWriter.flush();
        }
        catch (Exception e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, format("Unable to add '%s' column comment for: %s.%s", deltaLakeColumnHandle.getBaseColumnName(), deltaLakeTableHandle.getSchemaName(), deltaLakeTableHandle.getTableName()), e);
        }
    }

    @Override
    public void setViewComment(ConnectorSession session, SchemaTableName viewName, Optional comment)
    {
        trinoViewHiveMetastore.updateViewComment(session, viewName, comment);
    }

    @Override
    public void setViewColumnComment(ConnectorSession session, SchemaTableName viewName, String columnName, Optional comment)
    {
        trinoViewHiveMetastore.updateViewColumnComment(session, viewName, columnName, comment);
    }

    @Override
    public void addColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnMetadata newColumnMetadata)
    {
        DeltaLakeTableHandle handle = checkValidTableHandle(tableHandle);
        ProtocolEntry protocolEntry = handle.getProtocolEntry();
        checkSupportedWriterVersion(handle);
        ColumnMappingMode columnMappingMode = getColumnMappingMode(handle.getMetadataEntry(), protocolEntry);
        if (changeDataFeedEnabled(handle.getMetadataEntry(), protocolEntry).orElse(false) && CHANGE_DATA_FEED_COLUMN_NAMES.contains(newColumnMetadata.getName())) {
            throw new TrinoException(NOT_SUPPORTED, "Column name %s is forbidden when change data feed is enabled".formatted(newColumnMetadata.getName()));
        }
        checkUnsupportedWriterFeatures(protocolEntry);

        if (!newColumnMetadata.isNullable() && !transactionLogAccess.getActiveFiles(getSnapshot(session, handle), handle.getMetadataEntry(), handle.getProtocolEntry(), session).isEmpty()) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, format("Unable to add NOT NULL column '%s' for non-empty table: %s.%s", newColumnMetadata.getName(), handle.getSchemaName(), handle.getTableName()));
        }

        try {
            long commitVersion = handle.getReadVersion() + 1;

            AtomicInteger maxColumnId = switch (columnMappingMode) {
                case NONE -> new AtomicInteger();
                case ID, NAME -> new AtomicInteger(getMaxColumnId(handle.getMetadataEntry()));
                default -> throw new IllegalArgumentException("Unexpected column mapping mode: " + columnMappingMode);
            };

            List columnNames = ImmutableList.builder()
                    .addAll(getExactColumnNames(handle.getMetadataEntry()))
                    .add(newColumnMetadata.getName())
                    .build();
            ImmutableMap.Builder columnComments = ImmutableMap.builder();
            columnComments.putAll(getColumnComments(handle.getMetadataEntry()));
            if (newColumnMetadata.getComment() != null) {
                columnComments.put(newColumnMetadata.getName(), newColumnMetadata.getComment());
            }
            ImmutableMap.Builder columnsNullability = ImmutableMap.builder();
            columnsNullability.putAll(getColumnsNullability(handle.getMetadataEntry()));
            columnsNullability.put(newColumnMetadata.getName(), newColumnMetadata.isNullable());
            Map columnTypes = ImmutableMap.builderWithExpectedSize(columnNames.size())
                    .putAll(getColumnTypes(handle.getMetadataEntry()))
                    .put(Map.entry(newColumnMetadata.getName(), serializeColumnType(columnMappingMode, maxColumnId, newColumnMetadata.getType())))
                    .buildOrThrow();

            ImmutableMap.Builder> columnMetadata = ImmutableMap.builder();
            columnMetadata.putAll(getColumnsMetadata(handle.getMetadataEntry()));
            columnMetadata.put(newColumnMetadata.getName(), generateColumnMetadata(columnMappingMode, maxColumnId));

            Map configuration = new HashMap<>(handle.getMetadataEntry().getConfiguration());
            if (columnMappingMode == ID || columnMappingMode == NAME) {
                checkArgument(maxColumnId.get() > 0, "maxColumnId must be larger than 0: %s", maxColumnId);
                configuration.put(MAX_COLUMN_ID_CONFIGURATION_KEY, String.valueOf(maxColumnId.get()));
            }

            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, handle.getLocation());
            appendTableEntries(
                    commitVersion,
                    transactionLogWriter,
                    handle.getMetadataEntry().getId(),
                    serializeSchemaAsJson(
                            columnNames,
                            columnTypes,
                            columnComments.buildOrThrow(),
                            columnsNullability.buildOrThrow(),
                            columnMetadata.buildOrThrow()),
                    handle.getMetadataEntry().getOriginalPartitionColumns(),
                    configuration,
                    ADD_COLUMN_OPERATION,
                    session,
                    Optional.ofNullable(handle.getMetadataEntry().getDescription()),
                    protocolEntry);
            transactionLogWriter.flush();
        }
        catch (Exception e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, format("Unable to add '%s' column for: %s.%s", newColumnMetadata.getName(), handle.getSchemaName(), handle.getTableName()), e);
        }
    }

    @Override
    public void dropColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle)
    {
        DeltaLakeTableHandle table = (DeltaLakeTableHandle) tableHandle;
        DeltaLakeColumnHandle deltaLakeColumn = (DeltaLakeColumnHandle) columnHandle;
        verify(deltaLakeColumn.isBaseColumn(), "Unexpected dereference: %s", deltaLakeColumn);
        String dropColumnName = deltaLakeColumn.getBaseColumnName();
        MetadataEntry metadataEntry = table.getMetadataEntry();
        ProtocolEntry protocolEntry = table.getProtocolEntry();
        checkUnsupportedWriterFeatures(protocolEntry);

        checkSupportedWriterVersion(table);
        ColumnMappingMode columnMappingMode = getColumnMappingMode(metadataEntry, protocolEntry);
        if (columnMappingMode != ColumnMappingMode.NAME && columnMappingMode != ColumnMappingMode.ID) {
            throw new TrinoException(NOT_SUPPORTED, "Cannot drop column from table using column mapping mode " + columnMappingMode);
        }

        long commitVersion = table.getReadVersion() + 1;
        List partitionColumns = metadataEntry.getOriginalPartitionColumns();
        if (partitionColumns.contains(dropColumnName)) {
            throw new TrinoException(NOT_SUPPORTED, "Cannot drop partition column: " + dropColumnName);
        }

        // Use equalsIgnoreCase because the remote column name can contain uppercase characters
        // Creating a table with ambiguous names (e.g. "a" and "A") is disallowed, so this should be safe
        List columns = extractSchema(metadataEntry, protocolEntry, typeManager);
        List columnNames = getExactColumnNames(metadataEntry).stream()
                .filter(name -> !name.equalsIgnoreCase(dropColumnName))
                .collect(toImmutableList());
        if (columns.size() == columnNames.size()) {
            throw new ColumnNotFoundException(table.schemaTableName(), dropColumnName);
        }
        if (columnNames.size() == partitionColumns.size()) {
            throw new TrinoException(NOT_SUPPORTED, "Dropping the last non-partition column is unsupported");
        }
        Map lowerCaseToExactColumnNames = getExactColumnNames(metadataEntry).stream()
                .collect(toImmutableMap(name -> name.toLowerCase(ENGLISH), name -> name));
        Map physicalColumnNameMapping = columns.stream()
                .collect(toImmutableMap(DeltaLakeColumnMetadata::getName, DeltaLakeColumnMetadata::getPhysicalName));

        Map columnTypes = filterKeys(getColumnTypes(metadataEntry), name -> !name.equalsIgnoreCase(dropColumnName));
        Map columnComments = filterKeys(getColumnComments(metadataEntry), name -> !name.equalsIgnoreCase(dropColumnName));
        Map columnsNullability = filterKeys(getColumnsNullability(metadataEntry), name -> !name.equalsIgnoreCase(dropColumnName));
        Map> columnMetadata = filterKeys(getColumnsMetadata(metadataEntry), name -> !name.equalsIgnoreCase(dropColumnName));
        try {
            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, table.getLocation());
            appendTableEntries(
                    commitVersion,
                    transactionLogWriter,
                    metadataEntry.getId(),
                    columnNames,
                    partitionColumns,
                    columnTypes,
                    columnComments,
                    columnsNullability,
                    columnMetadata,
                    metadataEntry.getConfiguration(),
                    DROP_COLUMN_OPERATION,
                    session,
                    Optional.ofNullable(metadataEntry.getDescription()),
                    protocolEntry);
            transactionLogWriter.flush();
        }
        catch (Exception e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, format("Unable to drop '%s' column from: %s.%s", dropColumnName, table.getSchemaName(), table.getTableName()), e);
        }

        try {
            statisticsAccess.readExtendedStatistics(session, table.getSchemaTableName(), table.getLocation()).ifPresent(existingStatistics -> {
                ExtendedStatistics statistics = new ExtendedStatistics(
                        existingStatistics.getAlreadyAnalyzedModifiedTimeMax(),
                        existingStatistics.getColumnStatistics().entrySet().stream()
                                .filter(stats -> !stats.getKey().equalsIgnoreCase(toPhysicalColumnName(dropColumnName, lowerCaseToExactColumnNames, Optional.of(physicalColumnNameMapping))))
                                .collect(toImmutableMap(Entry::getKey, Entry::getValue)),
                        existingStatistics.getAnalyzedColumns()
                                .map(analyzedColumns -> analyzedColumns.stream().filter(column -> !column.equalsIgnoreCase(dropColumnName)).collect(toImmutableSet())));
                statisticsAccess.updateExtendedStatistics(session, table.getSchemaTableName(), table.getLocation(), statistics);
            });
        }
        catch (Exception e) {
            LOG.warn(e, "Failed to update extended statistics when dropping %s column from %s table", dropColumnName, table.schemaTableName());
        }
    }

    @Override
    public void renameColumn(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle, String newColumnName)
    {
        DeltaLakeTableHandle table = (DeltaLakeTableHandle) tableHandle;
        DeltaLakeColumnHandle deltaLakeColumn = (DeltaLakeColumnHandle) columnHandle;
        verify(deltaLakeColumn.isBaseColumn(), "Unexpected dereference: %s", deltaLakeColumn);
        String sourceColumnName = deltaLakeColumn.getBaseColumnName();
        ProtocolEntry protocolEntry = table.getProtocolEntry();
        checkUnsupportedWriterFeatures(protocolEntry);

        checkSupportedWriterVersion(table);
        if (changeDataFeedEnabled(table.getMetadataEntry(), protocolEntry).orElse(false)) {
            throw new TrinoException(NOT_SUPPORTED, "Cannot rename column when change data feed is enabled");
        }

        MetadataEntry metadataEntry = table.getMetadataEntry();
        ColumnMappingMode columnMappingMode = getColumnMappingMode(metadataEntry, protocolEntry);
        if (columnMappingMode != ColumnMappingMode.NAME && columnMappingMode != ColumnMappingMode.ID) {
            throw new TrinoException(NOT_SUPPORTED, "Cannot rename column in table using column mapping mode " + columnMappingMode);
        }

        long commitVersion = table.getReadVersion() + 1;

        // Use equalsIgnoreCase because the remote column name can contain uppercase characters
        // Creating a table with ambiguous names (e.g. "a" and "A") is disallowed, so this should be safe
        List partitionColumns = metadataEntry.getOriginalPartitionColumns().stream()
                .map(columnName -> columnName.equalsIgnoreCase(sourceColumnName) ? newColumnName : columnName)
                .collect(toImmutableList());

        List columnNames = getExactColumnNames(metadataEntry).stream()
                .map(name -> name.equalsIgnoreCase(sourceColumnName) ? newColumnName : name)
                .collect(toImmutableList());
        Map columnTypes = getColumnTypes(metadataEntry).entrySet().stream()
                .map(column -> column.getKey().equalsIgnoreCase(sourceColumnName) ? Map.entry(newColumnName, column.getValue()) : column)
                .collect(toImmutableMap(Entry::getKey, Entry::getValue));
        Map columnComments = getColumnComments(metadataEntry).entrySet().stream()
                .map(column -> column.getKey().equalsIgnoreCase(sourceColumnName) ? Map.entry(newColumnName, column.getValue()) : column)
                .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue));
        Map columnsNullability = getColumnsNullability(metadataEntry).entrySet().stream()
                .map(column -> column.getKey().equalsIgnoreCase(sourceColumnName) ? Map.entry(newColumnName, column.getValue()) : column)
                .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue));
        Map> columnMetadata = getColumnsMetadata(metadataEntry).entrySet().stream()
                .map(column -> column.getKey().equalsIgnoreCase(sourceColumnName) ? Map.entry(newColumnName, column.getValue()) : column)
                .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue));
        try {
            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, table.getLocation());
            appendTableEntries(
                    commitVersion,
                    transactionLogWriter,
                    metadataEntry.getId(),
                    columnNames,
                    partitionColumns,
                    columnTypes,
                    columnComments,
                    columnsNullability,
                    columnMetadata,
                    metadataEntry.getConfiguration(),
                    RENAME_COLUMN_OPERATION,
                    session,
                    Optional.ofNullable(metadataEntry.getDescription()),
                    protocolEntry);
            transactionLogWriter.flush();
            // Don't update extended statistics because it uses physical column names internally
        }
        catch (Exception e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, format("Unable to rename '%s' column for: %s.%s", sourceColumnName, table.getSchemaName(), table.getTableName()), e);
        }
    }

    private void appendTableEntries(
            long commitVersion,
            TransactionLogWriter transactionLogWriter,
            String tableId,
            List columnNames,
            List partitionColumnNames,
            Map columnTypes,
            Map columnComments,
            Map columnNullability,
            Map> columnMetadata,
            Map configuration,
            String operation,
            ConnectorSession session,
            Optional comment,
            ProtocolEntry protocolEntry)
    {
        appendTableEntries(
                commitVersion,
                transactionLogWriter,
                tableId,
                serializeSchemaAsJson(columnNames, columnTypes, columnComments, columnNullability, columnMetadata),
                partitionColumnNames,
                configuration,
                operation,
                session,
                comment,
                protocolEntry);
    }

    private void appendTableEntries(
            long commitVersion,
            TransactionLogWriter transactionLogWriter,
            String tableId,
            String schemaString,
            List partitionColumnNames,
            Map configuration,
            String operation,
            ConnectorSession session,
            Optional comment,
            ProtocolEntry protocolEntry)
    {
        long createdTime = System.currentTimeMillis();
        transactionLogWriter.appendCommitInfoEntry(getCommitInfoEntry(session, commitVersion, createdTime, operation, 0));

        transactionLogWriter.appendProtocolEntry(protocolEntry);

        transactionLogWriter.appendMetadataEntry(
                new MetadataEntry(
                        tableId,
                        null,
                        comment.orElse(null),
                        new Format("parquet", ImmutableMap.of()),
                        schemaString,
                        partitionColumnNames,
                        ImmutableMap.copyOf(configuration),
                        createdTime));
    }

    private static void appendAddFileEntries(TransactionLogWriter transactionLogWriter, List dataFileInfos, List partitionColumnNames, List originalColumnNames, boolean dataChange)
            throws JsonProcessingException
    {
        Map toOriginalColumnNames = originalColumnNames.stream()
                .collect(toImmutableMap(name -> name.toLowerCase(ENGLISH), identity()));
        for (DataFileInfo info : dataFileInfos) {
            // using Hashmap because partition values can be null
            Map partitionValues = new HashMap<>();
            for (int i = 0; i < partitionColumnNames.size(); i++) {
                partitionValues.put(partitionColumnNames.get(i), info.getPartitionValues().get(i));
            }

            Optional> minStats = toOriginalColumnNames(info.getStatistics().getMinValues(), toOriginalColumnNames);
            Optional> maxStats = toOriginalColumnNames(info.getStatistics().getMaxValues(), toOriginalColumnNames);
            Optional> nullStats = toOriginalColumnNames(info.getStatistics().getNullCount(), toOriginalColumnNames);
            DeltaLakeJsonFileStatistics statisticsWithExactNames = new DeltaLakeJsonFileStatistics(info.getStatistics().getNumRecords(), minStats, maxStats, nullStats);

            partitionValues = unmodifiableMap(partitionValues);

            transactionLogWriter.appendAddFileEntry(
                    new AddFileEntry(
                            toUriFormat(info.getPath()), // Paths are RFC 2396 URI encoded https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file
                            partitionValues,
                            info.getSize(),
                            info.getCreationTime(),
                            dataChange,
                            Optional.of(serializeStatsAsJson(statisticsWithExactNames)),
                            Optional.empty(),
                            ImmutableMap.of(),
                            Optional.empty()));
        }
    }

    private static Optional> toOriginalColumnNames(Optional> statistics, Map lowerCaseToExactColumnNames)
    {
        return statistics.map(statsMap -> statsMap.entrySet().stream()
                .collect(toImmutableMap(
                        // Lowercase column names because io.trino.parquet.reader.MetadataReader lowercase the path
                        stats -> lowerCaseToExactColumnNames.getOrDefault(stats.getKey().toLowerCase(ENGLISH), stats.getKey()),
                        Entry::getValue)));
    }

    @Override
    public ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle tableHandle, List columns, RetryMode retryMode)
    {
        DeltaLakeTableHandle table = (DeltaLakeTableHandle) tableHandle;
        checkWriteAllowed(session, table);
        checkWriteSupported(table);

        List inputColumns = columns.stream()
                .map(handle -> (DeltaLakeColumnHandle) handle)
                .collect(toImmutableList());

        ConnectorTableMetadata tableMetadata = getTableMetadata(session, table);

        // This check acts as a safeguard in cases where the input columns may differ from the table metadata case-sensitively
        checkAllColumnsPassedOnInsert(tableMetadata, inputColumns);

        return createInsertHandle(session, retryMode, table, inputColumns);
    }

    private DeltaLakeInsertTableHandle createInsertHandle(ConnectorSession session, RetryMode retryMode, DeltaLakeTableHandle table, List inputColumns)
    {
        String tableLocation = table.getLocation();
        try {
            TrinoFileSystem fileSystem = fileSystemFactory.create(session);
            return new DeltaLakeInsertTableHandle(
                    table.getSchemaTableName(),
                    tableLocation,
                    table.getMetadataEntry(),
                    table.getProtocolEntry(),
                    inputColumns,
                    getMandatoryCurrentVersion(fileSystem, tableLocation),
                    retryMode != NO_RETRIES);
        }
        catch (IOException e) {
            throw new TrinoException(GENERIC_INTERNAL_ERROR, e);
        }
    }

    private void checkAllColumnsPassedOnInsert(ConnectorTableMetadata tableMetadata, List insertColumns)
    {
        List allColumnNames = tableMetadata.getColumns().stream()
                .filter(not(ColumnMetadata::isHidden))
                .map(ColumnMetadata::getName)
                .collect(toImmutableList());

        List insertColumnNames = insertColumns.stream()
                // Lowercase because the above allColumnNames uses lowercase
                .map(column -> column.getBaseColumnName().toLowerCase(ENGLISH))
                .collect(toImmutableList());

        checkArgument(allColumnNames.equals(insertColumnNames), "Not all table columns passed on INSERT; table columns=%s; insert columns=%s", allColumnNames, insertColumnNames);
    }

    @Override
    public Optional finishInsert(
            ConnectorSession session,
            ConnectorInsertTableHandle insertHandle,
            Collection fragments,
            Collection computedStatistics)
    {
        DeltaLakeInsertTableHandle handle = (DeltaLakeInsertTableHandle) insertHandle;

        List dataFileInfos = fragments.stream()
                .map(Slice::getBytes)
                .map(dataFileInfoCodec::fromJson)
                .collect(toImmutableList());

        if (handle.isRetriesEnabled()) {
            cleanExtraOutputFiles(session, Location.of(handle.getLocation()), dataFileInfos);
        }

        boolean writeCommitted = false;
        try {
            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, handle.getLocation());

            long createdTime = Instant.now().toEpochMilli();

            TrinoFileSystem fileSystem = fileSystemFactory.create(session);
            long commitVersion = getMandatoryCurrentVersion(fileSystem, handle.getLocation()) + 1;
            if (commitVersion != handle.getReadVersion() + 1) {
                throw new TransactionConflictException(format("Conflicting concurrent writes found. Expected transaction log version: %s, actual version: %s",
                        handle.getReadVersion(),
                        commitVersion - 1));
            }
            Optional checkpointInterval = handle.getMetadataEntry().getCheckpointInterval();
            // it is not obvious why we need to persist this readVersion
            transactionLogWriter.appendCommitInfoEntry(getCommitInfoEntry(session, commitVersion, createdTime, INSERT_OPERATION, handle.getReadVersion()));

            ColumnMappingMode columnMappingMode = getColumnMappingMode(handle.getMetadataEntry(), handle.getProtocolEntry());
            List partitionColumns = getPartitionColumns(
                    handle.getMetadataEntry().getOriginalPartitionColumns(),
                    handle.getInputColumns(),
                    columnMappingMode);
            List exactColumnNames = getExactColumnNames(handle.getMetadataEntry());
            appendAddFileEntries(transactionLogWriter, dataFileInfos, partitionColumns, exactColumnNames, true);

            transactionLogWriter.flush();
            writeCommitted = true;
            writeCheckpointIfNeeded(session, handle.getTableName(), handle.getLocation(), handle.getReadVersion(), checkpointInterval, commitVersion);

            if (isCollectExtendedStatisticsColumnStatisticsOnWrite(session) && !computedStatistics.isEmpty() && !dataFileInfos.isEmpty()) {
                // TODO (https://github.com/trinodb/trino/issues/16088) Add synchronization when version conflict for INSERT is resolved.
                Optional maxFileModificationTime = dataFileInfos.stream()
                        .map(DataFileInfo::getCreationTime)
                        .max(Long::compare)
                        .map(Instant::ofEpochMilli);
                updateTableStatistics(
                        session,
                        Optional.empty(),
                        handle.getTableName(),
                        handle.getLocation(),
                        maxFileModificationTime,
                        computedStatistics,
                        exactColumnNames,
                        Optional.of(extractSchema(handle.getMetadataEntry(), handle.getProtocolEntry(), typeManager).stream()
                                .collect(toImmutableMap(DeltaLakeColumnMetadata::getName, DeltaLakeColumnMetadata::getPhysicalName))));
            }
        }
        catch (Exception e) {
            if (!writeCommitted) {
                // TODO perhaps it should happen in a background thread (https://github.com/trinodb/trino/issues/12011)
                cleanupFailedWrite(session, handle.getLocation(), dataFileInfos);
            }
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, "Failed to write Delta Lake transaction log entry", e);
        }

        return Optional.empty();
    }

    private static List getPartitionColumns(List originalPartitionColumns, List dataColumns, ColumnMappingMode columnMappingMode)
    {
        return switch (columnMappingMode) {
            case NAME, ID -> getPartitionColumnsForNameOrIdMapping(originalPartitionColumns, dataColumns);
            case NONE -> originalPartitionColumns;
            case UNKNOWN -> throw new TrinoException(NOT_SUPPORTED, "Unsupported column mapping mode");
        };
    }

    private static List getPartitionColumnsForNameOrIdMapping(List originalPartitionColumns, List dataColumns)
    {
        Map nameToDataColumns = dataColumns.stream()
                .collect(toImmutableMap(DeltaLakeColumnHandle::getColumnName, Function.identity()));
        return originalPartitionColumns.stream()
                .map(columnName -> {
                    DeltaLakeColumnHandle dataColumn = nameToDataColumns.get(columnName);
                    // During writes we want to preserve original case of partition columns, if the name is not different from the physical name
                    if (dataColumn.getBasePhysicalColumnName().equalsIgnoreCase(columnName)) {
                        return columnName;
                    }
                    return dataColumn.getBasePhysicalColumnName();
                })
                .collect(toImmutableList());
    }

    @Override
    public RowChangeParadigm getRowChangeParadigm(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        return DELETE_ROW_AND_INSERT_ROW;
    }

    @Override
    public ColumnHandle getMergeRowIdColumnHandle(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        return mergeRowIdColumnHandle();
    }

    @Override
    public Optional getUpdateLayout(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        return Optional.of(DeltaLakeUpdateHandle.INSTANCE);
    }

    @Override
    public ConnectorMergeTableHandle beginMerge(ConnectorSession session, ConnectorTableHandle tableHandle, RetryMode retryMode)
    {
        DeltaLakeTableHandle handle = (DeltaLakeTableHandle) tableHandle;
        if (isAppendOnly(handle.getMetadataEntry(), handle.getProtocolEntry())) {
            throw new TrinoException(NOT_SUPPORTED, "Cannot modify rows from a table with '" + APPEND_ONLY_CONFIGURATION_KEY + "' set to true");
        }
        checkWriteAllowed(session, handle);
        checkWriteSupported(handle);

        List inputColumns = getColumns(handle.getMetadataEntry(), handle.getProtocolEntry()).stream()
                .filter(column -> column.getColumnType() != SYNTHESIZED)
                .collect(toImmutableList());

        DeltaLakeInsertTableHandle insertHandle = createInsertHandle(session, retryMode, handle, inputColumns);

        return new DeltaLakeMergeTableHandle(handle, insertHandle);
    }

    @Override
    public void finishMerge(ConnectorSession session, ConnectorMergeTableHandle mergeTableHandle, Collection fragments, Collection computedStatistics)
    {
        DeltaLakeMergeTableHandle mergeHandle = (DeltaLakeMergeTableHandle) mergeTableHandle;
        DeltaLakeTableHandle handle = mergeHandle.getTableHandle();

        List mergeResults = fragments.stream()
                .map(Slice::getBytes)
                .map(mergeResultJsonCodec::fromJson)
                .collect(toImmutableList());

        List oldFiles = mergeResults.stream()
                .map(DeltaLakeMergeResult::getOldFile)
                .flatMap(Optional::stream)
                .collect(toImmutableList());

        List allFiles = mergeResults.stream()
                .map(DeltaLakeMergeResult::getNewFile)
                .flatMap(Optional::stream)
                .collect(toImmutableList());

        Map> split = allFiles.stream()
                .collect(partitioningBy(dataFile -> dataFile.getDataFileType() == DATA));

        List newFiles = ImmutableList.copyOf(split.get(true));
        List cdcFiles = ImmutableList.copyOf(split.get(false));

        if (mergeHandle.getInsertTableHandle().isRetriesEnabled()) {
            cleanExtraOutputFiles(session, Location.of(handle.getLocation()), allFiles);
        }

        Optional checkpointInterval = handle.getMetadataEntry().getCheckpointInterval();

        String tableLocation = handle.getLocation();
        boolean writeCommitted = false;
        try {
            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, tableLocation);

            long createdTime = Instant.now().toEpochMilli();

            TrinoFileSystem fileSystem = fileSystemFactory.create(session);
            long currentVersion = getMandatoryCurrentVersion(fileSystem, tableLocation);
            if (currentVersion != handle.getReadVersion()) {
                throw new TransactionConflictException(format("Conflicting concurrent writes found. Expected transaction log version: %s, actual version: %s", handle.getReadVersion(), currentVersion));
            }
            long commitVersion = currentVersion + 1;

            transactionLogWriter.appendCommitInfoEntry(getCommitInfoEntry(session, commitVersion, createdTime, MERGE_OPERATION, handle.getReadVersion()));
            // TODO: Delta writes another field "operationMetrics" (https://github.com/trinodb/trino/issues/12005)

            long writeTimestamp = Instant.now().toEpochMilli();

            ColumnMappingMode columnMappingMode = getColumnMappingMode(handle.getMetadataEntry(), handle.getProtocolEntry());
            List partitionColumns = getPartitionColumns(
                    handle.getMetadataEntry().getOriginalPartitionColumns(),
                    mergeHandle.getInsertTableHandle().getInputColumns(),
                    columnMappingMode);

            if (!cdcFiles.isEmpty()) {
                appendCdcFilesInfos(transactionLogWriter, cdcFiles, partitionColumns);
            }

            for (String file : oldFiles) {
                transactionLogWriter.appendRemoveFileEntry(new RemoveFileEntry(toUriFormat(file), writeTimestamp, true));
            }

            appendAddFileEntries(transactionLogWriter, newFiles, partitionColumns, getExactColumnNames(handle.getMetadataEntry()), true);

            transactionLogWriter.flush();
            writeCommitted = true;

            writeCheckpointIfNeeded(session, handle.getSchemaTableName(), handle.getLocation(), handle.getReadVersion(), checkpointInterval, commitVersion);
        }
        catch (IOException | RuntimeException e) {
            if (!writeCommitted) {
                // TODO perhaps it should happen in a background thread (https://github.com/trinodb/trino/issues/12011)
                cleanupFailedWrite(session, tableLocation, allFiles);
            }
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, "Failed to write Delta Lake transaction log entry", e);
        }
    }

    private static void appendCdcFilesInfos(
            TransactionLogWriter transactionLogWriter,
            List cdcFilesInfos,
            List partitionColumnNames)
    {
        for (DataFileInfo info : cdcFilesInfos) {
            // using Hashmap because partition values can be null
            Map partitionValues = new HashMap<>();
            for (int i = 0; i < partitionColumnNames.size(); i++) {
                partitionValues.put(partitionColumnNames.get(i), info.getPartitionValues().get(i));
            }
            partitionValues = unmodifiableMap(partitionValues);

            transactionLogWriter.appendCdcEntry(
                    new CdcEntry(
                            toUriFormat(info.getPath()),
                            partitionValues,
                            info.getSize()));
        }
    }

    @Override
    public Optional getTableHandleForExecute(
            ConnectorSession session,
            ConnectorTableHandle connectorTableHandle,
            String procedureName,
            Map executeProperties,
            RetryMode retryMode)
    {
        DeltaLakeTableHandle tableHandle = checkValidTableHandle(connectorTableHandle);
        checkUnsupportedWriterFeatures(tableHandle.getProtocolEntry());

        DeltaLakeTableProcedureId procedureId;
        try {
            procedureId = DeltaLakeTableProcedureId.valueOf(procedureName);
        }
        catch (IllegalArgumentException e) {
            throw new IllegalArgumentException("Unknown procedure '" + procedureName + "'");
        }

        return switch (procedureId) {
            case OPTIMIZE -> getTableHandleForOptimize(tableHandle, executeProperties, retryMode);
        };
    }

    private Optional getTableHandleForOptimize(DeltaLakeTableHandle tableHandle, Map executeProperties, RetryMode retryMode)
    {
        DataSize maxScannedFileSize = (DataSize) executeProperties.get("file_size_threshold");

        List columns = getColumns(tableHandle.getMetadataEntry(), tableHandle.getProtocolEntry()).stream()
                .filter(column -> column.getColumnType() != SYNTHESIZED)
                .collect(toImmutableList());

        return Optional.of(new DeltaLakeTableExecuteHandle(
                tableHandle.getSchemaTableName(),
                OPTIMIZE,
                new DeltaTableOptimizeHandle(
                        tableHandle.getMetadataEntry(),
                        tableHandle.getProtocolEntry(),
                        columns,
                        tableHandle.getMetadataEntry().getOriginalPartitionColumns(),
                        maxScannedFileSize,
                        Optional.empty(),
                        retryMode != NO_RETRIES),
                tableHandle.getLocation()));
    }

    @Override
    public Optional getLayoutForTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle)
    {
        DeltaLakeTableExecuteHandle executeHandle = (DeltaLakeTableExecuteHandle) tableExecuteHandle;
        switch (executeHandle.getProcedureId()) {
            case OPTIMIZE:
                return getLayoutForOptimize(executeHandle);
        }
        throw new IllegalArgumentException("Unknown procedure '" + executeHandle.getProcedureId() + "'");
    }

    private Optional getLayoutForOptimize(DeltaLakeTableExecuteHandle executeHandle)
    {
        DeltaTableOptimizeHandle optimizeHandle = (DeltaTableOptimizeHandle) executeHandle.getProcedureHandle();
        List partitionColumnNames = optimizeHandle.getMetadataEntry().getLowercasePartitionColumns();
        if (partitionColumnNames.isEmpty()) {
            return Optional.empty();
        }
        Map columnsByName = optimizeHandle.getTableColumns().stream()
                .collect(toImmutableMap(columnHandle -> columnHandle.getColumnName(), identity()));
        ImmutableList.Builder partitioningColumns = ImmutableList.builder();
        for (String columnName : partitionColumnNames) {
            partitioningColumns.add(columnsByName.get(columnName));
        }
        DeltaLakePartitioningHandle partitioningHandle = new DeltaLakePartitioningHandle(partitioningColumns.build());
        return Optional.of(new ConnectorTableLayout(partitioningHandle, partitionColumnNames, true));
    }

    @Override
    public BeginTableExecuteResult beginTableExecute(
            ConnectorSession session,
            ConnectorTableExecuteHandle tableExecuteHandle,
            ConnectorTableHandle updatedSourceTableHandle)
    {
        DeltaLakeTableExecuteHandle executeHandle = (DeltaLakeTableExecuteHandle) tableExecuteHandle;
        DeltaLakeTableHandle table = (DeltaLakeTableHandle) updatedSourceTableHandle;
        switch (executeHandle.getProcedureId()) {
            case OPTIMIZE:
                return beginOptimize(session, executeHandle, table);
        }
        throw new IllegalArgumentException("Unknown procedure '" + executeHandle.getProcedureId() + "'");
    }

    private BeginTableExecuteResult beginOptimize(
            ConnectorSession session,
            DeltaLakeTableExecuteHandle executeHandle,
            DeltaLakeTableHandle table)
    {
        DeltaTableOptimizeHandle optimizeHandle = (DeltaTableOptimizeHandle) executeHandle.getProcedureHandle();

        checkWriteAllowed(session, table);
        checkSupportedWriterVersion(table);

        return new BeginTableExecuteResult<>(
                executeHandle.withProcedureHandle(optimizeHandle.withCurrentVersion(table.getReadVersion())),
                table.forOptimize(true, optimizeHandle.getMaxScannedFileSize()));
    }

    @Override
    public void finishTableExecute(ConnectorSession session, ConnectorTableExecuteHandle tableExecuteHandle, Collection fragments, List splitSourceInfo)
    {
        DeltaLakeTableExecuteHandle executeHandle = (DeltaLakeTableExecuteHandle) tableExecuteHandle;
        switch (executeHandle.getProcedureId()) {
            case OPTIMIZE:
                finishOptimize(session, executeHandle, fragments, splitSourceInfo);
                return;
        }
        throw new IllegalArgumentException("Unknown procedure '" + executeHandle.getProcedureId() + "'");
    }

    private void finishOptimize(ConnectorSession session, DeltaLakeTableExecuteHandle executeHandle, Collection fragments, List splitSourceInfo)
    {
        DeltaTableOptimizeHandle optimizeHandle = (DeltaTableOptimizeHandle) executeHandle.getProcedureHandle();
        long readVersion = optimizeHandle.getCurrentVersion().orElseThrow(() -> new IllegalArgumentException("currentVersion not set"));
        String tableLocation = executeHandle.getTableLocation();

        // paths to be deleted
        Set scannedPaths = splitSourceInfo.stream()
                .map(String.class::cast)
                .collect(toImmutableSet());

        // files to be added
        List dataFileInfos = fragments.stream()
                .map(Slice::getBytes)
                .map(dataFileInfoCodec::fromJson)
                .collect(toImmutableList());

        if (optimizeHandle.isRetriesEnabled()) {
            cleanExtraOutputFiles(session, Location.of(executeHandle.getTableLocation()), dataFileInfos);
        }

        boolean writeCommitted = false;
        try {
            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, tableLocation);

            long createdTime = Instant.now().toEpochMilli();
            long commitVersion = readVersion + 1;
            transactionLogWriter.appendCommitInfoEntry(getCommitInfoEntry(session, commitVersion, createdTime, OPTIMIZE_OPERATION, readVersion));
            // TODO: Delta writes another field "operationMetrics" that I haven't
            //   seen before. It contains delete/update metrics. Investigate/include it.

            long writeTimestamp = Instant.now().toEpochMilli();

            for (String scannedPath : scannedPaths) {
                String relativePath = relativePath(tableLocation, scannedPath);
                transactionLogWriter.appendRemoveFileEntry(new RemoveFileEntry(toUriFormat(relativePath), writeTimestamp, false));
            }

            // Note: during writes we want to preserve original case of partition columns
            List partitionColumns = getPartitionColumns(
                    optimizeHandle.getMetadataEntry().getOriginalPartitionColumns(),
                    optimizeHandle.getTableColumns(),
                    getColumnMappingMode(optimizeHandle.getMetadataEntry(), optimizeHandle.getProtocolEntry()));
            appendAddFileEntries(transactionLogWriter, dataFileInfos, partitionColumns, getExactColumnNames(optimizeHandle.getMetadataEntry()), false);

            transactionLogWriter.flush();
            writeCommitted = true;
            Optional checkpointInterval = Optional.of(1L); // force checkpoint
            writeCheckpointIfNeeded(
                    session,
                    executeHandle.getSchemaTableName(),
                    executeHandle.getTableLocation(),
                    optimizeHandle.getCurrentVersion().orElseThrow(),
                    checkpointInterval,
                    commitVersion);
        }
        catch (Exception e) {
            if (!writeCommitted) {
                // TODO perhaps it should happen in a background thread (https://github.com/trinodb/trino/issues/12011)
                cleanupFailedWrite(session, tableLocation, dataFileInfos);
            }
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, "Failed to write Delta Lake transaction log entry", e);
        }
    }

    private void checkWriteAllowed(ConnectorSession session, DeltaLakeTableHandle table)
    {
        if (!allowWrite(session, table)) {
            String fileSystem = Location.of(table.getLocation()).scheme().orElse("unknown");
            throw new TrinoException(
                    NOT_SUPPORTED,
                    format("Writes are not enabled on the %1$s filesystem in order to avoid eventual data corruption which may be caused by concurrent data modifications on the table. " +
                            "Writes to the %1$s filesystem can be however enabled with the '%2$s' configuration property.", fileSystem, ENABLE_NON_CONCURRENT_WRITES_CONFIGURATION_KEY));
        }
    }

    private boolean allowWrite(ConnectorSession session, DeltaLakeTableHandle tableHandle)
    {
        try {
            String tableMetadataDirectory = getTransactionLogDir(tableHandle.getLocation());
            boolean requiresOptIn = transactionLogWriterFactory.newWriter(session, tableMetadataDirectory).isUnsafe();
            return !requiresOptIn || unsafeWritesEnabled;
        }
        catch (TrinoException e) {
            if (e.getErrorCode() == NOT_SUPPORTED.toErrorCode()) {
                return false;
            }
            throw e;
        }
    }

    private void checkWriteSupported(DeltaLakeTableHandle handle)
    {
        checkSupportedWriterVersion(handle);
        checkUnsupportedGeneratedColumns(handle.getMetadataEntry());
        ColumnMappingMode columnMappingMode = getColumnMappingMode(handle.getMetadataEntry(), handle.getProtocolEntry());
        if (!(columnMappingMode == NONE || columnMappingMode == ColumnMappingMode.NAME || columnMappingMode == ColumnMappingMode.ID)) {
            throw new TrinoException(NOT_SUPPORTED, "Writing with column mapping %s is not supported".formatted(columnMappingMode));
        }
        if (getColumnIdentities(handle.getMetadataEntry(), handle.getProtocolEntry()).values().stream().anyMatch(identity -> identity)) {
            throw new TrinoException(NOT_SUPPORTED, "Writing to tables with identity columns is not supported");
        }
        checkUnsupportedWriterFeatures(handle.getProtocolEntry());
    }

    private static void checkUnsupportedWriterFeatures(ProtocolEntry protocolEntry)
    {
        Set unsupportedWriterFeatures = unsupportedWriterFeatures(protocolEntry.getWriterFeatures().orElse(ImmutableSet.of()));
        if (!unsupportedWriterFeatures.isEmpty()) {
            throw new TrinoException(NOT_SUPPORTED, "Unsupported writer features: " + unsupportedWriterFeatures);
        }
    }

    private void checkUnsupportedGeneratedColumns(MetadataEntry metadataEntry)
    {
        Map columnGeneratedExpressions = getGeneratedColumnExpressions(metadataEntry);
        if (!columnGeneratedExpressions.isEmpty()) {
            throw new TrinoException(NOT_SUPPORTED, "Writing to tables with generated columns is not supported");
        }
    }

    private void checkSupportedWriterVersion(DeltaLakeTableHandle handle)
    {
        int requiredWriterVersion = handle.getProtocolEntry().getMinWriterVersion();
        if (requiredWriterVersion > MAX_WRITER_VERSION) {
            throw new TrinoException(
                    NOT_SUPPORTED,
                    format("Table %s requires Delta Lake writer version %d which is not supported", handle.getSchemaTableName(), requiredWriterVersion));
        }
    }

    private TableSnapshot getSnapshot(ConnectorSession session, DeltaLakeTableHandle table)
    {
        return getSnapshot(session, table.getSchemaTableName(), table.getLocation(), Optional.of(table.getReadVersion()));
    }

    private ProtocolEntry protocolEntryForNewTable(boolean containsTimestampType, Map properties)
    {
        int readerVersion = DEFAULT_READER_VERSION;
        int writerVersion = DEFAULT_WRITER_VERSION;
        Set readerFeatures = new HashSet<>();
        Set writerFeatures = new HashSet<>();
        Optional changeDataFeedEnabled = getChangeDataFeedEnabled(properties);
        if (changeDataFeedEnabled.isPresent() && changeDataFeedEnabled.get()) {
            // Enabling cdf (change data feed) requires setting the writer version to 4
            writerVersion = CDF_SUPPORTED_WRITER_VERSION;
        }
        ColumnMappingMode columnMappingMode = DeltaLakeTableProperties.getColumnMappingMode(properties);
        if (columnMappingMode == ID || columnMappingMode == NAME) {
            readerVersion = max(readerVersion, COLUMN_MAPPING_MODE_SUPPORTED_READER_VERSION);
            writerVersion = max(writerVersion, COLUMN_MAPPING_MODE_SUPPORTED_WRITER_VERSION);
        }
        if (containsTimestampType) {
            readerVersion = max(readerVersion, TIMESTAMP_NTZ_SUPPORTED_READER_VERSION);
            writerVersion = max(writerVersion, TIMESTAMP_NTZ_SUPPORTED_WRITER_VERSION);
            readerFeatures.add(TIMESTAMP_NTZ_FEATURE_NAME);
            writerFeatures.add(TIMESTAMP_NTZ_FEATURE_NAME);
        }
        return new ProtocolEntry(
                readerVersion,
                writerVersion,
                readerFeatures.isEmpty() ? Optional.empty() : Optional.of(readerFeatures),
                writerFeatures.isEmpty() ? Optional.empty() : Optional.of(writerFeatures));
    }

    private void writeCheckpointIfNeeded(
            ConnectorSession session,
            SchemaTableName table,
            String tableLocation,
            long readVersion,
            Optional checkpointInterval,
            long newVersion)
    {
        try {
            // We are writing checkpoint synchronously. It should not be long lasting operation for tables where transaction log is not humongous.
            // Tables with really huge transaction logs would behave poorly in read flow already.
            TableSnapshot snapshot = getSnapshot(session, table, tableLocation, Optional.of(readVersion));
            long lastCheckpointVersion = snapshot.getLastCheckpointVersion().orElse(0L);
            if (newVersion - lastCheckpointVersion < checkpointInterval.orElse(defaultCheckpointInterval)) {
                return;
            }

            // TODO: There is a race possibility here(https://github.com/trinodb/trino/issues/12004),
            // which may result in us not writing checkpoints at exactly the planned frequency.
            // The snapshot obtained above may already be on a version higher than `newVersion` because some other transaction could have just been committed.
            // This does not pose correctness issue but may be confusing if someone looks into transaction log.
            // To fix that we should allow for getting snapshot for given version.
            if (snapshot.getVersion() > newVersion) {
                LOG.info("Snapshot for table %s already at version %s when checkpoint requested for version %s", table, snapshot.getVersion(), newVersion);
            }

            TableSnapshot updatedSnapshot = snapshot.getUpdatedSnapshot(fileSystemFactory.create(session), Optional.of(newVersion)).orElseThrow();
            checkpointWriterManager.writeCheckpoint(session, updatedSnapshot);
        }
        catch (Exception e) {
            // We can't fail here as transaction was already committed, in case of INSERT this could result
            // in inserting data twice if client saw an error and decided to retry
            LOG.error(e, "Failed to write checkpoint for table %s for version %s", table, newVersion);
        }
    }

    private void cleanupFailedWrite(ConnectorSession session, String tableLocation, List dataFiles)
    {
        Location location = Location.of(tableLocation);
        List filesToDelete = dataFiles.stream()
                .map(DataFileInfo::getPath)
                .map(location::appendPath)
                .collect(toImmutableList());
        try {
            TrinoFileSystem fileSystem = fileSystemFactory.create(session);
            fileSystem.deleteFiles(filesToDelete);
        }
        catch (Exception e) {
            // Can be safely ignored since a VACUUM from DeltaLake will take care of such orphaned files
            LOG.warn(e, "Failed cleanup of leftover files from failed write, files are: %s", filesToDelete);
        }
    }

    @Override
    public Optional getInfo(ConnectorTableHandle table)
    {
        boolean isPartitioned = !((DeltaLakeTableHandle) table).getMetadataEntry().getLowercasePartitionColumns().isEmpty();
        return Optional.of(new DeltaLakeInputInfo(isPartitioned));
    }

    @Override
    public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        LocatedTableHandle handle = (LocatedTableHandle) tableHandle;
        boolean deleteData = handle.managed();
        metastore.dropTable(session, handle.schemaTableName(), handle.location(), deleteData);
        if (deleteData) {
            try {
                fileSystemFactory.create(session).deleteDirectory(Location.of(handle.location()));
            }
            catch (IOException e) {
                throw new TrinoException(DELTA_LAKE_FILESYSTEM_ERROR, format("Failed to delete directory %s of the table %s", handle.location(), handle.schemaTableName()), e);
            }
        }
        // As a precaution, clear the caches
        statisticsAccess.invalidateCache(handle.schemaTableName(), Optional.of(handle.location()));
        transactionLogAccess.invalidateCache(handle.schemaTableName(), Optional.of(handle.location()));
    }

    @Override
    public void renameTable(ConnectorSession session, ConnectorTableHandle tableHandle, SchemaTableName newTableName)
    {
        DeltaLakeTableHandle handle = checkValidTableHandle(tableHandle);
        DeltaMetastoreTable table = metastore.getTable(handle.getSchemaName(), handle.getTableName())
                .orElseThrow(() -> new TableNotFoundException(handle.getSchemaTableName()));
        if (table.managed() && !allowManagedTableRename) {
            throw new TrinoException(NOT_SUPPORTED, "Renaming managed tables is not allowed with current metastore configuration");
        }
        metastore.renameTable(session, handle.getSchemaTableName(), newTableName);
    }

    private CommitInfoEntry getCommitInfoEntry(
            ConnectorSession session,
            long commitVersion,
            long createdTime,
            String operation,
            long readVersion)
    {
        return new CommitInfoEntry(
                commitVersion,
                createdTime,
                session.getUser(),
                session.getUser(),
                operation,
                ImmutableMap.of("queryId", session.getQueryId()),
                null,
                null,
                "trino-" + nodeVersion + "-" + nodeId,
                readVersion,
                ISOLATION_LEVEL,
                Optional.of(true));
    }

    @Override
    public void setTableProperties(ConnectorSession session, ConnectorTableHandle tableHandle, Map> properties)
    {
        DeltaLakeTableHandle handle = checkValidTableHandle(tableHandle);
        Set unsupportedProperties = difference(properties.keySet(), UPDATABLE_TABLE_PROPERTIES);
        if (!unsupportedProperties.isEmpty()) {
            throw new TrinoException(NOT_SUPPORTED, "The following properties cannot be updated: " + String.join(", ", unsupportedProperties));
        }

        ProtocolEntry currentProtocolEntry = handle.getProtocolEntry();

        long createdTime = Instant.now().toEpochMilli();

        int requiredWriterVersion = currentProtocolEntry.getMinWriterVersion();
        Optional metadataEntry = Optional.empty();
        if (properties.containsKey(CHANGE_DATA_FEED_ENABLED_PROPERTY)) {
            boolean changeDataFeedEnabled = (Boolean) properties.get(CHANGE_DATA_FEED_ENABLED_PROPERTY)
                    .orElseThrow(() -> new IllegalArgumentException("The change_data_feed_enabled property cannot be empty"));
            if (changeDataFeedEnabled) {
                Set columnNames = getColumns(handle.getMetadataEntry(), handle.getProtocolEntry()).stream().map(DeltaLakeColumnHandle::getBaseColumnName).collect(toImmutableSet());
                Set conflicts = Sets.intersection(columnNames, CHANGE_DATA_FEED_COLUMN_NAMES);
                if (!conflicts.isEmpty()) {
                    throw new TrinoException(NOT_SUPPORTED, "Unable to enable change data feed because table contains %s columns".formatted(conflicts));
                }
                requiredWriterVersion = max(requiredWriterVersion, CDF_SUPPORTED_WRITER_VERSION);
            }
            Map configuration = new HashMap<>(handle.getMetadataEntry().getConfiguration());
            configuration.put(DELTA_CHANGE_DATA_FEED_ENABLED_PROPERTY, String.valueOf(changeDataFeedEnabled));
            metadataEntry = Optional.of(buildMetadataEntry(handle.getMetadataEntry(), configuration, createdTime));
        }

        long readVersion = handle.getReadVersion();
        long commitVersion = readVersion + 1;

        Optional protocolEntry = Optional.empty();
        if (requiredWriterVersion != currentProtocolEntry.getMinWriterVersion()) {
            protocolEntry = Optional.of(new ProtocolEntry(currentProtocolEntry.getMinReaderVersion(), requiredWriterVersion, currentProtocolEntry.getReaderFeatures(), currentProtocolEntry.getWriterFeatures()));
        }

        try {
            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, handle.getLocation());
            transactionLogWriter.appendCommitInfoEntry(getCommitInfoEntry(session, commitVersion, createdTime, SET_TBLPROPERTIES_OPERATION, readVersion));
            protocolEntry.ifPresent(transactionLogWriter::appendProtocolEntry);

            metadataEntry.ifPresent(transactionLogWriter::appendMetadataEntry);

            transactionLogWriter.flush();
        }
        catch (IOException e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, "Failed to write Delta Lake transaction log entry", e);
        }
    }

    private MetadataEntry buildMetadataEntry(MetadataEntry metadataEntry, Map configuration, long createdTime)
    {
        return new MetadataEntry(
                metadataEntry.getId(),
                metadataEntry.getName(),
                metadataEntry.getDescription(),
                metadataEntry.getFormat(),
                metadataEntry.getSchemaString(),
                metadataEntry.getOriginalPartitionColumns(),
                configuration,
                createdTime);
    }

    @Override
    public Map getSchemaProperties(ConnectorSession session, String schemaName)
    {
        if (isHiveSystemSchema(schemaName)) {
            throw new TrinoException(NOT_SUPPORTED, "Schema properties are not supported for system schema: " + schemaName);
        }
        return metastore.getDatabase(schemaName)
                .map(DeltaLakeSchemaProperties::fromDatabase)
                .orElseThrow(() -> new SchemaNotFoundException(schemaName));
    }

    @Override
    public void createView(ConnectorSession session, SchemaTableName viewName, ConnectorViewDefinition definition, boolean replace)
    {
        trinoViewHiveMetastore.createView(session, viewName, definition, replace);
    }

    @Override
    public void dropView(ConnectorSession session, SchemaTableName viewName)
    {
        trinoViewHiveMetastore.dropView(viewName);
    }

    @Override
    public List listViews(ConnectorSession session, Optional schemaName)
    {
        return trinoViewHiveMetastore.listViews(schemaName);
    }

    @Override
    public Map getViews(ConnectorSession session, Optional schemaName)
    {
        return trinoViewHiveMetastore.getViews(schemaName);
    }

    @Override
    public Optional getView(ConnectorSession session, SchemaTableName viewName)
    {
        return trinoViewHiveMetastore.getView(viewName);
    }

    @Override
    public void createRole(ConnectorSession session, String role, Optional grantor)
    {
        accessControlMetadata.createRole(session, role, grantor.map(HivePrincipal::from));
    }

    @Override
    public void dropRole(ConnectorSession session, String role)
    {
        accessControlMetadata.dropRole(session, role);
    }

    @Override
    public Set listRoles(ConnectorSession session)
    {
        return accessControlMetadata.listRoles(session);
    }

    @Override
    public Set listRoleGrants(ConnectorSession session, TrinoPrincipal principal)
    {
        return ImmutableSet.copyOf(accessControlMetadata.listRoleGrants(session, HivePrincipal.from(principal)));
    }

    @Override
    public void grantRoles(ConnectorSession session, Set roles, Set grantees, boolean withAdminOption, Optional grantor)
    {
        accessControlMetadata.grantRoles(session, roles, HivePrincipal.from(grantees), withAdminOption, grantor.map(HivePrincipal::from));
    }

    @Override
    public void revokeRoles(ConnectorSession session, Set roles, Set grantees, boolean adminOptionFor, Optional grantor)
    {
        accessControlMetadata.revokeRoles(session, roles, HivePrincipal.from(grantees), adminOptionFor, grantor.map(HivePrincipal::from));
    }

    @Override
    public Set listApplicableRoles(ConnectorSession session, TrinoPrincipal principal)
    {
        return accessControlMetadata.listApplicableRoles(session, HivePrincipal.from(principal));
    }

    @Override
    public Set listEnabledRoles(ConnectorSession session)
    {
        return accessControlMetadata.listEnabledRoles(session);
    }

    @Override
    public void grantTablePrivileges(ConnectorSession session, SchemaTableName schemaTableName, Set privileges, TrinoPrincipal grantee, boolean grantOption)
    {
        accessControlMetadata.grantTablePrivileges(session, schemaTableName, privileges, HivePrincipal.from(grantee), grantOption);
    }

    @Override
    public void revokeTablePrivileges(ConnectorSession session, SchemaTableName schemaTableName, Set privileges, TrinoPrincipal grantee, boolean grantOption)
    {
        accessControlMetadata.revokeTablePrivileges(session, schemaTableName, privileges, HivePrincipal.from(grantee), grantOption);
    }

    @Override
    public List listTablePrivileges(ConnectorSession session, SchemaTablePrefix schemaTablePrefix)
    {
        return accessControlMetadata.listTablePrivileges(session, listTables(session, schemaTablePrefix));
    }

    private List listTables(ConnectorSession session, SchemaTablePrefix prefix)
    {
        if (prefix.getTable().isEmpty()) {
            return listTables(session, prefix.getSchema());
        }
        SchemaTableName tableName = prefix.toSchemaTableName();
        return metastore.getTable(tableName.getSchemaName(), tableName.getTableName())
                .map(table -> ImmutableList.of(tableName))
                .orElse(ImmutableList.of());
    }

    private void setRollback(Runnable action)
    {
        checkState(rollbackAction.compareAndSet(null, action), "rollback action is already set");
    }

    private static String toUriFormat(String path)
    {
        verify(!path.startsWith("/") && !path.contains(":/"), "unexpected path: %s", path);
        try {
            return new URI(null, null, path, null).toString();
        }
        catch (URISyntaxException e) {
            throw new IllegalArgumentException("Invalid path: " + path, e);
        }
    }

    static String relativePath(String basePath, String path)
    {
        String basePathDirectory = basePath.endsWith("/") ? basePath : basePath + "/";
        checkArgument(path.startsWith(basePathDirectory) && (path.length() > basePathDirectory.length()),
                "path [%s] must be a subdirectory of basePath [%s]", path, basePath);
        return path.substring(basePathDirectory.length());
    }

    public void rollback()
    {
        // The actions are responsible for cleanup in case operation is aborted.
        // So far this is used by CTAS flow which does not require an explicit commit operation therefore
        // DeltaLakeMetadata does not define a commit() method.
        Optional.ofNullable(rollbackAction.getAndSet(null)).ifPresent(Runnable::run);
    }

    @Override
    public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle handle, Constraint constraint)
    {
        DeltaLakeTableHandle tableHandle = (DeltaLakeTableHandle) handle;
        SchemaTableName tableName = tableHandle.getSchemaTableName();

        checkArgument(constraint.getSummary().getDomains().isPresent(), "constraint summary is NONE");

        UtcConstraintExtractor.ExtractionResult extractionResult = extractTupleDomain(constraint);
        TupleDomain predicate = extractionResult.tupleDomain();

        if (predicate.isAll() && constraint.getPredicateColumns().isEmpty()) {
            return Optional.empty();
        }

        TupleDomain newEnforcedConstraint;
        TupleDomain newUnenforcedConstraint;
        Set newConstraintColumns;
        if (predicate.isNone()) {
            // Engine does not pass none Constraint.summary. It can become none when combined with the expression and connector's domain knowledge.
            newEnforcedConstraint = TupleDomain.none();
            newUnenforcedConstraint = TupleDomain.all();
            newConstraintColumns = constraint.getPredicateColumns().stream()
                    .flatMap(Collection::stream)
                    .map(DeltaLakeColumnHandle.class::cast)
                    .collect(toImmutableSet());
        }
        else {
            Set partitionColumns = ImmutableSet.copyOf(extractPartitionColumns(tableHandle.getMetadataEntry(), tableHandle.getProtocolEntry(), typeManager));
            Map constraintDomains = predicate.getDomains().orElseThrow();

            ImmutableMap.Builder enforceableDomains = ImmutableMap.builder();
            ImmutableMap.Builder unenforceableDomains = ImmutableMap.builder();
            ImmutableSet.Builder constraintColumns = ImmutableSet.builder();
            // We need additional field to track partition columns used in queries as enforceDomains seem to be not catching
            // cases when partition columns is used within complex filter as 'partitionColumn % 2 = 0'
            constraint.getPredicateColumns().stream()
                    .flatMap(Collection::stream)
                    .map(DeltaLakeColumnHandle.class::cast)
                    .forEach(constraintColumns::add);
            for (Entry domainEntry : constraintDomains.entrySet()) {
                DeltaLakeColumnHandle column = (DeltaLakeColumnHandle) domainEntry.getKey();
                if (!partitionColumns.contains(column)) {
                    unenforceableDomains.put(column, domainEntry.getValue());
                }
                else {
                    enforceableDomains.put(column, domainEntry.getValue());
                }
                constraintColumns.add(column);
            }

            newEnforcedConstraint = TupleDomain.withColumnDomains(enforceableDomains.buildOrThrow());
            newUnenforcedConstraint = TupleDomain.withColumnDomains(unenforceableDomains.buildOrThrow());
            newConstraintColumns = constraintColumns.build();
        }

        DeltaLakeTableHandle newHandle = new DeltaLakeTableHandle(
                tableName.getSchemaName(),
                tableName.getTableName(),
                tableHandle.isManaged(),
                tableHandle.getLocation(),
                tableHandle.getMetadataEntry(),
                tableHandle.getProtocolEntry(),
                // Do not simplify the enforced constraint, the connector is guaranteeing the constraint will be applied as is.
                // The unenforced constraint will still be checked by the engine.
                tableHandle.getEnforcedPartitionConstraint()
                        .intersect(newEnforcedConstraint),
                tableHandle.getNonPartitionConstraint()
                        .intersect(newUnenforcedConstraint)
                        .simplify(domainCompactionThreshold),
                Sets.union(tableHandle.getConstraintColumns(), newConstraintColumns),
                tableHandle.getWriteType(),
                tableHandle.getProjectedColumns(),
                tableHandle.getUpdatedColumns(),
                tableHandle.getUpdateRowIdColumns(),
                Optional.empty(),
                false,
                false,
                Optional.empty(),
                tableHandle.getReadVersion());

        if (tableHandle.getEnforcedPartitionConstraint().equals(newHandle.getEnforcedPartitionConstraint()) &&
                tableHandle.getNonPartitionConstraint().equals(newHandle.getNonPartitionConstraint()) &&
                tableHandle.getConstraintColumns().equals(newHandle.getConstraintColumns())) {
            return Optional.empty();
        }

        return Optional.of(new ConstraintApplicationResult<>(
                newHandle,
                newUnenforcedConstraint.transformKeys(ColumnHandle.class::cast),
                extractionResult.remainingExpression(),
                false));
    }

    @Override
    public Optional> applyProjection(
            ConnectorSession session,
            ConnectorTableHandle tableHandle,
            List projections,
            Map assignments)
    {
        DeltaLakeTableHandle deltaLakeTableHandle = (DeltaLakeTableHandle) tableHandle;

        // Create projected column representations for supported sub expressions. Simple column references and chain of
        // dereferences on a variable are supported right now.
        Set projectedExpressions = projections.stream()
                .flatMap(expression -> extractSupportedProjectedColumns(expression).stream())
                .collect(toImmutableSet());

        Map columnProjections = projectedExpressions.stream()
                .collect(toImmutableMap(Function.identity(), ApplyProjectionUtil::createProjectedColumnRepresentation));

        // all references are simple variables
        if (!isProjectionPushdownEnabled(session)
                || columnProjections.values().stream().allMatch(ProjectedColumnRepresentation::isVariable)) {
            Set projectedColumns = assignments.values().stream()
                    .map(DeltaLakeColumnHandle.class::cast)
                    .collect(toImmutableSet());
            // Check if column was projected already in previous call
            if (deltaLakeTableHandle.getProjectedColumns().isPresent()
                    && deltaLakeTableHandle.getProjectedColumns().get().equals(projectedColumns)) {
                return Optional.empty();
            }

            List newColumnAssignments = assignments.entrySet().stream()
                    .map(assignment -> new Assignment(
                            assignment.getKey(),
                            assignment.getValue(),
                            ((DeltaLakeColumnHandle) assignment.getValue()).getBaseType()))
                    .collect(toImmutableList());

            return Optional.of(new ProjectionApplicationResult<>(
                    deltaLakeTableHandle.withProjectedColumns(projectedColumns),
                    projections,
                    newColumnAssignments,
                    false));
        }

        Map newAssignments = new HashMap<>();
        ImmutableMap.Builder newVariablesBuilder = ImmutableMap.builder();
        ImmutableSet.Builder projectedColumnsBuilder = ImmutableSet.builder();

        for (Map.Entry entry : columnProjections.entrySet()) {
            ConnectorExpression expression = entry.getKey();
            ProjectedColumnRepresentation projectedColumn = entry.getValue();

            DeltaLakeColumnHandle projectedColumnHandle;
            String projectedColumnName;

            // See if input already contains a columnhandle for this projected column, avoid creating duplicates.
            Optional existingColumn = find(assignments, projectedColumn);

            if (existingColumn.isPresent()) {
                projectedColumnName = existingColumn.get();
                projectedColumnHandle = (DeltaLakeColumnHandle) assignments.get(projectedColumnName);
            }
            else {
                // Create a new column handle
                DeltaLakeColumnHandle oldColumnHandle = (DeltaLakeColumnHandle) assignments.get(projectedColumn.getVariable().getName());
                projectedColumnHandle = projectColumn(oldColumnHandle, projectedColumn.getDereferenceIndices(), expression.getType(), getColumnMappingMode(deltaLakeTableHandle.getMetadataEntry(), deltaLakeTableHandle.getProtocolEntry()));
                projectedColumnName = projectedColumnHandle.getQualifiedPhysicalName();
            }

            Variable projectedColumnVariable = new Variable(projectedColumnName, expression.getType());
            Assignment newAssignment = new Assignment(projectedColumnName, projectedColumnHandle, expression.getType());
            newAssignments.putIfAbsent(projectedColumnName, newAssignment);

            newVariablesBuilder.put(expression, projectedColumnVariable);
            projectedColumnsBuilder.add(projectedColumnHandle);
        }

        // Modify projections to refer to new variables
        Map newVariables = newVariablesBuilder.buildOrThrow();
        List newProjections = projections.stream()
                .map(expression -> replaceWithNewVariables(expression, newVariables))
                .collect(toImmutableList());

        List outputAssignments = ImmutableList.copyOf(newAssignments.values());
        return Optional.of(new ProjectionApplicationResult<>(
                deltaLakeTableHandle.withProjectedColumns(projectedColumnsBuilder.build()),
                newProjections,
                outputAssignments,
                false));
    }

    private static DeltaLakeColumnHandle projectColumn(DeltaLakeColumnHandle column, List indices, Type projectedColumnType, ColumnMappingMode columnMappingMode)
    {
        if (indices.isEmpty()) {
            return column;
        }
        Optional existingProjectionInfo = column.getProjectionInfo();
        ImmutableList.Builder dereferenceNames = ImmutableList.builder();
        ImmutableList.Builder dereferenceIndices = ImmutableList.builder();

        if (!column.isBaseColumn()) {
            dereferenceNames.addAll(existingProjectionInfo.get().getDereferencePhysicalNames());
            dereferenceIndices.addAll(existingProjectionInfo.get().getDereferenceIndices());
        }

        Type columnType = switch (columnMappingMode) {
            case ID, NAME -> column.getBasePhysicalType();
            case NONE -> column.getBaseType();
            default -> throw new TrinoException(NOT_SUPPORTED, "Projecting columns with column mapping %s is not supported".formatted(columnMappingMode));
        };

        for (int index : dereferenceIndices.build()) {
            RowType.Field field = ((RowType) columnType).getFields().get(index);
            columnType = field.getType();
        }

        for (int index : indices) {
            RowType.Field field = ((RowType) columnType).getFields().get(index);
            dereferenceNames.add(field.getName().orElseThrow());
            columnType = field.getType();
        }
        dereferenceIndices.addAll(indices);

        DeltaLakeColumnProjectionInfo projectionInfo = new DeltaLakeColumnProjectionInfo(
                projectedColumnType,
                dereferenceIndices.build(),
                dereferenceNames.build());

        return new DeltaLakeColumnHandle(
                column.getBaseColumnName(),
                column.getBaseType(),
                column.getBaseFieldId(),
                column.getBasePhysicalColumnName(),
                column.getBasePhysicalType(),
                REGULAR,
                Optional.of(projectionInfo));
    }

    /**
     * Returns the assignment key corresponding to the column represented by {@param projectedColumn} in the {@param assignments}, if one exists.
     * The variable in the {@param projectedColumn} can itself be a representation of another projected column. For example,
     * say a projected column representation has variable "x" and a dereferenceIndices=[0]. "x" can in-turn map to a projected
     * column handle with base="a" and [1, 2] as dereference indices. Then the method searches for a column handle in
     * {@param assignments} with base="a" and dereferenceIndices=[1, 2, 0].
     */
    private static Optional find(Map assignments, ProjectedColumnRepresentation projectedColumn)
    {
        DeltaLakeColumnHandle variableColumn = (DeltaLakeColumnHandle) assignments.get(projectedColumn.getVariable().getName());

        requireNonNull(variableColumn, "variableColumn is null");

        String baseColumnName = variableColumn.getBaseColumnName();

        List variableColumnIndices = variableColumn.getProjectionInfo()
                .map(DeltaLakeColumnProjectionInfo::getDereferenceIndices)
                .orElse(ImmutableList.of());

        List projectionIndices = ImmutableList.builder()
                .addAll(variableColumnIndices)
                .addAll(projectedColumn.getDereferenceIndices())
                .build();

        for (Map.Entry entry : assignments.entrySet()) {
            DeltaLakeColumnHandle column = (DeltaLakeColumnHandle) entry.getValue();
            if (column.getBaseColumnName().equals(baseColumnName) &&
                    column.getProjectionInfo()
                            .map(DeltaLakeColumnProjectionInfo::getDereferenceIndices)
                            .orElse(ImmutableList.of())
                            .equals(projectionIndices)) {
                return Optional.of(entry.getKey());
            }
        }

        return Optional.empty();
    }

    @Override
    public void validateScan(ConnectorSession session, ConnectorTableHandle handle)
    {
        DeltaLakeTableHandle deltaLakeTableHandle = (DeltaLakeTableHandle) handle;

        if (isQueryPartitionFilterRequired(session)) {
            List partitionColumns = deltaLakeTableHandle.getMetadataEntry().getOriginalPartitionColumns();
            if (!partitionColumns.isEmpty()) {
                if (deltaLakeTableHandle.getAnalyzeHandle().isPresent()) {
                    throw new TrinoException(
                            QUERY_REJECTED,
                            "ANALYZE statement can not be performed on partitioned tables because filtering is required on at least one partition. However, the partition filtering check can be disabled with the catalog session property 'query_partition_filter_required'.");
                }
                Set referencedColumns =
                        deltaLakeTableHandle.getConstraintColumns().stream()
                                .map(DeltaLakeColumnHandle::getBaseColumnName)
                                .collect(toImmutableSet());
                if (Collections.disjoint(referencedColumns, partitionColumns)) {
                    throw new TrinoException(
                            QUERY_REJECTED,
                            format("Filter required on %s for at least one partition column: %s", deltaLakeTableHandle.getSchemaTableName(), String.join(", ", partitionColumns)));
                }
            }
        }
    }

    @Override
    public Optional applyTableScanRedirect(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        return deltaLakeRedirectionsProvider.getTableScanRedirection(session, (DeltaLakeTableHandle) tableHandle);
    }

    @Override
    public ConnectorAnalyzeMetadata getStatisticsCollectionMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, Map analyzeProperties)
    {
        if (!isExtendedStatisticsEnabled(session)) {
            throw new TrinoException(NOT_SUPPORTED, format(
                    "ANALYZE not supported if extended statistics are disabled. Enable via %s config property or %s session property.",
                    DeltaLakeConfig.EXTENDED_STATISTICS_ENABLED,
                    DeltaLakeSessionProperties.EXTENDED_STATISTICS_ENABLED));
        }

        DeltaLakeTableHandle handle = checkValidTableHandle(tableHandle);
        MetadataEntry metadata = handle.getMetadataEntry();

        Optional filesModifiedAfterFromProperties = getFilesModifiedAfterProperty(analyzeProperties);
        AnalyzeMode analyzeMode = getRefreshMode(analyzeProperties);

        Optional statistics = Optional.empty();
        if (analyzeMode == INCREMENTAL) {
            statistics = statisticsAccess.readExtendedStatistics(session, handle.getSchemaTableName(), handle.getLocation());
        }

        Optional alreadyAnalyzedModifiedTimeMax = statistics.map(ExtendedStatistics::getAlreadyAnalyzedModifiedTimeMax);

        // determine list of files we want to read based on what caller requested via files_modified_after and what files were already analyzed in the past
        Optional filesModifiedAfter = Optional.empty();
        if (filesModifiedAfterFromProperties.isPresent() || alreadyAnalyzedModifiedTimeMax.isPresent()) {
            filesModifiedAfter = Optional.of(Comparators.max(
                    filesModifiedAfterFromProperties.orElse(EPOCH),
                    alreadyAnalyzedModifiedTimeMax.orElse(EPOCH)));
        }

        List columnsMetadata = extractSchema(metadata, handle.getProtocolEntry(), typeManager);
        Set allColumnNames = columnsMetadata.stream().map(columnMetadata -> columnMetadata.getName().toLowerCase(ENGLISH)).collect(Collectors.toSet());
        Optional> analyzeColumnNames = getColumnNames(analyzeProperties);
        if (analyzeColumnNames.isPresent()) {
            Set columnNames = analyzeColumnNames.get();
            // validate that proper column names are passed via `columns` analyze property
            if (columnNames.isEmpty()) {
                throw new TrinoException(INVALID_ANALYZE_PROPERTY, "Cannot specify empty list of columns for analysis");
            }

            if (!allColumnNames.containsAll(columnNames)) {
                throw new TrinoException(
                        INVALID_ANALYZE_PROPERTY,
                        format("Invalid columns specified for analysis: %s", Sets.difference(columnNames, allColumnNames)));
            }
        }

        // verify that we do not extend set of analyzed columns
        Optional> oldAnalyzeColumnNames = statistics.flatMap(ExtendedStatistics::getAnalyzedColumns);
        if (oldAnalyzeColumnNames.isPresent()) {
            if (analyzeColumnNames.isEmpty() || !oldAnalyzeColumnNames.get().containsAll(analyzeColumnNames.get())) {
                throw new TrinoException(INVALID_ANALYZE_PROPERTY, format(
                        "List of columns to be analyzed must be a subset of previously used: %s. To extend list of analyzed columns drop table statistics",
                        oldAnalyzeColumnNames.get()));
            }
        }

        AnalyzeHandle analyzeHandle = new AnalyzeHandle(statistics.isEmpty() ? FULL_REFRESH : INCREMENTAL, filesModifiedAfter, analyzeColumnNames);
        DeltaLakeTableHandle newHandle = new DeltaLakeTableHandle(
                handle.getSchemaTableName().getSchemaName(),
                handle.getSchemaTableName().getTableName(),
                handle.isManaged(),
                handle.getLocation(),
                metadata,
                handle.getProtocolEntry(),
                TupleDomain.all(),
                TupleDomain.all(),
                Optional.empty(),
                Optional.empty(),
                Optional.empty(),
                Optional.empty(),
                Optional.of(analyzeHandle),
                handle.getReadVersion());

        TableStatisticsMetadata statisticsMetadata = getStatisticsCollectionMetadata(
                columnsMetadata.stream().map(DeltaLakeColumnMetadata::getColumnMetadata).collect(toImmutableList()),
                analyzeColumnNames.orElse(allColumnNames),
                statistics.isPresent(),
                false);

        return new ConnectorAnalyzeMetadata(newHandle, statisticsMetadata);
    }

    @Override
    public TableStatisticsMetadata getStatisticsCollectionMetadataForWrite(ConnectorSession session, ConnectorTableMetadata tableMetadata)
    {
        if (!isCollectExtendedStatisticsColumnStatisticsOnWrite(session)) {
            return TableStatisticsMetadata.empty();
        }

        Set allColumnNames = tableMetadata.getColumns().stream()
                .map(ColumnMetadata::getName)
                .collect(toImmutableSet());

        Optional> analyzeColumnNames = Optional.empty();
        String tableLocation = getLocation(tableMetadata.getProperties());
        Optional existingStatistics = Optional.empty();
        if (tableLocation != null) {
            existingStatistics = statisticsAccess.readExtendedStatistics(session, tableMetadata.getTable(), tableLocation);
            analyzeColumnNames = existingStatistics.flatMap(ExtendedStatistics::getAnalyzedColumns);
        }

        return getStatisticsCollectionMetadata(
                tableMetadata.getColumns(),
                analyzeColumnNames.orElse(allColumnNames),
                existingStatistics.isPresent(),
                true);
    }

    private TableStatisticsMetadata getStatisticsCollectionMetadata(
            List tableColumns,
            Set analyzeColumnNames,
            boolean extendedStatisticsExists,
            boolean isCollectionOnWrite)
    {
        // Collect file statistics only when performing ANALYZE on a table without extended statistics
        boolean collectFileStatistics = !extendedStatisticsExists && !isCollectionOnWrite;

        ImmutableSet.Builder columnStatistics = ImmutableSet.builder();
        tableColumns.stream()
                .filter(DeltaLakeMetadata::shouldCollectExtendedStatistics)
                .filter(columnMetadata -> analyzeColumnNames.contains(columnMetadata.getName()))
                .forEach(columnMetadata -> {
                    if (!(columnMetadata.getType() instanceof FixedWidthType)) {
                        columnStatistics.add(new ColumnStatisticMetadata(columnMetadata.getName(), TOTAL_SIZE_IN_BYTES));
                    }
                    columnStatistics.add(new ColumnStatisticMetadata(columnMetadata.getName(), NUMBER_OF_DISTINCT_VALUES_SUMMARY));
                    if (collectFileStatistics) {
                        // TODO: (https://github.com/trinodb/trino/issues/17055) Collect file level stats for VARCHAR type
                        if (!columnMetadata.getType().equals(VARCHAR)
                                && !columnMetadata.getType().equals(BOOLEAN)
                                && !columnMetadata.getType().equals(VARBINARY)) {
                            columnStatistics.add(new ColumnStatisticMetadata(columnMetadata.getName(), MIN_VALUE));
                            columnStatistics.add(new ColumnStatisticMetadata(columnMetadata.getName(), MAX_VALUE));
                        }
                        columnStatistics.add(new ColumnStatisticMetadata(columnMetadata.getName(), NUMBER_OF_NON_NULL_VALUES));
                    }
                });

        if (!isCollectionOnWrite) {
            // collect max(file modification time) for sake of incremental ANALYZE
            // File modified time does not need to be collected as a statistics because it gets derived directly from files being written
            columnStatistics.add(new ColumnStatisticMetadata(FILE_MODIFIED_TIME_COLUMN_NAME, MAX_VALUE));
        }

        Set tableStatistics = ImmutableSet.of();
        List groupingColumns = ImmutableList.of();
        if (collectFileStatistics) {
            tableStatistics = ImmutableSet.of(TableStatisticType.ROW_COUNT);
            groupingColumns = ImmutableList.of(PATH_COLUMN_NAME);
        }

        return new TableStatisticsMetadata(
                columnStatistics.build(),
                tableStatistics,
                groupingColumns);
    }

    private static boolean shouldCollectExtendedStatistics(ColumnMetadata columnMetadata)
    {
        if (columnMetadata.isHidden()) {
            return false;
        }
        Type type = columnMetadata.getType();
        if (type instanceof MapType || type instanceof RowType || type instanceof ArrayType) {
            return false;
        }
        return true;
    }

    @Override
    public ConnectorTableHandle beginStatisticsCollection(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        // nothing to be done here
        return tableHandle;
    }

    @Override
    public void finishStatisticsCollection(ConnectorSession session, ConnectorTableHandle table, Collection computedStatistics)
    {
        DeltaLakeTableHandle tableHandle = (DeltaLakeTableHandle) table;
        AnalyzeHandle analyzeHandle = tableHandle.getAnalyzeHandle().orElseThrow(() -> new IllegalArgumentException("analyzeHandle not set"));
        if (analyzeHandle.getAnalyzeMode() == FULL_REFRESH) {
            // TODO: Populate stats for incremental ANALYZE https://github.com/trinodb/trino/issues/18110
            generateMissingFileStatistics(session, tableHandle, computedStatistics);
        }
        Optional maxFileModificationTime = getMaxFileModificationTime(computedStatistics);
        Map physicalColumnNameMapping = extractSchema(tableHandle.getMetadataEntry(), tableHandle.getProtocolEntry(), typeManager).stream()
                .collect(toImmutableMap(DeltaLakeColumnMetadata::getName, DeltaLakeColumnMetadata::getPhysicalName));
        updateTableStatistics(
                session,
                Optional.of(analyzeHandle),
                tableHandle.getSchemaTableName(),
                tableHandle.getLocation(),
                maxFileModificationTime,
                computedStatistics,
                getExactColumnNames(tableHandle.getMetadataEntry()),
                Optional.of(physicalColumnNameMapping));
    }

    private void generateMissingFileStatistics(ConnectorSession session, DeltaLakeTableHandle tableHandle, Collection computedStatistics)
    {
        Map addFileEntriesWithNoStats = transactionLogAccess.getActiveFiles(
                        getSnapshot(session, tableHandle), tableHandle.getMetadataEntry(), tableHandle.getProtocolEntry(), session)
                .stream()
                .filter(addFileEntry -> addFileEntry.getStats().isEmpty()
                        || addFileEntry.getStats().get().getNumRecords().isEmpty()
                        || addFileEntry.getStats().get().getMaxValues().isEmpty()
                        || addFileEntry.getStats().get().getMinValues().isEmpty()
                        || addFileEntry.getStats().get().getNullCount().isEmpty())
                .filter(addFileEntry -> !URI.create(addFileEntry.getPath()).isAbsolute()) // TODO: Support absolute paths https://github.com/trinodb/trino/issues/18277
                // Statistics returns whole path to file build in DeltaLakeSplitManager, so we need to create corresponding map key for AddFileEntry.
                .collect(toImmutableMap(addFileEntry -> DeltaLakeSplitManager.buildSplitPath(Location.of(tableHandle.getLocation()), addFileEntry).toString(), identity()));

        if (addFileEntriesWithNoStats.isEmpty()) {
            return;
        }

        Map lowercaseToColumnsHandles = getColumns(tableHandle.getMetadataEntry(), tableHandle.getProtocolEntry()).stream()
                .filter(column -> column.getColumnType() == REGULAR)
                .collect(toImmutableMap(columnHandle -> columnHandle.getBaseColumnName().toLowerCase(ENGLISH), identity()));

        List updatedAddFileEntries = computedStatistics.stream()
                .map(statistics -> {
                    // Grouping by `PATH_COLUMN_NAME`.
                    String filePathFromStatistics = VARCHAR.getSlice(statistics.getGroupingValues().get(0), 0).toStringUtf8();
                    // Check if collected statistics are for files without stats.
                    // If AddFileEntry is present in addFileEntriesWithNoStats means that it does not have statistics so prepare updated entry.
                    // If null is returned from addFileEntriesWithNoStats means that statistics are present, and we don't need to do anything.
                    AddFileEntry addFileEntry = addFileEntriesWithNoStats.get(filePathFromStatistics);
                    if (addFileEntry != null) {
                        return Optional.of(prepareUpdatedAddFileEntry(statistics, addFileEntry, lowercaseToColumnsHandles));
                    }
                    return Optional.empty();
                })
                .flatMap(Optional::stream)
                .collect(toImmutableList());

        if (updatedAddFileEntries.isEmpty()) {
            return;
        }
        try {
            long createdTime = Instant.now().toEpochMilli();
            long readVersion = tableHandle.getReadVersion();
            long commitVersion = readVersion + 1;
            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, tableHandle.getLocation());
            transactionLogWriter.appendCommitInfoEntry(getCommitInfoEntry(session, commitVersion, createdTime, OPTIMIZE_OPERATION, readVersion));
            updatedAddFileEntries.forEach(transactionLogWriter::appendAddFileEntry);
            transactionLogWriter.flush();
        }
        catch (Throwable e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, "Unable to access file system for: " + tableHandle.getLocation(), e);
        }
    }

    private AddFileEntry prepareUpdatedAddFileEntry(ComputedStatistics stats, AddFileEntry addFileEntry, Map lowercaseToColumnsHandles)
    {
        DeltaLakeJsonFileStatistics deltaLakeJsonFileStatistics = DeltaLakeComputedStatistics.toDeltaLakeJsonFileStatistics(stats, lowercaseToColumnsHandles);
        try {
            return new AddFileEntry(
                    addFileEntry.getPath(),
                    addFileEntry.getPartitionValues(), // preserve original case without canonicalization
                    addFileEntry.getSize(),
                    addFileEntry.getModificationTime(),
                    false,
                    Optional.of(serializeStatsAsJson(deltaLakeJsonFileStatistics)),
                    Optional.empty(),
                    addFileEntry.getTags(),
                    addFileEntry.getDeletionVector());
        }
        catch (JsonProcessingException e) {
            throw new TrinoException(GENERIC_INTERNAL_ERROR, "Statistics serialization error", e);
        }
    }

    private void updateTableStatistics(
            ConnectorSession session,
            Optional analyzeHandle,
            SchemaTableName schemaTableName,
            String location,
            Optional maxFileModificationTime,
            Collection computedStatistics,
            List originalColumnNames,
            Optional> physicalColumnNameMapping)
    {
        Optional oldStatistics = Optional.empty();
        boolean loadExistingStats = analyzeHandle.isEmpty() || analyzeHandle.get().getAnalyzeMode() == INCREMENTAL;
        if (loadExistingStats) {
            oldStatistics = statisticsAccess.readExtendedStatistics(session, schemaTableName, location);
        }

        // more elaborate logic for handling statistics model evaluation may need to be introduced in the future
        // for now let's have a simple check rejecting update
        oldStatistics.ifPresent(statistics ->
                checkArgument(
                        statistics.getModelVersion() == ExtendedStatistics.CURRENT_MODEL_VERSION,
                        "Existing table statistics are incompatible, run the drop statistics procedure on this table before re-analyzing"));

        Map lowerCaseToExactColumnNames = originalColumnNames.stream()
                .collect(toImmutableMap(name -> name.toLowerCase(ENGLISH), identity()));

        Map oldColumnStatistics = oldStatistics.map(ExtendedStatistics::getColumnStatistics)
                .orElseGet(ImmutableMap::of);
        Map newColumnStatistics = toDeltaLakeColumnStatistics(computedStatistics);

        Map mergedColumnStatistics = newColumnStatistics.entrySet().stream()
                .map(entry -> {
                    String columnName = entry.getKey();
                    String physicalColumnName = toPhysicalColumnName(columnName, lowerCaseToExactColumnNames, physicalColumnNameMapping);
                    return Map.entry(physicalColumnName, entry.getValue());
                })
                .collect(toImmutableMap(
                        Entry::getKey,
                        entry -> {
                            String columnName = entry.getKey();
                            DeltaLakeColumnStatistics newStats = entry.getValue();
                            DeltaLakeColumnStatistics oldStats = oldColumnStatistics.get(columnName);
                            return oldStats == null
                                    ? newStats
                                    : oldStats.update(newStats);
                        }));

        // We do not want to hinder our future calls to ANALYZE if one of the files we analyzed have modification time far in the future.
        // Therefore we cap the value we store in extended_stats.json to current_time as observed on Trino coordinator.
        Instant finalAlreadyAnalyzedModifiedTimeMax = Instant.now();
        if (maxFileModificationTime.isPresent()) {
            finalAlreadyAnalyzedModifiedTimeMax = Comparators.min(maxFileModificationTime.get(), finalAlreadyAnalyzedModifiedTimeMax);
        }
        // also ensure that we are not traveling back in time
        if (oldStatistics.isPresent()) {
            finalAlreadyAnalyzedModifiedTimeMax = Comparators.max(oldStatistics.get().getAlreadyAnalyzedModifiedTimeMax(), finalAlreadyAnalyzedModifiedTimeMax);
        }

        Optional> analyzedColumns = analyzeHandle.flatMap(AnalyzeHandle::getColumns);
        // If update is invoked by other command than ANALYZE, statistics should preserve previous columns set.
        if (analyzeHandle.isEmpty()) {
            analyzedColumns = oldStatistics.flatMap(ExtendedStatistics::getAnalyzedColumns);
        }

        analyzedColumns.ifPresent(analyzeColumns -> {
            Set analyzePhysicalColumns = analyzeColumns.stream()
                    .map(columnName -> toPhysicalColumnName(columnName, lowerCaseToExactColumnNames, physicalColumnNameMapping))
                    .collect(toImmutableSet());
            if (!mergedColumnStatistics.keySet().equals(analyzePhysicalColumns)) {
                // sanity validation
                throw new IllegalStateException(format("Unexpected columns in in mergedColumnStatistics %s; expected %s", mergedColumnStatistics.keySet(), analyzePhysicalColumns));
            }
        });

        ExtendedStatistics mergedExtendedStatistics = new ExtendedStatistics(
                finalAlreadyAnalyzedModifiedTimeMax,
                mergedColumnStatistics,
                analyzedColumns);

        statisticsAccess.updateExtendedStatistics(session, schemaTableName, location, mergedExtendedStatistics);
    }

    private static String toPhysicalColumnName(String columnName, Map lowerCaseToExactColumnNames, Optional> physicalColumnNameMapping)
    {
        String originalColumnName = lowerCaseToExactColumnNames.get(columnName.toLowerCase(ENGLISH));
        checkArgument(originalColumnName != null, "%s doesn't contain '%s'", lowerCaseToExactColumnNames.keySet(), columnName);
        if (physicalColumnNameMapping.isPresent()) {
            String physicalColumnName = physicalColumnNameMapping.get().get(originalColumnName);
            return requireNonNull(physicalColumnName, () -> "%s doesn't exist in %s".formatted(columnName, physicalColumnNameMapping));
        }
        return originalColumnName;
    }

    private void cleanExtraOutputFiles(ConnectorSession session, Location baseLocation, List validDataFiles)
    {
        Set writtenFilePaths = validDataFiles.stream()
                .map(dataFileInfo -> baseLocation.appendPath(dataFileInfo.getPath()))
                .collect(toImmutableSet());

        cleanExtraOutputFiles(session, writtenFilePaths);
    }

    private void cleanExtraOutputFiles(ConnectorSession session, Set validWrittenFilePaths)
    {
        Set fileLocations = validWrittenFilePaths.stream()
                .map(Location::parentDirectory)
                .collect(toImmutableSet());

        for (Location location : fileLocations) {
            cleanExtraOutputFiles(session, session.getQueryId(), location, validWrittenFilePaths);
        }
    }

    private void cleanExtraOutputFiles(ConnectorSession session, String queryId, Location location, Set filesToKeep)
    {
        Deque filesToDelete = new ArrayDeque<>();
        try {
            LOG.debug("Deleting failed attempt files from %s for query %s", location, queryId);
            TrinoFileSystem fileSystem = fileSystemFactory.create(session);

            // files within given partition are written flat into location; we need to list recursively
            FileIterator iterator = fileSystem.listFiles(location);
            while (iterator.hasNext()) {
                Location file = iterator.next().location();
                if (!file.parentDirectory().equals(location)) {
                    // we do not want recursive listing
                    continue;
                }
                if (isFileCreatedByQuery(file, queryId) && !filesToKeep.contains(file)) {
                    filesToDelete.add(file);
                }
            }

            if (filesToDelete.isEmpty()) {
                return;
            }

            LOG.info("Found %s files to delete and %s to retain in location %s for query %s", filesToDelete.size(), filesToKeep.size(), location, queryId);
            fileSystem.deleteFiles(filesToDelete);
        }
        catch (IOException e) {
            throw new TrinoException(DELTA_LAKE_FILESYSTEM_ERROR, "Failed to clean up extraneous output files", e);
        }
    }

    private static boolean isFileCreatedByQuery(Location file, String queryId)
    {
        verify(!queryId.contains("-"), "queryId(%s) should not contain hyphens", queryId);
        return file.fileName().startsWith(queryId + "-");
    }

    @Override
    public Optional getSystemTable(ConnectorSession session, SchemaTableName tableName)
    {
        return getRawSystemTable(tableName).map(systemTable -> new ClassLoaderSafeSystemTable(systemTable, getClass().getClassLoader()));
    }

    private Optional getRawSystemTable(SchemaTableName systemTableName)
    {
        Optional tableType = DeltaLakeTableName.tableTypeFrom(systemTableName.getTableName());
        if (tableType.isEmpty() || tableType.get() == DeltaLakeTableType.DATA) {
            return Optional.empty();
        }

        String tableName = DeltaLakeTableName.tableNameFrom(systemTableName.getTableName());
        Optional table;
        try {
            table = metastore.getTable(systemTableName.getSchemaName(), tableName);
        }
        catch (NotADeltaLakeTableException e) {
            return Optional.empty();
        }
        if (table.isEmpty()) {
            return Optional.empty();
        }

        String tableLocation = table.get().location();

        return switch (tableType.get()) {
            case DATA -> throw new VerifyException("Unexpected DATA table type"); // Handled above.
            case HISTORY -> Optional.of(new DeltaLakeHistoryTable(
                    systemTableName,
                    tableLocation,
                    fileSystemFactory,
                    transactionLogAccess,
                    typeManager));
            case PROPERTIES -> Optional.of(new DeltaLakePropertiesTable(systemTableName, tableLocation, transactionLogAccess));
        };
    }

    @Override
    public WriterScalingOptions getNewTableWriterScalingOptions(ConnectorSession session, SchemaTableName tableName, Map tableProperties)
    {
        return WriterScalingOptions.ENABLED;
    }

    @Override
    public WriterScalingOptions getInsertWriterScalingOptions(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        return WriterScalingOptions.ENABLED;
    }

    @Override
    public void truncateTable(ConnectorSession session, ConnectorTableHandle tableHandle)
    {
        executeDelete(session, checkValidTableHandle(tableHandle), TRUNCATE_OPERATION);
    }

    @Override
    public Optional applyDelete(ConnectorSession session, ConnectorTableHandle handle)
    {
        DeltaLakeTableHandle tableHandle = (DeltaLakeTableHandle) handle;
        if (changeDataFeedEnabled(tableHandle.getMetadataEntry(), tableHandle.getProtocolEntry()).orElse(false)) {
            // For tables with CDF enabled the DELETE operation can't be performed only on metadata files
            return Optional.empty();
        }

        return Optional.of(tableHandle);
    }

    @Override
    public OptionalLong executeDelete(ConnectorSession session, ConnectorTableHandle handle)
    {
        return executeDelete(session, handle, DELETE_OPERATION);
    }

    private OptionalLong executeDelete(ConnectorSession session, ConnectorTableHandle handle, String operation)
    {
        DeltaLakeTableHandle tableHandle = (DeltaLakeTableHandle) handle;
        if (isAppendOnly(tableHandle.getMetadataEntry(), tableHandle.getProtocolEntry())) {
            throw new TrinoException(NOT_SUPPORTED, "Cannot modify rows from a table with '" + APPEND_ONLY_CONFIGURATION_KEY + "' set to true");
        }
        checkWriteAllowed(session, tableHandle);
        checkWriteSupported(tableHandle);

        String tableLocation = tableHandle.location();
        List activeFiles = getAddFileEntriesMatchingEnforcedPartitionConstraint(session, tableHandle);

        try {
            TransactionLogWriter transactionLogWriter = transactionLogWriterFactory.newWriter(session, tableLocation);

            long writeTimestamp = Instant.now().toEpochMilli();
            TrinoFileSystem fileSystem = fileSystemFactory.create(session);
            long currentVersion = getMandatoryCurrentVersion(fileSystem, tableLocation);
            if (currentVersion != tableHandle.getReadVersion()) {
                throw new TransactionConflictException(format("Conflicting concurrent writes found. Expected transaction log version: %s, actual version: %s", tableHandle.getReadVersion(), currentVersion));
            }
            long commitVersion = currentVersion + 1;
            transactionLogWriter.appendCommitInfoEntry(getCommitInfoEntry(session, commitVersion, writeTimestamp, operation, tableHandle.getReadVersion()));

            long deletedRecords = 0L;
            boolean allDeletedFilesStatsPresent = true;
            for (AddFileEntry addFileEntry : activeFiles) {
                transactionLogWriter.appendRemoveFileEntry(new RemoveFileEntry(addFileEntry.getPath(), writeTimestamp, true));

                Optional fileRecords = addFileEntry.getStats().flatMap(DeltaLakeFileStatistics::getNumRecords);
                allDeletedFilesStatsPresent &= fileRecords.isPresent();
                deletedRecords += fileRecords.orElse(0L);
            }

            transactionLogWriter.flush();
            writeCheckpointIfNeeded(
                    session,
                    tableHandle.getSchemaTableName(),
                    tableHandle.location(),
                    tableHandle.getReadVersion(),
                    tableHandle.getMetadataEntry().getCheckpointInterval(),
                    commitVersion);
            return allDeletedFilesStatsPresent ? OptionalLong.of(deletedRecords) : OptionalLong.empty();
        }
        catch (Exception e) {
            throw new TrinoException(DELTA_LAKE_BAD_WRITE, "Failed to write Delta Lake transaction log entry", e);
        }
    }

    private List getAddFileEntriesMatchingEnforcedPartitionConstraint(ConnectorSession session, DeltaLakeTableHandle tableHandle)
    {
        TableSnapshot tableSnapshot = getSnapshot(session, tableHandle);
        List validDataFiles = transactionLogAccess.getActiveFiles(
                tableSnapshot,
                tableHandle.getMetadataEntry(),
                tableHandle.getProtocolEntry(),
                tableHandle.getEnforcedPartitionConstraint(),
                tableHandle.getProjectedColumns(),
                session);
        TupleDomain enforcedPartitionConstraint = tableHandle.getEnforcedPartitionConstraint();
        if (enforcedPartitionConstraint.isAll()) {
            return validDataFiles;
        }
        Map enforcedDomains = enforcedPartitionConstraint.getDomains().orElseThrow();
        return validDataFiles.stream()
                .filter(addAction -> partitionMatchesPredicate(addAction.getCanonicalPartitionValues(), enforcedDomains))
                .collect(toImmutableList());
    }

    private static Map toDeltaLakeColumnStatistics(Collection computedStatistics)
    {
        return computedStatistics.stream()
                .map(statistics -> createColumnToComputedStatisticsMap(statistics.getColumnStatistics()).entrySet().stream()
                        .collect(toImmutableMap(Entry::getKey, entry -> createDeltaLakeColumnStatistics(entry.getValue()))))
                .map(Map::entrySet)
                .flatMap(Collection::stream)
                .collect(toImmutableMap(Entry::getKey, Entry::getValue, DeltaLakeColumnStatistics::update));
    }

    private static Map> createColumnToComputedStatisticsMap(Map computedStatistics)
    {
        ImmutableTable.Builder result = ImmutableTable.builder();
        computedStatistics.forEach((metadata, block) -> {
            if (metadata.getColumnName().equals(FILE_MODIFIED_TIME_COLUMN_NAME)) {
                return;
            }
            if (!SUPPORTED_STATISTICS_TYPE.contains(metadata.getStatisticType())) {
                throw new TrinoException(GENERIC_INTERNAL_ERROR, "Unexpected statistics collection: " + metadata);
            }

            result.put(metadata.getColumnName(), metadata.getStatisticType(), block);
        });
        return result.buildOrThrow().rowMap();
    }

    private static DeltaLakeColumnStatistics createDeltaLakeColumnStatistics(Map computedStatistics)
    {
        OptionalLong totalSize = OptionalLong.empty();
        if (computedStatistics.containsKey(TOTAL_SIZE_IN_BYTES)) {
            totalSize = getLongValue(computedStatistics.get(TOTAL_SIZE_IN_BYTES));
        }
        HyperLogLog ndvSummary = getHyperLogLogForNdv(computedStatistics.get(NUMBER_OF_DISTINCT_VALUES_SUMMARY));
        return DeltaLakeColumnStatistics.create(totalSize, ndvSummary);
    }

    private static OptionalLong getLongValue(Block block)
    {
        if (block.isNull(0)) {
            return OptionalLong.empty();
        }
        return OptionalLong.of(BIGINT.getLong(block, 0));
    }

    private static HyperLogLog getHyperLogLogForNdv(Block block)
    {
        if (block.isNull(0)) {
            return HyperLogLog.newInstance(4096); // number of buckets used by $approx_set
        }
        Slice serializedSummary = (Slice) blockToNativeValue(HyperLogLogType.HYPER_LOG_LOG, block);
        return HyperLogLog.newInstance(serializedSummary);
    }

    private static Optional getMaxFileModificationTime(Collection computedStatistics)
    {
        return computedStatistics.stream()
                .map(ComputedStatistics::getColumnStatistics)
                .map(Map::entrySet)
                .flatMap(Collection::stream)
                .filter(entry -> entry.getKey().getColumnName().equals(FILE_MODIFIED_TIME_COLUMN_NAME))
                .flatMap(entry -> {
                    ColumnStatisticMetadata columnStatisticMetadata = entry.getKey();
                    if (columnStatisticMetadata.getStatisticType() != MAX_VALUE) {
                        throw new TrinoException(GENERIC_INTERNAL_ERROR, "Unexpected statistics collection: " + columnStatisticMetadata);
                    }
                    if (entry.getValue().isNull(0)) {
                        return Stream.of();
                    }
                    return Stream.of(Instant.ofEpochMilli(unpackMillisUtc(TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS.getLong(entry.getValue(), 0))));
                })
                .max(naturalOrder());
    }

    public DeltaLakeMetastore getMetastore()
    {
        return metastore;
    }

    private static ColumnMetadata getColumnMetadata(DeltaLakeColumnHandle column, @Nullable String comment, boolean nullability, @Nullable String generation)
    {
        String columnName;
        Type columnType;
        if (column.isBaseColumn()) {
            columnName = column.getBaseColumnName();
            columnType = column.getBaseType();
        }
        else {
            DeltaLakeColumnProjectionInfo projectionInfo = column.getProjectionInfo().get();
            columnName = column.getQualifiedPhysicalName();
            columnType = projectionInfo.getType();
        }
        return ColumnMetadata.builder()
                .setName(columnName)
                .setType(columnType)
                .setHidden(column.getColumnType() == SYNTHESIZED)
                .setComment(Optional.ofNullable(comment))
                .setNullable(nullability)
                .setExtraInfo(generation == null ? Optional.empty() : Optional.of("generated: " + generation))
                .build();
    }

    public static DeltaLakeTableHandle checkValidTableHandle(ConnectorTableHandle tableHandle)
    {
        requireNonNull(tableHandle, "tableHandle is null");
        if (tableHandle instanceof CorruptedDeltaLakeTableHandle corruptedTableHandle) {
            throw corruptedTableHandle.createException();
        }
        return ((DeltaLakeTableHandle) tableHandle);
    }

    public static TupleDomain createStatisticsPredicate(
            AddFileEntry addFileEntry,
            List schema,
            List canonicalPartitionColumns)
    {
        return addFileEntry.getStats()
                .map(deltaLakeFileStatistics -> withColumnDomains(
                        schema.stream()
                                .filter(column -> canUseInPredicate(column.getColumnMetadata()))
                                .collect(toImmutableMap(
                                        column -> DeltaLakeMetadata.toColumnHandle(column.getName(), column.getType(), column.getFieldId(), column.getPhysicalName(), column.getPhysicalColumnType(), canonicalPartitionColumns),
                                        column -> buildColumnDomain(column, deltaLakeFileStatistics, canonicalPartitionColumns)))))
                .orElseGet(TupleDomain::all);
    }

    private static boolean canUseInPredicate(ColumnMetadata column)
    {
        Type type = column.getType();
        return type.equals(TINYINT)
                || type.equals(SMALLINT)
                || type.equals(INTEGER)
                || type.equals(BIGINT)
                || type.equals(REAL)
                || type.equals(DOUBLE)
                || type.equals(BOOLEAN)
                || type.equals(DATE)
                || type instanceof TimestampWithTimeZoneType
                || type instanceof DecimalType
                || type.equals(VARCHAR);
    }

    private static Domain buildColumnDomain(DeltaLakeColumnMetadata column, DeltaLakeFileStatistics stats, List canonicalPartitionColumns)
    {
        Optional nullCount = stats.getNullCount(column.getPhysicalName());
        if (nullCount.isEmpty()) {
            // No stats were collected for this column; this can happen in 2 scenarios:
            // 1. The column didn't exist in the schema when the data file was created
            // 2. The column does exist in the file, but Spark property 'delta.dataSkippingNumIndexedCols'
            //    was used to limit the number of columns for which stats are collected
            // Since we don't know which scenario we're dealing with, we can't make a decision to prune.
            return Domain.all(column.getType());
        }
        if (stats.getNumRecords().equals(nullCount)) {
            return Domain.onlyNull(column.getType());
        }

        boolean hasNulls = nullCount.get() > 0;
        DeltaLakeColumnHandle deltaLakeColumnHandle = toColumnHandle(column.getName(), column.getType(), column.getFieldId(), column.getPhysicalName(), column.getPhysicalColumnType(), canonicalPartitionColumns);
        Optional minValue = stats.getMinColumnValue(deltaLakeColumnHandle);
        if (minValue.isPresent() && isFloatingPointNaN(column.getType(), minValue.get())) {
            return allValues(column.getType(), hasNulls);
        }
        if (isNotFinite(minValue, column.getType())) {
            minValue = Optional.empty();
        }
        Optional maxValue = stats.getMaxColumnValue(deltaLakeColumnHandle);
        if (maxValue.isPresent() && isFloatingPointNaN(column.getType(), maxValue.get())) {
            return allValues(column.getType(), hasNulls);
        }
        if (isNotFinite(maxValue, column.getType())) {
            maxValue = Optional.empty();
        }
        if (minValue.isPresent() && maxValue.isPresent()) {
            return Domain.create(
                    ofRanges(range(column.getType(), minValue.get(), true, maxValue.get(), true)),
                    hasNulls);
        }
        if (minValue.isPresent()) {
            return Domain.create(ofRanges(greaterThanOrEqual(column.getType(), minValue.get())), hasNulls);
        }

        return maxValue
                .map(value -> Domain.create(ofRanges(lessThanOrEqual(column.getType(), value)), hasNulls))
                .orElseGet(() -> Domain.all(column.getType()));
    }

    private static boolean isNotFinite(Optional value, Type type)
    {
        if (type.equals(DOUBLE)) {
            return value
                    .map(Double.class::cast)
                    .filter(val -> !Double.isFinite(val))
                    .isPresent();
        }
        if (type.equals(REAL)) {
            return value
                    .map(Long.class::cast)
                    .map(Math::toIntExact)
                    .map(Float::intBitsToFloat)
                    .filter(val -> !Float.isFinite(val))
                    .isPresent();
        }
        return false;
    }

    private static Domain allValues(Type type, boolean includeNull)
    {
        if (includeNull) {
            return Domain.all(type);
        }
        return Domain.notNull(type);
    }

    private static DeltaLakeColumnHandle toColumnHandle(String originalName, Type type, OptionalInt fieldId, String physicalName, Type physicalType, Collection partitionColumns)
    {
        boolean isPartitionKey = partitionColumns.stream().anyMatch(partition -> partition.equalsIgnoreCase(originalName));
        return new DeltaLakeColumnHandle(
                originalName,
                type,
                fieldId,
                physicalName,
                physicalType,
                isPartitionKey ? PARTITION_KEY : REGULAR,
                Optional.empty());
    }

    private static Optional getQueryId(Database database)
    {
        return Optional.ofNullable(database.getParameters().get(TRINO_QUERY_ID_NAME));
    }

    public static Optional getQueryId(Table table)
    {
        return Optional.ofNullable(table.getParameters().get(TRINO_QUERY_ID_NAME));
    }
}