You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by ya...@apache.org on 2021/03/08 02:22:32 UTC

[incubator-doris] branch branch-0.14 updated (0203ce7 -> 9aba744)

This is an automated email from the ASF dual-hosted git repository.

yangzhg pushed a change to branch branch-0.14
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git.


 discard 0203ce7  [Bug] Fix bug that data lost when doing backup job  (#5473)
 discard 48444ae  [Bug] Fix bug that the image cannot be pulled after the new fe nodes added (#5418)
 discard 2b43e4e  0.14.0-release-rc02
     new f36547d  [Bug] Fix bug that the image cannot be pulled after the new fe nodes added (#5418)
     new 9aba744  [Bug] Fix bug that data lost when doing backup job  (#5473)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (0203ce7)
            \
             N -- N -- N   refs/heads/branch-0.14 (9aba744)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 gensrc/script/gen_build_version.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 02/02: [Bug] Fix bug that data lost when doing backup job (#5473)

Posted by ya...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yangzhg pushed a commit to branch branch-0.14
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 9aba744a56ce215b6c9a30b43ad27cb830bc5401
Author: Mingyu Chen <mo...@gmail.com>
AuthorDate: Mon Mar 8 09:32:16 2021 +0800

    [Bug] Fix bug that data lost when doing backup job  (#5473)
    
    Backup job may delete the tablet of existing table, which cause data lost.
    
    (cherry picked from commit a1160bcd99ec527827a1a03177343ea775670ae6)
---
 .../java/org/apache/doris/backup/RestoreJob.java   |  11 +-
 .../java/org/apache/doris/catalog/Catalog.java     |  10 +-
 .../java/org/apache/doris/catalog/OlapTable.java   |  74 ++++++------
 .../apache/doris/catalog/DropPartitionTest.java    | 132 +++++++++++++++++++++
 4 files changed, 180 insertions(+), 47 deletions(-)

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 3be93d3..8c246fd 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
@@ -71,6 +71,9 @@ import org.apache.doris.thrift.TStorageMedium;
 import org.apache.doris.thrift.TStorageType;
 import org.apache.doris.thrift.TTaskType;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
@@ -82,9 +85,6 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.Range;
 import com.google.common.collect.Table.Cell;
 
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -622,8 +622,7 @@ public class RestoreJob extends AbstractJob {
                     Set<String> allPartNames = remoteOlapTbl.getPartitionNames();
                     for (String partName : allPartNames) {
                         if (!tblInfo.containsPart(partName)) {
-                            remoteOlapTbl.dropPartition(-1 /* db id is useless here */, partName,
-                                    true /* act like replay to disable recycle bin action */);
+                            remoteOlapTbl.dropPartitionAndReserveTablet(partName);
                         }
                     }
 
@@ -1567,7 +1566,7 @@ public class RestoreJob extends AbstractJob {
                         restoreTbl.getName(), entry.second.getName());
                 restoreTbl.writeLock();
                 try {
-                    restoreTbl.dropPartition(dbId, entry.second.getName(), true /* is restore */);
+                    restoreTbl.dropPartition(dbId, entry.second.getName(), true /* force drop */);
                 } finally {
                     restoreTbl.writeUnlock();
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
index 200f9d3..7f79320 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
@@ -217,6 +217,11 @@ import org.apache.doris.transaction.GlobalTransactionMgr;
 import org.apache.doris.transaction.PublishVersionDaemon;
 import org.apache.doris.transaction.UpdateDbUsedDataQuotaDaemon;
 
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.codehaus.jackson.map.ObjectMapper;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
@@ -231,11 +236,6 @@ import com.sleepycat.je.rep.InsufficientLogException;
 import com.sleepycat.je.rep.NetworkRestore;
 import com.sleepycat.je.rep.NetworkRestoreConfig;
 
-import org.apache.commons.collections.CollectionUtils;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-import org.codehaus.jackson.map.ObjectMapper;
-
 import java.io.BufferedInputStream;
 import java.io.BufferedReader;
 import java.io.DataInputStream;
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 b146204..e4afdca 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
@@ -667,7 +667,15 @@ public class OlapTable extends Table {
         nameToPartition.put(partition.getName(), partition);
     }
 
-    public Partition dropPartition(long dbId, String partitionName, boolean isForceDrop) {
+    // This is a private methid.
+    // Call public "dropPartitionAndReserveTablet" and "dropPartition"
+    private Partition dropPartition(long dbId, String partitionName, boolean isForceDrop, boolean reserveTablets) {
+        // 1. If "isForceDrop" is false, the partition will be added to the Catalog Recyle bin, and all tablets of this
+        //    partition will not be deleted.
+        // 2. If "ifForceDrop" is true, the partition will be dropped the immediately, but whether to drop the tablets
+        //    of this partition depends on "reserveTablets"
+        //    If "reserveTablets" is true, the tablets of this partition will not to deleted.
+        //    Otherwise, the tablets of this partition will be deleted immediately.
         Partition partition = nameToPartition.get(partitionName);
         if (partition != null) {
             idToPartition.remove(partition.getId());
@@ -675,15 +683,15 @@ public class OlapTable extends Table {
 
             Preconditions.checkState(partitionInfo.getType() == PartitionType.RANGE);
             RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo;
-            
+
             if (!isForceDrop) {
                 // recycle partition
                 Catalog.getCurrentRecycleBin().recyclePartition(dbId, id, partition,
-                                          rangePartitionInfo.getRange(partition.getId()),
-                                          rangePartitionInfo.getDataProperty(partition.getId()),
-                                          rangePartitionInfo.getReplicationNum(partition.getId()),
-                                          rangePartitionInfo.getIsInMemory(partition.getId()));
-            } else {
+                        rangePartitionInfo.getRange(partition.getId()),
+                        rangePartitionInfo.getDataProperty(partition.getId()),
+                        rangePartitionInfo.getReplicationNum(partition.getId()),
+                        rangePartitionInfo.getIsInMemory(partition.getId()));
+            } else if (!reserveTablets) {
                 Catalog.getCurrentCatalog().onErasePartition(partition);
             }
 
@@ -693,54 +701,58 @@ public class OlapTable extends Table {
         return partition;
     }
 
-    public Partition dropPartitionForBackup(String partitionName) {
-        return dropPartition(-1, partitionName, true);
+    public Partition dropPartitionAndReserveTablet(String partitionName) {
+        return dropPartition(-1, partitionName, true, true);
+    }
+
+    public Partition dropPartition(long dbId, String partitionName, boolean isForceDrop) {
+        return dropPartition(dbId, partitionName, isForceDrop, !isForceDrop);
     }
 
     /*
      * A table may contain both formal and temporary partitions.
      * There are several methods to get the partition of a table.
      * Typically divided into two categories:
-     * 
+     *
      * 1. Get partition by id
      * 2. Get partition by name
-     * 
+     *
      * According to different requirements, the caller may want to obtain
      * a formal partition or a temporary partition. These methods are
      * described below in order to obtain the partition by using the correct method.
-     * 
+     *
      * 1. Get by name
-     * 
+     *
      * This type of request usually comes from a user with partition names. Such as
      * `select * from tbl partition(p1);`.
      * This type of request has clear information to indicate whether to obtain a
      * formal or temporary partition.
      * Therefore, we need to get the partition through this method:
-     * 
+     *
      * `getPartition(String partitionName, boolean isTemp)`
-     * 
+     *
      * To avoid modifying too much code, we leave the `getPartition(String
      * partitionName)`, which is same as:
-     * 
+     *
      * `getPartition(partitionName, false)`
-     * 
+     *
      * 2. Get by id
-     * 
+     *
      * This type of request usually means that the previous step has obtained
      * certain partition ids in some way,
      * so we only need to get the corresponding partition through this method:
-     * 
+     *
      * `getPartition(long partitionId)`.
-     * 
+     *
      * This method will try to get both formal partitions and temporary partitions.
-     * 
+     *
      * 3. Get all partition instances
-     * 
+     *
      * Depending on the requirements, the caller may want to obtain all formal
      * partitions,
      * all temporary partitions, or all partitions. Therefore we provide 3 methods,
      * the caller chooses according to needs.
-     * 
+     *
      * `getPartitions()`
      * `getTempPartitions()`
      * `getAllPartitions()`
@@ -1248,7 +1260,7 @@ public class OlapTable extends Table {
         
         for (String partName : partNames) {
             if (!reservedPartitions.contains(partName)) {
-                copied.dropPartitionForBackup(partName);
+                copied.dropPartitionAndReserveTablet(partName);
             }
         }
         
@@ -1555,10 +1567,9 @@ public class OlapTable extends Table {
         
         // begin to replace
         // 1. drop old partitions
-        List<Partition> droppedPartitions = Lists.newArrayList();
         for (String partitionName : partitionNames) {
-            Partition partition = dropPartition(-1, partitionName, true);
-            droppedPartitions.add(partition);
+            // This will also drop all tablets of the partition from TabletInvertedIndex
+            dropPartition(-1, partitionName, true);
         }
 
         // 2. add temp partitions' range info to rangeInfo, and remove them from tempPartitionInfo
@@ -1572,15 +1583,6 @@ public class OlapTable extends Table {
             rangeInfo.moveRangeFromTempToFormal(partition.getId());
         }
 
-        // 3. delete old partition's tablets in inverted index
-        for (Partition partition : droppedPartitions) {
-            for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) {
-                for (Tablet tablet : index.getTablets()) {
-                    Catalog.getCurrentInvertedIndex().deleteTablet(tablet.getId());
-                }
-            }
-        }
-
         // change the name so that after replacing, the partition name remain unchanged
         if (!useTempPartitionName && partitionNames.size() == tempPartitionNames.size()) {
             for (int i = 0; i < tempPartitionNames.size(); i++) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java
new file mode 100644
index 0000000..ba2ef86
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java
@@ -0,0 +1,132 @@
+// 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.catalog;
+
+import org.apache.doris.analysis.AlterTableStmt;
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.analysis.RecoverPartitionStmt;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ExceptionChecker;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.utframe.UtFrameUtils;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.List;
+import java.util.UUID;
+
+public class DropPartitionTest {
+    private static String runningDir = "fe/mocked/DropPartitionTest/" + UUID.randomUUID().toString() + "/";
+
+    private static ConnectContext connectContext;
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        UtFrameUtils.createMinDorisCluster(runningDir);
+
+        // create connect context
+        connectContext = UtFrameUtils.createDefaultCtx();
+        // create database
+        String createDbStmtStr = "create database test;";
+        String createTablleStr = "create table test.tbl1(d1 date, k1 int, k2 bigint) duplicate key(d1, k1) "
+                + "PARTITION BY RANGE(d1) (PARTITION p20210201 VALUES [('2021-02-01'), ('2021-02-02')),"
+                + "PARTITION p20210202 VALUES [('2021-02-02'), ('2021-02-03')),"
+                + "PARTITION p20210203 VALUES [('2021-02-03'), ('2021-02-04'))) distributed by hash(k1) "
+                + "buckets 1 properties('replication_num' = '1');";
+        createDb(createDbStmtStr);
+        createTable(createTablleStr);
+    }
+
+    @AfterClass
+    public static void tearDown() {
+        File file = new File(runningDir);
+        file.delete();
+    }
+
+    private static void createDb(String sql) throws Exception {
+        CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
+        Catalog.getCurrentCatalog().createDb(createDbStmt);
+    }
+
+    private static void createTable(String sql) throws Exception {
+        CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
+        Catalog.getCurrentCatalog().createTable(createTableStmt);
+    }
+
+    private static void dropPartition(String sql) throws Exception {
+        AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
+        Catalog.getCurrentCatalog().alterTable(alterTableStmt);
+    }
+
+    @Test
+    public void testNormalDropPartition() throws Exception {
+        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        OlapTable table = (OlapTable) db.getTable("tbl1");
+        Partition partition = table.getPartition("p20210201");
+        long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
+        String dropPartitionSql = " alter table test.tbl1 drop partition p20210201;";
+        dropPartition(dropPartitionSql);
+        List<Replica> replicaList = Catalog.getCurrentCatalog().getTabletInvertedIndex().getReplicasByTabletId(tabletId);
+        partition = table.getPartition("p20210201");
+        Assert.assertEquals(1, replicaList.size());
+        Assert.assertNull(partition);
+        String recoverPartitionSql = "recover partition p20210201 from test.tbl1";
+        RecoverPartitionStmt recoverPartitionStmt = (RecoverPartitionStmt) UtFrameUtils.parseAndAnalyzeStmt(recoverPartitionSql, connectContext);
+        Catalog.getCurrentCatalog().recoverPartition(recoverPartitionStmt);
+        partition = table.getPartition("p20210201");
+        Assert.assertNotNull(partition);
+        Assert.assertEquals("p20210201", partition.getName());
+    }
+
+    @Test
+    public void testForceDropPartition() throws Exception {
+        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        OlapTable table = (OlapTable) db.getTable("tbl1");
+        Partition partition = table.getPartition("p20210202");
+        long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
+        String dropPartitionSql = " alter table test.tbl1 drop partition p20210202 force;";
+        dropPartition(dropPartitionSql);
+        List<Replica> replicaList = Catalog.getCurrentCatalog().getTabletInvertedIndex().getReplicasByTabletId(tabletId);
+        partition = table.getPartition("p20210202");
+        Assert.assertTrue(replicaList.isEmpty());
+        Assert.assertNull(partition);
+        String recoverPartitionSql = "recover partition p20210202 from test.tbl1";
+        RecoverPartitionStmt recoverPartitionStmt = (RecoverPartitionStmt) UtFrameUtils.parseAndAnalyzeStmt(recoverPartitionSql, connectContext);
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class,
+                "No partition named p20210202 in table tbl1",
+                () -> Catalog.getCurrentCatalog().recoverPartition(recoverPartitionStmt));
+    }
+
+    @Test
+    public void testDropPartitionAndReserveTablets() throws Exception {
+        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        OlapTable table = (OlapTable) db.getTable("tbl1");
+        Partition partition = table.getPartition("p20210203");
+        long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
+        table.dropPartitionAndReserveTablet("p20210203");
+        List<Replica> replicaList = Catalog.getCurrentCatalog().getTabletInvertedIndex().getReplicasByTabletId(tabletId);
+        partition = table.getPartition("p20210203");
+        Assert.assertEquals(1, replicaList.size());
+        Assert.assertNull(partition);
+    }
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[incubator-doris] 01/02: [Bug] Fix bug that the image cannot be pulled after the new fe nodes added (#5418)

Posted by ya...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yangzhg pushed a commit to branch branch-0.14
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit f36547dcd940d7c3bf524d3bd6ac523df4de717e
Author: Mingyu Chen <mo...@gmail.com>
AuthorDate: Fri Mar 5 10:50:54 2021 +0800

    [Bug] Fix bug that the image cannot be pulled after the new fe nodes added (#5418)
    
    * [Bug] Fix bug that the image cannot be pulled after the new fe node is added
    
    This is because httpv2 modified the response body of the "/info" api,
    causing FE to fail to obtain info from this api.
    And the system did not exit correctly.
    
    This will also cause issues in issue #5292
    
    (cherry picked from commit c95f00d5081082d328fa857b49a8bd0b58404611)
---
 .../java/org/apache/doris/catalog/Catalog.java     | 26 +++++++++++++++++--
 .../org/apache/doris/persist/StorageInfoV2.java    | 30 ++++++++++++++++++++++
 2 files changed, 54 insertions(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
index 7ea431f..200f9d3 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
@@ -185,6 +185,7 @@ import org.apache.doris.persist.ReplicaPersistInfo;
 import org.apache.doris.persist.SetReplicaStatusOperationLog;
 import org.apache.doris.persist.Storage;
 import org.apache.doris.persist.StorageInfo;
+import org.apache.doris.persist.StorageInfoV2;
 import org.apache.doris.persist.TableInfo;
 import org.apache.doris.persist.TablePropertyInfo;
 import org.apache.doris.persist.TruncateTableInfo;
@@ -236,12 +237,14 @@ import org.apache.logging.log4j.Logger;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import java.io.BufferedInputStream;
+import java.io.BufferedReader;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.util.ArrayList;
@@ -1394,7 +1397,7 @@ public class Catalog {
                 MetaHelper.complete(filename, dir);
             }
         } catch (Exception e) {
-            return;
+            throw new IOException(e);
         }
     }
 
@@ -1425,7 +1428,26 @@ public class Catalog {
             connection = (HttpURLConnection) url.openConnection();
             connection.setConnectTimeout(HTTP_TIMEOUT_SECOND * 1000);
             connection.setReadTimeout(HTTP_TIMEOUT_SECOND * 1000);
-            return mapper.readValue(connection.getInputStream(), StorageInfo.class);
+
+            String response;
+            try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(connection.getInputStream()))) {
+                String line;
+                StringBuilder sb = new StringBuilder();
+                while ((line = bufferedReader.readLine()) != null) {
+                    sb.append(line);
+                }
+                response = sb.toString();
+            }
+
+            // For http v2, the response body for "/info" api changed from
+            // StorageInfo to StorageInfoV2.
+            // So we need to make it compatible with old api.
+            try {
+                return mapper.readValue(response, StorageInfo.class);
+            } catch (Exception e) {
+                // try new response body
+                return mapper.readValue(response, StorageInfoV2.class).data;
+            }
         } finally {
             if (connection != null) {
                 connection.disconnect();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/StorageInfoV2.java b/fe/fe-core/src/main/java/org/apache/doris/persist/StorageInfoV2.java
new file mode 100644
index 0000000..8f1bba7
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/StorageInfoV2.java
@@ -0,0 +1,30 @@
+// 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.persist;
+
+/**
+ * This class is wrapper of StorageInfo.
+ * Because for http v2, the response body of "/info" api changed to:
+ * {"msg":"success","code":0,"data":{"clusterID":1464655034,"imageSeq":809779,"editsSeq":0},"count":0}
+ */
+public class StorageInfoV2 {
+    public String msg;
+    public int code;
+    public StorageInfo data;
+    public int count;
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org