Re: Adding new connectors

2023-03-27 Thread Russell Spitzer
External Catalog is old and you probably shouldn't touch it, it's mostly for wrapping the old Hive interface. Almost all connectors now are built using the DSV2 interface either via a catalog, or directly via DataSourceRegister. I think Datasource register is mostly good for backwards

Re: Syndicate Apache Spark Twitter to Mastodon?

2022-12-01 Thread Russell Spitzer
Since this is just syndication I don't think arguments on the benefits of Twitter vs Mastodon are that important, it's really just what are the costs of additionally posting to Mastodon. I'm assuming those costs are basically 0 since this can be done by a bot? So I don't think there is any

Re: Missing string replace function

2022-10-02 Thread russell . spitzer
-+|                           ooo zzz|+----------+WDYT?On Sun, Oct 2, 2022 at 6:24 PM Russell Spitzer <russell.spit...@gmail.com> wrote:Quick test on on 3.2 confirms everything should be working as expectedscala> spark.createDataset(Seq(("foo"

Re: Missing string replace function

2022-10-02 Thread Russell Spitzer
eateTempView("temp") scala> spark.sql("SELECT replace(_1, 'fo', 'bo') from temp").show +---+ |replace(_1, fo, bo)| +---+ | boo| +---+ > On Oct 2, 2022, at 12:21 PM, Russell Spitzer > wrote: &g

Re: Missing string replace function

2022-10-02 Thread Russell Spitzer
https://spark.apache.org/docs/3.3.0/api/sql/index.html#replace This was added in Spark 2.3.0 as far as I can tell. https://github.com/apache/spark/pull/18047 > On Oct 2, 2022, at

Re: Supports Dynamic Table Options for Spark SQL

2021-11-15 Thread Russell Spitzer
I think since we probably will end up using this same syntax on write, this makes a lot of sense. Unless there is another good way to express a similar concept during a write operation I think going forward with this would be ok. On Mon, Nov 15, 2021 at 10:44 AM Ryan Blue wrote: > The proposed

Re: [VOTE] SPIP: Row-level operations in Data Source V2

2021-11-13 Thread Russell Spitzer
+1 (never binding) On Sat, Nov 13, 2021 at 1:10 AM Dongjoon Hyun wrote: > +1 > > On Fri, Nov 12, 2021 at 6:58 PM huaxin gao wrote: > >> +1 >> >> On Fri, Nov 12, 2021 at 6:44 PM Yufei Gu >> wrote: >> >>> +1 >>> >>> > On Nov 12, 2021, at 6:25 PM, L. C. Hsieh wrote: >>> > >>> > Hi all, >>> >

Re: [VOTE] SPIP: Storage Partitioned Join for Data Source V2

2021-10-29 Thread Russell Spitzer
+1 This is a great idea, (I have no Apache Spark voting points) On Fri, Oct 29, 2021 at 12:41 PM L. C. Hsieh wrote: > > I'll start with my +1. > > On 2021/10/29 17:30:03, L. C. Hsieh wrote: > > Hi all, > > > > I’d like to start a vote for SPIP: Storage Partitioned Join for Data > Source V2. >

Re: [VOTE] SPIP: Add FunctionCatalog

2021-03-08 Thread Russell Spitzer
+1 (for what it's worth) Thanks for making such a robust proposal, i'm excited to see the new work coming from this > On Mar 8, 2021, at 11:44 PM, Dongjoon Hyun wrote: > > +1 (binding) > > Thank you, Ryan. > > Bests, > Dongjoon. > > > On Mon, Mar 8, 2021 at 5:20 PM Chao Sun

Re: [DISCUSS] preferred behavior when fails to instantiate configured v2 session catalog

2020-10-23 Thread Russell Spitzer
I was convinced that we should probably just fail, but if that is too much of a change, then logging the exception is also acceptable. On Thu, Oct 22, 2020, 10:32 PM Jungtaek Lim wrote: > Hi devs, > > I got another report regarding configuring v2 session catalog, when Spark > fails to

Re: Official support of CREATE EXTERNAL TABLE

2020-10-06 Thread Russell Spitzer
I don't feel differently than I did on the thread linked above, I think treating "External" as a table option is still the safest way to go about things. For the Cassandra catalog this option wouldn't appear on our whitelist of allowed options, the same as "path" and other options that don't apply

Re: My I report a special comparaison of executions leading on issues on Spark JIRA ?

2020-10-01 Thread Russell Spitzer
You are always welcome to create a jira or jiras, but you may find you get a faster response by asking about your issues on the mailing list first. That may help in identifying whether your issues are already logged or not, or whether there is a solution that can be applied right away. On Thu,

Re: Why is V2SessionCatalog not a CatalogExtension?

2020-08-08 Thread Russell Spitzer
A v2catalog can stand alone, it doesn't have to extend the built in catalog. On Sat, Aug 8, 2020, 7:36 AM Jacek Laskowski wrote: > Hi, > > Just started exploring Catalog Plugin API and noticed these two classes: > CatalogExtension and V2SessionCatalog. > > Why is V2SessionCatalog not a

Re: [SparkSql] Casting of Predicate Literals

2020-08-04 Thread Russell Spitzer
Thanks! That's exactly what I was hoping for! Thanks for finding the Jira for me! On Tue, Aug 4, 2020 at 11:46 AM Wenchen Fan wrote: > I think this is not a problem in 3.0 anymore, see > https://issues.apache.org/jira/browse/SPARK-27638 > > On Wed, Aug 5, 2020 at 12:08 AM Rus

[SparkSql] Casting of Predicate Literals

2020-08-04 Thread Russell Spitzer
I've just run into this issue again with another user and I feel like most folks here have seen some flavor of this at some point. The user registers a Datasource with a column of type Date (or some non string) then performs a query that looks like. *SELECT * from Source WHERE date_col >

Re: Removing references to Master

2020-08-04 Thread Russell Spitzer
I think we should use Scheduler or Comptroller or Leader; something that evokes better describes the purpose as a resource management service. I would rather we didn't use controller, coordinator, application manager, primary because I feel that those terms make it seem like the process is central

Re: Spark-submit --files option help

2020-08-01 Thread Russell Spitzer
You can use SparkFiles.get(path) Example here https://github.com/datastax/spark-cassandra-connector/blob/master/connector/src/main/scala/com/datastax/spark/connector/cql/CassandraConnectionFactory.scala#L152 Also this is probably a better question for the user list than the dev one On Sat, Aug

Re: [DISCUSS][SQL] What is the best practice to add catalog support for customized storage format.

2020-07-22 Thread Russell Spitzer
There is now a full catalog API you can implement which should give you the control you are looking for. It is in Spark 3.0 and here is an example implementation for supporting Cassandra.

Re: Bridging gap between Spark UI and Code

2020-07-21 Thread Russell Spitzer
Have you looked in the DAG visualization? Each block refer to the code line invoking it. For Dataframes the execution plan will let you know explicitly which operations are in which stages. On Tue, Jul 21, 2020, 8:18 AM Michal Sankot wrote: > Hi, > when I analyze and debug our Spark batch jobs

Re: Removing references to slave (and maybe in the future master)

2020-06-18 Thread Russell Spitzer
I really dislike the use of "worker" in the code base since it describes a process which doesn't actually do work, but I don't think it's in the scope for this ticket. I would definitely prefer we use "agent" instead of "worker" (or some other name) and have master switched to something like

Re: [vote] Apache Spark 3.0 RC3

2020-06-08 Thread Russell Spitzer
+1 (non-binding) ran the new SCC DSV2 suite and all other tests, no issues On Sun, Jun 7, 2020 at 11:12 PM Yin Huai wrote: > Hello everyone, > > I am wondering if it makes more sense to not count Saturday and Sunday. I > doubt that any serious testing work was done during this past weekend. Can

Re: [DatasourceV2] Default Mode for DataFrameWriter not Dependent on DataSource Version

2020-05-21 Thread Russell Spitzer
message in case of a V2 source recommending the DSV2 api On Wed, May 20, 2020 at 5:07 PM Russell Spitzer wrote: > I think those are fair concerns, I was mostly just updating tests for RC2 > and adding in "append" everywhere > > Code like > > spark

Re: [DatasourceV2] Default Mode for DataFrameWriter not Dependent on DataSource Version

2020-05-20 Thread Russell Spitzer
atement for ErrorIfExists > mode. > https://github.com/apache/spark/blob/master/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsCatalogOptions.java > > On Wed, May 20, 2020 at 2:50 PM Russell Spitzer > wrote: > >> >&g

[DatasourceV2] Default Mode for DataFrameWriter not Dependent on DataSource Version

2020-05-20 Thread Russell Spitzer
While the ScalaDocs for DataFrameWriter say /** * Specifies the behavior when data or table already exists. Options include: * * `SaveMode.Overwrite`: overwrite the existing data. * `SaveMode.Append`: append the data. * `SaveMode.Ignore`: ignore the operation (i.e. no-op). *

Re: [DatasourceV2] Allowing Partial Writes to DSV2 Tables

2020-05-14 Thread Russell Spitzer
Yeah! That is working for me. Thanks! On Thu, May 14, 2020 at 12:10 AM Wenchen Fan wrote: > I think we already have this table capacity: ACCEPT_ANY_SCHEMA. Can you > try that? > > On Thu, May 14, 2020 at 6:17 AM Russell Spitzer > wrote: > >> I would really appreciate

Re: [DatasourceV2] Allowing Partial Writes to DSV2 Tables

2020-05-13 Thread Russell Spitzer
ing pipeline writing data to a table. I can contribute the > changes to validation if people are interested. > > On Wed, May 13, 2020 at 2:57 PM Russell Spitzer > wrote: > >> In DSV1 this was pretty easy to do because of the burden of verification >> for writes had to be

[DatasourceV2] Allowing Partial Writes to DSV2 Tables

2020-05-13 Thread Russell Spitzer
In DSV1 this was pretty easy to do because of the burden of verification for writes had to be in the datasource, the new setup makes partial writes difficult. resolveOuptutColumns checks the table schema against the writeplan's output and will fail any requests which don't contain every column as

Re: [DISCUSS] Resolve ambiguous parser rule between two "create table"s

2020-05-12 Thread Russell Spitzer
e than option 2, but it doesn't mean I >>>>>>> fully >>>>>>> agree with option 1. >>>>>>> >>>>>>> Let's make below things clear if we really go with option 1, >>>>>>> otherwise please consid

[Datasource V2] Exception Handling for Catalogs - Naming Suggestions

2020-05-12 Thread Russell Spitzer
Currently the way some actions work, we receive an error during analysis phase. For example, doing a "SELECT * FROM non_existent_table" will return an analysis exception as the NoSuchTableException is captured and replaced. Other actions like the "ShowNamespaceExec" call catalog methods directly

Re: [DISCUSS] Resolve ambiguous parser rule between two "create table"s

2020-05-11 Thread Russell Spitzer
I think reverting 30098 is the right decision here if we want to unblock 3.0. We shouldn't ship with features which we know do not function in the way we intend, regardless of how little exposure most users have to them. Even if it's off my default, we should probably work to avoid switches that

Re: BUG: spark.readStream .schema(staticSchema) not receiving schema information

2020-03-28 Thread Russell Spitzer
This is probably more of a question for the user support list, but I believe I understand the issue. Schema inside of spark refers to the structure of the output rows, for example the schema for a particular dataframe could be (User: Int, Password: String) - Two Columns the first is User of type

Re: [VOTE][SPARK-28885] Follow ANSI store assignment rules in table insertion by default

2019-10-08 Thread Russell Spitzer
+1 (non-binding). Sounds good to me On Mon, Oct 7, 2019 at 11:58 PM Wenchen Fan wrote: > +1 > > I think this is the most reasonable default behavior among the three. > > On Mon, Oct 7, 2019 at 6:06 PM Alessandro Solimando < > alessandro.solima...@gmail.com> wrote: > >> +1 (non-binding) >> >> I

Re: DSv2 sync notes - 21 August 2019

2019-08-30 Thread Russell Spitzer
Sorry all, I'm out on vacation till the week after next, I'll chime in then unless we need consensus sooner On Fri, Aug 30, 2019, 4:05 PM Ryan Blue wrote: > Sorry these notes were delayed. Here’s what we talked about in the last > DSv2 sync. > > *Attendees*: > > Ryan Blue > John Zhuge > Burak

Re: [Discuss] Follow ANSI SQL on table insertion

2019-07-29 Thread Russell Spitzer
ether invalid cast should return null or fail. The sink is only > responsible for writing data, not the type coercion/cast stuff. > > On Sun, Jul 28, 2019 at 12:24 AM Russell Spitzer < > russell.spit...@gmail.com> wrote: > >> I'm a big -1 on null values for invalid casts. This can

Re: [Discuss] Follow ANSI SQL on table insertion

2019-07-27 Thread Russell Spitzer
I'm a big -1 on null values for invalid casts. This can lead to a lot of even more unexpected errors and runtime behavior since null is 1. Not allowed in all schemas (Leading to a runtime error anyway) 2. Is the same as delete in some systems (leading to data loss) And this would be dependent on

Re: Sample date_trunc error for webpage (https://spark.apache.org/docs/2.3.0/api/sql/#date_trunc )

2019-07-07 Thread Russell Spitzer
The args look like they are in the wrong order in the doc On Sun, Jul 7, 2019, 1:50 PM Sean Owen wrote: > binggan1989, I don't see any problem in that snippet. What are you > referring to? > > On Sun, Jul 7, 2019, 2:22 PM Chris Lambertus wrote: > >> Spark, >> >> We received this message. I

Re: FW: Stage 152 contains a task of very large size (12747 KB). The maximum recommended task size is 100 KB

2019-04-25 Thread Russell Spitzer
I usually only see that in regards to folks parallelizing very large objects. From what I know, it's really just the data inside the "Partition" class of the RDD that is being sent back and forth. So usually something like spark.parallelize(Seq(reallyBigMap)) or something like that. The

Re: Sort order in bucketing in a custom datasource

2019-04-16 Thread Russell Spitzer
Please join the DataSource V2 meetings, the next one is tomorrow since we are discussing these very topics right now. Datasource v1 cannot provide this information but any source which just generates RDDs can specify a partitioner. This is only useful though if you are only using RDDs, for

Re: [VOTE] SPIP: Spark API for Table Metadata

2019-02-27 Thread Russell Spitzer
+1 (non-binding) On Wed, Feb 27, 2019, 6:28 PM Ryan Blue wrote: > Hi everyone, > > In the last DSv2 sync, the consensus was that the table metadata SPIP was > ready to bring up for a vote. Now that the multi-catalog identifier SPIP > vote has passed, I'd like to start one for the table metadata

Re: [DISCUSS] Default values and data sources

2018-12-21 Thread Russell Spitzer
kes me think that we should delegate this behavior to the source and > have a way for sources to signal that they accept default values in DDL (a > capability) and assume that they do not in most cases. > >> > >> On Thu, Dec 20, 2018 at 1:32 PM Russell Spitzer <

Re: [DISCUSS] Default values and data sources

2018-12-20 Thread Russell Spitzer
t;>> know which value will be used for missing columns, when they write data. >>>>> >>>>> If we do want to go with the "data source decides default value" >>>>> approach, we should create a new SQL syntax for ADD COLUMN, as its >&g

Re: [DISCUSS] Default values and data sources

2018-12-19 Thread Russell Spitzer
I'm not sure why 1) wouldn't be fine. I'm guessing the reason we want 2 is for a unified way of dealing with missing columns? I feel like that probably should be left up to the underlying datasource implementation. For example if you have missing columns with a database the Datasource can choose a

Re: DataSourceV2 hangouts sync

2018-10-28 Thread Russell Spitzer
Responding for invite On Fri, Oct 26, 2018, 12:34 PM Ryan Blue wrote: > Looks like the majority opinion is for Wednesday. I've sent out an invite > to everyone that replied and will add more people as I hear more responses. > > Thanks, everyone! > > On Fri, Oct 26, 2018 at 3:23 AM Gengliang

Re: Adding Extension to Load Custom functions into Thriftserver/SqlShell

2018-09-27 Thread Russell Spitzer
8 at 11:32 PM Mark Hamstra > wrote: > >> Yes, the "startWithContext" code predates SparkSessions in Thriftserver, >> so it doesn't really work the way you want it to with Session initiation. >> >> On Thu, Sep 27, 2018 at 11:13 AM Russell Spitzer < >>

Re: Adding Extension to Load Custom functions into Thriftserver/SqlShell

2018-09-27 Thread Russell Spitzer
I wrote a quick patch and attached it if anyone wants to think about this in context. I can always rebase this to master. On Thu, Sep 27, 2018 at 1:39 PM Russell Spitzer wrote: > And incase anyone is wondering, the reason I want this may be avoided with > DataSourceV2 depending o

Re: Adding Extension to Load Custom functions into Thriftserver/SqlShell

2018-09-27 Thread Russell Spitzer
and analysis rules, but I want to make sure it gets into the SQL interface. On Thu, Sep 27, 2018 at 1:35 PM Russell Spitzer wrote: > It would be a @dev internal api I think > > If we wanted to go extremely general with post session init, it could be > added to SparkExtensions > > d

Re: Adding Extension to Load Custom functions into Thriftserver/SqlShell

2018-09-27 Thread Russell Spitzer
would look like? > > On Thu, Sep 27, 2018 at 11:13 AM Russell Spitzer < > russell.spit...@gmail.com> wrote: > >> While that's easy for some users, we basically want to load up some >> functions by default into all session catalogues regardless of who made &g

Re: Adding Extension to Load Custom functions into Thriftserver/SqlShell

2018-09-27 Thread Russell Spitzer
ntext. > > On Wed, Sep 26, 2018 at 3:30 PM Russell Spitzer > wrote: > >> I've been looking recently on possible avenues to load new functions into >> the Thriftserver and SqlShell at launch time. I basically want to preload a >> set of functions in addition to t

Adding Extension to Load Custom functions into Thriftserver/SqlShell

2018-09-26 Thread Russell Spitzer
I've been looking recently on possible avenues to load new functions into the Thriftserver and SqlShell at launch time. I basically want to preload a set of functions in addition to those already present in the Spark Code. I'm not sure there is at present a way to do this and I was wondering if

Re: DataSourceWriter V2 Api questions

2018-09-11 Thread Russell Spitzer
/skip the data to be > moved/retained into the final destination. > > > > *Misc* > > Another option is to use Spark to stage data into a filesystem > (distributed, HDFS) and then use RDBMS utilities to transactionally load > data into the destination table. > > &

Re: DataSourceWriter V2 Api questions

2018-09-11 Thread Russell Spitzer
rnatives, like "replay + >> deduplicate write (per a batch/partition)" which ensures "eventually" >> exactly-once but cannot ensure the contract? >> >> Btw, unless achieving exactly-once is light enough for given sink, I >> think the sink should prov

Re: DataSourceWriter V2 Api questions

2018-09-10 Thread Russell Spitzer
Why is atomic operations a requirement? I feel like doubling the amount of writes (with staging tables) is probably a tradeoff that the end user should make. On Mon, Sep 10, 2018, 10:43 PM Wenchen Fan wrote: > Regardless the API, to use Spark to write data atomically, it requires > 1. Write

Re: DataSourceWriter V2 Api questions

2018-09-10 Thread Russell Spitzer
I think I mentioned on the Design Doc that with the Cassandra connector we have similar issues. There is no "transaction" or "staging table" capable of really doing that the api requires. On Mon, Sep 10, 2018 at 12:26 PM Reynold Xin wrote: > I don't think the problem is just whether we have a

Re: Need help with HashAggregateExec, TungstenAggregationIterator and UnsafeFixedWidthAggregationMap

2018-09-07 Thread Russell Spitzer
That's my understanding :) doExecute is for non-codegen while doProduce and Consume are for generating code On Fri, Sep 7, 2018 at 2:59 PM Jacek Laskowski wrote: > Hi Devs, > > Sorry for bothering you with my questions (and concerns), but I really > need to understand this piece of code (= my

Re: [DISCUSS] SPIP: APIs for Table Metadata Operations

2018-09-04 Thread Russell Spitzer
Spark would have to fail them in analysis without an actual physical > column. > > On Tue, Sep 4, 2018 at 9:12 PM Russell Spitzer > wrote: > >> I'm a big fan of 1 as well. I had to implement something similar using >> custom expressions and it was a bit more work than i

Re: [DISCUSS] SPIP: APIs for Table Metadata Operations

2018-09-04 Thread Russell Spitzer
I'm a big fan of 1 as well. I had to implement something similar using custom expressions and it was a bit more work than it should be. In particular our use case is that columns have certain metadata (ttl, writetime) which exist not as separate columns but as special values which can be surfaced.

Re: Select top (100) percent equivalent in spark

2018-09-04 Thread Russell Spitzer
RDD: Top http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.rdd.RDD@top(num:Int)(implicitord:Ordering[T]):Array[T ] Which is pretty much what Sean suggested For Dataframes I think doing a order and limit would be equivalent after optimizations. On Tue, Sep 4, 2018 at 2:28

Re: [DISCUSS] USING syntax for Datasource V2

2018-08-21 Thread Russell Spitzer
data in Spark’s metastore catalog. > > You could certainly build a catalog implementation that stores its data in > Cassandra or JDBC and supports the same tables that Spark does today. > That’s just not what I’m talking about here. > > On Mon, Aug 20, 2018 at 7:31 PM Russel

Re: [DISCUSS] USING syntax for Datasource V2

2018-08-20 Thread Russell Spitzer
I'm not sure I follow what the discussion topic is here > For example, a Cassandra catalog or a JDBC catalog that exposes tables in those systems will definitely not support users marking tables with the “parquet” data source. I don't understand why a Cassandra Catalogue wouldn't be able to

Re: [VOTE] SPIP: Standardize SQL logical plans

2018-07-18 Thread Russell Spitzer
+1 (non-binding) On Wed, Jul 18, 2018 at 1:32 AM Marco Gaido wrote: > +1 (non-binding) > > > On Wed, 18 Jul 2018, 07:43 Takeshi Yamamuro, > wrote: > >> +1 (non-binding) >> >> On Wed, Jul 18, 2018 at 2:41 PM John Zhuge wrote: >> >>> +1 (non-binding) >>> >>> On Tue, Jul 17, 2018 at 8:06 PM

Re: DataSourceV2 write input requirements

2018-03-29 Thread Russell Spitzer
t;> populated by a series of independent writes. Each write could have a global >>> order, but once those files are written, you have to deal with multiple >>> sorted data sets. I think it makes sense to focus on order within data >>> files, not order between data files.

Re: DataSourceV2 write input requirements

2018-03-28 Thread Russell Spitzer
; Cheers > > On Wed, Mar 28, 2018 at 7:50 PM, Russell Spitzer < > russell.spit...@gmail.com> wrote: > >> For added color, one thing that I may want to consider as a data source >> implementer is the cost / benefit of applying a particular clustering. For >> e

Re: DataSourceV2 write input requirements

2018-03-28 Thread Russell Spitzer
on - a >>> cost threshold? And yes, it would be good to flesh out what information we >>> get from Spark in the datasource when providing these >>> recommendations/requirements - I could see statistics and the existing >>> outputPartitioning/Ordering of the child

Re: DataSourceV2 write input requirements

2018-03-27 Thread Russell Spitzer
ing would not, but a required sort would. Clustering is > asking for the input dataframe's partitioning, and sorting would be how > each partition is sorted. > > On Tue, Mar 27, 2018 at 4:53 PM, Russell Spitzer < > russell.spit...@gmail.com> wrote: > >> I forgot since it's be

Re: DataSourceV2 write input requirements

2018-03-27 Thread Russell Spitzer
I forgot since it's been a while, but does Clustering support allow requesting that partitions contain elements in order as well? That would be a useful trick for me. IE Request/Require(SortedOn(Col1)) Partition 1 -> ((A,1), (A, 2), (B,1) , (B,2) , (C,1) , (C,2)) On Tue, Mar 27, 2018 at 4:38 PM

Reserved Words in Spark SQL as TableAliases

2018-03-19 Thread Russell Spitzer
I found https://issues.apache.org/jira/browse/SPARK-20964 but currently it seems like strictIdentifiers are allowed to contain any reserved key words https://github.com/apache/spark/blob/master/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4#L501-L503

Re: data source v2 online meetup

2018-02-01 Thread Russell Spitzer
-- >> *From:* Xiao Li <gatorsm...@gmail.com> >> *Sent:* Wednesday, January 31, 2018 10:46:26 PM >> *To:* Ryan Blue >> *Cc:* Reynold Xin; dev; Wenchen Fen; Russell Spitzer >> *Subject:* Re: data source v2 online meetup >> >> Hi, Ryan, >> &g

Re: [discuss] Data Source V2 write path

2017-09-27 Thread Russell Spitzer
On an unrelated note, is there any appetite for making the write path also include an option to return elements that were not able to be processed for some reason. Usage might be like saveAndIgnoreFailures() : Dataset So that if some records cannot be parsed by the datasource for writing, or

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-28 Thread Russell Spitzer
+1 (Non-binding) The clustering approach covers most of my requirements on saving some shuffles. We kind of left the "should the user be allowed to provide a full partitioner" discussion on the table. I understand that would require exposing a lot of internals so this is perhaps a good

Re: [VOTE] [SPIP] SPARK-15689: Data Source API V2

2017-08-17 Thread Russell Spitzer
-1, I don't think there has really been any discussion of this api change yet or at least it hasn't occurred on the jira ticket On Thu, Aug 17, 2017 at 8:05 AM Wenchen Fan wrote: > adding my own +1 (binding) > > On Thu, Aug 17, 2017 at 9:02 PM, Wenchen Fan

Re: PlanLater not being optimized out of Query Plan

2017-06-29 Thread Russell Spitzer
Figured it out, it was in my Exec, I hadn't defined it as a case class (Just a normal class) and just left in stubs for the Product trait methods. This just led to some... unwanted behaviors. On Thu, Jun 29, 2017 at 4:26 PM Russell Spitzer <russell.spit...@gmail.com> wrote: > I've bee

PlanLater not being optimized out of Query Plan

2017-06-29 Thread Russell Spitzer
I've been writing some toy experimental strategies which end up adding UnaryExec nodes to the plan. For some reason though my "PlanLater" nodes end up being ignored and end up in the final physical plan. Is there anything in general that I might be missing? I'm doing my sample work on 2.1.X and

Custom Partitioning in Catalyst

2017-06-16 Thread Russell Spitzer
I've been trying to work with making Catalyst Cassandra partitioning aware. There seem to be two major blocks on this. The first is that DataSourceScanExec is unable to learn what the underlying partitioning should be from the BaseRelation it comes from. I'm currently able to get around this by

Re: Straw poll: dropping support for things like Scala 2.10

2017-03-02 Thread Russell Spitzer
+1 on removing 2.10 On Thu, Mar 2, 2017 at 8:51 AM Koert Kuipers wrote: given the issues with scala 2.10 and java 8 i am in favor of dropping scala 2.10 in next release On Sat, Feb 25, 2017 at 2:10 PM, Sean Owen wrote: I want to bring up the issue of

Re: welcoming Burak and Holden as committers

2017-01-24 Thread Russell Spitzer
Great news! Congratulations! On Tue, Jan 24, 2017 at 10:25 AM Dean Wampler wrote: > Congratulations to both of you! > > dean > > *Dean Wampler, Ph.D.* > Author: Programming Scala, 2nd Edition > , Fast Data > Architectures

Re: Anyone seeing a lot of Spark emails go to Gmail spam?

2016-11-02 Thread Russell Spitzer
I had one bounce message last week, but haven't seen anything else, I also do the skip inbox filter thing though. On Wed, Nov 2, 2016 at 10:16 AM Matei Zaharia wrote: > It might be useful to ask Apache Infra whether they have any information > on these (e.g. what do

Re: Official Stance on Not Using Spark Submit

2016-10-10 Thread Russell Spitzer
I actually had not seen SparkLauncher before, that looks pretty great :) On Mon, Oct 10, 2016 at 10:17 AM Russell Spitzer <russell.spit...@gmail.com> wrote: > I'm definitely only talking about non-embedded uses here as I also use > embedded Spark (cassandra, and kafka)

Re: Official Stance on Not Using Spark Submit

2016-10-10 Thread Russell Spitzer
own. I think that's consistent > with all the JIRA discussions I have seen over time. > > > On Mon, Oct 10, 2016, 17:33 Russell Spitzer <russell.spit...@gmail.com> > wrote: > > I've seen a variety of users attempting to work around using Spark Submit > with at best middling lev

Re: spark cassandra issue

2016-09-04 Thread Russell Spitzer
This would also be a better question for the SCC user list :) https://groups.google.com/a/lists.datastax.com/forum/#!forum/spark-connector-user On Sun, Sep 4, 2016 at 9:31 AM Russell Spitzer <russell.spit...@gmail.com> wrote: > > https://github.com/datastax/spark-cassandra-connector

Re: spark cassandra issue

2016-09-04 Thread Russell Spitzer
https://github.com/datastax/spark-cassandra-connector/blob/v1.3.1/doc/14_data_frames.md In Spark 1.3 it was illegal to use "table" as a key in Spark SQL so in that version of Spark the connector needed to use the option "c_table" val df = sqlContext.read. |

Re: Insert non-null values from dataframe

2016-08-26 Thread Russell Spitzer
Cassandra does not differentiate between null and empty, so when reading from C* all empty values are reported as null. To avoid inserting nulls (avoiding tombstones) see https://github.com/datastax/spark-cassandra-connector/blob/master/doc/5_saving.md#globally-treating-all-nulls-as-unset This

Re: Dataframes: PrunedFilteredScan without Spark Side Filtering

2015-10-07 Thread Russell Spitzer
Should I make up a new ticket for this? Or is there something already underway? On Mon, Oct 5, 2015 at 4:31 PM Russell Spitzer <russell.spit...@gmail.com> wrote: > That sounds fine to me, we already do the filtering so populating that > field would be pretty simple. > > On Sun,

Re: Dataframes: PrunedFilteredScan without Spark Side Filtering

2015-10-05 Thread Russell Spitzer
ill a > chance that this would conflict with existing methods, but hopefully that > would not be a problem in practice. > > Thoughts? > > Michael > > On Fri, Sep 25, 2015 at 10:02 PM, Russell Spitzer < > russell.spit...@gmail.com> wrote: > >> Hi! First tim

Dataframes: PrunedFilteredScan without Spark Side Filtering

2015-09-25 Thread Russell Spitzer
Hi! First time poster, long time reader. I'm wondering if there is a way to let cataylst know that it doesn't need to repeat a filter on the spark side after a filter has been applied by the Source Implementing PrunedFilterScan. This is for a usecase in which we except a filter on a