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]