You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/04/14 00:29:56 UTC

[GitHub] [arrow-datafusion] wangxiaoying opened a new issue, #2230: Panic while running inner join with predicate only on single relation

wangxiaoying opened a new issue, #2230:
URL: https://github.com/apache/arrow-datafusion/issues/2230

   **Describe the bug**
   The following error would occur when join condition is only filtered on single realtion:
   `thread 'tokio-runtime-worker' panicked at 'index out of bounds: the len is 0 but the index is 0', /Users/momo/.cargo/registry/src/github.com-1ecc6299db9ec823/datafusion-7.0.0/src/physical_plan/repartition.rs:354:39`
   
   **To Reproduce**
   ```rust
   fn main() {
       let id_array = Int32Array::from(vec![1, 2, 3, 4, 5]);
       let str_array = StringArray::from(vec!["1", "2", "3", "4", "5"]);
       let schema = Schema::new(vec![
           Field::new("id", DataType::Int32, false),
           Field::new("str", DataType::Utf8, false),
       ]);
       let batch = RecordBatch::try_new(
           Arc::new(schema),
           vec![
               Arc::new(id_array),
               Arc::new(str_array),
           ],
       )
       .unwrap();
   
       let mut ctx = ExecutionContext::new();
       let sql = "select * from tmp t1 inner join tmp t2 on t1.str = '1' and t2.str = '2'";
       let db1 = MemTable::try_new(batch.schema(), vec![vec![batch]]).unwrap();
       ctx.register_table("tmp", Arc::new(db1)).unwrap();
   
       let rt = Arc::new(tokio::runtime::Runtime::new().expect("Failed to create runtime"));
       let df = rt.block_on(ctx.sql(sql)).unwrap();
       rt.block_on(df.limit(5).unwrap().show()).unwrap();
       let num_rows = rt
           .block_on(df.collect())
           .unwrap()
           .into_iter()
           .map(|rb| rb.num_rows())
           .sum::<usize>();
       println!("Final # rows: {}", num_rows);
   }
   ```
   
   **Expected behavior**
   The following queries should have same result:
   q1: `select * from tmp t1 inner join tmp t2 on t1.str = '1' and t2.str = '2'`
   q2: `select * from tmp t1 cross join tmp t2 where t1.str = '1' and t2.str = '2'`
   q3: `select * from tmp t1, tmp t2 where t1.str = '1' and t2.str = '2'`
   
   q2 and q3 works fine in datafusion.
   
   **Additional context**
   Add any other context about the problem here.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] Dandandan commented on issue #2230: Panic while running inner join with predicate only on single relation

Posted by GitBox <gi...@apache.org>.
Dandandan commented on issue #2230:
URL: https://github.com/apache/arrow-datafusion/issues/2230#issuecomment-1099665076

   Reproducing:
   
   ```
   ❯ create table y as select 1 c;
   +---+
   | c |
   +---+
   | 1 |
   +---+
   1 row in set. Query took 0.005 seconds.
   ❯ select * from y t1 join y t2 on t1.c = 1 and  t2.c = 1;
   thread 'tokio-runtime-worker' panicked at 'index out of bounds: the len is 0 but the index is 0', /home/danielheres/Code/gdd/arrow-datafusion/datafusion/core/src/physical_plan/repartition.rs:349:39
   note: run with `RUST_BACKTRACE=1` environment variable to display a backtrace
   ArrowError(ExternalError(ArrowError(ExternalError(Execution("Join Error: panic")))))
   ❯ explain select * from y t1 join y t2 on t1.c = 1 and  t2.c = 1;
   +---------------+-----------------------------------------------------------------+
   | plan_type     | plan                                                            |
   +---------------+-----------------------------------------------------------------+
   | logical_plan  | Projection: #t1.c, #t2.c                                        |
   |               |   Inner Join:                                                   |
   |               |     Filter: #t1.c = Int64(1)                                    |
   |               |       TableScan: t1 projection=Some([0])                        |
   |               |     Filter: #t2.c = Int64(1)                                    |
   |               |       TableScan: t2 projection=Some([0])                        |
   | physical_plan | ProjectionExec: expr=[c@0 as c, c@1 as c]                       |
   |               |   CoalesceBatchesExec: target_batch_size=4096                   |
   |               |     HashJoinExec: mode=Partitioned, join_type=Inner, on=[]      |
   |               |       CoalesceBatchesExec: target_batch_size=4096               |
   |               |         RepartitionExec: partitioning=Hash([], 16)              |
   |               |           CoalesceBatchesExec: target_batch_size=4096           |
   |               |             FilterExec: c@0 = 1                                 |
   |               |               RepartitionExec: partitioning=RoundRobinBatch(16) |
   |               |                 MemoryExec: partitions=1, partition_sizes=[1]   |
   |               |       CoalesceBatchesExec: target_batch_size=4096               |
   |               |         RepartitionExec: partitioning=Hash([], 16)              |
   |               |           CoalesceBatchesExec: target_batch_size=4096           |
   |               |             FilterExec: c@0 = 1                                 |
   |               |               RepartitionExec: partitioning=RoundRobinBatch(16) |
   |               |                 MemoryExec: partitions=1, partition_sizes=[1]   |
   |               |                                                                 |
   +---------------+-----------------------------------------------------------------+
   2 rows in set. Query took 0.007 seconds.
   ```
   
   I think the reason is we are creating a hash join based on a empty list of keys.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-datafusion] yjshen closed issue #2230: Panic while running inner join with predicate only on single relation

Posted by GitBox <gi...@apache.org>.
yjshen closed issue #2230: Panic while running inner join with predicate only on single relation
URL: https://github.com/apache/arrow-datafusion/issues/2230


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org