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

morrysnow pushed a commit to branch branch-3.1
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-3.1 by this push:
     new 002fd51b0aa branch-3.1: [opt](filecache) Let base compaction output 
rowsets write filecache when cache size is enough #53801 (#54692)
002fd51b0aa is described below

commit 002fd51b0aaa3dcf15e869e023e79c7df81be22c
Author: Lei Zhang <[email protected]>
AuthorDate: Thu Aug 14 13:51:36 2025 +0800

    branch-3.1: [opt](filecache) Let base compaction output rowsets write 
filecache when cache size is enough #53801 (#54692)
    
    picked from #53801
---
 be/src/common/config.cpp                           |   6 +
 be/src/common/config.h                             |   1 +
 be/src/io/cache/block_file_cache.h                 |   6 +
 be/src/io/fs/broker_file_writer.h                  |   1 -
 be/src/io/fs/file_writer.h                         |  37 +++-
 be/src/io/fs/hdfs_file_writer.cpp                  |   8 +-
 be/src/io/fs/hdfs_file_writer.h                    |   6 -
 be/src/io/fs/local_file_writer.h                   |   3 -
 be/src/io/fs/s3_file_writer.cpp                    |   9 +-
 be/src/io/fs/s3_file_writer.h                      |   6 -
 be/src/io/fs/stream_sink_file_writer.h             |   3 -
 be/src/olap/compaction.cpp                         |   2 +
 be/src/olap/rowset/rowset_writer_context.h         |   6 +-
 be/test/olap/tablet_cooldown_test.cpp              |   2 -
 .../test_filecache_with_base_compaction.groovy     | 195 +++++++++++++++++++++
 15 files changed, 252 insertions(+), 39 deletions(-)

diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp
index c7e068d0e59..f599434441a 100644
--- a/be/src/common/config.cpp
+++ b/be/src/common/config.cpp
@@ -1121,7 +1121,13 @@ DEFINE_mBool(enbale_dump_error_file, "false");
 DEFINE_mInt64(file_cache_error_log_limit_bytes, "209715200"); // 200MB
 DEFINE_mInt64(cache_lock_wait_long_tail_threshold_us, "30000000");
 DEFINE_mInt64(cache_lock_held_long_tail_threshold_us, "30000000");
+
+// enable_file_cache_keep_base_compaction_output true means force base 
compaction output rowsets
+// write to file cache, enable_file_cache_adaptive_write true means when file 
cache is enough, it
+// will write to file cache; satisfying any of the two conditions will write 
to file cache.
 DEFINE_mBool(enable_file_cache_keep_base_compaction_output, "false");
+DEFINE_mBool(enable_file_cache_adaptive_write, "true");
+
 DEFINE_mInt64(file_cache_remove_block_qps_limit, "1000");
 DEFINE_mInt64(file_cache_background_gc_interval_ms, "100");
 DEFINE_mBool(enable_reader_dryrun_when_download_file_cache, "true");
diff --git a/be/src/common/config.h b/be/src/common/config.h
index 2a9ba7bd293..d65047a7301 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -1161,6 +1161,7 @@ DECLARE_mInt64(cache_lock_held_long_tail_threshold_us);
 // If your file cache is ample enough to accommodate all the data in your 
database,
 // enable this option; otherwise, it is recommended to leave it disabled.
 DECLARE_mBool(enable_file_cache_keep_base_compaction_output);
+DECLARE_mBool(enable_file_cache_adaptive_write);
 DECLARE_mInt64(file_cache_remove_block_qps_limit);
 DECLARE_mInt64(file_cache_background_gc_interval_ms);
 DECLARE_mBool(enable_reader_dryrun_when_download_file_cache);
diff --git a/be/src/io/cache/block_file_cache.h 
b/be/src/io/cache/block_file_cache.h
index eaf62c1a82c..c0fe2cd2a09 100644
--- a/be/src/io/cache/block_file_cache.h
+++ b/be/src/io/cache/block_file_cache.h
@@ -20,6 +20,7 @@
 #include <bvar/bvar.h>
 #include <concurrentqueue.h>
 
+#include <algorithm>
 #include <boost/lockfree/spsc_queue.hpp>
 #include <memory>
 #include <mutex>
@@ -283,6 +284,11 @@ public:
     using QueryFileCacheContextHolderPtr = 
std::unique_ptr<QueryFileCacheContextHolder>;
     QueryFileCacheContextHolderPtr get_query_context_holder(const TUniqueId& 
query_id);
 
+    int64_t approximate_available_cache_size() const {
+        return std::max<int64_t>(
+                _cache_capacity_metrics->get_value() - 
_cur_cache_size_metrics->get_value(), 0);
+    }
+
 private:
     struct FileBlockCell {
         FileBlockSPtr file_block;
diff --git a/be/src/io/fs/broker_file_writer.h 
b/be/src/io/fs/broker_file_writer.h
index 173f1df1088..6c46ee2d2fa 100644
--- a/be/src/io/fs/broker_file_writer.h
+++ b/be/src/io/fs/broker_file_writer.h
@@ -49,7 +49,6 @@ public:
     const Path& path() const override { return _path; }
     size_t bytes_appended() const override { return _cur_offset; }
     State state() const override { return _state; }
-    FileCacheAllocatorBuilder* cache_builder() const override { return 
nullptr; }
 
 private:
     Status _write(const uint8_t* buf, size_t buf_len, size_t* written_bytes);
diff --git a/be/src/io/fs/file_writer.h b/be/src/io/fs/file_writer.h
index c388dcd40d8..9ff06bf5a37 100644
--- a/be/src/io/fs/file_writer.h
+++ b/be/src/io/fs/file_writer.h
@@ -22,6 +22,9 @@
 
 #include "common/status.h"
 #include "gutil/macros.h"
+#include "io/cache/block_file_cache.h"
+#include "io/cache/block_file_cache_factory.h"
+#include "io/cache/file_cache_common.h"
 #include "io/fs/file_reader.h"
 #include "io/fs/file_reader_writer_fwd.h"
 #include "io/fs/path.h"
@@ -43,8 +46,9 @@ struct FileWriterOptions {
     bool used_by_s3_committer = false;
     bool write_file_cache = false;
     bool is_cold_data = false;
-    bool sync_file_data = true;         // Whether flush data into storage 
system
-    uint64_t file_cache_expiration = 0; // Absolute time
+    bool sync_file_data = true;              // Whether flush data into 
storage system
+    uint64_t file_cache_expiration = 0;      // Absolute time
+    uint64_t approximate_bytes_to_write = 0; // Approximate bytes to write, 
used for file cache
 };
 
 struct AsyncCloseStatusPack {
@@ -79,7 +83,34 @@ public:
 
     virtual State state() const = 0;
 
-    virtual FileCacheAllocatorBuilder* cache_builder() const = 0;
+    FileCacheAllocatorBuilder* cache_builder() const {
+        return _cache_builder == nullptr ? nullptr : _cache_builder.get();
+    }
+
+protected:
+    void init_cache_builder(const FileWriterOptions* opts, const Path& path) {
+        if (!config::enable_file_cache || opts == nullptr) {
+            return;
+        }
+
+        io::UInt128Wrapper path_hash = 
BlockFileCache::hash(path.filename().native());
+        BlockFileCache* file_cache_ptr = 
FileCacheFactory::instance()->get_by_path(path_hash);
+
+        bool has_enough_file_cache_space = 
config::enable_file_cache_adaptive_write &&
+                                           (opts->approximate_bytes_to_write > 
0) &&
+                                           
(file_cache_ptr->approximate_available_cache_size() >
+                                            opts->approximate_bytes_to_write);
+
+        if (opts->write_file_cache || has_enough_file_cache_space) {
+            _cache_builder = 
std::make_unique<FileCacheAllocatorBuilder>(FileCacheAllocatorBuilder {
+                    opts ? opts->is_cold_data : false, opts ? 
opts->file_cache_expiration : 0,
+                    path_hash, file_cache_ptr});
+        }
+        return;
+    }
+
+    std::unique_ptr<FileCacheAllocatorBuilder> _cache_builder =
+            nullptr; // nullptr if disable write file cache
 };
 
 } // namespace doris::io
diff --git a/be/src/io/fs/hdfs_file_writer.cpp 
b/be/src/io/fs/hdfs_file_writer.cpp
index ff68d1c837a..2f3863179ee 100644
--- a/be/src/io/fs/hdfs_file_writer.cpp
+++ b/be/src/io/fs/hdfs_file_writer.cpp
@@ -145,13 +145,7 @@ HdfsFileWriter::HdfsFileWriter(Path path, 
std::shared_ptr<HdfsHandler> handler,
           _fs_name(std::move(fs_name)),
           _sync_file_data(opts ? opts->sync_file_data : true),
           _batch_buffer(MB * config::hdfs_write_batch_buffer_size_mb) {
-    if (config::enable_file_cache && opts != nullptr && 
opts->write_file_cache) {
-        _cache_builder = 
std::make_unique<FileCacheAllocatorBuilder>(FileCacheAllocatorBuilder {
-                opts ? opts->is_cold_data : false, opts ? 
opts->file_cache_expiration : 0,
-                BlockFileCache::hash(_path.filename().native()),
-                FileCacheFactory::instance()->get_by_path(
-                        BlockFileCache::hash(_path.filename().native()))});
-    }
+    init_cache_builder(opts, _path);
     hdfs_file_writer_total << 1;
 
     TEST_SYNC_POINT("HdfsFileWriter");
diff --git a/be/src/io/fs/hdfs_file_writer.h b/be/src/io/fs/hdfs_file_writer.h
index 25835658f53..fb6b778a2ff 100644
--- a/be/src/io/fs/hdfs_file_writer.h
+++ b/be/src/io/fs/hdfs_file_writer.h
@@ -52,10 +52,6 @@ public:
 
     Status close(bool non_block = false) override;
 
-    FileCacheAllocatorBuilder* cache_builder() const override {
-        return _cache_builder == nullptr ? nullptr : _cache_builder.get();
-    }
-
 private:
     Status _close_impl();
     // Flush buffered data into HDFS client and write local file cache if 
enabled
@@ -73,8 +69,6 @@ private:
     std::string _fs_name;
     size_t _bytes_appended = 0;
     bool _sync_file_data;
-    std::unique_ptr<FileCacheAllocatorBuilder>
-            _cache_builder; // nullptr if disable write file cache
     class BatchBuffer {
     public:
         BatchBuffer(size_t capacity);
diff --git a/be/src/io/fs/local_file_writer.h b/be/src/io/fs/local_file_writer.h
index c27ebb931a8..66456b325b6 100644
--- a/be/src/io/fs/local_file_writer.h
+++ b/be/src/io/fs/local_file_writer.h
@@ -35,9 +35,6 @@ public:
     const Path& path() const override { return _path; }
     size_t bytes_appended() const override;
     State state() const override { return _state; }
-
-    FileCacheAllocatorBuilder* cache_builder() const override { return 
nullptr; }
-
     Status close(bool non_block = false) override;
 
 private:
diff --git a/be/src/io/fs/s3_file_writer.cpp b/be/src/io/fs/s3_file_writer.cpp
index 5eabeea888d..756f83c78da 100644
--- a/be/src/io/fs/s3_file_writer.cpp
+++ b/be/src/io/fs/s3_file_writer.cpp
@@ -61,13 +61,8 @@ S3FileWriter::S3FileWriter(std::shared_ptr<ObjClientHolder> 
client, std::string
     s3_file_writer_total << 1;
     s3_file_being_written << 1;
     Aws::Http::SetCompliantRfc3986Encoding(true);
-    if (config::enable_file_cache && opts != nullptr && 
opts->write_file_cache) {
-        _cache_builder = 
std::make_unique<FileCacheAllocatorBuilder>(FileCacheAllocatorBuilder {
-                opts ? opts->is_cold_data : false, opts ? 
opts->file_cache_expiration : 0,
-                
BlockFileCache::hash(_obj_storage_path_opts.path.filename().native()),
-                FileCacheFactory::instance()->get_by_path(
-                        
BlockFileCache::hash(_obj_storage_path_opts.path.filename().native()))});
-    }
+
+    init_cache_builder(opts, _obj_storage_path_opts.path);
 }
 
 S3FileWriter::~S3FileWriter() {
diff --git a/be/src/io/fs/s3_file_writer.h b/be/src/io/fs/s3_file_writer.h
index 95ad52ddb67..c02f7bc1c7f 100644
--- a/be/src/io/fs/s3_file_writer.h
+++ b/be/src/io/fs/s3_file_writer.h
@@ -58,10 +58,6 @@ public:
     size_t bytes_appended() const override { return _bytes_appended; }
     State state() const override { return _state; }
 
-    FileCacheAllocatorBuilder* cache_builder() const override {
-        return _cache_builder == nullptr ? nullptr : _cache_builder.get();
-    }
-
     const std::vector<ObjectCompleteMultiPart>& completed_parts() const { 
return _completed_parts; }
 
     const std::string& key() const { return _obj_storage_path_opts.key; }
@@ -103,8 +99,6 @@ private:
     size_t _bytes_appended = 0;
 
     std::shared_ptr<FileBuffer> _pending_buf;
-    std::unique_ptr<FileCacheAllocatorBuilder>
-            _cache_builder; // nullptr if disable write file cache
 
     // S3 committer will start multipart uploading all files on BE side,
     // and then complete multipart upload these files on FE side.
diff --git a/be/src/io/fs/stream_sink_file_writer.h 
b/be/src/io/fs/stream_sink_file_writer.h
index 0950039077b..f092319e7fa 100644
--- a/be/src/io/fs/stream_sink_file_writer.h
+++ b/be/src/io/fs/stream_sink_file_writer.h
@@ -53,9 +53,6 @@ public:
         static Path dummy;
         return dummy;
     }
-
-    FileCacheAllocatorBuilder* cache_builder() const override { return 
nullptr; }
-
     Status close(bool non_block = false) override;
 
 private:
diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp
index 7fdb3e1d52b..73a295a590e 100644
--- a/be/src/olap/compaction.cpp
+++ b/be/src/olap/compaction.cpp
@@ -1475,6 +1475,8 @@ Status 
CloudCompactionMixin::construct_output_rowset_writer(RowsetWriterContext&
                            
(config::enable_file_cache_keep_base_compaction_output &&
                             compaction_type() == 
ReaderType::READER_BASE_COMPACTION);
     ctx.file_cache_ttl_sec = _tablet->ttl_seconds();
+    ctx.approximate_bytes_to_write = _input_rowsets_total_size;
+
     _output_rs_writer = DORIS_TRY(_tablet->create_rowset_writer(ctx, 
_is_vertical));
     RETURN_IF_ERROR(
             
_engine.meta_mgr().prepare_rowset(*_output_rs_writer->rowset_meta().get(), 
_uuid));
diff --git a/be/src/olap/rowset/rowset_writer_context.h 
b/be/src/olap/rowset/rowset_writer_context.h
index ab62fc6eda9..f304191b191 100644
--- a/be/src/olap/rowset/rowset_writer_context.h
+++ b/be/src/olap/rowset/rowset_writer_context.h
@@ -100,6 +100,7 @@ struct RowsetWriterContext {
     bool write_file_cache = false;
     bool is_hot_data = false;
     uint64_t file_cache_ttl_sec = 0;
+    uint64_t approximate_bytes_to_write = 0;
     /// end file cache opts
 
     // segcompaction for this RowsetWriter, disable it for some transient 
writers
@@ -155,7 +156,10 @@ struct RowsetWriterContext {
                 .is_cold_data = is_hot_data,
                 .file_cache_expiration = file_cache_ttl_sec > 0 && 
newest_write_timestamp > 0
                                                  ? newest_write_timestamp + 
file_cache_ttl_sec
-                                                 : 0};
+                                                 : 0,
+                .approximate_bytes_to_write = approximate_bytes_to_write,
+        };
+
         return opts;
     }
 };
diff --git a/be/test/olap/tablet_cooldown_test.cpp 
b/be/test/olap/tablet_cooldown_test.cpp
index fbcbb443131..fc167500f63 100644
--- a/be/test/olap/tablet_cooldown_test.cpp
+++ b/be/test/olap/tablet_cooldown_test.cpp
@@ -111,8 +111,6 @@ public:
 
     const Path& path() const override { return _local_file_writer->path(); }
 
-    io::FileCacheAllocatorBuilder* cache_builder() const override { return 
nullptr; }
-
 private:
     std::unique_ptr<io::FileWriter> _local_file_writer;
 };
diff --git 
a/regression-test/suites/compaction/test_filecache_with_base_compaction.groovy 
b/regression-test/suites/compaction/test_filecache_with_base_compaction.groovy
new file mode 100644
index 00000000000..7c067afd768
--- /dev/null
+++ 
b/regression-test/suites/compaction/test_filecache_with_base_compaction.groovy
@@ -0,0 +1,195 @@
+// 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.
+
+import java.util.concurrent.atomic.AtomicBoolean
+import org.apache.doris.regression.suite.ClusterOptions
+import org.codehaus.groovy.runtime.IOGroovyMethods
+import org.apache.doris.regression.util.Http
+
+suite("test_filecache_with_base_compaction", "docker") {
+    def options = new ClusterOptions()
+    options.cloudMode = true
+    options.setFeNum(1)
+    options.setBeNum(1)
+
+    options.beConfigs.add('enable_flush_file_cache_async=false')
+    
options.beConfigs.add('file_cache_enter_disk_resource_limit_mode_percent=99')
+    options.beConfigs.add('enable_evict_file_cache_in_advance=false')
+    options.beConfigs.add('')
+
+    def testTable = "test_filecache_with_base_compaction"
+    def backendId_to_backendIP = [:]
+    def backendId_to_backendHttpPort = [:]
+    def backendId_to_backendBrpcPort = [:]
+
+    def triggerCumulativeCompaction = { tablet ->
+        String tablet_id = tablet.TabletId
+        String trigger_backend_id = tablet.BackendId
+        def be_host = backendId_to_backendIP[trigger_backend_id]
+        def be_http_port = backendId_to_backendHttpPort[trigger_backend_id]
+        def (code_1, out_1, err_1) = be_run_cumulative_compaction(be_host, 
be_http_port, tablet_id)
+        logger.info("Run compaction: code=" + code_1 + ", out=" + out_1 + ", 
err=" + err_1)
+        assertEquals(code_1, 0)
+        return out_1
+    }
+
+    def triggerBaseCompaction = { tablet ->
+        String tablet_id = tablet.TabletId
+        String trigger_backend_id = tablet.BackendId
+        def be_host = backendId_to_backendIP[trigger_backend_id]
+        def be_http_port = backendId_to_backendHttpPort[trigger_backend_id]
+        def (code_1, out_1, err_1) = be_run_base_compaction(be_host, 
be_http_port, tablet_id)
+        logger.info("Run compaction: code=" + code_1 + ", out=" + out_1 + ", 
err=" + err_1)
+        assertEquals(code_1, 0)
+        return out_1
+    }
+
+    def getTabletStatus = { tablet ->
+        String tablet_id = tablet.TabletId
+        String trigger_backend_id = tablet.BackendId
+        def be_host = backendId_to_backendIP[trigger_backend_id]
+        def be_http_port = backendId_to_backendHttpPort[trigger_backend_id]
+        StringBuilder sb = new StringBuilder();
+        sb.append("curl -X GET http://${be_host}:${be_http_port}";)
+        sb.append("/api/compaction/show?tablet_id=")
+        sb.append(tablet_id)
+
+        String command = sb.toString()
+        logger.info(command)
+        def process = command.execute()
+        def code = process.waitFor()
+        def out = process.getText()
+        logger.info("Get tablet status:  =" + code + ", out=" + out)
+        assertEquals(code, 0)
+        def tabletStatus = parseJson(out.trim())
+        return tabletStatus
+    }
+
+    def waitForCompaction = { tablet ->
+        String tablet_id = tablet.TabletId
+        String trigger_backend_id = tablet.BackendId
+        def be_host = backendId_to_backendIP[trigger_backend_id]
+        def be_http_port = backendId_to_backendHttpPort[trigger_backend_id]
+        def running = true
+        do {
+            Thread.sleep(1000)
+            StringBuilder sb = new StringBuilder();
+            sb.append("curl -X GET http://${be_host}:${be_http_port}";)
+            sb.append("/api/compaction/run_status?tablet_id=")
+            sb.append(tablet_id)
+
+            String command = sb.toString()
+            logger.info(command)
+            def process = command.execute()
+            def code = process.waitFor()
+            def out = process.getText()
+            logger.info("Get compaction status: code=" + code + ", out=" + out)
+            assertEquals(code, 0)
+            def compactionStatus = parseJson(out.trim())
+            assertEquals("success", compactionStatus.status.toLowerCase())
+            running = compactionStatus.run_status
+        } while (running)
+    }
+
+    docker(options) {
+        def fes = sql_return_maparray "show frontends"
+        logger.info("frontends: ${fes}")
+        def url = "jdbc:mysql://${fes[0].Host}:${fes[0].QueryPort}/"
+        logger.info("url: " + url)
+
+        def result = sql 'SELECT DATABASE()'
+
+        sql """ DROP TABLE IF EXISTS ${testTable} """
+
+        sql """ CREATE TABLE ${testTable}
+                (
+                    siteid INT DEFAULT '10',
+                    citycode SMALLINT NOT NULL,
+                    username VARCHAR(32) DEFAULT '',
+                    pv BIGINT DEFAULT '0'
+                )
+                DUPLICATE KEY(siteid, citycode, username)
+                DISTRIBUTED BY HASH(siteid) BUCKETS 1
+                PROPERTIES (
+                "disable_auto_compaction" = "true"
+                )
+        """
+
+        // getBackendIpHttpPort(backendId_to_backendIP, 
backendId_to_backendHttpPort);
+        getBackendIpHttpAndBrpcPort(backendId_to_backendIP, 
backendId_to_backendHttpPort, backendId_to_backendBrpcPort);
+
+        def tablets = sql_return_maparray """ show tablets from ${testTable}; 
"""
+        logger.info("tablets: " + tablets)
+        assertEquals(1, tablets.size())
+        def tablet = tablets[0]
+        String tablet_id = tablet.TabletId
+
+        sql """ insert into ${testTable}(siteid, citycode, username, pv) 
values (1, 1, "xxx", 1); """
+        sql """ delete from ${testTable} where siteid=2; """
+        sql """ delete from ${testTable} where siteid=2; """
+        sql """ delete from ${testTable} where siteid=2; """
+        sql """ delete from ${testTable} where siteid=2; """
+        sql """ delete from ${testTable} where siteid=2; """
+        sql """ delete from ${testTable} where siteid=2; """
+        sql """ delete from ${testTable} where siteid=2; """
+
+        sql """ sync """
+        sql "select * from ${testTable}"
+        def tablet_status = getTabletStatus(tablet)
+        logger.info("tablet status: ${tablet_status}")
+
+        triggerCumulativeCompaction(tablet)
+        waitForCompaction(tablet)
+        triggerCumulativeCompaction(tablet)
+        waitForCompaction(tablet)
+        triggerBaseCompaction(tablet)
+        waitForCompaction(tablet)
+
+        tablet_status = getTabletStatus(tablet)
+        logger.info("tablet status: ${tablet_status}")
+        def base_compaction_finished = false
+        Set<String> final_rowsets = new HashSet<>();
+        for (int i = 0; i < 100; i++) {
+            tablet_status = getTabletStatus(tablet)
+            if (tablet_status["rowsets"].size() == 2) {
+                base_compaction_finished = true
+                final_rowsets.addAll(tablet_status["rowsets"])
+                break
+            }
+            sleep(500)
+        }
+        assertTrue(base_compaction_finished)
+
+        def be_host = backendId_to_backendIP[tablet.BackendId]
+        def be_http_port = backendId_to_backendHttpPort[tablet.BackendId]
+
+        for (int i = 0; i < final_rowsets.size(); i++) {
+            def rowsetStr = final_rowsets[i]
+            def start_version = rowsetStr.split(" ")[0].replace('[', 
'').replace(']', '').split("-")[0].toInteger()
+            def end_version = rowsetStr.split(" ")[0].replace('[', 
'').replace(']', '').split("-")[1].toInteger()
+            def rowset_id = rowsetStr.split(" ")[4]
+            if (start_version == 0) {
+                continue
+            }
+
+            logger.info("final rowset ${i}, start: ${start_version}, end: 
${end_version}, id: ${rowset_id}")
+            def data = 
Http.GET("http://${be_host}:${be_http_port}/api/file_cache?op=list_cache&value=${rowset_id}_0.dat";,
 true)
+            logger.info("file cache data: ${data}")
+            assertTrue(data.size() > 0)
+        }
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to