Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[data] Dataset.write_sql is not compatible with DB API2 executemany method #46160

Open
voe09 opened this issue Jun 20, 2024 · 0 comments
Open
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P1 Issue that should be fixed within a few weeks

Comments

@voe09
Copy link

voe09 commented Jun 20, 2024

What happened + What you expected to happen

Python DB API2 executemany method could be sync or async. Modules are free to implement it using different ways. The example code leverages sqlite3 and does not have the issue since sqlite3 implement executemany in a sync way. However, in another widely used library PyHive, all connections implement executemany in a async way (lazy execution). When using PyHive connections in write_sql, the query would be queued but not be executed.

The expected behavior of write_sql should be similar to read_sql's implementation. read_sql leverages execute() and fetchall() API to ensure the execution of the query (code).

Versions / Dependencies

ray == 2.24.0

Reproduction script

import ray
from pyhive import presto

def create_connection():
   return presto.connect('localhost')

def create_hive_table_using_presto_connection(conn):
   cursor = conn.cursor()
   sql = """
   CREATE TABLE IF NOT EXISTS movie (
       title varchar,
       year bigint,
       score double
   )
   WITH (FORMAT = 'parquet')
   """
   cursor.execute(sql)
   print(cursor.fetchall())

def write_data_into_hive_table_using_ray():
   dataset = ray.data.from_items([
       {"title": "Monty Python and the Holy Grail", "year": 1975, "score": 8.2},
       {"title": "And Now for Something Completely Different", "year": 1971, "score": 7.5}
   ])
   dataset.write_sql(
       "INSERT INTO movie VALUES(%s, %d, %f)", create_connection
   )

def read_data_from_hive_table_using_ray():
   dataset = ray.data.read_sql(
       "SELECT * FROM movie", create_connection
   )
   dataset.show()

if __name__ == '__main__':
   ray.init()

   create_hive_table_using_presto_connection(create_connection())
   write_data_into_hive_table_using_ray()
   read_data_from_hive_table_using_ray()

Issue Severity

High: It blocks me from completing my task.

@voe09 voe09 added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Jun 20, 2024
@anyscalesam anyscalesam added the data Ray Data-related issues label Jun 21, 2024
@scottjlee scottjlee added P1 Issue that should be fixed within a few weeks and removed triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Jun 26, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P1 Issue that should be fixed within a few weeks
Projects
None yet
Development

No branches or pull requests

3 participants