Re: [DISCUSS] Iceberg Summit 2025 ?

2024-09-27 Thread Russell Spitzer
I am really excited about the prospect of another Summit and also had a
great time last year. I think we had a great selection of talks and I'm
hoping we can do so again.

I'm very much in support of having an in person element, I would love to
have a chance to talk face to face with other members of the community. I
do think we should
preserve online viewing as well since I know not everyone has the ability
to travel.

I do hope that we can have more talks about users with Iceberg in
production as well. I think we did a really good job of covering Iceberg
development last time but didn't
have as many practitioner discussions as I would have liked. I also think
it would be great if we had a section that was purely just "ideas for
Iceberg" where folks can pitch
their features and proposals to a much broader audience.

I also would love to have some workshops this time as well, showing folks
how to use the project, how to make their first tables, and how to
contribute to the Iceberg project.

Things I'd like to avoid: Sales pitches, Talks not focused on Iceberg or
its ecosystem (Personally I don't really want to hear anything about AI or
LLMS but I know that might not be everyone). Ideally I would like this to
be a vendor neutral event where planning is as transparent as possible for
the community.

I'd love to hear what other folks are thinking,
Russ

On Fri, Sep 27, 2024 at 12:51 PM Jean-Baptiste Onofré 
wrote:

> Hi folks,
>
> Last year in June we started to discuss the first edition of the Iceberg
> Summit (https://lists.apache.org/thread/cbgx1jlc9ywn618yod2487g498lgrkt3).
>
>
> The Iceberg Summit was in May 2024, and it was clearly a great community
> event, with a lot of nice talks.
> This first edition was fully virtual.
>
> I think it would be great to have Iceberg Summit 2025, community event,
> but maybe this time a hybrid event.
> Also, regarding the number of talks received by the selection committee
> for Iceberg Summit 2024, I would suggest (for the future Selection
> Committee) to have new talk tracks (like user stories, practitioners, ...).
>
> The process would be similar of Iceberg Summit 2024:
> - first the community discuss here about the idea, kind of event (virtual,
> in person, hybrid), ...
>* should we have another event ?
>* would you like there to be an in-person event ?
>* what kind of talks would you like to hear at such an event ?
>* what kind of talks would you not like to hear at such an event ?
> - if there's no objections, the Iceberg PMC should approve the use of
> Iceberg and the ASF VP M&P should be notified. I can help on the paperwork
> and process again.
> - the PMC will appoint two committees (at least selection and sponsoring
> committees)
>
> Thoughts ?
>
> Regards
> JB
>


Re: [EXTERNAL] Re: [DISCUSS] Column to Column filtering

2024-09-27 Thread Baldwin, Jennifer
Please see attached, I hope this provides you with more clarity on the use case 
we hope to support.  Let me know if you have any further questions

From: Russell Spitzer 
Date: Wednesday, September 18, 2024 at 6:15 PM
To: dev@iceberg.apache.org 
Cc: jennifer.bald...@teradata.com.invalid 

Subject: [EXTERNAL] Re: [DISCUSS] Column to Column filtering
[CAUTION: External Email]

I have similar concerns to Ryan although I could see that if we were writing 
smaller and better correlated files that this could be a big help. Specifically 
with variant use cases this may be very useful. I would love to hear more about 
the use cases and rationale for adding this. Do you have any specific examples 
you can go into detail on?

On Wed, Sep 18, 2024 at 4:48 PM rdb...@gmail.com 
mailto:rdb...@gmail.com>> wrote:
I'm curious to learn more about this feature. Is there a driving use case that 
you're implementing it for? Are there common situations in which these filters 
are helpful and selective?

My initial impression is that this kind of expression would have limited 
utility at the table format level. Iceberg tracks column ranges for data files 
and the primary use case for filtering is to skip data files at the scan 
planning phase. For a column-to-column comparison, you would only be able to 
eliminate data files that have non-overlapping ranges. That is, if you're 
looking for rows where x < y, you can only eliminate a file when max(x) < 
min(y). To me, it seems unlikely that this would be generic enough to be worth 
it, but if there are use cases where this can happen and speed up queries I 
think it may make sense.

Ryan

On Tue, Sep 17, 2024 at 6:21 AM Baldwin, Jennifer 
 wrote:
I’m starting a thread to discuss a feature for comparisons using column 
references on the left and right side of an expression wherever iceberg 
supports column reference to literal value(s) comparisons.  The use case we 
want to support is filtering of date columns from a single table.  For instance:

select * from travel_table
where expected_date > travel_date;

select * from travel_table
where payment_date <>  due_date;


The changes will impact row and scan file filtering.  Impacted jars are 
iceberg-api, iceberg-core, iceberg-orc and iceberg-parquet.

Is this a feature the Iceberg community would be willing to accept?

Here is a link to a Draft PR with current changes, Thanks.
https://github.com/apache/iceberg/pull/11152



[VOTE] Table v3 spec: Add unknown and new type promotion

2024-09-27 Thread rdb...@gmail.com
Hi everyone,

I'd like to vote on PR #10955
 that
has been open for a while with the changes to add new type promotion cases.
After discussion, the PR has been scoped down to keep complexity low. It
now adds:

* An `unknown` type for cases when only `null` values have been observed
* Type promotion from `unknown` to any other type
* Type promotion from `date` to `timestamp` or `timestamp_ns`
* Clarification that promotion is not allowed if it breaks transform results

The set of changes is quite a bit smaller than originally proposed because
of the issue already discussed about lower and upper bounds values, and it
no longer includes variant. I think that we can add more type promotion
cases after we improve bounds metadata. This adds what we can now to keep
v3 moving forward.

Please vote in the next 72 hours:

[ ] +1, commit the proposed spec changes
[ ] -0
[ ] -1, do not make these changes because . . .

Thanks,

Ryan


Re: [VOTE] Table v3 spec: Add unknown and new type promotion

2024-09-27 Thread Russell Spitzer
+1 (binding)

On Fri, Sep 27, 2024 at 4:37 PM rdb...@gmail.com  wrote:

> Hi everyone,
>
> I'd like to vote on PR #10955
>  that has been open for a
> while with the changes to add new type promotion cases. After discussion,
> the PR has been scoped down to keep complexity low. It now adds:
>
> * An `unknown` type for cases when only `null` values have been observed
> * Type promotion from `unknown` to any other type
> * Type promotion from `date` to `timestamp` or `timestamp_ns`
> * Clarification that promotion is not allowed if it breaks transform
> results
>
> The set of changes is quite a bit smaller than originally proposed because
> of the issue already discussed about lower and upper bounds values, and it
> no longer includes variant. I think that we can add more type promotion
> cases after we improve bounds metadata. This adds what we can now to keep
> v3 moving forward.
>
> Please vote in the next 72 hours:
>
> [ ] +1, commit the proposed spec changes
> [ ] -0
> [ ] -1, do not make these changes because . . .
>
> Thanks,
>
> Ryan
>


Re: [DISCUSS] Modify ThreadPools.newWorkerPool to avoid unnecessary Shutdown Hook registration

2024-09-27 Thread rdb...@gmail.com
I'm okay with adding newFixedThreadPool as Steven suggests, but I don't
think that solves the problem that these are used more widely than intended
and without people knowing the behavior. Even though "non-exiting" is
awkward, it is maybe a good option to call out behavior. +1 for Javadoc,
and +1 for doing something here since there are improper uses throughout
Iceberg. Thanks for raising this, Peter!

On Thu, Sep 26, 2024 at 1:52 AM Jean-Baptiste Onofré 
wrote:

> Hi Steven,
>
> I agree with you here. I think we can use semantics similar to
> ThreadPoolExecutor/ScheduledThreadPoolExecutor (like
> newFixedThreadPool, newWorkStealingPool, ...).
>
> Regards
> JB
>
> On Thu, Sep 26, 2024 at 2:05 AM Steven Wu  wrote:
> >
> >
> > First, we should definitely add Javadoc to `ThreadPools.newWorkerPool`
> on its behavior with a shutdown hook. It is not obvious from the method
> name. I would actually go further to deprecate `newWorkerPool` with
> `newExitingWorkerPool`. `newWorkerPool` method name is easy to cause the
> misuage, as the intention is not obvious from the name.
> >
> > `newNonExitingWorkerPool` is a little awkward to me. `NonExiting` should
> be the default behavior. Maybe we can call this new method as
> `newFixedThreadPool(int poolSize, String prefix)`. Alternatively, we can
> just make `ThreadPools.newDaemonThreadFactory` public as the proposed
> `newNonExitingWorkerPool` really just saved one line on the thread factory
> construction.
> >
> >
> > On Wed, Sep 18, 2024 at 10:25 PM Péter Váry 
> wrote:
> >>
> >> Here are the cases where we call the `newWorkerPool` in our code:
> >>
> >> Correctly:
> >>
> >> S3FileIO.executorService
> >> HadoopFileIO.executorService
> >>
> >> Incorrectly:
> >>
> >> CountersBenchmark.defaultCounterMultipleThreads (core module)
> >> BaseDistributedDataScan.newMonitorPool (core module)
> >> FlinkInputFormat.createInputSplits (flink module)
> >> IcebergInputFormat.getSplits (flink module)
> >>
> >> Incorrectly, but typically called only once in the JVM lifecycle
> >>
> >> TableMigrationUtil.migrationService - the pool management is abandoned,
> and nothing prevents multiple pool creations (data module)
> >> IcebergCommitter (flink module)
> >> IcebergFilesCommitter.open (flink module)
> >> IcebergSource.planSplitsForBatch (flink module)
> >> StreamingMonitorFunction.open (flink module)
> >> ContinuousSplitPlannerImpl (flink module)
> >> Coordinator - Kafka coordinator - I'm not sure that this belongs
> to here (kafka-connect)
> >>
> >> The code we need to duplicate in core/data/flink/kafka module is:
> >>
> >>   public static ExecutorService newNonExitingWorkerPool(String
> namePrefix, int poolSize) {
> >> return Executors.newFixedThreadPool(
> >> poolSize,
> >> new
> ThreadFactoryBuilder().setDaemon(true).setNameFormat(namePrefix +
> "-%d").build());
> >>   }
> >>
> >>
> >> Maybe adding another utility method to the `ThreadPools` would help
> future contributors to think twice about the need for using the `Exiting`
> solution, so I would prefer to add this method to the core `ThreadPools`
> with enough javadoc to highlight the intended usage.
> >>
> >> Thanks,
> >> Peter
> >>
> >> rdb...@gmail.com  ezt írta (időpont: 2024. szept.
> 18., Sze, 23:26):
> >>>
> >>> I think this is the intended behavior. The code calls
> `MoreExecutors.getExitingExecutorService` internally to ensure the pool
> exits. I think the right fix is for callers to create their own
> `ExecutorService` rather than using `newWorkerPool`. That allows for
> customization without making Iceberg more complicated. `ThreadPools` isn't
> doing anything special here. It's just a convenience method to create an
> exiting, fixed-size thread pool that runs daemon threads. If that's not
> what you're looking for then isn't it reasonable to make your own
> convenience method?
> >>>
> >>> On Wed, Sep 18, 2024 at 1:22 PM Péter Váry <
> peter.vary.apa...@gmail.com> wrote:
> 
>  This is not just a Flink issue, tha calls are spread out in multiple
> packages. We checked the code, and in many of the current use-cases in the
> Iceberg repo the pool is not used in a static environment, and closed
> manually. In this cases we should switch to a thread pool without a
> shutdown hook. So I think minimally we need to create a utility method to
> create such a pool.
> 
>  The main question is:
>  - Is it a bug, or a feature, that we always provide a pool with a
> hook?
> 
>  If this is a bug, then we create a "newExitingWorkerPool", and change
> the callers to use the correct one.
>  If this is a feature, then we create a "newNotExitingWorkerPool"
> (which is gross IMHO, but we should consider API compatibility), and change
> the callers to use the correct one.
> 
>  Thanks,
>  Peter
> 
>  On Wed, Sep 18, 2024, 21:53 rdb...@gmail.com 
> wrote:
> >
> > Since we're using standard interfaces, maybe we should just document
> this behavior and you can con

[DISCUSS] Iceberg Summit 2025 ?

2024-09-27 Thread Jean-Baptiste Onofré
Hi folks,

Last year in June we started to discuss the first edition of the Iceberg
Summit (https://lists.apache.org/thread/cbgx1jlc9ywn618yod2487g498lgrkt3).

The Iceberg Summit was in May 2024, and it was clearly a great community
event, with a lot of nice talks.
This first edition was fully virtual.

I think it would be great to have Iceberg Summit 2025, community event, but
maybe this time a hybrid event.
Also, regarding the number of talks received by the selection committee for
Iceberg Summit 2024, I would suggest (for the future Selection Committee)
to have new talk tracks (like user stories, practitioners, ...).

The process would be similar of Iceberg Summit 2024:
- first the community discuss here about the idea, kind of event (virtual,
in person, hybrid), ...
   * should we have another event ?
   * would you like there to be an in-person event ?
   * what kind of talks would you like to hear at such an event ?
   * what kind of talks would you not like to hear at such an event ?
- if there's no objections, the Iceberg PMC should approve the use of
Iceberg and the ASF VP M&P should be notified. I can help on the paperwork
and process again.
- the PMC will appoint two committees (at least selection and sponsoring
committees)

Thoughts ?

Regards
JB


Re: Clarification on DayTransform Result Type

2024-09-27 Thread rdb...@gmail.com
The background is that the result of the day function and dates are
basically the same: the number of days from the Unix epoch. When we started
using metadata tables, we realized that a lot of people use the day
function but then get a weird ordinal value out, but if we just change the
type to `date`, engines could correctly display the value. This isn't
required by the spec, it's just a convenience.

On Fri, Sep 27, 2024 at 8:30 AM Russell Spitzer 
wrote:

> Good thing DateType is an Integer :)
> https://github.com/apache/iceberg/blob/113c6e7d62e53d3e3cb15b1712f3a1db473ca940/api/src/main/java/org/apache/iceberg/types/Type.java#L37
>
> On Thu, Sep 26, 2024 at 8:38 PM Kevin Liu  wrote:
>
>> Hey folks,
>>
>> While reviewing a PR to fix DayTransform in PyIceberg (#1208
>> ), we found an
>> inconsistency between the spec and the Java Iceberg library.
>>
>> According to the spec
>> , the result type
>> for the "day partition transform" should be `int`, similar to other
>> time-based partition transforms (year/month/hour). However, in the Java
>> Iceberg library, the result type for day partition transform is `DateType` (
>> source
>> ).
>> This seems to be a discrepancy from the spec, as the day partition
>> transform is the only time-based transform with a non-int result
>> type—whereas the others use IntegerType (source
>> 
>> ).
>>
>> Could someone confirm if my understanding is correct? If so, is there any
>> historical context for this difference? Lastly, how should we approach
>> resolving this moving forward?
>>
>> Best,
>> Kevin
>>
>>


Re: Clarification on DayTransform Result Type

2024-09-27 Thread Russell Spitzer
Good thing DateType is an Integer :)
https://github.com/apache/iceberg/blob/113c6e7d62e53d3e3cb15b1712f3a1db473ca940/api/src/main/java/org/apache/iceberg/types/Type.java#L37

On Thu, Sep 26, 2024 at 8:38 PM Kevin Liu  wrote:

> Hey folks,
>
> While reviewing a PR to fix DayTransform in PyIceberg (#1208
> ), we found an
> inconsistency between the spec and the Java Iceberg library.
>
> According to the spec
> , the result type
> for the "day partition transform" should be `int`, similar to other
> time-based partition transforms (year/month/hour). However, in the Java
> Iceberg library, the result type for day partition transform is `DateType` (
> source
> ).
> This seems to be a discrepancy from the spec, as the day partition
> transform is the only time-based transform with a non-int result
> type—whereas the others use IntegerType (source
> 
> ).
>
> Could someone confirm if my understanding is correct? If so, is there any
> historical context for this difference? Lastly, how should we approach
> resolving this moving forward?
>
> Best,
> Kevin
>
>


Re: V3 Spec Changes

2024-09-27 Thread Micah Kornfield
For variant, the current plan on moving to Parquet is to mark the variant
type as experimental. Would Iceberg depend on the experimental type or is
V3 going to wait for a variant to be deemed non-experimental by the Parquet
community?

Thanks,
Micah

On Tue, Sep 24, 2024 at 9:52 AM Russell Spitzer 
wrote:

> Hi y’all!
>
> I’m excited to say that we have a lot of great Iceberg V3 Spec PR’s out
> right now. V3 Looks like it’s going to be awesome!
>
> A reminder if you haven’t had a chance yet to check them out:
>
> Row Lineage
> Materialized Views
> Geometric Types
> Type Promotion and Variant
> Type
>
> I’m hoping we can get
> consensus on all of these ASAP so we can start working more on
> implementations and getting closer to the Iceberg 1.7 release (not that
> these are blockers I’m just hoping we can have some functionality ready.)
> So if you are interested in any of these please check them out and add any
> comments if you have any or just leave a note saying you are in on board.
>
> I also know we should have one additional PR for the new Delete File spec
> changes coming soon!
>
> Thanks everyone for your time,
> Russ
>


Re: [DISCUSS] Modify ThreadPools.newWorkerPool to avoid unnecessary Shutdown Hook registration

2024-09-27 Thread Steven Wu
>  I don't think that solves the problem that these are used more widely
than intended and without people knowing the behavior.

Ryan, to solve this problem, I suggest we deprecate the current
`newWorkerPool` with `newExitingWorkerPool`. This way, when people calls
`newExitingWorkerPool`, the intended behavior is clear from the method name.

On Fri, Sep 27, 2024 at 1:58 PM rdb...@gmail.com  wrote:

> I'm okay with adding newFixedThreadPool as Steven suggests, but I don't
> think that solves the problem that these are used more widely than intended
> and without people knowing the behavior. Even though "non-exiting" is
> awkward, it is maybe a good option to call out behavior. +1 for Javadoc,
> and +1 for doing something here since there are improper uses throughout
> Iceberg. Thanks for raising this, Peter!
>
> On Thu, Sep 26, 2024 at 1:52 AM Jean-Baptiste Onofré 
> wrote:
>
>> Hi Steven,
>>
>> I agree with you here. I think we can use semantics similar to
>> ThreadPoolExecutor/ScheduledThreadPoolExecutor (like
>> newFixedThreadPool, newWorkStealingPool, ...).
>>
>> Regards
>> JB
>>
>> On Thu, Sep 26, 2024 at 2:05 AM Steven Wu  wrote:
>> >
>> >
>> > First, we should definitely add Javadoc to `ThreadPools.newWorkerPool`
>> on its behavior with a shutdown hook. It is not obvious from the method
>> name. I would actually go further to deprecate `newWorkerPool` with
>> `newExitingWorkerPool`. `newWorkerPool` method name is easy to cause the
>> misuage, as the intention is not obvious from the name.
>> >
>> > `newNonExitingWorkerPool` is a little awkward to me. `NonExiting`
>> should be the default behavior. Maybe we can call this new method as
>> `newFixedThreadPool(int poolSize, String prefix)`. Alternatively, we can
>> just make `ThreadPools.newDaemonThreadFactory` public as the proposed
>> `newNonExitingWorkerPool` really just saved one line on the thread factory
>> construction.
>> >
>> >
>> > On Wed, Sep 18, 2024 at 10:25 PM Péter Váry <
>> peter.vary.apa...@gmail.com> wrote:
>> >>
>> >> Here are the cases where we call the `newWorkerPool` in our code:
>> >>
>> >> Correctly:
>> >>
>> >> S3FileIO.executorService
>> >> HadoopFileIO.executorService
>> >>
>> >> Incorrectly:
>> >>
>> >> CountersBenchmark.defaultCounterMultipleThreads (core module)
>> >> BaseDistributedDataScan.newMonitorPool (core module)
>> >> FlinkInputFormat.createInputSplits (flink module)
>> >> IcebergInputFormat.getSplits (flink module)
>> >>
>> >> Incorrectly, but typically called only once in the JVM lifecycle
>> >>
>> >> TableMigrationUtil.migrationService - the pool management is
>> abandoned, and nothing prevents multiple pool creations (data module)
>> >> IcebergCommitter (flink module)
>> >> IcebergFilesCommitter.open (flink module)
>> >> IcebergSource.planSplitsForBatch (flink module)
>> >> StreamingMonitorFunction.open (flink module)
>> >> ContinuousSplitPlannerImpl (flink module)
>> >> Coordinator - Kafka coordinator - I'm not sure that this belongs
>> to here (kafka-connect)
>> >>
>> >> The code we need to duplicate in core/data/flink/kafka module is:
>> >>
>> >>   public static ExecutorService newNonExitingWorkerPool(String
>> namePrefix, int poolSize) {
>> >> return Executors.newFixedThreadPool(
>> >> poolSize,
>> >> new
>> ThreadFactoryBuilder().setDaemon(true).setNameFormat(namePrefix +
>> "-%d").build());
>> >>   }
>> >>
>> >>
>> >> Maybe adding another utility method to the `ThreadPools` would help
>> future contributors to think twice about the need for using the `Exiting`
>> solution, so I would prefer to add this method to the core `ThreadPools`
>> with enough javadoc to highlight the intended usage.
>> >>
>> >> Thanks,
>> >> Peter
>> >>
>> >> rdb...@gmail.com  ezt írta (időpont: 2024. szept.
>> 18., Sze, 23:26):
>> >>>
>> >>> I think this is the intended behavior. The code calls
>> `MoreExecutors.getExitingExecutorService` internally to ensure the pool
>> exits. I think the right fix is for callers to create their own
>> `ExecutorService` rather than using `newWorkerPool`. That allows for
>> customization without making Iceberg more complicated. `ThreadPools` isn't
>> doing anything special here. It's just a convenience method to create an
>> exiting, fixed-size thread pool that runs daemon threads. If that's not
>> what you're looking for then isn't it reasonable to make your own
>> convenience method?
>> >>>
>> >>> On Wed, Sep 18, 2024 at 1:22 PM Péter Váry <
>> peter.vary.apa...@gmail.com> wrote:
>> 
>>  This is not just a Flink issue, tha calls are spread out in multiple
>> packages. We checked the code, and in many of the current use-cases in the
>> Iceberg repo the pool is not used in a static environment, and closed
>> manually. In this cases we should switch to a thread pool without a
>> shutdown hook. So I think minimally we need to create a utility method to
>> create such a pool.
>> 
>>  The main question is:
>>  - Is it a bug, or a feature, that we always provide a 

Re: [DISCUSS] Iceberg Materialzied Views

2024-09-27 Thread Benny Chow
>>  storing the lineage is an optimization that can avoid
recomputation/re-parsing.
I don't think having the lineage is optimizing much over re-parsing the
SQL.  The most expensive part of SQL parsing is catalog access which has to
happen with lineage anyway.  Once the planner has the query tree, it can
validate the freshness.  It's not like the planner needs to complete
logical and physical planning.

>> We could also have the catalog name/alias problem for the same engine.
Yes, this is a general problem with the Iceberg View spec.  I guess if two
different Spark clusters wanted to share the same view, they best not
reference the catalog name in their SQLs.  Even then, cross catalog joins
are not going to work.  Again, these are problems with the Iceberg View
spec.  I think for the MV spec, as long as we don't propose something that
involves SQL identifiers, then the MV spec isn't making this different
engine problem worse.

There's another issue I'd like to bring up about using UUIDs which is that
these UUIDs are client generated and there's no validation that they are
indeed globally unique identifiers.  The catalog just persists whatever it
is given without validating that the UUIDs are indeed UUIDs and unique
across the catalog.  (I know Nessie is not doing this validation).   We are
assuming this UUID is not only unique within a catalog but is also unique
across catalogs.  Thoughts on this?

Thanks
Benny



On Wed, Sep 25, 2024 at 8:01 PM Steven Wu  wrote:

> I agree that it is reasonable to assume/restrict view definition and
> storage table in the same catalog. Hence the storage table reference in the
> view metadata can include only namespace and table (excluding the engine
> dependent catalog name/alias).
>
> Regarding the question of having lineage metadata in view definition vs
> re-parsing SQL, I guess storing the lineage is an optimization that can
> avoid recomputation/re-parsing. would be good to have more input.
>
> Thinking about catalog name/alias again. For the same engine (like Spark),
> different applications/jobs may configure the catalog name differently.
> E.g. Spark catalogs are configured using properties under
> spark.sql.catalog.(catalog_name). We could also have the catalog
> name/alias problem for the same engine.
>
>
>
>
> On Fri, Sep 20, 2024 at 12:16 AM Jan Kaul 
> wrote:
>
>> Hi Walaa,
>>
>> It appears that you would like to maintain the lineage structure and not
>> revert to parsing the SQL to obtain identifiers.
>>
>> Initially, one of the reasons for avoiding SQL parsing was to enable
>> consumers who don't understand the SQL dialect of any representation to
>> determine the freshness of the Materialized View (MV). However, with the
>> "catalog alias" issue, having an identifier for some representation is
>> insufficient, as the *catalog_name* is unlikely to work for the
>> consumer. Therefore, supporting consumers that don't use a query engine of
>> any representation seems impossible.
>>
>> Given this, parsing the SQL definition becomes a less significant
>> drawback, as the consumer must understand the dialect anyway. In fact,
>> simply parsing the SQL definition seems like a more robust and
>> straightforward solution than using a lineage for every representation. I
>> believe this is why Benny suggested reverting to SQL parsing, and I agree
>> with him.
>>
>> Regarding the Storage table identifier: Its design as a
>> *PartialIdentifier* with only namespace and name fields was intentional,
>> to avoid the *catalog_name* issue.
>>
>> Best regards,
>>
>> Jan
>> On 19.09.24 23:16, Benny Chow wrote:
>>
>> If Spark added the storage table identifier to the MV, I'm not sure how
>> it could also add a full identifier to the Dremio representation.
>> Spark doesn't know what name Dremio used for the catalog.
>>
>> For the UX issue, I think Jan cleverly called it a "PartialIdentifier"
>> and not a "FullIdentifier" to indicate that catalog name is not even a
>> property of the identifier.
>>
>> Requirement 3 is for the view's SQL.  I'm not sure there is a very strong
>> use case to put the storage table into a different catalog than the view.
>> If we had an engine agnostic solution for it, I'm all for it though...
>>
>> Thanks
>> Benny
>>
>>
>> On Thu, Sep 19, 2024 at 1:56 PM Walaa Eldin Moustafa <
>> wa.moust...@gmail.com> wrote:
>>
>>> I think the solution for the storage identifier might be shared with the
>>> end state solution for the lineage. One could imagine a "full identifier"
>>> can be used for the storage table; however, it is
>>> "representation"-dependent (i.e., it changes according to
>>> which representation it is part of, or rather which engine uses it).
>>>
>>> Also, are we asking engines (or their Iceberg implementation) to throw
>>> an exception if the full storage table identifier was provided as part of
>>> the MV definition? Sounds like a not very ideal UX. Note that it also
>>> conflicts with the spirit of requirement #3.
>>>
>>> Thanks,
>>> Walaa.
>>