morrySnow commented on code in PR #31276:
URL: https://github.com/apache/doris/pull/31276#discussion_r1498870440
##########
fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java:
##########
@@ -151,7 +149,7 @@ default int getBaseColumnIdxByName(String colName) {
BaseAnalysisTask createAnalysisTask(AnalysisInfo info);
- long estimatedRowCount();
+ long getRowCountForNereids();
Review Comment:
could u add some comment to explain why we need this inferface
##########
fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalRowCountCache.java:
##########
@@ -0,0 +1,113 @@
+// 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.doris.datasource;
+
+import org.apache.doris.catalog.TableIf;
+import org.apache.doris.common.Config;
+import org.apache.doris.statistics.StatisticConstants;
+import org.apache.doris.statistics.StatisticsCacheLoader;
+import org.apache.doris.statistics.util.StatisticsUtil;
+
+import com.github.benmanes.caffeine.cache.AsyncLoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.time.Duration;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+public class ExternalRowCountCache {
+
+ private static final Logger LOG =
LogManager.getLogger(ExternalRowCountCache.class);
+ private final AsyncLoadingCache<RowCountKey, Optional<Long>> rowCountCache;
+
+ public ExternalRowCountCache(ExecutorService executor) {
+ rowCountCache = Caffeine.newBuilder()
+ .maximumSize(Config.max_external_table_row_count_cache_num)
+
.expireAfterWrite(Duration.ofHours(StatisticConstants.EXTERNAL_ROW_COUNT_EXPIRE_INTERVAL_HOURS))
+ .executor(executor)
+ .buildAsync(new RowCountLoader());
+ }
+
+ public static class RowCountKey {
+ private final long catalogId;
+ private final long dbId;
+ private final long tableId;
+
+ public RowCountKey(long catalogId, long dbId, long tableId) {
+ this.catalogId = catalogId;
+ this.dbId = dbId;
+ this.tableId = tableId;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (!(obj instanceof RowCountKey)) {
+ return false;
+ }
+ return ((RowCountKey) obj).tableId == this.tableId;
+ }
+
+ @Override
+ public int hashCode() {
+ return (int) tableId;
+ }
+ }
+
+ public static class RowCountLoader extends
StatisticsCacheLoader<RowCountKey, Optional<Long>> {
+
+ @Override
+ protected Optional<Long> doLoad(RowCountKey rowCountKey) {
+ try {
+ TableIf table =
StatisticsUtil.findTable(rowCountKey.catalogId, rowCountKey.dbId,
rowCountKey.tableId);
+ return Optional.of(table.fetchRowCount());
+ } catch (Exception e) {
+ LOG.warn("Failed to get table with catalogId {}, dbId {},
tableId {}", rowCountKey.catalogId,
+ rowCountKey.dbId, rowCountKey.tableId);
+ return Optional.empty();
+ }
+ }
+ }
+
+ /**
+ * Get cached row count for the given table. Return 0 if cached not loaded
or table not exists.
+ * Cached will be loaded async.
+ * @param catalogId
+ * @param dbId
+ * @param tableId
+ * @return Cached row count or 0 if not exist
+ */
+ public long getCachedRowCount(long catalogId, long dbId, long tableId) {
+ RowCountKey key = new RowCountKey(catalogId, dbId, tableId);
+ try {
+ CompletableFuture<Optional<Long>> f = rowCountCache.get(key);
+ if (f.isDone()) {
+ return f.get().orElse(0L);
+ }
+ } catch (Exception e) {
+ LOG.warn("Unexpected exception while returning ColumnStatistic",
e);
Review Comment:
```suggestion
LOG.warn("Unexpected exception while returning row count", e);
```
##########
fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java:
##########
@@ -297,10 +297,16 @@ public String getMysqlType() {
@Override
public long getRowCount() {
- return 0;
+ // All external table should get external row count from cache.
+ return
Env.getCurrentEnv().getExtMetaCacheMgr().getRowCountCache().getCachedRowCount(catalog.getId(),
dbId, id);
}
- public long getCacheRowCount() {
+ @Override
+ /**
+ * Default return 0. Subclass need to implement this interface.
+ * This is called by ExternalRowCountCache to load row count cache.
+ */
+ public long fetchRowCount() {
return 0;
}
Review Comment:
if all sub class need implement this interface, why not keep it as abstract?
why external table is not abstract?
##########
fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java:
##########
@@ -352,8 +358,8 @@ public BaseAnalysisTask createAnalysisTask(AnalysisInfo
info) {
}
@Override
- public long estimatedRowCount() {
- return 1;
+ public long getRowCountForNereids() {
+ return Math.max(getRowCount(), 1);
}
Review Comment:
same implement for all type table? just implement in TableIf?
##########
fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalRowCountCache.java:
##########
@@ -0,0 +1,113 @@
+// 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.doris.datasource;
+
+import org.apache.doris.catalog.TableIf;
+import org.apache.doris.common.Config;
+import org.apache.doris.statistics.StatisticConstants;
+import org.apache.doris.statistics.StatisticsCacheLoader;
+import org.apache.doris.statistics.util.StatisticsUtil;
+
+import com.github.benmanes.caffeine.cache.AsyncLoadingCache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.time.Duration;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
+public class ExternalRowCountCache {
+
+ private static final Logger LOG =
LogManager.getLogger(ExternalRowCountCache.class);
+ private final AsyncLoadingCache<RowCountKey, Optional<Long>> rowCountCache;
+
+ public ExternalRowCountCache(ExecutorService executor) {
+ rowCountCache = Caffeine.newBuilder()
+ .maximumSize(Config.max_external_table_row_count_cache_num)
+
.expireAfterWrite(Duration.ofHours(StatisticConstants.EXTERNAL_ROW_COUNT_EXPIRE_INTERVAL_HOURS))
+ .executor(executor)
+ .buildAsync(new RowCountLoader());
+ }
+
+ public static class RowCountKey {
+ private final long catalogId;
+ private final long dbId;
+ private final long tableId;
+
+ public RowCountKey(long catalogId, long dbId, long tableId) {
+ this.catalogId = catalogId;
+ this.dbId = dbId;
+ this.tableId = tableId;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (!(obj instanceof RowCountKey)) {
+ return false;
+ }
+ return ((RowCountKey) obj).tableId == this.tableId;
+ }
+
+ @Override
+ public int hashCode() {
+ return (int) tableId;
+ }
+ }
+
+ public static class RowCountLoader extends
StatisticsCacheLoader<RowCountKey, Optional<Long>> {
Review Comment:
since the basic cache loader extended by RowCountLoader. it is better give
StatisticsCacheLoader a better name
##########
fe/fe-common/src/main/java/org/apache/doris/common/Config.java:
##########
@@ -1902,6 +1902,10 @@ public class Config extends ConfigBase {
"Max cache number of remote file system."})
public static long max_remote_file_system_cache_num = 100;
+ @ConfField(mutable = false, masterOnly = false, description =
{"外表行数缓存最大数量",
+ "Max cache number of external table row count"})
+ public static long max_external_table_row_count_cache_num = 100000;
Review Comment:
use session variable?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]