>,
37 | shutdown_handle: Shutdown,
38 | ) -> Self {
39 | Self {
40 | rebalance_rx,
41 | stream_state,
42 | shutdown_handle,
43 | }
44 | }
45 |
46 | pub fn start(self) {
47 | let mut rebalance_receiver = self.rebalance_rx;
48 | let stream_state = self.stream_state.clone();
49 | let shutdown_handle = self.shutdown_handle.clone();
50 | let tokio_runtime_handle = Handle::current();
51 |
52 | std::thread::Builder::new().name("rebalance-listener-thread".to_string()).spawn(move || {
53 | tokio_runtime_handle.block_on(async move {
54 | loop {
55 | tokio::select! {
56 | rebalance = rebalance_receiver.recv() => {
57 | match rebalance {
58 | Some(RebalanceEvent::Assign(tpl)) => info!("RebalanceEvent Assign: {:?}", tpl),
59 | Some(RebalanceEvent::Revoke(tpl, callback)) => {
60 | info!("RebalanceEvent Revoke: {:?}", tpl);
61 | if let Ok(mut stream_state) = stream_state.try_write() {
62 | stream_state.terminate_partition_streams(tpl).await;
63 | drop(stream_state);
64 | } else {
65 | warn!("Stream state lock is busy, skipping rebalance revoke for {:?}", tpl);
66 | }
67 | if let Err(err) = callback.send(()) {
68 | warn!("Error during sending response to context. Cause: {:?}", err);
69 | }
70 | info!("Finished Rebalance Revoke");
71 | }
72 | None => {
73 | info!("Rebalance event sender is closed!");
74 | break
75 | }
76 | }
77 | },
78 | _ = shutdown_handle.recv() => {
79 | info!("Gracefully stopping rebalance listener!");
80 | break;
81 | },
82 | }
83 | }
84 | })
85 | }).expect("Failed to start rebalance listener thread");
86 | }
87 | }
88 |
--------------------------------------------------------------------------------
/src/connectors/kafka/sink.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 | use crate::connectors::common::build_runtime;
19 | use crate::connectors::common::processor::Processor;
20 | use crate::connectors::kafka::consumer::KafkaStreams;
21 | use crate::connectors::kafka::processor::StreamWorker;
22 | use crate::connectors::kafka::ConsumerRecord;
23 | use anyhow::Result;
24 | use futures_util::StreamExt;
25 | use rdkafka::consumer::Consumer;
26 | use std::sync::Arc;
27 | use tokio::runtime::Runtime;
28 | use tracing::{error, info};
29 |
30 | pub struct KafkaSinkConnector
31 | where
32 | P: Processor, ()>,
33 | {
34 | streams: KafkaStreams,
35 | stream_processor: Arc>,
36 | runtime: Runtime,
37 | }
38 |
39 | impl KafkaSinkConnector
40 | where
41 | P: Processor, ()> + Send + Sync + 'static,
42 | {
43 | pub fn new(kafka_streams: KafkaStreams, processor: P) -> Self {
44 | let consumer = kafka_streams.consumer();
45 | let stream_processor = Arc::new(StreamWorker::new(
46 | Arc::new(processor),
47 | Arc::clone(&consumer),
48 | ));
49 |
50 | let runtime = build_runtime(
51 | consumer.context().config.partition_listener_concurrency,
52 | "kafka-sink-worker",
53 | )
54 | .expect("Failed to build runtime");
55 | let _ = runtime.enter();
56 |
57 | Self {
58 | streams: kafka_streams,
59 | stream_processor,
60 | runtime,
61 | }
62 | }
63 |
64 | pub async fn run(self) -> Result<()> {
65 | self.streams
66 | .partitioned()
67 | .map(|partition_stream| {
68 | let worker = Arc::clone(&self.stream_processor);
69 | let tp = partition_stream.topic_partition().clone();
70 | self.runtime.spawn(async move {
71 | partition_stream
72 | .run_drain(|partition_records| async {
73 | info!("Starting task for partition: {:?}", tp);
74 |
75 | worker
76 | .process_partition(tp.clone(), partition_records)
77 | .await
78 | .unwrap();
79 | })
80 | .await;
81 |
82 | info!("Task completed for partition: {:?}", tp);
83 | })
84 | })
85 | .for_each_concurrent(None, |task| async {
86 | if let Err(e) = task.await {
87 | error!("Task failed: {:?}", e);
88 | }
89 | })
90 | .await;
91 |
92 | Ok(())
93 | }
94 | }
95 |
--------------------------------------------------------------------------------
/src/connectors/kafka/state.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use crate::connectors::kafka::partition_stream::PartitionStreamSender;
20 | use crate::connectors::kafka::{TopicPartition, TopicPartitionList};
21 | use std::collections::HashMap;
22 | use tracing::info;
23 |
24 | pub struct StreamState {
25 | partition_senders: HashMap,
26 | }
27 |
28 | impl StreamState {
29 | pub fn new(capacity: usize) -> Self {
30 | Self {
31 | partition_senders: HashMap::with_capacity(capacity),
32 | }
33 | }
34 |
35 | pub fn insert_partition_sender(
36 | &mut self,
37 | tp: TopicPartition,
38 | sender: PartitionStreamSender,
39 | ) -> Option {
40 | self.partition_senders.insert(tp, sender)
41 | }
42 |
43 | pub fn get_partition_sender(&self, tp: &TopicPartition) -> Option<&PartitionStreamSender> {
44 | self.partition_senders.get(tp)
45 | }
46 |
47 | pub async fn terminate_partition_streams(&mut self, tpl: TopicPartitionList) {
48 | info!("Terminating streams: {:?}", tpl);
49 |
50 | for tp in tpl.tpl {
51 | if let Some(sender) = self.partition_senders.remove(&tp) {
52 | info!("Terminating stream for {:?}", tp);
53 | sender.terminate();
54 | drop(sender);
55 | info!("Stream terminated for {:?}", tp);
56 | } else {
57 | info!("Stream already completed for {:?}", tp);
58 | }
59 | }
60 |
61 | info!("All streams terminated!");
62 | }
63 |
64 | pub fn clear(&mut self) {
65 | info!("Clearing all stream states...");
66 | self.partition_senders.clear();
67 | }
68 | }
69 |
--------------------------------------------------------------------------------
/src/connectors/mod.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use std::sync::Arc;
20 |
21 | use actix_web_prometheus::PrometheusMetrics;
22 | use common::{processor::Processor, shutdown::Shutdown};
23 | use kafka::{
24 | config::KafkaConfig, consumer::KafkaStreams, metrics::KafkaMetricsCollector,
25 | processor::ParseableSinkProcessor, rebalance_listener::RebalanceListener,
26 | sink::KafkaSinkConnector, state::StreamState, ConsumerRecord, KafkaContext,
27 | };
28 | use prometheus::Registry;
29 | use tokio::sync::RwLock;
30 | use tracing::{info, warn};
31 |
32 | use crate::{option::Mode, parseable::PARSEABLE};
33 |
34 | pub mod common;
35 | pub mod kafka;
36 |
37 | pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> {
38 | if matches!(PARSEABLE.options.mode, Mode::Ingest | Mode::All) {
39 | match PARSEABLE.kafka_config.validate() {
40 | Err(e) => {
41 | warn!("Kafka connector configuration invalid. {}", e);
42 | }
43 | Ok(_) => {
44 | let config = PARSEABLE.kafka_config.clone();
45 | let shutdown_handle = Shutdown::default();
46 | let registry = prometheus.registry.clone();
47 | let processor = ParseableSinkProcessor;
48 |
49 | tokio::spawn({
50 | let shutdown_handle = shutdown_handle.clone();
51 | async move {
52 | shutdown_handle.signal_listener().await;
53 | info!("Connector received shutdown signal!");
54 | }
55 | });
56 |
57 | run_kafka2parseable(config, registry, processor, shutdown_handle).await?;
58 | }
59 | }
60 | }
61 |
62 | Ok(())
63 | }
64 |
65 | async fn run_kafka2parseable(
66 | config: KafkaConfig,
67 | registry: Registry,
68 | processor: P,
69 | shutdown_handle: Shutdown,
70 | ) -> anyhow::Result<()>
71 | where
72 | P: Processor, ()> + Send + Sync + 'static,
73 | {
74 | info!("Initializing KafkaSink connector...");
75 |
76 | let kafka_config = Arc::new(config.clone());
77 | let (kafka_context, rebalance_rx) = KafkaContext::new(kafka_config);
78 |
79 | //TODO: fetch topics metadata from kafka then give dynamic value to StreamState
80 | let stream_state = Arc::new(RwLock::new(StreamState::new(60)));
81 | let rebalance_listener = RebalanceListener::new(
82 | rebalance_rx,
83 | Arc::clone(&stream_state),
84 | shutdown_handle.clone(),
85 | );
86 |
87 | let kafka_streams = KafkaStreams::init(kafka_context, stream_state, shutdown_handle.clone())?;
88 |
89 | let stats = kafka_streams.statistics();
90 | registry.register(Box::new(KafkaMetricsCollector::new(stats)?))?;
91 |
92 | let kafka_parseable_sink_connector = KafkaSinkConnector::new(kafka_streams, processor);
93 |
94 | rebalance_listener.start();
95 | kafka_parseable_sink_connector.run().await?;
96 |
97 | Ok(())
98 | }
99 |
--------------------------------------------------------------------------------
/src/enterprise/mod.rs:
--------------------------------------------------------------------------------
1 | pub mod utils;
2 |
--------------------------------------------------------------------------------
/src/handlers/http/about.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use actix_web::web::Json;
20 | use serde_json::{json, Value};
21 |
22 | use crate::{
23 | about::{self, get_latest_release},
24 | parseable::PARSEABLE,
25 | storage::StorageMetadata,
26 | };
27 | use std::path::PathBuf;
28 |
29 | /// {
30 | /// "version": current_version,
31 | /// "uiVersion": ui_version,
32 | /// "commit": commit,
33 | /// "deploymentId": deployment_id,
34 | /// "updateAvailable": update_available,
35 | /// "latestVersion": latest_release,
36 | /// "llmActive": is_llm_active,
37 | /// "llmProvider": llm_provider,
38 | /// "oidcActive": is_oidc_active,
39 | /// "license": "AGPL-3.0-only",
40 | /// "mode": mode,
41 | /// "staging": staging,
42 | /// "grpcPort": grpc_port,
43 | /// "store": {
44 | /// "type": PARSEABLE.get_storage_mode_string(),
45 | /// "path": store_endpoint
46 | /// }
47 | /// }
48 | pub async fn about() -> Json {
49 | let meta = StorageMetadata::global();
50 |
51 | let current_release = about::current();
52 | let latest_release = get_latest_release();
53 | let (update_available, latest_release) = match latest_release {
54 | Some(latest_release) => (
55 | latest_release.version > current_release.released_version,
56 | Some(format!("v{}", latest_release.version)),
57 | ),
58 | None => (false, None),
59 | };
60 |
61 | let current_version = format!("v{}", current_release.released_version);
62 | let commit = current_release.commit_hash;
63 | let deployment_id = meta.deployment_id.to_string();
64 | let mode = PARSEABLE.get_server_mode_string();
65 | let staging = PARSEABLE.options.staging_dir().display().to_string();
66 | let grpc_port = PARSEABLE.options.grpc_port;
67 |
68 | let store_endpoint = PARSEABLE.storage.get_endpoint();
69 | let is_llm_active = &PARSEABLE.options.open_ai_key.is_some();
70 | let llm_provider = is_llm_active.then_some("OpenAI");
71 | let is_oidc_active = PARSEABLE.options.openid().is_some();
72 | let ui_version = option_env!("UI_VERSION").unwrap_or("development");
73 |
74 | let hot_tier_details: String = if PARSEABLE.hot_tier_dir().is_none() {
75 | "Disabled".to_string()
76 | } else {
77 | let hot_tier_dir: &Option = PARSEABLE.hot_tier_dir();
78 | format!(
79 | "Enabled, Path: {}",
80 | hot_tier_dir.as_ref().unwrap().display(),
81 | )
82 | };
83 |
84 | let ms_clarity_tag = &PARSEABLE.options.ms_clarity_tag;
85 |
86 | Json(json!({
87 | "version": current_version,
88 | "uiVersion": ui_version,
89 | "commit": commit,
90 | "deploymentId": deployment_id,
91 | "updateAvailable": update_available,
92 | "latestVersion": latest_release,
93 | "llmActive": is_llm_active,
94 | "llmProvider": llm_provider,
95 | "oidcActive": is_oidc_active,
96 | "license": "AGPL-3.0-only",
97 | "mode": mode,
98 | "staging": staging,
99 | "hotTier": hot_tier_details,
100 | "grpcPort": grpc_port,
101 | "store": {
102 | "type": PARSEABLE.get_storage_mode_string(),
103 | "path": store_endpoint
104 | },
105 | "analytics": {
106 | "clarityTag": ms_clarity_tag
107 | },
108 | }))
109 | }
110 |
--------------------------------------------------------------------------------
/src/handlers/http/correlation.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use actix_web::web::{Json, Path};
20 | use actix_web::{web, HttpRequest, HttpResponse, Responder};
21 | use anyhow::Error;
22 | use itertools::Itertools;
23 |
24 | use crate::rbac::Users;
25 | use crate::utils::actix::extract_session_key_from_req;
26 | use crate::utils::{get_hash, get_user_from_request, user_auth_for_datasets};
27 |
28 | use crate::correlation::{CorrelationConfig, CorrelationError, CORRELATIONS};
29 |
30 | pub async fn list(req: HttpRequest) -> Result {
31 | let session_key = extract_session_key_from_req(&req)
32 | .map_err(|err| CorrelationError::AnyhowError(Error::msg(err.to_string())))?;
33 |
34 | let correlations = CORRELATIONS.list_correlations(&session_key).await?;
35 |
36 | Ok(web::Json(correlations))
37 | }
38 |
39 | pub async fn get(
40 | req: HttpRequest,
41 | correlation_id: Path,
42 | ) -> Result {
43 | let correlation_id = correlation_id.into_inner();
44 | let session_key = extract_session_key_from_req(&req)
45 | .map_err(|err| CorrelationError::AnyhowError(Error::msg(err.to_string())))?;
46 |
47 | let correlation = CORRELATIONS.get_correlation(&correlation_id).await?;
48 |
49 | let permissions = Users.get_permissions(&session_key);
50 |
51 | let tables = &correlation
52 | .table_configs
53 | .iter()
54 | .map(|t| t.table_name.clone())
55 | .collect_vec();
56 |
57 | user_auth_for_datasets(&permissions, tables)?;
58 |
59 | Ok(web::Json(correlation))
60 | }
61 |
62 | pub async fn post(
63 | req: HttpRequest,
64 | Json(mut correlation): Json,
65 | ) -> Result {
66 | let session_key = extract_session_key_from_req(&req)
67 | .map_err(|err| CorrelationError::AnyhowError(anyhow::Error::msg(err.to_string())))?;
68 | let user_id = get_user_from_request(&req)
69 | .map(|s| get_hash(&s.to_string()))
70 | .map_err(|err| CorrelationError::AnyhowError(Error::msg(err.to_string())))?;
71 | correlation.user_id = user_id;
72 |
73 | let correlation = CORRELATIONS.create(correlation, &session_key).await?;
74 |
75 | Ok(web::Json(correlation))
76 | }
77 |
78 | pub async fn modify(
79 | req: HttpRequest,
80 | correlation_id: Path,
81 | Json(mut correlation): Json,
82 | ) -> Result {
83 | correlation.id = correlation_id.into_inner();
84 | correlation.user_id = get_user_from_request(&req)
85 | .map(|s| get_hash(&s.to_string()))
86 | .map_err(|err| CorrelationError::AnyhowError(Error::msg(err.to_string())))?;
87 |
88 | let session_key = extract_session_key_from_req(&req)
89 | .map_err(|err| CorrelationError::AnyhowError(anyhow::Error::msg(err.to_string())))?;
90 |
91 | let correlation = CORRELATIONS.update(correlation, &session_key).await?;
92 |
93 | Ok(web::Json(correlation))
94 | }
95 |
96 | pub async fn delete(
97 | req: HttpRequest,
98 | correlation_id: Path,
99 | ) -> Result {
100 | let correlation_id = correlation_id.into_inner();
101 | let user_id = get_user_from_request(&req)
102 | .map(|s| get_hash(&s.to_string()))
103 | .map_err(|err| CorrelationError::AnyhowError(Error::msg(err.to_string())))?;
104 |
105 | CORRELATIONS.delete(&correlation_id, &user_id).await?;
106 |
107 | Ok(HttpResponse::Ok().finish())
108 | }
109 |
--------------------------------------------------------------------------------
/src/handlers/http/health_check.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use std::sync::Arc;
20 |
21 | use actix_web::{
22 | body::MessageBody,
23 | dev::{ServiceRequest, ServiceResponse},
24 | error::Error,
25 | error::ErrorServiceUnavailable,
26 | middleware::Next,
27 | HttpResponse,
28 | };
29 | use http::StatusCode;
30 | use once_cell::sync::Lazy;
31 | use tokio::{sync::Mutex, task::JoinSet};
32 | use tracing::{error, info, warn};
33 |
34 | use crate::parseable::PARSEABLE;
35 |
36 | // Create a global variable to store signal status
37 | static SIGNAL_RECEIVED: Lazy>> = Lazy::new(|| Arc::new(Mutex::new(false)));
38 |
39 | pub async fn liveness() -> HttpResponse {
40 | HttpResponse::new(StatusCode::OK)
41 | }
42 |
43 | pub async fn check_shutdown_middleware(
44 | req: ServiceRequest,
45 | next: Next,
46 | ) -> Result, Error> {
47 | // Acquire the shutdown flag to check if the server is shutting down.
48 | if *SIGNAL_RECEIVED.lock().await {
49 | // Return 503 Service Unavailable if the server is shutting down.
50 | Err(ErrorServiceUnavailable("Server is shutting down"))
51 | } else {
52 | // Continue processing the request if the server is not shutting down.
53 | next.call(req).await
54 | }
55 | }
56 |
57 | // This function is called when the server is shutting down
58 | pub async fn shutdown() {
59 | // Set the shutdown flag to true
60 | let mut shutdown_flag = SIGNAL_RECEIVED.lock().await;
61 | *shutdown_flag = true;
62 |
63 | let mut joinset = JoinSet::new();
64 |
65 | // Sync staging
66 | PARSEABLE.streams.flush_and_convert(&mut joinset, true);
67 |
68 | while let Some(res) = joinset.join_next().await {
69 | match res {
70 | Ok(Ok(_)) => info!("Successfully converted arrow files to parquet."),
71 | Ok(Err(err)) => warn!("Failed to convert arrow files to parquet. {err:?}"),
72 | Err(err) => error!("Failed to join async task: {err}"),
73 | }
74 | }
75 |
76 | if let Err(e) = PARSEABLE
77 | .storage
78 | .get_object_store()
79 | .upload_files_from_staging()
80 | .await
81 | {
82 | warn!("Failed to sync local data with object store. {:?}", e);
83 | } else {
84 | info!("Successfully synced all data to S3.");
85 | }
86 | }
87 |
88 | pub async fn readiness() -> HttpResponse {
89 | // Check the object store connection
90 | if PARSEABLE.storage.get_object_store().check().await.is_ok() {
91 | HttpResponse::new(StatusCode::OK)
92 | } else {
93 | HttpResponse::new(StatusCode::SERVICE_UNAVAILABLE)
94 | }
95 | }
96 |
--------------------------------------------------------------------------------
/src/handlers/http/kinesis.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use base64::{engine::general_purpose::STANDARD, Engine as _};
20 | use serde::{Deserialize, Serialize};
21 | use serde_json::{Map, Value};
22 | use std::str;
23 |
24 | use crate::utils::json::flatten::{generic_flattening, has_more_than_max_allowed_levels};
25 |
26 | #[derive(Serialize, Deserialize, Debug)]
27 | #[serde(rename_all = "camelCase")]
28 | pub struct Message {
29 | records: Vec,
30 | request_id: String,
31 | timestamp: u64,
32 | }
33 | #[derive(Serialize, Deserialize, Debug)]
34 | struct Data {
35 | data: String,
36 | }
37 |
38 | // Flatten Kinesis logs is used to flatten the Kinesis logs into a queryable JSON format.
39 | // Kinesis logs are in the format
40 | // {
41 | // "requestId": "9b848d8a-2d89-474b-b073-04b8e5232210",
42 | // "timestamp": 1705026780451,
43 | // "records": [
44 | // {
45 | // "data": "eyJDSEFOR0UiOi0wLjQ1LCJQUklDRSI6NS4zNiwiVElDS0VSX1NZTUJPTCI6IkRFRyIsIlNFQ1RPUiI6IkVORVJHWSJ9"
46 | // }
47 | // ]
48 | // }
49 | // The data field is base64 encoded JSON (there can be multiple data fields), and there is a requestId and timestamp field.
50 | // Kinesis logs are flattened to the following format:
51 | // {
52 | // "CHANGE": 3.16,
53 | // "PRICE": 73.76,
54 | // "SECTOR": "RETAIL",
55 | // "TICKER_SYMBOL": "WMT",
56 | // "p_metadata": "",
57 | // "p_tags": "",
58 | // "p_timestamp": "2024-01-11T09:08:34.290",
59 | // "requestId": "b858288a-f5d8-4181-a746-3f3dd716be8a",
60 | // "timestamp": "1704964113659"
61 | // }
62 | pub async fn flatten_kinesis_logs(message: Message) -> Result, anyhow::Error> {
63 | let mut vec_kinesis_json = Vec::new();
64 |
65 | for record in message.records.iter() {
66 | let bytes = STANDARD.decode(record.data.clone())?;
67 | if let Ok(json_string) = String::from_utf8(bytes) {
68 | let json: serde_json::Value = serde_json::from_str(&json_string)?;
69 | // Check if the JSON has more than the allowed levels of nesting
70 | // If it has less than or equal to the allowed levels, we flatten it.
71 | // If it has more than the allowed levels, we just push it as is
72 | // without flattening or modifying it.
73 | if !has_more_than_max_allowed_levels(&json, 1) {
74 | let flattened_json_arr = generic_flattening(&json)?;
75 | for flattened_json in flattened_json_arr {
76 | let mut kinesis_json: Map =
77 | serde_json::from_value(flattened_json)?;
78 | kinesis_json.insert(
79 | "requestId".to_owned(),
80 | Value::String(message.request_id.clone()),
81 | );
82 | kinesis_json.insert(
83 | "timestamp".to_owned(),
84 | Value::String(message.timestamp.to_string()),
85 | );
86 |
87 | vec_kinesis_json.push(Value::Object(kinesis_json));
88 | }
89 | } else {
90 | // If the JSON has more than the allowed levels, we just push it as is
91 | // without flattening or modifying it.
92 | // This is a fallback to ensure we don't lose data.
93 | tracing::warn!(
94 | "Kinesis log with requestId {} and timestamp {} has more than the allowed levels of nesting, skipping flattening for this record.",
95 | message.request_id, message.timestamp
96 | );
97 | vec_kinesis_json.push(json);
98 | }
99 | } else {
100 | tracing::error!(
101 | "Failed to decode base64 data for kinesis log with requestId {} and timestamp {}",
102 | message.request_id,
103 | message.timestamp
104 | );
105 | return Err(anyhow::anyhow!(
106 | "Failed to decode base64 data for record with requestId {} and timestamp {}",
107 | message.request_id,
108 | message.timestamp
109 | ));
110 | }
111 | }
112 |
113 | Ok(vec_kinesis_json)
114 | }
115 |
--------------------------------------------------------------------------------
/src/handlers/http/modal/ingest/ingestor_ingest.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use actix_web::{HttpRequest, HttpResponse};
20 | use bytes::Bytes;
21 |
22 | use crate::{handlers::http::{ingest::PostError, modal::utils::ingest_utils::flatten_and_push_logs}, metadata::PARSEABLE.streams};
23 |
24 |
25 | // Handler for POST /api/v1/logstream/{logstream}
26 | // only ingests events into the specified logstream
27 | // fails if the logstream does not exist
28 | pub async fn post_event(req: HttpRequest, body: Bytes) -> Result {
29 | let stream_name: String = req.match_info().get("logstream").unwrap().parse().unwrap();
30 | let internal_stream_names = PARSEABLE.streams.list_internal_streams();
31 | if internal_stream_names.contains(&stream_name) {
32 | return Err(PostError::Invalid(anyhow::anyhow!(
33 | "Stream {} is an internal stream and cannot be ingested into",
34 | stream_name
35 | )));
36 | }
37 | if !PARSEABLE.streams.stream_exists(&stream_name) {
38 | return Err(PostError::StreamNotFound(stream_name));
39 | }
40 |
41 | flatten_and_push_logs(req, body, stream_name).await?;
42 | Ok(HttpResponse::Ok().finish())
43 | }
--------------------------------------------------------------------------------
/src/handlers/http/modal/ingest/ingestor_logstream.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use std::fs;
20 |
21 | use actix_web::{
22 | web::{Json, Path},
23 | HttpRequest, Responder,
24 | };
25 | use bytes::Bytes;
26 | use http::StatusCode;
27 | use tracing::warn;
28 |
29 | use crate::{
30 | catalog::remove_manifest_from_snapshot,
31 | handlers::http::logstream::error::StreamError,
32 | parseable::{StreamNotFound, PARSEABLE},
33 | stats,
34 | };
35 |
36 | pub async fn retention_cleanup(
37 | stream_name: Path,
38 | Json(date_list): Json>,
39 | ) -> Result {
40 | let stream_name = stream_name.into_inner();
41 | let storage = PARSEABLE.storage.get_object_store();
42 | // if the stream not found in memory map,
43 | //check if it exists in the storage
44 | //create stream and schema from storage
45 | if !PARSEABLE.streams.contains(&stream_name)
46 | && !PARSEABLE
47 | .create_stream_and_schema_from_storage(&stream_name)
48 | .await
49 | .unwrap_or(false)
50 | {
51 | return Err(StreamNotFound(stream_name.clone()).into());
52 | }
53 |
54 | let res = remove_manifest_from_snapshot(storage.clone(), &stream_name, date_list).await;
55 | let first_event_at: Option = res.unwrap_or_default();
56 |
57 | Ok((first_event_at, StatusCode::OK))
58 | }
59 |
60 | pub async fn delete(stream_name: Path) -> Result {
61 | let stream_name = stream_name.into_inner();
62 |
63 | // Delete from staging
64 | let stream_dir = PARSEABLE.get_stream(&stream_name)?;
65 | if let Err(err) = fs::remove_dir_all(&stream_dir.data_path) {
66 | warn!(
67 | "failed to delete local data for stream {} with error {err}. Clean {} manually",
68 | stream_name,
69 | stream_dir.data_path.to_string_lossy()
70 | )
71 | }
72 |
73 | // Delete from memory
74 | PARSEABLE.streams.delete(&stream_name);
75 | stats::delete_stats(&stream_name, "json")
76 | .unwrap_or_else(|e| warn!("failed to delete stats for stream {}: {:?}", stream_name, e));
77 |
78 | Ok((format!("log stream {stream_name} deleted"), StatusCode::OK))
79 | }
80 |
81 | pub async fn put_stream(
82 | req: HttpRequest,
83 | stream_name: Path,
84 | body: Bytes,
85 | ) -> Result {
86 | let stream_name = stream_name.into_inner();
87 | PARSEABLE
88 | .create_update_stream(req.headers(), &body, &stream_name)
89 | .await?;
90 |
91 | Ok(("Log stream created", StatusCode::OK))
92 | }
93 |
--------------------------------------------------------------------------------
/src/handlers/http/modal/ingest/ingestor_rbac.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use std::collections::HashSet;
20 |
21 | use actix_web::{web, Responder};
22 | use tokio::sync::Mutex;
23 |
24 | use crate::{
25 | handlers::http::{modal::utils::rbac_utils::get_metadata, rbac::RBACError},
26 | rbac::{
27 | user::{self, User as ParseableUser},
28 | Users,
29 | },
30 | storage,
31 | };
32 |
33 | // async aware lock for updating storage metadata and user map atomicically
34 | static UPDATE_LOCK: Mutex<()> = Mutex::const_new(());
35 |
36 | // Handler for POST /api/v1/user/{username}
37 | // Creates a new user by username if it does not exists
38 | pub async fn post_user(
39 | username: web::Path,
40 | body: Option>,
41 | ) -> Result {
42 | let username = username.into_inner();
43 |
44 | let generated_password = String::default();
45 | let metadata = get_metadata().await?;
46 | if let Some(body) = body {
47 | let user: ParseableUser = serde_json::from_value(body.into_inner())?;
48 | let _ = storage::put_staging_metadata(&metadata);
49 | let created_role = user.roles.clone();
50 | Users.put_user(user.clone());
51 | Users.put_role(&username, created_role.clone());
52 | }
53 |
54 | Ok(generated_password)
55 | }
56 |
57 | // Handler for DELETE /api/v1/user/delete/{username}
58 | pub async fn delete_user(username: web::Path) -> Result {
59 | let username = username.into_inner();
60 | let _ = UPDATE_LOCK.lock().await;
61 | // fail this request if the user does not exists
62 | if !Users.contains(&username) {
63 | return Err(RBACError::UserDoesNotExist);
64 | };
65 | // delete from parseable.json first
66 | let mut metadata = get_metadata().await?;
67 | metadata.users.retain(|user| user.username() != username);
68 |
69 | let _ = storage::put_staging_metadata(&metadata);
70 |
71 | // update in mem table
72 | Users.delete_user(&username);
73 | Ok(format!("deleted user: {username}"))
74 | }
75 |
76 | // Handler PUT /user/{username}/roles => Put roles for user
77 | // Put roles for given user
78 | pub async fn put_role(
79 | username: web::Path,
80 | role: web::Json>,
81 | ) -> Result {
82 | let username = username.into_inner();
83 | let role = role.into_inner();
84 |
85 | if !Users.contains(&username) {
86 | return Err(RBACError::UserDoesNotExist);
87 | };
88 | // update parseable.json first
89 | let mut metadata = get_metadata().await?;
90 | if let Some(user) = metadata
91 | .users
92 | .iter_mut()
93 | .find(|user| user.username() == username)
94 | {
95 | user.roles.clone_from(&role);
96 | } else {
97 | // should be unreachable given state is always consistent
98 | return Err(RBACError::UserDoesNotExist);
99 | }
100 |
101 | let _ = storage::put_staging_metadata(&metadata);
102 | // update in mem table
103 | Users.put_role(&username.clone(), role.clone());
104 |
105 | Ok(format!("Roles updated successfully for {username}"))
106 | }
107 |
108 | // Handler for POST /api/v1/user/{username}/generate-new-password
109 | // Resets password for the user to a newly generated one and returns it
110 | pub async fn post_gen_password(username: web::Path) -> Result {
111 | let username = username.into_inner();
112 | let mut new_hash = String::default();
113 | let mut metadata = get_metadata().await?;
114 |
115 | let _ = storage::put_staging_metadata(&metadata);
116 | if let Some(user) = metadata
117 | .users
118 | .iter_mut()
119 | .filter_map(|user| match user.ty {
120 | user::UserType::Native(ref mut user) => Some(user),
121 | _ => None,
122 | })
123 | .find(|user| user.username == username)
124 | {
125 | new_hash.clone_from(&user.password_hash);
126 | } else {
127 | return Err(RBACError::UserDoesNotExist);
128 | }
129 | Users.change_password_hash(&username, &new_hash);
130 |
131 | Ok("Updated")
132 | }
133 |
--------------------------------------------------------------------------------
/src/handlers/http/modal/ingest/ingestor_role.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use actix_web::{
20 | web::{self, Json},
21 | HttpResponse, Responder,
22 | };
23 |
24 | use crate::{
25 | handlers::http::{modal::utils::rbac_utils::get_metadata, role::RoleError},
26 | rbac::{map::mut_roles, role::model::DefaultPrivilege},
27 | storage,
28 | };
29 |
30 | // Handler for PUT /api/v1/role/{name}
31 | // Creates a new role or update existing one
32 | pub async fn put(
33 | name: web::Path,
34 | Json(privileges): Json>,
35 | ) -> Result {
36 | let name = name.into_inner();
37 | let mut metadata = get_metadata().await?;
38 | metadata.roles.insert(name.clone(), privileges.clone());
39 |
40 | let _ = storage::put_staging_metadata(&metadata);
41 | mut_roles().insert(name.clone(), privileges);
42 |
43 | Ok(HttpResponse::Ok().finish())
44 | }
45 |
--------------------------------------------------------------------------------
/src/handlers/http/modal/ingest/mod.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | pub mod ingestor_logstream;
20 | pub mod ingestor_rbac;
21 | pub mod ingestor_role;
22 |
--------------------------------------------------------------------------------
/src/handlers/http/modal/query/mod.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | pub mod querier_ingest;
20 | pub mod querier_logstream;
21 | pub mod querier_rbac;
22 | pub mod querier_role;
23 |
--------------------------------------------------------------------------------
/src/handlers/http/modal/query/querier_ingest.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use actix_web::HttpResponse;
20 |
21 | use crate::handlers::http::ingest::PostError;
22 |
23 | // Handler for POST /api/v1/logstream/{logstream}
24 | // only ingests events into the specified logstream
25 | // fails if the logstream does not exist
26 | #[allow(unused)]
27 | pub async fn post_event() -> Result {
28 | Err(PostError::IngestionNotAllowed)
29 | }
30 |
--------------------------------------------------------------------------------
/src/handlers/http/modal/query/querier_role.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use actix_web::{
20 | web::{self, Json},
21 | HttpResponse, Responder,
22 | };
23 |
24 | use crate::{
25 | handlers::http::{
26 | cluster::sync_role_update_with_ingestors,
27 | modal::utils::rbac_utils::{get_metadata, put_metadata},
28 | role::RoleError,
29 | },
30 | rbac::{map::mut_roles, role::model::DefaultPrivilege},
31 | };
32 |
33 | // Handler for PUT /api/v1/role/{name}
34 | // Creates a new role or update existing one
35 | pub async fn put(
36 | name: web::Path,
37 | Json(privileges): Json>,
38 | ) -> Result {
39 | let name = name.into_inner();
40 | let mut metadata = get_metadata().await?;
41 | metadata.roles.insert(name.clone(), privileges.clone());
42 |
43 | put_metadata(&metadata).await?;
44 | mut_roles().insert(name.clone(), privileges.clone());
45 |
46 | sync_role_update_with_ingestors(name.clone(), privileges.clone()).await?;
47 |
48 | Ok(HttpResponse::Ok().finish())
49 | }
50 |
--------------------------------------------------------------------------------
/src/handlers/http/modal/ssl_acceptor.rs:
--------------------------------------------------------------------------------
1 | /*
2 | * Parseable Server (C) 2022 - 2024 Parseable, Inc.
3 | *
4 | * This program is free software: you can redistribute it and/or modify
5 | * it under the terms of the GNU Affero General Public License as
6 | * published by the Free Software Foundation, either version 3 of the
7 | * License, or (at your option) any later version.
8 | *
9 | * This program is distributed in the hope that it will be useful,
10 | * but WITHOUT ANY WARRANTY; without even the implied warranty of
11 | * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12 | * GNU Affero General Public License for more details.
13 | *
14 | * You should have received a copy of the GNU Affero General Public License
15 | * along with this program. If not, see .
16 | *
17 | */
18 |
19 | use std::{
20 | fs::{self, File},
21 | io::BufReader,
22 | path::PathBuf,
23 | };
24 |
25 | use rustls::ServerConfig;
26 |
27 | pub fn get_ssl_acceptor(
28 | tls_cert: &Option,
29 | tls_key: &Option,
30 | other_certs: &Option,
31 | ) -> anyhow::Result