Skip to content

Commit

Permalink
refactor!: remove opentsdb tcp server (#3828)
Browse files Browse the repository at this point in the history
* refactor: remove opentsdb tcp server

* refactor: remove config and add test

* refactor: update docs and remove unused code
  • Loading branch information
shuiyisong authored May 6, 2024
1 parent 6e12e1b commit 6e9e8fa
Show file tree
Hide file tree
Showing 17 changed files with 15 additions and 953 deletions.
8 changes: 2 additions & 6 deletions config/config.md
Original file line number Diff line number Diff line change
Expand Up @@ -33,9 +33,7 @@
| `postgres.tls.key_path` | String | `None` | Private key file path. |
| `postgres.tls.watch` | Bool | `false` | Watch for Certificate and key file change and auto reload |
| `opentsdb` | -- | -- | OpenTSDB protocol options. |
| `opentsdb.enable` | Bool | `true` | Whether to enable |
| `opentsdb.addr` | String | `127.0.0.1:4242` | OpenTSDB telnet API server address. |
| `opentsdb.runtime_size` | Integer | `2` | The number of server worker threads. |
| `opentsdb.enable` | Bool | `true` | Whether to enable OpenTSDB put in HTTP API. |
| `influxdb` | -- | -- | InfluxDB protocol options. |
| `influxdb.enable` | Bool | `true` | Whether to enable InfluxDB protocol in HTTP API. |
| `prom_store` | -- | -- | Prometheus remote storage options |
Expand Down Expand Up @@ -168,9 +166,7 @@
| `postgres.tls.key_path` | String | `None` | Private key file path. |
| `postgres.tls.watch` | Bool | `false` | Watch for Certificate and key file change and auto reload |
| `opentsdb` | -- | -- | OpenTSDB protocol options. |
| `opentsdb.enable` | Bool | `true` | Whether to enable |
| `opentsdb.addr` | String | `127.0.0.1:4242` | OpenTSDB telnet API server address. |
| `opentsdb.runtime_size` | Integer | `2` | The number of server worker threads. |
| `opentsdb.enable` | Bool | `true` | Whether to enable OpenTSDB put in HTTP API. |
| `influxdb` | -- | -- | InfluxDB protocol options. |
| `influxdb.enable` | Bool | `true` | Whether to enable InfluxDB protocol in HTTP API. |
| `prom_store` | -- | -- | Prometheus remote storage options |
Expand Down
6 changes: 1 addition & 5 deletions config/frontend.example.toml
Original file line number Diff line number Diff line change
Expand Up @@ -88,12 +88,8 @@ watch = false

## OpenTSDB protocol options.
[opentsdb]
## Whether to enable
## Whether to enable OpenTSDB put in HTTP API.
enable = true
## OpenTSDB telnet API server address.
addr = "127.0.0.1:4242"
## The number of server worker threads.
runtime_size = 2

## InfluxDB protocol options.
[influxdb]
Expand Down
6 changes: 1 addition & 5 deletions config/standalone.example.toml
Original file line number Diff line number Diff line change
Expand Up @@ -83,12 +83,8 @@ watch = false

## OpenTSDB protocol options.
[opentsdb]
## Whether to enable
## Whether to enable OpenTSDB put in HTTP API.
enable = true
## OpenTSDB telnet API server address.
addr = "127.0.0.1:4242"
## The number of server worker threads.
runtime_size = 2

## InfluxDB protocol options.
[influxdb]
Expand Down
17 changes: 4 additions & 13 deletions src/cmd/src/frontend.rs
Original file line number Diff line number Diff line change
Expand Up @@ -126,8 +126,6 @@ pub struct StartCommand {
mysql_addr: Option<String>,
#[clap(long)]
postgres_addr: Option<String>,
#[clap(long)]
opentsdb_addr: Option<String>,
#[clap(short, long)]
config_file: Option<String>,
#[clap(short, long)]
Expand Down Expand Up @@ -198,11 +196,6 @@ impl StartCommand {
opts.postgres.tls = tls_opts;
}

if let Some(addr) = &self.opentsdb_addr {
opts.opentsdb.enable = true;
opts.opentsdb.addr.clone_from(addr);
}

if let Some(enable) = self.influxdb_enable {
opts.influxdb.enable = enable;
}
Expand Down Expand Up @@ -319,7 +312,6 @@ mod tests {
http_addr: Some("127.0.0.1:1234".to_string()),
mysql_addr: Some("127.0.0.1:5678".to_string()),
postgres_addr: Some("127.0.0.1:5432".to_string()),
opentsdb_addr: Some("127.0.0.1:4321".to_string()),
influxdb_enable: Some(false),
disable_dashboard: Some(false),
..Default::default()
Expand All @@ -333,7 +325,6 @@ mod tests {
assert_eq!(ReadableSize::mb(64), opts.http.body_limit);
assert_eq!(opts.mysql.addr, "127.0.0.1:5678");
assert_eq!(opts.postgres.addr, "127.0.0.1:5432");
assert_eq!(opts.opentsdb.addr, "127.0.0.1:4321");

let default_opts = FrontendOptions::default();

Expand All @@ -346,10 +337,6 @@ mod tests {
default_opts.postgres.runtime_size
);
assert!(opts.opentsdb.enable);
assert_eq!(
opts.opentsdb.runtime_size,
default_opts.opentsdb.runtime_size
);

assert!(!opts.influxdb.enable);
}
Expand All @@ -365,6 +352,9 @@ mod tests {
timeout = "30s"
body_limit = "2GB"
[opentsdb]
enable = false
[logging]
level = "debug"
dir = "/tmp/greptimedb/test/logs"
Expand All @@ -389,6 +379,7 @@ mod tests {

assert_eq!("debug", fe_opts.logging.level.as_ref().unwrap());
assert_eq!("/tmp/greptimedb/test/logs".to_string(), fe_opts.logging.dir);
assert!(!fe_opts.opentsdb.enable);
}

#[tokio::test]
Expand Down
11 changes: 4 additions & 7 deletions src/cmd/src/standalone.rs
Original file line number Diff line number Diff line change
Expand Up @@ -256,8 +256,6 @@ pub struct StartCommand {
mysql_addr: Option<String>,
#[clap(long)]
postgres_addr: Option<String>,
#[clap(long)]
opentsdb_addr: Option<String>,
#[clap(short, long)]
influxdb_enable: bool,
#[clap(short, long)]
Expand Down Expand Up @@ -343,11 +341,6 @@ impl StartCommand {
opts.postgres.tls = tls_opts;
}

if let Some(addr) = &self.opentsdb_addr {
opts.opentsdb.enable = true;
opts.opentsdb.addr.clone_from(addr);
}

if self.influxdb_enable {
opts.influxdb.enable = self.influxdb_enable;
}
Expand Down Expand Up @@ -610,6 +603,9 @@ mod tests {
timeout = "33s"
body_limit = "128MB"
[opentsdb]
enable = true
[logging]
level = "debug"
dir = "/tmp/greptimedb/test/logs"
Expand Down Expand Up @@ -637,6 +633,7 @@ mod tests {
assert_eq!(2, fe_opts.mysql.runtime_size);
assert_eq!(None, fe_opts.mysql.reject_no_database);
assert!(fe_opts.influxdb.enable);
assert!(fe_opts.opentsdb.enable);

let DatanodeWalConfig::RaftEngine(raft_engine_config) = dn_opts.wal else {
unreachable!()
Expand Down
19 changes: 0 additions & 19 deletions src/frontend/src/server.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ use servers::grpc::{GrpcServer, GrpcServerConfig};
use servers::http::{HttpServer, HttpServerBuilder};
use servers::metrics_handler::MetricsHandler;
use servers::mysql::server::{MysqlServer, MysqlSpawnConfig, MysqlSpawnRef};
use servers::opentsdb::OpentsdbServer;
use servers::postgres::PostgresServer;
use servers::query_handler::grpc::ServerGrpcQueryHandlerAdapter;
use servers::query_handler::sql::ServerSqlQueryHandlerAdapter;
Expand Down Expand Up @@ -258,24 +257,6 @@ where
handlers.insert((pg_server, pg_addr)).await;
}

if opts.opentsdb.enable {
// Init OpenTSDB server
let opts = &opts.opentsdb;
let addr = parse_addr(&opts.addr)?;

let io_runtime = Arc::new(
RuntimeBuilder::default()
.worker_threads(opts.runtime_size)
.thread_name("opentsdb-io-handlers")
.build()
.context(error::RuntimeResourceSnafu)?,
);

let server = OpentsdbServer::create_server(instance.clone(), io_runtime);

handlers.insert((server, addr)).await;
}

Ok(handlers)
}
}
Expand Down
8 changes: 1 addition & 7 deletions src/frontend/src/service_config/opentsdb.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,10 @@ use serde::{Deserialize, Serialize};
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq)]
pub struct OpentsdbOptions {
pub enable: bool,
pub addr: String,
pub runtime_size: usize,
}

impl Default for OpentsdbOptions {
fn default() -> Self {
Self {
enable: true,
addr: "127.0.0.1:4242".to_string(),
runtime_size: 2,
}
Self { enable: true }
}
}
9 changes: 0 additions & 9 deletions src/servers/src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -190,13 +190,6 @@ pub enum Error {
error: hyper::Error,
},

#[snafu(display("Invalid OpenTSDB line"))]
InvalidOpentsdbLine {
#[snafu(source)]
error: FromUtf8Error,
location: Location,
},

#[snafu(display("Invalid OpenTSDB Json request"))]
InvalidOpentsdbJsonRequest {
#[snafu(source)]
Expand Down Expand Up @@ -508,7 +501,6 @@ impl ErrorExt for Error {
| InvalidQuery { .. }
| InfluxdbLineProtocol { .. }
| ConnResetByPeer { .. }
| InvalidOpentsdbLine { .. }
| InvalidOpentsdbJsonRequest { .. }
| DecodePromRemoteRequest { .. }
| DecodeOtlpRequest { .. }
Expand Down Expand Up @@ -664,7 +656,6 @@ impl IntoResponse for Error {
Error::InfluxdbLineProtocol { .. }
| Error::RowWriter { .. }
| Error::PromSeriesWrite { .. }
| Error::InvalidOpentsdbLine { .. }
| Error::InvalidOpentsdbJsonRequest { .. }
| Error::DecodePromRemoteRequest { .. }
| Error::DecodeOtlpRequest { .. }
Expand Down
1 change: 0 additions & 1 deletion src/servers/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,6 @@ pub mod query_handler;
pub mod repeated_field;
mod row_writer;
pub mod server;
mod shutdown;
pub mod tls;

pub use common_config::Mode;
Expand Down
9 changes: 2 additions & 7 deletions src/servers/src/metrics.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@ use axum::response::IntoResponse;
use hyper::Body;
use lazy_static::lazy_static;
use prometheus::{
register_histogram, register_histogram_vec, register_int_counter, register_int_counter_vec,
register_int_gauge, Histogram, HistogramVec, IntCounter, IntCounterVec, IntGauge,
register_histogram_vec, register_int_counter, register_int_counter_vec, register_int_gauge,
Histogram, HistogramVec, IntCounter, IntCounterVec, IntGauge,
};
use tonic::body::BoxBody;
use tower::{Layer, Service};
Expand Down Expand Up @@ -130,11 +130,6 @@ lazy_static! {
&[METRIC_DB_LABEL]
)
.unwrap();
pub static ref METRIC_TCP_OPENTSDB_LINE_WRITE_ELAPSED: Histogram = register_histogram!(
"greptime_servers_opentsdb_line_write_elapsed",
"servers opentsdb line write elapsed"
)
.unwrap();
pub static ref METRIC_MYSQL_CONNECTIONS: IntGauge = register_int_gauge!(
"greptime_servers_mysql_connection_count",
"servers mysql connection count"
Expand Down
113 changes: 0 additions & 113 deletions src/servers/src/opentsdb.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,127 +13,14 @@
// limitations under the License.

pub mod codec;
pub mod connection;
mod handler;

use std::future::Future;
use std::net::SocketAddr;
use std::sync::Arc;

use api::v1::RowInsertRequests;
use async_trait::async_trait;
use common_error::ext::ErrorExt;
use common_grpc::precision::Precision;
use common_query::prelude::{GREPTIME_TIMESTAMP, GREPTIME_VALUE};
use common_runtime::Runtime;
use common_telemetry::logging::{debug, error, warn};
use futures::StreamExt;
use tokio::sync::broadcast;

use self::codec::DataPoint;
use crate::error::Result;
use crate::opentsdb::connection::Connection;
use crate::opentsdb::handler::Handler;
use crate::query_handler::OpentsdbProtocolHandlerRef;
use crate::row_writer::{self, MultiTableData};
use crate::server::{AbortableStream, BaseTcpServer, Server};
use crate::shutdown::Shutdown;

pub struct OpentsdbServer {
base_server: BaseTcpServer,
query_handler: OpentsdbProtocolHandlerRef,

/// Broadcasts a shutdown signal to all active connections.
///
/// When a connection task is spawned, it is passed a broadcast receiver handle. We can send
/// a `()` value via `notify_shutdown` or just drop `notify_shutdown`, then each active
/// connection receives it, reaches a safe terminal state, and completes the task.
notify_shutdown: Option<broadcast::Sender<()>>,
}

impl OpentsdbServer {
pub fn create_server(
query_handler: OpentsdbProtocolHandlerRef,
io_runtime: Arc<Runtime>,
) -> Box<dyn Server> {
// When the provided `shutdown` future completes, we must send a shutdown
// message to all active connections. We use a broadcast channel for this
// purpose. The call below ignores the receiver of the broadcast pair, and when
// a receiver is needed, the subscribe() method on the sender is used to create
// one.
let (notify_shutdown, _) = broadcast::channel(1);

Box::new(OpentsdbServer {
base_server: BaseTcpServer::create_server("OpenTSDB", io_runtime),
query_handler,
notify_shutdown: Some(notify_shutdown),
})
}

fn accept(
&self,
io_runtime: Arc<Runtime>,
stream: AbortableStream,
) -> impl Future<Output = ()> {
let query_handler = self.query_handler.clone();
let notify_shutdown = self
.notify_shutdown
.clone()
.expect("`notify_shutdown` must be present when accepting connection!");
stream.for_each(move |stream| {
let io_runtime = io_runtime.clone();
let query_handler = query_handler.clone();
let shutdown = Shutdown::new(notify_shutdown.subscribe());
async move {
match stream {
Ok(stream) => {
if let Err(e) = stream.set_nodelay(true) {
warn!(e; "Failed to set TCP nodelay");
}
let connection = Connection::new(stream);
let mut handler = Handler::new(query_handler, connection, shutdown);

let _handle = io_runtime.spawn(async move {
if let Err(e) = handler.run().await {
if e.status_code().should_log_error() {
error!(e; "Unexpected error when handling OpenTSDB connection");
}
}
});
}
Err(error) => debug!("Broken pipe: {}", error), // IoError doesn't impl ErrorExt.
};
}
})
}
}

pub const OPENTSDB_SERVER: &str = "OPENTSDB_SERVER";

#[async_trait]
impl Server for OpentsdbServer {
async fn shutdown(&self) -> Result<()> {
if let Some(tx) = &self.notify_shutdown {
// Err of broadcast sender does not mean that future calls to send will fail, so
// its return value is ignored here.
let _ = tx.send(());
}
self.base_server.shutdown().await?;
Ok(())
}

async fn start(&self, listening: SocketAddr) -> Result<SocketAddr> {
let (stream, addr) = self.base_server.bind(listening).await?;

let io_runtime = self.base_server.io_runtime();
let join_handle = common_runtime::spawn_read(self.accept(io_runtime, stream));
self.base_server.start_with(join_handle).await?;
Ok(addr)
}
fn name(&self) -> &str {
OPENTSDB_SERVER
}
}

pub fn data_point_to_grpc_row_insert_requests(
data_points: Vec<DataPoint>,
Expand Down
Loading

0 comments on commit 6e9e8fa

Please sign in to comment.