Thanks for the detailed report !

One more thing:  We now have made a lot of progress in integrating Alibaba
Cloud (https://www.aliyun.com/), Please see
https://github.com/apache/iceberg/projects/21 (Thanks @xingbowu -
https://github.com/xingbowu).

On Thu, Oct 21, 2021 at 11:30 PM Sam Redai <s...@tabular.io> wrote:

> Good Morning Everyone,
>
> Here are the minutes from our Iceberg Sync that took place on October
> 20th, 9am-10am PT. Please remember that anyone can join the discussion so
> feel free to share the Iceberg-Sync
> <https://groups.google.com/g/iceberg-sync> google group with anyone who
> is seeking an invite. As usual, the notes and the agenda are posted in the 
> live
> doc
> <https://docs.google.com/document/d/1YuGhUdukLP5gGiqCbk0A5_Wifqe2CZWgOd3TbhY3UQg/edit?usp=drive_web>
>  that's
> also attached to the meeting invitation.
>
> We covered a lot of topics...here we go!:
>
> Top of the Meeting Highlights
>
>    -
>
>    Sort based compaction - This is finished, reviewed, and merged. When
>    you compact data files, you can now also have Spark re-sort them, either by
>    the table’s sort order or the sort order given when you create the
>    compaction job.
>    -
>
>    Spark build refactor: Thank you to Jack for getting us started on the
>    Spark build refactor and also thanks to Anton for reviewing and helping get
>    these changes in. We’ve gone with a variant of option 3 from our last
>    discussions where we include all of the spark modules in our build but make
>    it easy to turn them off. This way we can get the CI to run Spark, Hive,
>    and Flink tests separately and only if necessary.
>    -
>
>    Delete files implementation for ORC: Thanks to Peter for adding
>    builders to store deletes in ORC (previously we could only store deletes in
>    Parquet or Avro). This means we now have support for all 3 formats for this
>    feature.
>    -
>
>    Flink Update: We’ve updated Flink to 1.13 so we’re back on a supported
>    version. 1.14 is out this week so we can aim to move to that at some point.
>
> Iceberg 0.12.1 Upcoming Patch Release (milestone
> <https://github.com/apache/iceberg/milestone/15?closed=1>)
>
>    -
>
>    Fix for the parquet map projection bug
>    -
>
>    Fix Flink CDC bug
>    -
>
>    A few other fixes that we also want to get out to the community so
>    we’re going to start a release candidate as soon as possible
>    -
>
>    Kyle will start a thread in the general slack channel so everyone
>    please feel free to mention any additional fixes that they want to see in
>    this patch release
>
> Snapshot Releases
>
>    -
>
>    Eduard will tackle adding snapshot releases
>    -
>
>    In our deploy.gradle file, it’s setup to deploy to the snapshot
>    repository
>    -
>
>    May require certain credentials so it may be required to reach out to
>    the ASF infrastructure team
>
> Iceberg 0.13.0 Upcoming Release
>
>    -
>
>    There’s agreement to switch to a time based release schedule so the
>    next release is roughly mid-November
>    -
>
>    Jack will cut a branch close to that time and any features that aren’t
>    in yet will be pushed to the next release
>    -
>
>    We agree not to hold up releases to squeeze features in and prefer
>    instead to aim for releasing sooner the next time
>
> Adding v3.2 to Spark Build Refactoring
>
>    -
>
>    Russell and Anton will coordinate on dropping in a Spark 3.2 module
>    -
>
>    We currently have 3.1 in the `spark3` module. We’ll move that out to
>    its own module and mirror what we do with the 3.2 module. (This will enable
>    cleaning up some mixed 3.0/3.1 code)
>
> Merge on Read
>
>    -
>
>    Anton has a bunch of PRs ready to queue up to contribute their
>    internal implementation. (Russell will work with him)
>    -
>
>    This feature will allow for a much lower write amplification
>    -
>
>    The expectation is that in Spark 3.3 we can rely on Spark’s internal
>    merge on read
>
> Snapshot Tagging (design doc
> <https://docs.google.com/document/d/1PvxK_0ebEoX3s7nS6-LOJJZdBYr_olTWH9oepNUfJ-A/edit>)
> (PR #3104 <https://github.com/apache/iceberg/pull/3104>)
>
>    -
>
>    We just had a meeting on Monday about that and made some conclusions
>    and designs, so anyone who is interested please take a look.
>    -
>
>    Next steps are to add the feature in the stack and Jack already has a
>    WIP implementation into the table metadata class
>
> Delete Compaction (design doc
> <https://docs.google.com/document/d/1-EyKSfwd_W9iI5jrzAvomVw3w1mb_kayVNT7f2I-SUg>
> )
>
>    -
>
>    Discussion happening at 5pm ET on 10/21 5-6pm PT for anyone interested
>    (meeting link <https://meet.google.com/nxx-nnvj-omx>)
>    -
>
>    Some more discussion is needed to hone in on a final design choice.
>    There are a few options that each have their own pros and cons.
>
> The New Source Interface for Flink (FLIP-27
> <https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface>
> )
>
>    -
>
>    Eventually everything will move to this new source interface (Kafka is
>    already using this and it will be the default in Flink 1.14)
>    -
>
>    A few PR’s for Iceberg are out there and are pending review and merge
>    (may not make the deadline for the next release but that’s ok)
>
> Encryption MVP
>
>    -
>
>    Just had a recent sync on this and we're currently waiting on a few
>    updates to the design
>    -
>
>       Flesh out how the new pushdown encryption into ORC and Parquet will
>       work
>       -
>
>       Need some people to review the stream based encryption,
>       particularly around splitability
>       -
>
>    A few offline discussions are currently happening and for the
>    interface we are expecting a few additional PRs separate from the main
>    encryption MVP PR
>
> Python Library Development
>
>    -
>
>    The high level design discussions have been concluded recently
>    -
>
>    We’ll delay the top level API discussions until some of the core is
>    implemented
>    -
>
>    We have a collection of issues created and a handful of engineers
>    working on it
>
> Iceberg Docsite Refactoring
>
>    -
>
>    Large refactoring coming for the Iceberg docsite
>    -
>
>       Versioned docs (In the future need to decide how to represent the
>       python versions)
>       -
>
>       Organized more by the persona of the visitor (Data Engineer,
>       Systems Engineer, etc.)
>       -
>
>       Searchable
>       -
>
>    Expect a PR from Sam, ready for review by the end of this week or
>    early next week
>
> Row-Level Support in the Vectorized Reader (PR #3141
> <https://github.com/apache/iceberg/issues/3141>)
>
>    -
>
>    Yufei is working on this and it’s part of the effort for merge on read
>    -
>
>    PR #3287 <https://github.com/apache/iceberg/pull/3287> is only for the
>    position delete in parquet
>    -
>
>    We should have something ready to add by next week
>
> View Spec (PR #3188 <https://github.com/apache/iceberg/pull/3188>)
>
>    -
>
>    There was a discussion on if we should we have just the SQL text
>    exactly as it was passed to the engine or should we also include the parsed
>    and analyze plan (includes column resolution). In theory, the resolved SQL
>    text should be very useful but it’s usefulness may be limited to certain
>    edge cases.
>    -
>
>    The broader discussion here is: Should we allow having multiple
>    dialects (Trino, Spark, etc..)
>    -
>
>       Adds complexity
>       -
>
>       Time traveling needs to be considered. What does time traveling a
>       view mean? If the underlying table is an Iceberg table we may be able 
> to,
>       but even that would require “as of” time travel to allow time travel 
> across
>       multiple tables.
>       -
>
>       Time traveling schemas needs to be added
>       -
>
>    Agreement that we should not try to solve everything at once but break
>    this into smaller problems.
>    -
>
>    Let’s keep an eye on upcoming engine features to see if this will be
>    implicitly solved and let's also refrain from over engineering this.
>
> That's it! Thanks everyone for the high level of participation and enjoy
> the rest of your week!
>
>

Reply via email to