Skip to content

Commit

Permalink
perf: avoid boxing in zipWithIndex
Browse files Browse the repository at this point in the history
  • Loading branch information
He-Pin committed Jan 7, 2025
1 parent 0f87351 commit 90a91d7
Show file tree
Hide file tree
Showing 9 changed files with 165 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import org.apache.pekko.stream.*;
import org.apache.pekko.stream.scaladsl.FlowSpec;
import org.apache.pekko.stream.testkit.javadsl.TestSink;
import org.apache.pekko.util.ConstantFun;
import org.apache.pekko.stream.javadsl.GraphDSL.Builder;
import org.apache.pekko.stream.stage.*;
import org.apache.pekko.testkit.PekkoSpec;
Expand Down Expand Up @@ -1689,4 +1688,41 @@ public void useFlatMapPrefixSubSource() {
.join();
Assert.assertEquals(Sets.newHashSet(1, 2), resultSet);
}

@Test
public void zipWithIndex() {
final List<Integer> input = Arrays.asList(1, 2, 3);
final List<Pair<Integer, Long>> expected =
Arrays.asList(new Pair<>(1, 0L), new Pair<>(2, 1L), new Pair<>(3, 2L));

final List<Pair<Integer, Long>> result =
Source.from(input)
.via(Flow.of(Integer.class).zipWithIndex())
.runWith(Sink.seq(), system)
.toCompletableFuture()
.join();

assertEquals(expected, result);
}

@Test
public void zipWithIndexInSubFlow() {

final Set<Pair<Integer, Long>> resultSet =
Source.range(1, 5)
.via(Flow.of(Integer.class).groupBy(2, i -> i % 2).zipWithIndex().mergeSubstreams())
.runWith(Sink.collect(Collectors.toSet()), system)
.toCompletableFuture()
.join();

Assert.assertEquals(
new HashSet<>(
Arrays.asList(
Pair.create(1, 0L),
Pair.create(3, 1L),
Pair.create(5, 2L),
Pair.create(2, 0L),
Pair.create(4, 1L))),
resultSet);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@
import org.apache.pekko.testkit.PekkoJUnitActorSystemResource;
import org.apache.pekko.testkit.PekkoSpec;
import org.apache.pekko.testkit.javadsl.TestKit;
import org.apache.pekko.util.ConstantFun;
import com.google.common.collect.Iterables;
import org.junit.Assert;
import org.junit.ClassRule;
Expand Down Expand Up @@ -1529,4 +1528,33 @@ public void useFlatMapPrefixSubSource() {
.join();
Assert.assertEquals(Sets.newHashSet(1, 2), resultSet);
}

@Test
public void zipWithIndex() {
final List<Pair<Integer, Long>> resultList =
Source.range(1, 3).zipWithIndex().runWith(Sink.seq(), system).toCompletableFuture().join();
assertEquals(
Arrays.asList(Pair.create(1, 0L), Pair.create(2, 1L), Pair.create(3, 2L)), resultList);
}

@Test
public void zipWithIndexOnSubSource() {
final Set<Pair<Integer, Long>> resultSet =
Source.range(1, 5)
.groupBy(2, i -> i % 2)
.zipWithIndex()
.mergeSubstreams()
.runWith(Sink.collect(Collectors.toSet()), system)
.toCompletableFuture()
.join();
Assert.assertEquals(
new HashSet<>(
Arrays.asList(
Pair.create(1, 0L),
Pair.create(3, 1L),
Pair.create(5, 2L),
Pair.create(2, 0L),
Pair.create(4, 1L))),
resultSet);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

# Should be java.lang.Long instead of java.lang.Object
ProblemFilters.exclude[IncompatibleSignatureProblem]("org.apache.pekko.stream.javadsl.SubSource.zipWithIndex")
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.pekko.stream.impl.fusing

import org.apache.pekko
import pekko.annotation.InternalApi
import pekko.japi.Pair
import pekko.stream.{ Attributes, FlowShape, Inlet, Outlet }
import pekko.stream.impl.Stages.DefaultAttributes
import pekko.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler }

/**
* INTERNAL API
*/
@InternalApi private[pekko] object ZipWithIndex extends GraphStage[FlowShape[Any, (Any, Long)]] {
val in = Inlet[Any]("ZipWithIndex.in")
val out = Outlet[(Any, Long)]("ZipWithIndex.out")
override val shape = FlowShape(in, out)
override def initialAttributes: Attributes = DefaultAttributes.zipWithIndex
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new GraphStageLogic(shape) with InHandler with OutHandler {
private var index = 0L
override def onPush(): Unit = {
push(out, (grab(in), index))
index += 1
}

override def onPull(): Unit = pull(in)
setHandlers(in, out, this)
}
}

/**
* INTERNAL API
*/
@InternalApi private[pekko] object ZipWithIndexJava extends GraphStage[FlowShape[Any, Pair[Any, Long]]] {
val in = Inlet[Any]("ZipWithIndex.in")
val out = Outlet[Pair[Any, Long]]("ZipWithIndex.out")
override val shape = FlowShape(in, out)
override def initialAttributes: Attributes = DefaultAttributes.zipWithIndex
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new GraphStageLogic(shape) with InHandler with OutHandler {
private var index = 0L
override def onPush(): Unit = {
push(out, new Pair(grab(in), index))
index += 1
}

override def onPull(): Unit = pull(in)
setHandlers(in, out, this)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import pekko.japi.Pair
import pekko.japi.function
import pekko.japi.function.Creator
import pekko.stream.{ javadsl, _ }
import pekko.stream.impl.fusing.ZipWithIndexJava
import pekko.util.ConstantFun
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
Expand Down Expand Up @@ -3691,7 +3692,7 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
* '''Cancels when''' downstream cancels
*/
def zipWithIndex: Flow[In, Pair[Out, java.lang.Long], Mat] =
new Flow(delegate.zipWithIndex.map { case (elem, index) => Pair[Out, java.lang.Long](elem, index) })
via(ZipWithIndexJava.asInstanceOf[Graph[FlowShape[Out, Pair[Out, java.lang.Long]], NotUsed]])

/**
* If the first element has not passed through this operator before the provided timeout, the stream is failed
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import pekko.japi.{ function, JavaPartialFunction, Pair }
import pekko.japi.function.Creator
import pekko.stream._
import pekko.stream.impl.{ LinearTraversalBuilder, UnfoldAsyncJava, UnfoldJava }
import pekko.stream.impl.fusing.ArraySource
import pekko.stream.impl.fusing.{ ArraySource, ZipWithIndexJava }
import pekko.util.{ unused, _ }
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
Expand Down Expand Up @@ -2173,7 +2173,8 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
* '''Cancels when''' downstream cancels
*/
def zipWithIndex: javadsl.Source[Pair[Out @uncheckedVariance, java.lang.Long], Mat] =
new Source(delegate.zipWithIndex.map { case (elem, index) => Pair[Out, java.lang.Long](elem, index) })
new Source(delegate.via(
ZipWithIndexJava.asInstanceOf[Graph[FlowShape[Out, Pair[Out, java.lang.Long]], NotUsed]]))

/**
* Shortcut for running this `Source` with a foreach procedure. The given procedure is invoked
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import pekko.annotation.ApiMayChange
import pekko.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
import pekko.japi.{ function, Pair }
import pekko.stream._
import pekko.stream.impl.fusing.ZipWithIndexJava
import pekko.util.ConstantFun
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
Expand Down Expand Up @@ -2272,7 +2273,8 @@ class SubFlow[In, Out, Mat](
* '''Cancels when''' downstream cancels
*/
def zipWithIndex: SubFlow[In, pekko.japi.Pair[Out @uncheckedVariance, java.lang.Long], Mat] =
new SubFlow(delegate.zipWithIndex.map { case (elem, index) => pekko.japi.Pair[Out, java.lang.Long](elem, index) })
new SubFlow(delegate.via(
ZipWithIndexJava.asInstanceOf[Graph[FlowShape[Out, Pair[Out, java.lang.Long]], NotUsed]]))

/**
* If the first element has not passed through this operator before the provided timeout, the stream is failed
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import pekko.annotation.ApiMayChange
import pekko.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter }
import pekko.japi.{ function, Pair }
import pekko.stream._
import pekko.stream.impl.fusing.ZipWithIndexJava
import pekko.util.ConstantFun
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
Expand Down Expand Up @@ -2246,8 +2247,9 @@ class SubSource[Out, Mat](
*
* '''Cancels when''' downstream cancels
*/
def zipWithIndex: javadsl.SubSource[pekko.japi.Pair[Out @uncheckedVariance, Long], Mat] =
new SubSource(delegate.zipWithIndex.map { case (elem, index) => pekko.japi.Pair(elem, index) })
def zipWithIndex: javadsl.SubSource[pekko.japi.Pair[Out @uncheckedVariance, java.lang.Long], Mat] =
new SubSource(delegate.via(
ZipWithIndexJava.asInstanceOf[Graph[FlowShape[Out, Pair[Out, java.lang.Long]], NotUsed]]))

/**
* If the first element has not passed through this operator before the provided timeout, the stream is failed
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3304,16 +3304,7 @@ trait FlowOps[+Out, +Mat] {
*
* '''Cancels when''' downstream cancels
*/
def zipWithIndex: Repr[(Out, Long)] = {
statefulMapConcat[(Out, Long)] { () =>
var index: Long = 0L
elem => {
val zipped = (elem, index)
index += 1
immutable.Iterable[(Out, Long)](zipped)
}
}
}
def zipWithIndex: Repr[(Out, Long)] = via(ZipWithIndex.asInstanceOf[Graph[FlowShape[Out, (Out, Long)], NotUsed]])

/**
* Interleave is a deterministic merge of the given [[Source]] with elements of this [[Flow]].
Expand Down

0 comments on commit 90a91d7

Please sign in to comment.