private static ColStats getPaimonColStats()

in paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableStatsUtil.java [90:280]


    private static ColStats<?> getPaimonColStats(
            DataField field, CatalogColumnStatisticsDataBase colStat) {
        DataTypeRoot typeRoot = field.type().getTypeRoot();
        if (colStat instanceof CatalogColumnStatisticsDataString) {
            CatalogColumnStatisticsDataString stringColStat =
                    (CatalogColumnStatisticsDataString) colStat;
            if (typeRoot.equals(DataTypeRoot.CHAR) || typeRoot.equals(DataTypeRoot.VARCHAR)) {
                return ColStats.newColStats(
                        field.id(),
                        null != stringColStat.getNdv() ? stringColStat.getNdv() : null,
                        null,
                        null,
                        null != stringColStat.getNullCount() ? stringColStat.getNullCount() : null,
                        null != stringColStat.getAvgLength()
                                ? stringColStat.getAvgLength().longValue()
                                : null,
                        null != stringColStat.getMaxLength() ? stringColStat.getMaxLength() : null);
            }
        } else if (colStat instanceof CatalogColumnStatisticsDataBoolean) {
            CatalogColumnStatisticsDataBoolean booleanColStat =
                    (CatalogColumnStatisticsDataBoolean) colStat;
            if (typeRoot.equals(DataTypeRoot.BOOLEAN)) {
                return ColStats.newColStats(
                        field.id(),
                        (booleanColStat.getFalseCount() > 0 ? 1L : 0)
                                + (booleanColStat.getTrueCount() > 0 ? 1L : 0),
                        null,
                        null,
                        booleanColStat.getNullCount(),
                        null,
                        null);
            }
        } else if (colStat instanceof CatalogColumnStatisticsDataLong) {
            CatalogColumnStatisticsDataLong longColStat = (CatalogColumnStatisticsDataLong) colStat;
            if (typeRoot.equals(DataTypeRoot.INTEGER)) {
                return ColStats.newColStats(
                        field.id(),
                        null != longColStat.getNdv() ? longColStat.getNdv() : null,
                        null != longColStat.getMin() ? longColStat.getMin().intValue() : null,
                        null != longColStat.getMax() ? longColStat.getMax().intValue() : null,
                        null != longColStat.getNullCount() ? longColStat.getNullCount() : null,
                        null,
                        null);
            } else if (typeRoot.equals(DataTypeRoot.TINYINT)) {
                return ColStats.newColStats(
                        field.id(),
                        null != longColStat.getNdv() ? longColStat.getNdv() : null,
                        null != longColStat.getMin() ? longColStat.getMin().byteValue() : null,
                        null != longColStat.getMax() ? longColStat.getMax().byteValue() : null,
                        null != longColStat.getNullCount() ? longColStat.getNullCount() : null,
                        null,
                        null);

            } else if (typeRoot.equals(DataTypeRoot.SMALLINT)) {
                return ColStats.newColStats(
                        field.id(),
                        null != longColStat.getNdv() ? longColStat.getNdv() : null,
                        null != longColStat.getMin() ? longColStat.getMin().shortValue() : null,
                        null != longColStat.getMax() ? longColStat.getMax().shortValue() : null,
                        null != longColStat.getNullCount() ? longColStat.getNullCount() : null,
                        null,
                        null);
            } else if (typeRoot.equals(DataTypeRoot.BIGINT)) {
                return ColStats.newColStats(
                        field.id(),
                        null != longColStat.getNdv() ? longColStat.getNdv() : null,
                        null != longColStat.getMin() ? longColStat.getMin() : null,
                        null != longColStat.getMax() ? longColStat.getMax() : null,
                        null != longColStat.getNullCount() ? longColStat.getNullCount() : null,
                        null,
                        null);
            } else if (typeRoot.equals(DataTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE)) {
                return ColStats.newColStats(
                        field.id(),
                        null != longColStat.getNdv() ? longColStat.getNdv() : null,
                        null != longColStat.getMin()
                                ? org.apache.paimon.data.Timestamp.fromSQLTimestamp(
                                        new Timestamp(longColStat.getMin()))
                                : null,
                        null != longColStat.getMax()
                                ? org.apache.paimon.data.Timestamp.fromSQLTimestamp(
                                        new Timestamp(longColStat.getMax()))
                                : null,
                        null != longColStat.getNullCount() ? longColStat.getNullCount() : null,
                        null,
                        null);
            }
        } else if (colStat instanceof CatalogColumnStatisticsDataDouble) {
            CatalogColumnStatisticsDataDouble doubleColumnStatsData =
                    (CatalogColumnStatisticsDataDouble) colStat;
            if (typeRoot.equals(DataTypeRoot.FLOAT)) {
                return ColStats.newColStats(
                        field.id(),
                        null != doubleColumnStatsData.getNdv()
                                ? doubleColumnStatsData.getNdv()
                                : null,
                        null != doubleColumnStatsData.getMin()
                                ? doubleColumnStatsData.getMin().floatValue()
                                : null,
                        null != doubleColumnStatsData.getMax()
                                ? doubleColumnStatsData.getMax().floatValue()
                                : null,
                        null != doubleColumnStatsData.getNullCount()
                                ? doubleColumnStatsData.getNullCount()
                                : null,
                        null,
                        null);
            } else if (typeRoot.equals(DataTypeRoot.DOUBLE)) {
                return ColStats.newColStats(
                        field.id(),
                        null != doubleColumnStatsData.getNdv()
                                ? doubleColumnStatsData.getNdv()
                                : null,
                        null != doubleColumnStatsData.getMin()
                                ? doubleColumnStatsData.getMin()
                                : null,
                        null != doubleColumnStatsData.getMax()
                                ? doubleColumnStatsData.getMax()
                                : null,
                        null != doubleColumnStatsData.getNullCount()
                                ? doubleColumnStatsData.getNullCount()
                                : null,
                        null,
                        null);
            } else if (typeRoot.equals(DataTypeRoot.DECIMAL)) {
                BigDecimal max = BigDecimal.valueOf(doubleColumnStatsData.getMax());
                BigDecimal min = BigDecimal.valueOf(doubleColumnStatsData.getMin());
                return ColStats.newColStats(
                        field.id(),
                        null != doubleColumnStatsData.getNdv()
                                ? doubleColumnStatsData.getNdv()
                                : null,
                        null != doubleColumnStatsData.getMin()
                                ? Decimal.fromBigDecimal(min, min.precision(), min.scale())
                                : null,
                        null != doubleColumnStatsData.getMax()
                                ? Decimal.fromBigDecimal(max, max.precision(), max.scale())
                                : null,
                        null != doubleColumnStatsData.getNullCount()
                                ? doubleColumnStatsData.getNullCount()
                                : null,
                        null,
                        null);
            }
        } else if (colStat instanceof CatalogColumnStatisticsDataDate) {
            CatalogColumnStatisticsDataDate dateColumnStatsData =
                    (CatalogColumnStatisticsDataDate) colStat;
            if (typeRoot.equals(DataTypeRoot.DATE)) {
                return ColStats.newColStats(
                        field.id(),
                        null != dateColumnStatsData.getNdv() ? dateColumnStatsData.getNdv() : null,
                        null != dateColumnStatsData.getMin()
                                ? new Long(dateColumnStatsData.getMin().getDaysSinceEpoch())
                                        .intValue()
                                : null,
                        null != dateColumnStatsData.getMax()
                                ? new Long(dateColumnStatsData.getMax().getDaysSinceEpoch())
                                        .intValue()
                                : null,
                        null != dateColumnStatsData.getNullCount()
                                ? dateColumnStatsData.getNullCount()
                                : null,
                        null,
                        null);
            }
        } else if (colStat instanceof CatalogColumnStatisticsDataBinary) {
            CatalogColumnStatisticsDataBinary binaryColumnStatsData =
                    (CatalogColumnStatisticsDataBinary) colStat;
            if (typeRoot.equals(DataTypeRoot.VARBINARY) || typeRoot.equals(DataTypeRoot.BINARY)) {
                return ColStats.newColStats(
                        field.id(),
                        null,
                        null,
                        null,
                        null != binaryColumnStatsData.getNullCount()
                                ? binaryColumnStatsData.getNullCount()
                                : null,
                        null != binaryColumnStatsData.getAvgLength()
                                ? binaryColumnStatsData.getAvgLength().longValue()
                                : null,
                        null != binaryColumnStatsData.getMaxLength()
                                ? binaryColumnStatsData.getMaxLength()
                                : null);
            }
        }
        throw new CatalogException(
                String.format(
                        "Flink does not support convert ColumnStats '%s' for Paimon column "
                                + "type '%s' yet",
                        colStat, field.type()));
    }