unsubscribe
ot sure if
this is necessary to move forward though)
Thanks for all the feedback. Please let me know if there are other concerns
that we would like to resolve before enabling blacklisting.
Thanks,
Ankur
On Tue, Apr 2, 2019 at 2:45 AM Steve Loughran
wrote:
>
>
> On Fri, Mar 29, 2019
bled by default already.
On Mon, Apr 1, 2019 at 3:08 PM Chris Stevens
wrote:
> Hey Ankur,
>
> I think the significant decrease in "spark.blacklist.timeout" (1 hr down
> to 5 minutes) in your updated suggestion is the key here.
>
> Looking at a few *successful* runs of
ves lot of unnecessary computation and
also alerts admins to look for transient/permanent hardware failures.
Please let me know if you think, we should enable blacklisting feature by
default with the higher threshold.
Thanks,
Ankur
On Fri, Mar 29, 2019 at 3:23 PM Chris Stevens
wrote:
> Hey All,
>
&g
>
> On Thu, Mar 28, 2019 at 3:32 PM, Ankur Gupta <
> ankur.gu...@cloudera.com.invalid> wrote:
>
>> Hi all,
>>
>> This is a follow-on to my PR: https://github.com/apache/spark/pull/24208,
>> where I aimed to enable blacklisting for fetch failure by defaul
blacklisting timeout config*
: spark.scheduler.executorTaskBlacklistTime
Thanks,
Ankur
;re not really that
> useful for debugging. So a solution than keeps that behavior, but
> writes INFO logs to this new sink, would be great.
>
> If you can come up with a solution to those problems I think this
> could be a good feature.
>
>
> On Wed, Aug 22, 2018 at 10:01 AM
during the
shutdown phase of Spark Application.
Thanks,
Ankur
On Wed, Aug 22, 2018 at 1:36 AM Marco Gaido wrote:
> I agree with Saisai. You can also configure log4j to append anywhere else
> other than the console. Many companies have their system for collecting and
> monitoring
application end. This way the logs will be available as part of Yarn Logs.
I am also interested in hearing about other ideas that the community may
have about this. Or if someone has already solved this problem, then I
would like them to contribute their solution to the community.
Thanks,
Ankur
Adding DEV mailing list to see if this is a defect with ConnectedComponent
or if they can recommend any solution.
Thanks
Ankur
On Thu, Jan 5, 2017 at 1:10 PM, Ankur Srivastava wrote:
> Yes I did try it out and it choses the local file system as my checkpoint
> location starts with s3n://
s not very obvious
that “data” is a supposed to be huge and thus need to be PairRDD and if it
is not you do not need to use the JavaPairRDD, instead use
a Map and return a List.
If it data is huge and has to be PairRDD pull out the logic to build the
data PairRDD and then invoke map function on tha
l, though it would be worthwhile when combined with other
improvements <https://github.com/apache/spark/pull/1217>.
Ankur <http://www.ankurdave.com/>
pregel-simplify-join.patch
Description: Binary data
-
To unsubscr
semantics rely on an
implementation detail (vertex replication).
Ankur <http://www.ankurdave.com/>
On Mon, Jun 1, 2015 at 8:54 AM, Tarek Auel wrote:
> Hello,
>
> Someone proposed in a Jira issue to implement new graph operations. Sean
> Owen recommended to check first with
ten faster since then. The performance gap is
because the MLlib version implements some ALS-specific optimizations that
are hard to do using GraphX, such as storing the edges twice (partitioned
by source and by destination) to reduce communication.
Ankur <http://www.ankurdave.com/>
On Tue, M
nly reduce the
replication factor by 1.
Ankur <http://www.ankurdave.com/>
On Mon, Jan 19, 2015 at 12:20 PM, Michael Malak <
michaelma...@yahoo.com.invalid> wrote:
> Does GraphX make an effort to co-locate vertices onto the same workers as
> the majority (or even some) of its edges?
>
+1 (binding)
Ankur <http://www.ankurdave.com/>
On Wed, Nov 5, 2014 at 5:31 PM, Matei Zaharia
wrote:
> I'd like to formally call a [VOTE] on this model, to last 72 hours. The
> [VOTE] will end on Nov 8, 2014 at 6 PM PST.
>
Here is how
to do that in the Spark shell:
https://gist.github.com/ankurdave/25732a493bc8c8541c97
Ankur
-
To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
For additional commands, e-mail: dev-h...@spark.apache.org
the code checked out up
>> to commit# db56f2df1b8027171da1b8d2571d1f2ef1e103b6.
The error is because I merged a GraphX PR that introduced a nondeterministic
bug [1]. I reverted the faulty PR, but it was too late for the 1.1.0 release.
The problem should go away if you use branch-1.1 or mas
I posted the fix on the JIRA ticket
(https://issues.apache.org/jira/browse/SPARK-3190). To update the user list,
this is indeed an integer overflow problem when summing up the partition sizes.
The fix is to use Longs for the sum: https://github.com/apache/spark/pull/2106.
Ankur
the
edges. The ShippableVertexPartition class encapsulates the vertex hash map
along with a RoutingTablePartition.
After joining the vertices with the edges, the edge partitions cache their
adjacent vertices in the mirror cache. They use the VertexPartition for
this, which provides only the hash map functionality and not the routing
table.
Ankur <http://www.ankurdave.com/>
=> (getTripletPartition(e), e))
.partitionBy(new HashPartitioner(numPartitions))
*.map(pair => Edge(pair._2.srcId, pair._2.dstId, pair._2.attr))*
val partitionedGraph = Graph(unpartitionedGraph.vertices, partitionedEdges)
Ankur <http://www.ankurdave.com/>
vertices, partitionedEdges)
A multipass partitioning algorithm could store its results in the edge
attribute, and then you could use the code above to do the partitioning.
Ankur <http://www.ankurdave.com/>
On Wed, Jul 23, 2014 at 11:59 PM, Larry Xiao wrote:
> Hi all,
>
> I'm imp
sist() and
graph.edges.unpersist().
By the way, the memory leak bug with Pregel (SPARK-2025
<https://issues.apache.org/jira/browse/SPARK-2025>) is fixed in master.
Ankur <http://www.ankurdave.com/>
You can achieve an equivalent effect by calling rdd.foreach(x => {}), which
is the lightest possible action that forces materialization of the whole
RDD.
Ankur <http://www.ankurdave.com/>
I agree, let's go ahead and remove it.
Ankur <http://www.ankurdave.com/>
0
OK, I withdraw my downvote.
Ankur <http://www.ankurdave.com/>
This is probably due to
SPARK-1931<https://issues.apache.org/jira/browse/SPARK-1931>,
which I just fixed in PR #885 <https://github.com/apache/spark/pull/885>.
Is the problem resolved if you use the current Spark master?
Ankur <http://www.ankurdave.com/>
29 matches
Mail list logo