test: sqlness upgrade compatibility tests (#5126)

* feat: simple version switch

* chore: remove debug print

* chore: add common folder

* tests: add drop table

* feat: pull versioned binary

* chore: don't use native-tls

* chore: rm outdated docs

* chore: new line

* fix: save old bin dir

* fix: switch version restart all node

* feat: use etcd

* fix: wait for election

* fix: normal sqlness

* refactor: hashmap for bin dir

* test: past 3 major version compat crate table

* refactor: allow using without setup etcd
This commit is contained in:
discord9
2024-12-17 15:00:02 +08:00
committed by Yingwen
parent bcecd8ce52
commit a59fef9ffb
18 changed files with 1420 additions and 33 deletions

70
Cargo.lock generated
View File

@@ -6026,6 +6026,18 @@ version = "0.4.14"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89"
[[package]]
name = "local-ip-address"
version = "0.6.3"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "3669cf5561f8d27e8fc84cc15e58350e70f557d4d65f70e3154e54cd2f8e1782"
dependencies = [
"libc",
"neli",
"thiserror 1.0.64",
"windows-sys 0.59.0",
]
[[package]]
name = "lock_api"
version = "0.4.12"
@@ -6992,6 +7004,31 @@ version = "0.1.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "27b02d87554356db9e9a873add8782d4ea6e3e58ea071a9adb9a2e8ddb884a8b"
[[package]]
name = "neli"
version = "0.6.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "1100229e06604150b3becd61a4965d5c70f3be1759544ea7274166f4be41ef43"
dependencies = [
"byteorder",
"libc",
"log",
"neli-proc-macros",
]
[[package]]
name = "neli-proc-macros"
version = "0.1.3"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "c168194d373b1e134786274020dae7fc5513d565ea2ebb9bc9ff17ffb69106d4"
dependencies = [
"either",
"proc-macro2",
"quote",
"serde",
"syn 1.0.109",
]
[[package]]
name = "new_debug_unreachable"
version = "1.0.6"
@@ -9380,9 +9417,9 @@ dependencies = [
[[package]]
name = "reqwest"
version = "0.12.8"
version = "0.12.9"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "f713147fbe92361e52392c73b8c9e48c04c6625bce969ef54dc901e58e042a7b"
checksum = "a77c62af46e79de0a562e1a9849205ffcb7fc1238876e9bd743357570e04046f"
dependencies = [
"base64 0.22.1",
"bytes",
@@ -11280,14 +11317,21 @@ dependencies = [
"common-recordbatch",
"common-time",
"datatypes",
"flate2",
"hex",
"local-ip-address",
"mysql",
"reqwest",
"serde",
"serde_json",
"sha2",
"sqlness",
"tar",
"tempfile",
"tinytemplate",
"tokio",
"tokio-postgres",
"tokio-stream",
]
[[package]]
@@ -12043,6 +12087,17 @@ version = "1.0.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369"
[[package]]
name = "tar"
version = "0.4.43"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "c65998313f8e17d0d553d28f91a0df93e4dbbbf770279c7bc21ca0f09ea1a1f6"
dependencies = [
"filetime",
"libc",
"xattr",
]
[[package]]
name = "target-lexicon"
version = "0.12.16"
@@ -14168,6 +14223,17 @@ dependencies = [
"zeroize",
]
[[package]]
name = "xattr"
version = "1.3.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "8da84f1a25939b27f6820d92aed108f83ff920fdf11a7b19366c27c4cda81d4f"
dependencies = [
"libc",
"linux-raw-sys",
"rustix",
]
[[package]]
name = "xml-rs"
version = "0.8.22"

View File

@@ -15,6 +15,7 @@
use std::any::Any;
use std::sync::Arc;
use common_telemetry::info;
use etcd_client::{
Client, DeleteOptions, GetOptions, PutOptions, Txn, TxnOp, TxnOpResponse, TxnResponse,
};
@@ -55,6 +56,7 @@ impl EtcdStore {
}
pub fn with_etcd_client(client: Client, max_txn_ops: usize) -> KvBackendRef {
info!("Connected to etcd");
Arc::new(Self {
client,
max_txn_ops,

View File

@@ -1,4 +1,14 @@
flush_stats_factor = 1
{{ if use_etcd }}
## Store server address default to etcd store.
store_addrs = [{store_addrs | unescaped}]
## Store data in memory.
use_memory_store = false
## The datastore for meta server.
backend = "EtcdStore"
{{ endif }}
[wal]
{{ if is_raft_engine }}
provider = "raft_engine"

View File

@@ -16,12 +16,18 @@ common-query.workspace = true
common-recordbatch.workspace = true
common-time.workspace = true
datatypes = { workspace = true }
flate2 = "1.0"
hex = "0.4"
local-ip-address = "0.6"
mysql = { version = "25.0.1", default-features = false, features = ["minimal", "rustls-tls"] }
reqwest = { version = "0.12", default-features = false, features = ["rustls-tls"] }
serde.workspace = true
serde_json.workspace = true
tokio-postgres = { workspace = true }
# sqlness 0.6.0 have a bug causing `cargo sqlness` to fail(see https://github.com/CeresDB/sqlness/issues/68) which is fixed in 0.6.1
sqlness = "0.6.1"
sha2 = "0.10"
sqlness = "0.6.1" # sqlness 0.6.0 have a bug causing `cargo sqlness` to fail(see https://github.com/CeresDB/sqlness/issues/68) which is fixed in 0.6.1
tar = "0.4"
tempfile.workspace = true
tinytemplate = "1.2"
tokio.workspace = true
tokio-postgres = { workspace = true }
tokio-stream.workspace = true

View File

@@ -13,6 +13,7 @@
// limitations under the License.
use std::borrow::Cow;
use std::collections::HashMap;
use std::fmt::Display;
use std::fs::OpenOptions;
use std::io;
@@ -45,6 +46,7 @@ use tokio::sync::Mutex as TokioMutex;
use tokio_postgres::{Client as PgClient, SimpleQueryMessage as PgRow};
use crate::protocol_interceptor::{MYSQL, PROTOCOL_KEY};
use crate::util::{get_workspace_root, maybe_pull_binary, PROGRAM};
use crate::{util, ServerAddr};
const METASRV_ADDR: &str = "127.0.0.1:29302";
@@ -64,6 +66,12 @@ pub enum WalConfig {
},
}
#[derive(Clone)]
pub struct StoreConfig {
pub store_addrs: Vec<String>,
pub setup_etcd: bool,
}
#[derive(Clone)]
pub struct Env {
sqlness_home: PathBuf,
@@ -74,6 +82,12 @@ pub struct Env {
/// When running in CI, this is expected to be set.
/// If not set, this runner will build the GreptimeDB binary itself when needed, and set this field by then.
bins_dir: Arc<Mutex<Option<PathBuf>>>,
/// The path to the directory that contains the old pre-built GreptimeDB binaries.
versioned_bins_dirs: Arc<Mutex<HashMap<String, PathBuf>>>,
/// Pull different versions of GreptimeDB on need.
pull_version_on_need: bool,
/// Store address for metasrv metadata
store_config: StoreConfig,
}
#[async_trait]
@@ -100,13 +114,21 @@ impl Env {
data_home: PathBuf,
server_addrs: ServerAddr,
wal: WalConfig,
pull_version_on_need: bool,
bins_dir: Option<PathBuf>,
store_config: StoreConfig,
) -> Self {
Self {
sqlness_home: data_home,
server_addrs,
wal,
bins_dir: Arc::new(Mutex::new(bins_dir)),
pull_version_on_need,
bins_dir: Arc::new(Mutex::new(bins_dir.clone())),
versioned_bins_dirs: Arc::new(Mutex::new(HashMap::from_iter([(
"latest".to_string(),
bins_dir.clone().unwrap_or(util::get_binary_dir("debug")),
)]))),
store_config,
}
}
@@ -117,7 +139,7 @@ impl Env {
self.build_db();
self.setup_wal();
let db_ctx = GreptimeDBContext::new(self.wal.clone());
let db_ctx = GreptimeDBContext::new(self.wal.clone(), self.store_config.clone());
let server_process = self.start_server("standalone", &db_ctx, true).await;
@@ -136,8 +158,9 @@ impl Env {
} else {
self.build_db();
self.setup_wal();
self.setup_etcd();
let db_ctx = GreptimeDBContext::new(self.wal.clone());
let db_ctx = GreptimeDBContext::new(self.wal.clone(), self.store_config.clone());
// start a distributed GreptimeDB
let meta_server = self.start_server("metasrv", &db_ctx, true).await;
@@ -152,12 +175,12 @@ impl Env {
let mut greptimedb = self.connect_db(&Default::default()).await;
greptimedb.metasrv_process = Some(meta_server);
greptimedb.metasrv_process = Some(meta_server).into();
greptimedb.server_processes = Some(Arc::new(Mutex::new(vec![
datanode_1, datanode_2, datanode_3,
])));
greptimedb.frontend_process = Some(frontend);
greptimedb.flownode_process = Some(flownode);
greptimedb.frontend_process = Some(frontend).into();
greptimedb.flownode_process = Some(flownode).into();
greptimedb.is_standalone = false;
greptimedb.ctx = db_ctx;
@@ -237,13 +260,14 @@ impl Env {
pg_client: TokioMutex::new(pg_client),
mysql_client: TokioMutex::new(mysql_client),
server_processes: None,
metasrv_process: None,
frontend_process: None,
flownode_process: None,
metasrv_process: None.into(),
frontend_process: None.into(),
flownode_process: None.into(),
ctx: GreptimeDBContext {
time: 0,
datanode_id: Default::default(),
wal: self.wal.clone(),
store_config: self.store_config.clone(),
},
is_standalone: false,
env: self.clone(),
@@ -341,7 +365,7 @@ impl Env {
)
}
"metasrv" => {
let args = vec![
let mut args = vec![
DEFAULT_LOG_LEVEL.to_string(),
subcommand.to_string(),
"start".to_string(),
@@ -349,8 +373,6 @@ impl Env {
"127.0.0.1:29302".to_string(),
"--server-addr".to_string(),
"127.0.0.1:29302".to_string(),
"--backend".to_string(),
"memory-store".to_string(),
"--enable-region-failover".to_string(),
"false".to_string(),
"--http-addr=127.0.0.1:29502".to_string(),
@@ -361,6 +383,9 @@ impl Env {
"-c".to_string(),
self.generate_config_file(subcommand, db_ctx),
];
if db_ctx.store_config().store_addrs.is_empty() {
args.extend(vec!["--backend".to_string(), "memory-store".to_string()])
}
(args, vec![METASRV_ADDR.to_string()])
}
_ => panic!("Unexpected subcommand: {subcommand}"),
@@ -375,23 +400,20 @@ impl Env {
}
}
#[cfg(not(windows))]
let program = "./greptime";
#[cfg(windows)]
let program = "greptime.exe";
let program = PROGRAM;
let bins_dir = self.bins_dir.lock().unwrap().clone().expect(
"GreptimeDB binary is not available. Please pass in the path to the directory that contains the pre-built GreptimeDB binary. Or you may call `self.build_db()` beforehand.",
);
let mut process = Command::new(program)
.current_dir(bins_dir)
.current_dir(bins_dir.clone())
.env("TZ", "UTC")
.args(args)
.stdout(stdout_file)
.spawn()
.unwrap_or_else(|error| {
panic!("Failed to start the DB with subcommand {subcommand},Error: {error}")
panic!("Failed to start the DB with subcommand {subcommand},Error: {error}, path: {:?}", bins_dir.join(program));
});
for check_ip_addr in &check_ip_addrs {
@@ -452,7 +474,7 @@ impl Env {
}
/// stop and restart the server process
async fn restart_server(&self, db: &GreptimeDB) {
async fn restart_server(&self, db: &GreptimeDB, is_full_restart: bool) {
{
if let Some(server_process) = db.server_processes.clone() {
let mut server_processes = server_process.lock().unwrap();
@@ -460,6 +482,23 @@ impl Env {
Env::stop_server(server_process);
}
}
if is_full_restart {
if let Some(mut metasrv_process) =
db.metasrv_process.lock().expect("poisoned lock").take()
{
Env::stop_server(&mut metasrv_process);
}
if let Some(mut frontend_process) =
db.frontend_process.lock().expect("poisoned lock").take()
{
Env::stop_server(&mut frontend_process);
}
if let Some(mut flownode_process) =
db.flownode_process.lock().expect("poisoned lock").take()
{
Env::stop_server(&mut flownode_process);
}
}
}
// check if the server is distributed or standalone
@@ -468,12 +507,37 @@ impl Env {
vec![new_server_process]
} else {
db.ctx.reset_datanode_id();
if is_full_restart {
let metasrv = self.start_server("metasrv", &db.ctx, false).await;
db.metasrv_process
.lock()
.expect("lock poisoned")
.replace(metasrv);
// wait for metasrv to start
// since it seems older version of db might take longer to complete election
tokio::time::sleep(Duration::from_secs(5)).await;
}
let mut processes = vec![];
for _ in 0..3 {
let new_server_process = self.start_server("datanode", &db.ctx, false).await;
processes.push(new_server_process);
}
if is_full_restart {
let frontend = self.start_server("frontend", &db.ctx, false).await;
db.frontend_process
.lock()
.expect("lock poisoned")
.replace(frontend);
let flownode = self.start_server("flownode", &db.ctx, false).await;
db.flownode_process
.lock()
.expect("lock poisoned")
.replace(flownode);
}
processes
};
@@ -493,6 +557,19 @@ impl Env {
}
}
/// Setup etcd if needed.
fn setup_etcd(&self) {
if self.store_config.setup_etcd {
let client_ports = self
.store_config
.store_addrs
.iter()
.map(|s| s.split(':').nth(1).unwrap().parse::<u16>().unwrap())
.collect::<Vec<_>>();
util::setup_etcd(client_ports, None, None);
}
}
/// Generate config file to `/tmp/{subcommand}-{current_time}.toml`
fn generate_config_file(&self, subcommand: &str, db_ctx: &GreptimeDBContext) -> String {
let mut tt = TinyTemplate::new();
@@ -509,6 +586,8 @@ impl Env {
procedure_dir: String,
is_raft_engine: bool,
kafka_wal_broker_endpoints: String,
use_etcd: bool,
store_addrs: String,
}
let data_home = self.sqlness_home.join(format!("greptimedb-{subcommand}"));
@@ -522,6 +601,15 @@ impl Env {
procedure_dir,
is_raft_engine: db_ctx.is_raft_engine(),
kafka_wal_broker_endpoints: db_ctx.kafka_wal_broker_endpoints(),
use_etcd: !self.store_config.store_addrs.is_empty(),
store_addrs: self
.store_config
.store_addrs
.clone()
.iter()
.map(|p| format!("\"{p}\""))
.collect::<Vec<_>>()
.join(","),
};
let rendered = tt.render(subcommand, &ctx).unwrap();
@@ -580,9 +668,9 @@ impl Env {
pub struct GreptimeDB {
server_processes: Option<Arc<Mutex<Vec<Child>>>>,
metasrv_process: Option<Child>,
frontend_process: Option<Child>,
flownode_process: Option<Child>,
metasrv_process: Mutex<Option<Child>>,
frontend_process: Mutex<Option<Child>>,
flownode_process: Mutex<Option<Child>>,
grpc_client: TokioMutex<DB>,
pg_client: TokioMutex<PgClient>,
mysql_client: TokioMutex<MySqlClient>,
@@ -693,8 +781,35 @@ impl GreptimeDB {
impl Database for GreptimeDB {
async fn query(&self, ctx: QueryContext, query: String) -> Box<dyn Display> {
if ctx.context.contains_key("restart") && self.env.server_addrs.server_addr.is_none() {
self.env.restart_server(self).await;
self.env.restart_server(self, false).await;
} else if let Some(version) = ctx.context.get("version") {
let version_bin_dir = self
.env
.versioned_bins_dirs
.lock()
.expect("lock poison")
.get(version.as_str())
.cloned();
match version_bin_dir {
Some(path) if path.clone().join(PROGRAM).is_file() => {
// use version in versioned_bins_dirs
*self.env.bins_dir.lock().unwrap() = Some(path.clone());
}
_ => {
// use version in dir files
maybe_pull_binary(version, self.env.pull_version_on_need).await;
let root = get_workspace_root();
let new_path = PathBuf::from_iter([&root, version]);
*self.env.bins_dir.lock().unwrap() = Some(new_path);
}
}
self.env.restart_server(self, true).await;
// sleep for a while to wait for the server to fully boot up
tokio::time::sleep(Duration::from_secs(5)).await;
}
if let Some(protocol) = ctx.context.get(PROTOCOL_KEY) {
// protocol is bound to be either "mysql" or "postgres"
if protocol == MYSQL {
@@ -720,15 +835,30 @@ impl GreptimeDB {
);
}
}
if let Some(mut metasrv) = self.metasrv_process.take() {
if let Some(mut metasrv) = self
.metasrv_process
.lock()
.expect("someone else panic when holding lock")
.take()
{
Env::stop_server(&mut metasrv);
println!("Metasrv (pid = {}) is stopped", metasrv.id());
}
if let Some(mut frontend) = self.frontend_process.take() {
if let Some(mut frontend) = self
.frontend_process
.lock()
.expect("someone else panic when holding lock")
.take()
{
Env::stop_server(&mut frontend);
println!("Frontend (pid = {}) is stopped", frontend.id());
}
if let Some(mut flownode) = self.flownode_process.take() {
if let Some(mut flownode) = self
.flownode_process
.lock()
.expect("someone else panic when holding lock")
.take()
{
Env::stop_server(&mut flownode);
println!("Flownode (pid = {}) is stopped", flownode.id());
}
@@ -752,14 +882,16 @@ struct GreptimeDBContext {
time: i64,
datanode_id: AtomicU32,
wal: WalConfig,
store_config: StoreConfig,
}
impl GreptimeDBContext {
pub fn new(wal: WalConfig) -> Self {
pub fn new(wal: WalConfig, store_config: StoreConfig) -> Self {
Self {
time: common_time::util::current_time_millis(),
datanode_id: AtomicU32::new(0),
wal,
store_config,
}
}
@@ -787,6 +919,10 @@ impl GreptimeDBContext {
fn reset_datanode_id(&self) {
self.datanode_id.store(0, Ordering::Relaxed);
}
fn store_config(&self) -> StoreConfig {
self.store_config.clone()
}
}
struct ResultDisplayer {

View File

@@ -22,6 +22,8 @@ use env::{Env, WalConfig};
use sqlness::interceptor::Registry;
use sqlness::{ConfigBuilder, Runner};
use crate::env::StoreConfig;
mod env;
mod protocol_interceptor;
mod util;
@@ -92,6 +94,18 @@ struct Args {
/// This may affect future test runs.
#[clap(long)]
preserve_state: bool,
/// Pull Different versions of GreptimeDB on need.
#[clap(long, default_value = "true")]
pull_version_on_need: bool,
/// The store addresses for metadata, if empty, will use memory store.
#[clap(long)]
store_addrs: Vec<String>,
/// Whether to setup etcd, by default it is false.
#[clap(long, default_value = "false")]
setup_etcd: bool,
}
#[tokio::main]
@@ -110,6 +124,11 @@ async fn main() {
Arc::new(protocol_interceptor::ProtocolInterceptorFactory),
);
if let Some(d) = &args.case_dir {
if !d.is_dir() {
panic!("{} is not a directory", d.display());
}
}
let config = ConfigBuilder::default()
.case_dir(util::get_case_dir(args.case_dir))
.fail_fast(args.fail_fast)
@@ -132,19 +151,30 @@ async fn main() {
},
};
let store = StoreConfig {
store_addrs: args.store_addrs.clone(),
setup_etcd: args.setup_etcd,
};
let runner = Runner::new(
config,
Env::new(
sqlness_home.clone(),
args.server_addr.clone(),
wal,
args.pull_version_on_need,
args.bins_dir,
store,
),
);
runner.run().await.unwrap();
// clean up and exit
if !args.preserve_state {
if args.setup_etcd {
println!("Stopping etcd");
util::stop_rm_etcd();
}
println!("Removing state in {:?}", sqlness_home);
tokio::fs::remove_dir_all(sqlness_home).await.unwrap();
}

View File

@@ -12,18 +12,299 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::io::Read;
use std::net::SocketAddr;
use std::path::PathBuf;
use std::path::{Path, PathBuf};
use std::process::Command;
use std::time::Duration;
use sha2::{Digest, Sha256};
use tokio::io::AsyncWriteExt;
use tokio::net::TcpSocket;
use tokio::time;
use tokio_stream::StreamExt;
/// Check port every 0.1 second.
const PORT_CHECK_INTERVAL: Duration = Duration::from_millis(100);
#[cfg(not(windows))]
pub const PROGRAM: &str = "./greptime";
#[cfg(windows)]
pub const PROGRAM: &str = "greptime.exe";
fn http_proxy() -> Option<String> {
for proxy in ["http_proxy", "HTTP_PROXY", "all_proxy", "ALL_PROXY"] {
if let Ok(proxy_addr) = std::env::var(proxy) {
println!("Getting Proxy from env var: {}={}", proxy, proxy_addr);
return Some(proxy_addr);
}
}
None
}
fn https_proxy() -> Option<String> {
for proxy in ["https_proxy", "HTTPS_PROXY", "all_proxy", "ALL_PROXY"] {
if let Ok(proxy_addr) = std::env::var(proxy) {
println!("Getting Proxy from env var: {}={}", proxy, proxy_addr);
return Some(proxy_addr);
}
}
None
}
async fn download_files(url: &str, path: &str) {
let proxy = if url.starts_with("http://") {
http_proxy().map(|proxy| reqwest::Proxy::http(proxy).unwrap())
} else if url.starts_with("https://") {
https_proxy().map(|proxy| reqwest::Proxy::https(proxy).unwrap())
} else {
None
};
let client = proxy
.map(|proxy| {
reqwest::Client::builder()
.proxy(proxy)
.build()
.expect("Failed to build client")
})
.unwrap_or(reqwest::Client::new());
let mut file = tokio::fs::File::create(path)
.await
.unwrap_or_else(|_| panic!("Failed to create file in {path}"));
println!("Downloading {}...", url);
let resp = client
.get(url)
.send()
.await
.expect("Failed to send download request");
let len = resp.content_length();
let mut stream = resp.bytes_stream();
let mut size_downloaded = 0;
while let Some(chunk_result) = stream.next().await {
let chunk = chunk_result.unwrap();
size_downloaded += chunk.len();
if let Some(len) = len {
print!("\rDownloading {}/{} bytes", size_downloaded, len);
} else {
print!("\rDownloaded {} bytes", size_downloaded);
}
file.write_all(&chunk).await.unwrap();
}
file.flush().await.unwrap();
println!("\nDownloaded {}", url);
}
fn decompress(archive: &str, dest: &str) {
let tar = std::fs::File::open(archive).unwrap();
let dec = flate2::read::GzDecoder::new(tar);
let mut a = tar::Archive::new(dec);
a.unpack(dest).unwrap();
}
/// Use curl to download the binary from the release page.
///
/// # Arguments
///
/// * `version` - The version of the binary to download. i.e. "v0.9.5"
pub async fn pull_binary(version: &str) {
let os = std::env::consts::OS;
let arch = match std::env::consts::ARCH {
"x86_64" => "amd64",
"aarch64" => "arm64",
_ => panic!("Unsupported arch: {}", std::env::consts::ARCH),
};
let triple = format!("greptime-{}-{}-{}", os, arch, version);
let filename = format!("{triple}.tar.gz");
let url = format!(
"https://github.com/GreptimeTeam/greptimedb/releases/download/{version}/{filename}"
);
println!("Downloading {version} binary from {}", url);
// mkdir {version}
let _ = std::fs::create_dir(version);
let archive = Path::new(version).join(filename);
let folder_path = Path::new(version);
// download the binary to the version directory
download_files(&url, &archive.to_string_lossy()).await;
let checksum_file = format!("{triple}.sha256sum");
let checksum_url = format!(
"https://github.com/GreptimeTeam/greptimedb/releases/download/{version}/{checksum_file}"
);
download_files(
&checksum_url,
&PathBuf::from_iter([version, &checksum_file]).to_string_lossy(),
)
.await;
// verify the checksum
let mut file = std::fs::File::open(&archive).unwrap();
let mut sha256 = Sha256::new();
std::io::copy(&mut file, &mut sha256).unwrap();
let checksum: Vec<u8> = sha256.finalize().to_vec();
let mut expected_checksum =
std::fs::File::open(PathBuf::from_iter([version, &checksum_file])).unwrap();
let mut buf = String::new();
expected_checksum.read_to_string(&mut buf).unwrap();
let expected_checksum = hex::decode(buf.lines().next().unwrap()).unwrap();
assert_eq!(
checksum, expected_checksum,
"Checksum mismatched, downloaded file is corrupted"
);
decompress(&archive.to_string_lossy(), &folder_path.to_string_lossy());
println!("Downloaded and extracted {version} binary to {folder_path:?}");
// move the binary to the version directory
std::fs::rename(
PathBuf::from_iter([version, &triple, "greptime"]),
PathBuf::from_iter([version, "greptime"]),
)
.unwrap();
// remove the archive and inner folder
std::fs::remove_file(&archive).unwrap();
std::fs::remove_dir(PathBuf::from_iter([version, &triple])).unwrap();
}
/// Pull the binary if it does not exist and `pull_version_on_need` is true.
pub async fn maybe_pull_binary(version: &str, pull_version_on_need: bool) {
let exist = Path::new(version).join(PROGRAM).is_file();
match (exist, pull_version_on_need){
(true, _) => println!("Binary {version} exists"),
(false, false) => panic!("Binary {version} does not exist, please run with --pull-version-on-need or manually download it"),
(false, true) => { pull_binary(version).await; },
}
}
/// Set up a standalone etcd in docker.
pub fn setup_etcd(client_ports: Vec<u16>, peer_port: Option<u16>, etcd_version: Option<&str>) {
if std::process::Command::new("docker")
.args(["-v"])
.status()
.is_err()
{
panic!("Docker is not installed");
}
let peer_port = peer_port.unwrap_or(2380);
let exposed_port: Vec<_> = client_ports.iter().chain(Some(&peer_port)).collect();
let exposed_port_str = exposed_port
.iter()
.flat_map(|p| ["-p".to_string(), format!("{p}:{p}")])
.collect::<Vec<_>>();
let etcd_version = etcd_version.unwrap_or("v3.5.17");
let etcd_image = format!("quay.io/coreos/etcd:{etcd_version}");
let peer_url = format!("http://0.0.0.0:{peer_port}");
let my_local_ip = local_ip_address::local_ip().unwrap();
let my_local_ip_str = my_local_ip.to_string();
let mut arg_list = vec![];
arg_list.extend([
"run",
"-d",
"-v",
"/usr/share/ca-certificates/:/etc/ssl/certs",
]);
arg_list.extend(exposed_port_str.iter().map(std::ops::Deref::deref));
arg_list.extend([
"--name",
"etcd",
&etcd_image,
"etcd",
"-name",
"etcd0",
"-advertise-client-urls",
]);
let adv_client_urls = client_ports
.iter()
.map(|p| format!("http://{my_local_ip_str}:{p}"))
.collect::<Vec<_>>()
.join(",");
arg_list.push(&adv_client_urls);
arg_list.extend(["-listen-client-urls"]);
let client_ports_fmt = client_ports
.iter()
.map(|p| format!("http://0.0.0.0:{p}"))
.collect::<Vec<_>>()
.join(",");
arg_list.push(&client_ports_fmt);
arg_list.push("-initial-advertise-peer-urls");
let advertise_peer_url = format!("http://{my_local_ip_str}:{peer_port}");
arg_list.push(&advertise_peer_url);
arg_list.extend(["-listen-peer-urls", &peer_url]);
arg_list.extend(["-initial-cluster-token", "etcd-cluster-1"]);
arg_list.push("-initial-cluster");
let init_cluster_url = format!("etcd0=http://{my_local_ip_str}:{peer_port}");
arg_list.push(&init_cluster_url);
arg_list.extend(["-initial-cluster-state", "new"]);
let mut cmd = std::process::Command::new("docker");
cmd.args(arg_list);
println!("Starting etcd with command: {:?}", cmd);
let status = cmd.status();
if status.is_err() {
panic!("Failed to start etcd: {:?}", status);
} else if let Ok(status) = status {
if status.success() {
println!(
"Started etcd with client ports {:?} and peer port {}, statues:{status:?}",
client_ports, peer_port
);
} else {
panic!("Failed to start etcd: {:?}", status);
}
}
}
/// Stop and remove the etcd container
pub fn stop_rm_etcd() {
let status = std::process::Command::new("docker")
.args(["container", "stop", "etcd"])
.status();
if status.is_err() {
panic!("Failed to stop etcd: {:?}", status);
} else {
println!("Stopped etcd");
}
// rm the container
let status = std::process::Command::new("docker")
.args(["container", "rm", "etcd"])
.status();
if status.is_err() {
panic!("Failed to remove etcd container: {:?}", status);
} else {
println!("Removed etcd container");
}
}
/// Get the dir of test cases. This function only works when the runner is run
/// under the project's dir because it depends on some envs set by cargo.
pub fn get_case_dir(case_dir: Option<PathBuf>) -> String {

View File

@@ -0,0 +1 @@
../standalone/common

View File

@@ -0,0 +1,137 @@
-- SQLNESS ARG version=v0.10.2
CREATE TABLE mito_system_metrics (
host STRING,
idc STRING,
cpu_util DOUBLE,
memory_util DOUBLE,
disk_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host, idc),
TIME INDEX(ts)
)ENGINE=mito;
Affected Rows: 0
INSERT INTO mito_system_metrics
VALUES
("host1", "idc_a", 11.8, 10.3, 10.3, 1667446797450),
("host2", "idc_a", 80.0, 70.3, 90.0, 1667446797450),
("host1", "idc_b", 50.0, 66.7, 40.6, 1667446797450);
Affected Rows: 3
CREATE TABLE phy (ts timestamp time index, cpu_util double) engine=metric with ("physical_metric_table" = "");
Affected Rows: 0
CREATE TABLE system_metrics (
host STRING,
cpu_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host),
TIME INDEX(ts)
)ENGINE=metric with ("on_physical_table" = "phy");
Affected Rows: 0
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host1', 11.8, 1667446797450),
('host2', 80.0, 1667446797450),
('host1', 50.0, 1667446797450);
Affected Rows: 3
-- SQLNESS ARG version=latest
SHOW CREATE TABLE mito_system_metrics;
+---------------------+-----------------------------------------------------------+
| Table | Create Table |
+---------------------+-----------------------------------------------------------+
| mito_system_metrics | CREATE TABLE IF NOT EXISTS "mito_system_metrics" ( |
| | "host" STRING NULL, |
| | "idc" STRING NULL, |
| | "cpu_util" DOUBLE NULL, |
| | "memory_util" DOUBLE NULL, |
| | "disk_util" DOUBLE NULL, |
| | "ts" TIMESTAMP(3) NOT NULL DEFAULT current_timestamp(), |
| | TIME INDEX ("ts"), |
| | PRIMARY KEY ("host", "idc") |
| | ) |
| | |
| | ENGINE=mito |
| | |
+---------------------+-----------------------------------------------------------+
SHOW CREATE TABLE system_metrics;
+----------------+-----------------------------------------------------------+
| Table | Create Table |
+----------------+-----------------------------------------------------------+
| system_metrics | CREATE TABLE IF NOT EXISTS "system_metrics" ( |
| | "cpu_util" DOUBLE NULL, |
| | "host" STRING NULL, |
| | "ts" TIMESTAMP(3) NOT NULL DEFAULT current_timestamp(), |
| | TIME INDEX ("ts"), |
| | PRIMARY KEY ("host") |
| | ) |
| | |
| | ENGINE=metric |
| | WITH( |
| | on_physical_table = 'phy' |
| | ) |
+----------------+-----------------------------------------------------------+
INSERT INTO mito_system_metrics
VALUES
("host3", "idc_a", 90.0, 70.3, 90.0, 1667446797450),
("host4", "idc_a", 70.0, 70.3, 90.0, 1667446797450),
("host5", "idc_a", 60.0, 70.3, 90.0, 1667446797450);
Affected Rows: 3
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host3', 90.0, 1667446797450),
('host4', 70.0, 1667446797450),
('host5', 60.0, 1667446797450);
Affected Rows: 3
SELECT * FROM mito_system_metrics;
+-------+-------+----------+-------------+-----------+-------------------------+
| host | idc | cpu_util | memory_util | disk_util | ts |
+-------+-------+----------+-------------+-----------+-------------------------+
| host1 | idc_a | 11.8 | 10.3 | 10.3 | 2022-11-03T03:39:57.450 |
| host1 | idc_b | 50.0 | 66.7 | 40.6 | 2022-11-03T03:39:57.450 |
| host2 | idc_a | 80.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
| host3 | idc_a | 90.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
| host4 | idc_a | 70.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
| host5 | idc_a | 60.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
+-------+-------+----------+-------------+-----------+-------------------------+
SELECT * FROM system_metrics;
+----------+-------+-------------------------+
| cpu_util | host | ts |
+----------+-------+-------------------------+
| 80.0 | host2 | 2022-11-03T03:39:57.450 |
| 70.0 | host4 | 2022-11-03T03:39:57.450 |
| 60.0 | host5 | 2022-11-03T03:39:57.450 |
| 90.0 | host3 | 2022-11-03T03:39:57.450 |
| 50.0 | host1 | 2022-11-03T03:39:57.450 |
+----------+-------+-------------------------+
DROP TABLE mito_system_metrics;
Affected Rows: 0
DROP TABLE system_metrics;
Affected Rows: 0
DROP TABLE phy;
Affected Rows: 0

View File

@@ -0,0 +1,60 @@
-- SQLNESS ARG version=v0.10.2
CREATE TABLE mito_system_metrics (
host STRING,
idc STRING,
cpu_util DOUBLE,
memory_util DOUBLE,
disk_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host, idc),
TIME INDEX(ts)
)ENGINE=mito;
INSERT INTO mito_system_metrics
VALUES
("host1", "idc_a", 11.8, 10.3, 10.3, 1667446797450),
("host2", "idc_a", 80.0, 70.3, 90.0, 1667446797450),
("host1", "idc_b", 50.0, 66.7, 40.6, 1667446797450);
CREATE TABLE phy (ts timestamp time index, cpu_util double) engine=metric with ("physical_metric_table" = "");
CREATE TABLE system_metrics (
host STRING,
cpu_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host),
TIME INDEX(ts)
)ENGINE=metric with ("on_physical_table" = "phy");
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host1', 11.8, 1667446797450),
('host2', 80.0, 1667446797450),
('host1', 50.0, 1667446797450);
-- SQLNESS ARG version=latest
SHOW CREATE TABLE mito_system_metrics;
SHOW CREATE TABLE system_metrics;
INSERT INTO mito_system_metrics
VALUES
("host3", "idc_a", 90.0, 70.3, 90.0, 1667446797450),
("host4", "idc_a", 70.0, 70.3, 90.0, 1667446797450),
("host5", "idc_a", 60.0, 70.3, 90.0, 1667446797450);
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host3', 90.0, 1667446797450),
('host4', 70.0, 1667446797450),
('host5', 60.0, 1667446797450);
SELECT * FROM mito_system_metrics;
SELECT * FROM system_metrics;
DROP TABLE mito_system_metrics;
DROP TABLE system_metrics;
DROP TABLE phy;

View File

@@ -0,0 +1,137 @@
-- SQLNESS ARG version=v0.11.0
CREATE TABLE mito_system_metrics (
host STRING,
idc STRING,
cpu_util DOUBLE,
memory_util DOUBLE,
disk_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host, idc),
TIME INDEX(ts)
)ENGINE=mito;
Affected Rows: 0
INSERT INTO mito_system_metrics
VALUES
("host1", "idc_a", 11.8, 10.3, 10.3, 1667446797450),
("host2", "idc_a", 80.0, 70.3, 90.0, 1667446797450),
("host1", "idc_b", 50.0, 66.7, 40.6, 1667446797450);
Affected Rows: 3
CREATE TABLE phy (ts timestamp time index, cpu_util double) engine=metric with ("physical_metric_table" = "");
Affected Rows: 0
CREATE TABLE system_metrics (
host STRING,
cpu_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host),
TIME INDEX(ts)
)ENGINE=metric with ("on_physical_table" = "phy");
Affected Rows: 0
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host1', 11.8, 1667446797450),
('host2', 80.0, 1667446797450),
('host1', 50.0, 1667446797450);
Affected Rows: 3
-- SQLNESS ARG version=latest
SHOW CREATE TABLE mito_system_metrics;
+---------------------+-----------------------------------------------------------+
| Table | Create Table |
+---------------------+-----------------------------------------------------------+
| mito_system_metrics | CREATE TABLE IF NOT EXISTS "mito_system_metrics" ( |
| | "host" STRING NULL, |
| | "idc" STRING NULL, |
| | "cpu_util" DOUBLE NULL, |
| | "memory_util" DOUBLE NULL, |
| | "disk_util" DOUBLE NULL, |
| | "ts" TIMESTAMP(3) NOT NULL DEFAULT current_timestamp(), |
| | TIME INDEX ("ts"), |
| | PRIMARY KEY ("host", "idc") |
| | ) |
| | |
| | ENGINE=mito |
| | |
+---------------------+-----------------------------------------------------------+
SHOW CREATE TABLE system_metrics;
+----------------+-----------------------------------------------------------+
| Table | Create Table |
+----------------+-----------------------------------------------------------+
| system_metrics | CREATE TABLE IF NOT EXISTS "system_metrics" ( |
| | "cpu_util" DOUBLE NULL, |
| | "host" STRING NULL, |
| | "ts" TIMESTAMP(3) NOT NULL DEFAULT current_timestamp(), |
| | TIME INDEX ("ts"), |
| | PRIMARY KEY ("host") |
| | ) |
| | |
| | ENGINE=metric |
| | WITH( |
| | on_physical_table = 'phy' |
| | ) |
+----------------+-----------------------------------------------------------+
INSERT INTO mito_system_metrics
VALUES
("host3", "idc_a", 90.0, 70.3, 90.0, 1667446797450),
("host4", "idc_a", 70.0, 70.3, 90.0, 1667446797450),
("host5", "idc_a", 60.0, 70.3, 90.0, 1667446797450);
Affected Rows: 3
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host3', 90.0, 1667446797450),
('host4', 70.0, 1667446797450),
('host5', 60.0, 1667446797450);
Affected Rows: 3
SELECT * FROM mito_system_metrics;
+-------+-------+----------+-------------+-----------+-------------------------+
| host | idc | cpu_util | memory_util | disk_util | ts |
+-------+-------+----------+-------------+-----------+-------------------------+
| host1 | idc_a | 11.8 | 10.3 | 10.3 | 2022-11-03T03:39:57.450 |
| host1 | idc_b | 50.0 | 66.7 | 40.6 | 2022-11-03T03:39:57.450 |
| host2 | idc_a | 80.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
| host3 | idc_a | 90.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
| host4 | idc_a | 70.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
| host5 | idc_a | 60.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
+-------+-------+----------+-------------+-----------+-------------------------+
SELECT * FROM system_metrics;
+----------+-------+-------------------------+
| cpu_util | host | ts |
+----------+-------+-------------------------+
| 80.0 | host2 | 2022-11-03T03:39:57.450 |
| 70.0 | host4 | 2022-11-03T03:39:57.450 |
| 60.0 | host5 | 2022-11-03T03:39:57.450 |
| 90.0 | host3 | 2022-11-03T03:39:57.450 |
| 50.0 | host1 | 2022-11-03T03:39:57.450 |
+----------+-------+-------------------------+
DROP TABLE mito_system_metrics;
Affected Rows: 0
DROP TABLE system_metrics;
Affected Rows: 0
DROP TABLE phy;
Affected Rows: 0

View File

@@ -0,0 +1,60 @@
-- SQLNESS ARG version=v0.11.0
CREATE TABLE mito_system_metrics (
host STRING,
idc STRING,
cpu_util DOUBLE,
memory_util DOUBLE,
disk_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host, idc),
TIME INDEX(ts)
)ENGINE=mito;
INSERT INTO mito_system_metrics
VALUES
("host1", "idc_a", 11.8, 10.3, 10.3, 1667446797450),
("host2", "idc_a", 80.0, 70.3, 90.0, 1667446797450),
("host1", "idc_b", 50.0, 66.7, 40.6, 1667446797450);
CREATE TABLE phy (ts timestamp time index, cpu_util double) engine=metric with ("physical_metric_table" = "");
CREATE TABLE system_metrics (
host STRING,
cpu_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host),
TIME INDEX(ts)
)ENGINE=metric with ("on_physical_table" = "phy");
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host1', 11.8, 1667446797450),
('host2', 80.0, 1667446797450),
('host1', 50.0, 1667446797450);
-- SQLNESS ARG version=latest
SHOW CREATE TABLE mito_system_metrics;
SHOW CREATE TABLE system_metrics;
INSERT INTO mito_system_metrics
VALUES
("host3", "idc_a", 90.0, 70.3, 90.0, 1667446797450),
("host4", "idc_a", 70.0, 70.3, 90.0, 1667446797450),
("host5", "idc_a", 60.0, 70.3, 90.0, 1667446797450);
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host3', 90.0, 1667446797450),
('host4', 70.0, 1667446797450),
('host5', 60.0, 1667446797450);
SELECT * FROM mito_system_metrics;
SELECT * FROM system_metrics;
DROP TABLE mito_system_metrics;
DROP TABLE system_metrics;
DROP TABLE phy;

View File

@@ -0,0 +1,137 @@
-- SQLNESS ARG version=v0.9.5
CREATE TABLE mito_system_metrics (
host STRING,
idc STRING,
cpu_util DOUBLE,
memory_util DOUBLE,
disk_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host, idc),
TIME INDEX(ts)
)ENGINE=mito;
Affected Rows: 0
INSERT INTO mito_system_metrics
VALUES
("host1", "idc_a", 11.8, 10.3, 10.3, 1667446797450),
("host2", "idc_a", 80.0, 70.3, 90.0, 1667446797450),
("host1", "idc_b", 50.0, 66.7, 40.6, 1667446797450);
Affected Rows: 3
CREATE TABLE phy (ts timestamp time index, cpu_util double) engine=metric with ("physical_metric_table" = "");
Affected Rows: 0
CREATE TABLE system_metrics (
host STRING,
cpu_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host),
TIME INDEX(ts)
)ENGINE=metric with ("on_physical_table" = "phy");
Affected Rows: 0
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host1', 11.8, 1667446797450),
('host2', 80.0, 1667446797450),
('host1', 50.0, 1667446797450);
Affected Rows: 3
-- SQLNESS ARG version=latest
SHOW CREATE TABLE mito_system_metrics;
+---------------------+-----------------------------------------------------------+
| Table | Create Table |
+---------------------+-----------------------------------------------------------+
| mito_system_metrics | CREATE TABLE IF NOT EXISTS "mito_system_metrics" ( |
| | "host" STRING NULL, |
| | "idc" STRING NULL, |
| | "cpu_util" DOUBLE NULL, |
| | "memory_util" DOUBLE NULL, |
| | "disk_util" DOUBLE NULL, |
| | "ts" TIMESTAMP(3) NOT NULL DEFAULT current_timestamp(), |
| | TIME INDEX ("ts"), |
| | PRIMARY KEY ("host", "idc") |
| | ) |
| | |
| | ENGINE=mito |
| | |
+---------------------+-----------------------------------------------------------+
SHOW CREATE TABLE system_metrics;
+----------------+-----------------------------------------------------------+
| Table | Create Table |
+----------------+-----------------------------------------------------------+
| system_metrics | CREATE TABLE IF NOT EXISTS "system_metrics" ( |
| | "cpu_util" DOUBLE NULL, |
| | "host" STRING NULL, |
| | "ts" TIMESTAMP(3) NOT NULL DEFAULT current_timestamp(), |
| | TIME INDEX ("ts"), |
| | PRIMARY KEY ("host") |
| | ) |
| | |
| | ENGINE=metric |
| | WITH( |
| | on_physical_table = 'phy' |
| | ) |
+----------------+-----------------------------------------------------------+
INSERT INTO mito_system_metrics
VALUES
("host3", "idc_a", 90.0, 70.3, 90.0, 1667446797450),
("host4", "idc_a", 70.0, 70.3, 90.0, 1667446797450),
("host5", "idc_a", 60.0, 70.3, 90.0, 1667446797450);
Affected Rows: 3
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host3', 90.0, 1667446797450),
('host4', 70.0, 1667446797450),
('host5', 60.0, 1667446797450);
Affected Rows: 3
SELECT * FROM mito_system_metrics;
+-------+-------+----------+-------------+-----------+-------------------------+
| host | idc | cpu_util | memory_util | disk_util | ts |
+-------+-------+----------+-------------+-----------+-------------------------+
| host1 | idc_a | 11.8 | 10.3 | 10.3 | 2022-11-03T03:39:57.450 |
| host1 | idc_b | 50.0 | 66.7 | 40.6 | 2022-11-03T03:39:57.450 |
| host2 | idc_a | 80.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
| host3 | idc_a | 90.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
| host4 | idc_a | 70.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
| host5 | idc_a | 60.0 | 70.3 | 90.0 | 2022-11-03T03:39:57.450 |
+-------+-------+----------+-------------+-----------+-------------------------+
SELECT * FROM system_metrics;
+----------+-------+-------------------------+
| cpu_util | host | ts |
+----------+-------+-------------------------+
| 80.0 | host2 | 2022-11-03T03:39:57.450 |
| 70.0 | host4 | 2022-11-03T03:39:57.450 |
| 60.0 | host5 | 2022-11-03T03:39:57.450 |
| 90.0 | host3 | 2022-11-03T03:39:57.450 |
| 50.0 | host1 | 2022-11-03T03:39:57.450 |
+----------+-------+-------------------------+
DROP TABLE mito_system_metrics;
Affected Rows: 0
DROP TABLE system_metrics;
Affected Rows: 0
DROP TABLE phy;
Affected Rows: 0

View File

@@ -0,0 +1,60 @@
-- SQLNESS ARG version=v0.9.5
CREATE TABLE mito_system_metrics (
host STRING,
idc STRING,
cpu_util DOUBLE,
memory_util DOUBLE,
disk_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host, idc),
TIME INDEX(ts)
)ENGINE=mito;
INSERT INTO mito_system_metrics
VALUES
("host1", "idc_a", 11.8, 10.3, 10.3, 1667446797450),
("host2", "idc_a", 80.0, 70.3, 90.0, 1667446797450),
("host1", "idc_b", 50.0, 66.7, 40.6, 1667446797450);
CREATE TABLE phy (ts timestamp time index, cpu_util double) engine=metric with ("physical_metric_table" = "");
CREATE TABLE system_metrics (
host STRING,
cpu_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host),
TIME INDEX(ts)
)ENGINE=metric with ("on_physical_table" = "phy");
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host1', 11.8, 1667446797450),
('host2', 80.0, 1667446797450),
('host1', 50.0, 1667446797450);
-- SQLNESS ARG version=latest
SHOW CREATE TABLE mito_system_metrics;
SHOW CREATE TABLE system_metrics;
INSERT INTO mito_system_metrics
VALUES
("host3", "idc_a", 90.0, 70.3, 90.0, 1667446797450),
("host4", "idc_a", 70.0, 70.3, 90.0, 1667446797450),
("host5", "idc_a", 60.0, 70.3, 90.0, 1667446797450);
INSERT INTO system_metrics (host, cpu_util, ts)
VALUES
('host3', 90.0, 1667446797450),
('host4', 70.0, 1667446797450),
('host5', 60.0, 1667446797450);
SELECT * FROM mito_system_metrics;
SELECT * FROM system_metrics;
DROP TABLE mito_system_metrics;
DROP TABLE system_metrics;
DROP TABLE phy;

View File

@@ -0,0 +1,47 @@
-- SQLNESS ARG version=v0.9.5
CREATE TABLE system_metrics (
host STRING,
idc STRING,
cpu_util DOUBLE,
memory_util DOUBLE,
disk_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host, idc),
TIME INDEX(ts)
);
Affected Rows: 0
INSERT INTO system_metrics
VALUES
("host1", "idc_a", 11.8, 10.3, 10.3, 1667446797450),
("host2", "idc_a", 80.0, 70.3, 90.0, 1667446797450),
("host1", "idc_b", 50.0, 66.7, 40.6, 1667446797450);
Affected Rows: 3
-- SQLNESS ARG version=latest
SHOW CREATE TABLE system_metrics;
+----------------+-----------------------------------------------------------+
| Table | Create Table |
+----------------+-----------------------------------------------------------+
| system_metrics | CREATE TABLE IF NOT EXISTS "system_metrics" ( |
| | "host" STRING NULL, |
| | "idc" STRING NULL, |
| | "cpu_util" DOUBLE NULL, |
| | "memory_util" DOUBLE NULL, |
| | "disk_util" DOUBLE NULL, |
| | "ts" TIMESTAMP(3) NOT NULL DEFAULT current_timestamp(), |
| | TIME INDEX ("ts"), |
| | PRIMARY KEY ("host", "idc") |
| | ) |
| | |
| | ENGINE=mito |
| | |
+----------------+-----------------------------------------------------------+
DROP TABLE system_metrics;
Affected Rows: 0

View File

@@ -0,0 +1,22 @@
-- SQLNESS ARG version=v0.9.5
CREATE TABLE system_metrics (
host STRING,
idc STRING,
cpu_util DOUBLE,
memory_util DOUBLE,
disk_util DOUBLE,
ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP(),
PRIMARY KEY(host, idc),
TIME INDEX(ts)
);
INSERT INTO system_metrics
VALUES
("host1", "idc_a", 11.8, 10.3, 10.3, 1667446797450),
("host2", "idc_a", 80.0, 70.3, 90.0, 1667446797450),
("host1", "idc_b", 50.0, 66.7, 40.6, 1667446797450);
-- SQLNESS ARG version=latest
SHOW CREATE TABLE system_metrics;
DROP TABLE system_metrics;

View File

@@ -0,0 +1,153 @@
-- SQLNESS ARG version=v0.9.5
CREATE TABLE test_ttl_0s(ts TIMESTAMP TIME INDEX, val INT) WITH (ttl = '0 second');
Affected Rows: 0
CREATE TABLE test_ttl_1s(ts TIMESTAMP TIME INDEX, val INT) WITH (ttl = '1 second');
Affected Rows: 0
CREATE TABLE test_ttl_none(ts TIMESTAMP TIME INDEX, val INT);
Affected Rows: 0
CREATE DATABASE ttl_db_1s WITH (ttl = '1 second');
Affected Rows: 1
CREATE DATABASE ttl_db_0s WITH (ttl = '0 second');
Affected Rows: 1
CREATE DATABASE ttl_db_none;
Affected Rows: 1
-- SQLNESS ARG version=latest
SHOW TABLES;
+---------------+
| Tables |
+---------------+
| numbers |
| test_ttl_0s |
| test_ttl_1s |
| test_ttl_none |
+---------------+
SHOW CREATE TABLE test_ttl_1s;
+-------------+--------------------------------------------+
| Table | Create Table |
+-------------+--------------------------------------------+
| test_ttl_1s | CREATE TABLE IF NOT EXISTS "test_ttl_1s" ( |
| | "ts" TIMESTAMP(3) NOT NULL, |
| | "val" INT NULL, |
| | TIME INDEX ("ts") |
| | ) |
| | |
| | ENGINE=mito |
| | WITH( |
| | ttl = '1s' |
| | ) |
+-------------+--------------------------------------------+
SHOW CREATE TABLE test_ttl_0s;
+-------------+--------------------------------------------+
| Table | Create Table |
+-------------+--------------------------------------------+
| test_ttl_0s | CREATE TABLE IF NOT EXISTS "test_ttl_0s" ( |
| | "ts" TIMESTAMP(3) NOT NULL, |
| | "val" INT NULL, |
| | TIME INDEX ("ts") |
| | ) |
| | |
| | ENGINE=mito |
| | WITH( |
| | ttl = '0s' |
| | ) |
+-------------+--------------------------------------------+
SHOW CREATE TABLE test_ttl_none;
+---------------+----------------------------------------------+
| Table | Create Table |
+---------------+----------------------------------------------+
| test_ttl_none | CREATE TABLE IF NOT EXISTS "test_ttl_none" ( |
| | "ts" TIMESTAMP(3) NOT NULL, |
| | "val" INT NULL, |
| | TIME INDEX ("ts") |
| | ) |
| | |
| | ENGINE=mito |
| | |
+---------------+----------------------------------------------+
DROP TABLE test_ttl_1s;
Affected Rows: 0
DROP TABLE test_ttl_0s;
Affected Rows: 0
DROP TABLE test_ttl_none;
Affected Rows: 0
SHOW DATABASES;
+--------------------+
| Database |
+--------------------+
| greptime_private |
| information_schema |
| public |
| ttl_db_0s |
| ttl_db_1s |
| ttl_db_none |
+--------------------+
SHOW CREATE DATABASE ttl_db_1s;
+-----------+-----------------------------------------+
| Database | Create Database |
+-----------+-----------------------------------------+
| ttl_db_1s | CREATE DATABASE IF NOT EXISTS ttl_db_1s |
| | WITH( |
| | ttl = '1s' |
| | ) |
+-----------+-----------------------------------------+
SHOW CREATE DATABASE ttl_db_0s;
+-----------+-----------------------------------------+
| Database | Create Database |
+-----------+-----------------------------------------+
| ttl_db_0s | CREATE DATABASE IF NOT EXISTS ttl_db_0s |
| | WITH( |
| | ttl = '0s' |
| | ) |
+-----------+-----------------------------------------+
SHOW CREATE DATABASE ttl_db_none;
+-------------+-------------------------------------------+
| Database | Create Database |
+-------------+-------------------------------------------+
| ttl_db_none | CREATE DATABASE IF NOT EXISTS ttl_db_none |
+-------------+-------------------------------------------+
DROP DATABASE ttl_db_1s;
Affected Rows: 0
DROP DATABASE ttl_db_0s;
Affected Rows: 0
DROP DATABASE ttl_db_none;
Affected Rows: 0

View File

@@ -0,0 +1,42 @@
-- SQLNESS ARG version=v0.9.5
CREATE TABLE test_ttl_0s(ts TIMESTAMP TIME INDEX, val INT) WITH (ttl = '0 second');
CREATE TABLE test_ttl_1s(ts TIMESTAMP TIME INDEX, val INT) WITH (ttl = '1 second');
CREATE TABLE test_ttl_none(ts TIMESTAMP TIME INDEX, val INT);
CREATE DATABASE ttl_db_1s WITH (ttl = '1 second');
CREATE DATABASE ttl_db_0s WITH (ttl = '0 second');
CREATE DATABASE ttl_db_none;
-- SQLNESS ARG version=latest
SHOW TABLES;
SHOW CREATE TABLE test_ttl_1s;
SHOW CREATE TABLE test_ttl_0s;
SHOW CREATE TABLE test_ttl_none;
DROP TABLE test_ttl_1s;
DROP TABLE test_ttl_0s;
DROP TABLE test_ttl_none;
SHOW DATABASES;
SHOW CREATE DATABASE ttl_db_1s;
SHOW CREATE DATABASE ttl_db_0s;
SHOW CREATE DATABASE ttl_db_none;
DROP DATABASE ttl_db_1s;
DROP DATABASE ttl_db_0s;
DROP DATABASE ttl_db_none;