AFAIK, should apply to all queries, irrespective of the source of the data
or the plugins involved within the query. So when this issue occurs, I
would expect any query to take long to execute.

On Thu, Dec 1, 2016 at 5:47 AM John Omernik <j...@omernik.com> wrote:

> @Abhishek,
>
> Do you think the issue is related to any storage plugin that is enabled and
> not available as it applies to all queries?  I guess if it's an issue where
> all queries are slow because the foreman is waiting to initialize ALL
> storage plugins, regardless of their applicability to the queried data,
> then that is a more general issue (that should still be resolved, does the
> foreman need to initialize all plugins before querying specific data?)
>  However, I am still concerned that the query on the CTAS parquet data is
> specifically slower because of it's source.  @Rahul could you test a
> different Parquet table, NOT loaded from the SQL server to see if the
> enabling or disabling the JDBC storage plugin (with the server unavailable)
> has any impact?  Basically, I want to ensure that data that is created as a
> Parquet table via CTAS is 100% free of any links to the source data. This
> is EXTREMELY important.
>
> John
>
>
>
> On Thu, Dec 1, 2016 at 12:46 AM, Abhishek Girish <
> abhishek.gir...@gmail.com>
> wrote:
>
> > Thanks for the update, Rahul!
> >
> > On Wed, Nov 30, 2016 at 9:45 PM Rahul Raj <
> rahul....@option3consulting.com
> > >
> > wrote:
> >
> > > Abhishek,
> > >
> > > Your observation is correct, we just verified that:
> > >
> > >    1. The queries run as expected(faster) with Jdbc plugin disabled.
> > >    2. Queries run as expected when the plugin's datasource is running.
> > >    3. With the datasource down, queries run very slow waiting for the
> > >    connection to fail
> > >
> > > Rahul
> > >
> > > On Thu, Dec 1, 2016 at 10:07 AM, Abhishek Girish <
> > > abhishek.gir...@gmail.com>
> > > wrote:
> > >
> > > > @John,
> > > >
> > > > I agree that this should work. While I am not certain, I don't think
> > the
> > > > issue is specific to a particular plugin, but the way in a query's
> > > > lifecycle, the foreman attempts to initialize every enabled storage
> > > plugin
> > > > before proceeding to execute the query. So when a particular plugin
> > isn't
> > > > configured correctly or the underlying datasource is not up, this
> could
> > > > drastically slow down the query execution time.
> > > >
> > > > I'll look up to see if we have a JIRA for this already - if not will
> > file
> > > > one.
> > > >
> > > > On Wed, Nov 30, 2016 at 8:12 AM, John Omernik <j...@omernik.com>
> > wrote:
> > > >
> > > > > So just my opinion in reading this thread.  (sorry for swooping in
> an
> > > > > opining)
> > > > >
> > > > > If a CTAS is done from any data source into Parquet files.... there
> > > > should
> > > > > be NO dependency on the original data source to query the resultant
> > > > Parquet
> > > > > files.   As a Drill user, as a Drill admin, this breaks the concept
> > of
> > > > > least surprise.  If I take data from one source, and create Parquet
> > > files
> > > > > in a distributed file system, it should just work.  If there are
> > > "issues"
> > > > > with JDBC plugins or the HBase/Hive plugins in a similar manner,
> > these
> > > > > needs to be hunted down by a large group of villages with
> pitchforks
> > > and
> > > > > torches.  I just can't see how this could be acceptable at any
> level.
> > > The
> > > > > whole idea of Parquet files is they are self describing, schema
> > > included
> > > > > files.... thus a read of a directory of Parquet files should have
> NO
> > > > > dependancies on anything but the parquet files... even the Parquet
> > > > > "additions" (such as the METADATA Cache) should be a fail open
> > thing...
> > > > if
> > > > > it exists great, use it, speed things up, but if it doesn't read
> the
> > > > > parquet files as normal (Which I believe is how it operates)
> > > > >
> > > > > John
> > > > >
> > > > > On Wed, Nov 30, 2016 at 12:12 AM, Abhishek Girish <
> > > > > abhishek.gir...@gmail.com
> > > > > > wrote:
> > > > >
> > > > > > Can you attempt to disable to jdbc plugin (configured with
> > SQLServer)
> > > > and
> > > > > > try the query (on parquet) when SQL Server is offline?
> > > > > >
> > > > > > I've seen a similar issue previously when the HBase / Hive plugin
> > was
> > > > > > enabled but either the plugin configuration was wrong or the
> > > underlying
> > > > > > data source was down.
> > > > > >
> > > > > > On Fri, Nov 25, 2016 at 3:21 AM, Rahul Raj
> > > > <rahul.raj@option3consulting.
> > > > > > com>
> > > > > > wrote:
> > > > > >
> > > > > > > I have created a parquet file using CTAS from a MS SQL Server.
> > The
> > > > > query
> > > > > > on
> > > > > > > parquet is getting stuck in STARTING state for a long time
> before
> > > > > > returning
> > > > > > > the results.
> > > > > > >
> > > > > > > We could see that drill was trying to connect to the MS SQL
> > server
> > > > from
> > > > > > > which the data was imported. The MSSQL server was down, drill
> > threw
> > > > an
> > > > > > > exception "Failure while attempting to load JDBC schema", and
> > then
> > > > > > returned
> > > > > > > the results. While SQL server is running, the query executes
> > > without
> > > > > > > issues.
> > > > > > >
> > > > > > > Why is drill querying the DB metadata externally and not the
> > > imported
> > > > > > > parquets?
> > > > > > >
> > > > > > > Rahul.
> > > > > > >
> > > > > > > --
> > > > > > > **** This email and any files transmitted with it are
> > confidential
> > > > and
> > > > > > > intended solely for the use of the individual or entity to whom
> > it
> > > is
> > > > > > > addressed. If you are not the named addressee then you should
> not
> > > > > > > disseminate, distribute or copy this e-mail. Please notify the
> > > sender
> > > > > > > immediately and delete this e-mail from your system.****
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > > --
> > > **** This email and any files transmitted with it are confidential and
> > > intended solely for the use of the individual or entity to whom it is
> > > addressed. If you are not the named addressee then you should not
> > > disseminate, distribute or copy this e-mail. Please notify the sender
> > > immediately and delete this e-mail from your system.****
> > >
> >
>

Reply via email to