This is an automated email from the ASF dual-hosted git repository.
dataroaring pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/master by this push:
new 0fd46fb7651 [feature](merge-cloud) rewrite cloud table property
(#30069)
0fd46fb7651 is described below
commit 0fd46fb76511af274004cfd9a26ee4b69f74819b
Author: yujun <[email protected]>
AuthorDate: Fri Jan 19 18:07:04 2024 +0800
[feature](merge-cloud) rewrite cloud table property (#30069)
---
.../doris/alter/MaterializedViewHandler.java | 2 +-
.../apache/doris/alter/SchemaChangeHandler.java | 2 +-
.../org/apache/doris/analysis/CreateTableStmt.java | 4 +-
.../java/org/apache/doris/backup/RestoreJob.java | 2 +-
.../doris/catalog/DynamicPartitionProperty.java | 50 +++++-----
.../main/java/org/apache/doris/catalog/Env.java | 9 +-
.../java/org/apache/doris/catalog/EnvFactory.java | 102 ++++++++++-----------
.../java/org/apache/doris/catalog/OlapTable.java | 2 +-
.../java/org/apache/doris/catalog/Partition.java | 2 +-
.../java/org/apache/doris/catalog/Replica.java | 2 +-
.../apache/doris/catalog/ReplicaAllocation.java | 2 +-
.../org/apache/doris/catalog/TableProperty.java | 5 +-
.../main/java/org/apache/doris/catalog/Tablet.java | 4 +-
.../catalog/CloudDynamicPartitionProperty.java | 44 +++++++++
.../doris/cloud/catalog/CloudEnvFactory.java | 102 +++++++++++++++++++++
.../cloud/common/util/CloudPropertyAnalyzer.java | 50 ++++++++++
.../cloud/datasource/CloudInternalCatalog.java | 2 +-
.../doris/cloud/system/CloudSystemInfoService.java | 43 +++++++++
.../apache/doris/common/util/PropertyAnalyzer.java | 45 ++++++++-
.../org/apache/doris/datasource/CatalogMgr.java | 2 +-
.../apache/doris/datasource/InternalCatalog.java | 2 +-
.../trees/plans/commands/info/CreateTableInfo.java | 13 ++-
.../apache/doris/analysis/CreateTableStmtTest.java | 12 ++-
.../org/apache/doris/binlog/BinlogManagerTest.java | 2 +-
.../org/apache/doris/catalog/EnvFactoryTest.java | 31 +++----
.../doris/cloud/catalog/CloudEnvFactoryTest.java | 52 +++++++++++
.../statistics/StatisticsAutoCollectorTest.java | 4 +-
27 files changed, 464 insertions(+), 128 deletions(-)
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
index 3f7122f3650..347da5f1a49 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
@@ -398,7 +398,7 @@ public class MaterializedViewHandler extends AlterHandler {
long baseTabletId = baseTablet.getId();
long mvTabletId = idGeneratorBuffer.getNextId();
- Tablet newTablet = EnvFactory.createTablet(mvTabletId);
+ Tablet newTablet =
EnvFactory.getInstance().createTablet(mvTabletId);
mvIndex.addTablet(newTablet, mvTabletMeta);
addedTablets.add(newTablet);
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
index b5895ead653..dbb9ee1466d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
@@ -1548,7 +1548,7 @@ public class SchemaChangeHandler extends AlterHandler {
long originTabletId = originTablet.getId();
long shadowTabletId = idGeneratorBuffer.getNextId();
- Tablet shadowTablet =
EnvFactory.createTablet(shadowTabletId);
+ Tablet shadowTablet =
EnvFactory.getInstance().createTablet(shadowTabletId);
shadowIndex.addTablet(shadowTablet, shadowTabletMeta);
addedTablets.add(shadowTablet);
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java
b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java
index 17dd5f396c8..b1f84326b44 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java
@@ -494,8 +494,8 @@ public class CreateTableStmt extends DdlStmt {
if (engineName.equalsIgnoreCase(DEFAULT_ENGINE_NAME)) {
// before analyzing partition, handle the replication allocation
info
- properties =
PropertyAnalyzer.rewriteReplicaAllocationProperties(tableName.getCtl(),
tableName.getDb(),
- properties);
+ properties =
PropertyAnalyzer.getInstance().rewriteOlapProperties(tableName.getCtl(),
+ tableName.getDb(), properties);
// analyze partition
if (partitionDesc != null) {
if (partitionDesc instanceof ListPartitionDesc ||
partitionDesc instanceof RangePartitionDesc) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
index 0398199e2e1..4d0afed9b8c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
@@ -1123,7 +1123,7 @@ public class RestoreJob extends AbstractJob {
for (int i = 0; i < remotetabletSize; i++) {
// generate new tablet id
long newTabletId = env.getNextId();
- Tablet newTablet = EnvFactory.createTablet(newTabletId);
+ Tablet newTablet =
EnvFactory.getInstance().createTablet(newTabletId);
// add tablet to index, but not add to TabletInvertedIndex
remoteIdx.addTablet(newTablet, null /* tablet meta */, true /*
is restore */);
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
index 2daeb7bc72c..816bb64e0ad 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
@@ -19,7 +19,6 @@ package org.apache.doris.catalog;
import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.Config;
import org.apache.doris.common.DdlException;
import org.apache.doris.common.FeConstants;
import org.apache.doris.common.util.DynamicPartitionUtil;
@@ -31,6 +30,7 @@ import com.google.common.base.Strings;
import java.util.Map;
import java.util.TimeZone;
+import java.util.function.BiConsumer;
public class DynamicPartitionProperty {
public static final String DYNAMIC_PARTITION_PROPERTY_PREFIX =
"dynamic_partition.";
@@ -106,6 +106,10 @@ public class DynamicPartitionProperty {
}
}
+ protected boolean supportProperty(String property) {
+ return true;
+ }
+
private ReplicaAllocation analyzeReplicaAllocation(Map<String, String>
properties) {
try {
return PropertyAnalyzer.analyzeReplicaAllocation(properties,
"dynamic_partition");
@@ -217,33 +221,35 @@ public class DynamicPartitionProperty {
* use table replication_num as dynamic_partition.replication_num default
value
*/
public String getProperties(ReplicaAllocation tableReplicaAlloc) {
+ ReplicaAllocation tmpAlloc = this.replicaAlloc.isNotSet() ?
tableReplicaAlloc : this.replicaAlloc;
StringBuilder sb = new StringBuilder();
- sb.append(",\n\"" + ENABLE + "\" = \"" + enable + "\"");
- sb.append(",\n\"" + TIME_UNIT + "\" = \"" + timeUnit + "\"");
- sb.append(",\n\"" + TIME_ZONE + "\" = \"" + tz.getID() + "\"");
- sb.append(",\n\"" + START + "\" = \"" + start + "\"");
- sb.append(",\n\"" + END + "\" = \"" + end + "\"");
- sb.append(",\n\"" + PREFIX + "\" = \"" + prefix + "\"");
- if (Config.isNotCloudMode()) {
- ReplicaAllocation tmpAlloc = this.replicaAlloc.isNotSet() ?
tableReplicaAlloc : this.replicaAlloc;
- sb.append(",\n\"" + REPLICATION_ALLOCATION + "\" = \"" +
tmpAlloc.toCreateStmt() + "\"");
- }
- sb.append(",\n\"" + BUCKETS + "\" = \"" + buckets + "\"");
- sb.append(",\n\"" + CREATE_HISTORY_PARTITION + "\" = \"" +
createHistoryPartition + "\"");
- sb.append(",\n\"" + HISTORY_PARTITION_NUM + "\" = \"" +
historyPartitionNum + "\"");
- sb.append(",\n\"" + HOT_PARTITION_NUM + "\" = \"" + hotPartitionNum +
"\"");
- sb.append(",\n\"" + RESERVED_HISTORY_PERIODS + "\" = \"" +
reservedHistoryPeriods + "\"");
- if (Config.isNotCloudMode()) {
- sb.append(",\n\"" + STORAGE_POLICY + "\" = \"" + storagePolicy +
"\"");
- if (!Strings.isNullOrEmpty(storageMedium)) {
- sb.append(",\n\"" + STORAGE_MEDIUM + "\" = \"" + storageMedium
+ "\"");
+ BiConsumer<String, Object> addProperty = (property, value) -> {
+ if (supportProperty(property)) {
+ sb.append(",\n\"" + property + "\" = \"" + value + "\"");
}
+ };
+ addProperty.accept(ENABLE, enable);
+ addProperty.accept(TIME_UNIT, timeUnit);
+ addProperty.accept(TIME_ZONE, tz.getID());
+ addProperty.accept(START, start);
+ addProperty.accept(END, end);
+ addProperty.accept(PREFIX, prefix);
+ addProperty.accept(REPLICATION_ALLOCATION, tmpAlloc.toCreateStmt());
+ addProperty.accept(BUCKETS, buckets);
+ addProperty.accept(CREATE_HISTORY_PARTITION, createHistoryPartition);
+ addProperty.accept(HISTORY_PARTITION_NUM, historyPartitionNum);
+ addProperty.accept(HOT_PARTITION_NUM, hotPartitionNum);
+ addProperty.accept(RESERVED_HISTORY_PERIODS, reservedHistoryPeriods);
+ addProperty.accept(STORAGE_POLICY, storagePolicy);
+ if (!Strings.isNullOrEmpty(storageMedium)) {
+ addProperty.accept(STORAGE_MEDIUM, storageMedium);
}
if (getTimeUnit().equalsIgnoreCase(TimeUnit.WEEK.toString())) {
- sb.append(",\n\"" + START_DAY_OF_WEEK + "\" = \"" +
startOfWeek.dayOfWeek + "\"");
+ addProperty.accept(START_DAY_OF_WEEK, startOfWeek.dayOfWeek);
} else if (getTimeUnit().equalsIgnoreCase(TimeUnit.MONTH.toString())) {
- sb.append(",\n\"" + START_DAY_OF_MONTH + "\" = \"" +
startOfMonth.day + "\"");
+ addProperty.accept(START_DAY_OF_MONTH, startOfMonth.day);
}
+
return sb.toString();
}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
index 37930708cc8..2c0ff9f692e 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
@@ -623,7 +623,7 @@ public class Env {
}
private static class SingletonHolder {
- private static final Env INSTANCE = EnvFactory.createEnv(false);
+ private static final Env INSTANCE =
EnvFactory.getInstance().createEnv(false);
}
private Env() {
@@ -673,7 +673,7 @@ public class Env {
this.journalObservable = new JournalObservable();
this.masterInfo = new MasterInfo();
- this.systemInfo = new SystemInfoService();
+ this.systemInfo = EnvFactory.getInstance().createSystemInfoService();
this.heartbeatMgr = new HeartbeatMgr(systemInfo, !isCheckpointCatalog);
this.tabletInvertedIndex = new TabletInvertedIndex();
this.colocateTableIndex = new ColocateTableIndex();
@@ -773,7 +773,7 @@ public class Env {
// only checkpoint thread it self will goes here.
// so no need to care about the thread safe.
if (CHECKPOINT == null) {
- CHECKPOINT = EnvFactory.createEnv(true);
+ CHECKPOINT = EnvFactory.getInstance().createEnv(true);
}
return CHECKPOINT;
} else {
@@ -3253,7 +3253,6 @@ public class Env {
// replicationNum
ReplicaAllocation replicaAlloc =
olapTable.getDefaultReplicaAllocation();
-
if (Config.isCloudMode()) {
sb.append("\"").append(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS).append("\"
= \"");
sb.append(olapTable.getTTLSeconds()).append("\"");
@@ -3312,7 +3311,7 @@ public class Env {
sb.append(olapTable.getDataSortInfo().toSql());
}
- if (Config.isCloudMode() && olapTable.getTTLSeconds() != 0) {
+ if (olapTable.getTTLSeconds() != 0) {
sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS).append("\"
= \"");
sb.append(olapTable.getTTLSeconds()).append("\"");
}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java
index c653159d7b6..9ecee918322 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java
@@ -17,79 +17,75 @@
package org.apache.doris.catalog;
-import org.apache.doris.cloud.catalog.CloudEnv;
-import org.apache.doris.cloud.catalog.CloudPartition;
-import org.apache.doris.cloud.catalog.CloudReplica;
-import org.apache.doris.cloud.catalog.CloudTablet;
-import org.apache.doris.cloud.datasource.CloudInternalCatalog;
+import org.apache.doris.cloud.catalog.CloudEnvFactory;
import org.apache.doris.common.Config;
+import org.apache.doris.common.util.PropertyAnalyzer;
import org.apache.doris.datasource.InternalCatalog;
+import org.apache.doris.system.SystemInfoService;
import java.lang.reflect.Type;
+import java.util.Map;
public class EnvFactory {
- public static Env createEnv(boolean isCheckpointCatalog) {
- if (Config.isCloudMode()) {
- return new CloudEnv(isCheckpointCatalog);
- } else {
- return new Env(isCheckpointCatalog);
- }
+ public EnvFactory() {
}
- public static InternalCatalog createInternalCatalog() {
- if (Config.isCloudMode()) {
- return new CloudInternalCatalog();
- } else {
- return new InternalCatalog();
- }
+ private static class SingletonHolder {
+ private static final EnvFactory INSTANCE =
+ Config.isCloudMode() ? new CloudEnvFactory() : new
EnvFactory();
}
- public static Type getPartitionClass() {
- if (Config.isCloudMode()) {
- return CloudPartition.class;
- } else {
- return Partition.class;
- }
+ public static EnvFactory getInstance() {
+ return SingletonHolder.INSTANCE;
}
- public static Partition createPartition() {
- if (Config.isCloudMode()) {
- return new CloudPartition();
- } else {
- return new Partition();
- }
+ public Env createEnv(boolean isCheckpointCatalog) {
+ return new Env(isCheckpointCatalog);
}
- public static Type getTabletClass() {
- if (Config.isCloudMode()) {
- return CloudTablet.class;
- } else {
- return Tablet.class;
- }
+ public InternalCatalog createInternalCatalog() {
+ return new InternalCatalog();
}
- public static Tablet createTablet() {
- if (Config.isCloudMode()) {
- return new CloudTablet();
- } else {
- return new Tablet();
- }
+ public SystemInfoService createSystemInfoService() {
+ return new SystemInfoService();
}
- public static Tablet createTablet(long tabletId) {
- if (Config.isCloudMode()) {
- return new CloudTablet(tabletId);
- } else {
- return new Tablet(tabletId);
- }
+ public Type getPartitionClass() {
+ return Partition.class;
}
- public static Replica createReplica() {
- if (Config.isCloudMode()) {
- return new CloudReplica();
- } else {
- return new Replica();
- }
+ public Partition createPartition() {
+ return new Partition();
}
+
+ public Type getTabletClass() {
+ return Tablet.class;
+ }
+
+ public Tablet createTablet() {
+ return new Tablet();
+ }
+
+ public Tablet createTablet(long tabletId) {
+ return new Tablet(tabletId);
+ }
+
+ public Replica createReplica() {
+ return new Replica();
+ }
+
+ public ReplicaAllocation createDefReplicaAllocation() {
+ return new ReplicaAllocation((short) 3);
+ }
+
+ public PropertyAnalyzer createPropertyAnalyzer() {
+ return new PropertyAnalyzer();
+ }
+
+ public DynamicPartitionProperty createDynamicPartitionProperty(Map<String,
String> properties) {
+ return new DynamicPartitionProperty(properties);
+ }
+
}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
index 2c221a45500..1d06b0305a0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
@@ -609,7 +609,7 @@ public class OlapTable extends Table {
idx.clearTabletsForRestore();
for (int i = 0; i < tabletNum; i++) {
long newTabletId = env.getNextId();
- Tablet newTablet = EnvFactory.createTablet(newTabletId);
+ Tablet newTablet =
EnvFactory.getInstance().createTablet(newTabletId);
idx.addTablet(newTablet, null /* tablet meta */, true /*
is restore */);
// replicas
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java
index cf4ac871c40..e1da0019c5a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java
@@ -331,7 +331,7 @@ public class Partition extends MetaObject implements
Writable {
}
public static Partition read(DataInput in) throws IOException {
- Partition partition = EnvFactory.createPartition();
+ Partition partition = EnvFactory.getInstance().createPartition();
partition.readFields(in);
return partition;
}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java
index e6b9066af78..f40396e23b8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java
@@ -671,7 +671,7 @@ public class Replica implements Writable {
}
public static Replica read(DataInput in) throws IOException {
- Replica replica = EnvFactory.createReplica();
+ Replica replica = EnvFactory.getInstance().createReplica();
replica.readFields(in);
return replica;
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java
index e81864402f5..ecb536e47d0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java
@@ -46,7 +46,7 @@ public class ReplicaAllocation implements Writable {
public static final ReplicaAllocation NOT_SET;
static {
- DEFAULT_ALLOCATION = new ReplicaAllocation((short) 3);
+ DEFAULT_ALLOCATION =
EnvFactory.getInstance().createDefReplicaAllocation();
NOT_SET = new ReplicaAllocation();
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
index 5e857acf884..23a844538a4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
@@ -56,7 +56,8 @@ public class TableProperty implements Writable {
private Map<String, String> properties;
// the follower variables are built from "properties"
- private DynamicPartitionProperty dynamicPartitionProperty = new
DynamicPartitionProperty(Maps.newHashMap());
+ private DynamicPartitionProperty dynamicPartitionProperty =
+
EnvFactory.getInstance().createDynamicPartitionProperty(Maps.newHashMap());
private ReplicaAllocation replicaAlloc =
ReplicaAllocation.DEFAULT_ALLOCATION;
private boolean isInMemory = false;
private short minLoadReplicaNum = -1;
@@ -180,7 +181,7 @@ public class TableProperty implements Writable {
dynamicPartitionProperties.put(entry.getKey(),
entry.getValue());
}
}
- dynamicPartitionProperty = new
DynamicPartitionProperty(dynamicPartitionProperties);
+ dynamicPartitionProperty =
EnvFactory.getInstance().createDynamicPartitionProperty(dynamicPartitionProperties);
return this;
}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
index badeb992465..984444a4f81 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
@@ -404,10 +404,10 @@ public class Tablet extends MetaObject implements
Writable {
public static Tablet read(DataInput in) throws IOException {
if (Env.getCurrentEnvJournalVersion() >= FeMetaVersion.VERSION_115) {
String json = Text.readString(in);
- return GsonUtils.GSON.fromJson(json, EnvFactory.getTabletClass());
+ return GsonUtils.GSON.fromJson(json,
EnvFactory.getInstance().getTabletClass());
}
- Tablet tablet = EnvFactory.createTablet();
+ Tablet tablet = EnvFactory.getInstance().createTablet();
tablet.readFields(in);
return tablet;
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudDynamicPartitionProperty.java
b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudDynamicPartitionProperty.java
new file mode 100644
index 00000000000..4f871cf6bac
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudDynamicPartitionProperty.java
@@ -0,0 +1,44 @@
+// 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.cloud.catalog;
+
+import org.apache.doris.catalog.DynamicPartitionProperty;
+
+import com.google.common.collect.Sets;
+
+import java.util.Map;
+import java.util.Set;
+
+public class CloudDynamicPartitionProperty extends DynamicPartitionProperty {
+
+ private static Set<String> unsupportedProperties = Sets.newHashSet(
+ DynamicPartitionProperty.REPLICATION_NUM,
+ DynamicPartitionProperty.REPLICATION_ALLOCATION,
+ DynamicPartitionProperty.STORAGE_POLICY);
+
+ public CloudDynamicPartitionProperty(Map<String, String> properties) {
+ super(properties);
+ }
+
+ @Override
+ protected boolean supportProperty(String property) {
+ return !unsupportedProperties.contains(property);
+ }
+
+}
+
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java
b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java
new file mode 100644
index 00000000000..46b916b9dbb
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/cloud/catalog/CloudEnvFactory.java
@@ -0,0 +1,102 @@
+// 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.cloud.catalog;
+
+import org.apache.doris.catalog.DynamicPartitionProperty;
+import org.apache.doris.catalog.Env;
+import org.apache.doris.catalog.EnvFactory;
+import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.Replica;
+import org.apache.doris.catalog.ReplicaAllocation;
+import org.apache.doris.catalog.Tablet;
+import org.apache.doris.cloud.common.util.CloudPropertyAnalyzer;
+import org.apache.doris.cloud.datasource.CloudInternalCatalog;
+import org.apache.doris.cloud.system.CloudSystemInfoService;
+import org.apache.doris.common.util.PropertyAnalyzer;
+import org.apache.doris.datasource.InternalCatalog;
+import org.apache.doris.system.SystemInfoService;
+
+import java.lang.reflect.Type;
+import java.util.Map;
+
+public class CloudEnvFactory extends EnvFactory {
+
+ public CloudEnvFactory() {
+ }
+
+ @Override
+ public Env createEnv(boolean isCheckpointCatalog) {
+ return new CloudEnv(isCheckpointCatalog);
+ }
+
+ @Override
+ public InternalCatalog createInternalCatalog() {
+ return new CloudInternalCatalog();
+ }
+
+ @Override
+ public SystemInfoService createSystemInfoService() {
+ return new CloudSystemInfoService();
+ }
+
+ @Override
+ public Type getPartitionClass() {
+ return CloudPartition.class;
+ }
+
+ @Override
+ public Partition createPartition() {
+ return new CloudPartition();
+ }
+
+ @Override
+ public Type getTabletClass() {
+ return CloudTablet.class;
+ }
+
+ @Override
+ public Tablet createTablet() {
+ return new CloudTablet();
+ }
+
+ @Override
+ public Tablet createTablet(long tabletId) {
+ return new CloudTablet(tabletId);
+ }
+
+ @Override
+ public Replica createReplica() {
+ return new CloudReplica();
+ }
+
+ @Override
+ public ReplicaAllocation createDefReplicaAllocation() {
+ return new ReplicaAllocation((short) 1);
+ }
+
+ @Override
+ public PropertyAnalyzer createPropertyAnalyzer() {
+ return new CloudPropertyAnalyzer();
+ }
+
+ @Override
+ public DynamicPartitionProperty createDynamicPartitionProperty(Map<String,
String> properties) {
+ return new CloudDynamicPartitionProperty(properties);
+ }
+
+}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/cloud/common/util/CloudPropertyAnalyzer.java
b/fe/fe-core/src/main/java/org/apache/doris/cloud/common/util/CloudPropertyAnalyzer.java
new file mode 100644
index 00000000000..0e6f0f7051f
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/cloud/common/util/CloudPropertyAnalyzer.java
@@ -0,0 +1,50 @@
+// 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.cloud.common.util;
+
+import org.apache.doris.catalog.DynamicPartitionProperty;
+import org.apache.doris.catalog.ReplicaAllocation;
+import org.apache.doris.common.util.PropertyAnalyzer;
+
+import com.google.common.collect.ImmutableMap;
+
+public class CloudPropertyAnalyzer extends PropertyAnalyzer {
+
+ public CloudPropertyAnalyzer() {
+ forceProperties = ImmutableMap.<String, String>builder()
+ .put(PropertyAnalyzer.PROPERTIES_INMEMORY, "true")
+ //.put(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM, null)
+ .put(PropertyAnalyzer.PROPERTIES_STORAGE_FORMAT, "")
+ .put(PropertyAnalyzer.PROPERTIES_STORAGE_POLICY, "")
+ .put(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TIME, "")
+ .put(PropertyAnalyzer.PROPERTIES_MIN_LOAD_REPLICA_NUM, "-1")
+ .put(PropertyAnalyzer.PROPERTIES_DISABLE_AUTO_COMPACTION,
"false")
+ .put(PropertyAnalyzer.PROPERTIES_ENABLE_LIGHT_SCHEMA_CHANGE,
"true")
+ .put(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM,
+
String.valueOf(ReplicaAllocation.DEFAULT_ALLOCATION.getTotalReplicaNum()))
+ .put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION,
+ ReplicaAllocation.DEFAULT_ALLOCATION.toCreateStmt())
+ //.put(DynamicPartitionProperty.PROPERTIES_STORAGE_MEDIUM, "")
+ .put(DynamicPartitionProperty.REPLICATION_NUM,
+
String.valueOf(ReplicaAllocation.DEFAULT_ALLOCATION.getTotalReplicaNum()))
+ .put(DynamicPartitionProperty.REPLICATION_ALLOCATION,
+ ReplicaAllocation.DEFAULT_ALLOCATION.toCreateStmt())
+ .build();
+ }
+
+}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java
b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java
index f2e3f10c280..d7efdefe775 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/cloud/datasource/CloudInternalCatalog.java
@@ -318,7 +318,7 @@ public class CloudInternalCatalog extends InternalCatalog {
DistributionInfo distributionInfo, long version, ReplicaAllocation
replicaAlloc,
TabletMeta tabletMeta, Set<Long> tabletIdSet) throws DdlException {
for (int i = 0; i < distributionInfo.getBucketNum(); ++i) {
- Tablet tablet =
EnvFactory.createTablet(Env.getCurrentEnv().getNextId());
+ Tablet tablet =
EnvFactory.getInstance().createTablet(Env.getCurrentEnv().getNextId());
// add tablet to inverted index first
index.addTablet(tablet, tabletMeta);
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java
b/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java
new file mode 100644
index 00000000000..5ae16c9f35b
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/cloud/system/CloudSystemInfoService.java
@@ -0,0 +1,43 @@
+// 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.cloud.system;
+
+import org.apache.doris.catalog.ReplicaAllocation;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.resource.Tag;
+import org.apache.doris.system.SystemInfoService;
+import org.apache.doris.thrift.TStorageMedium;
+
+import com.google.common.collect.Maps;
+
+import java.util.List;
+import java.util.Map;
+
+public class CloudSystemInfoService extends SystemInfoService {
+
+ @Override
+ public Map<Tag, List<Long>> selectBackendIdsForReplicaCreation(
+ ReplicaAllocation replicaAlloc, Map<Tag, Integer> nextIndexs,
+ TStorageMedium storageMedium, boolean isStorageMediumSpecified,
+ boolean isOnlyForCheck)
+ throws DdlException {
+ return Maps.newHashMap();
+ }
+
+}
+
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
index f1e457336f2..afebd26ee06 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
@@ -23,6 +23,7 @@ import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.DataProperty;
import org.apache.doris.catalog.DatabaseIf;
import org.apache.doris.catalog.Env;
+import org.apache.doris.catalog.EnvFactory;
import org.apache.doris.catalog.EsResource;
import org.apache.doris.catalog.KeysType;
import org.apache.doris.catalog.Partition;
@@ -48,6 +49,7 @@ import org.apache.doris.thrift.TTabletType;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.commons.lang3.StringUtils;
@@ -194,6 +196,27 @@ public class PropertyAnalyzer {
public static final long
TIME_SERIES_COMPACTION_TIME_THRESHOLD_SECONDS_DEFAULT_VALUE = 3600;
public static final long
TIME_SERIES_COMPACTION_EMPTY_ROWSETS_THRESHOLD_DEFAULT_VALUE = 5;
+ // Use forceProperties to rewrite olap's property.
+ // For a key-value pair in forceProperties,
+ // if the value is null, then delete this property from properties and
skip check this property,
+ // otherwise rewrite this property into properties and check property
using the force value.
+ //
+ // In most cases, specified a none-null force value is better then
specified a null force value.
+ protected ImmutableMap<String, String> forceProperties;
+
+ public PropertyAnalyzer() {
+ forceProperties = ImmutableMap.<String, String>builder()
+ .put(PROPERTIES_FILE_CACHE_TTL_SECONDS, "0")
+ .build();
+ }
+
+ private static class SingletonHolder {
+ private static final PropertyAnalyzer INSTANCE =
EnvFactory.getInstance().createPropertyAnalyzer();
+ }
+
+ public static PropertyAnalyzer getInstance() {
+ return SingletonHolder.INSTANCE;
+ }
/**
* check and replace members of DataProperty by properties.
@@ -1285,7 +1308,27 @@ public class PropertyAnalyzer {
}
}
- public static Map<String, String> rewriteReplicaAllocationProperties(
+ public Map<String, String> rewriteOlapProperties(
+ String ctl, String db, Map<String, String> properties) {
+ if (properties == null) {
+ properties = Maps.newHashMap();
+ }
+ rewriteReplicaAllocationProperties(ctl, db, properties);
+ rewriteForceProperties(properties);
+ return properties;
+ }
+
+ private void rewriteForceProperties(Map<String, String> properties) {
+ forceProperties.forEach((property, value) -> {
+ if (value == null) {
+ properties.remove(property);
+ } else {
+ properties.put(property, value);
+ }
+ });
+ }
+
+ private static Map<String, String> rewriteReplicaAllocationProperties(
String ctl, String db, Map<String, String> properties) {
if (Config.force_olap_table_replication_num <= 0) {
return rewriteReplicaAllocationPropertiesByDatabase(ctl, db,
properties);
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java
b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java
index 76d7702fef4..219f29ebec1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java
@@ -115,7 +115,7 @@ public class CatalogMgr implements Writable,
GsonPostProcessable {
}
private void initInternalCatalog() {
- internalCatalog = EnvFactory.createInternalCatalog();
+ internalCatalog = EnvFactory.getInstance().createInternalCatalog();
addCatalog(internalCatalog);
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
index ae32e45aa81..51787b66353 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
@@ -2877,7 +2877,7 @@ public class InternalCatalog implements
CatalogIf<Database> {
for (int i = 0; i < distributionInfo.getBucketNum(); ++i) {
// create a new tablet with random chosen backends
- Tablet tablet =
EnvFactory.createTablet(idGeneratorBuffer.getNextId());
+ Tablet tablet =
EnvFactory.getInstance().createTablet(idGeneratorBuffer.getNextId());
// add tablet to inverted index first
index.addTablet(tablet, tabletMeta);
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
index a1a5dd9836c..ffaba171b29 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateTableInfo.java
@@ -284,16 +284,15 @@ public class CreateTableInfo {
}
if (engineName.equalsIgnoreCase("olap")) {
- properties =
PropertyAnalyzer.rewriteReplicaAllocationProperties(ctlName, dbName,
- properties);
boolean enableDuplicateWithoutKeysByDefault = false;
- if (properties != null) {
- try {
+ properties =
PropertyAnalyzer.getInstance().rewriteOlapProperties(ctlName, dbName,
properties);
+ try {
+ if (properties != null) {
enableDuplicateWithoutKeysByDefault =
-
PropertyAnalyzer.analyzeEnableDuplicateWithoutKeysByDefault(properties);
- } catch (Exception e) {
- throw new AnalysisException(e.getMessage(), e.getCause());
+
PropertyAnalyzer.analyzeEnableDuplicateWithoutKeysByDefault(properties);
}
+ } catch (Exception e) {
+ throw new AnalysisException(e.getMessage(), e.getCause());
}
if (keys.isEmpty()) {
boolean hasAggColumn = false;
diff --git
a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java
b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java
index ccd865dd6d4..fed3f0d6771 100644
---
a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java
+++
b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateTableStmtTest.java
@@ -111,7 +111,8 @@ public class CreateTableStmtTest {
stmt.analyze(analyzer);
Assert.assertEquals("db1", stmt.getDbName());
Assert.assertEquals("table1", stmt.getTableName());
- Assert.assertTrue(stmt.getProperties() == null ||
stmt.getProperties().isEmpty());
+
Assert.assertEquals(PropertyAnalyzer.getInstance().rewriteOlapProperties("",
"", null),
+ stmt.getProperties());
}
@Test
@@ -121,7 +122,8 @@ public class CreateTableStmtTest {
stmt.analyze(analyzer);
Assert.assertEquals("db1", stmt.getDbName());
Assert.assertEquals("table1", stmt.getTableName());
- Assert.assertTrue(stmt.getProperties() == null ||
stmt.getProperties().isEmpty());
+
Assert.assertEquals(PropertyAnalyzer.getInstance().rewriteOlapProperties("",
"", null),
+ stmt.getProperties());
Assert.assertTrue(stmt.toSql().contains("DISTRIBUTED BY
RANDOM\nBUCKETS 6"));
}
@@ -273,7 +275,8 @@ public class CreateTableStmtTest {
stmt.analyze(analyzer);
Assert.assertEquals("db1", stmt.getDbName());
Assert.assertEquals("table1", stmt.getTableName());
- Assert.assertTrue(stmt.getProperties() == null ||
stmt.getProperties().isEmpty());
+
Assert.assertEquals(PropertyAnalyzer.getInstance().rewriteOlapProperties("",
"", null),
+ stmt.getProperties());
Assert.assertTrue(stmt.toSql()
.contains("rollup( `index1` (`col1`, `col2`) FROM `table1`,
`index2` (`col2`, `col3`) FROM `table1`)"));
}
@@ -287,7 +290,8 @@ public class CreateTableStmtTest {
Assert.assertEquals("testDb", stmt.getDbName());
Assert.assertEquals("table1", stmt.getTableName());
Assert.assertNull(stmt.getPartitionDesc());
- Assert.assertTrue(stmt.getProperties() == null ||
stmt.getProperties().isEmpty());
+
Assert.assertEquals(PropertyAnalyzer.getInstance().rewriteOlapProperties("",
"", null),
+ stmt.getProperties());
}
@Test(expected = AnalysisException.class)
diff --git
a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java
b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java
index 03f8d325d77..1031e3d81e3 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java
@@ -119,7 +119,7 @@ public class BinlogManagerTest {
new MockUp<Env>() {
@Mock
public InternalCatalog getCurrentInternalCatalog() {
- return EnvFactory.createInternalCatalog();
+ return EnvFactory.getInstance().createInternalCatalog();
}
};
diff --git
a/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java
b/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java
index 1a939457f5e..9701989b7ee 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/EnvFactoryTest.java
@@ -18,6 +18,7 @@
package org.apache.doris.catalog;
import org.apache.doris.cloud.catalog.CloudEnv;
+import org.apache.doris.cloud.catalog.CloudEnvFactory;
import org.apache.doris.cloud.catalog.CloudPartition;
import org.apache.doris.cloud.catalog.CloudReplica;
import org.apache.doris.cloud.catalog.CloudTablet;
@@ -33,27 +34,23 @@ public class EnvFactoryTest {
@Test
public void testCreate() throws Exception {
Config.cloud_unique_id = "";
+ EnvFactory envFactory = EnvFactory.getInstance();
+ Assert.assertTrue(envFactory instanceof EnvFactory);
+ Assert.assertFalse(envFactory instanceof CloudEnvFactory);
Assert.assertTrue(Env.getCurrentEnv() instanceof Env);
Assert.assertFalse(Env.getCurrentEnv() instanceof CloudEnv);
Assert.assertTrue(Env.getCurrentInternalCatalog() instanceof
InternalCatalog);
Assert.assertFalse(Env.getCurrentInternalCatalog() instanceof
CloudInternalCatalog);
- Assert.assertTrue(EnvFactory.createEnv(false) instanceof Env);
- Assert.assertFalse(EnvFactory.createEnv(false) instanceof CloudEnv);
- Assert.assertTrue(EnvFactory.createInternalCatalog() instanceof
InternalCatalog);
- Assert.assertFalse(EnvFactory.createInternalCatalog() instanceof
CloudInternalCatalog);
- Assert.assertTrue(EnvFactory.createPartition() instanceof Partition);
- Assert.assertFalse(EnvFactory.createPartition() instanceof
CloudPartition);
- Assert.assertTrue(EnvFactory.createTablet() instanceof Tablet);
- Assert.assertFalse(EnvFactory.createTablet() instanceof CloudTablet);
- Assert.assertTrue(EnvFactory.createReplica() instanceof Replica);
- Assert.assertFalse(EnvFactory.createReplica() instanceof CloudReplica);
-
- Config.cloud_unique_id = "test_cloud";
- Assert.assertTrue(EnvFactory.createEnv(false) instanceof CloudEnv);
- Assert.assertTrue(EnvFactory.createInternalCatalog() instanceof
CloudInternalCatalog);
- Assert.assertTrue(EnvFactory.createPartition() instanceof
CloudPartition);
- Assert.assertTrue(EnvFactory.createTablet() instanceof CloudTablet);
- Assert.assertTrue(EnvFactory.createReplica() instanceof CloudReplica);
+ Assert.assertTrue(envFactory.createEnv(false) instanceof Env);
+ Assert.assertFalse(envFactory.createEnv(false) instanceof CloudEnv);
+ Assert.assertTrue(envFactory.createInternalCatalog() instanceof
InternalCatalog);
+ Assert.assertFalse(envFactory.createInternalCatalog() instanceof
CloudInternalCatalog);
+ Assert.assertTrue(envFactory.createPartition() instanceof Partition);
+ Assert.assertFalse(envFactory.createPartition() instanceof
CloudPartition);
+ Assert.assertTrue(envFactory.createTablet() instanceof Tablet);
+ Assert.assertFalse(envFactory.createTablet() instanceof CloudTablet);
+ Assert.assertTrue(envFactory.createReplica() instanceof Replica);
+ Assert.assertFalse(envFactory.createReplica() instanceof CloudReplica);
}
}
diff --git
a/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/CloudEnvFactoryTest.java
b/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/CloudEnvFactoryTest.java
new file mode 100644
index 00000000000..6c3d0d3ab9b
--- /dev/null
+++
b/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/CloudEnvFactoryTest.java
@@ -0,0 +1,52 @@
+// 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.cloud.catalog;
+
+import org.apache.doris.catalog.Env;
+import org.apache.doris.catalog.EnvFactory;
+import org.apache.doris.cloud.datasource.CloudInternalCatalog;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.util.PropertyAnalyzer;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Map;
+
+public class CloudEnvFactoryTest {
+
+ @Test
+ public void testCreate() throws Exception {
+ Config.cloud_unique_id = "test_cloud";
+ EnvFactory envFactory = EnvFactory.getInstance();
+ Assert.assertTrue(envFactory instanceof CloudEnvFactory);
+ Assert.assertTrue(Env.getCurrentEnv() instanceof CloudEnv);
+ Assert.assertTrue(Env.getCurrentInternalCatalog() instanceof
CloudInternalCatalog);
+ Assert.assertTrue(envFactory.createEnv(false) instanceof CloudEnv);
+ Assert.assertTrue(envFactory.createInternalCatalog() instanceof
CloudInternalCatalog);
+ Assert.assertTrue(envFactory.createPartition() instanceof
CloudPartition);
+ Assert.assertTrue(envFactory.createTablet() instanceof CloudTablet);
+ Assert.assertTrue(envFactory.createReplica() instanceof CloudReplica);
+
+ Map<String, String> properties =
PropertyAnalyzer.getInstance().rewriteOlapProperties(
+ "catalog_not_exist", "db_not_exist", null);
+ Assert.assertEquals("1",
properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM));
+ }
+
+}
+
diff --git
a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java
b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java
index bffdfbd2c68..83dcdd68824 100644
---
a/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java
+++
b/fe/fe-core/src/test/java/org/apache/doris/statistics/StatisticsAutoCollectorTest.java
@@ -465,7 +465,7 @@ public class StatisticsAutoCollectorTest {
@Test
public void testDisableAuto1() throws Exception {
- InternalCatalog catalog1 = EnvFactory.createInternalCatalog();
+ InternalCatalog catalog1 =
EnvFactory.getInstance().createInternalCatalog();
List<CatalogIf> catalogs = Lists.newArrayList();
catalogs.add(catalog1);
@@ -493,7 +493,7 @@ public class StatisticsAutoCollectorTest {
@Test
public void testDisableAuto2() throws Exception {
- InternalCatalog catalog1 = EnvFactory.createInternalCatalog();
+ InternalCatalog catalog1 =
EnvFactory.getInstance().createInternalCatalog();
List<CatalogIf> catalogs = Lists.newArrayList();
catalogs.add(catalog1);
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]