-
Notifications
You must be signed in to change notification settings - Fork 7
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Implement basic PoC demonstrating append-only do put endpoint
- Loading branch information
Showing
10 changed files
with
292 additions
and
7 deletions.
There are no files selected for viewing
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
syntax = "proto3"; | ||
|
||
package clade.flight; | ||
|
||
message DoPutCommand { | ||
enum CommandType { | ||
// Protobuf pattern, not used | ||
UNKNOWN = 0; | ||
|
||
// Data: updated or inserted rows. If a column is omitted, | ||
// this assumes it's unchanged. | ||
UPSERT = 1; | ||
|
||
// Data: columns forming the primary keys / replica identity | ||
// of the deleted rows. | ||
DELETE = 2; | ||
} | ||
|
||
CommandType type = 1; | ||
|
||
// TODO: Table ref or URL of the Delta Table? | ||
string table = 2; | ||
|
||
// Primary key columns | ||
repeated string pk_column = 3; | ||
|
||
// Opaque string identifying PGD origin | ||
optional string origin = 4; | ||
|
||
// Monotonically-increasing transaction number (e.g. an LSN) | ||
optional uint64 sequence_number = 5; | ||
} | ||
|
||
message DoPutResult { | ||
enum AckType { | ||
UNKNOWN = 0; | ||
|
||
// Seafowl is overloaded and can't accept the change (e.g. | ||
// can't flush fast enough). The client should wait and retry. | ||
// TODO: we can block when receiving FlightData instead? | ||
BACKPRESSURE = 1; | ||
|
||
// Change acknowledged and is in-memory. Will get lost if | ||
// the Seafowl instance is restarted | ||
MEMORY = 2; | ||
|
||
// Change, and all changes from this origin, has been flushed | ||
// to durable storage (i.e. Delta Lake). | ||
DURABLE = 3; | ||
} | ||
|
||
AckType type = 1; | ||
} | ||
|
||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,105 @@ | ||
use crate::flight::*; | ||
use arrow::array::{Int32Array, StringArray}; | ||
use arrow_flight::encode::FlightDataEncoderBuilder; | ||
use arrow_schema::{DataType, Field, Schema}; | ||
use clade::flight::do_put_result::AckType; | ||
use clade::flight::{do_put_command::CommandType, DoPutCommand, DoPutResult}; | ||
use futures::StreamExt; | ||
|
||
#[tokio::test] | ||
async fn test_basic_upload() -> Result<()> { | ||
let (_context, mut client) = flight_server().await; | ||
|
||
let schema = Arc::new(Schema::new(vec![ | ||
Field::new("col_1", DataType::Int32, true), | ||
Field::new("col_2", DataType::Utf8, true), | ||
])); | ||
|
||
let batch = RecordBatch::try_new( | ||
schema.clone(), | ||
vec![ | ||
Arc::new(Int32Array::from(vec![1, 2])), | ||
Arc::new(StringArray::from(vec![Some("one"), Some("two")])), | ||
], | ||
) | ||
.unwrap(); | ||
|
||
let cmd = DoPutCommand { | ||
r#type: CommandType::Upsert.into(), | ||
table: "do_put_test".to_string(), | ||
origin: Some("test-origin".to_string()), | ||
pk_column: vec![], | ||
sequence_number: Some(123), | ||
}; | ||
|
||
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec()); | ||
let flight_stream_builder = | ||
FlightDataEncoderBuilder::new().with_flight_descriptor(Some(descriptor)); | ||
let flight_data = flight_stream_builder.build(futures::stream::iter(vec![Ok(batch)])); | ||
let response = client.do_put(flight_data).await?.next().await.unwrap()?; | ||
|
||
// Changes are still in memory | ||
let ack_type = AckType::try_from( | ||
DoPutResult::decode(response.app_metadata) | ||
.expect("DoPutResult") | ||
.r#type, | ||
) | ||
.expect("AckType"); | ||
assert_eq!(ack_type, AckType::Memory); | ||
|
||
let err = get_flight_batches(&mut client, "SELECT * FROM do_put_test".to_string()) | ||
.await | ||
.unwrap_err() | ||
.to_string(); | ||
assert!(err.contains("Tonic error: status: Internal, message: \"Error during planning: table 'default.public.do_put_test' not found\"")); | ||
|
||
// Try to append some more data and pass the flush threshold | ||
let batch = RecordBatch::try_new( | ||
schema.clone(), | ||
vec![ | ||
Arc::new(Int32Array::from(vec![3, 4])), | ||
Arc::new(StringArray::from(vec![Some("three"), Some("four")])), | ||
], | ||
) | ||
.unwrap(); | ||
|
||
let cmd = DoPutCommand { | ||
r#type: CommandType::Upsert.into(), | ||
table: "do_put_test".to_string(), | ||
origin: Some("test-origin".to_string()), | ||
pk_column: vec![], | ||
sequence_number: Some(456), | ||
}; | ||
|
||
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec()); | ||
let flight_stream_builder = | ||
FlightDataEncoderBuilder::new().with_flight_descriptor(Some(descriptor)); | ||
let flight_data = flight_stream_builder.build(futures::stream::iter(vec![Ok(batch)])); | ||
let response = client.do_put(flight_data).await?.next().await.unwrap()?; | ||
|
||
// Changes are now flushed to storage | ||
let ack_type = AckType::try_from( | ||
DoPutResult::decode(response.app_metadata) | ||
.expect("DoPutResult") | ||
.r#type, | ||
) | ||
.expect("AckType"); | ||
assert_eq!(ack_type, AckType::Durable); | ||
|
||
let results = | ||
get_flight_batches(&mut client, "SELECT * FROM do_put_test".to_string()).await?; | ||
let expected = [ | ||
"+-------+-------+", | ||
"| col_1 | col_2 |", | ||
"+-------+-------+", | ||
"| 1 | one |", | ||
"| 2 | two |", | ||
"| 3 | three |", | ||
"| 4 | four |", | ||
"+-------+-------+", | ||
]; | ||
|
||
assert_batches_eq!(expected, &results); | ||
|
||
Ok(()) | ||
} |
Oops, something went wrong.