Return-Path: X-Original-To: apmail-flink-dev-archive@www.apache.org Delivered-To: apmail-flink-dev-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id A75B3180B9 for ; Wed, 28 Oct 2015 17:45:34 +0000 (UTC) Received: (qmail 3556 invoked by uid 500); 28 Oct 2015 17:45:29 -0000 Delivered-To: apmail-flink-dev-archive@flink.apache.org Received: (qmail 3494 invoked by uid 500); 28 Oct 2015 17:45:29 -0000 Mailing-List: contact dev-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list dev@flink.apache.org Received: (qmail 3481 invoked by uid 99); 28 Oct 2015 17:45:29 -0000 Received: from Unknown (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 28 Oct 2015 17:45:29 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id BBA7B1A2955 for ; Wed, 28 Oct 2015 17:45:28 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 2.899 X-Spam-Level: ** X-Spam-Status: No, score=2.899 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, HTML_MESSAGE=3, RCVD_IN_MSPIKE_H2=-0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd2-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=gmail.com Received: from mx1-eu-west.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id mrvglRLk9Sze for ; Wed, 28 Oct 2015 17:45:22 +0000 (UTC) Received: from mail-lb0-f171.google.com (mail-lb0-f171.google.com [209.85.217.171]) by mx1-eu-west.apache.org (ASF Mail Server at mx1-eu-west.apache.org) with ESMTPS id 7EB4F20604 for ; Wed, 28 Oct 2015 17:45:21 +0000 (UTC) Received: by lbbes7 with SMTP id es7so11354696lbb.2 for ; Wed, 28 Oct 2015 10:45:21 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=BG0tTvwP4WY5CG805Zf3DfMQdhMQS3rVBaAsvBpx/eM=; b=gIZjZjitM3O+5xRZXYp1yh9Wl4Y9M5goBqr647R/ONclL1SBPpQAwfsEHCdwfgfSeL RFBY3juLzmHXvySmG6OupGyMkotZn+jVTvYr144DW2U0RzR9+IlrEVMvT7KG8pKjxvIs xoQCyZ3gXEfs69H6FS9oJW42Gur54tqgmcsYzkct/JyNyQbGktKPuAU7qXMH/4/OF28g dOVnffPsijh95tyatg0ZuSAcWPAWaJ0xMspktw4GCm08f2BrgTvKsTQnIOsK0Nqygtc6 dh0KzBlWcMaIShjGaYWpap41s+g2eljLOlSrc6WaDX8/zwUXo22InF0Bt31gM4yDVm26 RH/A== MIME-Version: 1.0 X-Received: by 10.112.204.67 with SMTP id kw3mr23537000lbc.60.1446054320853; Wed, 28 Oct 2015 10:45:20 -0700 (PDT) Received: by 10.25.73.11 with HTTP; Wed, 28 Oct 2015 10:45:20 -0700 (PDT) In-Reply-To: <5630BC44.4030805@mailbox.org> References: <5630BC44.4030805@mailbox.org> Date: Wed, 28 Oct 2015 18:45:20 +0100 Message-ID: Subject: Re: Fast strategy for intersect From: Vasiliki Kalavri To: dev@flink.apache.org Content-Type: multipart/alternative; boundary=001a11c31b8e1ed9b005232dc270 --001a11c31b8e1ed9b005232dc270 Content-Type: text/plain; charset=UTF-8 Hi Martin, isn't finding the intersection of edges enough in this case? And assuming there are no duplicate edges, I believe a join should do the trick. Cheers, -Vasia. On 28 October 2015 at 13:15, Martin Junghanns wrote: > Hi all! > > While working on FLINK-2905, I was wondering what a good (and fast) way to > compute the intersect between two data sets (Gelly vertices in my case) > with unknown size would be. > > I came up with three ways to solve this: > > Consider two sets: > > DataSet> verticesLeft = this.getVertices(); > DataSet> verticesRight = graph.getVertices(); > > Way 1 (join) > > intersectVertices = verticesLeft.join(verticesRight) > .where(0) > .equalTo(0) > .with(new JoinFunction, /* .. * ./>() { > @Override > public Vertex join(Vertex first, Vertex second) > throws Exception { > return first; > } > }); > > Way 2 (coGroup) > > intersectVertices = verticesLeft.coGroup(verticesRight) > .where(0) > .equalTo(0) > .with(new CoGroupFunction, /* .. */ >() { > @Override > public void coGroup(Iterable> first, > Iterable> second, > Collector> out) throws Exception { > Iterator> leftIt = first.iterator(); > Iterator> rightIt = second.iterator(); > if (leftIt.hasNext() && rightIt.hasNext()) { > out.collect(leftIt.next()); > } > } > }); > > Way 3 (union + groupBy + aggregate) > > intersectVertices = verticesLeft.union(verticesRight) > .map(new MapFunction, Tuple3>() { > @Override > public Tuple3 map(Vertex vertex) > throws Exception { > return new Tuple3<>(vertex.f0, vertex.f1, 1); > } > }).withForwardedFields("f0;f1") > .groupBy(0) // vertex id > .aggregate(Aggregations.SUM, 2) > .filter(new FilterFunction>() { > @Override > public boolean filter(Tuple3 value) { > return value.f2 == 2; > } > }) > .map(new MapFunction, Vertex>() { > @Override > public Vertex map(Tuple3 vertexWithAggregate) { > return new Vertex<>(vertexWithAggregate.f0, vertexWithAggregate.f1); > } > }).withForwardedFields("f0;f1"); > > Thanks for your input. > > Best, > > Martin > > > > --001a11c31b8e1ed9b005232dc270--