From bc53d712d8c5667df677b892d552ccd470dce86d Mon Sep 17 00:00:00 2001 From: LinMingQiang Date: Sun, 8 Dec 2024 14:59:10 +0800 Subject: [PATCH 1/2] [core] support summary system table to show tables detail info --- .../paimon/table/system/SummaryTable.java | 363 ++++++++++++++++++ .../table/system/SystemTableLoader.java | 2 + .../paimon/flink/SystemTableITCase.java | 101 ++++- 3 files changed, 465 insertions(+), 1 deletion(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/table/system/SummaryTable.java diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SummaryTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SummaryTable.java new file mode 100644 index 000000000000..524e1f9ed0bb --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SummaryTable.java @@ -0,0 +1,363 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.table.system; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.disk.IOManager; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.BucketSpec; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.ReadonlyTable; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.InnerTableRead; +import org.apache.paimon.table.source.InnerTableScan; +import org.apache.paimon.table.source.ReadOnceTableScan; +import org.apache.paimon.table.source.SingletonSplit; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.table.source.TableRead; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.TimestampType; +import org.apache.paimon.utils.IteratorRecordReader; +import org.apache.paimon.utils.JsonSerdeUtil; +import org.apache.paimon.utils.Pair; +import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.utils.ProjectedRow; +import org.apache.paimon.utils.RowDataToObjectArrayConverter; +import org.apache.paimon.utils.SerializationUtils; + +import org.apache.paimon.shade.guava30.com.google.common.collect.Iterators; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER; + +/** A {@link Table} for showing summary of the specific table. */ +public class SummaryTable implements ReadonlyTable { + + private static final long serialVersionUID = 1L; + + public static final String SUMMARY = "summary"; + + public static final RowType TABLE_TYPE = + new RowType( + Arrays.asList( + // table summary. + new DataField(0, "table_name", SerializationUtils.newStringType(false)), + new DataField(1, "table_path", SerializationUtils.newStringType(false)), + new DataField(2, "table_type", SerializationUtils.newStringType(false)), + new DataField( + 3, "primary_keys", SerializationUtils.newStringType(true)), + new DataField( + 4, "partition_keys", SerializationUtils.newStringType(true)), + new DataField(5, "comment", SerializationUtils.newStringType(true)), + new DataField(6, "schema_id", new BigIntType(false)), + new DataField( + 7, "bucket_mode", SerializationUtils.newStringType(false)), + new DataField(8, "bucket_num", new IntType(false)), + new DataField( + 9, "snapshot_range", SerializationUtils.newStringType(true)), + new DataField( + 10, + "latest_commit_kind", + SerializationUtils.newStringType(true)), + new DataField(11, "tag_nums", new IntType(false)), + new DataField(12, "options", SerializationUtils.newStringType(true)), + + // partition summary. + new DataField(13, "partition_nums", new IntType(false)), + new DataField( + 14, + "max_record_partition", + SerializationUtils.newStringType(true)), + new DataField( + 15, + "max_filenums_partition", + SerializationUtils.newStringType(true)), + new DataField( + 16, + "max_filesize_partition", + SerializationUtils.newStringType(true)), + + // datafile summary. + new DataField(17, "file_nums", new BigIntType(false)), + new DataField(18, "file_size", new BigIntType(false)), + new DataField(19, "estimate_record_count", new BigIntType(false)), + new DataField(20, "last_commit_time", new TimestampType(true, 3)))); + + private final FileStoreTable storeTable; + + public SummaryTable(FileStoreTable storeTable) { + this.storeTable = storeTable; + } + + @Override + public InnerTableScan newScan() { + return new SummaryScan(); + } + + @Override + public InnerTableRead newRead() { + return new SummaryRead(storeTable); + } + + @Override + public String name() { + return storeTable.name() + SYSTEM_TABLE_SPLITTER + SUMMARY; + } + + @Override + public RowType rowType() { + return TABLE_TYPE; + } + + @Override + public List primaryKeys() { + return Collections.singletonList("table_name"); + } + + @Override + public Table copy(Map dynamicOptions) { + return new SummaryTable(storeTable.copy(dynamicOptions)); + } + + private static class SummaryScan extends ReadOnceTableScan { + + @Override + protected Plan innerPlan() { + return () -> Collections.singletonList(new SummarySplit()); + } + + @Override + public InnerTableScan withFilter(Predicate predicate) { + // no need to filter. + return this; + } + } + + private static class SummarySplit extends SingletonSplit { + private static final long serialVersionUID = 1L; + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + return obj != null && getClass() == obj.getClass(); + } + } + + private static class SummaryRead implements InnerTableRead { + + private RowType readType; + + private final FileStoreTable storeTable; + + private SummaryRead(FileStoreTable storeTable) { + this.storeTable = storeTable; + } + + @Override + public InnerTableRead withFilter(Predicate predicate) { + return this; + } + + @Override + public TableRead withIOManager(IOManager ioManager) { + return this; + } + + @Override + public InnerTableRead withReadType(RowType readType) { + this.readType = readType; + return this; + } + + @Override + public RecordReader createReader(Split split) { + Preconditions.checkArgument( + split instanceof SummarySplit, "Unsupported split: " + split.getClass()); + + Iterator iterator = + Collections.singletonList(toRow(storeTable)).iterator(); + + if (readType != null) { + iterator = + Iterators.transform( + iterator, + row -> + ProjectedRow.from(readType, PartitionsTable.TABLE_TYPE) + .replaceRow(row)); + } + return new IteratorRecordReader<>(iterator); + } + + public InternalRow toRow(FileStoreTable storeTable) { + GenericRow result = new GenericRow(21); + setTableSummary(result); + setPartitionSummary(result); + setDataFilesSummary(result); + + Snapshot latestSnapshot = storeTable.snapshotManager().latestSnapshot(); + result.setField( + 20, + latestSnapshot != null + ? Timestamp.fromLocalDateTime( + LocalDateTime.ofInstant( + Instant.ofEpochMilli(latestSnapshot.timeMillis()), + ZoneId.systemDefault())) + : null); + + return result; + } + + private void setTableSummary(GenericRow result) { + List pk = storeTable.primaryKeys(); + BucketSpec bucketSpec = storeTable.bucketSpec(); + Snapshot latestSnapshot = storeTable.snapshotManager().latestSnapshot(); + + result.setField(0, BinaryString.fromString(storeTable.fullName())); + result.setField(1, BinaryString.fromString(storeTable.location().toString())); + result.setField( + 2, + BinaryString.fromString(pk.isEmpty() ? "Append table" : "Primary key table")); + + result.setField(3, toJson(pk)); + result.setField(4, toJson(storeTable.partitionKeys())); + result.setField(5, BinaryString.fromString(storeTable.comment().orElse(""))); + result.setField(6, storeTable.schema().id()); + result.setField(7, BinaryString.fromString(bucketSpec.getBucketMode().toString())); + result.setField(8, bucketSpec.getNumBuckets()); + result.setField( + 9, + BinaryString.fromString( + String.format( + "[%s,%s]", + storeTable.snapshotManager().earliestSnapshotId(), + latestSnapshot != null ? latestSnapshot.id() : null))); + result.setField( + 10, + latestSnapshot != null + ? BinaryString.fromString(latestSnapshot.commitKind().name()) + : null); + result.setField(11, storeTable.tagManager().allTagNames().size()); + result.setField(12, toJson(storeTable.options())); + } + + private void setPartitionSummary(GenericRow result) { + RowDataToObjectArrayConverter converter = + new RowDataToObjectArrayConverter(storeTable.schema().logicalPartitionType()); + + List partitionEntryList = storeTable.newScan().listPartitionEntries(); + result.setField(13, partitionEntryList.size()); + if (storeTable.partitionKeys().isEmpty()) { + result.setField(14, null); + result.setField(15, null); + result.setField(16, null); + return; + } + + Pair maxRecord = Pair.of(null, 0L); + Pair maxFileNums = Pair.of(null, 0L); + Pair maxFilesize = Pair.of(null, 0L); + + for (PartitionEntry partitionEntry : partitionEntryList) { + if (partitionEntry.recordCount() > maxRecord.getRight()) { + maxRecord.setRight(partitionEntry.recordCount()); + maxRecord.setLeft(partitionEntry.partition()); + } + if (partitionEntry.fileCount() > maxFileNums.getRight()) { + maxFileNums.setRight(partitionEntry.fileCount()); + maxFileNums.setLeft(partitionEntry.partition()); + } + if (partitionEntry.fileSizeInBytes() > maxFilesize.getRight()) { + maxFilesize.setRight(partitionEntry.fileSizeInBytes()); + maxFilesize.setLeft(partitionEntry.partition()); + } + } + + String partitionInfoFormat = "{\"partition\":\"%s\",\"%s\": %s}"; + + result.setField( + 14, + BinaryString.fromString( + String.format( + partitionInfoFormat, + Arrays.toString(converter.convert(maxRecord.getLeft())), + "record_count", + maxRecord.getRight()))); + result.setField( + 15, + BinaryString.fromString( + String.format( + partitionInfoFormat, + Arrays.toString(converter.convert(maxFileNums.getLeft())), + "file_nums", + maxFileNums.getRight()))); + result.setField( + 16, + BinaryString.fromString( + String.format( + partitionInfoFormat, + Arrays.toString(converter.convert(maxFilesize.getLeft())), + "file_size", + maxFilesize.getRight()))); + } + + private void setDataFilesSummary(GenericRow result) { + // If the latest snapshot type is COMPACT, this will be exact, otherwise it is slightly + // larger than the actual value, because these data are not merged. + long estimateRecordCount = 0L; + long fileNums = 0L; + long fileSizeInBytes = 0L; + for (DataSplit dataSplit : storeTable.newSnapshotReader().read().dataSplits()) { + estimateRecordCount += dataSplit.rowCount(); + fileNums += dataSplit.dataFiles().size(); + for (DataFileMeta dataFile : dataSplit.dataFiles()) { + fileSizeInBytes += dataFile.fileSize(); + } + } + result.setField(17, fileNums); + result.setField(18, fileSizeInBytes); + result.setField(19, estimateRecordCount); + } + + private BinaryString toJson(Object obj) { + return BinaryString.fromString(JsonSerdeUtil.toFlatJson(obj)); + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java index 763e4d121673..6f88d2d4607b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java @@ -52,6 +52,7 @@ import static org.apache.paimon.table.system.SchemasTable.SCHEMAS; import static org.apache.paimon.table.system.SnapshotsTable.SNAPSHOTS; import static org.apache.paimon.table.system.StatisticTable.STATISTICS; +import static org.apache.paimon.table.system.SummaryTable.SUMMARY; import static org.apache.paimon.table.system.TagsTable.TAGS; /** Loader to load system {@link Table}s. */ @@ -74,6 +75,7 @@ public class SystemTableLoader { .put(AGGREGATION_FIELDS, AggregationFieldsTable::new) .put(STATISTICS, StatisticTable::new) .put(BINLOG, BinlogTable::new) + .put(SUMMARY, SummaryTable::new) .build(); public static final List SYSTEM_TABLES = new ArrayList<>(SYSTEM_TABLE_LOADERS.keySet()); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java index 771f4acc5e58..0bcd53010a45 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java @@ -50,7 +50,7 @@ public void testBinlogTableStreamRead() throws Exception { } @Test - public void testBinlogTableBatchRead() throws Exception { + public void testBinlogTableBatchRead() { sql( "CREATE TABLE T (a INT, b INT, primary key (a) NOT ENFORCED) with ('changelog-producer' = 'lookup', " + "'bucket' = '2')"); @@ -63,4 +63,103 @@ public void testBinlogTableBatchRead() throws Exception { Row.of("+I", new Integer[] {1}, new Integer[] {3}), Row.of("+I", new Integer[] {2}, new Integer[] {2})); } + + @Test + public void testSummaryTableAppendOnlyTable() { + sql( + "CREATE TABLE T (" + + "a INT," + + " b INT" + + ") comment 'this is comment'" + + " with (" + + "'bucket' = '2'," + + "'bucket-key' = 'a')"); + sql("INSERT INTO T VALUES (1, 2), (1, 2), (1, 2), (1, 2)"); + tEnv.executeSql("SELECT * FROM T$summary").print(); + + // append table with unaware_bucket. + sql( + "CREATE TABLE T_unaware_bucket (" + + "a INT," + + " b INT" + + ") with (" + + "'bucket' = '-1'" + + ")"); + + sql("INSERT INTO T_unaware_bucket VALUES (1, 2)"); + tEnv.executeSql("SELECT * FROM T_unaware_bucket$summary").print(); + + // append table with partitioned. + sql( + "CREATE TABLE T_with_partition (" + + "a INT," + + " b INT," + + " dt string," + + " hm string" + + ") PARTITIONED BY (dt, hm) with (" + + "'bucket' = '2'," + + " 'bucket-key' = 'a')"); + sql( + "INSERT INTO T_with_partition VALUES (1, 2, '20240101', '11'),(1, 2, '20240101', '11')"); + tEnv.executeSql("SELECT * FROM T_with_partition$summary").print(); + } + + @Test + public void testSummaryTablePrimaryKeyTable() { + sql( + "CREATE TABLE T (a INT," + + " b INT," + + " primary key (a) NOT ENFORCED" + + ") with (" + + "'bucket' = '2')"); + // sql("INSERT INTO T VALUES (1, 2)"); + tEnv.executeSql("SELECT * FROM T$summary").print(); + + sql( + "CREATE TABLE T_unaware_bucket (a INT," + + " b INT," + + " primary key (a) NOT ENFORCED" + + ") with (" + + "'bucket' = '-1')"); + sql("INSERT INTO T_unaware_bucket VALUES (1, 2)"); + tEnv.executeSql("SELECT * FROM T_unaware_bucket$summary").print(); + + sql( + "CREATE TABLE T_with_partition (" + + "a INT," + + " b INT," + + " dt string," + + " hm string," + + " primary key (a, dt, hm) NOT ENFORCED" + + ") PARTITIONED BY (dt, hm) with ( " + + "'bucket' = '2')"); + + sql( + "INSERT INTO T_with_partition VALUES" + + " (1, 2, '20240101', '11')" + + ",(1, 2, '20240101', '12')"); + sql("INSERT INTO T_with_partition VALUES (1, 2, '20240101', '13')"); + sql("INSERT INTO T_with_partition VALUES (1, 2, '20240101', '13')"); + tEnv.executeSql("SELECT * FROM T_with_partition$summary").print(); + } + + @Test + public void testSummaryTableWithCompact() { + sql( + "CREATE TABLE T (" + + "a INT," + + " b INT," + + " dt string," + + " hm string," + + " primary key (a, dt, hm) NOT ENFORCED" + + ") PARTITIONED BY (dt, hm) with ('changelog-producer' = 'lookup', " + + "'bucket' = '2')"); + sql("INSERT INTO T VALUES (1, 2, '20240101', '11')" + ",(1, 2, '20240101', '11')"); + sql( + "INSERT INTO T /*+ OPTIONS(" + + "'full-compaction.delta-commits' = '1'" + + ") */" + + " VALUES (1, 2, '20240101', '11')"); + tEnv.executeSql("SELECT * FROM T$summary").print(); + } } From ffc0597fe8cb540c4756042955d3f988e9ea0bee Mon Sep 17 00:00:00 2001 From: LinMingQiang <1356469429@qq.com> Date: Wed, 11 Dec 2024 16:55:16 +0800 Subject: [PATCH 2/2] [core] support summary system table to show tables detail info. --- docs/content/concepts/system-tables.md | 21 ++++++++++++++++ .../paimon/table/system/SummaryTable.java | 2 +- .../paimon/flink/SystemTableITCase.java | 24 ++++++++++++++++++- 3 files changed, 45 insertions(+), 2 deletions(-) diff --git a/docs/content/concepts/system-tables.md b/docs/content/concepts/system-tables.md index 5795aea419fb..9e55ae87b489 100644 --- a/docs/content/concepts/system-tables.md +++ b/docs/content/concepts/system-tables.md @@ -43,6 +43,27 @@ In some cases, the table name needs to be enclosed with back quotes to avoid syn SELECT * FROM my_catalog.my_db.`my_table$snapshots`; ``` +### Summary Table + +```sql +SELECT * FROM my_table$summary; + +/* ++--------------------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+----------------------+--------------------------------+-------------+--------------------------------+--------------------------------+-------------+--------------------------------+----------------+--------------------------------+--------------------------------+--------------------------------+----------------------+----------------------+-----------------------+-------------------------+ +| table_name | table_path | table_type | primary_keys | partition_keys | comment | schema_id | bucket_mode | bucket_num | snapshot_range | latest_commit_kind | tag_nums | options | partition_nums | max_recordcount_partition | max_filenums_partition | max_filesize_partition | file_nums | file_size | estimate_record_count | last_commit_time | ++--------------------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+----------------------+--------------------------------+-------------+--------------------------------+--------------------------------+-------------+--------------------------------+----------------+--------------------------------+--------------------------------+--------------------------------+----------------------+----------------------+-----------------------+-------------------------+ +| default.T | /var/folders/n8/t55hm8l56wl... | Primary key table | ["a","dt","hm"] | ["dt","hm"] | | 0 | HASH_FIXED | 2 | [1,4] | COMPACT | 0 | {"bucket":"2","path":"/var/... | 1 | {"partition":"[20240101, 11... | {"partition":"[20240101, 11... | {"partition":"[20240101, 11... | 1 | 1379 | 1 | 2024-12-10 10:55:58.395 | ++--------------------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+----------------------+--------------------------------+-------------+--------------------------------+--------------------------------+-------------+--------------------------------+----------------+--------------------------------+--------------------------------+--------------------------------+----------------------+----------------------+-----------------------+-------------------------+ +1 rows in set +*/ + +Comment : +max_recordcount_partition : The partition with the most record count. +max_filenums_partition : The partition with the most file nums. +max_filesize_partition : The partition with the most file size. +estimate_record_count: If the latest snapshot type is COMPACT, this will be exact, otherwise it is slightly larger than the actual value, because these data are not merged. +``` + ### Snapshots Table You can query the snapshot history information of the table through snapshots table, including the record count occurred in the snapshot. diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SummaryTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SummaryTable.java index 524e1f9ed0bb..f0e97eb44726 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SummaryTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SummaryTable.java @@ -102,7 +102,7 @@ public class SummaryTable implements ReadonlyTable { new DataField(13, "partition_nums", new IntType(false)), new DataField( 14, - "max_record_partition", + "max_recordcount_partition", SerializationUtils.newStringType(true)), new DataField( 15, diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java index 0bcd53010a45..15de1cbfa21a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java @@ -74,6 +74,10 @@ public void testSummaryTableAppendOnlyTable() { + " with (" + "'bucket' = '2'," + "'bucket-key' = 'a')"); + // no data, no partition. + tEnv.executeSql("SELECT * FROM T$summary").print(); + + // 1 file, 4 record. sql("INSERT INTO T VALUES (1, 2), (1, 2), (1, 2), (1, 2)"); tEnv.executeSql("SELECT * FROM T$summary").print(); @@ -85,10 +89,14 @@ public void testSummaryTableAppendOnlyTable() { + ") with (" + "'bucket' = '-1'" + ")"); - sql("INSERT INTO T_unaware_bucket VALUES (1, 2)"); tEnv.executeSql("SELECT * FROM T_unaware_bucket$summary").print(); + + } + + @Test + public void testSummaryTableAppendOnlyTableWithPartition() { // append table with partitioned. sql( "CREATE TABLE T_with_partition (" @@ -102,6 +110,19 @@ public void testSummaryTableAppendOnlyTable() { sql( "INSERT INTO T_with_partition VALUES (1, 2, '20240101', '11'),(1, 2, '20240101', '11')"); tEnv.executeSql("SELECT * FROM T_with_partition$summary").print(); + + // append table with partitioned and unaware bucket. + sql( + "CREATE TABLE T_with_partition_unaware_bucket (" + + "a INT," + + " b INT," + + " dt string," + + " hm string" + + ") PARTITIONED BY (dt, hm) with (" + + "'bucket' = '-1')"); + sql( + "INSERT INTO T_with_partition_unaware_bucket VALUES (1, 2, '20240101', '11'),(1, 2, '20240101', '11')"); + tEnv.executeSql("SELECT * FROM T_with_partition_unaware_bucket$summary").print(); } @Test @@ -114,6 +135,7 @@ public void testSummaryTablePrimaryKeyTable() { + "'bucket' = '2')"); // sql("INSERT INTO T VALUES (1, 2)"); tEnv.executeSql("SELECT * FROM T$summary").print(); + tEnv.executeSql("desc T$summary").print(); sql( "CREATE TABLE T_unaware_bucket (a INT,"