@@ -106,6 +106,21 @@ void SQLiteFlightSqlServer::ExecuteSql(const std::string& sql) {
106106 }
107107}
108108
109+ Status DoGetSQLiteQuery (sqlite3* db, const std::string& query,
110+ const std::shared_ptr<Schema>& schema,
111+ std::unique_ptr<FlightDataStream>* result) {
112+ std::shared_ptr<SqliteStatement> statement;
113+ ARROW_RETURN_NOT_OK (SqliteStatement::Create (db, query, &statement));
114+
115+ std::shared_ptr<SqliteStatementBatchReader> reader;
116+ ARROW_RETURN_NOT_OK (SqliteStatementBatchReader::Create (
117+ statement, schema, &reader));
118+
119+ *result = std::unique_ptr<FlightDataStream>(new RecordBatchStream (reader));
120+
121+ return Status::OK ();
122+ }
123+
109124Status GetFlightInfoForCommand (const FlightDescriptor& descriptor,
110125 std::unique_ptr<FlightInfo>* info,
111126 const google::protobuf::Message& command,
@@ -299,16 +314,7 @@ Status SQLiteFlightSqlServer::DoGetTableTypes(const ServerCallContext& context,
299314 std::unique_ptr<FlightDataStream>* result) {
300315 std::string query = " SELECT DISTINCT type as table_type FROM sqlite_master" ;
301316
302- std::shared_ptr<SqliteStatement> statement;
303- ARROW_RETURN_NOT_OK (SqliteStatement::Create (db_, query, &statement));
304-
305- std::shared_ptr<SqliteStatementBatchReader> reader;
306- ARROW_RETURN_NOT_OK (SqliteStatementBatchReader::Create (
307- statement, SqlSchema::GetTableTypesSchema (), &reader));
308-
309- *result = std::unique_ptr<FlightDataStream>(new RecordBatchStream (reader));
310-
311- return Status::OK ();
317+ return DoGetSQLiteQuery (db_, query, SqlSchema::GetTableTypesSchema (), result);
312318}
313319
314320Status SQLiteFlightSqlServer::GetFlightInfoPrimaryKeys (
@@ -343,17 +349,86 @@ SQLiteFlightSqlServer::DoGetPrimaryKeys(const pb::sql::CommandGetPrimaryKeys &co
343349
344350 table_query << " and table_name LIKE '" << command.table () << " '" ;
345351
346- std::shared_ptr<SqliteStatement> statement;
347- ARROW_RETURN_NOT_OK (SqliteStatement::Create (db_, table_query.str (), &statement));
352+ return DoGetSQLiteQuery (db_, table_query.str (), SqlSchema::GetPrimaryKeysSchema (),
353+ result);
354+ }
348355
349- std::shared_ptr<SqliteStatementBatchReader> reader;
350- ARROW_RETURN_NOT_OK (SqliteStatementBatchReader::Create (
351- statement, SqlSchema::GetPrimaryKeysSchema (), &reader));
356+ std::string PrepareQueryForGetImportedOrExportedKeys (const std::string& filter) {
357+ return R"( SELECT * FROM (SELECT NULL AS pk_catalog_name,
358+ NULL AS pk_schema_name,
359+ p."table" AS pk_table_name,
360+ p."to" AS pk_column_name,
361+ NULL AS fk_catalog_name,
362+ NULL AS fk_schema_name,
363+ m.name AS fk_table_name,
364+ p."from" AS fk_column_name,
365+ p.seq AS key_sequence,
366+ NULL AS pk_key_name,
367+ NULL AS fk_key_name,
368+ CASE
369+ WHEN p.on_update = 'CASCADE' THEN 0
370+ WHEN p.on_update = 'RESTRICT' THEN 1
371+ WHEN p.on_update = 'SET NULL' THEN 2
372+ WHEN p.on_update = 'NO ACTION' THEN 3
373+ WHEN p.on_update = 'SET DEFAULT' THEN 4
374+ END AS update_rule,
375+ CASE
376+ WHEN p.on_delete = 'CASCADE' THEN 0
377+ WHEN p.on_delete = 'RESTRICT' THEN 1
378+ WHEN p.on_delete = 'SET NULL' THEN 2
379+ WHEN p.on_delete = 'NO ACTION' THEN 3
380+ WHEN p.on_delete = 'SET DEFAULT' THEN 4
381+ END AS delete_rule
382+ FROM sqlite_master m
383+ JOIN pragma_foreign_key_list(m.name) p ON m.name != p."table"
384+ WHERE m.type = 'table') WHERE )" + filter + R"( ORDER BY
385+ pk_catalog_name, pk_schema_name, pk_table_name, pk_key_name, key_sequence)" ;
386+ }
352387
353- *result = std::unique_ptr<FlightDataStream>(
354- new RecordBatchStream (reader));
388+ Status SQLiteFlightSqlServer::GetFlightInfoImportedKeys (
389+ const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context,
390+ const FlightDescriptor& descriptor, std::unique_ptr<FlightInfo>* info) {
391+ return GetFlightInfoForCommand (descriptor, info, command,
392+ SqlSchema::GetImportedAndExportedKeysSchema ());
393+ }
355394
356- return Status::OK ();
395+ Status SQLiteFlightSqlServer::DoGetImportedKeys (
396+ const pb::sql::CommandGetImportedKeys& command, const ServerCallContext& context,
397+ std::unique_ptr<FlightDataStream>* result) {
398+ std::string filter = " fk_table_name = '" + command.table () + " '" ;
399+ if (command.has_catalog ()) {
400+ filter += " AND fk_catalog_name = '" + command.catalog () + " '" ;
401+ }
402+ if (command.has_schema ()) {
403+ filter += " AND fk_schema_name = '" + command.schema () + " '" ;
404+ }
405+ std::string query = PrepareQueryForGetImportedOrExportedKeys (filter);
406+
407+ return DoGetSQLiteQuery (db_, query, SqlSchema::GetImportedAndExportedKeysSchema (),
408+ result);
409+ }
410+
411+ Status SQLiteFlightSqlServer::GetFlightInfoExportedKeys (
412+ const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context,
413+ const FlightDescriptor& descriptor, std::unique_ptr<FlightInfo>* info) {
414+ return GetFlightInfoForCommand (descriptor, info, command,
415+ SqlSchema::GetImportedAndExportedKeysSchema ());
416+ }
417+
418+ Status SQLiteFlightSqlServer::DoGetExportedKeys (
419+ const pb::sql::CommandGetExportedKeys& command, const ServerCallContext& context,
420+ std::unique_ptr<FlightDataStream>* result) {
421+ std::string filter = " pk_table_name = '" + command.table () + " '" ;
422+ if (command.has_catalog ()) {
423+ filter += " AND pk_catalog_name = '" + command.catalog () + " '" ;
424+ }
425+ if (command.has_schema ()) {
426+ filter += " AND pk_schema_name = '" + command.schema () + " '" ;
427+ }
428+ std::string query = PrepareQueryForGetImportedOrExportedKeys (filter);
429+
430+ return DoGetSQLiteQuery (db_, query, SqlSchema::GetImportedAndExportedKeysSchema (),
431+ result);
357432}
358433
359434} // namespace example
0 commit comments