[ 
https://issues.apache.org/jira/browse/FLINK-8607?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16362804#comment-16362804
 ] 

ASF GitHub Bot commented on FLINK-8607:
---------------------------------------

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5441#discussion_r167947410
  
    --- Diff: 
flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/DynamicResult.java
 ---
    @@ -0,0 +1,253 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.client.gateway;
    +
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.table.sinks.TableSink;
    +import org.apache.flink.types.Row;
    +
    +import java.net.InetAddress;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * A result of a dynamic table program.
    + */
    +public class DynamicResult {
    +
    +   private final boolean isChangelog;
    +   private final CollectTableSink collectTableSink;
    +   private final Object resultLock = new Object();
    +   private final ResultRetrievalThread retrievalThread;
    +   private final JobMonitoringThread monitoringThread;
    +   private Runnable program;
    +   private SqlExecutionException executionException;
    +
    +   // for table materialization
    +   private final List<Row> materializedTable;
    +   private final Map<Row, List<Integer>> rowPositions; // positions of 
rows in table for faster access
    +   private final List<Row> snapshot;
    +   private int pageSize;
    +   private boolean isLastSnapshot;
    +
    +   // for changelog
    +   private Tuple2<Boolean, Row> nextChangeRecord;
    --- End diff --
    
    Also fetch change rows in a List for more efficient retrieval?


> Add a basic embedded SQL CLI client
> -----------------------------------
>
>                 Key: FLINK-8607
>                 URL: https://issues.apache.org/jira/browse/FLINK-8607
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API &amp; SQL
>            Reporter: Timo Walther
>            Assignee: Timo Walther
>            Priority: Major
>
> This issue describes the Implementation Plan 1 of FLIP-24.
> Goal: Add the basic features to play around with Flink's streaming SQL.
> {code}
> - Add CLI component that reads the configuration files
>     - "Pre-registered table sources"
>     - "Job parameters"
> - Add executor for retrieving pre-flight information and corresponding CLI 
> SQL parser
>     - SHOW TABLES
>     - DESCRIBE TABLE
>     - EXPLAIN
> - Add streaming append query submission to executor
> - Submit jars and run SELECT query using the ClusterClient
> - Collect results on heap and serve them on the CLI side (Internal Mode with 
> SELECT)
> - SOURCE (for executing a SQL statement stored in a local file)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to