No, type factories belong to a connection (or shorter duration if you like). By 
design, the tables in a schema don’t have row types; they have a method you 
call to get a row type for a particular type factory. So type factories are 
“short term” objects, schemas and tables are “long term”.

> On Aug 31, 2017, at 12:29 AM, Christian Beikov <[email protected]> 
> wrote:
> 
> My CAS scheme was merely meant for Materialization registration. The retry is 
> an implementation detail and would happen inside of the registration method, 
> the user wouldn't notice that. APIs stay the way they are, I'd only change 
> the way the MaterializationActor is accessed.
> 
> The (root) schema is one part that I'd like to see being shared, but I guess 
> the type factory as well as the CalciteServer should be shared too between 
> connections. Is there anything else you think that can/should be shared?
> 
> I could implement the discussed sharing as a javax.sql.DataSource if you want 
> so we can discuss specifics. Along the way I'd try to do some general 
> performance improvements regarding concurrency synchronization mechanisms. 
> Would that be ok?
> 
> 
> Mit freundlichen Grüßen,
> ------------------------------------------------------------------------
> *Christian Beikov*
> Am 31.08.2017 um 00:02 schrieb Julian Hyde:
>> Schema is the context you are referring to. Schema has a longer
>> lifespan than Connection, and if you make an immutable one (which we
>> recommend) you can share it among connections.
>> 
>> Your CAS scheme would work but requires each user to create a copy of
>> all of the materialization state. This is potentially large (thousands
>> of tables) and rapidly changing. Also, your scheme requires the user
>> to re-try. I think the actor model is better suited for this.
>> 
>> On Wed, Aug 30, 2017 at 2:14 PM, Christian Beikov
>> <[email protected]> wrote:
>>> Of course steps 2 and 3 depend on what you read, but if a change happens in
>>> the meantime you'r CAS will fail in step 3 since all changes are done
>>> through such a CAS, so you have to "redo" the transaction or parts of it.
>>> That's basically optimistic locking :)
>>> 
>>> The important part is, that the whole holder is replaced so you can do
>>> guarantee safety by doing a single CAS. Imagine the actor field in
>>> MaterializationService is wrapped by an AtomicReference and all maps in the
>>> MaterializationActor are immutable. The only way to change a thing is to
>>> read the actor, create a copy of it with the new state and do a CAS through
>>> the atomic reference. That would already solve all thread safety issues that
>>> the current design has.
>>> 
>>> Could you maybe comment on the context sharing between connections part too?
>>> 
>>> 
>>> Mit freundlichen Grüßen,
>>> ------------------------------------------------------------------------
>>> *Christian Beikov*
>>> Am 30.08.2017 um 21:31 schrieb Julian Hyde:
>>>> Consider a “transaction” that involves reads and writes:
>>>> 
>>>>    Read from a data structure
>>>>    Do some stuff
>>>>    Write to the data structure
>>>> 
>>>> If steps 2 and 3 depend on what you read in step 1, then you need to
>>>> prevent anyone from writing until you have written. A simple CAS won’t 
>>>> solve
>>>> this. The simplest solution is for the whole transaction to be in a 
>>>> critical
>>>> section. It doesn’t really matter whether that is implemented using an 
>>>> actor
>>>> or synchronized blocks.
>>>> 
>>>> We are mostly in agreement - especially about using immutable data
>>>> structures for anything shared between threads.
>>>> 
>>>> Julian
>>>> 
>>>> 
>>>>> On Aug 29, 2017, at 2:01 PM, Christian Beikov
>>>>> <[email protected]> wrote:
>>>>> 
>>>>> Imagine the holder of the various hash maps is immutable, let's call it
>>>>> "actor". When a new registration is done, we create a copy of that holder
>>>>> and CAS it. When we query, we simply get the current value and access it's
>>>>> maps. So MaterializationService could have an AtomicReference to a holder
>>>>> "actor" just like right now, but we make the maps immutable and create
>>>>> copies whenever a change occurs. We could hide such details behind a 
>>>>> message
>>>>> passing interface so that remote models can be implemented too, but that
>>>>> seems like a next step.
>>>>> 
>>>>> The materialization concurrency issues isn't the only problem, what about
>>>>> the general usage in multithreaded environments? The whole schema is
>>>>> currently bound to a CalciteConnection. It would be nice if all the 
>>>>> context
>>>>> could be shared between multiple connections so that we avoid having to
>>>>> initialize every connection. Do you have any plans to tackle that or am I
>>>>> not seeing how to achieve this?
>>>>> 
>>>>> 
>>>>> Mit freundlichen Grüßen,
>>>>> ------------------------------------------------------------------------
>>>>> *Christian Beikov*
>>>>> Am 29.08.2017 um 19:40 schrieb Julian Hyde:
>>>>>>> I'd rather have immutable state being CASed(compare-and-swap) to make
>>>>>>> the querying cheap and do updates in an optimistic concurrency control
>>>>>>> manner.
>>>>>> Compare and swap only works for one memory address. You can't use it
>>>>>> to, say, debit one bank account and credit another.
>>>>>> 
>>>>>> The set of valid materializations is just about the only mutable state
>>>>>> in Calcite and I think it will need to be several interconnected data
>>>>>> structures. So, compare-and-swap (or its high-level equivalent,
>>>>>> ConcurrentHashMap) won't cut it.
>>>>>> 
>>>>>> So we could use locks/monitors (the "synchronized" keyword) or we
>>>>>> could use an actor. The key difference between the two is who does the
>>>>>> work. With a monitor, each customer grabs the key (there is only one
>>>>>> key), walks into the bank vault, and moves the money from one deposit
>>>>>> box to another. With an actor, there is a bank employee in the vault
>>>>>> who is the only person allowed to move money around.
>>>>>> 
>>>>>> The work done is the same in both models. There are performance
>>>>>> advantages of the actor model (the data structures will tend to exist
>>>>>> in one core's cache) and there are code simplicity advantages (the
>>>>>> critical code is all in one class or package).
>>>>>> 
>>>>>> The overhead of two puts/gets on an ArrayBlockingQueue per request is
>>>>>> negligible. And besides, you can switch to a non-actor implementation
>>>>>> of the service if Calcite is single-threaded.
>>>>>> 
>>>>>> I haven't thought out the details of multi-tenant. It is not true to
>>>>>> say that this is "not a primary requirement for
>>>>>> the Calcite project." Look at the "data grid (cache)" on the diagram
>>>>>> in my "Optiq" talk [1] from 2013. Dynamic materialized views were in
>>>>>> from the very start. There can be multiple instances of the actor
>>>>>> (each with their own request/response queues), so you could have one
>>>>>> per tenant. Also, it is very straightforward to make the actors
>>>>>> remote, replacing the queues with RPC over a message broker. Remote
>>>>>> actors are called services.
>>>>>> 
>>>>>> Julian
>>>>>> 
>>>>>> [1]
>>>>>> https://www.slideshare.net/julianhyde/optiq-a-dynamic-data-management-framework
>>>>>> 
>>>>>> On Tue, Aug 29, 2017 at 8:25 AM, Jesus Camacho Rodriguez
>>>>>> <[email protected]> wrote:
>>>>>>> LGTM, I think by the time we have support for the outer joins, I might
>>>>>>> have
>>>>>>> had time to finish the filter tree index implementation too.
>>>>>>> 
>>>>>>> -Jesús
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> On 8/29/17, 3:11 AM, "Christian Beikov" <[email protected]>
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> I'd like to stick to trying to figure out how to support outer joins
>>>>>>>> for
>>>>>>>> now and when I have an implementation for that, I'd look into the
>>>>>>>> filter
>>>>>>>> tree index if you haven't done it by then.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Mit freundlichen Grüßen,
>>>>>>>> 
>>>>>>>> ------------------------------------------------------------------------
>>>>>>>> *Christian Beikov*
>>>>>>>> Am 28.08.2017 um 20:01 schrieb Jesus Camacho Rodriguez:
>>>>>>>>> Christian,
>>>>>>>>> 
>>>>>>>>> The implementation of the filter tree index is what I was referring
>>>>>>>>> to
>>>>>>>>> indeed. In the initial implementation I focused on the rewriting
>>>>>>>>> coverage,
>>>>>>>>> but now that the first part is finished, it is at the top of my list
>>>>>>>>> as
>>>>>>>>> I think it is critical to make the whole query rewriting algorithm
>>>>>>>>> work
>>>>>>>>> at scale. However, I have not started yet.
>>>>>>>>> 
>>>>>>>>> The filter tree index will help to filter not only based on the
>>>>>>>>> tables used
>>>>>>>>> by a given query, but also for queries that do not meet the
>>>>>>>>> equivalence
>>>>>>>>> classes conditions, filter conditions, etc. We could implement all
>>>>>>>>> the
>>>>>>>>> preconditions mentioned in the paper, and we could add our own
>>>>>>>>> additional
>>>>>>>>> ones. I also think that in a second version, we might need to maybe
>>>>>>>>> add
>>>>>>>>> some kind of ranking/limit as many views might meet the preconditions
>>>>>>>>> for
>>>>>>>>> a given query.
>>>>>>>>> 
>>>>>>>>> It seems you understood how it should work, so if you could help to
>>>>>>>>> quickstart that work by maybe implementing a first version of the
>>>>>>>>> filter
>>>>>>>>> tree index with a couple of basic conditions (table matching and EC
>>>>>>>>> matching?),
>>>>>>>>> that would be great. I could review any of the contributions you
>>>>>>>>> make.
>>>>>>>>> 
>>>>>>>>> -Jesús
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On 8/28/17, 3:22 AM, "Christian Beikov" <[email protected]>
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> If the metadata was cached, that would be awesome, especially
>>>>>>>>>> because
>>>>>>>>>> that would also improve the prformance regarding the metadata
>>>>>>>>>> retrival
>>>>>>>>>> for the query currently being planned, although I am not sure how
>>>>>>>>>> the
>>>>>>>>>> caching would work since the RelNodes are mutable.
>>>>>>>>>> 
>>>>>>>>>> Have you considered implementing the filter tree index explained in
>>>>>>>>>> the
>>>>>>>>>> paper? As far as I understood, the whole thing only works when a
>>>>>>>>>> redundant table elimination is implemented. Is that the case? If so,
>>>>>>>>>> or
>>>>>>>>>> if it can be done easily, I'd propose we initialize all the lookup
>>>>>>>>>> structures during registration and use them during planning. This
>>>>>>>>>> will
>>>>>>>>>> improve planning time drastically and essentially handle the
>>>>>>>>>> scalability
>>>>>>>>>> problem you mention.
>>>>>>>>>> 
>>>>>>>>>> What other MV-related issues are on your personal todo list Jesus? I
>>>>>>>>>> read the paper now and think I can help you in one place or another
>>>>>>>>>> if
>>>>>>>>>> you want.
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> Mit freundlichen Grüßen,
>>>>>>>>>> 
>>>>>>>>>> ------------------------------------------------------------------------
>>>>>>>>>> *Christian Beikov*
>>>>>>>>>> Am 28.08.2017 um 08:13 schrieb Jesus Camacho Rodriguez:
>>>>>>>>>>> Hive does not use the Calcite SQL parser, thus we follow a
>>>>>>>>>>> different path
>>>>>>>>>>> and did not experience the problem on the Calcite end. However,
>>>>>>>>>>> FWIW we
>>>>>>>>>>> avoided reparsing the SQL every time a query was being planned by
>>>>>>>>>>> creating/managing our own cache too.
>>>>>>>>>>> 
>>>>>>>>>>> The metadata providers implement some caching, thus I would expect
>>>>>>>>>>> that once
>>>>>>>>>>> you avoid reparsing every MV, the retrieval time of predicates,
>>>>>>>>>>> lineage, etc.
>>>>>>>>>>> would improve (at least after using the MV for the first time).
>>>>>>>>>>> However,
>>>>>>>>>>> I agree that the information should be inferred when the MV is
>>>>>>>>>>> loaded.
>>>>>>>>>>> In fact, maybe just making some calls to the metadata providers
>>>>>>>>>>> while the MVs
>>>>>>>>>>> are being loaded would do the trick (Julian should confirm this).
>>>>>>>>>>> 
>>>>>>>>>>> Btw, probably you will find another scalability issue as the number
>>>>>>>>>>> of MVs
>>>>>>>>>>> grows large with the current implementation of the rewriting, since
>>>>>>>>>>> the´
>>>>>>>>>>> pre-filtering implementation in place does not discard many of the
>>>>>>>>>>> views that
>>>>>>>>>>> are not valid to rewrite a given query, and rewriting is attempted
>>>>>>>>>>> with all
>>>>>>>>>>> of them.
>>>>>>>>>>> This last bit is work that I would like to tackle shortly, but I
>>>>>>>>>>> have not
>>>>>>>>>>> created the corresponding JIRA yet.
>>>>>>>>>>> 
>>>>>>>>>>> -Jesús
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On 8/27/17, 10:43 PM, "Rajat Venkatesh" <[email protected]>
>>>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Thread Safety and repeated parsing is a problem. We have
>>>>>>>>>>>> experience with
>>>>>>>>>>>> managing 10s of materialized views. Repeated parsing takes more
>>>>>>>>>>>> time than
>>>>>>>>>>>> execution of the query itself. We also have a similar problem
>>>>>>>>>>>> where
>>>>>>>>>>>> concurrent queries (with a different set of materialized views
>>>>>>>>>>>> potentailly)
>>>>>>>>>>>> maybe planned at the same time. We solved it through maintaining a
>>>>>>>>>>>> cache
>>>>>>>>>>>> and carefully setting the cache in a thread local.
>>>>>>>>>>>> Relevant code for inspiration:
>>>>>>>>>>>> 
>>>>>>>>>>>> https://github.com/qubole/quark/blob/master/optimizer/src/main/java/org/apache/calcite/prepare/Materializer.java
>>>>>>>>>>>> 
>>>>>>>>>>>> https://github.com/qubole/quark/blob/master/optimizer/src/main/java/org/apache/calcite/plan/QuarkMaterializeCluster.java
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Sun, Aug 27, 2017 at 6:50 PM Christian Beikov
>>>>>>>>>>>> <[email protected]>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Hey, I have been looking a bit into how materialized views
>>>>>>>>>>>>> perform
>>>>>>>>>>>>> during the planning because of a very long test
>>>>>>>>>>>>> run(MaterializationTest#testJoinMaterializationUKFK6) and the
>>>>>>>>>>>>> current
>>>>>>>>>>>>> state is problematic.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> CalcitePrepareImpl#getMaterializations always reparses the SQL
>>>>>>>>>>>>> and down
>>>>>>>>>>>>> the line, there is a lot of expensive work(e.g. predicate and
>>>>>>>>>>>>> lineage
>>>>>>>>>>>>> determination) done during planning that could easily be
>>>>>>>>>>>>> pre-calculated
>>>>>>>>>>>>> and cached during materialization creation.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> There is also a bit of a thread safety problem with the current
>>>>>>>>>>>>> implementation. Unless there is a different safety mechanism that
>>>>>>>>>>>>> I
>>>>>>>>>>>>> don't see, the sharing of the MaterializationService and thus
>>>>>>>>>>>>> also the
>>>>>>>>>>>>> maps in MaterializationActor via a static instance between
>>>>>>>>>>>>> multiple
>>>>>>>>>>>>> threads is problematic.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Since I mentioned thread safety, how is Calcite supposed to be
>>>>>>>>>>>>> used in a
>>>>>>>>>>>>> multi-threaded environment? Currently I use a connection pool
>>>>>>>>>>>>> that
>>>>>>>>>>>>> initializes the schema on new connections, but that is not really
>>>>>>>>>>>>> nice.
>>>>>>>>>>>>> I suppose caches are also bound to the connection? A thread safe
>>>>>>>>>>>>> context
>>>>>>>>>>>>> that can be shared between connections would be nice to avoid all
>>>>>>>>>>>>> that
>>>>>>>>>>>>> repetitive work.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Are these known issues which you have thought about how to fix or
>>>>>>>>>>>>> should
>>>>>>>>>>>>> I log JIRAs for these and fix them to the best of my knowledge?
>>>>>>>>>>>>> I'd more
>>>>>>>>>>>>> or less keep the service shared but would implement it using a
>>>>>>>>>>>>> copy on
>>>>>>>>>>>>> write strategy since I'd expect seldom schema changes after
>>>>>>>>>>>>> startup.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Regarding the repetitive work that partly happens during
>>>>>>>>>>>>> planning, I'd
>>>>>>>>>>>>> suggest doing that during materialization registration instead
>>>>>>>>>>>>> like it
>>>>>>>>>>>>> is already mentioned CalcitePrepareImpl#populateMaterializations.
>>>>>>>>>>>>> Would
>>>>>>>>>>>>> that be ok?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> --
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Mit freundlichen Grüßen,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> ------------------------------------------------------------------------
>>>>>>>>>>>>> *Christian Beikov*
>>>>>>>>>>>>> 
> 

Reply via email to