Compare commits

...

22 Commits

Author SHA1 Message Date
Hengfei Yang 473b908746
Merge branch 'main' into feat/search-event-ctx 2024-10-31 18:21:27 +08:00
Hengfei Yang 26bafe90f7
fix: prefix partition key with inverted index (#4962) 2024-10-31 18:10:21 +08:00
Taiming Liu ffe469657e Merge branch 'main' into feat/search-event-ctx 2024-10-30 22:52:21 -07:00
Taiming Liu a27421f442 clippy: box usage buffer enum 2024-10-30 22:17:12 -07:00
Sai Nikhil 0ebaf21f33
feat: Pipeline integration v2 (#4732)
This PR implements the new design of **Pipeline** introduced by #4432 

---

**2 types of pipelines differentiates by the source** 

Type I: **Realtime**
- source is a stream (Logs, Metrics, & Traces)
- takes effect when the source stream is being ingested

Type II: **Scheduled** (aka DerivedSteram)
- source is a SQL query
- takes effect based on the given schedule

The new pipeline is represented as a graph consists of different types
of Nodes that are connected by edges.
**Pipeline Data Model**
```rs
pub struct Pipeline {
    pub id: String,
    pub version: i32,
    pub enabled: bool,
    pub org: String,
    pub name: String,
    pub description: String,
    pub source: PipelineSource,
    pub nodes: Vec<Node>,
    pub edges: Vec<Edge>,
}
```
**4 types of node**
I. StreamNode: either used for source node of realtime pipeline or for
destination nodes
II. QueryNode: used for source node of scheduled pipeline
III. FunctionNode: used for executing vrl functions
IV. ConditionNode: used for checking routing conditions

Rules applied to validating a pipeline when it's created:
1. non-empty nodes list
2. non-empty edges list
3. 1st node in nodes list is either StreamNode or QueryNode
4. non-empty `conditions` in all ConditionNode nodes in nodes list
5. all leaf nodes are of type StreamNode
6. In the same branch, unchecked `after_flattened` FunctionNode can't
follow checked
   `after_flattened` checked FunctionNode

Pipeline execution is implemented with the struct `ExecutablePipeline`,
a ready-execute pipeline object cached in memory based on the Pipeline
objects modified on the UI side.
`ExecutablePipeline` object
- builds the relationships among all the nodes based on the edges
- topologically sorts the nodes based on the level to determine which
node to process first
- registers all the vrl function node once

`ExecutablePipeline` object processes ingested records in batch.
Starting from the source node, each connecting edge has a channel
connecting the parent and child node, used for passing the records. This
approach enables batch processing in parallel.

---

**Deprecated Features**
1. previous implementation of pipeline
2. Function x Stream associations
 - Functions can still be added/edited/removed as before
- Needs to create a pipeline in order to apply chosen functions to the
desired stream

The new release when this pr is merged will automatically migrate **old
pipelines** and **Function x Stream associations** to new pipeline
format and can be found in `Stream Pipelines` tab with `Migrated-`
prefix names.
**_Note: Auto generated pipelines are paused by default. Please verify
the pipelines before enabling them_**


Co-authored-by: Taiming Liu <liutaiming3@gmail.com>
Co-authored-by: Bhargav <BJP232004@GMAIL.COM>
2024-10-30 21:53:28 -07:00
Sai Nikhil 61ce5a0bf5
fix: issue #4935 (#4940)
This PR fixes 
1.#4935 
2.This also tries to add wrap the stream name into double quotes so
whenever any field also contain the stream name it ignores that

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## Summary by CodeRabbit

- **New Features**
- Enhanced error handling for search results, improving user feedback
based on various error states.
- Improved accuracy in SQL query modifications related to stream names,
preventing unintended replacements.
- More precise filtering of fields during removal, enhancing the field
management process.
- Adjusted histogram interval conditions for better data generation,
resulting in finer granularity.
- More descriptive error messages for improved user notifications during
search operations.

- **Bug Fixes**
- Refined query cancellation process to notify users effectively if a
search query is interrupted.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
2024-10-31 09:48:05 +05:30
Taiming Liu a634feb205
feat: channel-based usage queue for async usage reporting (#4949)
impl #4947 

Two existing `env` to control Usage data ingestion
1. `ZO_USAGE_BATCH_SIZE`: in-memory cache size threshold to initiate
ingestion process
2. `ZO_USAGE_PUBLISH_INTERVAL`: duration in seconds between each usage
reporting
2024-10-30 20:21:14 -07:00
Taiming Liu 3966498e55
fix: parquet move_files split prefix to more columns for prefix_date (#4958) 2024-10-31 08:43:04 +08:00
Taiming Liu e8d7b3e994
Update src/common/utils/http.rs
Co-authored-by: coderabbitai[bot] <136622811+coderabbitai[bot]@users.noreply.github.com>
2024-10-30 12:30:36 -07:00
Taiming Liu d1a53600c8 fix: parquet move_files split prefix to more columns for prefix_date 2024-10-30 11:21:49 -07:00
Taiming Liu aea054e68e update parameter name to folder_id and add report_id to report query url 2024-10-30 11:10:08 -07:00
Sai Nikhil 322ed67a1e added reportId from route 2024-10-30 10:09:03 -07:00
Sai Nikhil 2b8948b7c7 fix: search event context , reportid and dashboard id added 2024-10-30 10:09:03 -07:00
Taiming Liu 5c40a1f37e update: add dashboard_id and dashboard_folder_id to dashboard serch context 2024-10-30 10:09:03 -07:00
Taiming Liu 3c06d33bd0 updte per review suggestions 2024-10-30 10:09:03 -07:00
Taiming Liu 7bd85e54fc feat: SearchEventContext - additional details for search usage reports 2024-10-30 10:09:03 -07:00
Taiming Liu be5e3c5b06 chore: remove search_event_type from grpc search 2024-10-30 10:09:03 -07:00
Hengfei Yang 3955ed02f2
feat: add jobs for compact old data (#4952) 2024-10-31 00:33:09 +08:00
dependabot[bot] fe32354aa4
chore(deps): bump elliptic from 6.5.7 to 6.6.0 in /web (#4953) 2024-10-30 23:09:01 +08:00
Omkar Kesarkhane c4d164c230
feat: added Logstash data source in logs (#4957) 2024-10-30 22:13:36 +08:00
Huaijin 757c9dcb7b
fix: add search_type to search_multi (#4955) 2024-10-30 21:21:43 +08:00
Neha P 5306ad899f
test: clone alert (#4931)
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## Summary by CodeRabbit

- **New Features**
- Enhanced alert creation and cloning functionality with improved
validation checks.
- Added support for comprehensive search capabilities in the alert list.
- Introduced functionality to verify user interactions with search
history.

- **Bug Fixes**
	- Improved error handling and user feedback during alert cloning.
	- Enhanced validation for required fields in alert creation.

- **Tests**
- New test cases added for alert management and search history
functionalities.
- Updated existing tests to ensure better coverage and error handling,
including the addition of new scenarios for invalid inputs.

- **Chores**
- Cleaned up the test suite by removing unused code and improving
structure.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Co-authored-by: Sai Nikhil <nikhil@openobserve.ai>
2024-10-30 17:53:27 +05:30
165 changed files with 12001 additions and 5298 deletions

2
Cargo.lock generated
View File

@ -2515,6 +2515,7 @@ dependencies = [
"serde",
"serde_json",
"sqlparser",
"sqlx",
"svix-ksuid",
"sysinfo",
"tokio",
@ -2523,6 +2524,7 @@ dependencies = [
"tracing-subscriber",
"urlencoding",
"utoipa",
"vrl",
"zstd",
]

View File

@ -99,6 +99,14 @@ pub async fn cli() -> Result<bool, anyhow::Error> {
.help("migrate to: sqlite, etcd, mysql, postgresql"),
]),
clap::Command::new("migrate-dashboards").about("migrate-dashboards"),
clap::Command::new("migrate-pipeline").about("migrate pipelines")
.arg(
clap::Arg::new("drop-table")
.long("drop-table")
.required(false)
.num_args(0)
.help("Drop existing Pipeline table first before migrating")
),
clap::Command::new("delete-parquet")
.about("delete parquet files from s3 and file_list")
.arg(
@ -246,6 +254,11 @@ pub async fn cli() -> Result<bool, anyhow::Error> {
println!("Running migration dashboard");
migration::dashboards::run().await?
}
"migrate-pipeline" => {
println!("Running migration pipeline");
let drop_table = command.get_flag("drop-table");
migration::pipeline_func::run(drop_table).await?;
}
"delete-parquet" => {
let file = command.get_one::<String>("file").unwrap();
match file_list::delete_parquet_file(file, true).await {

View File

@ -24,7 +24,10 @@ use config::meta::{
use crate::{
cli::data::{cli::Cli, Context},
common::utils::http::{get_search_type_from_request, get_stream_type_from_request},
common::utils::http::{
get_search_event_context_from_request, get_search_type_from_request,
get_stream_type_from_request,
},
service::search as SearchService,
};
@ -34,18 +37,22 @@ pub struct Export {}
impl Context for Export {
async fn operator(c: Cli) -> Result<bool, anyhow::Error> {
let map = HashMap::from([("type".to_string(), c.stream_type)]);
let query_map = Query(map);
let stream_type = match get_stream_type_from_request(&Query(map.clone())) {
let stream_type = match get_stream_type_from_request(&query_map) {
Ok(v) => v.unwrap_or(StreamType::Logs),
Err(_) => return Ok(false),
};
let cfg = config::get_config();
let table = c.stream_name;
let search_type = match get_search_type_from_request(&Query(map.clone())) {
let search_type = match get_search_type_from_request(&query_map) {
Ok(v) => v,
Err(_) => return Ok(false),
};
let search_event_context = search_type
.as_ref()
.and_then(|event_type| get_search_event_context_from_request(event_type, &query_map));
let query = search::Query {
sql: format!("select * from {}", table),
from: 0,
@ -68,6 +75,7 @@ impl Context for Export {
clusters: vec![],
timeout: 0,
search_type,
search_event_context,
index_type: "".to_string(),
};

View File

@ -15,7 +15,14 @@
use std::sync::Arc;
use config::{RwAHashMap, RwHashMap};
use config::{
meta::{
alerts::{alert::Alert, destinations::Destination, templates::Template},
function::Transform,
stream::StreamParams,
},
RwAHashMap, RwHashMap,
};
use dashmap::DashMap;
use hashbrown::HashMap;
use infra::table::short_urls::ShortUrlRecord;
@ -25,18 +32,12 @@ use vector_enrichment::TableRegistry;
use crate::{
common::meta::{
alerts::{alert::Alert, destinations::Destination, templates::Template},
dashboards::reports,
functions::{StreamFunctionsList, Transform},
maxmind::MaxmindClient,
organization::OrganizationSetting,
pipelines::PipeLine,
prom::ClusterLeader,
syslog::SyslogRoute,
user::User,
dashboards::reports, maxmind::MaxmindClient, organization::OrganizationSetting,
prom::ClusterLeader, syslog::SyslogRoute, user::User,
},
service::{
db::scheduler as db_scheduler, enrichment::StreamTable, enrichment_table::geoip::Geoip,
pipeline::batch_execution::ExecutablePipeline,
},
};
@ -47,8 +48,6 @@ pub static BUILD_DATE: &str = env!("GIT_BUILD_DATE");
// global cache variables
pub static KVS: Lazy<RwHashMap<String, bytes::Bytes>> = Lazy::new(Default::default);
pub static STREAM_FUNCTIONS: Lazy<RwHashMap<String, StreamFunctionsList>> =
Lazy::new(DashMap::default);
pub static QUERY_FUNCTIONS: Lazy<RwHashMap<String, Transform>> = Lazy::new(DashMap::default);
pub static USERS: Lazy<RwHashMap<String, User>> = Lazy::new(DashMap::default);
pub static USERS_RUM_TOKEN: Lazy<Arc<RwHashMap<String, User>>> =
@ -83,6 +82,7 @@ pub static GEOIP_CITY_TABLE: Lazy<Arc<RwLock<Option<Geoip>>>> =
pub static GEOIP_ASN_TABLE: Lazy<Arc<RwLock<Option<Geoip>>>> =
Lazy::new(|| Arc::new(RwLock::new(None)));
pub static STREAM_EXECUTABLE_PIPELINES: Lazy<RwAHashMap<StreamParams, ExecutablePipeline>> =
Lazy::new(Default::default);
pub static USER_SESSIONS: Lazy<RwHashMap<String, String>> = Lazy::new(Default::default);
pub static STREAM_PIPELINES: Lazy<RwHashMap<String, PipeLine>> = Lazy::new(DashMap::default);
pub static SHORT_URLS: Lazy<RwHashMap<String, ShortUrlRecord>> = Lazy::new(DashMap::default);

View File

@ -13,10 +13,8 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
pub mod alerts;
pub mod authz;
pub mod dashboards;
pub mod functions;
pub mod http;
pub mod ingestion;
pub mod maxmind;

View File

@ -13,11 +13,10 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use config::meta::{alerts::alert::Alert, function::Transform};
use serde::{Deserialize, Serialize};
use utoipa::ToSchema;
use super::{alerts::alert::Alert, functions::Transform};
pub const DEFAULT_ORG: &str = "default";
pub const CUSTOM: &str = "custom";
pub const THRESHOLD: i64 = 9383939382;

View File

@ -16,16 +16,15 @@
use std::collections::HashMap;
use config::{
meta::stream::{RoutingCondition, StreamType},
meta::{
alerts::derived_streams::DerivedStreamMeta,
stream::{RoutingCondition, StreamType},
},
utils::json::Value,
};
use serde::{Deserialize, Serialize};
use utoipa::ToSchema;
use crate::common::meta::{
alerts::derived_streams::DerivedStreamMeta, functions::StreamFunctionsList,
};
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema, PartialEq)]
pub struct PipeLine {
pub name: String,
@ -42,42 +41,3 @@ pub struct PipeLine {
#[serde(skip_serializing_if = "Option::is_none")]
pub meta: Option<HashMap<String, Value>>,
}
impl PipeLine {
pub fn into_response(self, functions: Option<StreamFunctionsList>) -> PipeLineResponse {
PipeLineResponse {
name: self.name,
description: self.description,
stream_name: self.stream_name,
stream_type: self.stream_type,
routing: self.routing,
derived_streams: self.derived_streams,
functions,
meta: self.meta,
}
}
}
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema, PartialEq)]
pub struct PipeLineResponse {
pub name: String,
#[serde(default)]
pub description: String,
#[serde(default)]
pub stream_name: String,
#[serde(default)]
pub stream_type: StreamType,
#[serde(skip_serializing_if = "Option::is_none")]
pub routing: Option<HashMap<String, Vec<RoutingCondition>>>,
#[serde(skip_serializing_if = "Option::is_none")]
pub functions: Option<StreamFunctionsList>,
#[serde(skip_serializing_if = "Option::is_none")]
pub derived_streams: Option<Vec<DerivedStreamMeta>>,
#[serde(skip_serializing_if = "Option::is_none")]
pub meta: Option<HashMap<String, Value>>,
}
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema)]
pub struct PipeLineList {
pub list: Vec<PipeLineResponse>,
}

View File

@ -251,12 +251,6 @@ pub async fn add_zo_info(mut data: HashMap<String, json::Value>) -> HashMap<Stri
format!("{:.0}", (traces_compressed_size / SIZE_IN_MB)).into(),
);
let iter = STREAM_FUNCTIONS.iter().clone();
let mut ingest_functions = 0;
for item in iter {
ingest_functions += item.value().list.len()
}
data.insert("num_ingest_functions".to_string(), ingest_functions.into());
data.insert(
"num_query_functions".to_string(),
QUERY_FUNCTIONS.len().into(),

View File

@ -20,6 +20,7 @@ use version_compare::Version;
pub mod dashboards;
pub mod file_list;
pub mod meta;
pub mod pipeline_func;
pub mod schema;
pub async fn check_upgrade(old_ver: &str, new_ver: &str) -> Result<(), anyhow::Error> {
@ -46,6 +47,11 @@ pub async fn check_upgrade(old_ver: &str, new_ver: &str) -> Result<(), anyhow::E
upgrade_092_093().await?;
}
let v131 = Version::from("v0.13.1").unwrap();
if old_ver < v131 {
upgrade_130_131().await?;
}
Ok(())
}
@ -66,6 +72,13 @@ async fn upgrade_092_093() -> Result<(), anyhow::Error> {
Ok(())
}
async fn upgrade_130_131() -> Result<(), anyhow::Error> {
// migrate pipelines and function associations
pipeline_func::run(false).await?;
Ok(())
}
pub async fn upgrade_resource_names() -> Result<(), anyhow::Error> {
// The below migration requires ofga init ready, but on Router node,
// we don't initialize ofga, hence the migration should not run on router

View File

@ -0,0 +1,364 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use std::collections::HashMap;
use config::{
ider,
meta::{
function,
pipeline::{
components::{
ConditionParams, DerivedStream, Edge, FunctionParams, Node, NodeData,
PipelineSource,
},
Pipeline,
},
stream::{StreamParams, StreamType},
},
utils::json,
};
use infra::db as infra_db;
pub async fn run(drop_table_first: bool) -> Result<(), anyhow::Error> {
if drop_table_first {
infra::pipeline::drop_table().await?;
}
infra::pipeline::init().await?;
migrate_pipelines().await?;
Ok(())
}
async fn migrate_pipelines() -> Result<(), anyhow::Error> {
let mut new_pipeline_by_source: HashMap<StreamParams, Pipeline> = HashMap::new();
let mut func_to_update = vec![];
// load all functions from meta table
let mut stream_funcs: HashMap<StreamParams, Vec<(u8, FunctionParams)>> = HashMap::new();
let db = infra_db::get_db().await;
let db_key = "/function/";
let data = db.list(db_key).await?;
for (key, val) in data {
let local_key = key.strip_prefix('/').unwrap_or(&key);
let key_col = local_key.split('/').collect::<Vec<&str>>();
let org_id = key_col[1];
let mut trans: function::Transform = json::from_slice(&val).unwrap();
if let Some(stream_orders) = trans.streams.clone() {
for stream_ord in stream_orders {
if !matches!(
stream_ord.stream_type,
StreamType::Logs | StreamType::Metrics | StreamType::Traces
) {
continue;
}
let func_params = FunctionParams {
name: trans.name.clone(),
num_args: trans.num_args,
after_flatten: !stream_ord.apply_before_flattening,
};
let entry = stream_funcs
.entry(StreamParams::new(
org_id,
&stream_ord.stream,
stream_ord.stream_type,
))
.or_default();
entry.push((stream_ord.order, func_params));
}
trans.streams = None;
func_to_update.push((org_id.to_string(), trans));
}
}
// load all old pipelines from meta table
let db = infra_db::get_db().await;
let db_key = "/pipeline/";
let data = db.list(db_key).await?;
for (key, val) in data {
let local_key = key.strip_prefix('/').unwrap_or(&key);
let key_col = local_key.split('/').collect::<Vec<&str>>();
let old_pipe: crate::common::meta::pipelines::PipeLine = json::from_slice(&val).unwrap();
// two scenarios:
// scenario 1: with DerivedStream info -> scheduled
if let Some(old_derived_streams) = old_pipe.derived_streams {
for old_derived_stream in old_derived_streams {
let new_derived_stream = DerivedStream {
org_id: old_derived_stream.source.org_id.to_string(),
stream_type: old_derived_stream.source.stream_type,
query_condition: old_derived_stream.query_condition,
trigger_condition: old_derived_stream.trigger_condition,
tz_offset: old_derived_stream.tz_offset,
};
let pipeline_source = PipelineSource::Scheduled(new_derived_stream.clone());
// construct the nodes and edges lists
let source_node_data = NodeData::Query(new_derived_stream);
let dest_node_data = NodeData::Stream(old_derived_stream.destination);
let (pos_x, pos_y): (f32, f32) = (50.0, 50.0);
let pos_offset: f32 = 200.0;
let source_node = Node::new(
ider::uuid(),
source_node_data,
pos_x,
pos_y,
"input".to_string(),
);
let dest_node = Node::new(
ider::uuid(),
dest_node_data,
pos_x + pos_offset,
pos_y + pos_offset,
"output".to_string(),
);
let nodes: Vec<Node> = vec![source_node, dest_node];
let edges = nodes
.windows(2)
.map(|pair| Edge::new(pair[0].id.clone(), pair[1].id.clone()))
.collect::<Vec<_>>();
let pl_id = ider::uuid();
let name = format!("Migrated-{pl_id}");
let description = "This pipeline was generated from previous found prior to OpenObserve v0.13.1. Please check and confirm before enabling it manually".to_string();
let pipeline = Pipeline {
id: pl_id,
version: 0,
enabled: false,
org: old_derived_stream.source.org_id.to_string(),
name,
description,
source: pipeline_source,
nodes,
edges,
};
new_pipeline_by_source.insert(
StreamParams::new(
&old_derived_stream.source.org_id,
&old_derived_stream.name,
old_derived_stream.source.stream_type,
),
pipeline,
);
}
}
// scenario 2: with functions or routing -> realtime
let source_params = StreamParams::new(key_col[1], key_col[3], StreamType::from(key_col[2]));
if stream_funcs.contains_key(&source_params) || old_pipe.routing.is_some() {
let (mut pos_x, mut pos_y): (f32, f32) = (50.0, 50.0);
let pos_offset: f32 = 200.0;
let new_pipeline = new_pipeline_by_source.entry(source_params.clone()).or_insert_with(|| {
let pipeline_source = PipelineSource::Realtime(source_params.clone());
let source_node = Node::new(
ider::uuid(),
NodeData::Stream(source_params.clone()),
pos_x,
pos_y,
"input".to_string(),
);
let pl_id = ider::uuid();
let name = format!("Migrated-{pl_id}");
let description = "This pipeline was generated from previous found prior to OpenObserve v0.12.2. Please check and confirm before enabling it manually".to_string();
Pipeline {
id: pl_id,
version: 0,
enabled: false,
org: key_col[1].to_string(),
name,
description,
source: pipeline_source,
nodes: vec![source_node],
edges: vec![],
}
});
let source_node_id = new_pipeline.nodes[0].id.clone();
if let Some(mut func_params) = stream_funcs.remove(&source_params) {
let dest_node = Node::new(
ider::uuid(),
NodeData::Stream(source_params.clone()),
pos_x,
pos_y + (pos_offset * (func_params.len() + 1) as f32),
"output".to_string(),
);
func_params.sort_by(|a, b| a.0.cmp(&b.0));
for (idx, (_, func_param)) in func_params.into_iter().enumerate() {
let func_node_data = NodeData::Function(func_param);
let func_node = Node::new(
ider::uuid(),
func_node_data,
pos_x,
pos_y + (pos_offset * (idx + 1) as f32),
"default".to_string(),
);
let new_edge = Edge::new(
new_pipeline.nodes.last().unwrap().id.clone(),
func_node.id.clone(),
);
new_pipeline.edges.push(new_edge);
new_pipeline.nodes.push(func_node);
}
let new_edge = Edge::new(
new_pipeline.nodes.last().unwrap().id.clone(),
dest_node.id.clone(),
);
new_pipeline.edges.push(new_edge);
new_pipeline.nodes.push(dest_node);
}
if let Some(routings) = old_pipe.routing {
pos_x += pos_offset;
for (dest_stream, routing_conditions) in routings {
pos_y += pos_offset;
let condition_node = Node::new(
ider::uuid(),
NodeData::Condition(ConditionParams {
conditions: routing_conditions,
}),
pos_x,
pos_y,
"default".to_string(),
);
pos_y += pos_offset;
let dest_node = Node::new(
ider::uuid(),
NodeData::Stream(StreamParams::new(
key_col[1],
&dest_stream,
StreamType::from(key_col[2]),
)),
pos_x,
pos_y,
"output".to_string(),
);
new_pipeline
.edges
.push(Edge::new(source_node_id.clone(), condition_node.id.clone()));
new_pipeline
.edges
.push(Edge::new(condition_node.id.clone(), dest_node.id.clone()));
new_pipeline.nodes.push(condition_node);
new_pipeline.nodes.push(dest_node);
pos_y += pos_offset;
}
}
}
}
// remaining function stream associations
for (stream_params, mut func_params) in stream_funcs {
func_params.sort_by(|a, b| a.0.cmp(&b.0));
let (pos_x, pos_y): (f32, f32) = (50.0, 50.0);
let pos_offset: f32 = 200.0;
let new_pipeline = new_pipeline_by_source.entry(stream_params.clone()).or_insert_with(|| {
let pipeline_source = PipelineSource::Realtime(stream_params.clone());
let source_node = Node::new(
ider::uuid(),
NodeData::Stream(stream_params.clone()),
pos_x,
pos_y,
"input".to_string(),
);
let pl_id = ider::uuid();
let name = format!("Migrated-{pl_id}");
let description = "This pipeline was generated based on Function x Stream Associations found prior to OpenObserve v0.12.2. Please check the correctness of the pipeline and enabling manually".to_string();
Pipeline {
id: pl_id,
version: 0,
enabled: false,
org: stream_params.org_id.to_string(),
name,
description,
source: pipeline_source,
nodes: vec![source_node],
edges: vec![],
}
});
let dest_node = Node::new(
ider::uuid(),
NodeData::Stream(stream_params.clone()),
pos_x,
pos_y + (pos_offset * (func_params.len() + 1) as f32),
"output".to_string(),
);
for (idx, (_, func_param)) in func_params.into_iter().enumerate() {
let func_node_data = NodeData::Function(func_param);
let func_node = Node::new(
ider::uuid(),
func_node_data,
pos_x,
pos_y + (pos_offset * (idx + 1) as f32),
"default".to_string(),
);
let new_edge = Edge::new(
new_pipeline.nodes.last().unwrap().id.clone(),
func_node.id.clone(),
);
new_pipeline.edges.push(new_edge);
new_pipeline.nodes.push(func_node);
}
let new_edge = Edge::new(
new_pipeline.nodes.last().unwrap().id.clone(),
dest_node.id.clone(),
);
new_pipeline.edges.push(new_edge);
new_pipeline.nodes.push(dest_node);
}
// save generated pipeline
let mut ok_to_remove = true;
for (_, pipeline) in new_pipeline_by_source {
if infra::pipeline::put(&pipeline).await.is_err() {
log::error!(
"[Migration]: Error migrating pipelines to the new pipeline format introduced in v0.12.2. Original data kept."
);
ok_to_remove = false;
continue;
}
}
if ok_to_remove {
// clear the old pipelines from the meta table
if let Err(e) = db
.delete("/pipeline/", true, infra_db::NO_NEED_WATCH, None)
.await
{
log::error!(
"[Migration-Pipeline] error deleting all pipelines from meta table: {}",
e
);
}
// update the functions by removing the stream associations
for (org_id, trans) in func_to_update {
if let Err(e) = crate::service::db::functions::set(&org_id, &trans.name, &trans).await {
log::error!(
"[Migration-Function] error saving updated version of function {}: {}",
trans.name,
e
);
}
}
}
Ok(())
}

View File

@ -14,7 +14,13 @@
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use chrono::Utc;
use config::{meta::stream::StreamType, utils::json};
use config::{
meta::{
alerts::{alert::Alert, destinations::Destination, templates::Template},
stream::StreamType,
},
utils::json,
};
use datafusion::arrow::datatypes::Schema;
use infra::{
db::{self as infra_db, NO_NEED_WATCH},
@ -32,10 +38,7 @@ use o2_enterprise::enterprise::openfga::authorizer::authz::get_ownership_tuple;
use crate::{
common::{
infra::config::VERSION,
meta::{
alerts::{alert::Alert, destinations::Destination, templates::Template},
dashboards::reports::Report,
},
meta::dashboards::reports::Report,
utils::auth::{into_ofga_supported_format, is_ofga_unsupported},
},
service::db,

View File

@ -15,12 +15,14 @@
use std::collections::HashMap;
use config::{GEO_IP_ASN_ENRICHMENT_TABLE, GEO_IP_CITY_ENRICHMENT_TABLE};
use config::{
meta::function::VRLCompilerConfig, GEO_IP_ASN_ENRICHMENT_TABLE, GEO_IP_CITY_ENRICHMENT_TABLE,
};
use vector_enrichment::{Table, TableRegistry};
use crate::common::{
infra::config::{ENRICHMENT_TABLES, GEOIP_ASN_TABLE, GEOIP_CITY_TABLE},
meta::{functions::VRLCompilerConfig, organization::DEFAULT_ORG},
meta::organization::DEFAULT_ORG,
};
pub async fn get_all_transform_keys(org_id: &str) -> Vec<String> {

View File

@ -23,7 +23,10 @@ use actix_web::{http::header::HeaderName, web::Query};
use awc::http::header::HeaderMap;
use config::{
get_config,
meta::{search::SearchEventType, stream::StreamType},
meta::{
search::{SearchEventContext, SearchEventType},
stream::StreamType,
},
};
use opentelemetry::{global, propagation::Extractor, trace::TraceContextExt};
use tracing_opentelemetry::OpenTelemetrySpanExt;
@ -79,6 +82,28 @@ pub(crate) fn get_search_type_from_request(
Ok(event_type)
}
#[inline(always)]
pub(crate) fn get_search_event_context_from_request(
search_event_type: &SearchEventType,
query: &Query<HashMap<String, String>>,
) -> Option<SearchEventContext> {
match search_event_type {
SearchEventType::Dashboards => Some(SearchEventContext::with_dashboard(
query.get("dashboard_id").map(String::from),
query.get("dashboard_name").map(String::from),
query.get("folder_id").map(String::from),
query.get("folder_name").map(String::from),
)),
SearchEventType::Alerts => Some(SearchEventContext::with_alert(
query.get("alert_key").map(String::from),
)),
SearchEventType::Reports => Some(SearchEventContext::with_report(
query.get("report_id").map(String::from),
)),
_ => None,
}
}
/// Index type for a search can be either `parquet` or `fst`. It's only effective when env
/// `ZO_INVERTED_INDEX_STORE_FORMAT` is set as `both`.
/// Otherwise 'index_type' is set by env `ZO_INVERTED_INDEX_SEARCH_FORMAT`, which is also

View File

@ -59,6 +59,7 @@ serde.workspace = true
serde_json.workspace = true
segment.workspace = true
sqlparser.workspace = true
sqlx.workspace = true
svix-ksuid.workspace = true
sysinfo.workspace = true
tokio.workspace = true
@ -67,4 +68,5 @@ tracing-log.workspace = true
tracing-subscriber.workspace = true
urlencoding.workspace = true
utoipa.workspace = true
vrl.workspace = true
zstd.workspace = true

View File

@ -662,7 +662,7 @@ pub struct Common {
pub usage_batch_size: usize,
#[env_config(
name = "ZO_USAGE_PUBLISH_INTERVAL",
default = 600,
default = 60,
help = "duration in seconds after last reporting usage will be published"
)]
// in seconds
@ -1044,18 +1044,6 @@ pub struct Limit {
help = "Maximum size of a single enrichment table in mb"
)]
pub max_enrichment_table_size: usize,
#[env_config(
name = "ZO_USE_UPPER_BOUND_FOR_MAX_TS",
default = false,
help = "use upper bound for max tx"
)]
pub use_upper_bound_for_max_ts: bool,
#[env_config(
name = "ZO_BUFFER_FOR_MAX_TS",
default = 60,
help = "buffer for upper bound in mins"
)]
pub upper_bound_for_max_ts: i64,
#[env_config(name = "ZO_SHORT_URL_RETENTION_DAYS", default = 30)] // days
pub short_url_retention_days: i64,
}
@ -1066,18 +1054,22 @@ pub struct Compact {
pub enabled: bool,
#[env_config(name = "ZO_COMPACT_INTERVAL", default = 60)] // seconds
pub interval: u64,
#[env_config(name = "ZO_COMPACT_OLD_DATA_INTERVAL", default = 3600)] // seconds
pub old_data_interval: u64,
#[env_config(name = "ZO_COMPACT_STRATEGY", default = "file_time")] // file_size, file_time
pub strategy: String,
#[env_config(name = "ZO_COMPACT_LOOKBACK_HOURS", default = 0)] // hours
pub lookback_hours: i64,
#[env_config(name = "ZO_COMPACT_STEP_SECS", default = 3600)] // seconds
pub step_secs: i64,
#[env_config(name = "ZO_COMPACT_SYNC_TO_DB_INTERVAL", default = 600)] // seconds
pub sync_to_db_interval: u64,
#[env_config(name = "ZO_COMPACT_MAX_FILE_SIZE", default = 512)] // MB
pub max_file_size: usize,
#[env_config(name = "ZO_COMPACT_DATA_RETENTION_DAYS", default = 3650)] // days
pub data_retention_days: i64,
#[env_config(name = "ZO_COMPACT_OLD_DATA_MAX_DAYS", default = 7)] // days
pub old_data_max_days: i64,
#[env_config(name = "ZO_COMPACT_OLD_DATA_MIN_RECORDS", default = 100)] // records
pub old_data_min_records: i64,
#[env_config(name = "ZO_COMPACT_OLD_DATA_MIN_FILES", default = 10)] // files
pub old_data_min_files: i64,
#[env_config(name = "ZO_COMPACT_DELETE_FILES_DELAY_HOURS", default = 2)] // hours
pub delete_files_delay_hours: i64,
#[env_config(name = "ZO_COMPACT_BLOCKED_ORGS", default = "")] // use comma to split
@ -1098,7 +1090,7 @@ pub struct Compact {
pub job_run_timeout: i64,
#[env_config(
name = "ZO_COMPACT_JOB_CLEAN_WAIT_TIME",
default = 86400, // 1 day
default = 7200, // 2 hours
help = "Clean the jobs which are finished more than this time"
)]
pub job_clean_wait_time: i64,
@ -1458,6 +1450,11 @@ pub fn init() -> Config {
panic!("disk cache config error: {e}");
}
// check compact config
if let Err(e) = check_compact_config(&mut cfg) {
panic!("compact config error: {e}");
}
// check etcd config
if let Err(e) = check_etcd_config(&mut cfg) {
panic!("etcd config error: {e}");
@ -1561,34 +1558,6 @@ fn check_common_config(cfg: &mut Config) -> Result<(), anyhow::Error> {
));
}
// check compact_max_file_size to MB
cfg.compact.max_file_size *= 1024 * 1024;
if cfg.compact.interval == 0 {
cfg.compact.interval = 60;
}
if cfg.compact.pending_jobs_metric_interval == 0 {
cfg.compact.pending_jobs_metric_interval = 300;
}
// check compact_step_secs, min value is 600s
if cfg.compact.step_secs == 0 {
cfg.compact.step_secs = 3600;
} else if cfg.compact.step_secs <= 600 {
cfg.compact.step_secs = 600;
}
if cfg.compact.data_retention_days > 0 && cfg.compact.data_retention_days < 3 {
return Err(anyhow::anyhow!(
"Data retention is not allowed to be less than 3 days."
));
}
if cfg.compact.delete_files_delay_hours < 1 {
return Err(anyhow::anyhow!(
"Delete files delay is not allowed to be less than 1 hour."
));
}
if cfg.compact.batch_size < 1 {
cfg.compact.batch_size = 100;
}
// If the default scrape interval is less than 5s, raise an error
if cfg.common.default_scrape_interval < 5 {
return Err(anyhow::anyhow!(
@ -1870,6 +1839,48 @@ fn check_disk_cache_config(cfg: &mut Config) -> Result<(), anyhow::Error> {
Ok(())
}
fn check_compact_config(cfg: &mut Config) -> Result<(), anyhow::Error> {
if cfg.compact.data_retention_days > 0 && cfg.compact.data_retention_days < 3 {
return Err(anyhow::anyhow!(
"Data retention is not allowed to be less than 3 days."
));
}
if cfg.compact.interval < 1 {
cfg.compact.interval = 60;
}
// check compact_max_file_size to MB
if cfg.compact.max_file_size < 1 {
cfg.compact.max_file_size = 512;
}
cfg.compact.max_file_size *= 1024 * 1024;
if cfg.compact.delete_files_delay_hours < 1 {
cfg.compact.delete_files_delay_hours = 2;
}
if cfg.compact.old_data_interval < 1 {
cfg.compact.old_data_interval = 3600;
}
if cfg.compact.old_data_max_days < 1 {
cfg.compact.old_data_max_days = 7;
}
if cfg.compact.old_data_min_records < 1 {
cfg.compact.old_data_min_records = 100;
}
if cfg.compact.old_data_min_files < 1 {
cfg.compact.old_data_min_files = 10;
}
if cfg.compact.batch_size < 1 {
cfg.compact.batch_size = 100;
}
if cfg.compact.pending_jobs_metric_interval == 0 {
cfg.compact.pending_jobs_metric_interval = 300;
}
Ok(())
}
fn check_sns_config(cfg: &mut Config) -> Result<(), anyhow::Error> {
// Validate endpoint URL if provided
if !cfg.sns.endpoint.is_empty()

View File

@ -14,12 +14,14 @@
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use chrono::{DateTime, FixedOffset};
use config::meta::stream::StreamType;
use hashbrown::HashMap;
use serde::{Deserialize, Serialize};
use utoipa::ToSchema;
use super::{QueryCondition, TriggerCondition};
use crate::meta::{
alerts::{QueryCondition, TriggerCondition},
stream::StreamType,
};
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema)]
pub struct Alert {

View File

@ -13,12 +13,14 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use config::meta::stream::StreamParams;
use hashbrown::HashMap;
use serde::{Deserialize, Serialize};
use utoipa::ToSchema;
use crate::common::meta::alerts::{QueryCondition, TriggerCondition};
use crate::meta::{
alerts::{QueryCondition, TriggerCondition},
stream::StreamParams,
};
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema, PartialEq)]
pub struct DerivedStreamMeta {

View File

@ -13,10 +13,11 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use config::{meta::search::SearchEventType, utils::json::Value};
use serde::{Deserialize, Serialize};
use utoipa::ToSchema;
use crate::{meta::search::SearchEventType, utils::json::Value};
pub mod alert;
pub mod derived_streams;
pub mod destinations;

View File

@ -216,3 +216,9 @@ pub fn get_internal_grpc_token() -> String {
cfg.grpc.internal_grpc_token.clone()
}
}
// CompactionJobType is used to distinguish between current and historical compaction jobs.
pub enum CompactionJobType {
Current,
Historical,
}

View File

@ -13,7 +13,6 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use config::meta::stream::StreamType;
use serde::{Deserialize, Serialize};
use utoipa::ToSchema;
use vrl::{
@ -21,6 +20,8 @@ use vrl::{
prelude::Function,
};
use crate::meta::stream::StreamType;
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema)]
#[serde(rename_all = "camelCase")]
pub struct Transform {
@ -53,52 +54,6 @@ pub struct StreamOrder {
pub apply_before_flattening: bool,
}
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema)]
pub struct StreamTransform {
#[serde(flatten)]
pub transform: Transform,
#[serde(default)]
pub stream: String,
#[serde(default)]
pub order: u8,
#[serde(default)]
pub stream_type: StreamType,
#[serde(default)]
pub is_removed: bool,
#[serde(default)]
#[serde(rename = "applyBeforeFlattening")]
pub apply_before_flattening: bool,
}
impl PartialEq for StreamTransform {
fn eq(&self, other: &Self) -> bool {
self.stream == other.stream
&& self.transform.name == other.transform.name
&& self.stream_type == other.stream_type
}
}
impl Transform {
pub fn to_stream_transform(&self) -> Vec<StreamTransform> {
let mut ret: Vec<StreamTransform> = vec![];
if let Some(streams) = &self.streams {
let mut func = self.clone();
func.streams = None;
for stream in streams {
ret.push(StreamTransform {
transform: func.clone(),
stream: stream.stream.clone(),
order: stream.order,
stream_type: stream.stream_type,
is_removed: stream.is_removed,
apply_before_flattening: stream.apply_before_flattening,
})
}
}
ret
}
}
impl PartialEq for Transform {
fn eq(&self, other: &Self) -> bool {
self.name == other.name && self.function == other.function && self.params == other.params
@ -115,11 +70,6 @@ pub struct FunctionList {
pub list: Vec<Transform>,
}
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema, PartialEq)]
pub struct StreamFunctionsList {
pub list: Vec<StreamTransform>,
}
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct VRLConfig {
pub runtime: VrlRuntime,
@ -141,6 +91,7 @@ pub struct VRLRuntimeConfig {
pub fields: Vec<String>,
}
#[derive(Clone, Debug)]
pub struct VRLResultResolver {
pub program: Program,
pub fields: Vec<String>,
@ -148,9 +99,8 @@ pub struct VRLResultResolver {
#[cfg(test)]
mod tests {
use config::utils::json;
use super::*;
use crate::utils::json;
#[test]
fn test_functions() {

View File

@ -13,11 +13,14 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
pub mod alerts;
pub mod bitvec;
pub mod cluster;
pub mod function;
pub mod inverted_index;
pub mod logger;
pub mod meta_store;
pub mod pipeline;
pub mod puffin;
pub mod search;
pub mod short_url;

View File

@ -0,0 +1,238 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use serde::{Deserialize, Serialize};
use utoipa::ToSchema;
use crate::meta::{
alerts::{QueryCondition, TriggerCondition},
stream::{RoutingCondition, StreamParams, StreamType},
};
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
#[serde(tag = "source_type")]
#[serde(rename_all = "snake_case")]
pub enum PipelineSource {
Realtime(StreamParams),
Scheduled(DerivedStream),
}
impl Default for PipelineSource {
fn default() -> Self {
Self::Realtime(StreamParams::default())
}
}
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Default)]
#[serde(rename_all = "snake_case")]
#[serde(default)]
pub struct DerivedStream {
#[serde(default)]
pub org_id: String,
pub stream_type: StreamType,
#[serde(default)]
pub query_condition: QueryCondition,
#[serde(default)]
pub trigger_condition: TriggerCondition, // Frequency type only supports minutes
/// Timezone offset in minutes.
/// The negative secs means the Western Hemisphere
#[serde(default)]
pub tz_offset: i32,
}
#[derive(Debug, Clone, Serialize, Deserialize)]
#[serde(rename_all = "snake_case")]
pub struct Node {
pub id: String,
pub data: NodeData,
position: Position,
io_type: String,
#[serde(skip_serializing_if = "Option::is_none")]
style: Option<NodeStyle>,
}
impl PartialEq for Node {
fn eq(&self, other: &Self) -> bool {
self.id == other.id && self.data == other.data && self.position == other.position
}
}
impl Node {
pub fn new(id: String, data: NodeData, pos_x: f32, pos_y: f32, io_type: String) -> Self {
Self {
id,
data,
position: Position { x: pos_x, y: pos_y },
io_type,
style: None,
}
}
pub fn get_node_data(&self) -> NodeData {
self.data.clone()
}
pub fn get_node_id(&self) -> String {
self.id.clone()
}
pub fn is_function_node(&self) -> bool {
matches!(&self.data, NodeData::Function(_))
}
}
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub struct Edge {
pub id: String,
pub source: String,
pub target: String,
}
impl Edge {
pub fn new(source: String, target: String) -> Self {
let id = format!("e{source}-{target}");
Self { id, source, target }
}
}
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
#[serde(tag = "node_type")]
#[serde(rename_all = "snake_case")]
pub enum NodeData {
Stream(StreamParams),
Query(DerivedStream),
Function(FunctionParams),
Condition(ConditionParams),
}
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, ToSchema)]
#[serde(rename_all = "snake_case")]
pub struct FunctionParams {
#[serde(default)]
pub name: String,
#[serde(default)]
pub after_flatten: bool,
#[serde(default)]
pub num_args: u8,
}
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub struct ConditionParams {
pub conditions: Vec<RoutingCondition>,
}
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
struct Position {
x: f32,
y: f32,
}
#[derive(Debug, Clone, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
struct NodeStyle {
background_color: Option<String>,
}
#[cfg(test)]
mod tests {
use super::*;
use crate::{meta::stream::StreamType, utils::json};
#[test]
fn test_pipeline_source_serialization() {
let data = json::json!({
"source_type": "realtime",
"org_id": "default",
"stream_name": "default",
"stream_type": "logs"
});
let from_json: PipelineSource = json::from_value(data).unwrap();
let stream_params = StreamParams {
org_id: "default".into(),
stream_name: "default".into(),
stream_type: StreamType::Logs,
};
let source = PipelineSource::Realtime(stream_params);
assert_eq!(from_json, source);
}
#[test]
fn test_pipeline_empty_source_serialization() {
let data = json::json!({
"source_type": "scheduled",
});
let from_json: PipelineSource = json::from_value(data).unwrap();
println!("result: {:?}", from_json);
let stream_params = DerivedStream::default();
let source = PipelineSource::Scheduled(stream_params);
assert_eq!(from_json, source);
}
#[test]
fn test_node_data_serialization() {
let data = json::json!({
"node_type": "stream",
"org_id": "default",
"stream_name": "default",
"stream_type": "logs"
});
let node: NodeData = json::from_value(data).unwrap();
let node_data = NodeData::Stream(StreamParams {
org_id: "default".into(),
stream_name: "default".into(),
stream_type: StreamType::Logs,
});
assert_eq!(node_data, node);
}
#[test]
fn test_function_node_serialization() {
let func = FunctionParams {
// vrl_script: "vrl_script".to_string(),
name: "func".to_string(),
after_flatten: false,
// params: "row".to_string(),
num_args: 0,
};
let func_node = NodeData::Function(func);
let payload = json::json!({
"node_type": "function",
"name": "func",
"after_flatten": false,
});
let node_data: NodeData = json::from_value(payload).unwrap();
assert_eq!(func_node, node_data);
}
#[test]
fn test_condition_node_serialization() {
let payload = json::json!({
"node_type": "condition", // required
"conditions": [ // required
{
"column": "body",
"operator": ">=",
"value": {
"key": "value"
},
"ignore_case": false // optional
}
]
});
let node_data = json::from_value::<NodeData>(payload);
assert!(node_data.is_ok());
}
}

View File

@ -0,0 +1,486 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use std::collections::{HashMap, HashSet};
use anyhow::{anyhow, Result};
use components::{DerivedStream, Edge, Node, NodeData, PipelineSource};
use serde::{Deserialize, Serialize};
use sqlx::{Decode, Error, FromRow, Row, Type};
use utoipa::ToSchema;
use crate::{
meta::{
function::VRLResultResolver,
stream::{StreamParams, StreamType},
},
utils::json,
};
pub mod components;
// (pipeline, node_map, graph, vrl_map)
pub type PipelineExecDFS = (
Pipeline,
HashMap<String, NodeData>,
HashMap<String, Vec<String>>,
HashMap<String, VRLResultResolver>,
);
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, ToSchema)]
pub struct Pipeline {
#[serde(default)]
#[serde(rename = "pipeline_id")]
pub id: String,
#[serde(default)]
pub version: i32,
#[serde(default = "default_status")]
pub enabled: bool,
#[serde(default)]
pub org: String, // org this pipeline belongs to. diff from source stream org_id
pub name: String,
#[serde(default)]
pub description: String,
#[serde(default)]
pub source: PipelineSource,
pub nodes: Vec<Node>,
pub edges: Vec<Edge>,
}
impl Pipeline {
pub fn get_cache_key(&self) -> String {
match &self.source {
PipelineSource::Realtime(stream_params) => {
format!("{}/{}", self.org, stream_params)
}
PipelineSource::Scheduled(_) => {
format!("{}/{}", self.org, self.id)
}
}
}
pub fn get_derived_stream(&self) -> Option<DerivedStream> {
match &self.source {
PipelineSource::Scheduled(derived_stream) => Some(derived_stream.to_owned()),
_ => None,
}
}
/// Verifies the pipeline is valid by:
/// 1. non-empty nodes list
/// 2. non-empty edges list
/// 3. 1st node in nodes list is either StreamNode or QueryNode
/// 4. non-empty `conditions` in all ConditionNode nodes in nodes list
/// 5. every node is reachable
/// 6. all leaf nodes are of type StreamNode
/// 7. In the same branch, unchecked `after_flattened` FunctionNode can't follow checked
/// `after_flattened` checked FunctionNode
///
/// If all satisfies, populates the [Pipeline::source] with the first node in nodes list
pub fn validate(&mut self) -> Result<()> {
// ck 1 & 2
match (self.nodes.is_empty(), self.edges.is_empty()) {
(true, true) | (true, false) => {
return Err(anyhow!(
"Empty pipeline. Please add Source/Destination nodes, or any applicable Transform Nodes"
));
}
(false, true) => {
return Err(anyhow!(
"Please connect all Nodes to complete pipeline creation"
));
}
_ => {}
};
// ck 3
match self.nodes.first().unwrap().get_node_data() {
NodeData::Stream(stream_params) => {
self.source = PipelineSource::Realtime(stream_params);
}
NodeData::Query(derived_stream) => {
if derived_stream.trigger_condition.period == 0 {
return Err(anyhow!(
"DerivedStream source's TriggerCondition period missing or is 0"
));
}
self.source = PipelineSource::Scheduled(derived_stream);
}
_ => return Err(anyhow!("Source must be either a StreamNode or QueryNode")),
};
// ck 4
if self.nodes.iter().any(|node| {
matches!(node.get_node_data(), NodeData::Condition(condition_params) if condition_params.conditions.is_empty())
}) {
return Err(anyhow!("ConditionNode must have non-empty conditions"));
}
// ck 5
if self.edges.len() < self.nodes.len() - 1 {
return Err(anyhow!(
"Insufficient number of edges to connect all nodes. Need at least {} for {} nodes, but got {}.",
self.nodes.len() - 1,
self.nodes.len(),
self.edges.len()
));
}
// build adjacency list for ck 6 & 7
let source_node_id = self.nodes[0].id.as_str();
let node_map = self.get_node_map();
let adjacency_list = self.build_adjacency_list(&node_map)?;
let mut visited = HashSet::new();
dfs_traversal_check(
source_node_id,
&adjacency_list,
&node_map,
false,
&mut visited,
)?;
Ok(())
}
/// Converts pipeline's node list to a map for quick lookup
pub fn get_node_map(&self) -> HashMap<String, NodeData> {
self.nodes
.iter()
.map(|node| (node.get_node_id(), node.get_node_data()))
.collect()
}
/// Builds the graph representation of this pipeline's nodes structure.
///
/// Used for pipeline validation and execution.
pub fn build_adjacency_list(
&self,
node_map: &HashMap<String, NodeData>,
) -> Result<HashMap<String, Vec<String>>> {
let mut adjacency_list = HashMap::new();
for (idx, edge) in self.edges.iter().enumerate() {
if !node_map.contains_key(&edge.source) {
return Err(anyhow!("Edge #{idx}'s source node not found in nodes list"));
}
if !node_map.contains_key(&edge.target) {
return Err(anyhow!("Edge #{idx}'s target node not found in nodes list"));
}
adjacency_list
.entry(edge.source.clone())
.or_insert_with(Vec::new)
.push(edge.target.clone());
}
Ok(adjacency_list)
}
/// Finds all the destination streams in the pipeline.
pub fn get_all_destination_streams(
&self,
node_map: &HashMap<String, NodeData>,
graph: &HashMap<String, Vec<String>>,
) -> Vec<StreamParams> {
node_map
.iter()
.filter_map(|(id, node_data)| {
if !graph.contains_key(id) {
if let NodeData::Stream(stream_params) = node_data {
Some(stream_params.clone())
} else {
None
}
} else {
None
}
})
.collect()
}
/// Returns the number of functions nodes in this pipeline.
///
/// Used for usage report.
pub fn num_of_func(&self) -> usize {
self.nodes
.iter()
.filter(|node| node.is_function_node())
.count()
}
}
impl<'r, R: Row> FromRow<'r, R> for Pipeline
where
&'r ::std::primitive::str: ::sqlx::ColumnIndex<R>,
String: Type<R::Database> + Decode<'r, R::Database>,
i32: Type<R::Database> + Decode<'r, R::Database>,
bool: Type<R::Database> + Decode<'r, R::Database>,
{
fn from_row(row: &'r R) -> Result<Self, Error> {
let id: String = row.try_get("id")?;
let version: i32 = row.try_get("version")?;
let enabled: bool = row.try_get("enabled")?;
let org: String = row.try_get("org")?;
let name: String = row.try_get("name")?;
let description: String = row.try_get("description")?;
let source_type: String = row.try_get("source_type")?;
let source = match source_type.as_str() {
"realtime" => {
let stream_org: String = row.try_get("stream_org")?;
let stream_name: String = row.try_get("stream_name")?;
let stream_type: String = row.try_get("stream_type")?;
let stream_params = StreamParams::new(
&stream_org,
&stream_name,
StreamType::from(stream_type.as_str()),
);
PipelineSource::Realtime(stream_params)
}
"scheduled" => {
let derived_stream_raw: String = row.try_get("derived_stream")?;
let derived_stream: DerivedStream = json::from_str(&derived_stream_raw)
.expect("Deserializing DerivedStream from ROW error");
PipelineSource::Scheduled(derived_stream)
}
_ => return Err(sqlx::Error::ColumnNotFound("Invalid source type".into())),
};
let (nodes, edges) = {
let nodes_raw: String = row.try_get("nodes")?;
let edges_raw: String = row.try_get("edges")?;
(
json::from_str(&nodes_raw).expect("Deserializing Nodes from ROW error"),
json::from_str(&edges_raw).expect("Deserializing Edges from ROW error"),
)
};
Ok(Pipeline {
id,
version,
enabled,
org,
name,
description,
source,
nodes,
edges,
})
}
}
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema)]
pub struct PipelineList {
pub list: Vec<Pipeline>,
}
/// DFS traversal to check:
/// 1. all leaf nodes are of StreamNode
/// 2. No `After Flattened` unchecked FunctionNode follows `After Flatten` checked FunctionNode in
/// the same branch
///
/// `graph` and `node_map` already validated for unwrap
fn dfs_traversal_check(
current_id: &str,
graph: &HashMap<String, Vec<String>>,
node_map: &HashMap<String, NodeData>,
mut flattened: bool,
visited: &mut HashSet<String>,
) -> Result<()> {
if visited.contains(current_id) {
return Err(anyhow!("Cyclical pipeline detected."));
}
visited.insert(current_id.to_string());
// Check if the current node is a leaf node
if !graph.contains_key(current_id) {
// Ensure leaf nodes are Stream nodes
if let Some(node_data) = node_map.get(current_id) {
if !matches!(node_data, NodeData::Stream(_)) {
return Err(anyhow!("All leaf nodes must be StreamNode"));
}
} else {
return Err(anyhow!("Node with id {} not found in node_map", current_id));
}
visited.remove(current_id);
return Ok(());
}
for next_node_id in graph.get(current_id).unwrap() {
if let NodeData::Function(func_params) = &node_map.get(next_node_id).unwrap() {
if flattened && !func_params.after_flatten {
return Err(anyhow!(
"After Flatten must be checked if a previous FunctionNode already checked it in the same branch."
));
}
flattened |= func_params.after_flatten;
};
dfs_traversal_check(next_node_id, graph, node_map, flattened, visited)?;
}
visited.remove(current_id);
Ok(())
}
fn default_status() -> bool {
true
}
#[cfg(test)]
mod tests {
use super::*;
use crate::utils::json;
#[test]
fn test_stream_pipeline_serialization() {
let payload = json::json!(
{
"pipeline_id": "uuid",
"name": "pipeline test",
"description": "with or without this field",
"org": "prod",
"nodes": [
{
"id": "1",
"data": {
"node_type": "stream",
"org_id": "default",
"stream_name": "default",
"stream_type": "logs"
},
"position": {
"x": 100,
"y": 100
},
"style": {
"backgroundColor": "#f0f0f0"
},
"io_type": "input",
},
{
"id": "2",
"data": {
"node_type": "stream",
"org_id": "default",
"stream_name": "default",
"stream_type": "logs"
},
"io_type": "input",
"position": {
"x": 300,
"y": 100
}
}
],
"edges": [
{
"id": "e1-2",
"source": "1",
"target": "2"
}
]
}
);
let from_value = json::from_value::<Pipeline>(payload);
assert!(from_value.is_ok());
let mut pl = from_value.unwrap();
assert!(pl.build_adjacency_list(&pl.get_node_map()).is_ok());
assert!(pl.validate().is_ok());
let nodes = json::to_string(&pl.nodes);
assert!(nodes.is_ok());
let nodes = nodes.unwrap();
let new_nodes = json::from_str::<Option<Vec<Node>>>(&nodes);
assert!(new_nodes.is_ok());
}
#[test]
fn test_query_pipeline_serialization() {
let payload = json::json!(
{
"pipeline_id": "uuid",
"name": "pipeline test",
"description": "with or without this field",
"org": "prod",
"nodes": [
{
"id": "1",
"data": {
"node_type": "query",
"org_id": "default",
"stream_type": "logs",
"query_condition": {
"type": "sql",
"conditions": null,
"sql": "SELECT min(_timestamp) min_ts,max(_timestamp) max_ts, count(_timestamp) as counts FROM \"default1\" ",
"promql": null,
"promql_condition": null,
"aggregation": null,
"vrl_function": null,
"search_event_type": "DerivedStream"
},
"trigger_condition": {
"period": 60,
"operator": "=",
"threshold": 0,
"frequency": 15,
"cron": "10 56 * * * *",
"frequency_type": "cron",
"silence": 0
},
"tz_offset": 0
},
"position": {
"x": 100,
"y": 100
},
"io_type": "input",
"style": {
"backgroundColor": "#f0f0f0"
}
},
{
"id": "2",
"data": {
"node_type": "stream",
"org_id": "default",
"stream_name": "default",
"stream_type": "logs"
},
"io_type": "output",
"position": {
"x": 300,
"y": 100
},
}
],
"edges": [
{
"id": "e1-2",
"source": "1",
"target": "2"
}
]
}
);
let from_value = json::from_value::<Pipeline>(payload);
assert!(from_value.is_ok());
let mut pl = from_value.unwrap();
assert!(pl.build_adjacency_list(&pl.get_node_map()).is_ok());
assert!(pl.validate().is_ok());
let nodes = json::to_string(&pl.nodes);
assert!(nodes.is_ok());
let nodes = nodes.unwrap();
let new_nodes = json::from_str::<Option<Vec<Node>>>(&nodes);
assert!(new_nodes.is_ok());
}
}

View File

@ -53,8 +53,12 @@ pub struct Request {
pub clusters: Vec<String>, // default query all clusters, local: only query local cluster
#[serde(default)]
pub timeout: i64,
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub search_type: Option<SearchEventType>,
#[serde(default, flatten)]
#[serde(skip_serializing_if = "Option::is_none")]
pub search_event_context: Option<SearchEventContext>,
#[serde(default)]
pub index_type: String,
}
@ -434,6 +438,7 @@ impl SearchHistoryRequest {
clusters: Vec::new(),
timeout: 0,
search_type: Some(SearchEventType::Other),
search_event_context: None,
index_type: "".to_string(),
};
Ok(search_req)
@ -747,6 +752,70 @@ impl FromStr for SearchEventType {
}
}
#[derive(Clone, Debug, PartialEq, Default, Serialize, Deserialize, ToSchema)]
#[serde(rename_all = "snake_case")]
pub struct SearchEventContext {
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub alert_key: Option<String>,
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub derived_stream_key: Option<String>,
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub report_key: Option<String>,
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub dashboard_id: Option<String>,
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub dashboard_name: Option<String>,
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub dashboard_folder_id: Option<String>,
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub dashboard_folder_name: Option<String>,
}
impl SearchEventContext {
pub fn with_alert(alert_key: Option<String>) -> Self {
Self {
alert_key,
..Default::default()
}
}
pub fn with_derived_stream(derived_stream_key: Option<String>) -> Self {
Self {
derived_stream_key,
..Default::default()
}
}
pub fn with_report(report_key: Option<String>) -> Self {
Self {
report_key,
..Default::default()
}
}
pub fn with_dashboard(
dashboard_id: Option<String>,
dashboard_name: Option<String>,
dashboard_folder_id: Option<String>,
dashboard_folder_name: Option<String>,
) -> Self {
Self {
dashboard_id,
dashboard_name,
dashboard_folder_id,
dashboard_folder_name,
..Default::default()
}
}
}
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema)]
pub struct MultiSearchPartitionRequest {
pub sql: Vec<String>,
@ -814,7 +883,12 @@ pub struct MultiStreamRequest {
pub regions: Vec<String>, // default query all regions, local: only query local region clusters
#[serde(default)]
pub clusters: Vec<String>, // default query all clusters, local: only query local cluster
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub search_type: Option<SearchEventType>,
#[serde(default, flatten)]
#[serde(skip_serializing_if = "Option::is_none")]
pub search_event_context: Option<SearchEventContext>,
#[serde(default)]
pub index_type: String, // parquet(default) or fst
#[serde(default)]
@ -886,6 +960,7 @@ impl MultiStreamRequest {
encoding: self.encoding,
timeout: self.timeout,
search_type: self.search_type,
search_event_context: self.search_event_context.clone(),
index_type: self.index_type.clone(),
});
}
@ -971,6 +1046,7 @@ mod tests {
clusters: vec![],
timeout: 0,
search_type: None,
search_event_context: None,
index_type: "".to_string(),
};
@ -1076,7 +1152,10 @@ mod search_history_utils {
let query = SearchHistoryQueryBuilder::new()
.with_org_id(&Some("org123".to_string()))
.build(SEARCH_STREAM_NAME);
assert_eq!(query, "SELECT * FROM usage WHERE event='Search' AND org_id = 'org123'");
assert_eq!(
query,
"SELECT * FROM usage WHERE event='Search' AND org_id = 'org123'"
);
}
#[test]

View File

@ -63,6 +63,18 @@ impl StreamType {
StreamType::Logs | StreamType::Metrics | StreamType::Traces
)
}
pub fn as_str(&self) -> &str {
match self {
StreamType::Logs => "logs",
StreamType::Metrics => "metrics",
StreamType::Traces => "traces",
StreamType::EnrichmentTables => "enrichment_tables",
StreamType::Filelist => "file_list",
StreamType::Metadata => "metadata",
StreamType::Index => "index",
}
}
}
impl From<&str> for StreamType {
@ -108,6 +120,53 @@ impl std::fmt::Display for StreamType {
}
}
#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash)]
#[serde(default)]
pub struct StreamParams {
pub org_id: faststr::FastStr,
pub stream_name: faststr::FastStr,
pub stream_type: StreamType,
}
impl Default for StreamParams {
fn default() -> Self {
Self {
org_id: String::default().into(),
stream_name: String::default().into(),
stream_type: StreamType::default(),
}
}
}
impl std::fmt::Display for StreamParams {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(
f,
"{}/{}/{}",
self.org_id, self.stream_name, self.stream_type
)
}
}
impl StreamParams {
pub fn new(org_id: &str, stream_name: &str, stream_type: StreamType) -> Self {
Self {
org_id: org_id.to_string().into(),
stream_name: stream_name.to_string().into(),
stream_type,
}
}
pub fn is_valid(&self) -> bool {
!(self.org_id.is_empty() || self.stream_name.is_empty())
}
}
#[derive(Clone, Debug, Serialize, Deserialize, ToSchema)]
pub struct ListStreamParams {
pub list: Vec<StreamParams>,
}
#[derive(Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize)]
pub struct FileKey {
pub key: String,
@ -769,11 +828,6 @@ pub struct PartitioningDetails {
pub partition_time_level: Option<PartitionTimeLevel>,
}
pub struct Routing {
pub destination: String,
pub routing: Vec<RoutingCondition>,
}
// Code Duplicated from alerts
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, ToSchema)]
pub struct RoutingCondition {
@ -786,7 +840,7 @@ pub struct RoutingCondition {
}
// Code Duplicated from alerts
impl RoutingCondition {
pub async fn evaluate(&self, row: &Map<String, Value>) -> bool {
pub fn evaluate(&self, row: &Map<String, Value>) -> bool {
let val = match row.get(&self.column) {
Some(val) => val,
None => {
@ -890,45 +944,6 @@ impl std::fmt::Display for Operator {
}
}
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct StreamParams {
pub org_id: faststr::FastStr,
pub stream_name: faststr::FastStr,
pub stream_type: StreamType,
}
impl PartialEq for StreamParams {
fn eq(&self, other: &Self) -> bool {
self.org_id == other.org_id
&& self.stream_name == other.stream_name
&& self.stream_type == other.stream_type
}
}
impl Default for StreamParams {
fn default() -> Self {
Self {
org_id: String::default().into(),
stream_name: String::default().into(),
stream_type: StreamType::default(),
}
}
}
impl StreamParams {
pub fn new(org_id: &str, stream_name: &str, stream_type: StreamType) -> Self {
Self {
org_id: org_id.to_string().into(),
stream_name: stream_name.to_string().into(),
stream_type,
}
}
pub fn is_valid(&self) -> bool {
!(self.org_id.is_empty() || self.stream_name.is_empty())
}
}
#[cfg(test)]
mod tests {
use super::*;
@ -1010,8 +1025,15 @@ mod tests {
#[test]
fn test_stream_params() {
let params = StreamParams::new("org_id", "stream_name", StreamType::Logs);
let param2 = StreamParams::new("org_id", "stream_name", StreamType::Logs);
let param3 = StreamParams::new("org_id", "stream_name", StreamType::Index);
assert_eq!(params.org_id, "org_id");
assert_eq!(params.stream_name, "stream_name");
assert_eq!(params.stream_type, StreamType::Logs);
let mut map = HashMap::new();
map.insert(params, 1);
map.insert(param2, 2);
map.insert(param3, 2);
assert_eq!(map.len(), 2);
}
}

View File

@ -15,7 +15,7 @@
use serde::{Deserialize, Serialize};
use super::search::SearchEventType;
use super::search::{SearchEventContext, SearchEventType};
use crate::{
meta::stream::{FileMeta, StreamType},
SIZE_IN_MB,
@ -99,6 +99,9 @@ pub struct UsageData {
pub max_ts: Option<i64>,
#[serde(skip_serializing_if = "Option::is_none")]
pub search_type: Option<SearchEventType>,
#[serde(default, flatten)]
#[serde(skip_serializing_if = "Option::is_none")]
pub search_event_context: Option<SearchEventContext>,
#[serde(skip_serializing_if = "Option::is_none")]
pub took_wait_in_queue: Option<usize>,
#[serde(skip_serializing_if = "Option::is_none")]
@ -265,6 +268,9 @@ pub struct RequestStats {
pub user_email: Option<String>,
#[serde(skip_serializing_if = "Option::is_none")]
pub search_type: Option<SearchEventType>,
#[serde(default, flatten)]
#[serde(skip_serializing_if = "Option::is_none")]
pub search_event_context: Option<SearchEventContext>,
#[serde(skip_serializing_if = "Option::is_none")]
pub trace_id: Option<String>,
#[serde(skip_serializing_if = "Option::is_none")]
@ -290,6 +296,7 @@ impl Default for RequestStats {
max_ts: None,
user_email: None,
search_type: None,
search_event_context: None,
trace_id: None,
took_wait_in_queue: None,
result_cache_ratio: None,
@ -313,6 +320,7 @@ impl From<FileMeta> for RequestStats {
max_ts: Some(meta.max_ts),
user_email: None,
search_type: None,
search_event_context: None,
trace_id: None,
took_wait_in_queue: None,
result_cache_ratio: None,

View File

@ -25,7 +25,7 @@ pub fn flatten(to_flatten: Value) -> Result<Value, anyhow::Error> {
/// Flattens the provided JSON object (`current`).
///
/// It will return an error if flattening the object would make two keys to be
/// the same, overwriting a value. It will alre return an error if the JSON
/// the same, overwriting a value. It will also return an error if the JSON
/// value passed it's not an object.
///
/// # Errors

View File

@ -13,7 +13,7 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use chrono::{DateTime, Datelike, NaiveDateTime, TimeZone, Utc};
use chrono::{DateTime, Datelike, Duration, NaiveDateTime, TimeZone, Utc};
use once_cell::sync::Lazy;
use crate::utils::json;
@ -46,6 +46,19 @@ pub fn now_micros() -> i64 {
Utc::now().timestamp_micros()
}
#[inline(always)]
pub fn hour_micros(n: i64) -> i64 {
Duration::try_hours(n).unwrap().num_microseconds().unwrap()
}
#[inline(always)]
pub fn second_micros(n: i64) -> i64 {
Duration::try_seconds(n)
.unwrap()
.num_microseconds()
.unwrap()
}
#[inline(always)]
pub fn parse_i64_to_timestamp_micros(v: i64) -> i64 {
if v == 0 {

View File

@ -29,7 +29,7 @@ use arrow_flight::{
HandshakeRequest, HandshakeResponse, PollInfo, PutResult, SchemaResult, Ticket,
};
use arrow_schema::Schema;
use config::meta::search::ScanStats;
use config::{meta::search::ScanStats, metrics};
use datafusion::{
common::{DataFusionError, Result},
execution::SendableRecordBatchStream,
@ -284,6 +284,16 @@ impl Drop for FlightSenderStream {
self.trace_id,
end
);
// metrics
let time = self.start.elapsed().as_secs_f64();
metrics::GRPC_RESPONSE_TIME
.with_label_values(&["/search/flight/do_get", "200", "", "", ""])
.observe(time);
metrics::GRPC_INCOMING_REQUESTS
.with_label_values(&["/search/flight/do_get", "200", "", "", ""])
.inc();
if let Some(defer) = self.defer.take() {
drop(defer);
} else {

View File

@ -14,7 +14,7 @@
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use actix_web::http::StatusCode;
use config::utils::json;
use config::{metrics, utils::json};
use proto::cluster_rpc::{
ingest_server::Ingest, IngestionRequest, IngestionResponse, IngestionType, StreamType,
};
@ -31,6 +31,7 @@ impl Ingest for Ingester {
&self,
request: Request<IngestionRequest>,
) -> Result<Response<IngestionResponse>, Status> {
let start = std::time::Instant::now();
// let metadata = request.metadata().clone();
let req = request.into_inner();
let org_id = req.org_id;
@ -131,6 +132,16 @@ impl Ingest for Ingester {
message: err.to_string(),
},
};
// metrics
let time = start.elapsed().as_secs_f64();
metrics::GRPC_RESPONSE_TIME
.with_label_values(&["/ingest/inner", "200", "", "", ""])
.observe(time);
metrics::GRPC_INCOMING_REQUESTS
.with_label_values(&["/ingest/inner", "200", "", "", ""])
.inc();
Ok(Response::new(reply))
}
}

View File

@ -13,6 +13,7 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use config::metrics;
use opentelemetry_proto::tonic::collector::logs::v1::{
logs_service_server::LogsService, ExportLogsServiceRequest, ExportLogsServiceResponse,
};
@ -27,7 +28,9 @@ impl LogsService for LogsServer {
&self,
request: tonic::Request<ExportLogsServiceRequest>,
) -> Result<tonic::Response<ExportLogsServiceResponse>, tonic::Status> {
let start = std::time::Instant::now();
let cfg = config::get_config();
let metadata = request.metadata().clone();
let msg = format!(
"Please specify organization id with header key '{}' ",
@ -64,9 +67,20 @@ impl LogsService for LogsServer {
)
.await
{
Ok(_) => Ok(Response::new(ExportLogsServiceResponse {
partial_success: None,
})),
Ok(_) => {
// metrics
let time = start.elapsed().as_secs_f64();
metrics::GRPC_RESPONSE_TIME
.with_label_values(&["/otlp/v1/logs", "200", "", "", ""])
.observe(time);
metrics::GRPC_INCOMING_REQUESTS
.with_label_values(&["/otlp/v1/logs", "200", "", "", ""])
.inc();
Ok(Response::new(ExportLogsServiceResponse {
partial_success: None,
}))
}
Err(e) => Err(Status::internal(e.to_string())),
}
}

View File

@ -13,6 +13,7 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use config::metrics;
use opentelemetry_proto::tonic::collector::metrics::v1::{
metrics_service_server::MetricsService, ExportMetricsServiceRequest,
ExportMetricsServiceResponse,
@ -28,7 +29,9 @@ impl MetricsService for MetricsIngester {
&self,
request: tonic::Request<ExportMetricsServiceRequest>,
) -> Result<tonic::Response<ExportMetricsServiceResponse>, tonic::Status> {
let start = std::time::Instant::now();
let cfg = config::get_config();
let metadata = request.metadata().clone();
let msg = format!(
"Please specify organization id with header key '{}' ",
@ -51,6 +54,14 @@ impl MetricsService for MetricsIngester {
)
.await;
if resp.is_ok() {
// metrics
let time = start.elapsed().as_secs_f64();
metrics::GRPC_RESPONSE_TIME
.with_label_values(&["/otlp/v1/metrics", "200", "", "", ""])
.observe(time);
metrics::GRPC_INCOMING_REQUESTS
.with_label_values(&["/otlp/v1/metrics", "200", "", "", ""])
.inc();
return Ok(Response::new(ExportMetricsServiceResponse {
partial_success: None,
}));

View File

@ -13,6 +13,7 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use config::metrics;
use opentelemetry_proto::tonic::collector::trace::v1::{
trace_service_server::TraceService, ExportTraceServiceRequest, ExportTraceServiceResponse,
};
@ -29,7 +30,9 @@ impl TraceService for TraceServer {
&self,
request: tonic::Request<ExportTraceServiceRequest>,
) -> Result<tonic::Response<ExportTraceServiceResponse>, tonic::Status> {
let start = std::time::Instant::now();
let cfg = config::get_config();
let metadata = request.metadata().clone();
let msg = format!(
"Please specify organization id with header key '{}' ",
@ -59,6 +62,14 @@ impl TraceService for TraceServer {
)
.await;
if resp.is_ok() {
// metrics
let time = start.elapsed().as_secs_f64();
metrics::GRPC_RESPONSE_TIME
.with_label_values(&["/otlp/v1/traces", "200", "", "", ""])
.observe(time);
metrics::GRPC_INCOMING_REQUESTS
.with_label_values(&["/otlp/v1/traces", "200", "", "", ""])
.inc();
return Ok(Response::new(ExportTraceServiceResponse {
partial_success: None,
}));

View File

@ -546,8 +546,10 @@ async fn oo_validator_internal(
path_prefix: &str,
) -> Result<ServiceRequest, (Error, ServiceRequest)> {
if auth_info.auth.starts_with("Basic") {
let decoded = base64::decode(auth_info.auth.strip_prefix("Basic").unwrap().trim())
.expect("Failed to decode base64 string");
let decoded = match base64::decode(auth_info.auth.strip_prefix("Basic").unwrap().trim()) {
Ok(val) => val,
Err(_) => return Err((ErrorUnauthorized("Unauthorized Access"), req)),
};
let (username, password) = match get_user_details(decoded) {
Some(value) => value,
@ -562,14 +564,16 @@ async fn oo_validator_internal(
if chrono::Utc::now().timestamp() - auth_tokens.request_time > auth_tokens.expires_in {
Err((ErrorUnauthorized("Unauthorized Access"), req))
} else {
let decoded = base64::decode(
let decoded = match base64::decode(
auth_tokens
.auth_ext
.strip_prefix("auth_ext")
.unwrap()
.trim(),
)
.expect("Failed to decode base64 string");
) {
Ok(val) => val,
Err(_) => return Err((ErrorUnauthorized("Unauthorized Access"), req)),
};
let (username, password) = match get_user_details(decoded) {
Some(value) => value,
None => return Err((ErrorUnauthorized("Unauthorized Access"), req)),
@ -584,8 +588,10 @@ async fn oo_validator_internal(
#[cfg(feature = "enterprise")]
pub async fn get_user_email_from_auth_str(auth_str: &str) -> Option<String> {
if auth_str.starts_with("Basic") {
let decoded = base64::decode(auth_str.strip_prefix("Basic").unwrap().trim())
.expect("Failed to decode base64 string");
let decoded = match base64::decode(auth_str.strip_prefix("Basic").unwrap().trim()) {
Ok(val) => val,
Err(_) => return None,
};
match get_user_details(decoded) {
Some(value) => Some(value.0),
@ -599,14 +605,16 @@ pub async fn get_user_email_from_auth_str(auth_str: &str) -> Option<String> {
if chrono::Utc::now().timestamp() - auth_tokens.request_time > auth_tokens.expires_in {
None
} else {
let decoded = base64::decode(
let decoded = match base64::decode(
auth_tokens
.auth_ext
.strip_prefix("auth_ext")
.unwrap()
.trim(),
)
.expect("Failed to decode base64 string");
) {
Ok(val) => val,
Err(_) => return None,
};
match get_user_details(decoded) {
Some(value) => Some(value.0),
None => None,
@ -618,9 +626,10 @@ pub async fn get_user_email_from_auth_str(auth_str: &str) -> Option<String> {
}
fn get_user_details(decoded: String) -> Option<(String, String)> {
let credentials = String::from_utf8(decoded.into())
.map_err(|_| ())
.expect("Failed to decode base64 string");
let credentials = match String::from_utf8(decoded.into()).map_err(|_| ()) {
Ok(val) => val,
Err(_) => return None,
};
let parts: Vec<&str> = credentials.splitn(2, ':').collect();
if parts.len() != 2 {
return None;

View File

@ -16,14 +16,11 @@
use std::{collections::HashMap, io::Error};
use actix_web::{delete, get, http, post, put, web, HttpRequest, HttpResponse};
use config::meta::alerts::alert::{Alert, AlertListFilter};
use crate::{
common::{
meta::{
alerts::alert::{Alert, AlertListFilter},
dashboards::datetime_now,
http::HttpResponse as MetaHttpResponse,
},
meta::{dashboards::datetime_now, http::HttpResponse as MetaHttpResponse},
utils::{auth::UserEmail, http::get_stream_type_from_request},
},
service::alerts::alert,

View File

@ -16,11 +16,9 @@
use std::io::Error;
use actix_web::{delete, get, http, post, put, web, HttpRequest, HttpResponse};
use config::meta::alerts::destinations::Destination;
use crate::{
common::meta::{alerts::destinations::Destination, http::HttpResponse as MetaHttpResponse},
service::alerts::destinations,
};
use crate::{common::meta::http::HttpResponse as MetaHttpResponse, service::alerts::destinations};
/// CreateDestination
#[utoipa::path(

View File

@ -16,11 +16,9 @@
use std::io::Error;
use actix_web::{delete, get, http, post, put, web, HttpRequest, HttpResponse};
use config::meta::alerts::templates::Template;
use crate::{
common::meta::{alerts::templates::Template, http::HttpResponse as MetaHttpResponse},
service::alerts::templates,
};
use crate::{common::meta::http::HttpResponse as MetaHttpResponse, service::alerts::templates};
/// CreateTemplate
#[utoipa::path(

View File

@ -13,15 +13,10 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use std::{collections::HashMap, io::Error};
use std::io::Error;
use actix_web::{delete, get, http, post, put, web, HttpRequest, HttpResponse};
use crate::common::{
meta,
meta::functions::{StreamOrder, Transform},
utils::http::get_stream_type_from_request,
};
use actix_web::{delete, get, post, put, web, HttpRequest, HttpResponse};
use config::meta::function::Transform;
/// CreateFunction
#[utoipa::path(
@ -153,128 +148,3 @@ pub async fn update_function(
transform.function = transform.function.trim().to_string();
crate::service::functions::update_function(&org_id, name, transform).await
}
/// ListStreamFunctions
#[utoipa::path(
context_path = "/api",
tag = "Functions",
operation_id = "listStreamFunctions",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
("stream_name" = String, Path, description = "Stream name"),
),
responses(
(status = 200, description = "Success", content_type = "application/json", body = StreamFunctionsList),
)
)]
#[get("/{org_id}/streams/{stream_name}/functions")]
async fn list_stream_functions(
path: web::Path<(String, String)>,
req: HttpRequest,
) -> Result<HttpResponse, Error> {
let (org_id, stream_name) = path.into_inner();
let query = web::Query::<HashMap<String, String>>::from_query(req.query_string()).unwrap();
let stream_type = match get_stream_type_from_request(&query) {
Ok(v) => v.unwrap_or_default(),
Err(e) => {
return Ok(
HttpResponse::BadRequest().json(meta::http::HttpResponse::error(
http::StatusCode::BAD_REQUEST.into(),
e.to_string(),
)),
);
}
};
crate::service::functions::list_stream_functions(&org_id, stream_type, &stream_name).await
}
/// RemoveStreamFunction
#[utoipa::path(
context_path = "/api",
tag = "Functions",
operation_id = "removeStreamFunction",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
("stream_name" = String, Path, description = "Stream name"),
("name" = String, Path, description = "Function name"),
),
responses(
(status = 200, description = "Success", content_type = "application/json", body = HttpResponse),
(status = 404, description = "NotFound", content_type = "application/json", body = HttpResponse),
)
)]
#[delete("/{org_id}/streams/{stream_name}/functions/{name}")]
async fn delete_stream_function(
path: web::Path<(String, String, String)>,
req: HttpRequest,
) -> Result<HttpResponse, Error> {
let (org_id, stream_name, name) = path.into_inner();
let query = web::Query::<HashMap<String, String>>::from_query(req.query_string()).unwrap();
let stream_type = match get_stream_type_from_request(&query) {
Ok(v) => v.unwrap_or_default(),
Err(e) => {
return Ok(
HttpResponse::BadRequest().json(meta::http::HttpResponse::error(
http::StatusCode::BAD_REQUEST.into(),
e.to_string(),
)),
);
}
};
crate::service::functions::delete_stream_function(&org_id, stream_type, &stream_name, &name)
.await
}
/// ApplyFunctionToStream
#[utoipa::path(
context_path = "/api",
tag = "Functions",
operation_id = "applyFunctionToStream",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
("stream_name" = String, Path, description = "Stream name"),
("name" = String, Path, description = "Function name"),
),
request_body(content = StreamOrder, description = "Function data", content_type = "application/json"),
responses(
(status = 200, description = "Success", content_type = "application/json", body = HttpResponse),
(status = 400, description = "Failure", content_type = "application/json", body = HttpResponse),
)
)]
#[put("/{org_id}/streams/{stream_name}/functions/{name}")]
pub async fn add_function_to_stream(
path: web::Path<(String, String, String)>,
stream_order: web::Json<StreamOrder>,
req: HttpRequest,
) -> Result<HttpResponse, Error> {
let (org_id, stream_name, name) = path.into_inner();
let query = web::Query::<HashMap<String, String>>::from_query(req.query_string()).unwrap();
let stream_type = match get_stream_type_from_request(&query) {
Ok(v) => v.unwrap_or_default(),
Err(e) => {
return Ok(
HttpResponse::BadRequest().json(meta::http::HttpResponse::error(
http::StatusCode::BAD_REQUEST.into(),
e.to_string(),
)),
);
}
};
crate::service::functions::add_function_to_stream(
&org_id,
stream_type,
&stream_name,
&name,
stream_order.into_inner(),
)
.await
}

View File

@ -23,7 +23,7 @@ pub mod kv;
pub mod logs;
pub mod metrics;
pub mod organization;
pub mod pipelines;
pub mod pipeline;
pub mod prom;
pub mod rum;
pub mod search;

View File

@ -0,0 +1,208 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use std::io::Error;
use actix_web::{delete, get, post, put, web, HttpRequest, HttpResponse};
use ahash::HashMap;
use config::{ider, meta::pipeline::Pipeline};
/// CreatePipeline
#[utoipa::path(
context_path = "/api",
tag = "Pipeline",
operation_id = "createPipeline",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
),
request_body(content = Pipeline, description = "Pipeline data", content_type = "application/json"),
responses(
(status = 200, description = "Success", content_type = "application/json", body = HttpResponse),
(status = 400, description = "Failure", content_type = "application/json", body = HttpResponse),
)
)]
#[post("/{org_id}/pipelines")]
pub async fn save_pipeline(
path: web::Path<String>,
pipeline: web::Json<Pipeline>,
_req: HttpRequest,
) -> Result<HttpResponse, Error> {
let org_id = path.into_inner();
let mut pipeline = pipeline.into_inner();
pipeline.name = pipeline.name.trim().to_string();
pipeline.org = org_id;
pipeline.id = ider::generate();
crate::service::pipeline::save_pipeline(pipeline).await
}
/// ListPipelines
#[utoipa::path(
context_path = "/api",
tag = "Pipelines",
operation_id = "listPipelines",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
),
responses(
(status = 200, description = "Success", content_type = "application/json", body = PipelineList),
)
)]
#[get("/{org_id}/pipelines")]
async fn list_pipelines(
org_id: web::Path<String>,
_req: HttpRequest,
) -> Result<HttpResponse, Error> {
let mut _permitted = None;
// Get List of allowed objects
#[cfg(feature = "enterprise")]
{
let user_id = _req.headers().get("user_id").unwrap();
match crate::handler::http::auth::validator::list_objects_for_user(
&org_id,
user_id.to_str().unwrap(),
"GET",
"logs",
)
.await
{
Ok(list) => {
_permitted = list;
}
Err(e) => {
return Ok(crate::common::meta::http::HttpResponse::forbidden(
e.to_string(),
));
}
}
// Get List of allowed objects ends
}
crate::service::pipeline::list_pipelines(org_id.into_inner(), _permitted).await
}
/// GetStreamsWithPipeline
#[utoipa::path(
context_path = "/api",
tag = "Pipelines",
operation_id = "getStreamsWithPipeline",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
),
responses(
(status = 200, description = "Success", content_type = "application/json", body = PipelineList),
)
)]
#[get("/{org_id}/pipelines/streams")]
async fn list_streams_with_pipeline(
path: web::Path<String>,
_req: HttpRequest,
) -> Result<HttpResponse, Error> {
let org_id = path.into_inner();
crate::service::pipeline::list_streams_with_pipeline(&org_id).await
}
/// DeletePipeline
#[utoipa::path(
context_path = "/api",
tag = "Pipelines",
operation_id = "deletePipeline",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
("pipeline_id" = String, Path, description = "Pipeline ID"),
),
responses(
(status = 200, description = "Success", content_type = "application/json", body = HttpResponse),
(status = 404, description = "NotFound", content_type = "application/json", body = HttpResponse),
)
)]
#[delete("/{org_id}/pipelines/{pipeline_id}")]
async fn delete_pipeline(
path: web::Path<(String, String)>,
_req: HttpRequest,
) -> Result<HttpResponse, Error> {
let (_org_id, pipeline_id) = path.into_inner();
crate::service::pipeline::delete_pipeline(&pipeline_id).await
}
/// UpdatePipeline
#[utoipa::path(
context_path = "/api",
tag = "Pipelines",
operation_id = "updatePipeline",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
),
request_body(content = Pipeline, description = "Pipeline data", content_type = "application/json"),
responses(
(status = 200, description = "Success", content_type = "application/json", body = HttpResponse),
(status = 400, description = "Failure", content_type = "application/json", body = HttpResponse),
)
)]
#[put("/{org_id}/pipelines")]
pub async fn update_pipeline(
pipeline: web::Json<Pipeline>,
_req: HttpRequest,
) -> Result<HttpResponse, Error> {
let pipeline = pipeline.into_inner();
crate::service::pipeline::update_pipeline(pipeline).await
}
/// EnablePipeline
#[utoipa::path(
context_path = "/api",
tag = "Pipelines",
operation_id = "enablePipeline",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
("pipeline_id" = String, Path, description = "Pipeline ID"),
("value" = bool, Query, description = "Enable or disable pipeline"),
),
responses(
(status = 200, description = "Success", content_type = "application/json", body = HttpResponse),
(status = 404, description = "NotFound", content_type = "application/json", body = HttpResponse),
(status = 500, description = "Failure", content_type = "application/json", body = HttpResponse),
)
)]
#[put("/{org_id}/pipelines/{pipeline_id}/enable")]
pub async fn enable_pipeline(
path: web::Path<(String, String)>,
req: HttpRequest,
) -> Result<HttpResponse, Error> {
let (org_id, pipeline_id) = path.into_inner();
let query = web::Query::<HashMap<String, String>>::from_query(req.query_string()).unwrap();
let enable = match query.get("value") {
Some(v) => v.parse::<bool>().unwrap_or_default(),
None => false,
};
crate::service::pipeline::enable_pipeline(&org_id, &pipeline_id, enable).await
}

View File

@ -1,221 +0,0 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use std::{collections::HashMap, io::Error};
use actix_web::{delete, get, http, post, put, web, HttpRequest, HttpResponse};
use config::{meta::stream::StreamParams, utils::schema::format_stream_name};
use crate::common::{
meta::{self, pipelines::PipeLine},
utils::http::get_stream_type_from_request,
};
/// CreatePipeline
#[utoipa::path(
context_path = "/api",
tag = "Pipelines",
operation_id = "createPipeline",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
),
request_body(content = PipeLine, description = "Pipeline data", content_type = "application/json"),
responses(
(status = 200, description = "Success", content_type = "application/json", body = HttpResponse),
(status = 400, description = "Failure", content_type = "application/json", body = HttpResponse),
)
)]
#[post("/{org_id}/streams/{stream_name}/pipelines")]
pub async fn save_pipeline(
path: web::Path<(String, String)>,
pipeline: web::Json<PipeLine>,
req: HttpRequest,
) -> Result<HttpResponse, Error> {
let (org_id, stream_name) = path.into_inner();
let query = web::Query::<HashMap<String, String>>::from_query(req.query_string()).unwrap();
let mut pipeline = pipeline.into_inner();
pipeline.name = pipeline.name.trim().to_string();
pipeline.stream_name = stream_name;
pipeline.stream_type = match get_stream_type_from_request(&query) {
Ok(v) => v.unwrap_or_default(),
Err(e) => {
return Ok(
HttpResponse::BadRequest().json(meta::http::HttpResponse::error(
http::StatusCode::BAD_REQUEST.into(),
e.to_string(),
)),
);
}
};
if let Some(ref mut routing) = &mut pipeline.routing {
let keys_to_update: Vec<_> = routing.keys().cloned().collect();
for key in keys_to_update {
let value = routing.remove(&key).unwrap();
if value.is_empty() {
return Ok(
HttpResponse::BadRequest().json(meta::http::HttpResponse::error(
http::StatusCode::BAD_REQUEST.into(),
format!("Routing condition for {} is empty", key),
)),
);
}
let formatted_key = format_stream_name(&key);
routing.insert(formatted_key, value);
}
}
crate::service::pipelines::save_pipeline(org_id, pipeline).await
}
/// ListPipelines
#[utoipa::path(
context_path = "/api",
tag = "Pipelines",
operation_id = "listPipelines",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
),
responses(
(status = 200, description = "Success", content_type = "application/json", body = PipelineList),
)
)]
#[get("/{org_id}/pipelines")]
async fn list_pipelines(
org_id: web::Path<String>,
_req: HttpRequest,
) -> Result<HttpResponse, Error> {
let mut _permitted = None;
// Get List of allowed objects
#[cfg(feature = "enterprise")]
{
let user_id = _req.headers().get("user_id").unwrap();
match crate::handler::http::auth::validator::list_objects_for_user(
&org_id,
user_id.to_str().unwrap(),
"GET",
"logs",
)
.await
{
Ok(list) => {
_permitted = list;
}
Err(e) => {
return Ok(crate::common::meta::http::HttpResponse::forbidden(
e.to_string(),
));
}
}
// Get List of allowed objects ends
}
crate::service::pipelines::list_pipelines(org_id.into_inner(), _permitted).await
}
/// DeletePipeline
#[utoipa::path(
context_path = "/api",
tag = "Pipelines",
operation_id = "deletePipeline",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
("name" = String, Path, description = "Pipeline name"),
),
responses(
(status = 200, description = "Success", content_type = "application/json", body = HttpResponse),
(status = 404, description = "NotFound", content_type = "application/json", body = HttpResponse),
)
)]
#[delete("/{org_id}/streams/{stream_name}/pipelines/{name}")]
async fn delete_pipeline(
path: web::Path<(String, String, String)>,
req: HttpRequest,
) -> Result<HttpResponse, Error> {
let (org_id, stream_name, name) = path.into_inner();
let query = web::Query::<HashMap<String, String>>::from_query(req.query_string()).unwrap();
let stream_type = match get_stream_type_from_request(&query) {
Ok(v) => v.unwrap_or_default(),
Err(e) => {
return Ok(crate::common::meta::http::HttpResponse::bad_request(e));
}
};
let pipeline_source = StreamParams::new(&org_id, &stream_name, stream_type);
crate::service::pipelines::delete_pipeline(&name, pipeline_source).await
}
/// UpdatePipeline
#[utoipa::path(
context_path = "/api",
tag = "Pipelines",
operation_id = "updatePipeline",
security(
("Authorization"= [])
),
params(
("org_id" = String, Path, description = "Organization name"),
("name" = String, Path, description = "Pipeline name"),
),
request_body(content = PipeLine, description = "Pipeline data", content_type = "application/json"),
responses(
(status = 200, description = "Success", content_type = "application/json", body = HttpResponse),
(status = 400, description = "Failure", content_type = "application/json", body = HttpResponse),
)
)]
#[put("/{org_id}/streams/{stream_name}/pipelines/{name}")]
pub async fn update_pipeline(
path: web::Path<(String, String, String)>,
pipeline: web::Json<PipeLine>,
req: HttpRequest,
) -> Result<HttpResponse, Error> {
let (org_id, stream_name, name) = path.into_inner();
let query = web::Query::<HashMap<String, String>>::from_query(req.query_string()).unwrap();
let stream_type = match get_stream_type_from_request(&query) {
Ok(v) => v.unwrap_or_default(),
Err(e) => {
return Ok(crate::common::meta::http::HttpResponse::bad_request(e));
}
};
let name = name.trim();
let mut pipeline = pipeline.into_inner();
pipeline.name = name.to_string();
pipeline.stream_type = stream_type;
pipeline.stream_name = stream_name;
if let Some(ref mut routing) = &mut pipeline.routing {
let keys_to_update: Vec<_> = routing.keys().cloned().collect();
for key in keys_to_update {
let value = routing.remove(&key).unwrap();
if value.is_empty() {
return Ok(
HttpResponse::BadRequest().json(meta::http::HttpResponse::error(
http::StatusCode::BAD_REQUEST.into(),
format!("Routing condition for {} is empty", key),
)),
);
}
let formatted_key = format_stream_name(&key);
routing.insert(formatted_key, value);
}
}
crate::service::pipelines::update_pipeline(&org_id, pipeline).await
}

View File

@ -39,7 +39,8 @@ use crate::{
utils::{
functions,
http::{
get_index_type_from_request, get_or_create_trace_id, get_search_type_from_request,
get_index_type_from_request, get_or_create_trace_id,
get_search_event_context_from_request, get_search_type_from_request,
get_stream_type_from_request, get_use_cache_from_request, get_work_group,
},
},
@ -150,6 +151,10 @@ pub async fn search(
Ok(v) => v,
Err(e) => return Ok(MetaHttpResponse::bad_request(e)),
};
req.search_event_context = req
.search_type
.as_ref()
.and_then(|event_type| get_search_event_context_from_request(event_type, &query));
// set index_type
req.index_type = match get_index_type_from_request(&query) {
@ -469,6 +474,7 @@ pub async fn around(
clusters: clusters.clone(),
timeout,
search_type: Some(SearchEventType::UI),
search_event_context: None,
index_type: "".to_string(),
};
let search_res = SearchService::search(&trace_id, &org_id, stream_type, user_id.clone(), &req)
@ -520,6 +526,7 @@ pub async fn around(
clusters,
timeout,
search_type: Some(SearchEventType::UI),
search_event_context: None,
index_type: "".to_string(),
};
let search_res = SearchService::search(&trace_id, &org_id, stream_type, user_id.clone(), &req)
@ -898,6 +905,7 @@ async fn values_v1(
clusters,
timeout,
search_type: Some(SearchEventType::Values),
search_event_context: None,
index_type: "".to_string(),
};
@ -1218,6 +1226,7 @@ async fn values_v2(
clusters,
timeout,
search_type: Some(SearchEventType::Values),
search_event_context: None,
index_type: "".to_string(),
};
let search_res = SearchService::search(

View File

@ -20,6 +20,7 @@ use chrono::{Duration, Utc};
use config::{
get_config,
meta::{
function::VRLResultResolver,
search,
sql::resolve_stream_names,
stream::StreamType,
@ -37,8 +38,8 @@ use crate::{
utils::{
functions,
http::{
get_or_create_trace_id, get_search_type_from_request, get_stream_type_from_request,
get_work_group,
get_or_create_trace_id, get_search_event_context_from_request,
get_search_type_from_request, get_stream_type_from_request, get_work_group,
},
},
},
@ -146,6 +147,9 @@ pub async fn search_multi(
return Ok(MetaHttpResponse::bad_request(e));
}
};
let search_event_context = search_type
.as_ref()
.and_then(|event_type| get_search_event_context_from_request(event_type, &query));
// handle encoding for query and aggs
let multi_req: search::MultiStreamRequest = match json::from_slice(&body) {
@ -275,6 +279,9 @@ pub async fn search_multi(
}
}
// add search type to request
req.search_type = search_type;
metrics::QUERY_PENDING_NUMS
.with_label_values(&[&org_id])
.inc();
@ -342,6 +349,7 @@ pub async fn search_multi(
max_ts: Some(req.query.end_time),
cached_ratio: Some(res.cached_ratio),
search_type,
search_event_context: search_event_context.clone(),
trace_id: Some(res.trace_id.clone()),
took_wait_in_queue: if res.took_detail.is_some() {
let resp_took = res.took_detail.as_ref().unwrap();
@ -468,11 +476,11 @@ pub async fn search_multi(
if apply_over_hits {
let ret_val = crate::service::ingestion::apply_vrl_fn(
&mut runtime,
&meta::functions::VRLResultResolver {
&VRLResultResolver {
program: program.program.clone(),
fields: program.fields.clone(),
},
&json::Value::Array(multi_res.hits),
json::Value::Array(multi_res.hits),
&org_id,
&[vrl_stream_name.clone()],
);
@ -504,11 +512,11 @@ pub async fn search_multi(
.filter_map(|hit| {
let ret_val = crate::service::ingestion::apply_vrl_fn(
&mut runtime,
&meta::functions::VRLResultResolver {
&VRLResultResolver {
program: program.program.clone(),
fields: program.fields.clone(),
},
&hit,
hit,
&org_id,
&[vrl_stream_name.clone()],
);
@ -559,6 +567,7 @@ pub async fn search_multi(
trace_id: None,
// took_wait_in_queue: multi_res.t,
search_type: multi_req.search_type,
search_event_context: multi_req.search_event_context.clone(),
..Default::default()
};
report_request_usage_stats(
@ -923,6 +932,7 @@ pub async fn around_multi(
clusters: clusters.clone(),
timeout,
search_type: Some(search::SearchEventType::UI),
search_event_context: None,
index_type: "".to_string(),
};
let search_res =
@ -996,6 +1006,7 @@ pub async fn around_multi(
clusters: clusters.clone(),
timeout,
search_type: Some(search::SearchEventType::UI),
search_event_context: None,
index_type: "".to_string(),
};
let search_res =

View File

@ -26,7 +26,7 @@ use arrow_schema::Schema;
use config::{
cluster::LOCAL_NODE,
get_config, get_instance_id,
meta::cluster::NodeStatus,
meta::{cluster::NodeStatus, function::ZoFunction},
utils::{json, schema_ext::SchemaExt},
Config, QUICK_MODEL_FIELDS, SQL_FULL_TEXT_SEARCH_FIELDS,
};
@ -62,7 +62,6 @@ use crate::{
common::{
infra::{cluster, config::*},
meta::{
functions::ZoFunction,
http::HttpResponse as MetaHttpResponse,
user::{AuthTokens, AuthTokensExt},
},

View File

@ -272,6 +272,7 @@ pub async fn get_latest_traces(
clusters: vec![],
timeout,
search_type: None,
search_event_context: None,
index_type: "".to_string(),
};
let stream_type = StreamType::Traces;

View File

@ -403,9 +403,6 @@ pub fn get_service_routes(cfg: &mut web::ServiceConfig) {
.service(functions::list_functions)
.service(functions::delete_function)
.service(functions::update_function)
.service(functions::add_function_to_stream)
.service(functions::list_stream_functions)
.service(functions::delete_stream_function)
.service(dashboards::create_dashboard)
.service(dashboards::update_dashboard)
.service(dashboards::list_dashboards)
@ -481,11 +478,12 @@ pub fn get_service_routes(cfg: &mut web::ServiceConfig) {
.service(authz::fga::delete_group)
.service(users::list_roles)
.service(clusters::list_clusters)
.service(pipelines::save_pipeline)
.service(pipelines::list_pipelines)
.service(pipelines::delete_pipeline)
.service(pipelines::update_pipeline)
.service(pipelines::update_pipeline)
.service(pipeline::save_pipeline)
.service(pipeline::update_pipeline)
.service(pipeline::list_pipelines)
.service(pipeline::list_streams_with_pipeline)
.service(pipeline::delete_pipeline)
.service(pipeline::enable_pipeline)
.service(search::multi_streams::search_multi)
.service(search::multi_streams::_search_partition_multi)
.service(search::multi_streams::around_multi)
@ -547,6 +545,6 @@ mod tests {
.uri("/proxy/org1/https://cloud.openobserve.ai/assets/flUhRq6tzZclQEJ-Vdg-IuiaDsNa.fd84f88b.woff")
.to_request();
let resp = call_service(&mut app, req).await;
assert_eq!(resp.status().as_u16(), 200);
assert_eq!(resp.status().as_u16(), 404);
}
}

View File

@ -74,9 +74,6 @@ use crate::{common::meta, handler::http::request};
request::functions::update_function,
request::functions::save_function,
request::functions::delete_function,
request::functions::list_stream_functions,
request::functions::add_function_to_stream,
request::functions::delete_stream_function,
request::dashboards::create_dashboard,
request::dashboards::update_dashboard,
request::dashboards::list_dashboards,
@ -154,6 +151,24 @@ use crate::{common::meta, handler::http::request};
meta::dashboards::Folder,
meta::dashboards::MoveDashboard,
meta::dashboards::FolderList,
config::meta::alerts::alert::Alert,
config::meta::alerts::Aggregation,
config::meta::alerts::AggFunction,
config::meta::alerts::Condition,
config::meta::alerts::CompareHistoricData,
config::meta::alerts::destinations::Destination,
config::meta::alerts::destinations::DestinationWithTemplate,
config::meta::alerts::destinations::HTTPType,
config::meta::alerts::destinations::DestinationType,
config::meta::alerts::FrequencyType,
config::meta::alerts::Operator,
config::meta::alerts::QueryType,
config::meta::alerts::QueryCondition,
config::meta::alerts::TriggerCondition,
config::meta::alerts::templates::Template,
config::meta::function::Transform,
config::meta::function::FunctionList,
config::meta::function::StreamOrder,
config::meta::sql::OrderBy,
config::meta::search::Query,
config::meta::search::Request,
@ -162,6 +177,7 @@ use crate::{common::meta, handler::http::request};
config::meta::search::ResponseTook,
config::meta::search::ResponseNodeTook,
config::meta::search::SearchEventType,
config::meta::search::SearchEventContext,
config::meta::search::SearchPartitionRequest,
config::meta::search::SearchPartitionResponse,
config::meta::search::SearchHistoryRequest,
@ -179,26 +195,6 @@ use crate::{common::meta, handler::http::request};
meta::saved_view::DeleteViewResponse,
meta::saved_view::CreateViewResponse,
meta::saved_view::UpdateViewRequest,
meta::alerts::alert::Alert,
meta::alerts::Condition,
meta::alerts::Operator,
meta::alerts::Aggregation,
meta::alerts::AggFunction,
meta::alerts::QueryType,
meta::alerts::TriggerCondition,
meta::alerts::FrequencyType,
meta::alerts::QueryCondition,
meta::alerts::CompareHistoricData,
meta::alerts::destinations::Destination,
meta::alerts::destinations::DestinationWithTemplate,
meta::alerts::destinations::HTTPType,
meta::alerts::destinations::DestinationType,
meta::alerts::templates::Template,
meta::functions::Transform,
meta::functions::FunctionList,
meta::functions::StreamFunctionsList,
meta::functions::StreamTransform,
meta::functions::StreamOrder,
meta::user::UpdateUser,
meta::user::UserRequest,
meta::user::UserRole,

View File

@ -293,7 +293,7 @@ impl super::Db for MysqlDb {
} else {
DB_QUERY_NUMS.with_label_values(&["select", "meta"]).inc();
match sqlx::query_as::<_,super::MetaRecord>(
r#"SELECT id, module, key1, key2, start_dt, value FROM meta WHERE module = ? AND key1 = ? AND key2 = ? ORDER BY id DESC;"#
r#"SELECT id, module, key1, key2, start_dt, value FROM meta WHERE module = ? AND key1 = ? AND key2 = ? ORDER BY start_dt DESC, id DESC;"#
)
.bind(&module)
.bind(&key1)

View File

@ -247,7 +247,7 @@ impl super::Db for PostgresDb {
} else {
DB_QUERY_NUMS.with_label_values(&["select", "meta"]).inc();
match sqlx::query_as::<_,super::MetaRecord>(
r#"SELECT id, module, key1, key2, start_dt, value FROM meta WHERE module = $1 AND key1 = $2 AND key2 = $3 ORDER BY id DESC;"#
r#"SELECT id, module, key1, key2, start_dt, value FROM meta WHERE module = $1 AND key1 = $2 AND key2 = $3 ORDER BY start_dt DESC, id DESC;"#
)
.bind(&module)
.bind(&key1)

View File

@ -349,7 +349,7 @@ impl super::Db for SqliteDb {
}
} else {
match sqlx::query_as::<_,super::MetaRecord>(
r#"SELECT id, module, key1, key2, start_dt, value FROM meta WHERE module = $1 AND key1 = $2 AND key2 = $3 ORDER BY id DESC;"#
r#"SELECT id, module, key1, key2, start_dt, value FROM meta WHERE module = $1 AND key1 = $2 AND key2 = $3 ORDER BY start_dt DESC, id DESC;"#
)
.bind(&module)
.bind(&key1)

View File

@ -56,6 +56,7 @@ impl From<DataFusionError> for Error {
};
}
if err.contains("parquet not found") {
log::error!("[Datafusion] Parquet file not found: {}", err);
return Error::ErrorCode(ErrorCodes::SearchParquetFileNotFound);
}
if err.contains("Invalid function ") {

View File

@ -85,6 +85,13 @@ pub trait FileList: Sync + Send + 'static {
stream_name: &str,
time_range: Option<(i64, i64)>,
) -> Result<Vec<FileId>>;
async fn query_old_data_hours(
&self,
org_id: &str,
stream_type: StreamType,
stream_name: &str,
time_range: Option<(i64, i64)>,
) -> Result<Vec<String>>;
async fn query_deleted(
&self,
org_id: &str,
@ -225,11 +232,7 @@ pub async fn list() -> Result<Vec<(String, FileMeta)>> {
}
#[inline]
#[tracing::instrument(
name = "infra:file_list:query_db",
skip_all,
fields(org_id = org_id, stream_name = stream_name)
)]
#[tracing::instrument(name = "infra:file_list:db:query")]
pub async fn query(
org_id: &str,
stream_type: StreamType,
@ -238,11 +241,7 @@ pub async fn query(
time_range: Option<(i64, i64)>,
flattened: Option<bool>,
) -> Result<Vec<(String, FileMeta)>> {
if let Some((start, end)) = time_range {
if start > end || start == 0 || end == 0 {
return Err(Error::Message("[file_list] invalid time range".to_string()));
}
}
validate_time_range(time_range)?;
CLIENT
.query(
org_id,
@ -262,27 +261,33 @@ pub async fn query_by_ids(ids: &[i64]) -> Result<Vec<(i64, String, FileMeta)>> {
}
#[inline]
#[tracing::instrument(
name = "infra:file_list:db:query_ids",
skip_all,
fields(org_id = org_id, stream_name = stream_name)
)]
#[tracing::instrument(name = "infra:file_list:db:query_ids")]
pub async fn query_ids(
org_id: &str,
stream_type: StreamType,
stream_name: &str,
time_range: Option<(i64, i64)>,
) -> Result<Vec<FileId>> {
if let Some((start, end)) = time_range {
if start > end || start == 0 || end == 0 {
return Err(Error::Message("[file_list] invalid time range".to_string()));
}
}
validate_time_range(time_range)?;
CLIENT
.query_ids(org_id, stream_type, stream_name, time_range)
.await
}
#[inline]
#[tracing::instrument(name = "infra:file_list:db:query_old_data_hours")]
pub async fn query_old_data_hours(
org_id: &str,
stream_type: StreamType,
stream_name: &str,
time_range: Option<(i64, i64)>,
) -> Result<Vec<String>> {
validate_time_range(time_range)?;
CLIENT
.query_old_data_hours(org_id, stream_type, stream_name, time_range)
.await
}
#[inline]
pub async fn query_deleted(org_id: &str, time_max: i64, limit: i64) -> Result<Vec<(String, bool)>> {
CLIENT.query_deleted(org_id, time_max, limit).await
@ -440,6 +445,24 @@ pub async fn local_cache_gc() -> Result<()> {
Ok(())
}
fn validate_time_range(time_range: Option<(i64, i64)>) -> Result<()> {
if let Some((start, end)) = time_range {
if start > end || start == 0 || end == 0 {
return Err(Error::Message("[file_list] invalid time range".to_string()));
}
}
Ok(())
}
fn calculate_max_ts_upper_bound(time_end: i64, stream_type: StreamType) -> i64 {
let ts = super::schema::unwrap_partition_time_level(None, stream_type).duration();
if ts > 0 {
time_end + ts
} else {
time_end + PartitionTimeLevel::Hourly.duration()
}
}
#[derive(Debug, Clone, PartialEq, sqlx::FromRow)]
pub struct FileRecord {
#[sqlx(default)]

View File

@ -363,11 +363,8 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
.await
} else {
let (time_start, time_end) = time_range.unwrap_or((0, 0));
let cfg = get_config();
if cfg.limit.use_upper_bound_for_max_ts {
let max_ts_upper_bound =
time_end + cfg.limit.upper_bound_for_max_ts * 60 * 1_000_000;
sqlx::query_as::<_, super::FileRecord>(
let max_ts_upper_bound = super::calculate_max_ts_upper_bound(time_end, stream_type);
sqlx::query_as::<_, super::FileRecord>(
r#"
SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, compressed_size, flattened
FROM file_list
@ -380,20 +377,6 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
.bind(time_end)
.fetch_all(&pool)
.await
} else {
sqlx::query_as::<_, super::FileRecord>(
r#"
SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, compressed_size, flattened
FROM file_list
WHERE stream = ? AND max_ts >= ? AND min_ts <= ?;
"#,
)
.bind(stream_key)
.bind(time_start)
.bind(time_end)
.fetch_all(&pool)
.await
}
};
let time = start.elapsed().as_secs_f64();
DB_QUERY_TIME
@ -495,12 +478,10 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
let stream_key = stream_key.clone();
tasks.push(tokio::task::spawn(async move {
let pool = CLIENT.clone();
let cfg = get_config();
DB_QUERY_NUMS
.with_label_values(&["select", "file_list"])
.inc();
if cfg.limit.use_upper_bound_for_max_ts {
let max_ts_upper_bound = time_end + cfg.limit.upper_bound_for_max_ts * 60 * 1_000_000;
let max_ts_upper_bound = super::calculate_max_ts_upper_bound(time_end, stream_type);
let query = "SELECT id, records, original_size FROM file_list WHERE stream = ? AND max_ts >= ? AND max_ts <= ? AND min_ts <= ?;";
sqlx::query_as::<_, super::FileId>(query)
.bind(stream_key)
@ -509,15 +490,6 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
.bind(time_end)
.fetch_all(&pool)
.await
} else {
let query = "SELECT id, records, original_size FROM file_list WHERE stream = ? AND max_ts >= ? AND min_ts <= ?;";
sqlx::query_as::<_, super::FileId>(query)
.bind(stream_key)
.bind(time_start)
.bind(time_end)
.fetch_all(&pool)
.await
}
}));
}
@ -540,6 +512,56 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
Ok(rets)
}
async fn query_old_data_hours(
&self,
org_id: &str,
stream_type: StreamType,
stream_name: &str,
time_range: Option<(i64, i64)>,
) -> Result<Vec<String>> {
if let Some((start, end)) = time_range {
if start == 0 && end == 0 {
return Ok(Vec::new());
}
}
let stream_key = format!("{org_id}/{stream_type}/{stream_name}");
let pool = CLIENT.clone();
DB_QUERY_NUMS
.with_label_values(&["select", "file_list"])
.inc();
let start = std::time::Instant::now();
let (time_start, time_end) = time_range.unwrap_or((0, 0));
let cfg = get_config();
let max_ts_upper_bound = super::calculate_max_ts_upper_bound(time_end, stream_type);
let sql = r#"
SELECT date
FROM file_list
WHERE stream = ? AND max_ts >= ? AND max_ts <= ? AND min_ts <= ? AND records < ?
GROUP BY date HAVING count(*) >= ?;
"#;
let ret = sqlx::query(sql)
.bind(stream_key)
.bind(time_start)
.bind(max_ts_upper_bound)
.bind(time_end)
.bind(cfg.compact.old_data_min_records)
.bind(cfg.compact.old_data_min_files)
.fetch_all(&pool)
.await?;
let time = start.elapsed().as_secs_f64();
DB_QUERY_TIME
.with_label_values(&["query_old_data_hours", "file_list"])
.observe(time);
Ok(ret
.into_iter()
.map(|r| r.try_get::<String, &str>("date").unwrap_or_default())
.collect())
}
async fn query_deleted(
&self,
org_id: &str,

View File

@ -376,37 +376,20 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
.fetch_all(&pool).await
} else {
let (time_start, time_end) = time_range.unwrap_or((0, 0));
let cfg = get_config();
if cfg.limit.use_upper_bound_for_max_ts {
let max_ts_upper_bound =
time_end + cfg.limit.upper_bound_for_max_ts * 60 * 1_000_000;
let sql = r#"
let max_ts_upper_bound = super::calculate_max_ts_upper_bound(time_end, stream_type);
let sql = r#"
SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, compressed_size, flattened
FROM file_list
WHERE stream = $1 AND max_ts >= $2 AND max_ts <= $3 AND min_ts <= $4;
"#;
sqlx::query_as::<_, super::FileRecord>(sql)
.bind(stream_key)
.bind(time_start)
.bind(max_ts_upper_bound)
.bind(time_end)
.fetch_all(&pool)
.await
} else {
let sql = r#"
SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, compressed_size, flattened
FROM file_list
WHERE stream = $1 AND max_ts >= $2 AND min_ts <= $3;
"#;
sqlx::query_as::<_, super::FileRecord>(sql)
.bind(stream_key)
.bind(time_start)
.bind(time_end)
.fetch_all(&pool)
.await
}
sqlx::query_as::<_, super::FileRecord>(sql)
.bind(stream_key)
.bind(time_start)
.bind(max_ts_upper_bound)
.bind(time_end)
.fetch_all(&pool)
.await
};
let time = start.elapsed().as_secs_f64();
DB_QUERY_TIME
@ -508,29 +491,18 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
let stream_key = stream_key.clone();
tasks.push(tokio::task::spawn(async move {
let pool = CLIENT.clone();
let cfg = get_config();
DB_QUERY_NUMS
.with_label_values(&["select", "file_list"])
.inc();
if cfg.limit.use_upper_bound_for_max_ts {
let max_ts_upper_bound = time_end + cfg.limit.upper_bound_for_max_ts * 60 * 1_000_000;
let query = "SELECT id, records, original_size FROM file_list WHERE stream = $1 AND max_ts >= $2 AND max_ts <= $3 AND min_ts <= $4;";
sqlx::query_as::<_, super::FileId>(query)
.bind(stream_key)
.bind(time_start)
.bind(max_ts_upper_bound)
.bind(time_end)
.fetch_all(&pool)
.await
} else {
let query = "SELECT id, records, original_size FROM file_list WHERE stream = $1 AND max_ts >= $2 AND min_ts <= $3;";
sqlx::query_as::<_, super::FileId>(query)
.bind(stream_key)
.bind(time_start)
.bind(time_end)
.fetch_all(&pool)
.await
}
let max_ts_upper_bound = super::calculate_max_ts_upper_bound(time_end, stream_type);
let query = "SELECT id, records, original_size FROM file_list WHERE stream = $1 AND max_ts >= $2 AND max_ts <= $3 AND min_ts <= $4;";
sqlx::query_as::<_, super::FileId>(query)
.bind(stream_key)
.bind(time_start)
.bind(max_ts_upper_bound)
.bind(time_end)
.fetch_all(&pool)
.await
}));
}
@ -553,6 +525,56 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
Ok(rets)
}
async fn query_old_data_hours(
&self,
org_id: &str,
stream_type: StreamType,
stream_name: &str,
time_range: Option<(i64, i64)>,
) -> Result<Vec<String>> {
if let Some((start, end)) = time_range {
if start == 0 && end == 0 {
return Ok(Vec::new());
}
}
let stream_key = format!("{org_id}/{stream_type}/{stream_name}");
let pool = CLIENT.clone();
DB_QUERY_NUMS
.with_label_values(&["select", "file_list"])
.inc();
let start = std::time::Instant::now();
let (time_start, time_end) = time_range.unwrap_or((0, 0));
let cfg = get_config();
let max_ts_upper_bound = super::calculate_max_ts_upper_bound(time_end, stream_type);
let sql = r#"
SELECT date
FROM file_list
WHERE stream = $1 AND max_ts >= $2 AND max_ts <= $3 AND min_ts <= $4 AND records < $5
GROUP BY date HAVING count(*) >= $6;
"#;
let ret = sqlx::query(sql)
.bind(stream_key)
.bind(time_start)
.bind(max_ts_upper_bound)
.bind(time_end)
.bind(cfg.compact.old_data_min_records)
.bind(cfg.compact.old_data_min_files)
.fetch_all(&pool)
.await?;
let time = start.elapsed().as_secs_f64();
DB_QUERY_TIME
.with_label_values(&["query_old_data_hours", "file_list"])
.observe(time);
Ok(ret
.into_iter()
.map(|r| r.try_get::<String, &str>("date").unwrap_or_default())
.collect())
}
async fn query_deleted(
&self,
org_id: &str,

View File

@ -326,11 +326,8 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
.await
} else {
let (time_start, time_end) = time_range.unwrap_or((0, 0));
let cfg = get_config();
if cfg.limit.use_upper_bound_for_max_ts {
let max_ts_upper_bound =
time_end + cfg.limit.upper_bound_for_max_ts * 60 * 1_000_000;
sqlx::query_as::<_, super::FileRecord>(
let max_ts_upper_bound = super::calculate_max_ts_upper_bound(time_end, stream_type);
sqlx::query_as::<_, super::FileRecord>(
r#"
SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, compressed_size, flattened
FROM file_list
@ -343,20 +340,6 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
.bind(time_end)
.fetch_all(&pool)
.await
} else {
sqlx::query_as::<_, super::FileRecord>(
r#"
SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, compressed_size, flattened
FROM file_list
WHERE stream = $1 AND max_ts >= $2 AND min_ts <= $3;
"#,
)
.bind(stream_key)
.bind(time_start)
.bind(time_end)
.fetch_all(&pool)
.await
}
};
Ok(ret?
.iter()
@ -446,9 +429,7 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
let stream_key = stream_key.clone();
tasks.push(tokio::task::spawn(async move {
let pool = CLIENT_RO.clone();
let cfg = get_config();
if cfg.limit.use_upper_bound_for_max_ts {
let max_ts_upper_bound = time_end + cfg.limit.upper_bound_for_max_ts * 60 * 1_000_000;
let max_ts_upper_bound = super::calculate_max_ts_upper_bound(time_end, stream_type);
let query = "SELECT id, records, original_size FROM file_list WHERE stream = $1 AND max_ts >= $2 AND max_ts <= $3 AND min_ts <= $4;";
sqlx::query_as::<_, super::FileId>(query)
.bind(stream_key)
@ -457,15 +438,6 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
.bind(time_end)
.fetch_all(&pool)
.await
} else {
let query = "SELECT id, records, original_size FROM file_list WHERE stream = $1 AND max_ts >= $2 AND min_ts <= $3;";
sqlx::query_as::<_, super::FileId>(query)
.bind(stream_key)
.bind(time_start)
.bind(time_end)
.fetch_all(&pool)
.await
}
}));
}
@ -484,6 +456,47 @@ SELECT stream, date, file, deleted, min_ts, max_ts, records, original_size, comp
Ok(rets)
}
async fn query_old_data_hours(
&self,
org_id: &str,
stream_type: StreamType,
stream_name: &str,
time_range: Option<(i64, i64)>,
) -> Result<Vec<String>> {
if let Some((start, end)) = time_range {
if start == 0 && end == 0 {
return Ok(Vec::new());
}
}
let stream_key = format!("{org_id}/{stream_type}/{stream_name}");
let pool = CLIENT_RO.clone();
let (time_start, time_end) = time_range.unwrap_or((0, 0));
let cfg = get_config();
let max_ts_upper_bound = super::calculate_max_ts_upper_bound(time_end, stream_type);
let sql = r#"
SELECT date
FROM file_list
WHERE stream = $1 AND max_ts >= $2 AND max_ts <= $3 AND min_ts <= $4 AND records < $5
GROUP BY date HAVING count(*) >= $6;
"#;
let ret = sqlx::query(sql)
.bind(stream_key)
.bind(time_start)
.bind(max_ts_upper_bound)
.bind(time_end)
.bind(cfg.compact.old_data_min_records)
.bind(cfg.compact.old_data_min_files)
.fetch_all(&pool)
.await?;
Ok(ret
.into_iter()
.map(|r| r.try_get::<String, &str>("date").unwrap_or_default())
.collect())
}
async fn query_deleted(
&self,
org_id: &str,

View File

@ -18,6 +18,7 @@ pub mod db;
pub mod dist_lock;
pub mod errors;
pub mod file_list;
pub mod pipeline;
pub mod queue;
pub mod scheduler;
pub mod schema;
@ -30,6 +31,7 @@ pub async fn init() -> Result<(), anyhow::Error> {
file_list::create_table().await?;
file_list::LOCAL_CACHE.create_table().await?;
file_list::local_cache_gc().await?;
pipeline::init().await?;
queue::init().await?;
scheduler::init().await?;
schema::init().await?;

View File

@ -0,0 +1,133 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use async_trait::async_trait;
use config::meta::{
meta_store::MetaStore,
pipeline::{components::PipelineSource, Pipeline},
stream::StreamParams,
};
use once_cell::sync::Lazy;
use crate::errors::Result;
pub mod mysql;
pub mod postgres;
pub mod sqlite;
static CLIENT: Lazy<Box<dyn PipelineTable>> = Lazy::new(connect);
pub fn connect() -> Box<dyn PipelineTable> {
match config::get_config().common.meta_store.as_str().into() {
MetaStore::MySQL => Box::<mysql::MySqlPipelineTable>::default(),
MetaStore::PostgreSQL => Box::<postgres::PostgresPipelineTable>::default(),
_ => Box::<sqlite::SqlitePipelineTable>::default(),
}
}
#[async_trait]
pub trait PipelineTable: Sync + Send + 'static {
async fn create_table(&self) -> Result<()>;
async fn create_table_index(&self) -> Result<()>;
async fn drop_table(&self) -> Result<()>;
async fn put(&self, pipeline: &Pipeline) -> Result<()>;
async fn update(&self, pipeline: &Pipeline) -> Result<()>;
async fn get_by_stream(&self, stream_params: &StreamParams) -> Result<Pipeline>;
async fn get_by_id(&self, pipeline_id: &str) -> Result<Pipeline>;
async fn get_with_same_source_stream(&self, pipeline: &Pipeline) -> Result<Pipeline>;
async fn list(&self) -> Result<Vec<Pipeline>>;
async fn list_by_org(&self, org: &str) -> Result<Vec<Pipeline>>;
async fn list_streams_with_pipeline(&self, org: &str) -> Result<Vec<Pipeline>>;
async fn delete(&self, pipeline_id: &str) -> Result<Pipeline>;
}
/// Initializes the PipelineTable - creates table and index
pub async fn init() -> Result<()> {
CLIENT.create_table().await?;
CLIENT.create_table_index().await?;
Ok(())
}
/// Creates a pipeline entry in the table
#[inline]
pub async fn put(pipeline: &Pipeline) -> Result<()> {
CLIENT.put(pipeline).await
}
/// Updates a pipeline entry by id
#[inline]
pub async fn update(pipeline: &Pipeline) -> Result<()> {
CLIENT.update(pipeline).await
}
/// Finds the pipeline associated with the StreamParams within an organization
#[inline]
pub async fn get_by_stream(stream_params: &StreamParams) -> Result<Pipeline> {
CLIENT.get_by_stream(stream_params).await
}
/// Finds all streams with existing pipelines.
#[inline]
pub async fn list_streams_with_pipeline(org: &str) -> Result<Vec<StreamParams>> {
CLIENT
.list_streams_with_pipeline(org)
.await
.map(|pipelines| {
pipelines
.into_iter()
.filter_map(|pl| match pl.source {
PipelineSource::Realtime(stream_params) => Some(stream_params),
PipelineSource::Scheduled(_) => None,
})
.collect()
})
}
/// Finds the pipeline by id
#[inline]
pub async fn get_by_id(pipeline_id: &str) -> Result<Pipeline> {
CLIENT.get_by_id(pipeline_id).await
}
/// Finds the pipeline with the same source and structure
#[inline]
pub async fn get_with_same_source_stream(pipeline: &Pipeline) -> Result<Pipeline> {
CLIENT.get_with_same_source_stream(pipeline).await
}
/// Lists all pipelines
#[inline]
pub async fn list() -> Result<Vec<Pipeline>> {
CLIENT.list().await
}
/// Lists all pipelines within an organization
#[inline]
pub async fn list_by_org(org: &str) -> Result<Vec<Pipeline>> {
CLIENT.list_by_org(org).await
}
/// Deletes the pipeline by id
#[inline]
pub async fn delete(pipeline_id: &str) -> Result<Pipeline> {
CLIENT.delete(pipeline_id).await
}
/// DropTable first. Used for testing migration.
#[inline]
pub async fn drop_table() -> Result<()> {
CLIENT.drop_table().await?;
Ok(())
}

View File

@ -0,0 +1,375 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use async_trait::async_trait;
use config::{
meta::{
pipeline::{components::PipelineSource, Pipeline},
stream::StreamParams,
},
utils::json,
};
use crate::{
db::mysql::CLIENT,
errors::{DbError, Error, Result},
};
pub struct MySqlPipelineTable {}
impl MySqlPipelineTable {
pub fn new() -> Self {
Self {}
}
}
impl Default for MySqlPipelineTable {
fn default() -> Self {
Self::new()
}
}
#[async_trait]
impl super::PipelineTable for MySqlPipelineTable {
async fn create_table(&self) -> Result<()> {
let pool = CLIENT.clone();
sqlx::query(
r#"
CREATE TABLE IF NOT EXISTS pipeline
(
id VARCHAR(256) not null primary key,
version INT not null,
enabled BOOLEAN default true not null,
name VARCHAR(256) not null,
description TEXT,
org VARCHAR(100) not null,
source_type VARCHAR(50) not null,
stream_org VARCHAR(100),
stream_name VARCHAR(256),
stream_type VARCHAR(50),
derived_stream TEXT,
nodes TEXT,
edges TEXT,
created_at TIMESTAMP DEFAULT CURRENT_TIMESTAMP
);
"#,
)
.execute(&pool)
.await?;
Ok(())
}
async fn create_table_index(&self) -> Result<()> {
let pool = CLIENT.clone();
let queries = vec![
"CREATE INDEX pipeline_org_idx ON pipeline (org);",
"CREATE INDEX pipeline_org_src_type_stream_params_idx ON pipeline (org, source_type, stream_org, stream_name, stream_type);",
];
for query in queries {
if let Err(e) = sqlx::query(query).execute(&pool).await {
if e.to_string().contains("Duplicate key") {
// index already exists
continue;
}
log::error!("[MYSQL] create index for pipeline table error: {}", e);
return Err(e.into());
}
}
Ok(())
}
async fn drop_table(&self) -> Result<()> {
let pool = CLIENT.clone();
sqlx::query("DROP TABLE IF EXISTS pipeline;")
.execute(&pool)
.await?;
Ok(())
}
async fn put(&self, pipeline: &Pipeline) -> Result<()> {
let pool = CLIENT.clone();
let mut tx = pool.begin().await?;
if let Err(e) = match &pipeline.source {
PipelineSource::Realtime(stream_params) => {
let (source_type, stream_org, stream_name, stream_type): (&str, &str, &str, &str) = (
"realtime",
stream_params.org_id.as_str(),
stream_params.stream_name.as_str(),
stream_params.stream_type.as_str(),
);
sqlx::query(
r#"
INSERT IGNORE INTO pipeline (id, version, enabled, name, description, org, source_type, stream_org, stream_name, stream_type, nodes, edges)
VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?);
"#,
)
.bind(&pipeline.id)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(stream_org)
.bind(stream_name)
.bind(stream_type)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.execute(&mut *tx)
.await
}
PipelineSource::Scheduled(derived_stream) => {
let (source_type, derived_stream_str) = (
"scheduled",
json::to_string(&derived_stream)
.expect("Serializing pipeline DerivedStream error"),
);
sqlx::query(
r#"
INSERT IGNORE INTO pipeline (id, version, enabled, name, description, org, source_type, derived_stream, nodes, edges)
VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?);
"#,
)
.bind(&pipeline.id)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(derived_stream_str)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.execute(&mut *tx)
.await
}
} {
if let Err(e) = tx.rollback().await {
log::error!("[MYSQL] rollback push pipeline error: {}", e);
}
return Err(e.into());
}
if let Err(e) = tx.commit().await {
log::error!("[MYSQL] commit push pipeline error: {}", e);
return Err(e.into());
}
Ok(())
}
async fn update(&self, pipeline: &Pipeline) -> Result<()> {
let pool = CLIENT.clone();
let mut tx = pool.begin().await?;
if let Err(e) = match &pipeline.source {
PipelineSource::Realtime(stream_params) => {
let (source_type, stream_org, stream_name, stream_type): (&str, &str, &str, &str) = (
"realtime",
stream_params.org_id.as_str(),
stream_params.stream_name.as_str(),
stream_params.stream_type.as_str(),
);
sqlx::query(
r#"
UPDATE pipeline
SET version = ?, enabled = ?, name = ?, description = ?, org = ?, source_type = ?, stream_org = ?, stream_name = ?, stream_type = ?, nodes = ?, edges = ?
WHERE id =?;
"#,
)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(stream_org)
.bind(stream_name)
.bind(stream_type)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.bind(&pipeline.id)
.execute(&mut *tx)
.await
}
PipelineSource::Scheduled(derived_stream) => {
let (source_type, derived_stream_str) = (
"scheduled",
json::to_string(&derived_stream)
.expect("Serializing pipeline DerivedStream error"),
);
sqlx::query(
r#"
UPDATE pipeline
SET version = ?, enabled = ?, name = ?, description = ?, org = ?, source_type = ?, derived_stream = ?, nodes = ?, edges = ?
WHERE id = ?;
"#,
)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(derived_stream_str)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.bind(&pipeline.id)
.execute(&mut *tx)
.await
}
} {
if let Err(e) = tx.rollback().await {
log::error!("[MYSQL] rollback push pipeline error: {}", e);
}
return Err(e.into());
}
if let Err(e) = tx.commit().await {
log::error!("[MYSQL] commit push pipeline error: {}", e);
return Err(e.into());
}
Ok(())
}
async fn get_by_stream(&self, stream_params: &StreamParams) -> Result<Pipeline> {
let pool = CLIENT.clone();
let query = r#"
SELECT * FROM pipeline WHERE org = ? AND source_type = ? AND stream_org = ? AND stream_name = ? AND stream_type = ?;
"#;
let pipeline = sqlx::query_as::<_, Pipeline>(query)
.bind(stream_params.org_id.as_str())
.bind("realtime")
.bind(stream_params.org_id.as_str())
.bind(stream_params.stream_name.as_str())
.bind(stream_params.stream_type.as_str())
.fetch_one(&pool)
.await?;
Ok(pipeline)
}
async fn get_by_id(&self, pipeline_id: &str) -> Result<Pipeline> {
let pool = CLIENT.clone();
let query = r#"SELECT * FROM pipeline WHERE id = ?;"#;
let pipeline = match sqlx::query_as::<_, Pipeline>(query)
.bind(pipeline_id)
.fetch_one(&pool)
.await
{
Ok(pipeline) => pipeline,
Err(e) => {
log::debug!("[MYSQL] get pipeline by id error: {}", e);
return Err(Error::from(DbError::KeyNotExists(pipeline_id.to_string())));
}
};
Ok(pipeline)
}
async fn get_with_same_source_stream(&self, pipeline: &Pipeline) -> Result<Pipeline> {
let pool = CLIENT.clone();
let similar_pipeline = match &pipeline.source {
PipelineSource::Realtime(stream_params) => {
sqlx::query_as::<_, Pipeline>(
r#"
SELECT * FROM pipeline
WHERE source_type = ? AND stream_org = ? AND stream_name = ? AND stream_type = ?;
"#,
)
.bind("realtime")
.bind(stream_params.org_id.as_str())
.bind(stream_params.stream_name.as_str())
.bind(stream_params.stream_type.as_str())
.fetch_one(&pool)
.await?
}
PipelineSource::Scheduled(_) => {
// only checks for realtime pipelines
return Err(Error::from(DbError::KeyNotExists("".to_string())));
}
};
Ok(similar_pipeline)
}
async fn list(&self) -> Result<Vec<Pipeline>> {
let pool = CLIENT.clone();
let query = r#"SELECT * FROM pipeline ORDER BY id;"#;
match sqlx::query_as::<_, Pipeline>(query).fetch_all(&pool).await {
Ok(pipelines) => Ok(pipelines),
Err(e) => {
log::debug!("[MYSQL] list all pipelines error: {}", e);
Ok(vec![])
}
}
}
async fn list_by_org(&self, org: &str) -> Result<Vec<Pipeline>> {
let pool = CLIENT.clone();
let query = r#"SELECT * FROM pipeline WHERE org = ? ORDER BY id;"#;
match sqlx::query_as::<_, Pipeline>(query)
.bind(org)
.fetch_all(&pool)
.await
{
Ok(pipelines) => Ok(pipelines),
Err(e) => {
log::debug!("[MYSQL] list pipelines by org error: {}", e);
Ok(vec![])
}
}
}
async fn list_streams_with_pipeline(&self, org: &str) -> Result<Vec<Pipeline>> {
let pool = CLIENT.clone();
let query = r#"
SELECT * FROM pipeline WHERE org = ? AND source_type = ? ORDER BY id;
"#;
match sqlx::query_as::<_, Pipeline>(query)
.bind(org)
.bind("realtime")
.fetch_all(&pool)
.await
{
Ok(pipelines) => Ok(pipelines),
Err(e) => {
log::debug!("[MYSQL] list streams with pipelines error: {}", e);
Ok(vec![])
}
}
}
async fn delete(&self, pipeline_id: &str) -> Result<Pipeline> {
let pool = CLIENT.clone();
let pipeline = sqlx::query_as::<_, Pipeline>("SELECT * FROM pipeline WHERE id = ?;")
.bind(pipeline_id)
.fetch_one(&pool)
.await?;
sqlx::query(r#"DELETE FROM pipeline WHERE id = ?;"#)
.bind(pipeline_id)
.execute(&pool)
.await?;
Ok(pipeline)
}
}

View File

@ -0,0 +1,375 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use async_trait::async_trait;
use config::{
meta::{
pipeline::{components::PipelineSource, Pipeline},
stream::StreamParams,
},
utils::json,
};
use crate::{
db::postgres::CLIENT,
errors::{DbError, Error, Result},
};
pub struct PostgresPipelineTable {}
impl PostgresPipelineTable {
pub fn new() -> Self {
Self {}
}
}
impl Default for PostgresPipelineTable {
fn default() -> Self {
Self::new()
}
}
#[async_trait]
impl super::PipelineTable for PostgresPipelineTable {
async fn create_table(&self) -> Result<()> {
let pool = CLIENT.clone();
sqlx::query(
r#"
CREATE TABLE IF NOT EXISTS pipeline
(
id VARCHAR(256) PRIMARY KEY,
version INT not null,
enabled BOOLEAN default true not null,
name VARCHAR(256) not null,
description TEXT,
org VARCHAR(100) not null,
source_type VARCHAR(50) not null,
stream_org VARCHAR(100),
stream_name VARCHAR(256),
stream_type VARCHAR(50),
derived_stream TEXT,
nodes TEXT,
edges TEXT,
created_at TIMESTAMP default CURRENT_TIMESTAMP
);
"#,
)
.execute(&pool)
.await?;
Ok(())
}
async fn create_table_index(&self) -> Result<()> {
let pool = CLIENT.clone();
let queries = vec![
"CREATE INDEX IF NOT EXISTS pipeline_org_idx ON pipeline (org);",
"CREATE INDEX IF NOT EXISTS pipeline_org_src_type_stream_params_idx ON pipeline (org, source_type, stream_org, stream_name, stream_type);",
];
for query in queries {
if let Err(e) = sqlx::query(query).execute(&pool).await {
log::error!(
"[POSTGRES] create table index for pipeline table error: {}",
e
);
return Err(e.into());
}
}
Ok(())
}
async fn drop_table(&self) -> Result<()> {
let pool = CLIENT.clone();
sqlx::query("DROP TABLE IF EXISTS pipeline;")
.execute(&pool)
.await?;
Ok(())
}
async fn put(&self, pipeline: &Pipeline) -> Result<()> {
let pool = CLIENT.clone();
let mut tx = pool.begin().await?;
if let Err(e) = match &pipeline.source {
PipelineSource::Realtime(stream_params) => {
let (source_type, stream_org, stream_name, stream_type): (&str, &str, &str, &str) = (
"realtime",
stream_params.org_id.as_str(),
stream_params.stream_name.as_str(),
stream_params.stream_type.as_str(),
);
sqlx::query(
r#"
INSERT INTO pipeline (id, version, enabled, name, description, org, source_type, stream_org, stream_name, stream_type, nodes, edges)
VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12)
ON CONFLICT DO NOTHING;
"#,
)
.bind(&pipeline.id)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(stream_org)
.bind(stream_name)
.bind(stream_type)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.execute(&mut *tx)
.await
}
PipelineSource::Scheduled(derived_stream) => {
let (source_type, derived_stream_str) = (
"scheduled",
json::to_string(&derived_stream)
.expect("Serializing pipeline DerivedStream error"),
);
sqlx::query(
r#"
INSERT INTO pipeline (id, version, enabled, name, description, org, source_type, derived_stream, nodes, edges)
VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10)
ON CONFLICT DO NOTHING;
"#,
)
.bind(&pipeline.id)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(derived_stream_str)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.execute(&mut *tx)
.await
}
} {
if let Err(e) = tx.rollback().await {
log::error!("[POSTGRES] rollback push pipeline error: {}", e);
}
return Err(e.into());
}
if let Err(e) = tx.commit().await {
log::error!("[POSTGRES] commit push pipeline error: {}", e);
return Err(e.into());
}
Ok(())
}
async fn update(&self, pipeline: &Pipeline) -> Result<()> {
let pool = CLIENT.clone();
let mut tx = pool.begin().await?;
if let Err(e) = match &pipeline.source {
PipelineSource::Realtime(stream_params) => {
let (source_type, stream_org, stream_name, stream_type): (&str, &str, &str, &str) = (
"realtime",
stream_params.org_id.as_str(),
stream_params.stream_name.as_str(),
stream_params.stream_type.as_str(),
);
sqlx::query(
r#"
UPDATE pipeline
SET version = $1, enabled = $2, name = $3, description = $4, org = $5, source_type = $6, stream_org = $7, stream_name = $8, stream_type = $9, nodes = $10, edges = $11
WHERE id = $12;
"#,
)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(stream_org)
.bind(stream_name)
.bind(stream_type)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.bind(&pipeline.id)
.execute(&mut *tx)
.await
}
PipelineSource::Scheduled(derived_stream) => {
let (source_type, derived_stream_str) = (
"scheduled",
json::to_string(&derived_stream)
.expect("Serializing pipeline DerivedStream error"),
);
sqlx::query(
r#"
UPDATE pipeline
SET version = $1, enabled = $2, name = $3, description = $4, org = $5, source_type = $6, derived_stream = $7, nodes = $8, edges = $9
WHERE id = $10;
"#,
)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(derived_stream_str)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.bind(&pipeline.id)
.execute(&mut *tx)
.await
}
} {
if let Err(e) = tx.rollback().await {
log::error!("[POSTGRES] rollback push pipeline error: {}", e);
}
return Err(e.into());
}
if let Err(e) = tx.commit().await {
log::error!("[POSTGRES] commit push pipeline error: {}", e);
return Err(e.into());
}
Ok(())
}
async fn get_by_stream(&self, stream_params: &StreamParams) -> Result<Pipeline> {
let pool = CLIENT.clone();
let query = r#"
SELECT * FROM pipeline WHERE org = $1 AND source_type = $2 AND stream_org = $3 AND stream_name = $4 AND stream_type = $5;
"#;
let pipeline = sqlx::query_as::<_, Pipeline>(query)
.bind(stream_params.org_id.as_str())
.bind("realtime")
.bind(stream_params.org_id.as_str())
.bind(stream_params.stream_name.as_str())
.bind(stream_params.stream_type.as_str())
.fetch_one(&pool)
.await?;
Ok(pipeline)
}
async fn get_by_id(&self, pipeline_id: &str) -> Result<Pipeline> {
let pool = CLIENT.clone();
let query = "SELECT * FROM pipeline WHERE id = $1;";
let pipeline = match sqlx::query_as::<_, Pipeline>(query)
.bind(pipeline_id)
.fetch_one(&pool)
.await
{
Ok(pipeline) => pipeline,
Err(e) => {
log::error!("[POSTGRES] get pipeline by id error: {}", e);
return Err(Error::from(DbError::KeyNotExists(pipeline_id.to_string())));
}
};
Ok(pipeline)
}
async fn get_with_same_source_stream(&self, pipeline: &Pipeline) -> Result<Pipeline> {
let pool = CLIENT.clone();
let similar_pipeline = match &pipeline.source {
PipelineSource::Realtime(stream_params) => {
sqlx::query_as::<_, Pipeline>(
r#"
SELECT * FROM pipeline
WHERE source_type = $1 AND stream_org = $2 AND stream_name = $3 AND stream_type = $4;
"#,
)
.bind("realtime")
.bind(stream_params.org_id.as_str())
.bind(stream_params.stream_name.as_str())
.bind(stream_params.stream_type.as_str())
.fetch_one(&pool)
.await?
}
PipelineSource::Scheduled(_) => {
// only checks for realtime pipelines
return Err(Error::from(DbError::KeyNotExists(
"No matching pipeline found for scheduled source".to_string(),
)));
}
};
Ok(similar_pipeline)
}
async fn list(&self) -> Result<Vec<Pipeline>> {
let pool = CLIENT.clone();
let query = "SELECT * FROM pipeline ORDER BY id;";
match sqlx::query_as::<_, Pipeline>(query).fetch_all(&pool).await {
Ok(pipelines) => Ok(pipelines),
Err(e) => {
log::debug!("[POSTGRES] list all pipelines error: {}", e);
Ok(vec![]) // Return empty vector instead of error
}
}
}
async fn list_by_org(&self, org: &str) -> Result<Vec<Pipeline>> {
let pool = CLIENT.clone();
let query = "SELECT * FROM pipeline WHERE org = $1 ORDER BY id;";
match sqlx::query_as::<_, Pipeline>(query)
.bind(org)
.fetch_all(&pool)
.await
{
Ok(pipelines) => Ok(pipelines),
Err(e) => {
log::debug!("[POSTGRES] list pipelines by org error: {}", e);
Ok(vec![])
}
}
}
async fn list_streams_with_pipeline(&self, org: &str) -> Result<Vec<Pipeline>> {
let pool = CLIENT.clone();
let query = r#"
SELECT * FROM pipeline WHERE org = $1 AND source_type = $2 ORDER BY id;
"#;
match sqlx::query_as::<_, Pipeline>(query)
.bind(org)
.bind("realtime")
.fetch_all(&pool)
.await
{
Ok(pipelines) => Ok(pipelines),
Err(e) => {
log::debug!("[POSTGRES] list streams with pipelines error: {}", e);
Ok(vec![])
}
}
}
async fn delete(&self, pipeline_id: &str) -> Result<Pipeline> {
let pool = CLIENT.clone();
let pipeline = sqlx::query_as::<_, Pipeline>("SELECT * FROM pipeline WHERE id = $1;")
.bind(pipeline_id)
.fetch_one(&pool)
.await?;
sqlx::query(r#"DELETE FROM pipeline WHERE id = $1;"#)
.bind(pipeline_id)
.execute(&pool)
.await?;
Ok(pipeline)
}
}

View File

@ -0,0 +1,387 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use async_trait::async_trait;
use config::{
meta::{
pipeline::{components::PipelineSource, Pipeline},
stream::StreamParams,
},
utils::json,
};
use crate::{
db::sqlite::{CLIENT_RO, CLIENT_RW},
errors::{DbError, Error, Result},
};
pub struct SqlitePipelineTable {}
impl SqlitePipelineTable {
pub fn new() -> Self {
Self {}
}
}
impl Default for SqlitePipelineTable {
fn default() -> Self {
Self::new()
}
}
#[async_trait]
impl super::PipelineTable for SqlitePipelineTable {
async fn create_table(&self) -> Result<()> {
let client = CLIENT_RW.clone();
let client = client.lock().await;
sqlx::query(
r#"
CREATE TABLE IF NOT EXISTS pipeline
(
id VARCHAR(256) not null primary key,
version INT not null,
enabled BOOLEAN default true not null,
name VARCHAR(256) not null,
description TEXT,
org VARCHAR(100) not null,
source_type VARCHAR(50) not null,
stream_org VARCHAR(100),
stream_name VARCHAR(256),
stream_type VARCHAR(50),
derived_stream TEXT,
nodes TEXT,
edges TEXT,
created_at TIMESTAMP default CURRENT_TIMESTAMP
);
"#,
)
.execute(&*client)
.await?;
Ok(())
}
async fn create_table_index(&self) -> Result<()> {
let client = CLIENT_RW.clone();
let client = client.lock().await;
let queries = vec![
"CREATE INDEX IF NOT EXISTS pipeline_org_idx ON pipeline (org);",
"CREATE INDEX IF NOT EXISTS pipeline_org_src_type_stream_params_idx ON pipeline (org, source_type, stream_org, stream_name, stream_type);",
];
for query in queries {
sqlx::query(query).execute(&*client).await?;
}
Ok(())
}
async fn drop_table(&self) -> Result<()> {
let client = CLIENT_RW.clone();
let client = client.lock().await;
sqlx::query("DROP TABLE IF EXISTS pipeline;")
.execute(&*client)
.await?;
Ok(())
}
async fn put(&self, pipeline: &Pipeline) -> Result<()> {
let client = CLIENT_RW.clone();
let client = client.lock().await;
let mut tx = client.begin().await?;
if let Err(e) = match &pipeline.source {
PipelineSource::Realtime(stream_params) => {
let (source_type, stream_org, stream_name, stream_type): (&str, &str, &str, &str) = (
"realtime",
stream_params.org_id.as_str(),
stream_params.stream_name.as_str(),
stream_params.stream_type.as_str(),
);
sqlx::query(
r#"
INSERT INTO pipeline (id, version, enabled, name, description, org, source_type, stream_org, stream_name, stream_type, nodes, edges)
VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12)
ON CONFLICT DO NOTHING;
"#,
)
.bind(&pipeline.id)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(stream_org)
.bind(stream_name)
.bind(stream_type)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.execute(&mut *tx)
.await
}
PipelineSource::Scheduled(derived_stream) => {
let (source_type, derived_stream_str) = (
"scheduled",
json::to_string(&derived_stream)
.expect("Serializing pipeline DerivedStream error"),
);
sqlx::query(
r#"
INSERT INTO pipeline (id, version, enabled, name, description, org, source_type, derived_stream, nodes, edges)
VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10)
ON CONFLICT DO NOTHING;
"#,
)
.bind(&pipeline.id)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(derived_stream_str)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.execute(&mut *tx)
.await
}
} {
if let Err(e) = tx.rollback().await {
log::error!("[SQLITE] rollback push pipeline error: {}", e);
}
return Err(e.into());
}
if let Err(e) = tx.commit().await {
log::error!("[SQLITE] commit push pipeline error: {}", e);
return Err(e.into());
}
// release lock
drop(client);
Ok(())
}
async fn update(&self, pipeline: &Pipeline) -> Result<()> {
let client = CLIENT_RW.clone();
let client = client.lock().await;
let mut tx = client.begin().await?;
if let Err(e) = match &pipeline.source {
PipelineSource::Realtime(stream_params) => {
let (source_type, stream_org, stream_name, stream_type): (&str, &str, &str, &str) = (
"realtime",
stream_params.org_id.as_str(),
stream_params.stream_name.as_str(),
stream_params.stream_type.as_str(),
);
sqlx::query(
r#"
UPDATE pipeline
SET version = $1, enabled = $2, name = $3, description = $4, org = $5, source_type = $6, stream_org = $7, stream_name = $8, stream_type = $9, nodes = $10, edges = $11
WHERE id = $12;
"#,
)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(stream_org)
.bind(stream_name)
.bind(stream_type)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.bind(&pipeline.id)
.execute(&mut *tx)
.await
}
PipelineSource::Scheduled(derived_stream) => {
let (source_type, derived_stream_str) = (
"scheduled",
json::to_string(&derived_stream)
.expect("Serializing pipeline DerivedStream error"),
);
sqlx::query(
r#"
UPDATE pipeline
SET version = $1, enabled = $2, name = $3, description = $4, org = $5, source_type = $6, derived_stream = $7, nodes = $8, edges = $9
WHERE id = $10;
"#,
)
.bind(pipeline.version)
.bind(pipeline.enabled)
.bind(&pipeline.name)
.bind(&pipeline.description)
.bind(&pipeline.org)
.bind(source_type)
.bind(derived_stream_str)
.bind(json::to_string(&pipeline.nodes).expect("Serializing pipeline nodes error"))
.bind(json::to_string(&pipeline.edges).expect("Serializing pipeline edges error"))
.bind(&pipeline.id)
.execute(&mut *tx)
.await
}
} {
if let Err(e) = tx.rollback().await {
log::error!("[SQLITE] rollback push pipeline error: {}", e);
}
return Err(e.into());
}
if let Err(e) = tx.commit().await {
log::error!("[SQLITE] commit push pipeline error: {}", e);
return Err(e.into());
}
// release lock
drop(client);
Ok(())
}
async fn get_by_stream(&self, stream_params: &StreamParams) -> Result<Pipeline> {
let pool = CLIENT_RO.clone();
let query = r#"
SELECT * FROM pipeline WHERE org = $1 AND source_type = $2 AND stream_org = $3 AND stream_name = $4 AND stream_type = $5;
"#;
let pipeline = sqlx::query_as::<_, Pipeline>(query)
.bind(stream_params.org_id.as_str())
.bind("realtime")
.bind(stream_params.org_id.as_str())
.bind(stream_params.stream_name.as_str())
.bind(stream_params.stream_type.as_str())
.fetch_one(&pool)
.await?;
Ok(pipeline)
}
async fn get_by_id(&self, pipeline_id: &str) -> Result<Pipeline> {
let pool = CLIENT_RO.clone();
let query = "SELECT * FROM pipeline WHERE id = $1;";
let pipeline = match sqlx::query_as::<_, Pipeline>(query)
.bind(pipeline_id)
.fetch_one(&pool)
.await
{
Ok(pipeline) => pipeline,
Err(e) => {
log::error!("[SQLITE] get pipeline by id error: {}", e);
return Err(Error::from(DbError::KeyNotExists(pipeline_id.to_string())));
}
};
Ok(pipeline)
}
async fn get_with_same_source_stream(&self, pipeline: &Pipeline) -> Result<Pipeline> {
let pool = CLIENT_RO.clone();
let similar_pipeline = match &pipeline.source {
PipelineSource::Realtime(stream_params) => {
sqlx::query_as::<_, Pipeline>(
r#"
SELECT * FROM pipeline
WHERE source_type = $1 AND stream_org = $2 AND stream_name = $3 AND stream_type = $4;
"#,
)
.bind("realtime")
.bind(stream_params.org_id.as_str())
.bind(stream_params.stream_name.as_str())
.bind(stream_params.stream_type.as_str())
.fetch_one(&pool)
.await?
}
PipelineSource::Scheduled(_) => {
// only checks for realtime pipelines
return Err(Error::from(DbError::KeyNotExists("".to_string())));
}
};
Ok(similar_pipeline)
}
async fn list(&self) -> Result<Vec<Pipeline>> {
let client = CLIENT_RO.clone();
let query = "SELECT * FROM pipeline ORDER BY id;";
match sqlx::query_as::<_, Pipeline>(query)
.fetch_all(&client)
.await
{
Ok(pipelines) => Ok(pipelines),
Err(e) => {
log::debug!("[SQLITE] list all pipelines error: {}", e);
Ok(vec![])
}
}
}
async fn list_by_org(&self, org: &str) -> Result<Vec<Pipeline>> {
let client = CLIENT_RO.clone();
let query = "SELECT * FROM pipeline WHERE org = $1 ORDER BY id;";
match sqlx::query_as::<_, Pipeline>(query)
.bind(org)
.fetch_all(&client)
.await
{
Ok(pipelines) => Ok(pipelines),
Err(e) => {
log::debug!("[SQLITE] list pipeline by org error: {}", e);
Ok(vec![])
}
}
}
async fn list_streams_with_pipeline(&self, org: &str) -> Result<Vec<Pipeline>> {
let client = CLIENT_RO.clone();
let query = r#"
SELECT * FROM pipeline WHERE org = $1 AND source_type = $2 ORDER BY id;
"#;
match sqlx::query_as::<_, Pipeline>(query)
.bind(org)
.bind("realtime")
.fetch_all(&client)
.await
{
Ok(pipelines) => Ok(pipelines),
Err(e) => {
log::debug!("[SQLITE] list streams with pipelines error: {}", e);
Ok(vec![])
}
}
}
async fn delete(&self, pipeline_id: &str) -> Result<Pipeline> {
let client = CLIENT_RW.clone();
let client = client.lock().await;
let mut tx = client.begin().await?;
let pipeline = sqlx::query_as::<_, Pipeline>("SELECT * FROM pipeline WHERE id = $1;")
.bind(pipeline_id)
.fetch_one(&mut *tx)
.await?;
sqlx::query(r#"DELETE FROM pipeline WHERE id = $1;"#)
.bind(pipeline_id)
.execute(&mut *tx)
.await?;
if let Err(e) = tx.commit().await {
log::error!("[SQLITE] commit delete pipeline error: {}", e);
return Err(e.into());
}
// release lock
drop(client);
Ok(pipeline)
}
}

View File

@ -15,7 +15,12 @@
use std::sync::Arc;
use config::{cluster::LOCAL_NODE, get_config, meta::stream::FileKey, metrics};
use config::{
cluster::LOCAL_NODE,
get_config,
meta::{cluster::CompactionJobType, stream::FileKey},
metrics,
};
use tokio::{
sync::{mpsc, Mutex},
time,
@ -94,6 +99,7 @@ pub async fn run() -> Result<(), anyhow::Error> {
}
tokio::task::spawn(async move { run_generate_job().await });
tokio::task::spawn(async move { run_generate_old_data_job().await });
tokio::task::spawn(async move { run_merge(tx).await });
tokio::task::spawn(async move { run_retention().await });
tokio::task::spawn(async move { run_delay_deletion().await });
@ -138,12 +144,27 @@ async fn run_generate_job() -> Result<(), anyhow::Error> {
loop {
time::sleep(time::Duration::from_secs(get_config().compact.interval)).await;
log::debug!("[COMPACTOR] Running generate merge job");
if let Err(e) = compact::run_generate_job().await {
if let Err(e) = compact::run_generate_job(CompactionJobType::Current).await {
log::error!("[COMPACTOR] run generate merge job error: {e}");
}
}
}
/// Generate merging jobs for old data
async fn run_generate_old_data_job() -> Result<(), anyhow::Error> {
loop {
// run every 1 hour at least
time::sleep(time::Duration::from_secs(
get_config().compact.old_data_interval,
))
.await;
log::debug!("[COMPACTOR] Running generate merge job for old data");
if let Err(e) = compact::run_generate_job(CompactionJobType::Historical).await {
log::error!("[COMPACTOR] run generate merge job for old data error: {e}");
}
}
}
/// Merge small files
async fn run_merge(tx: mpsc::Sender<(MergeSender, MergeBatch)>) -> Result<(), anyhow::Error> {
loop {

View File

@ -22,8 +22,10 @@ use arrow::array::{
use arrow_schema::{DataType, Schema};
use config::{
ider,
meta::stream::{FileMeta, StreamPartition, StreamType},
utils::parquet::new_parquet_writer,
meta::stream::{FileMeta, StreamPartition, StreamPartitionType, StreamType},
utils::{
parquet::new_parquet_writer, record_batch_ext::concat_batches, schema::format_partition_key,
},
FILE_EXT_PARQUET,
};
use hashbrown::HashMap;
@ -46,15 +48,19 @@ fn generate_index_file_name_from_compacted_file(
file_columns[4], file_columns[5], file_columns[6], file_columns[7]
);
let file_name = ider::generate();
format!("files/{stream_key}/{file_date}/{prefix}/{file_name}{FILE_EXT_PARQUET}")
if prefix.is_empty() {
format!("files/{stream_key}/{file_date}/{file_name}{FILE_EXT_PARQUET}")
} else {
format!("files/{stream_key}/{file_date}/{prefix}/{file_name}{FILE_EXT_PARQUET}")
}
}
pub(crate) async fn write_parquet_index_to_disk(
batches: Vec<arrow::record_batch::RecordBatch>,
file_size: u64,
org_id: &str,
stream_name: &str,
stream_type: StreamType,
stream_name: &str,
file_name: &str,
caller: &str,
) -> Result<Vec<(String, FileMeta)>, anyhow::Error> {
@ -69,18 +75,26 @@ pub(crate) async fn write_parquet_index_to_disk(
batches.iter().map(|b| b.num_rows()).sum::<usize>()
);
let stream_settings = infra::schema::get_settings(org_id, stream_name, stream_type)
.await
.unwrap_or_default();
let term_prefix_partition = stream_settings.partition_keys.iter().any(|partition| {
partition.field == "term"
&& partition.types == StreamPartitionType::Prefix
&& !partition.disabled
});
// partition the record batches
let partitioned_batches = generate_prefixed_batches(schema.clone(), batches)?;
let partitioned_batches = if term_prefix_partition {
generate_prefixed_batches(schema.clone(), batches)?
} else {
let mut partitioned_batches = HashMap::new();
let batch = concat_batches(schema.clone(), batches)?;
partitioned_batches.insert("".to_string(), batch);
partitioned_batches
};
for (prefix, batch) in partitioned_batches.iter() {
log::debug!(
"write_parquet_index_to_disk: prefix: {}, row count: {}",
prefix,
batch.num_rows()
);
}
let mut ret = Vec::new();
for (prefix, batch) in partitioned_batches.into_iter() {
// write metadata
let mut file_meta = FileMeta {
@ -140,7 +154,7 @@ pub(crate) async fn write_parquet_index_to_disk(
Ok(ret)
}
/// Generate prefix batches from record batches
/// Generate prefix batches from record batches
fn generate_prefixed_batches(
schema: Arc<Schema>,
batches: Vec<RecordBatch>,
@ -174,7 +188,7 @@ fn generate_prefixed_batches(
let prefix = if term.is_empty() {
String::new()
} else {
partition.get_partition_key(term)
format_partition_key(&partition.get_partition_key(term))
};
let entry = partition_buf.entry(prefix).or_default();
let builder = entry
@ -198,7 +212,7 @@ fn generate_prefixed_batches(
let prefix = if term.is_empty() {
String::new()
} else {
partition.get_partition_key(term)
format_partition_key(&partition.get_partition_key(term))
};
let entry = partition_buf.entry(prefix).or_default();
let builder = entry
@ -219,7 +233,7 @@ fn generate_prefixed_batches(
let prefix = if term.is_empty() {
String::new()
} else {
partition.get_partition_key(term)
format_partition_key(&partition.get_partition_key(term))
};
let entry = partition_buf.entry(prefix).or_default();
let builder = entry
@ -243,7 +257,7 @@ fn generate_prefixed_batches(
let prefix = if term.is_empty() {
String::new()
} else {
partition.get_partition_key(term)
format_partition_key(&partition.get_partition_key(term))
};
let entry = partition_buf.entry(prefix).or_default();
let builder = entry

View File

@ -153,19 +153,19 @@ async fn scan_wal_files(
}
});
let mut files_num = 0;
let mut last_time = start.elapsed().as_millis();
// let mut last_time = start.elapsed().as_millis();
loop {
match rx.recv().await {
None => {
break;
}
Some(files) => {
log::debug!(
"[INGESTER:JOB] scan files get batch: {}, took: {} ms",
files.len(),
start.elapsed().as_millis() - last_time
);
last_time = start.elapsed().as_millis();
// log::debug!(
// "[INGESTER:JOB] scan files get batch: {}, took: {} ms",
// files.len(),
// start.elapsed().as_millis() - last_time
// );
// last_time = start.elapsed().as_millis();
files_num += files.len();
match prepare_files(files).await {
Err(e) => {
@ -268,16 +268,21 @@ async fn prepare_files(
continue;
}
let prefix = file_key[..file_key.rfind('/').unwrap()].to_string();
// remove thread_id from prefix
// eg: files/default/logs/olympics/0/2023/08/21/08/8b8a5451bbe1c44b/
let mut columns = prefix.split('/').collect::<Vec<&str>>();
columns.remove(4);
let prefix = columns.join("/");
let partition = partition_files_with_size.entry(prefix).or_default();
partition.push(FileKey::new(&file_key, parquet_meta, false));
// mark the file as processing
// log::debug!("Processing files created: {:?}", file_key);
PROCESSING_FILES.write().await.insert(file_key);
}
log::debug!(
"[INGESTER:JOB] move files get partitions: {}",
partition_files_with_size.len()
);
// log::debug!(
// "[INGESTER:JOB] move files get partitions: {}",
// partition_files_with_size.len()
// );
Ok(partition_files_with_size)
}
@ -291,13 +296,15 @@ async fn move_files(
return Ok(());
}
let columns = prefix.splitn(5, '/').collect::<Vec<&str>>();
// eg: files/default/logs/olympics/0/2023/08/21/08/8b8a5451bbe1c44b/
// eg: files/default/traces/default/0/2023/09/04/05/default/service_name=ingester/
let columns = prefix.split('/').collect::<Vec<&str>>();
// removed thread_id from prefix, so there is no thread_id in the path
// eg: files/default/logs/olympics/2023/08/21/08/8b8a5451bbe1c44b/
// eg: files/default/traces/default/2023/09/04/05/default/service_name=ingester/
// let _ = columns[0].to_string(); // files/
let org_id = columns[1].to_string();
let stream_type = StreamType::from(columns[2]);
let stream_name = columns[3].to_string();
let prefix_date = format!("{}-{}-{}", columns[4], columns[5], columns[6]);
// log::debug!("[INGESTER:JOB:{thread_id}] check deletion for partition: {}", prefix);
@ -372,6 +379,42 @@ async fn move_files(
return Ok(());
}
// check data retention
let stream_settings = infra::schema::get_settings(&org_id, &stream_name, stream_type)
.await
.unwrap_or_default();
let mut stream_data_retention_days = cfg.compact.data_retention_days;
if stream_settings.data_retention > 0 {
stream_data_retention_days = stream_settings.data_retention;
}
if stream_data_retention_days > 0 {
let date =
config::utils::time::now() - Duration::try_days(stream_data_retention_days).unwrap();
let stream_data_retention_end = date.format("%Y-%m-%d").to_string();
if prefix_date < stream_data_retention_end {
for file in files {
log::warn!(
"[INGESTER:JOB:{thread_id}] the file [{}/{}/{}] was exceed the data retention, just delete file: {}",
&org_id,
stream_type,
&stream_name,
file.key,
);
if let Err(e) = tokio::fs::remove_file(wal_dir.join(&file.key)).await {
log::error!(
"[INGESTER:JOB:{thread_id}] Failed to remove parquet file from disk: {}, {}",
file.key,
e
);
}
// delete metadata from cache
WAL_PARQUET_METADATA.write().await.remove(&file.key);
PROCESSING_FILES.write().await.remove(&file.key);
}
return Ok(());
}
}
// log::debug!("[INGESTER:JOB:{thread_id}] start processing for partition: {}", prefix);
let wal_dir = wal_dir.clone();
@ -422,10 +465,10 @@ async fn move_files(
}
}
log::debug!(
"[INGESTER:JOB:{thread_id}] start merging for partition: {}",
prefix
);
// log::debug!(
// "[INGESTER:JOB:{thread_id}] start merging for partition: {}",
// prefix
// );
// start merge files and upload to s3
loop {
@ -743,8 +786,8 @@ async fn merge_files(
inverted_idx_batch.clone(),
new_file_key.clone(),
&org_id,
&stream_name,
stream_type,
&stream_name,
&full_text_search_fields,
&index_fields,
)
@ -781,8 +824,8 @@ pub(crate) async fn generate_index_on_ingester(
inverted_idx_batch: RecordBatch,
new_file_key: String,
org_id: &str,
stream_name: &str,
stream_type: StreamType,
stream_name: &str,
full_text_search_fields: &[String],
index_fields: &[String],
) -> Result<(), anyhow::Error> {
@ -795,6 +838,7 @@ pub(crate) async fn generate_index_on_ingester(
let record_batches = prepare_index_record_batches(
inverted_idx_batch,
org_id,
stream_type,
stream_name,
&new_file_key,
full_text_search_fields,
@ -950,8 +994,8 @@ pub(crate) async fn generate_index_on_compactor(
inverted_idx_batch: RecordBatch,
new_file_key: String,
org_id: &str,
stream_name: &str,
stream_type: StreamType,
stream_name: &str,
full_text_search_fields: &[String],
index_fields: &[String],
) -> Result<Vec<(String, FileMeta)>, anyhow::Error> {
@ -964,6 +1008,7 @@ pub(crate) async fn generate_index_on_compactor(
let mut record_batches = prepare_index_record_batches(
inverted_idx_batch,
org_id,
stream_type,
stream_name,
&new_file_key,
full_text_search_fields,
@ -974,7 +1019,7 @@ pub(crate) async fn generate_index_on_compactor(
}
let schema = record_batches.first().unwrap().schema();
let prefix_to_remove = format!("files/{}/logs/{}/", org_id, stream_name);
let prefix_to_remove = format!("files/{}/{}/{}/", org_id, stream_type, stream_name);
let len_of_columns_to_invalidate = file_list_to_invalidate.len();
let _timestamp: ArrayRef = Arc::new(Int64Array::from(
@ -1035,8 +1080,8 @@ pub(crate) async fn generate_index_on_compactor(
record_batches,
original_file_size,
org_id,
&index_stream_name,
StreamType::Index,
&index_stream_name,
&new_file_key,
"index_creator",
)
@ -1049,6 +1094,7 @@ pub(crate) async fn generate_index_on_compactor(
fn prepare_index_record_batches(
inverted_idx_batch: RecordBatch,
org_id: &str,
stream_type: StreamType,
stream_name: &str,
new_file_key: &str,
full_text_search_fields: &[String],
@ -1069,7 +1115,7 @@ fn prepare_index_record_batches(
Field::new("segment_ids", DataType::Binary, true), // bitmap
]));
let prefix_to_remove = format!("files/{}/logs/{}/", org_id, stream_name);
let prefix_to_remove = format!("files/{}/{}/{}/", org_id, stream_type, stream_name);
let file_name_without_prefix = new_file_key.trim_start_matches(&prefix_to_remove);
let mut indexed_record_batches_to_merge = Vec::new();

View File

@ -122,9 +122,6 @@ pub async fn init() -> Result<(), anyhow::Error> {
tokio::task::spawn(async move { db::organization::watch().await });
#[cfg(feature = "enterprise")]
tokio::task::spawn(async move { db::ofga::watch().await });
if LOCAL_NODE.is_ingester() {
tokio::task::spawn(async move { db::pipelines::watch().await });
}
#[cfg(feature = "enterprise")]
if !LOCAL_NODE.is_compactor() || LOCAL_NODE.is_single_node() {
@ -168,9 +165,9 @@ pub async fn init() -> Result<(), anyhow::Error> {
db::syslog::cache_syslog_settings()
.await
.expect("syslog settings cache failed");
if LOCAL_NODE.is_ingester() {
db::pipelines::cache().await.expect("syslog cache failed");
}
// cache pipeline
db::pipeline::cache().await.expect("Pipeline cache failed");
// cache file list
if !cfg.common.meta_store_external {

View File

@ -16,7 +16,7 @@
use config::{cluster::LOCAL_NODE, get_config};
use tokio::time;
use crate::service::{compact::stats::update_stats_from_file_list, db, usage};
use crate::service::{compact::stats::update_stats_from_file_list, db};
pub async fn run() -> Result<(), anyhow::Error> {
// tokio::task::spawn(async move { usage_report_stats().await });
@ -25,25 +25,6 @@ pub async fn run() -> Result<(), anyhow::Error> {
Ok(())
}
async fn _usage_report_stats() -> Result<(), anyhow::Error> {
let cfg = get_config();
if !LOCAL_NODE.is_compactor() || !cfg.common.usage_enabled {
return Ok(());
}
// should run it every 10 minutes
let mut interval = time::interval(time::Duration::from_secs(
cfg.limit.calculate_stats_interval,
));
interval.tick().await; // trigger the first run
loop {
interval.tick().await;
if let Err(e) = usage::stats::publish_stats().await {
log::error!("[STATS] run publish stats error: {}", e);
}
}
}
// get stats from file_list to update stream_stats
async fn file_list_update_stats() -> Result<(), anyhow::Error> {
let cfg = get_config();

View File

@ -136,6 +136,7 @@ pub static SMTP_CLIENT: Lazy<AsyncSmtpTransport<Tokio1Executor>> = Lazy::new(||
transport_builder.build()
});
#[allow(clippy::too_many_arguments)]
pub async fn generate_report(
dashboard: &models::ReportDashboard,
org_id: &str,
@ -144,6 +145,7 @@ pub async fn generate_report(
web_url: &str,
timezone: &str,
report_type: ReportType,
report_name: &str,
) -> Result<(Vec<u8>, String), anyhow::Error> {
let dashboard_id = &dashboard.dashboard;
let folder_id = &dashboard.folder;
@ -219,9 +221,9 @@ pub async fn generate_report(
sleep(Duration::from_secs(5)).await;
let timerange = &dashboard.timerange;
let search_type = match report_type.clone() {
ReportType::Cache => "ui",
_ => "reports",
let search_type_params = match report_type.clone() {
ReportType::Cache => "search_type=ui".to_string(),
_ => format!("search_type=reports&report_id={org_id}-{report_name}"),
};
// dashboard link in the email should contain data of the same period as the report
@ -230,7 +232,7 @@ pub async fn generate_report(
let period = &timerange.period;
let (time_duration, time_unit) = period.split_at(period.len() - 1);
let dashb_url = format!(
"{web_url}/dashboards/view?org_identifier={org_id}&dashboard={dashboard_id}&folder={folder_id}&tab={tab_id}&refresh=Off&searchtype={search_type}&period={period}&timezone={timezone}&var-Dynamic+filters=%255B%255D&print=true{dashb_vars}",
"{web_url}/dashboards/view?org_identifier={org_id}&dashboard={dashboard_id}&folder={folder_id}&tab={tab_id}&refresh=Off&{search_type_params}&period={period}&timezone={timezone}&var-Dynamic+filters=%255B%255D&print=true{dashb_vars}",
);
log::debug!("dashb_url for dashboard {folder_id}/{dashboard_id}: {dashb_url}");
@ -281,7 +283,7 @@ pub async fn generate_report(
}
models::ReportTimerangeType::Absolute => {
let url = format!(
"{web_url}/dashboards/view?org_identifier={org_id}&dashboard={dashboard_id}&folder={folder_id}&tab={tab_id}&refresh=Off&searchtype={search_type}&from={}&to={}&timezone={timezone}&var-Dynamic+filters=%255B%255D&print=true{dashb_vars}",
"{web_url}/dashboards/view?org_identifier={org_id}&dashboard={dashboard_id}&folder={folder_id}&tab={tab_id}&refresh=Off&{search_type_params}&from={}&to={}&timezone={timezone}&var-Dynamic+filters=%255B%255D&print=true{dashb_vars}",
&timerange.from, &timerange.to
);
log::debug!("dashb_url for dashboard {folder_id}/{dashboard_id}: {url}");

View File

@ -79,6 +79,7 @@ pub async fn send_report(
&report.email_details.dashb_url,
timezone,
report_type.clone(),
&report_name,
)
.await
{

View File

@ -19,10 +19,19 @@ use std::{
};
use actix_web::http;
use async_trait::async_trait;
use chrono::{Duration, Local, TimeZone, Timelike, Utc};
use config::{
get_config,
meta::stream::{StreamParams, StreamType},
meta::{
alerts::{
alert::{Alert, AlertListFilter},
destinations::{DestinationType, DestinationWithTemplate, HTTPType},
FrequencyType, Operator, QueryType,
},
search::{SearchEventContext, SearchEventType},
stream::StreamType,
},
utils::{
base64,
json::{Map, Value},
@ -34,18 +43,11 @@ use lettre::{message::SinglePart, AsyncTransport, Message};
use crate::{
common::{
meta::{
alerts::{
alert::{Alert, AlertListFilter},
destinations::{DestinationType, DestinationWithTemplate, HTTPType},
FrequencyType, Operator, QueryType,
},
authz::Authz,
},
meta::authz::Authz,
utils::auth::{is_ofga_unsupported, remove_ownership, set_ownership},
},
service::{
alerts::{build_sql, destinations},
alerts::{build_sql, destinations, QueryConditionExt},
db,
search::sql::RE_ONLY_SELECT,
short_url,
@ -343,10 +345,29 @@ pub async fn trigger(
.map_err(|e| (http::StatusCode::INTERNAL_SERVER_ERROR, e))
}
impl Alert {
#[async_trait]
pub trait AlertExt: Sync + Send + 'static {
/// Returns the evaluated row data and the end time of the search timerange,
/// for realtime this is 0. `start_time` is the start time of the search timerange.
pub async fn evaluate(
async fn evaluate(
&self,
row: Option<&Map<String, Value>>,
start_time: Option<i64>,
) -> Result<(Option<Vec<Map<String, Value>>>, i64), anyhow::Error>;
/// Returns a tuple containing a boolean - if all the send notification jobs succeeded
/// and the error message if any
async fn send_notification(
&self,
rows: &[Map<String, Value>],
rows_end_time: i64,
start_time: Option<i64>,
) -> Result<(String, String), anyhow::Error>;
}
#[async_trait]
impl AlertExt for Alert {
async fn evaluate(
&self,
row: Option<&Map<String, Value>>,
start_time: Option<i64>,
@ -354,19 +375,25 @@ impl Alert {
if self.is_real_time {
self.query_condition.evaluate_realtime(row).await
} else {
let search_event_ctx = SearchEventContext::with_alert(Some(format!(
"/alerts/{}/{}/{}/{}",
self.org_id, self.stream_type, self.stream_name, self.name
)));
self.query_condition
.evaluate_scheduled(
&self.get_stream_params(),
&self.org_id,
Some(&self.stream_name),
self.stream_type,
&self.trigger_condition,
start_time,
Some(SearchEventType::Alerts),
Some(search_event_ctx),
)
.await
}
}
/// Returns a tuple containing a boolean - if all the send notification jobs succeeded
/// and the error message if any
pub async fn send_notification(
async fn send_notification(
&self,
rows: &[Map<String, Value>],
rows_end_time: i64,
@ -406,14 +433,6 @@ impl Alert {
Ok((success_message, err_message))
}
}
pub fn get_stream_params(&self) -> StreamParams {
StreamParams {
org_id: self.org_id.clone().into(),
stream_name: self.stream_name.clone().into(),
stream_type: self.stream_type,
}
}
}
pub async fn send_notification(
@ -811,7 +830,9 @@ async fn process_dest_template(
QueryType::Custom => {
if let Some(conditions) = &alert.query_condition.conditions {
if let Ok(v) = build_sql(
&alert.get_stream_params(),
&alert.org_id,
&alert.stream_name,
alert.stream_type,
&alert.query_condition,
conditions,
)

View File

@ -15,46 +15,28 @@
use std::str::FromStr;
use async_trait::async_trait;
use chrono::Utc;
use config::{
get_config,
meta::{
alerts::{FrequencyType, QueryType},
pipeline::components::DerivedStream,
search::{SearchEventContext, SearchEventType},
},
utils::json::{Map, Value},
};
use cron::Schedule;
use hashbrown::HashMap;
use crate::{
common::{
meta::{
// authz::Authz,
alerts::{derived_streams::DerivedStreamMeta, FrequencyType, QueryType},
},
// utils::auth::{remove_ownership, set_ownership},
},
service::db,
};
use crate::service::{alerts::QueryConditionExt, db};
pub async fn save(
mut derived_stream: DerivedStreamMeta,
mut derived_stream: DerivedStream,
pipeline_name: &str,
pipeline_id: &str,
) -> Result<(), anyhow::Error> {
derived_stream.name = derived_stream.name.trim().to_string();
// 1. Start validate DerivedStream
if !derived_stream.is_valid() {
return Err(anyhow::anyhow!(
"DerivedStream Name, destination, and Trigger period required"
));
}
// query_type for realtime
if derived_stream.is_real_time && derived_stream.query_condition.query_type != QueryType::Custom
{
return Err(anyhow::anyhow!(
"Realtime DerivedStream should use Custom query type"
));
}
// other checks for query type
// checks for query type
match derived_stream.query_condition.query_type {
QueryType::SQL => {
if derived_stream
@ -82,22 +64,7 @@ pub async fn save(
}
}
_ => {}
}
// check source stream schema
let schema = infra::schema::get(
&derived_stream.source.org_id,
&derived_stream.source.stream_name,
derived_stream.source.stream_type,
)
.await?;
if schema.fields().is_empty() {
return Err(anyhow::anyhow!(
"Source Stream {}/{} schema is empty.",
derived_stream.source.org_id,
derived_stream.source.stream_name,
));
}
};
// End input validation
// 2. update the frequency
@ -110,17 +77,9 @@ pub async fn save(
std::cmp::max(1, get_config().limit.derived_stream_schedule_interval / 60);
}
// 3. clean up DerivedStream context attributes
if let Some(attrs) = &derived_stream.context_attributes {
let mut new_attrs = HashMap::with_capacity(attrs.len());
for (key, val) in attrs.iter() {
new_attrs.insert(key.trim().to_string(), val.to_string());
}
derived_stream.context_attributes = Some(new_attrs);
}
// test derived_stream
if let Err(e) = &derived_stream.evaluate(None, None).await {
let trigger_module_key = derived_stream.get_scheduler_module_key(pipeline_name, pipeline_id);
if let Err(e) = &derived_stream.evaluate(None, &trigger_module_key).await {
return Err(anyhow::anyhow!(
"DerivedStream not saved due to failed test run caused by {}",
e.to_string()
@ -130,11 +89,11 @@ pub async fn save(
// Save the trigger to db
let next_run_at = Utc::now().timestamp_micros();
let trigger = db::scheduler::Trigger {
org: derived_stream.source.org_id.to_string(),
org: derived_stream.org_id.to_string(),
module: db::scheduler::TriggerModule::DerivedStream,
module_key: derived_stream.get_scheduler_module_key(pipeline_name),
module_key: trigger_module_key,
next_run_at,
is_realtime: derived_stream.is_real_time,
is_realtime: false,
is_silenced: false,
..Default::default()
};
@ -150,45 +109,55 @@ pub async fn save(
}
pub async fn delete(
derived_stream: DerivedStreamMeta,
derived_stream: DerivedStream,
pipeline_name: &str,
pipeline_id: &str,
) -> Result<(), anyhow::Error> {
db::scheduler::delete(
&derived_stream.source.org_id,
&derived_stream.org_id,
db::scheduler::TriggerModule::DerivedStream,
&derived_stream.get_scheduler_module_key(pipeline_name),
&derived_stream.get_scheduler_module_key(pipeline_name, pipeline_id),
)
.await
.map_err(|e| anyhow::anyhow!("Error deleting derived stream trigger: {e}"))
}
impl DerivedStreamMeta {
pub fn is_valid(&self) -> bool {
!self.name.is_empty()
&& !self.is_real_time // TODO(taiming): support realtime DerivedStream
&& self.source.is_valid()
&& self.destination.is_valid()
&& self.trigger_condition.period != 0
}
#[async_trait]
pub trait DerivedStreamExt: Sync + Send + 'static {
fn get_scheduler_module_key(&self, pipeline_name: &str, pipeline_id: &str) -> String;
async fn evaluate(
&self,
start_time: Option<i64>,
module_key: &str,
) -> Result<(Option<Vec<Map<String, Value>>>, i64), anyhow::Error>;
}
pub fn get_scheduler_module_key(&self, pipeline_name: &str) -> String {
#[async_trait]
impl DerivedStreamExt for DerivedStream {
fn get_scheduler_module_key(&self, pipeline_name: &str, pipeline_id: &str) -> String {
format!(
"{}/{}/{}/{}",
self.source.stream_type, self.source.stream_name, pipeline_name, self.name
self.stream_type, self.org_id, pipeline_name, pipeline_id
)
}
pub async fn evaluate(
async fn evaluate(
&self,
row: Option<&Map<String, Value>>,
start_time: Option<i64>,
module_key: &str,
) -> Result<(Option<Vec<Map<String, Value>>>, i64), anyhow::Error> {
if self.is_real_time {
self.query_condition.evaluate_realtime(row).await
} else {
self.query_condition
.evaluate_scheduled(&self.source, &self.trigger_condition, start_time)
.await
}
self.query_condition
.evaluate_scheduled(
&self.org_id,
None,
self.stream_type,
&self.trigger_condition,
start_time,
Some(SearchEventType::DerivedStream),
Some(SearchEventContext::with_derived_stream(Some(
module_key.to_string(),
))),
)
.await
}
}

View File

@ -14,14 +14,12 @@
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use actix_web::http;
use config::meta::alerts::destinations::{Destination, DestinationType, DestinationWithTemplate};
use crate::{
common::{
infra::config::STREAM_ALERTS,
meta::{
alerts::destinations::{Destination, DestinationType, DestinationWithTemplate},
authz::Authz,
},
meta::authz::Authz,
utils::auth::{is_ofga_unsupported, remove_ownership, set_ownership},
},
service::db::{self, user},

View File

@ -15,12 +15,14 @@
use alert::to_float;
use arrow_schema::DataType;
use async_trait::async_trait;
use chrono::{Duration, Utc};
use config::{
get_config, ider,
meta::{
search::{SearchEventType, SqlQuery},
stream::StreamParams,
alerts::{AggFunction, Condition, Operator, QueryCondition, QueryType, TriggerCondition},
search::{SearchEventContext, SearchEventType, SqlQuery},
stream::StreamType,
},
utils::{
base64,
@ -29,12 +31,7 @@ use config::{
};
use super::promql;
use crate::{
common::meta::alerts::{
AggFunction, Condition, Operator, QueryCondition, QueryType, TriggerCondition,
},
service::search as SearchService,
};
use crate::service::search as SearchService;
pub mod alert;
pub mod derived_streams;
@ -42,8 +39,29 @@ pub mod destinations;
pub mod scheduler;
pub mod templates;
impl QueryCondition {
pub async fn evaluate_realtime(
#[async_trait]
pub trait QueryConditionExt: Sync + Send + 'static {
async fn evaluate_realtime(
&self,
row: Option<&Map<String, Value>>,
) -> Result<(Option<Vec<Map<String, Value>>>, i64), anyhow::Error>;
#[allow(clippy::too_many_arguments)]
async fn evaluate_scheduled(
&self,
org_id: &str,
stream_name: Option<&str>,
stream_type: StreamType,
trigger_condition: &TriggerCondition,
start_time: Option<i64>,
search_type: Option<SearchEventType>,
search_event_context: Option<SearchEventContext>,
) -> Result<(Option<Vec<Map<String, Value>>>, i64), anyhow::Error>;
}
#[async_trait]
impl QueryConditionExt for QueryCondition {
async fn evaluate_realtime(
&self,
row: Option<&Map<String, Value>>,
) -> Result<(Option<Vec<Map<String, Value>>>, i64), anyhow::Error> {
@ -69,19 +87,25 @@ impl QueryCondition {
Ok((Some(vec![row.to_owned()]), now))
}
pub async fn evaluate_scheduled(
async fn evaluate_scheduled(
&self,
stream_param: &StreamParams,
org_id: &str,
stream_name: Option<&str>,
stream_type: StreamType,
trigger_condition: &TriggerCondition,
start_time: Option<i64>,
search_type: Option<SearchEventType>,
search_event_context: Option<SearchEventContext>,
) -> Result<(Option<Vec<Map<String, Value>>>, i64), anyhow::Error> {
let now = Utc::now().timestamp_micros();
let sql = match self.query_type {
QueryType::Custom => {
let Some(v) = self.conditions.as_ref() else {
let (Some(stream_name), Some(v)) = (stream_name, self.conditions.as_ref()) else {
// CustomQuery type needs to provide source StreamName.
// CustomQuery is only used by Alerts' triggers.
return Ok((None, now));
};
build_sql(stream_param, self, v).await?
build_sql(org_id, stream_name, stream_type, self, v).await?
}
QueryType::SQL => {
let Some(v) = self.sql.as_ref() else {
@ -127,7 +151,7 @@ impl QueryCondition {
(end - start) / promql::MAX_DATA_POINTS,
),
};
let resp = match promql::search::search(&stream_param.org_id, &req, 0, "").await {
let resp = match promql::search::search(org_id, &req, 0, "").await {
Ok(v) => v,
Err(_) => {
return Ok((None, now));
@ -259,7 +283,8 @@ impl QueryCondition {
regions: vec![],
clusters: vec![],
timeout: 0,
search_type: Some(SearchEventType::Alerts),
search_type,
search_event_context,
from: 0,
size,
start_time: 0, // ignored
@ -275,14 +300,7 @@ impl QueryCondition {
per_query_response: false, // Will return results in single array
};
SearchService::search_multi(
&trace_id,
&stream_param.org_id,
stream_param.stream_type,
None,
&req,
)
.await
SearchService::search_multi(&trace_id, org_id, stream_type, None, &req).await
} else {
// fire the query
let req = config::meta::search::Request {
@ -316,18 +334,11 @@ impl QueryCondition {
regions: vec![],
clusters: vec![],
timeout: 0,
search_type: Some(SearchEventType::Alerts), /* TODO(taiming): change the name to
* scheduled & inform FE */
search_type,
search_event_context,
index_type: "".to_string(),
};
SearchService::search(
&trace_id,
&stream_param.org_id,
stream_param.stream_type,
None,
&req,
)
.await
SearchService::search(&trace_id, org_id, stream_type, None, &req).await
};
// Resp hits can be of two types -
@ -406,8 +417,14 @@ impl QueryCondition {
}
}
impl Condition {
pub async fn evaluate(&self, row: &Map<String, Value>) -> bool {
#[async_trait]
pub trait ConditionExt: Sync + Send + 'static {
async fn evaluate(&self, row: &Map<String, Value>) -> bool;
}
#[async_trait]
impl ConditionExt for Condition {
async fn evaluate(&self, row: &Map<String, Value>) -> bool {
let val = match row.get(&self.column) {
Some(val) => val,
None => {
@ -473,16 +490,13 @@ impl Condition {
}
async fn build_sql(
stream_params: &StreamParams,
org_id: &str,
stream_name: &str,
stream_type: StreamType,
query_condition: &QueryCondition,
conditions: &[Condition],
) -> Result<String, anyhow::Error> {
let schema = infra::schema::get(
&stream_params.org_id,
&stream_params.stream_name,
stream_params.stream_type,
)
.await?;
let schema = infra::schema::get(org_id, stream_name, stream_type).await?;
let mut wheres = Vec::with_capacity(conditions.len());
for cond in conditions.iter() {
let data_type = match schema.field_with_name(&cond.column) {
@ -491,7 +505,7 @@ async fn build_sql(
return Err(anyhow::anyhow!(
"Column {} not found on stream {}",
&cond.column,
&stream_params.stream_name
stream_name
));
}
};
@ -504,10 +518,7 @@ async fn build_sql(
String::new()
};
if query_condition.aggregation.is_none() {
return Ok(format!(
"SELECT * FROM \"{}\" {}",
stream_params.stream_name, where_sql
));
return Ok(format!("SELECT * FROM \"{}\" {}", stream_name, where_sql));
}
// handle aggregation
@ -520,7 +531,7 @@ async fn build_sql(
return Err(anyhow::anyhow!(
"Aggregation column {} not found on stream {}",
&agg.having.column,
&stream_params.stream_name
&stream_name
));
}
};
@ -550,7 +561,7 @@ async fn build_sql(
func_expr,
cfg.common.column_timestamp,
cfg.common.column_timestamp,
stream_params.stream_name,
stream_name,
where_sql,
group.join(", "),
having_expr
@ -563,7 +574,7 @@ async fn build_sql(
func_expr,
cfg.common.column_timestamp,
cfg.common.column_timestamp,
stream_params.stream_name,
stream_name,
where_sql,
having_expr
);

View File

@ -13,16 +13,21 @@
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use std::{str::FromStr, time::Instant};
use std::{collections::HashMap, str::FromStr, time::Instant};
use chrono::{Duration, FixedOffset, Utc};
use config::{
get_config,
meta::{
stream::StreamType,
alerts::FrequencyType,
stream::{StreamParams, StreamType},
usage::{TriggerData, TriggerDataStatus, TriggerDataType},
},
utils::{json, rand::get_rand_num_within},
utils::{
json,
rand::get_rand_num_within,
time::{hour_micros, second_micros},
},
};
use cron::Schedule;
use futures::future::try_join_all;
@ -30,11 +35,15 @@ use infra::scheduler::get_scheduler_max_retries;
use proto::cluster_rpc;
use crate::{
common::meta::{alerts::FrequencyType, dashboards::reports::ReportFrequencyType},
common::meta::dashboards::reports::ReportFrequencyType,
service::{
alerts::alert::{get_alert_start_end_time, get_row_column_map},
alerts::{
alert::{get_alert_start_end_time, get_row_column_map, AlertExt},
derived_streams::DerivedStreamExt,
},
db::{self, scheduler::ScheduledTriggerData},
ingestion::ingestion_service,
pipeline::batch_execution::ExecutablePipeline,
usage::publish_triggers_usage,
},
};
@ -86,11 +95,8 @@ fn get_max_considerable_delay(frequency: i64) -> i64 {
// If the delay is more than this, the alert will be skipped.
// The maximum delay is the lowest of 1 hour or 20% of the frequency.
// E.g. if the frequency is 5 mins, the maximum delay is 1 min.
let frequency = Duration::try_seconds(frequency)
.unwrap()
.num_microseconds()
.unwrap();
let max_delay = Duration::try_hours(1).unwrap().num_microseconds().unwrap();
let frequency = second_micros(frequency);
let max_delay = hour_micros(1);
// limit.alert_considerable_delay is in percentage, convert into float
let considerable_delay = get_config().limit.alert_considerable_delay as f64 * 0.01;
let max_considerable_delay = (frequency as f64 * considerable_delay) as i64;
@ -773,64 +779,14 @@ async fn handle_derived_stream_triggers(
);
let (_, max_retries) = get_scheduler_max_retries();
// module_key format: stream_type/stream_name/pipeline_name/derived_stream_name
// module_key format: stream_type/org_id/pipeline_name/pipeline_id
let columns = trigger.module_key.split('/').collect::<Vec<_>>();
assert_eq!(columns.len(), 4);
let org_id = &trigger.org;
let stream_type: StreamType = columns[0].into();
let stream_name = columns[1];
let org_id = columns[1];
let pipeline_name = columns[2];
let name = columns[3];
let pipeline_id = columns[3];
let is_real_time = trigger.is_realtime;
let is_silenced = trigger.is_silenced;
if is_real_time && is_silenced {
log::debug!(
"Realtime derived_stream needs to wake up, {}/{}",
org_id,
trigger.module_key
);
let new_trigger = db::scheduler::Trigger {
next_run_at: Utc::now().timestamp_micros(),
is_silenced: false,
status: db::scheduler::TriggerStatus::Waiting,
..trigger.clone()
};
db::scheduler::update_trigger(new_trigger).await?;
return Ok(());
}
let Ok(pipeline) = db::pipelines::get(org_id, stream_type, stream_name, pipeline_name).await
else {
return Err(anyhow::anyhow!(
"Pipeline associated with trigger not found: {}/{}/{}/{}",
org_id,
stream_name,
stream_type,
pipeline_name
));
};
let Some(derived_stream) = pipeline
.derived_streams
.and_then(|ds| ds.into_iter().find(|ds| ds.name == name))
else {
return Err(anyhow::anyhow!(
"DerivedStream associated with the trigger not found in pipeline: {}/{}/{}/{}",
org_id,
stream_name,
stream_type,
name,
));
};
let trigger_data: Option<ScheduledTriggerData> = json::from_str(&trigger.data).ok();
let start_time = if let Some(trigger_data) = trigger_data {
trigger_data
.period_end_time
.map(|period_end_time| period_end_time + 1)
} else {
None
};
let mut new_trigger = db::scheduler::Trigger {
next_run_at: Utc::now().timestamp_micros(),
is_silenced: false,
@ -839,6 +795,53 @@ async fn handle_derived_stream_triggers(
..trigger.clone()
};
let Ok(pipeline) = db::pipeline::get_by_id(pipeline_id).await else {
log::warn!(
"Pipeline associated with trigger not found: {}/{}/{}/{}. Deleting this trigger",
org_id,
stream_type,
pipeline_name,
pipeline_id
);
db::scheduler::delete(&trigger.org, trigger.module, &trigger.module_key).await?;
return Err(anyhow::anyhow!(
"Pipeline associated with trigger not found: {}/{}/{}/{}",
org_id,
stream_type,
pipeline_name,
pipeline_id
));
};
if !pipeline.enabled {
// remove the trigger from scheduler. Trigger will be added back when the pipeline is
// enabled again
log::info!("Pipeline associated with trigger not enabled. Removing trigger from Scheduler");
db::scheduler::delete(&trigger.org, trigger.module, &trigger.module_key).await?;
return Ok(());
}
let Some(derived_stream) = pipeline.get_derived_stream() else {
return Err(anyhow::anyhow!(
"DerivedStream associated with the trigger not found in pipeline: {}/{}/{}",
org_id,
pipeline_name,
pipeline_id,
));
};
let start_time = if trigger.data.is_empty() {
None
} else {
let trigger_data: Option<ScheduledTriggerData> = json::from_str(&trigger.data).ok();
if let Some(trigger_data) = trigger_data {
trigger_data
.period_end_time
.map(|period_end_time| period_end_time + 1)
} else {
None
}
};
if trigger.retries >= max_retries {
// It has been tried the maximum time, just update the
// next_run_at to the next expected trigger time
@ -869,7 +872,9 @@ async fn handle_derived_stream_triggers(
}
// evaluate trigger and configure trigger next run time
let (ret, end_time) = derived_stream.evaluate(None, start_time).await?;
let (ret, end_time) = derived_stream
.evaluate(start_time, &trigger.module_key)
.await?;
if ret.is_some() {
log::info!(
"DerivedStream conditions satisfied, org: {}, module_key: {}",
@ -962,36 +967,87 @@ async fn handle_derived_stream_triggers(
db::scheduler::update_trigger(new_trigger).await?;
trigger_data_stream.status = TriggerDataStatus::ConditionNotSatisfied;
} else {
// Ingest result into destination stream
let (org_id, stream_name, stream_type): (String, String, i32) = {
(
derived_stream.destination.org_id.into(),
derived_stream.destination.stream_name.into(),
cluster_rpc::StreamType::from(derived_stream.destination.stream_type).into(),
)
};
let req = cluster_rpc::IngestionRequest {
org_id: org_id.clone(),
stream_name: stream_name.clone(),
stream_type,
data: Some(cluster_rpc::IngestionData::from(local_val)),
ingestion_type: Some(cluster_rpc::IngestionType::Json.into()), /* TODO(taiming): finalize IngestionType for derived_stream */
};
match ingestion_service::ingest(&org_id, req).await {
Ok(resp) if resp.status_code == 200 => {
log::info!(
"DerivedStream result ingested to destination {org_id}/{stream_name}/{stream_type}",
// pass search results to pipeline to get modified results before ingesting
let mut json_data_by_stream: HashMap<StreamParams, Vec<json::Value>> = HashMap::new();
let mut ingestion_error_msg = None;
match ExecutablePipeline::new(&pipeline).await {
Err(e) => {
let err_msg = format!(
"Pipeline: {}/{} associated with the DerivedStream failed to initialize ExecutablePipeline. Caused by: {}",
org_id, pipeline_name, e
);
db::scheduler::update_trigger(new_trigger).await?;
log::error!("{err_msg}");
ingestion_error_msg = Some(err_msg);
}
error => {
let err = error.map_or_else(|e| e.to_string(), |resp| resp.message);
log::error!(
"Error in ingesting DerivedStream result to destination {:?}, org: {}, module_key: {}",
err,
new_trigger.org,
new_trigger.module_key
);
Ok(exec_pl) => match exec_pl.process_batch(org_id, local_val).await {
Err(e) => {
let err_msg = format!(
"DerivedStream query results failed to pass through the associated pipeline: {}/{}. Caused by: {}",
org_id, pipeline_name, e
);
log::error!("{err_msg}");
ingestion_error_msg = Some(err_msg);
}
Ok(pl_results) => {
for (stream_params, stream_pl_results) in pl_results {
if matches!(
stream_params.stream_type,
StreamType::Logs | StreamType::EnrichmentTables
) {
let (_, results): (Vec<_>, Vec<_>) =
stream_pl_results.into_iter().unzip();
json_data_by_stream
.entry(stream_params)
.or_default()
.extend(results);
}
}
}
},
};
// Ingest result into destination stream
if ingestion_error_msg.is_none() {
for (dest_stream, records) in json_data_by_stream {
let (org_id, stream_name, stream_type): (String, String, i32) = {
(
dest_stream.org_id.into(),
dest_stream.stream_name.into(),
cluster_rpc::StreamType::from(dest_stream.stream_type).into(),
)
};
let req = cluster_rpc::IngestionRequest {
org_id: org_id.clone(),
stream_name: stream_name.clone(),
stream_type,
data: Some(cluster_rpc::IngestionData::from(records)),
ingestion_type: Some(cluster_rpc::IngestionType::Json.into()),
};
match ingestion_service::ingest(&org_id, req).await {
Ok(resp) if resp.status_code == 200 => {
log::info!(
"DerivedStream result ingested to destination {org_id}/{stream_name}/{stream_type}",
);
}
error => {
let err = error.map_or_else(|e| e.to_string(), |resp| resp.message);
log::error!(
"Error in ingesting DerivedStream result to destination {:?}, org: {}, module_key: {}",
err,
new_trigger.org,
new_trigger.module_key
);
ingestion_error_msg = Some(err);
break;
}
};
}
}
match ingestion_error_msg {
None => db::scheduler::update_trigger(new_trigger).await?,
Some(err) => {
if trigger.retries + 1 >= max_retries {
// It has been tried the maximum time, just update the
// next_run_at to the next expected trigger time

View File

@ -14,11 +14,12 @@
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use actix_web::http;
use config::meta::alerts::templates::Template;
use crate::{
common::{
infra::config::ALERTS_DESTINATIONS,
meta::{alerts::templates::Template, authz::Authz},
meta::authz::Authz,
utils::auth::{is_ofga_unsupported, remove_ownership, set_ownership},
},
service::db,

View File

@ -20,7 +20,12 @@ use std::{
use bytes::Buf;
use chrono::{DateTime, Datelike, Duration, TimeZone, Timelike, Utc};
use config::{cluster::LOCAL_NODE, ider, meta::stream::FileKey, utils::json};
use config::{
cluster::LOCAL_NODE,
ider,
meta::stream::FileKey,
utils::{json, time::hour_micros},
};
use hashbrown::HashMap;
use infra::{dist_lock, schema::STREAM_SCHEMAS_LATEST, storage};
use tokio::sync::{RwLock, Semaphore};
@ -106,7 +111,7 @@ pub async fn run_merge(offset: i64) -> Result<(), anyhow::Error> {
// offset
let mut is_waiting_streams = false;
for (key, val) in offsets {
if (val - Duration::try_hours(1).unwrap().num_microseconds().unwrap()) < offset {
if (val - hour_micros(1)) < offset {
log::info!("[COMPACT] file_list is waiting for stream: {key}, offset: {val}");
is_waiting_streams = true;
break;
@ -162,7 +167,7 @@ pub async fn run_merge(offset: i64) -> Result<(), anyhow::Error> {
merge_file_list(offset).await?;
// write new sync offset
offset = offset_time_hour + Duration::try_hours(1).unwrap().num_microseconds().unwrap();
offset = offset_time_hour + hour_micros(1);
db::compact::file_list::set_offset(offset).await
}

View File

@ -16,7 +16,8 @@
use std::io::{BufRead, BufReader};
use bytes::Buf;
use chrono::{DateTime, Duration, TimeZone, Utc};
use chrono::{DateTime, TimeZone, Utc};
use config::utils::time::hour_micros;
use futures::future::try_join_all;
use hashbrown::HashMap;
use infra::{file_list as infra_file_list, storage};
@ -201,7 +202,7 @@ async fn query_deleted_from_s3(
let entry = files.entry(file).or_insert_with(Vec::new);
entry.extend(records);
}
cur_time += Duration::try_hours(1).unwrap().num_microseconds().unwrap();
cur_time += hour_micros(1);
}
Ok(files)
}

View File

@ -34,6 +34,7 @@ use config::{
},
record_batch_ext::{concat_batches, merge_record_batches},
schema_ext::SchemaExt,
time::hour_micros,
},
FILE_EXT_PARQUET,
};
@ -143,26 +144,18 @@ pub async fn generate_job_by_stream(
)
.unwrap()
.timestamp_micros();
// 1. if step_secs less than 1 hour, must wait for at least max_file_retention_time
// 2. if step_secs greater than 1 hour, must wait for at least 3 * max_file_retention_time
// must wait for at least 3 * max_file_retention_time
// -- first period: the last hour local file upload to storage, write file list
// -- second period, the last hour file list upload to storage
// -- third period, we can do the merge, so, at least 3 times of
// max_file_retention_time
if (cfg.compact.step_secs < 3600
&& time_now.timestamp_micros() - offset
if offset >= time_now_hour
|| time_now.timestamp_micros() - offset
<= Duration::try_seconds(cfg.limit.max_file_retention_time as i64)
.unwrap()
.num_microseconds()
.unwrap())
|| (cfg.compact.step_secs >= 3600
&& (offset >= time_now_hour
|| time_now.timestamp_micros() - offset
<= Duration::try_seconds(cfg.limit.max_file_retention_time as i64)
.unwrap()
.num_microseconds()
.unwrap()
* 3))
.unwrap()
* 3
{
return Ok(()); // the time is future, just wait
}
@ -184,11 +177,9 @@ pub async fn generate_job_by_stream(
}
// write new offset
let offset = offset
+ Duration::try_seconds(cfg.compact.step_secs)
.unwrap()
.num_microseconds()
.unwrap();
let offset = offset + hour_micros(1);
// format to hour with zero minutes, seconds
let offset = offset - offset % hour_micros(1);
db::compact::files::set_offset(
org_id,
stream_type,
@ -201,6 +192,113 @@ pub async fn generate_job_by_stream(
Ok(())
}
/// Generate merging job by stream
/// 1. get old data by hour
/// 2. check if other node is processing
/// 3. create job or return
pub async fn generate_old_data_job_by_stream(
org_id: &str,
stream_type: StreamType,
stream_name: &str,
) -> Result<(), anyhow::Error> {
// get last compacted offset
let (offset, node) = db::compact::files::get_offset(org_id, stream_type, stream_name).await;
if !node.is_empty() && LOCAL_NODE.uuid.ne(&node) && get_node_by_uuid(&node).await.is_some() {
return Ok(()); // other node is processing
}
if node.is_empty() || LOCAL_NODE.uuid.ne(&node) {
let lock_key = format!("/compact/merge/{}/{}/{}", org_id, stream_type, stream_name);
let locker = dist_lock::lock(&lock_key, 0, None).await?;
// check the working node again, maybe other node locked it first
let (offset, node) = db::compact::files::get_offset(org_id, stream_type, stream_name).await;
if !node.is_empty() && LOCAL_NODE.uuid.ne(&node) && get_node_by_uuid(&node).await.is_some()
{
dist_lock::unlock(&locker).await?;
return Ok(()); // other node is processing
}
// set to current node
let ret = db::compact::files::set_offset(
org_id,
stream_type,
stream_name,
offset,
Some(&LOCAL_NODE.uuid.clone()),
)
.await;
dist_lock::unlock(&locker).await?;
drop(locker);
ret?;
}
if offset == 0 {
return Ok(()); // no data
}
let cfg = get_config();
let stream_settings = infra::schema::get_settings(org_id, stream_name, stream_type)
.await
.unwrap_or_default();
let mut stream_data_retention_days = cfg.compact.data_retention_days;
if stream_settings.data_retention > 0 {
stream_data_retention_days = stream_settings.data_retention;
}
if stream_data_retention_days > cfg.compact.old_data_max_days {
stream_data_retention_days = cfg.compact.old_data_max_days;
}
if stream_data_retention_days == 0 {
return Ok(()); // no need to check old data
}
// get old data by hour, `offset - 2 hours` as old data
let end_time = offset - hour_micros(2);
let start_time = end_time
- Duration::try_days(stream_data_retention_days as i64)
.unwrap()
.num_microseconds()
.unwrap();
let hours = infra_file_list::query_old_data_hours(
org_id,
stream_type,
stream_name,
Some((start_time, end_time)),
)
.await?;
// generate merging job
for hour in hours {
let column = hour.split('/').collect::<Vec<_>>();
if column.len() != 4 {
return Err(anyhow::anyhow!(
"Unexpected hour format in {}, Expected format YYYY/MM/DD/HH",
hour
));
}
let offset = DateTime::parse_from_rfc3339(&format!(
"{}-{}-{}T{}:00:00Z",
column[0], column[1], column[2], column[3]
))?
.with_timezone(&Utc);
let offset = offset.timestamp_micros();
log::debug!(
"[COMPACTOR] generate_old_data_job_by_stream [{}/{}/{}] hours: {}, offset: {}",
org_id,
stream_type,
stream_name,
hour,
offset
);
if let Err(e) = infra_file_list::add_job(org_id, stream_type, stream_name, offset).await {
return Err(anyhow::anyhow!(
"[COMAPCT] add file_list_jobs for old data failed: {}",
e
));
}
}
Ok(())
}
/// compactor run steps on a stream:
/// 3. get a cluster lock for compactor stream
/// 4. read last compacted offset: year/month/day/hour
@ -277,17 +375,11 @@ pub async fn merge_by_stream(
// get current hour(day) all files
let (partition_offset_start, partition_offset_end) =
if partition_time_level == PartitionTimeLevel::Daily {
(
offset_time_day,
offset_time_day + Duration::try_hours(24).unwrap().num_microseconds().unwrap() - 1,
)
(offset_time_day, offset_time_day + hour_micros(24) - 1)
} else {
(
offset_time_hour,
offset_time_hour + Duration::try_hours(1).unwrap().num_microseconds().unwrap() - 1,
)
(offset_time_hour, offset_time_hour + hour_micros(1) - 1)
};
let mut files = file_list::query(
let files = file_list::query(
org_id,
stream_name,
stream_type,
@ -298,31 +390,6 @@ pub async fn merge_by_stream(
.await
.map_err(|e| anyhow::anyhow!("query file list failed: {}", e))?;
// check lookback files
if cfg.compact.lookback_hours > 0 {
let lookback_offset = Duration::try_hours(cfg.compact.lookback_hours)
.unwrap()
.num_microseconds()
.unwrap();
let lookback_offset_start = partition_offset_start - lookback_offset;
let mut lookback_offset_end = partition_offset_end - lookback_offset;
if lookback_offset_end > partition_offset_start {
// the lookback period is overlap with current period
lookback_offset_end = partition_offset_start;
}
let lookback_files = file_list::query(
org_id,
stream_name,
stream_type,
partition_time_level,
lookback_offset_start,
lookback_offset_end,
)
.await
.map_err(|e| anyhow::anyhow!("query lookback file list failed: {}", e))?;
files.extend(lookback_files);
}
log::debug!(
"[COMPACTOR] merge_by_stream [{}/{}/{}] time range: [{},{}], files: {}",
org_id,
@ -399,13 +466,6 @@ pub async fn merge_by_stream(
}
new_file_size += file.meta.original_size;
new_file_list.push(file.clone());
// metrics
metrics::COMPACT_MERGED_FILES
.with_label_values(&[&org_id, stream_type.to_string().as_str()])
.inc();
metrics::COMPACT_MERGED_BYTES
.with_label_values(&[&org_id, stream_type.to_string().as_str()])
.inc_by(file.meta.original_size as u64);
}
if new_file_list.len() > 1 {
batch_groups.push(MergeBatch {
@ -450,22 +510,14 @@ pub async fn merge_by_stream(
let new_file_name = std::mem::take(&mut new_file.key);
let new_file_meta = std::mem::take(&mut new_file.meta);
let new_file_list = batch_groups.get(batch_id).unwrap().files.as_slice();
if new_file_name.is_empty() {
if new_file_list.is_empty() {
// no file need to merge
break;
} else {
// delete files from file_list and continue
files_with_size.retain(|f| !&new_file_list.contains(f));
continue;
}
continue;
}
// delete small files keys & write big files keys, use transaction
let mut events = Vec::with_capacity(new_file_list.len() + 1);
events.push(FileKey {
key: new_file_name.clone(),
key: new_file_name,
meta: new_file_meta,
deleted: false,
segment_ids: None,
@ -536,16 +588,12 @@ pub async fn merge_by_stream(
.inc_by(time);
metrics::COMPACT_DELAY_HOURS
.with_label_values(&[org_id, stream_name, stream_type.to_string().as_str()])
.set(
(time_now_hour - offset_time_hour)
/ Duration::try_hours(1).unwrap().num_microseconds().unwrap(),
);
.set((time_now_hour - offset_time_hour) / hour_micros(1));
Ok(())
}
/// merge some small files into one big file, upload to storage, returns the big
/// file key and merged files
/// merge small files into big file, upload to storage, returns the big file key and merged files
pub async fn merge_files(
thread_id: usize,
org_id: &str,
@ -825,8 +873,8 @@ pub async fn merge_files(
inverted_idx_batch.clone(),
new_file_key.clone(),
org_id,
stream_name,
stream_type,
stream_name,
&full_text_search_fields,
&index_fields,
)

View File

@ -18,7 +18,7 @@ use config::{
cluster::LOCAL_NODE,
get_config,
meta::{
cluster::Role,
cluster::{CompactionJobType, Role},
stream::{PartitionTimeLevel, StreamType, ALL_STREAM_TYPES},
},
};
@ -63,10 +63,12 @@ pub async fn run_retention() -> Result<(), anyhow::Error> {
continue; // not this node
}
let schema = infra::schema::get(&org_id, &stream_name, stream_type).await?;
let stream = super::stream::stream_res(&stream_name, stream_type, schema, None);
let stream_data_retention_end = if stream.settings.data_retention > 0 {
let date = now - Duration::try_days(stream.settings.data_retention).unwrap();
let stream_settings =
infra::schema::get_settings(&org_id, &stream_name, stream_type)
.await
.unwrap_or_default();
let stream_data_retention_end = if stream_settings.data_retention > 0 {
let date = now - Duration::try_days(stream_settings.data_retention).unwrap();
date.format("%Y-%m-%d").to_string()
} else {
data_lifecycle_end.clone()
@ -137,7 +139,7 @@ pub async fn run_retention() -> Result<(), anyhow::Error> {
}
/// Generate job for compactor
pub async fn run_generate_job() -> Result<(), anyhow::Error> {
pub async fn run_generate_job(job_type: CompactionJobType) -> Result<(), anyhow::Error> {
let orgs = db::schema::list_organizations_from_cache().await;
for org_id in orgs {
// check backlist
@ -191,16 +193,37 @@ pub async fn run_generate_job() -> Result<(), anyhow::Error> {
continue;
}
if let Err(e) =
merge::generate_job_by_stream(&org_id, stream_type, &stream_name).await
{
log::error!(
"[COMPACTOR] generate_job_by_stream [{}/{}/{}] error: {}",
org_id,
stream_type,
stream_name,
e
);
match job_type {
CompactionJobType::Current => {
if let Err(e) =
merge::generate_job_by_stream(&org_id, stream_type, &stream_name).await
{
log::error!(
"[COMPACTOR] generate_job_by_stream [{}/{}/{}] error: {}",
org_id,
stream_type,
stream_name,
e
);
}
}
CompactionJobType::Historical => {
if let Err(e) = merge::generate_old_data_job_by_stream(
&org_id,
stream_type,
&stream_name,
)
.await
{
log::error!(
"[COMPACTOR] generate_old_data_job_by_stream [{}/{}/{}] error: {}",
org_id,
stream_type,
stream_name,
e
);
}
}
}
}
}
@ -234,7 +257,7 @@ pub async fn run_merge(
.unwrap_or_default();
let partition_time_level =
unwrap_partition_time_level(stream_setting.partition_time_level, stream_type);
if partition_time_level == PartitionTimeLevel::Daily || cfg.compact.step_secs < 3600 {
if partition_time_level == PartitionTimeLevel::Daily {
// check if this stream need process by this node
let Some(node_name) =
get_node_from_consistent_hash(&stream_name, &Role::Compactor, None).await

View File

@ -51,6 +51,15 @@ pub async fn delete_by_stream(
return Ok(()); // created_at is after lifecycle_end, just skip
}
log::debug!(
"[COMPACT] delete_by_stream {}/{}/{}/{},{}",
org_id,
stream_type,
stream_name,
lifecycle_start,
lifecycle_end
);
// Hack for 1970-01-01
if lifecycle_start.le("1970-01-01") {
let lifecycle_end = created_at + Duration::try_days(1).unwrap();
@ -83,7 +92,7 @@ pub async fn delete_all(
let locker = dist_lock::lock(&lock_key, 0, None).await?;
let node = db::compact::retention::get_stream(org_id, stream_type, stream_name, None).await;
if !node.is_empty() && LOCAL_NODE.uuid.ne(&node) && get_node_by_uuid(&node).await.is_some() {
log::error!("[COMPACT] stream {org_id}/{stream_type}/{stream_name} is deleting by {node}");
log::warn!("[COMPACT] stream {org_id}/{stream_type}/{stream_name} is deleting by {node}");
dist_lock::unlock(&locker).await?;
return Ok(()); // not this node, just skip
}
@ -179,7 +188,7 @@ pub async fn delete_by_date(
db::compact::retention::get_stream(org_id, stream_type, stream_name, Some(date_range))
.await;
if !node.is_empty() && LOCAL_NODE.uuid.ne(&node) && get_node_by_uuid(&node).await.is_some() {
log::error!(
log::warn!(
"[COMPACT] stream {org_id}/{stream_type}/{stream_name}/{:?} is deleting by {node}",
date_range
);

View File

@ -352,6 +352,7 @@ impl Report {
&cfg.common.report_user_password,
&self.timezone,
no_of_recipients,
&self.name,
)
.await?;
self.send_email(&report.0, report.1).await
@ -423,6 +424,7 @@ async fn generate_report(
user_pass: &str,
timezone: &str,
no_of_recipients: usize,
report_name: &str,
) -> Result<(Vec<u8>, String), anyhow::Error> {
let cfg = get_config();
// Check if Chrome is enabled, otherwise don't save the report
@ -488,10 +490,10 @@ async fn generate_report(
tokio::time::sleep(Duration::from_secs(5)).await;
let timerange = &dashboard.timerange;
let search_type = if no_of_recipients == 0 {
"ui"
let search_type_params = if no_of_recipients == 0 {
"search_type=ui".to_string()
} else {
"reports"
format!("search_type=reports&report_id={org_id}-{report_name}")
};
// dashboard link in the email should contain data of the same period as the report
@ -500,7 +502,7 @@ async fn generate_report(
let period = &timerange.period;
let (time_duration, time_unit) = period.split_at(period.len() - 1);
let dashb_url = format!(
"{web_url}/dashboards/view?org_identifier={org_id}&dashboard={dashboard_id}&folder={folder_id}&tab={tab_id}&refresh=Off&searchtype={search_type}&period={period}&timezone={timezone}&var-Dynamic+filters=%255B%255D&print=true{dashb_vars}",
"{web_url}/dashboards/view?org_identifier={org_id}&dashboard={dashboard_id}&folder={folder_id}&tab={tab_id}&refresh=Off&{search_type_params}&period={period}&timezone={timezone}&var-Dynamic+filters=%255B%255D&print=true{dashb_vars}",
);
let time_duration: i64 = time_duration.parse()?;
@ -550,7 +552,7 @@ async fn generate_report(
}
ReportTimerangeType::Absolute => {
let url = format!(
"{web_url}/dashboards/view?org_identifier={org_id}&dashboard={dashboard_id}&folder={folder_id}&tab={tab_id}&refresh=Off&searchtype={search_type}&from={}&to={}&timezone={timezone}&var-Dynamic+filters=%255B%255D&print=true{dashb_vars}",
"{web_url}/dashboards/view?org_identifier={org_id}&dashboard={dashboard_id}&folder={folder_id}&tab={tab_id}&refresh=Off&{search_type_params}&from={}&to={}&timezone={timezone}&var-Dynamic+filters=%255B%255D&print=true{dashb_vars}",
&timerange.from, &timerange.to
);
(url.clone(), url)

View File

@ -15,13 +15,13 @@
use std::sync::Arc;
use config::{meta::stream::StreamType, utils::json};
use crate::{
common::{infra::config::STREAM_ALERTS, meta::alerts::alert::Alert},
service::db,
use config::{
meta::{alerts::alert::Alert, stream::StreamType},
utils::json,
};
use crate::{common::infra::config::STREAM_ALERTS, service::db};
pub async fn get(
org_id: &str,
stream_type: StreamType,

View File

@ -15,13 +15,10 @@
use std::sync::Arc;
use config::utils::json;
use config::{meta::alerts::destinations::Destination, utils::json};
use itertools::Itertools;
use crate::{
common::{infra::config::ALERTS_DESTINATIONS, meta::alerts::destinations::Destination},
service::db,
};
use crate::{common::infra::config::ALERTS_DESTINATIONS, service::db};
pub async fn get(org_id: &str, name: &str) -> Result<Destination, anyhow::Error> {
let map_key = format!("{org_id}/{name}");

View File

@ -15,14 +15,11 @@
use std::sync::Arc;
use config::utils::json;
use config::{meta::alerts::templates::Template, utils::json};
use itertools::Itertools;
use crate::{
common::{
infra::config::ALERTS_TEMPLATES,
meta::{alerts::templates::Template, organization::DEFAULT_ORG},
},
common::{infra::config::ALERTS_TEMPLATES, meta::organization::DEFAULT_ORG},
service::db,
};

View File

@ -15,12 +15,16 @@
use std::sync::Arc;
use config::{meta::stream::StreamType, RwHashSet};
use config::{
meta::stream::StreamType,
utils::time::{hour_micros, now_micros},
RwHashMap,
};
use once_cell::sync::Lazy;
use crate::service::db;
static CACHE: Lazy<RwHashSet<String>> = Lazy::new(Default::default);
static CACHE: Lazy<RwHashMap<String, i64>> = Lazy::new(Default::default);
#[inline]
fn mk_key(
@ -47,12 +51,14 @@ pub async fn delete_stream(
let key = mk_key(org_id, stream_type, stream_name, date_range);
// write in cache
if CACHE.contains(&key) {
return Ok(()); // already in cache, just skip
if let Some(v) = CACHE.get(&key) {
if v.value() + hour_micros(1) > now_micros() {
return Ok(()); // already in cache, don't create same task in one hour
}
}
let db_key = format!("/compact/delete/{key}");
CACHE.insert(key);
CACHE.insert(key, now_micros());
Ok(db::put(&db_key, "OK".into(), db::NEED_WATCH, None).await?)
}
@ -92,7 +98,7 @@ pub fn is_deleting_stream(
stream_name: &str,
date_range: Option<(&str, &str)>,
) -> bool {
CACHE.contains(&mk_key(org_id, stream_type, stream_name, date_range))
CACHE.contains_key(&mk_key(org_id, stream_type, stream_name, date_range))
}
pub async fn delete_stream_done(
@ -102,9 +108,7 @@ pub async fn delete_stream_done(
date_range: Option<(&str, &str)>,
) -> Result<(), anyhow::Error> {
let key = mk_key(org_id, stream_type, stream_name, date_range);
db::delete_if_exists(&format!("/compact/delete/{key}"), false, db::NEED_WATCH)
.await
.map_err(|e| anyhow::anyhow!(e))?;
db::delete_if_exists(&format!("/compact/delete/{key}"), false, db::NEED_WATCH).await?;
// remove in cache
CACHE.remove(&key);
@ -128,19 +132,19 @@ pub async fn watch() -> Result<(), anyhow::Error> {
let cluster_coordinator = db::get_coordinator().await;
let mut events = cluster_coordinator.watch(key).await?;
let events = Arc::get_mut(&mut events).unwrap();
log::info!("Start watching stream deleting");
log::info!("Start watching compact deleting");
loop {
let ev = match events.recv().await {
Some(ev) => ev,
None => {
log::error!("watch_stream_deleting: event channel closed");
log::error!("watch_compact_deleting: event channel closed");
break;
}
};
match ev {
db::Event::Put(ev) => {
let item_key = ev.key.strip_prefix(key).unwrap();
CACHE.insert(item_key.to_string());
CACHE.insert(item_key.to_string(), now_micros());
}
db::Event::Delete(ev) => {
let item_key = ev.key.strip_prefix(key).unwrap();
@ -157,7 +161,7 @@ pub async fn cache() -> Result<(), anyhow::Error> {
let ret = db::list(key).await?;
for (item_key, _) in ret {
let item_key = item_key.strip_prefix(key).unwrap();
CACHE.insert(item_key.to_string());
CACHE.insert(item_key.to_string(), now_micros());
}
Ok(())
}

View File

@ -51,6 +51,7 @@ pub async fn get(org_id: &str, name: &str) -> Result<Vec<vrl::value::Value>, any
clusters: vec![],
timeout: 0,
search_type: None,
search_event_context: None,
index_type: "".to_string(),
};
// do search

View File

@ -20,8 +20,11 @@ use std::{
};
use bytes::Buf;
use chrono::{DateTime, Duration, TimeZone, Utc};
use config::{meta::stream::FileKey, utils::json};
use chrono::{DateTime, TimeZone, Utc};
use config::{
meta::stream::FileKey,
utils::{json, time::hour_micros},
};
use futures::future::try_join_all;
use infra::{cache::stats, file_list as infra_file_list, storage};
use once_cell::sync::Lazy;
@ -218,7 +221,7 @@ pub async fn cache_time_range(time_min: i64, time_max: i64) -> Result<(), anyhow
let offset_time: DateTime<Utc> = Utc.timestamp_nanos(cur_time * 1000);
let file_list_prefix = offset_time.format("%Y/%m/%d/%H/").to_string();
cache(&file_list_prefix, false).await?;
cur_time += Duration::try_hours(1).unwrap().num_microseconds().unwrap();
cur_time += hour_micros(1);
}
Ok(())
}

View File

@ -15,15 +15,9 @@
use std::sync::Arc;
use config::utils::json;
use config::{meta::function::Transform, utils::json};
use crate::{
common::{
infra::config::{QUERY_FUNCTIONS, STREAM_FUNCTIONS},
meta::functions::{StreamFunctionsList, Transform},
},
service::db,
};
use crate::{common::infra::config::QUERY_FUNCTIONS, service::db};
pub async fn set(org_id: &str, name: &str, js_func: &Transform) -> Result<(), anyhow::Error> {
let key = format!("/function/{org_id}/{name}");
@ -87,7 +81,6 @@ pub async fn watch() -> Result<(), anyhow::Error> {
match ev {
db::Event::Put(ev) => {
let item_key = ev.key.strip_prefix(key).unwrap();
let org_id = &item_key[0..item_key.find('/').unwrap()];
let item_value: Transform = if config::get_config().common.meta_store_external {
match db::get(&ev.key).await {
Ok(val) => match json::from_slice(&val) {
@ -105,29 +98,7 @@ pub async fn watch() -> Result<(), anyhow::Error> {
} else {
json::from_slice(&ev.value.unwrap()).unwrap()
};
if item_value.streams.is_some() {
for stream_fn in item_value.to_stream_transform() {
let mut group = STREAM_FUNCTIONS
.entry(format!(
"{}/{}/{}",
org_id, stream_fn.stream_type, stream_fn.stream
))
.or_insert_with(|| StreamFunctionsList { list: vec![] });
if group.list.contains(&stream_fn) {
let stream_name =
group.list.iter().position(|x| x.eq(&stream_fn)).unwrap();
if stream_fn.is_removed {
group.list.remove(stream_name);
} else {
let _ = std::mem::replace(&mut group.list[stream_name], stream_fn);
}
} else if !stream_fn.is_removed {
group.list.push(stream_fn);
}
}
} else {
QUERY_FUNCTIONS.insert(item_key.to_owned(), item_value);
}
QUERY_FUNCTIONS.insert(item_key.to_owned(), item_value);
}
db::Event::Delete(ev) => {
let item_key = ev.key.strip_prefix(key).unwrap();
@ -145,25 +116,7 @@ pub async fn cache() -> Result<(), anyhow::Error> {
for (item_key, item_value) in ret {
let item_key = item_key.strip_prefix(key).unwrap();
let json_val: Transform = json::from_slice(&item_value).unwrap();
let org_id = &item_key[0..item_key.find('/').unwrap()];
if json_val.streams.is_some() {
for stream_fn in json_val.to_stream_transform() {
let mut group = STREAM_FUNCTIONS
.entry(format!(
"{}/{}/{}",
org_id, stream_fn.stream_type, stream_fn.stream
))
.or_insert_with(|| StreamFunctionsList { list: vec![] });
if !stream_fn.is_removed {
group.list.push(stream_fn);
}
}
let mut func = json_val.clone();
func.streams = None;
QUERY_FUNCTIONS.insert(item_key.to_string(), func);
} else {
QUERY_FUNCTIONS.insert(item_key.to_string(), json_val);
}
QUERY_FUNCTIONS.insert(item_key.to_string(), json_val);
}
log::info!("Functions Cached");
Ok(())

View File

@ -34,7 +34,7 @@ pub mod metrics;
#[cfg(feature = "enterprise")]
pub mod ofga;
pub mod organization;
pub mod pipelines;
pub mod pipeline;
pub mod saved_view;
pub mod scheduler;
pub mod schema;

231
src/service/db/pipeline.rs Normal file
View File

@ -0,0 +1,231 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
// use std::sync::Arc;
use anyhow::Result;
use config::meta::{
pipeline::{components::PipelineSource, Pipeline},
stream::StreamParams,
};
use infra::pipeline::{self as infra_pipeline};
use crate::{
common::infra::config::STREAM_EXECUTABLE_PIPELINES,
service::pipeline::batch_execution::ExecutablePipeline,
};
/// Stores a new pipeline to database.
///
/// Pipeline validation should be handled by the caller.
pub async fn set(pipeline: &Pipeline) -> Result<()> {
if let Err(e) = infra_pipeline::put(pipeline).await {
log::error!("Error saving pipeline: {}", e);
return Err(anyhow::anyhow!("Error saving pipeline: {}", e));
}
// save to cache if realtime pipeline
if let PipelineSource::Realtime(stream_params) = &pipeline.source {
if pipeline.enabled {
update_cache(stream_params, pipeline, PipelineTableEvent::Add).await;
}
}
Ok(())
}
/// Updates a pipeline entry with the sane values.
///
/// Pipeline validation should be handled by the caller.
pub async fn update(pipeline: &Pipeline) -> Result<()> {
if let Err(e) = infra_pipeline::update(pipeline).await {
log::error!("Error updating pipeline: {}", e);
return Err(anyhow::anyhow!("Error updating pipeline: {}", e));
}
// save to cache if realtime pipeline
if let PipelineSource::Realtime(stream_params) = &pipeline.source {
let db_event = if pipeline.enabled {
PipelineTableEvent::Add
} else {
PipelineTableEvent::Remove
};
update_cache(stream_params, pipeline, db_event).await;
}
Ok(())
}
/// Returns all streams with existing pipelines.
pub async fn list_streams_with_pipeline(org: &str) -> Result<Vec<StreamParams>> {
infra_pipeline::list_streams_with_pipeline(org)
.await
.map_err(|e| {
log::error!("Error getting streams with pipeline for org({org}): {}", e);
anyhow::anyhow!("Error getting streams with pipeline for org({org}): {}", e)
})
}
/// Transform the initialized and enabled pipeline into ExecutablePipeline struct that's ready for
/// batch processing records.
///
/// Used for pipeline execution.
pub async fn get_executable_pipeline(stream_params: &StreamParams) -> Option<ExecutablePipeline> {
if let Some(exec_pl) = STREAM_EXECUTABLE_PIPELINES.read().await.get(stream_params) {
return Some(exec_pl.clone());
}
let pipeline = infra_pipeline::get_by_stream(stream_params).await.ok();
match pipeline {
Some(pl) if pl.enabled => match ExecutablePipeline::new(&pl).await {
Ok(exec_pl) => {
let mut stream_exec_pl_cache = STREAM_EXECUTABLE_PIPELINES.write().await;
stream_exec_pl_cache.insert(stream_params.to_owned(), exec_pl.clone());
drop(stream_exec_pl_cache);
Some(exec_pl)
}
Err(e) => {
log::error!(
"[Pipeline]: failed to initialize ExecutablePipeline from Pipeline read from database, {}",
e
);
None
}
},
_ => None,
}
}
/// Returns the pipeline by id.
///
/// Used to get the pipeline associated with the ID when scheduled job is ran.
pub async fn get_by_id(pipeline_id: &str) -> Result<Pipeline> {
infra_pipeline::get_by_id(pipeline_id).await.map_err(|e| {
log::error!("Error getting pipeline with ID({pipeline_id}): {}", e);
anyhow::anyhow!("Error getting pipeline with ID({pipeline_id}): {}", e)
})
}
/// Finds the pipeline with the same source
///
/// Used to validate if a duplicate pipeline exists.
pub async fn get_with_same_source_stream(pipeline: &Pipeline) -> Result<Pipeline> {
infra_pipeline::get_with_same_source_stream(pipeline)
.await
.map_err(|_| anyhow::anyhow!("No pipeline with the same source found"))
}
/// Lists all pipelines across all orgs.
pub async fn list() -> Result<Vec<Pipeline>> {
infra_pipeline::list().await.map_err(|e| {
log::debug!("Error listing pipelines for all orgs: {}", e);
anyhow::anyhow!("Error listing pipelines for all orgs: {}", e)
})
}
/// Lists all pipelines for a given organization.
pub async fn list_by_org(org: &str) -> Result<Vec<Pipeline>> {
infra_pipeline::list_by_org(org).await.map_err(|e| {
log::debug!("Error listing pipelines for org({org}): {}", e);
anyhow::anyhow!("Error listing pipelines for org({org}): {}", e)
})
}
/// Deletes a pipeline by ID.
pub async fn delete(pipeline_id: &str) -> Result<()> {
match infra_pipeline::delete(pipeline_id).await {
Err(e) => {
log::error!("Error deleting pipeline with ID({pipeline_id}): {}", e);
return Err(anyhow::anyhow!(
"Error deleting pipeline with ID({pipeline_id}): {}",
e
));
}
Ok(pipeline) => {
// remove from cache if realtime pipeline
if let PipelineSource::Realtime(stream_params) = &pipeline.source {
update_cache(stream_params, &pipeline, PipelineTableEvent::Remove).await;
}
}
}
Ok(())
}
/// Preload all enabled pipelines into the cache at startup.
pub async fn cache() -> Result<(), anyhow::Error> {
let pipelines = list().await?;
let mut stream_exec_pl = STREAM_EXECUTABLE_PIPELINES.write().await;
for pipeline in pipelines.into_iter() {
if pipeline.enabled {
if let PipelineSource::Realtime(stream_params) = &pipeline.source {
match ExecutablePipeline::new(&pipeline).await {
Err(e) => {
log::error!(
"[Pipeline] error initializing ExecutablePipeline from pipeline {}/{}. {}. Not cached",
pipeline.org,
pipeline.id,
e
)
}
Ok(exec_pl) => {
stream_exec_pl.insert(stream_params.clone(), exec_pl);
}
};
}
}
}
log::info!("[Pipeline] Cached with len: {}", stream_exec_pl.len());
Ok(())
}
/// Update STREAM_PIPELINES cache for realtime pipelines
async fn update_cache(
stream_params: &StreamParams,
pipeline: &Pipeline,
event: PipelineTableEvent,
) {
match event {
PipelineTableEvent::Remove => {
log::info!("[Pipeline]: pipeline {} removed from cache.", &pipeline.id);
STREAM_EXECUTABLE_PIPELINES
.write()
.await
.remove(stream_params);
}
PipelineTableEvent::Add => {
let mut stream_pl_exec = STREAM_EXECUTABLE_PIPELINES.write().await;
match ExecutablePipeline::new(pipeline).await {
Err(e) => {
log::error!(
"[Pipeline] {}/{}/{}: Error initializing pipeline into ExecutablePipeline when updating cache: {}",
pipeline.org,
pipeline.name,
pipeline.id,
e
);
}
Ok(exec_pl) => {
stream_pl_exec.insert(stream_params.clone(), exec_pl);
}
};
log::info!("[Pipeline]: pipeline {} added to cache.", &pipeline.id);
}
}
}
enum PipelineTableEvent {
Add,
Remove,
}

View File

@ -1,162 +0,0 @@
// Copyright 2024 OpenObserve Inc.
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU Affero General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Affero General Public License for more details.
//
// You should have received a copy of the GNU Affero General Public License
// along with this program. If not, see <http://www.gnu.org/licenses/>.
use std::sync::Arc;
use config::{meta::stream::StreamType, utils::json};
use crate::{
common::{infra::config::STREAM_PIPELINES, meta::pipelines::PipeLine},
service::db,
};
pub async fn set(org_id: &str, pipeline: &PipeLine) -> Result<(), anyhow::Error> {
let key = format!(
"/pipeline/{org_id}/{}/{}/{}",
pipeline.stream_type, pipeline.stream_name, pipeline.name
);
match db::put(
&key,
json::to_vec(pipeline).unwrap().into(),
db::NEED_WATCH,
None,
)
.await
{
Ok(_) => {}
Err(e) => {
log::error!("Error saving pipeline: {}", e);
return Err(anyhow::anyhow!("Error saving pipeline: {}", e));
}
}
Ok(())
}
pub async fn get(
org_id: &str,
stream_type: StreamType,
stream_name: &str,
name: &str,
) -> Result<PipeLine, anyhow::Error> {
let val = db::get(&format!(
"/pipeline/{org_id}/{stream_type}/{stream_name}/{name}"
))
.await?;
Ok(json::from_slice(&val).unwrap())
}
pub async fn delete(
org_id: &str,
stream_type: StreamType,
stream_name: &str,
name: &str,
) -> Result<(), anyhow::Error> {
let key = format!("/pipeline/{org_id}/{stream_type}/{stream_name}/{name}");
match db::delete(&key, false, db::NEED_WATCH, None).await {
Ok(_) => {}
Err(e) => {
log::error!("Error deleting pipeline: {}", e);
return Err(anyhow::anyhow!("Error deleting pipeline: {}", e));
}
}
Ok(())
}
pub async fn list(org_id: &str) -> Result<Vec<PipeLine>, anyhow::Error> {
Ok(db::list(&format!("/pipeline/{org_id}/"))
.await?
.values()
.map(|val| json::from_slice(val).unwrap())
.collect())
}
pub async fn watch() -> Result<(), anyhow::Error> {
let key = "/pipeline/";
let cluster_coordinator = db::get_coordinator().await;
let mut events = cluster_coordinator.watch(key).await?;
let events = Arc::get_mut(&mut events).unwrap();
log::info!("Start watching pipeline");
loop {
let ev = match events.recv().await {
Some(ev) => ev,
None => {
log::error!("watch_pipelines: event channel closed");
break;
}
};
match ev {
db::Event::Put(ev) => {
let item_key = ev.key.strip_prefix(key).unwrap();
let org_id = &item_key[0..item_key.find('/').unwrap()];
let item_value: PipeLine = if config::get_config().common.meta_store_external {
match db::get(&ev.key).await {
Ok(val) => match json::from_slice(&val) {
Ok(val) => val,
Err(e) => {
log::error!("Error getting value: {}", e);
continue;
}
},
Err(e) => {
log::error!("Error getting value: {}", e);
continue;
}
}
} else {
json::from_slice(&ev.value.unwrap()).unwrap()
};
let key = format!(
"{org_id}/{}/{}",
item_value.stream_type, item_value.stream_name
);
STREAM_PIPELINES.insert(key.to_owned(), item_value);
}
db::Event::Delete(ev) => {
let item_key = ev.key.strip_prefix(key).unwrap();
let removal_key = match item_key.rfind('/') {
Some(index) => &item_key[0..index],
None => item_key,
};
STREAM_PIPELINES.remove(removal_key);
}
db::Event::Empty => {}
}
}
Ok(())
}
pub async fn cache() -> Result<(), anyhow::Error> {
let key = "/pipeline/";
let ret = db::list(key).await?;
for (item_key, item_value) in ret {
let item_key = item_key.strip_prefix(key).unwrap();
let json_val: PipeLine = json::from_slice(&item_value).unwrap();
let org_id = &item_key[0..item_key.find('/').unwrap()];
let key = format!("{org_id}/{}/{}", json_val.stream_type, json_val.stream_name);
STREAM_PIPELINES.insert(key.to_string(), json_val);
}
log::info!("Pipelines Cached");
Ok(())
}
pub async fn reset() -> Result<(), anyhow::Error> {
let key = "/pipeline/";
db::delete(key, true, db::NO_NEED_WATCH, None).await?;
Ok(())
}

View File

@ -75,10 +75,16 @@ pub async fn query_by_ids(trace_id: &str, ids: &[i64]) -> Result<Vec<FileKey>> {
// 1. first query from local cache
let (mut files, ids) = if !cfg.common.local_mode && cfg.common.meta_store_external {
let ids_set: HashSet<_> = ids.iter().cloned().collect();
let cached_files = file_list::LOCAL_CACHE
.query_by_ids(ids)
.await
.unwrap_or_default();
let cached_files = match file_list::LOCAL_CACHE.query_by_ids(ids).await {
Ok(files) => files,
Err(e) => {
log::error!(
"[trace_id {trace_id}] file_list query cache failed: {:?}",
e
);
Vec::new()
}
};
let cached_ids = cached_files
.iter()
.map(|(id, ..)| *id)

View File

@ -19,18 +19,11 @@ use actix_web::{
http::{self, StatusCode},
HttpResponse,
};
use config::meta::stream::StreamType;
use config::meta::function::{FunctionList, Transform};
use crate::{
common::{
infra::config::STREAM_FUNCTIONS,
meta::{
authz::Authz,
functions::{
FunctionList, StreamFunctionsList, StreamOrder, StreamTransform, Transform,
},
http::HttpResponse as MetaHttpResponse,
},
meta::{authz::Authz, http::HttpResponse as MetaHttpResponse},
utils::auth::{remove_ownership, set_ownership},
},
service::{db, ingestion::compile_vrl_function},
@ -38,8 +31,6 @@ use crate::{
const FN_SUCCESS: &str = "Function saved successfully";
const FN_NOT_FOUND: &str = "Function not found";
const FN_ADDED: &str = "Function applied to stream";
const FN_REMOVED: &str = "Function removed from stream";
const FN_DELETED: &str = "Function deleted";
const FN_ALREADY_EXIST: &str = "Function already exist";
const FN_IN_USE: &str =
@ -101,10 +92,6 @@ pub async fn update_function(
return Ok(HttpResponse::Ok().json(func));
}
// UI mostly like in 1st version won't send streams, so we need to add them back
// from existing function
func.streams = existing_fn.streams;
if !func.function.ends_with('.') {
func.function = format!("{} \n .", func.function);
}
@ -206,118 +193,6 @@ pub async fn delete_function(org_id: String, fn_name: String) -> Result<HttpResp
}
}
pub async fn list_stream_functions(
org_id: &str,
stream_type: StreamType,
stream_name: &str,
) -> Result<HttpResponse, Error> {
if let Some(val) = STREAM_FUNCTIONS.get(&format!("{}/{}/{}", org_id, stream_type, stream_name))
{
Ok(HttpResponse::Ok().json(val.value()))
} else {
Ok(HttpResponse::Ok().json(StreamFunctionsList { list: vec![] }))
}
}
pub async fn delete_stream_function(
org_id: &str,
stream_type: StreamType,
stream_name: &str,
fn_name: &str,
) -> Result<HttpResponse, Error> {
let mut existing_fn = match check_existing_fn(org_id, fn_name).await {
Some(function) => function,
None => {
return Ok(HttpResponse::NotFound().json(MetaHttpResponse::error(
StatusCode::NOT_FOUND.into(),
FN_NOT_FOUND.to_string(),
)));
}
};
if let Some(ref mut val) = existing_fn.streams {
for stream in val.iter_mut() {
if stream.stream == stream_name && stream.stream_type == stream_type {
stream.is_removed = true;
stream.order = 0;
break;
}
}
if let Err(error) = db::functions::set(org_id, fn_name, &existing_fn).await {
Ok(
HttpResponse::InternalServerError().json(MetaHttpResponse::message(
http::StatusCode::INTERNAL_SERVER_ERROR.into(),
error.to_string(),
)),
)
} else {
// can't be removed from watcher of function as stream name & type won't be
// available , hence being removed here
// let key = format!("{}/{}/{}", org_id, stream_type, stream_name);
// remove_stream_fn_from_cache(&key, fn_name);
Ok(HttpResponse::Ok().json(MetaHttpResponse::message(
http::StatusCode::OK.into(),
FN_REMOVED.to_string(),
)))
}
} else {
Ok(HttpResponse::NotFound().json(MetaHttpResponse::error(
StatusCode::NOT_FOUND.into(),
FN_NOT_FOUND.to_string(),
)))
}
}
pub async fn add_function_to_stream(
org_id: &str,
stream_type: StreamType,
stream_name: &str,
fn_name: &str,
mut stream_order: StreamOrder,
) -> Result<HttpResponse, Error> {
let mut existing_fn = match check_existing_fn(org_id, fn_name).await {
Some(function) => function,
None => {
return Ok(HttpResponse::NotFound().json(MetaHttpResponse::error(
StatusCode::NOT_FOUND.into(),
FN_NOT_FOUND.to_string(),
)));
}
};
stream_order.stream = stream_name.to_owned();
stream_order.stream_type = stream_type;
if let Some(mut val) = existing_fn.streams {
if let Some(existing) = val.iter_mut().find(|x| {
x.stream == stream_order.stream && x.stream_type.eq(&stream_order.stream_type)
}) {
existing.is_removed = false;
existing.order = stream_order.order;
existing.apply_before_flattening = stream_order.apply_before_flattening;
} else {
val.push(stream_order);
}
existing_fn.streams = Some(val);
} else {
existing_fn.streams = Some(vec![stream_order]);
}
if let Err(error) = db::functions::set(org_id, fn_name, &existing_fn).await {
Ok(
HttpResponse::InternalServerError().json(MetaHttpResponse::message(
http::StatusCode::INTERNAL_SERVER_ERROR.into(),
error.to_string(),
)),
)
} else {
Ok(HttpResponse::Ok().json(MetaHttpResponse::message(
http::StatusCode::OK.into(),
FN_ADDED.to_string(),
)))
}
}
fn extract_num_args(func: &mut Transform) {
if func.trans_type.unwrap() == 1 {
let src: String = func.function.to_owned();
@ -342,24 +217,10 @@ async fn check_existing_fn(org_id: &str, fn_name: &str) -> Option<Transform> {
}
}
fn _remove_stream_fn_from_cache(key: &str, fn_name: &str) {
if let Some(val) = STREAM_FUNCTIONS.clone().get(key) {
if val.list.len() > 1 {
let final_list = val
.clone()
.list
.into_iter()
.filter(|x| x.transform.name != fn_name)
.collect::<Vec<StreamTransform>>();
STREAM_FUNCTIONS.insert(key.to_string(), StreamFunctionsList { list: final_list });
} else {
STREAM_FUNCTIONS.remove(key);
}
}
}
#[cfg(test)]
mod tests {
use config::meta::{function::StreamOrder, stream::StreamType};
use super::*;
#[tokio::test]

View File

@ -25,9 +25,10 @@ use config::{
get_config,
ider::SnowflakeIdGenerator,
meta::{
alerts::alert::Alert,
function::{VRLResultResolver, VRLRuntimeConfig},
stream::{
PartitionTimeLevel, PartitioningDetails, Routing, StreamParams, StreamPartition,
StreamType,
PartitionTimeLevel, PartitioningDetails, StreamParams, StreamPartition, StreamType,
},
usage::{RequestStats, TriggerData, TriggerDataStatus, TriggerDataType},
},
@ -38,27 +39,21 @@ use futures::future::try_join_all;
use infra::schema::STREAM_RECORD_ID_GENERATOR;
use proto::cluster_rpc::IngestionType;
use tokio::sync::Semaphore;
use vector_enrichment::TableRegistry;
use vrl::{
compiler::{runtime::Runtime, CompilationResult, TargetValueRef},
prelude::state,
};
use super::usage::publish_triggers_usage;
use super::{
db::pipeline, pipeline::batch_execution::ExecutablePipeline, usage::publish_triggers_usage,
};
use crate::{
common::{
infra::config::{
REALTIME_ALERT_TRIGGERS, STREAM_ALERTS, STREAM_FUNCTIONS, STREAM_PIPELINES,
},
meta::{
alerts::alert::Alert,
functions::{StreamTransform, VRLResultResolver, VRLRuntimeConfig},
ingestion::IngestionRequest,
stream::SchemaRecords,
},
infra::config::{REALTIME_ALERT_TRIGGERS, STREAM_ALERTS},
meta::{ingestion::IngestionRequest, stream::SchemaRecords},
utils::functions::get_vrl_compiler_config,
},
service::db,
service::{alerts::alert::AlertExt, db},
};
pub mod grpc;
@ -101,13 +96,13 @@ pub fn compile_vrl_function(func: &str, org_id: &str) -> Result<VRLRuntimeConfig
pub fn apply_vrl_fn(
runtime: &mut Runtime,
vrl_runtime: &VRLResultResolver,
row: &Value,
row: Value,
org_id: &str,
stream_name: &[String],
) -> Value {
let mut metadata = vrl::value::Value::from(BTreeMap::new());
let mut target = TargetValueRef {
value: &mut vrl::value::Value::from(row),
value: &mut vrl::value::Value::from(&row),
metadata: &mut metadata,
secrets: &mut vrl::value::Secrets::new(),
};
@ -127,7 +122,7 @@ pub fn apply_vrl_fn(
stream_name,
err,
);
row.clone()
row
}
},
Err(err) => {
@ -137,43 +132,11 @@ pub fn apply_vrl_fn(
stream_name,
err,
);
row.clone()
row
}
}
}
pub async fn get_stream_functions<'a>(
streams: &[StreamParams],
stream_before_functions_map: &mut HashMap<String, Vec<StreamTransform>>,
stream_after_functions_map: &mut HashMap<String, Vec<StreamTransform>>,
stream_vrl_map: &mut HashMap<String, VRLResultResolver>,
) {
for stream in streams {
let key = format!(
"{}/{}/{}",
stream.org_id, stream.stream_type, stream.stream_name
);
if stream_after_functions_map.contains_key(&key)
|| stream_before_functions_map.contains_key(&key)
{
// functions for this stream already fetched
continue;
}
// let mut _local_trans: Vec<StreamTransform> = vec![];
// let local_stream_vrl_map;
let (before_local_trans, after_local_trans, local_stream_vrl_map) =
crate::service::ingestion::register_stream_functions(
&stream.org_id,
&stream.stream_type,
&stream.stream_name,
);
stream_vrl_map.extend(local_stream_vrl_map);
stream_before_functions_map.insert(key.clone(), before_local_trans);
stream_after_functions_map.insert(key, after_local_trans);
}
}
pub async fn get_stream_partition_keys(
org_id: &str,
stream_type: &StreamType,
@ -188,6 +151,15 @@ pub async fn get_stream_partition_keys(
}
}
pub async fn get_stream_executable_pipeline(
org_id: &str,
stream_name: &str,
stream_type: &StreamType,
) -> Option<ExecutablePipeline> {
let stream_params = StreamParams::new(org_id, stream_name, *stream_type);
pipeline::get_executable_pipeline(&stream_params).await
}
pub async fn get_stream_alerts(
streams: &[StreamParams],
stream_alerts_map: &mut HashMap<String, Vec<Alert>>,
@ -353,74 +325,6 @@ pub fn get_write_partition_key(
time_key
}
pub fn register_stream_functions(
org_id: &str,
stream_type: &StreamType,
stream_name: &str,
) -> (
Vec<StreamTransform>,
Vec<StreamTransform>,
HashMap<String, VRLResultResolver>,
) {
let mut before_local_trans = vec![];
let mut after_local_trans = vec![];
let mut stream_vrl_map: HashMap<String, VRLResultResolver> = HashMap::new();
let key = format!("{}/{}/{}", org_id, stream_type, stream_name);
if let Some(transforms) = STREAM_FUNCTIONS.get(&key) {
(before_local_trans, after_local_trans) = (*transforms.list)
.iter()
.cloned()
.partition(|elem| elem.apply_before_flattening);
before_local_trans.sort_by(|a, b| a.order.cmp(&b.order));
after_local_trans.sort_by(|a, b| a.order.cmp(&b.order));
for trans in before_local_trans.iter().chain(after_local_trans.iter()) {
let func_key = format!("{}/{}", &stream_name, trans.transform.name);
if let Ok(vrl_runtime_config) = compile_vrl_function(&trans.transform.function, org_id)
{
let registry = vrl_runtime_config
.config
.get_custom::<TableRegistry>()
.unwrap();
registry.finish_load();
stream_vrl_map.insert(
func_key,
VRLResultResolver {
program: vrl_runtime_config.program,
fields: vrl_runtime_config.fields,
},
);
}
}
}
(before_local_trans, after_local_trans, stream_vrl_map)
}
pub fn apply_stream_functions(
local_trans: &[StreamTransform],
mut value: Value,
stream_vrl_map: &HashMap<String, VRLResultResolver>,
org_id: &str,
stream_name: &str,
runtime: &mut Runtime,
) -> Result<Value> {
for trans in local_trans {
let func_key = format!("{stream_name}/{}", trans.transform.name);
if stream_vrl_map.contains_key(&func_key) && !value.is_null() {
let vrl_runtime = stream_vrl_map.get(&func_key).unwrap();
value = apply_vrl_fn(
runtime,
vrl_runtime,
&value,
org_id,
&[stream_name.to_string()],
);
}
}
flatten::flatten_with_level(value, get_config().limit.ingest_flatten_level)
}
pub fn init_functions_runtime() -> Runtime {
crate::common::utils::functions::init_vrl_runtime()
}
@ -592,30 +496,6 @@ pub fn get_val_with_type_retained(val: &Value) -> Value {
Value::Null => Value::Null,
}
}
pub async fn get_stream_routing(
stream_params: StreamParams,
stream_routing_map: &mut HashMap<String, Vec<Routing>>,
) {
if let Some(pipeline) = STREAM_PIPELINES.get(&format!(
"{}/{}/{}",
&stream_params.org_id, stream_params.stream_type, &stream_params.stream_name,
)) {
let Some(routing) = pipeline.routing.as_ref() else {
return;
};
let res: Vec<Routing> = routing
.iter()
.map(|(k, v)| Routing {
destination: k.to_string(),
routing: v.clone(),
})
.collect();
stream_routing_map.insert(stream_params.stream_name.to_string(), res);
}
}
pub async fn get_uds_and_original_data_streams(
streams: &[StreamParams],
user_defined_schema_map: &mut HashMap<String, HashSet<String>>,

View File

@ -24,7 +24,7 @@ use chrono::{Duration, Utc};
use config::{
get_config,
meta::{
stream::{Routing, StreamParams, StreamType},
stream::{StreamParams, StreamType},
usage::UsageType,
},
metrics,
@ -33,18 +33,19 @@ use config::{
};
use crate::{
common::meta::{
functions::{StreamTransform, VRLResultResolver},
ingestion::{BulkResponse, BulkResponseError, BulkResponseItem, IngestionStatus},
},
common::meta::ingestion::{BulkResponse, BulkResponseError, BulkResponseItem, IngestionStatus},
service::{
format_stream_name, ingestion::check_ingestion_allowed, schema::get_upto_discard_error,
format_stream_name,
ingestion::check_ingestion_allowed,
pipeline::batch_execution::{ExecutablePipeline, ExecutablePipelineBulkInputs},
schema::get_upto_discard_error,
},
};
pub const TRANSFORM_FAILED: &str = "document_failed_transform";
pub const TS_PARSE_FAILED: &str = "timestamp_parsing_failed";
pub const SCHEMA_CONFORMANCE_FAILED: &str = "schema_conformance_failed";
pub const PIPELINE_EXEC_FAILED: &str = "pipeline_execution_failed";
pub async fn ingest(
thread_id: usize,
@ -69,18 +70,16 @@ pub async fn ingest(
let min_ts = (Utc::now() - Duration::try_hours(cfg.limit.ingest_allowed_upto).unwrap())
.timestamp_micros();
let mut runtime = crate::service::ingestion::init_functions_runtime();
let mut stream_vrl_map: HashMap<String, VRLResultResolver> = HashMap::new();
let mut stream_before_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut stream_after_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut action = String::from("");
let mut stream_name = String::from("");
let mut doc_id = None;
let mut blocked_stream_warnings: HashMap<String, bool> = HashMap::new();
let mut stream_routing_map: HashMap<String, Vec<Routing>> = HashMap::new();
let mut stream_executable_pipelines: HashMap<String, Option<ExecutablePipeline>> =
HashMap::new();
let mut stream_pipeline_inputs: HashMap<String, ExecutablePipelineBulkInputs> = HashMap::new();
let mut user_defined_schema_map: HashMap<String, HashSet<String>> = HashMap::new();
let mut streams_need_original_set: HashSet<String> = HashSet::new();
@ -118,33 +117,27 @@ pub async fn ingest(
continue; // skip
}
// Start get routing keys
crate::service::ingestion::get_stream_routing(
StreamParams {
org_id: org_id.to_owned().into(),
stream_type: StreamType::Logs,
stream_name: stream_name.to_owned().into(),
},
&mut stream_routing_map,
)
.await;
let mut streams = vec![StreamParams {
org_id: org_id.to_owned().into(),
stream_type: StreamType::Logs,
stream_name: stream_name.to_owned().into(),
}];
if let Some(routes) = stream_routing_map.get(&stream_name) {
for route in routes {
streams.push(StreamParams {
org_id: org_id.to_owned().into(),
stream_type: StreamType::Logs,
stream_name: route.destination.clone().into(),
});
// Start retrieve associated pipeline and initialize ExecutablePipeline
if !stream_executable_pipelines.contains_key(&stream_name) {
let exec_pl_option = crate::service::ingestion::get_stream_executable_pipeline(
org_id,
&stream_name,
&StreamType::Logs,
)
.await;
if let Some(exec_pl) = &exec_pl_option {
let pl_destinations = exec_pl.get_all_destination_streams();
streams.extend(pl_destinations);
}
stream_executable_pipelines.insert(stream_name.clone(), exec_pl_option);
}
// End get stream keys
// End pipeline params construction
crate::service::ingestion::get_uds_and_original_data_streams(
&streams,
@ -154,217 +147,265 @@ pub async fn ingest(
.await;
next_line_is_data = true;
// Start Register functions for stream
crate::service::ingestion::get_stream_functions(
&streams,
&mut stream_before_functions_map,
&mut stream_after_functions_map,
&mut stream_vrl_map,
)
.await;
// End Register functions for index
} else {
next_line_is_data = false;
// store a copy of original data before it's being transformed and/or flattened, unless
// store a copy of original data before it's being transformed and/or flattened, when
// 1. original data is not an object -> won't be flattened.
// 2. no routing and current StreamName not in streams_need_original_set
let original_data = if value.is_object() {
if stream_routing_map.is_empty()
&& !streams_need_original_set.contains(&stream_name)
// 2. current stream does not have pipeline
if stream_executable_pipelines
.get(&stream_name)
.unwrap()
.is_none()
{
None
// current stream requires original
streams_need_original_set
.contains(&stream_name)
.then_some(value.to_string())
} else {
// otherwise, make a copy in case the routed stream needs original data
Some(value.to_string())
// 3. with pipeline, storing original as long as streams_need_original_set is
// not empty
// because not sure the pipeline destinations
(!streams_need_original_set.is_empty()).then_some(value.to_string())
}
} else {
None // `item` won't be flattened, no need to store original
};
let main_stream_key = format!("{org_id}/{}/{stream_name}", StreamType::Logs);
// Start row based transform before flattening the value
if let Some(transforms) = stream_before_functions_map.get(&main_stream_key) {
if !transforms.is_empty() {
let mut ret_value = value.clone();
ret_value = crate::service::ingestion::apply_stream_functions(
transforms,
ret_value,
&stream_vrl_map,
if stream_executable_pipelines
.get(&stream_name)
.unwrap()
.is_some()
{
// current stream has pipeline. buff the record for batch processing later
let inputs = stream_pipeline_inputs
.entry(stream_name.clone())
.or_default();
inputs.add_input(value, doc_id.to_owned(), original_data);
} else {
// JSON Flattening
value = flatten::flatten_with_level(value, cfg.limit.ingest_flatten_level)?;
// get json object
let mut local_val = match value.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
// set _id
if let Some(doc_id) = &doc_id {
local_val.insert("_id".to_string(), json::Value::String(doc_id.to_owned()));
}
if let Some(fields) = user_defined_schema_map.get(&stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&stream_name) && original_data.is_some() {
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&stream_name,
&mut runtime,
)?;
if ret_value.is_null() || !ret_value.is_object() {
bulk_res.errors = true;
add_record_status(
stream_name.clone(),
&doc_id,
action.clone(),
Some(value),
&mut bulk_res,
Some(TRANSFORM_FAILED.to_owned()),
Some(TRANSFORM_FAILED.to_owned()),
);
continue;
} else {
value = ret_value;
}
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
}
// end row based transformation
// JSON Flattening
let mut value = flatten::flatten_with_level(value, cfg.limit.ingest_flatten_level)?;
let mut routed_stream_name = stream_name.clone();
// Start re-routing if exists
if let Some(routing) = stream_routing_map.get(&stream_name) {
if !routing.is_empty() {
for route in routing {
let mut is_routed = true;
let val = &route.routing;
for q_condition in val.iter() {
if !q_condition.evaluate(value.as_object().unwrap()).await {
is_routed = false;
break;
}
// handle timestamp
let timestamp = match local_val.get(&cfg.common.column_timestamp) {
Some(v) => match parse_timestamp_micro_from_value(v) {
Ok(t) => t,
Err(_e) => {
bulk_res.errors = true;
add_record_status(
stream_name.clone(),
&doc_id,
action.clone(),
Some(value),
&mut bulk_res,
Some(TS_PARSE_FAILED.to_string()),
Some(TS_PARSE_FAILED.to_string()),
);
continue;
}
if is_routed && !val.is_empty() {
routed_stream_name = route.destination.clone();
break;
}
}
},
None => Utc::now().timestamp_micros(),
};
// check ingestion time
if timestamp < min_ts {
bulk_res.errors = true;
let failure_reason = Some(get_upto_discard_error().to_string());
add_record_status(
stream_name.clone(),
&doc_id,
action.clone(),
Some(value),
&mut bulk_res,
Some(TS_PARSE_FAILED.to_string()),
failure_reason,
);
continue;
}
}
// End re-routing
let key = format!("{org_id}/{}/{routed_stream_name}", StreamType::Logs);
// Start row based transform
if let Some(transforms) = stream_after_functions_map.get(&key) {
if !transforms.is_empty() {
let mut ret_value = value.clone();
ret_value = crate::service::ingestion::apply_stream_functions(
transforms,
ret_value,
&stream_vrl_map,
org_id,
&routed_stream_name,
&mut runtime,
)?;
if ret_value.is_null() || !ret_value.is_object() {
bulk_res.errors = true;
add_record_status(
routed_stream_name.clone(),
&doc_id,
action.clone(),
Some(value),
&mut bulk_res,
Some(TRANSFORM_FAILED.to_owned()),
Some(TRANSFORM_FAILED.to_owned()),
);
continue;
} else {
value = ret_value;
}
}
}
// End row based transform
// get json object
let mut local_val = match value.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
// set _id
if let Some(doc_id) = &doc_id {
local_val.insert("_id".to_string(), json::Value::String(doc_id.to_owned()));
}
if let Some(fields) = user_defined_schema_map.get(&routed_stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&routed_stream_name) && original_data.is_some() {
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&routed_stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
cfg.common.column_timestamp.clone(),
json::Value::Number(timestamp.into()),
);
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_name.clone())
.or_insert((Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = Some(0); // no pl -> no func
}
// handle timestamp
let timestamp = match local_val.get(&cfg.common.column_timestamp) {
Some(v) => match parse_timestamp_micro_from_value(v) {
Ok(t) => t,
Err(_e) => {
bulk_res.errors = true;
add_record_status(
routed_stream_name.clone(),
&doc_id,
action.clone(),
Some(value),
&mut bulk_res,
Some(TS_PARSE_FAILED.to_string()),
Some(TS_PARSE_FAILED.to_string()),
);
continue;
}
},
None => Utc::now().timestamp_micros(),
};
// check ingestion time
if timestamp < min_ts {
bulk_res.errors = true;
let failure_reason = Some(get_upto_discard_error().to_string());
add_record_status(
routed_stream_name.clone(),
&doc_id,
action.clone(),
Some(value),
&mut bulk_res,
Some(TS_PARSE_FAILED.to_string()),
failure_reason,
);
continue;
}
local_val.insert(
cfg.common.column_timestamp.clone(),
json::Value::Number(timestamp.into()),
);
let fns_length = stream_before_functions_map
.get(&main_stream_key)
.map(|v| v.len())
.unwrap_or_default()
+ stream_after_functions_map
.get(&key)
.map(|v| v.len())
.unwrap_or_default();
let (ts_data, fn_num) = json_data_by_stream
.entry(routed_stream_name.clone())
.or_insert((Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = Some(fns_length);
}
}
// batch process records through pipeline
for (stream_name, exec_pl_option) in stream_executable_pipelines {
if let Some(exec_pl) = exec_pl_option {
let Some(pipeline_inputs) = stream_pipeline_inputs.remove(&stream_name) else {
log::error!(
"[Ingestion]: Stream {} has pipeline, but inputs failed to be buffered. BUG",
stream_name
);
continue;
};
let (records, doc_ids, originals) = pipeline_inputs.into_parts();
match exec_pl.process_batch(org_id, records).await {
Err(e) => {
log::error!(
"[Pipeline] for stream {}/{}: Batch execution error: {}.",
org_id,
stream_name,
e
);
bulk_res.errors = true;
add_record_status(
stream_name.clone(),
&None,
action.clone(),
None,
&mut bulk_res,
Some(PIPELINE_EXEC_FAILED.to_string()),
Some(PIPELINE_EXEC_FAILED.to_string()),
);
continue;
}
Ok(pl_results) => {
let function_no = exec_pl.num_of_func();
for (stream_params, stream_pl_results) in pl_results {
if stream_params.stream_type != StreamType::Logs {
continue;
}
for (idx, mut res) in stream_pl_results {
// get json object
let mut local_val = match res.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
// set _id
if let Some(doc_id) = &doc_ids[idx] {
local_val.insert(
"_id".to_string(),
json::Value::String(doc_id.to_owned()),
);
}
if let Some(fields) =
user_defined_schema_map.get(stream_params.stream_name.as_str())
{
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set
.contains(stream_params.stream_name.as_str())
&& originals[idx].is_some()
{
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
originals[idx].clone().unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&stream_params.stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
// handle timestamp
let timestamp = match local_val.get(&cfg.common.column_timestamp) {
Some(v) => match parse_timestamp_micro_from_value(v) {
Ok(t) => t,
Err(_e) => {
bulk_res.errors = true;
add_record_status(
stream_params.stream_name.to_string(),
&doc_ids[idx],
action.clone(),
Some(res),
&mut bulk_res,
Some(TS_PARSE_FAILED.to_string()),
Some(TS_PARSE_FAILED.to_string()),
);
continue;
}
},
None => Utc::now().timestamp_micros(),
};
// check ingestion time
if timestamp < min_ts {
bulk_res.errors = true;
let failure_reason = Some(get_upto_discard_error().to_string());
add_record_status(
stream_params.stream_name.to_string(),
&doc_ids[idx],
action.clone(),
Some(res),
&mut bulk_res,
Some(TS_PARSE_FAILED.to_string()),
failure_reason,
);
continue;
}
local_val.insert(
cfg.common.column_timestamp.clone(),
json::Value::Number(timestamp.into()),
);
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_params.stream_name.to_string())
.or_insert((Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = Some(function_no)
}
}
}
}
}
}
// drop memory-intensive variables
drop(stream_pipeline_inputs);
drop(streams_need_original_set);
drop(user_defined_schema_map);
let (metric_rpt_status_code, response_body) = {
let mut status = IngestionStatus::Bulk(bulk_res);
let write_result = super::write_logs_by_stream(

View File

@ -24,7 +24,7 @@ use chrono::{Duration, Utc};
use config::{
get_config,
meta::{
stream::{Routing, StreamParams, StreamType},
stream::{StreamParams, StreamType},
usage::UsageType,
},
metrics,
@ -39,16 +39,12 @@ use opentelemetry_proto::tonic::{
};
use prost::Message;
use serde_json::json;
use vrl::compiler::runtime::Runtime;
use crate::{
common::meta::{
functions::{StreamTransform, VRLResultResolver},
ingestion::{
AWSRecordType, GCPIngestionResponse, IngestionData, IngestionDataIter, IngestionError,
IngestionRequest, IngestionResponse, IngestionStatus, KinesisFHIngestionResponse,
StreamStatus,
},
common::meta::ingestion::{
AWSRecordType, GCPIngestionResponse, IngestionData, IngestionDataIter, IngestionError,
IngestionRequest, IngestionResponse, IngestionStatus, KinesisFHIngestionResponse,
StreamStatus,
},
service::{
format_stream_name, get_formatted_stream_name, ingestion::check_ingestion_allowed,
@ -81,31 +77,23 @@ pub async fn ingest(
let min_ts = (Utc::now() - Duration::try_hours(cfg.limit.ingest_allowed_upto).unwrap())
.timestamp_micros();
let mut runtime = crate::service::ingestion::init_functions_runtime();
let mut stream_vrl_map: HashMap<String, VRLResultResolver> = HashMap::new();
let mut stream_before_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut stream_after_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut stream_params = vec![StreamParams::new(org_id, &stream_name, StreamType::Logs)];
// Start get routing keys
let mut stream_routing_map: HashMap<String, Vec<Routing>> = HashMap::new();
crate::service::ingestion::get_stream_routing(
StreamParams::new(org_id, &stream_name, StreamType::Logs),
&mut stream_routing_map,
// Start retrieve associated pipeline and construct pipeline components
let executable_pipeline = crate::service::ingestion::get_stream_executable_pipeline(
org_id,
&stream_name,
&StreamType::Logs,
)
.await;
let mut pipeline_inputs = Vec::new();
let mut original_options = Vec::new();
// End pipeline params construction
if let Some(routes) = stream_routing_map.get(&stream_name) {
for route in routes {
stream_params.push(StreamParams::new(
org_id,
&route.destination,
StreamType::Logs,
));
}
if let Some(exec_pl) = &executable_pipeline {
let pl_destinations = exec_pl.get_all_destination_streams();
stream_params.extend(pl_destinations);
}
// End get routing keys
// Start get user defined schema
let mut user_defined_schema_map: HashMap<String, HashSet<String>> = HashMap::new();
@ -118,16 +106,6 @@ pub async fn ingest(
.await;
// End get user defined schema
// Start Register functions for stream
crate::service::ingestion::get_stream_functions(
&stream_params,
&mut stream_before_functions_map,
&mut stream_after_functions_map,
&mut stream_vrl_map,
)
.await;
// End Register functions for index
let json_req: Vec<json::Value>; // to hold json request because of borrow checker
let (endpoint, usage_type, data) = match in_req {
IngestionRequest::JSON(req) => {
@ -186,7 +164,6 @@ pub async fn ingest(
return Err(anyhow::anyhow!("Failed processing: {:?}", e));
}
};
let mut routed_stream_name = stream_name.clone();
if let Some(extend) = extend_json.as_ref() {
for (key, val) in extend.iter() {
@ -194,142 +171,149 @@ pub async fn ingest(
}
}
// store a copy of original data before it's being transformed and/or flattened, unless
// 1. original data is not an object -> won't be flattened.
// 2. no routing and current StreamName not in streams_need_original_set
// store a copy of original data before it's being transformed and/or flattened, when
// 1. original data is an object
let original_data = if item.is_object() {
if stream_routing_map.is_empty()
&& !streams_need_original_set.contains(&routed_stream_name)
{
None
// 2. current stream does not have pipeline
if executable_pipeline.is_none() {
// current stream requires original
streams_need_original_set
.contains(&stream_name)
.then_some(item.to_string())
} else {
// otherwise, make a copy in case the routed stream needs original data
Some(item.to_string())
// 3. with pipeline, storing original as long as streams_need_original_set is not
// empty
// because not sure the pipeline destinations
(!streams_need_original_set.is_empty()).then_some(item.to_string())
}
} else {
None // `item` won't be flattened, no need to store original
};
let main_stream_key = format!("{org_id}/{}/{stream_name}", StreamType::Logs);
// Start row based transform before flattening the value
if let Some(transforms) = stream_before_functions_map.get(&main_stream_key) {
if !transforms.is_empty() {
item = match apply_functions(
item,
transforms,
&stream_vrl_map,
if executable_pipeline.is_some() {
// buffer the records and originals for pipeline batch processing
pipeline_inputs.push(item);
original_options.push(original_data);
} else {
// JSON Flattening
let mut res = flatten::flatten_with_level(item, cfg.limit.ingest_flatten_level)?;
// get json object
let mut local_val = match res.take() {
json::Value::Object(val) => val,
_ => unreachable!(),
};
if let Some(fields) = user_defined_schema_map.get(&stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&stream_name) && original_data.is_some() {
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&routed_stream_name,
&mut runtime,
) {
Ok(res) => res,
Err(e) => {
stream_status.status.failed += 1;
stream_status.status.error = e.to_string();
continue;
}
}
&stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
}
// end row based transformation
// JSON Flattening
let item = flatten::flatten_with_level(item, cfg.limit.ingest_flatten_level)?;
// Start re-routing if exists
if let Some(routings) = stream_routing_map.get(&routed_stream_name) {
if !routings.is_empty() {
for route in routings {
let mut is_routed = true;
let val = &route.routing;
for q_condition in val.iter() {
if !q_condition.evaluate(item.as_object().unwrap()).await {
is_routed = false;
break;
}
}
if !val.is_empty() && is_routed {
routed_stream_name = route.destination.clone();
break;
}
}
}
}
// End re-routing
let key = format!("{org_id}/{}/{routed_stream_name}", StreamType::Logs);
// Start row based transform
let mut res = if let Some(transforms) = stream_after_functions_map.get(&key) {
match apply_functions(
item,
transforms,
&stream_vrl_map,
org_id,
&routed_stream_name,
&mut runtime,
) {
Ok(res) => res,
// handle timestamp
let timestamp = match handle_timestamp(&mut local_val, min_ts) {
Ok(ts) => ts,
Err(e) => {
stream_status.status.failed += 1;
stream_status.status.error = e.to_string();
continue;
}
}
} else {
item
};
// end row based transform
};
// get json object
let mut local_val = match res.take() {
json::Value::Object(val) => val,
_ => unreachable!(),
};
if let Some(fields) = user_defined_schema_map.get(&routed_stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_name.clone())
.or_insert_with(|| (Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = need_usage_report.then_some(0); // no pl -> no func
}
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&routed_stream_name) && original_data.is_some() {
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&routed_stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
// handle timestamp
let timestamp = match handle_timestamp(&mut local_val, min_ts) {
Ok(ts) => ts,
// batch process records through pipeline
if let Some(exec_pl) = &executable_pipeline {
let records_count = pipeline_inputs.len();
match exec_pl.process_batch(org_id, pipeline_inputs).await {
Err(e) => {
stream_status.status.failed += 1;
stream_status.status.error = e.to_string();
continue;
log::error!(
"[Pipeline] for stream {}/{}: Batch execution error: {}.",
org_id,
stream_name,
e
);
stream_status.status.failed += records_count as u32;
stream_status.status.error = format!("Pipeline batch execution error: {}", e);
}
};
Ok(pl_results) => {
let function_no = exec_pl.num_of_func();
for (stream_params, stream_pl_results) in pl_results {
if stream_params.stream_type != StreamType::Logs {
continue;
}
let function_no = stream_before_functions_map
.get(&main_stream_key)
.map(|v| v.len())
.unwrap_or_default()
+ stream_after_functions_map
.get(&key)
.map(|v| v.len())
.unwrap_or_default();
let (ts_data, fn_num) = json_data_by_stream
.entry(routed_stream_name.clone())
.or_insert((Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = need_usage_report.then_some(function_no);
for (idx, mut res) in stream_pl_results {
// get json object
let mut local_val = match res.take() {
json::Value::Object(val) => val,
_ => unreachable!(),
};
if let Some(fields) =
user_defined_schema_map.get(stream_params.stream_name.as_str())
{
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(stream_params.stream_name.as_str())
&& original_options[idx].is_some()
{
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_options[idx].clone().unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&stream_params.stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
// handle timestamp
let timestamp = match handle_timestamp(&mut local_val, min_ts) {
Ok(ts) => ts,
Err(e) => {
stream_status.status.failed += 1;
stream_status.status.error = e.to_string();
continue;
}
};
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_params.stream_name.to_string())
.or_insert_with(|| (Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = need_usage_report.then_some(function_no);
}
}
}
}
}
// if no data, fast return
@ -340,6 +324,12 @@ pub async fn ingest(
));
}
// drop memory-intensive variables
drop(streams_need_original_set);
drop(executable_pipeline);
drop(original_options);
drop(user_defined_schema_map);
let (metric_rpt_status_code, response_body) = {
let mut status = IngestionStatus::Record(stream_status.status);
let write_result = super::write_logs_by_stream(
@ -386,45 +376,12 @@ pub async fn ingest(
])
.inc();
// drop variables
drop(runtime);
drop(stream_vrl_map);
drop(stream_routing_map);
drop(user_defined_schema_map);
Ok(IngestionResponse::new(
http::StatusCode::OK.into(),
vec![response_body],
))
}
pub fn apply_functions<'a>(
item: json::Value,
local_trans: &[StreamTransform],
stream_vrl_map: &'a HashMap<String, VRLResultResolver>,
org_id: &'a str,
stream_name: &'a str,
runtime: &mut Runtime,
) -> Result<json::Value> {
let mut value = item;
if !local_trans.is_empty() {
value = crate::service::ingestion::apply_stream_functions(
local_trans,
value,
stream_vrl_map,
org_id,
stream_name,
runtime,
)?;
}
if value.is_null() || !value.is_object() {
Err(anyhow::Error::msg("apply functions failure"))
} else {
Ok(value)
}
}
pub fn handle_timestamp(
local_val: &mut json::Map<String, json::Value>,
min_ts: i64,

View File

@ -25,6 +25,7 @@ use arrow_schema::{DataType, Field};
use config::{
get_config,
meta::{
alerts::alert::Alert,
stream::{PartitionTimeLevel, StreamParams, StreamPartition, StreamType},
usage::{RequestStats, UsageType},
},
@ -42,9 +43,9 @@ use super::{
schema::stream_schema_exists,
};
use crate::{
common::meta::{alerts::alert::Alert, ingestion::IngestionStatus, stream::SchemaRecords},
common::meta::{ingestion::IngestionStatus, stream::SchemaRecords},
service::{
db, ingestion::get_write_partition_key, schema::check_for_schema,
alerts::alert::AlertExt, db, ingestion::get_write_partition_key, schema::check_for_schema,
usage::report_request_usage_stats,
},
};
@ -57,7 +58,7 @@ pub mod syslog;
static BULK_OPERATORS: [&str; 3] = ["create", "index", "update"];
type LogJsonData = (Vec<(i64, Map<String, Value>)>, Option<usize>);
pub type O2IngestJsonData = (Vec<(i64, Map<String, Value>)>, Option<usize>);
fn parse_bulk_index(v: &Value) -> Option<(String, String, Option<String>)> {
let local_val = v.as_object().unwrap();
@ -190,7 +191,7 @@ async fn write_logs_by_stream(
time_stats: (i64, &Instant), // started_at
usage_type: UsageType,
status: &mut IngestionStatus,
json_data_by_stream: HashMap<String, LogJsonData>,
json_data_by_stream: HashMap<String, O2IngestJsonData>,
) -> Result<()> {
for (stream_name, (json_data, fn_num)) in json_data_by_stream {
// check if we are allowed to ingest

View File

@ -22,7 +22,7 @@ use chrono::{Duration, Utc};
use config::{
get_config,
meta::{
stream::{Routing, StreamParams, StreamType},
stream::{StreamParams, StreamType},
usage::UsageType,
},
metrics,
@ -36,10 +36,7 @@ use opentelemetry_proto::tonic::collector::logs::v1::{
use prost::Message;
use crate::{
common::meta::{
functions::{StreamTransform, VRLResultResolver},
ingestion::{IngestionStatus, StreamStatus},
},
common::meta::ingestion::{IngestionStatus, StreamStatus},
handler::http::request::CONTENT_TYPE_PROTO,
service::{
format_stream_name,
@ -61,7 +58,6 @@ pub async fn handle_grpc_request(
) -> Result<HttpResponse> {
let start = std::time::Instant::now();
let started_at = Utc::now().timestamp_micros();
let cfg = get_config();
// check stream
let stream_name = match in_stream_name {
@ -70,34 +66,28 @@ pub async fn handle_grpc_request(
};
check_ingestion_allowed(org_id, Some(&stream_name))?;
let cfg = get_config();
let min_ts = (Utc::now() - Duration::try_hours(cfg.limit.ingest_allowed_upto).unwrap())
.timestamp_micros();
let mut runtime = crate::service::ingestion::init_functions_runtime();
let mut stream_vrl_map: HashMap<String, VRLResultResolver> = HashMap::new();
let mut stream_before_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut stream_after_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut stream_params = vec![StreamParams::new(org_id, &stream_name, StreamType::Logs)];
// Start get routing keys
let mut stream_routing_map: HashMap<String, Vec<Routing>> = HashMap::new();
crate::service::ingestion::get_stream_routing(
StreamParams::new(org_id, &stream_name, StreamType::Logs),
&mut stream_routing_map,
// Start retrieve associated pipeline and construct pipeline components
let executable_pipeline = crate::service::ingestion::get_stream_executable_pipeline(
org_id,
&stream_name,
&StreamType::Logs,
)
.await;
let mut pipeline_inputs = Vec::new();
let mut original_options = Vec::new();
let mut timestamps = Vec::new();
// End pipeline construction
if let Some(routes) = stream_routing_map.get(&stream_name) {
for route in routes {
stream_params.push(StreamParams::new(
org_id,
&route.destination,
StreamType::Logs,
));
}
if let Some(exec_pl) = &executable_pipeline {
let pl_destinations = exec_pl.get_all_destination_streams();
stream_params.extend(pl_destinations);
}
// End get routing keys
// Start get user defined schema
let mut user_defined_schema_map: HashMap<String, HashSet<String>> = HashMap::new();
@ -110,19 +100,13 @@ pub async fn handle_grpc_request(
.await;
// End get user defined schema
// Start Register functions for stream
crate::service::ingestion::get_stream_functions(
&stream_params,
&mut stream_before_functions_map,
&mut stream_after_functions_map,
&mut stream_vrl_map,
)
.await;
// End Register functions for index
let mut stream_status = StreamStatus::new(&stream_name);
let mut json_data_by_stream = HashMap::new();
let mut res = ExportLogsServiceResponse {
partial_success: None,
};
for resource_log in &request.resource_logs {
for instrumentation_logs in &resource_log.scope_logs {
for log_record in &instrumentation_logs.log_records {
@ -205,131 +189,149 @@ pub async fn handle_grpc_request(
.into();
}
};
let main_stream_key = format!("{org_id}/{}/{stream_name}", StreamType::Logs);
// store a copy of original data before it's being transformed and/or flattened,
// unless
// 1. original data is not an object -> won't be flattened.
// 2. no routing and current StreamName not in streams_need_original_set
// store a copy of original data before it's modified, when
// 1. original data is an object
let original_data = if rec.is_object() {
if stream_routing_map.is_empty()
&& !streams_need_original_set.contains(&stream_name)
{
None
// 2. current stream does not have pipeline
if executable_pipeline.is_none() {
// current stream requires original
streams_need_original_set
.contains(&stream_name)
.then_some(rec.to_string())
} else {
// otherwise, make a copy in case the routed stream needs original data
Some(rec.to_string())
// 3. with pipeline, storing original as long as streams_need_original_set
// is not empty
// because not sure the pipeline destinations
(!streams_need_original_set.is_empty()).then_some(rec.to_string())
}
} else {
None // `item` won't be flattened, no need to store original
};
// Start row based transform before flattening the value
if let Some(transforms) = stream_before_functions_map.get(&main_stream_key) {
if !transforms.is_empty() {
rec = crate::service::ingestion::apply_stream_functions(
transforms,
rec,
&stream_vrl_map,
if executable_pipeline.is_some() {
// buffer the records and originals for pipeline batch processing
pipeline_inputs.push(rec);
original_options.push(original_data);
timestamps.push(timestamp);
} else {
// flattening
rec = flatten::flatten_with_level(rec, cfg.limit.ingest_flatten_level)?;
// get json object
let mut local_val = match rec.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
if let Some(fields) = user_defined_schema_map.get(&stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&stream_name) && original_data.is_some() {
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&stream_name,
&mut runtime,
)?;
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_name.clone())
.or_insert((Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = Some(0); // no pl -> no func
}
// end row based transformation
// flattening
rec = flatten::flatten_with_level(rec, cfg.limit.ingest_flatten_level)?;
let mut routed_stream_name = stream_name.clone();
// Start re-routing if exists
if let Some(routings) = stream_routing_map.get(&stream_name) {
if !routings.is_empty() {
for route in routings {
let mut is_routed = true;
let val = &route.routing;
for q_condition in val.iter() {
if !q_condition.evaluate(rec.as_object().unwrap()).await {
is_routed = false;
break;
}
}
if !val.is_empty() && is_routed {
routed_stream_name = route.destination.clone();
break;
}
}
}
}
// End re-routing
let key = format!("{org_id}/{}/{routed_stream_name}", StreamType::Logs);
// Start row based transform
if let Some(transforms) = stream_after_functions_map.get(&key) {
if !transforms.is_empty() {
rec = crate::service::ingestion::apply_stream_functions(
transforms,
rec,
&stream_vrl_map,
org_id,
&routed_stream_name,
&mut runtime,
)?;
}
}
// end row based transform
// get json object
let mut local_val = match rec.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
if let Some(fields) = user_defined_schema_map.get(&routed_stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&routed_stream_name)
&& original_data.is_some()
{
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&routed_stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
let function_no = stream_before_functions_map
.get(&main_stream_key)
.map(|v| v.len())
.unwrap_or_default()
+ stream_after_functions_map
.get(&key)
.map(|v| v.len())
.unwrap_or_default();
let (ts_data, fn_num) = json_data_by_stream
.entry(routed_stream_name.clone())
.or_insert((Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = Some(function_no);
}
}
}
let mut res = ExportLogsServiceResponse {
partial_success: None,
};
// batch process records through pipeline
if let Some(exec_pl) = &executable_pipeline {
let records_count = pipeline_inputs.len();
match exec_pl.process_batch(org_id, pipeline_inputs).await {
Err(e) => {
log::error!(
"[Pipeline] for stream {}/{}: Batch execution error: {}.",
org_id,
stream_name,
e
);
stream_status.status.failed += records_count as u32;
stream_status.status.error = format!("Pipeline batch execution error: {}", e);
}
Ok(pl_results) => {
let function_no = exec_pl.num_of_func();
for (stream_params, stream_pl_results) in pl_results {
if stream_params.stream_type != StreamType::Logs {
continue;
}
for (idx, mut res) in stream_pl_results {
// get json object
let mut local_val = match res.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
if let Some(fields) =
user_defined_schema_map.get(stream_params.stream_name.as_str())
{
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(stream_params.stream_name.as_str())
&& original_options[idx].is_some()
{
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_options[idx].clone().unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&stream_params.stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_params.stream_name.to_string())
.or_insert((Vec::new(), None));
ts_data.push((timestamps[idx], local_val));
*fn_num = Some(function_no); // no pl -> no func
}
}
}
}
}
// drop variables
drop(executable_pipeline);
drop(original_options);
drop(timestamps);
drop(user_defined_schema_map);
// Update partial success
if stream_status.status.failed > 0 {
res.partial_success = Some(ExportLogsPartialSuccess {
rejected_log_records: stream_status.status.failed as i64,
error_message: stream_status.status.error.clone(),
});
}
// if no data, fast return
if json_data_by_stream.is_empty() {
@ -400,12 +402,6 @@ pub async fn handle_grpc_request(
])
.inc();
// drop variables
drop(runtime);
drop(stream_vrl_map);
drop(stream_routing_map);
drop(user_defined_schema_map);
return Ok(HttpResponse::Ok()
.status(http::StatusCode::OK)
.content_type(CONTENT_TYPE_PROTO)

View File

@ -22,7 +22,7 @@ use chrono::{Duration, Utc};
use config::{
get_config,
meta::{
stream::{Routing, StreamParams, StreamType},
stream::{StreamParams, StreamType},
usage::UsageType,
},
metrics,
@ -37,7 +37,6 @@ use prost::Message;
use crate::{
common::meta::{
functions::{StreamTransform, VRLResultResolver},
http::HttpResponse as MetaHttpResponse,
ingestion::{IngestionStatus, StreamStatus},
},
@ -106,31 +105,24 @@ pub async fn logs_json_handler(
let min_ts = (Utc::now() - Duration::try_hours(cfg.limit.ingest_allowed_upto).unwrap())
.timestamp_micros();
let mut runtime = crate::service::ingestion::init_functions_runtime();
let mut stream_vrl_map: HashMap<String, VRLResultResolver> = HashMap::new();
let mut stream_before_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut stream_after_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut stream_params = vec![StreamParams::new(org_id, &stream_name, StreamType::Logs)];
// Start get routing keys
let mut stream_routing_map: HashMap<String, Vec<Routing>> = HashMap::new();
crate::service::ingestion::get_stream_routing(
StreamParams::new(org_id, &stream_name, StreamType::Logs),
&mut stream_routing_map,
// Start retrieve associated pipeline and construct pipeline components
let executable_pipeline = crate::service::ingestion::get_stream_executable_pipeline(
org_id,
&stream_name,
&StreamType::Logs,
)
.await;
let mut pipeline_inputs = Vec::new();
let mut original_options = Vec::new();
let mut timestamps = Vec::new();
// End pipeline params construction
if let Some(routes) = stream_routing_map.get(&stream_name) {
for route in routes {
stream_params.push(StreamParams::new(
org_id,
&route.destination,
StreamType::Logs,
));
}
if let Some(pl) = &executable_pipeline {
let pl_destinations = pl.get_all_destination_streams();
stream_params.extend(pl_destinations);
}
// End get routing keys
// Start get user defined schema
let mut user_defined_schema_map: HashMap<String, HashSet<String>> = HashMap::new();
@ -143,16 +135,6 @@ pub async fn logs_json_handler(
.await;
// End get user defined schema
// Start Register functions for stream
crate::service::ingestion::get_stream_functions(
&stream_params,
&mut stream_before_functions_map,
&mut stream_after_functions_map,
&mut stream_vrl_map,
)
.await;
// End Register functions for stream
let mut stream_status = StreamStatus::new(&stream_name);
let mut json_data_by_stream = HashMap::new();
@ -195,6 +177,10 @@ pub async fn logs_json_handler(
},
};
let mut res = ExportLogsServiceResponse {
partial_success: None,
};
for res_log in logs.iter() {
let mut service_att_map: json::Map<String, json::Value> = json::Map::new();
if res_log.get("resource").is_some() {
@ -328,133 +314,149 @@ pub async fn logs_json_handler(
value = json::to_value(local_val)?;
// store a copy of original data before it's being transformed and/or flattened,
// unless
// 1. original data is not an object -> won't be flattened.
// 2. no routing and current StreamName not in streams_need_original_set
// store a copy of original data before it's modified, when
// 1. original data is an object
let original_data = if value.is_object() {
if stream_routing_map.is_empty()
&& !streams_need_original_set.contains(&stream_name)
{
None
// 2. current stream does not have pipeline
if executable_pipeline.is_none() {
// current stream requires original
streams_need_original_set
.contains(&stream_name)
.then_some(value.to_string())
} else {
// otherwise, make a copy in case the routed stream needs original data
Some(value.to_string())
// 3. with pipeline, storing original as long as streams_need_original_set
// is not empty
// because not sure the pipeline destinations
(!streams_need_original_set.is_empty()).then_some(value.to_string())
}
} else {
None // `item` won't be flattened, no need to store original
};
let main_stream_key = format!("{org_id}/{}/{stream_name}", StreamType::Logs);
if executable_pipeline.is_some() {
// buffer the records and originals for pipeline batch processing
pipeline_inputs.push(value);
original_options.push(original_data);
timestamps.push(timestamp);
} else {
// JSON Flattening
value =
flatten::flatten_with_level(value, cfg.limit.ingest_flatten_level).unwrap();
// Start row based transform before flattening the value
if let Some(transforms) = stream_before_functions_map.get(&main_stream_key) {
if !transforms.is_empty() {
value = crate::service::ingestion::apply_stream_functions(
transforms,
value,
&stream_vrl_map,
// get json object
let mut local_val = match value.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
if let Some(fields) = user_defined_schema_map.get(&stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&stream_name) && original_data.is_some() {
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&stream_name,
&mut runtime,
)
.unwrap();
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_name.clone())
.or_insert((Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = Some(0); // no pl -> no func
}
// end row based transformation
// JSON Flattening
value = flatten::flatten_with_level(value, cfg.limit.ingest_flatten_level).unwrap();
let mut routed_stream_name = stream_name.clone();
// Start re-routing if exists
if let Some(routing) = stream_routing_map.get(&stream_name) {
if !routing.is_empty() {
for route in routing {
let mut is_routed = true;
let val = &route.routing;
for q_condition in val.iter() {
if !q_condition.evaluate(value.as_object().unwrap()).await {
is_routed = false;
break;
}
}
if is_routed && !val.is_empty() {
routed_stream_name = route.destination.clone();
break;
}
}
}
}
// End re-routing
let key = format!("{org_id}/{}/{routed_stream_name}", StreamType::Logs);
// Start row based transform
if let Some(transforms) = stream_after_functions_map.get(&key) {
if !transforms.is_empty() {
value = crate::service::ingestion::apply_stream_functions(
transforms,
value,
&stream_vrl_map,
org_id,
&routed_stream_name,
&mut runtime,
)
.unwrap();
}
}
// end row based transform
// get json object
let mut local_val = match value.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
if let Some(fields) = user_defined_schema_map.get(&routed_stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&routed_stream_name)
&& original_data.is_some()
{
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&routed_stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
let function_no = stream_before_functions_map
.get(&main_stream_key)
.map(|v| v.len())
.unwrap_or_default()
+ stream_after_functions_map
.get(&key)
.map(|v| v.len())
.unwrap_or_default();
let (ts_data, fn_num) = json_data_by_stream
.entry(routed_stream_name.clone())
.or_insert((Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = Some(function_no);
}
}
}
let mut res = ExportLogsServiceResponse {
partial_success: None,
};
// batch process records through pipeline
if let Some(exec_pl) = &executable_pipeline {
let records_count = pipeline_inputs.len();
match exec_pl.process_batch(org_id, pipeline_inputs).await {
Err(e) => {
log::error!(
"[Pipeline] for stream {}/{}: Batch execution error: {}.",
org_id,
stream_name,
e
);
stream_status.status.failed += records_count as u32;
stream_status.status.error = format!("Pipeline batch execution error: {}", e);
}
Ok(pl_results) => {
let function_no = exec_pl.num_of_func();
for (stream_params, stream_pl_results) in pl_results {
if stream_params.stream_type != StreamType::Logs {
continue;
}
for (idx, mut res) in stream_pl_results {
// get json object
let mut local_val = match res.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
if let Some(fields) =
user_defined_schema_map.get(stream_params.stream_name.as_str())
{
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(stream_params.stream_name.as_str())
&& original_options[idx].is_some()
{
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_options[idx].clone().unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&stream_params.stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_params.stream_name.to_string())
.or_insert((Vec::new(), None));
ts_data.push((timestamps[idx], local_val));
*fn_num = Some(function_no); // no pl -> no func
}
}
}
}
}
// drop variables
drop(executable_pipeline);
drop(original_options);
drop(timestamps);
drop(user_defined_schema_map);
// Update partial success
if stream_status.status.failed > 0 {
res.partial_success = Some(ExportLogsPartialSuccess {
rejected_log_records: stream_status.status.failed as i64,
error_message: stream_status.status.error.clone(),
});
}
// if no data, fast return
if json_data_by_stream.is_empty() {
@ -520,12 +522,6 @@ pub async fn logs_json_handler(
])
.inc();
// drop variables
drop(runtime);
drop(stream_vrl_map);
drop(stream_routing_map);
drop(user_defined_schema_map);
return Ok(HttpResponse::Ok()
.status(http::StatusCode::OK)
.content_type(CONTENT_TYPE_JSON)

View File

@ -23,7 +23,10 @@ use anyhow::Result;
use chrono::{Duration, Utc};
use config::{
get_config,
meta::stream::{Routing, StreamParams, StreamType},
meta::{
stream::{StreamParams, StreamType},
usage::UsageType,
},
metrics,
utils::{flatten, json},
ID_COL_NAME, ORIGINAL_DATA_COL_NAME,
@ -35,7 +38,6 @@ use crate::{
common::{
infra::config::SYSLOG_ROUTES,
meta::{
functions::{StreamTransform, VRLResultResolver},
http::HttpResponse as MetaHttpResponse,
ingestion::{IngestionResponse, IngestionStatus, StreamStatus},
syslog::SyslogRoute,
@ -46,6 +48,7 @@ use crate::{
pub async fn ingest(msg: &str, addr: SocketAddr) -> Result<HttpResponse> {
let start = std::time::Instant::now();
let started_at: i64 = Utc::now().timestamp_micros();
let ip = addr.ip();
let matching_route = get_org_for_ip(ip).await;
@ -80,31 +83,23 @@ pub async fn ingest(msg: &str, addr: SocketAddr) -> Result<HttpResponse> {
let min_ts = (Utc::now() - Duration::try_hours(cfg.limit.ingest_allowed_upto).unwrap())
.timestamp_micros();
let mut runtime = crate::service::ingestion::init_functions_runtime();
let mut stream_vrl_map: HashMap<String, VRLResultResolver> = HashMap::new();
let mut stream_before_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut stream_after_functions_map: HashMap<String, Vec<StreamTransform>> = HashMap::new();
let mut stream_params = vec![StreamParams::new(org_id, &stream_name, StreamType::Logs)];
// Start get routing keys
let mut stream_routing_map: HashMap<String, Vec<Routing>> = HashMap::new();
crate::service::ingestion::get_stream_routing(
StreamParams::new(org_id, &stream_name, StreamType::Logs),
&mut stream_routing_map,
// Start retrieve associated pipeline and construct pipeline components
let executable_pipeline = crate::service::ingestion::get_stream_executable_pipeline(
org_id,
&stream_name,
&StreamType::Logs,
)
.await;
let mut pipeline_inputs = Vec::new();
let mut original_options = Vec::new();
// End pipeline construction
if let Some(routes) = stream_routing_map.get(&stream_name) {
for route in routes {
stream_params.push(StreamParams::new(
org_id,
&route.destination,
StreamType::Logs,
));
}
if let Some(pl) = &executable_pipeline {
let pl_destinations = pl.get_all_destination_streams();
stream_params.extend(pl_destinations);
}
// End get routing keys
// Start get user defined schema
let mut user_defined_schema_map: HashMap<String, HashSet<String>> = HashMap::new();
@ -117,153 +112,184 @@ pub async fn ingest(msg: &str, addr: SocketAddr) -> Result<HttpResponse> {
.await;
// End get user defined schema
// Start Register functions for stream
crate::service::ingestion::get_stream_functions(
&stream_params,
&mut stream_before_functions_map,
&mut stream_after_functions_map,
&mut stream_vrl_map,
)
.await;
// End Register functions for stream
let mut stream_status = StreamStatus::new(&stream_name);
let mut json_data_by_stream = HashMap::new();
// parse msg to json::Value
let parsed_msg = syslog_loose::parse_message(msg);
let mut value = message_to_value(parsed_msg);
// store a copy of original data before it's being transformed and/or flattened, unless
// 1. original data is not an object -> won't be flattened.
// 2. no routing and current StreamName not in streams_need_original_set
// store a copy of original data before it's modified, when
// 1. original data is an object
let original_data = if value.is_object() {
if stream_routing_map.is_empty() && !streams_need_original_set.contains(&stream_name) {
None
// 2. current stream does not have pipeline
if executable_pipeline.is_none() {
// current stream requires original
streams_need_original_set
.contains(&stream_name)
.then_some(value.to_string())
} else {
// otherwise, make a copy in case the routed stream needs original data
Some(value.to_string())
// 3. with pipeline, storing original as long as streams_need_original_set is not empty
// because not sure the pipeline destinations
(!streams_need_original_set.is_empty()).then_some(value.to_string())
}
} else {
None // `item` won't be flattened, no need to store original
};
let main_stream_key = format!("{org_id}/{}/{stream_name}", StreamType::Logs);
if executable_pipeline.is_some() {
// buffer the records and originals for pipeline batch processing
pipeline_inputs.push(value);
original_options.push(original_data);
} else {
// JSON Flattening
value = flatten::flatten_with_level(value, cfg.limit.ingest_flatten_level).unwrap();
// Start row based transform. Apply vrl functions with apply_before_flattening flag
if let Some(transforms) = stream_before_functions_map.get(&main_stream_key) {
if !transforms.is_empty() {
value = crate::service::ingestion::apply_stream_functions(
transforms,
value,
&stream_vrl_map,
// get json object
let mut local_val = match value.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
if let Some(fields) = user_defined_schema_map.get(&stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&stream_name) && original_data.is_some() {
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&stream_name,
&mut runtime,
)?;
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
// handle timestamp
let timestamp = match handle_timestamp(&mut local_val, min_ts) {
Ok(ts) => ts,
Err(e) => {
stream_status.status.failed += 1;
stream_status.status.error = e.to_string();
return Ok(HttpResponse::Ok().json(IngestionResponse::new(
http::StatusCode::OK.into(),
vec![stream_status],
))); // just return
}
};
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_name.clone())
.or_insert((Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = Some(0); // no pl -> no func
}
// end row based transformation
// JSON Flattening
value = flatten::flatten_with_level(value, cfg.limit.ingest_flatten_level).unwrap();
let mut routed_stream_name = stream_name.clone();
// Start re-rerouting if exists
if let Some(routings) = stream_routing_map.get(&stream_name) {
if !routings.is_empty() {
for route in routings {
let mut is_routed = true;
let val = &route.routing;
for q_condition in val.iter() {
if !q_condition.evaluate(value.as_object().unwrap()).await {
is_routed = false;
break;
// batch process records through pipeline
if let Some(exec_pl) = &executable_pipeline {
let records_count = pipeline_inputs.len();
match exec_pl.process_batch(org_id, pipeline_inputs).await {
Err(e) => {
log::error!(
"[Pipeline] for stream {}/{}: Batch execution error: {}.",
org_id,
stream_name,
e
);
stream_status.status.failed += records_count as u32;
stream_status.status.error = format!("Pipeline batch execution error: {}", e);
}
Ok(pl_results) => {
let function_no = exec_pl.num_of_func();
for (stream_params, stream_pl_results) in pl_results {
if stream_params.stream_type != StreamType::Logs {
continue;
}
for (idx, mut res) in stream_pl_results {
// get json object
let mut local_val = match res.take() {
json::Value::Object(val) => val,
_ => unreachable!(),
};
if let Some(fields) =
user_defined_schema_map.get(stream_params.stream_name.as_str())
{
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(stream_params.stream_name.as_str())
&& original_options[idx].is_some()
{
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_options[idx].clone().unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&stream_params.stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
// handle timestamp
let timestamp = match handle_timestamp(&mut local_val, min_ts) {
Ok(ts) => ts,
Err(e) => {
stream_status.status.failed += 1;
stream_status.status.error = e.to_string();
continue;
}
};
let (ts_data, fn_num) = json_data_by_stream
.entry(stream_params.stream_name.to_string())
.or_insert_with(|| (Vec::new(), None));
ts_data.push((timestamp, local_val));
*fn_num = Some(function_no);
}
}
if !val.is_empty() && is_routed {
routed_stream_name = route.destination.clone();
break;
}
}
}
}
// End re-routing
let key = format!("{org_id}/{}/{routed_stream_name}", StreamType::Logs);
// Start row based transform
if let Some(transforms) = stream_after_functions_map.get(&key) {
if !transforms.is_empty() {
value = crate::service::ingestion::apply_stream_functions(
transforms,
value,
&stream_vrl_map,
org_id,
&routed_stream_name,
&mut runtime,
)
.unwrap();
}
}
// end row based transform
if value.is_null() || !value.is_object() {
stream_status.status.failed += 1; // transform failed or dropped
// if no data, fast return
if json_data_by_stream.is_empty() {
return Ok(HttpResponse::Ok().json(IngestionResponse::new(
http::StatusCode::OK.into(),
vec![stream_status],
))); // just return
}
// End row based transform
// get json object
let mut local_val = match value.take() {
json::Value::Object(v) => v,
_ => unreachable!(),
};
if let Some(fields) = user_defined_schema_map.get(&routed_stream_name) {
local_val = crate::service::logs::refactor_map(local_val, fields);
}
// add `_original` and '_record_id` if required by StreamSettings
if streams_need_original_set.contains(&routed_stream_name) && original_data.is_some() {
local_val.insert(
ORIGINAL_DATA_COL_NAME.to_string(),
original_data.unwrap().into(),
);
let record_id = crate::service::ingestion::generate_record_id(
org_id,
&routed_stream_name,
&StreamType::Logs,
);
local_val.insert(
ID_COL_NAME.to_string(),
json::Value::String(record_id.to_string()),
);
}
// handle timestamp
let timestamp = match handle_timestamp(&mut local_val, min_ts) {
Ok(ts) => ts,
Err(e) => {
stream_status.status.failed += 1;
stream_status.status.error = e.to_string();
return Ok(HttpResponse::Ok().json(IngestionResponse::new(
http::StatusCode::OK.into(),
vec![stream_status],
))); // just return
}
};
// drop memory-intensive variables
drop(streams_need_original_set);
drop(executable_pipeline);
drop(original_options);
drop(user_defined_schema_map);
let (metric_rpt_status_code, response_body) = {
let mut status = IngestionStatus::Record(stream_status.status);
let write_result = super::write_logs(
let write_result = super::write_logs_by_stream(
0,
org_id,
&routed_stream_name,
"",
(started_at, &start),
UsageType::Syslog,
&mut status,
vec![(timestamp, local_val)],
json_data_by_stream,
)
.await;
stream_status.status = match status {
@ -299,12 +325,6 @@ pub async fn ingest(msg: &str, addr: SocketAddr) -> Result<HttpResponse> {
])
.inc();
// drop variables
drop(runtime);
drop(stream_vrl_map);
drop(stream_routing_map);
drop(user_defined_schema_map);
Ok(HttpResponse::Ok().json(IngestionResponse::new(
http::StatusCode::OK.into(),
vec![response_body],

Some files were not shown because too many files have changed in this diff Show More