Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -57,33 +57,23 @@ public class SqlQuery5 extends NexmarkQueryTransform<AuctionCount> {
private static final String QUERY_TEMPLATE =
Joiner.on("\n\t")
.join(
" SELECT AuctionBids.auction, AuctionBids.num",
" FROM (",
" SELECT",
" B1.auction,",
" count(*) AS num,",
" HOP_START(B1.dateTime, INTERVAL '%1$d' SECOND, INTERVAL '%2$d' SECOND) AS starttime",
" FROM Bid B1 ",
" GROUP BY ",
" B1.auction,",
" HOP(B1.dateTime, INTERVAL '%1$d' SECOND, INTERVAL '%2$d' SECOND)",
" ) AS AuctionBids",
" JOIN (",
" WITH BidCount (auction, num, hopstart) AS (",
" SELECT ",
" max(CountBids.num) AS maxnum, ",
" CountBids.starttime",
" FROM (",
" SELECT",
" count(*) AS num,",
" HOP_START(B2.dateTime, INTERVAL '%1$d' SECOND, INTERVAL '%2$d' SECOND) AS starttime",
" FROM Bid B2 ",
" GROUP BY ",
" B2.auction, ",
" HOP(B2.dateTime, INTERVAL '%1$d' SECOND, INTERVAL '%2$d' SECOND)",
" ) AS CountBids",
" GROUP BY CountBids.starttime",
" ) AS MaxBids ",
" ON AuctionBids.starttime = MaxBids.starttime AND AuctionBids.num >= MaxBids.maxnum ");
" Bid.auction,",
" count(*),",
" HOP_START(Bid.dateTime, INTERVAL '%1$d' SECOND, INTERVAL '%2$d' SECOND)",
" FROM Bid ",
" GROUP BY ",
" Bid.auction,",
" HOP(Bid.dateTime, INTERVAL '%1$d' SECOND, INTERVAL '%2$d' SECOND)",
" )",
" SELECT HotAuction.auction, HotAuction.num, HotAuction.hopstart",
" FROM BidCount as HotAuction ",
" WHERE HotAuction.num >= ALL (",
" SELECT num",
" FROM BidCount",
" WHERE HotAuction.hopstart = BidCount.hopstart",
" ) " );

private final PTransform<PInput, PCollection<Row>> query;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,11 @@
import org.joda.time.Instant;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Unit tests for {@link SqlQuery0}. */
@RunWith(JUnit4.class)
public class SqlQuery0Test {

private static final Bid BID1 = new Bid(5L, 3L, 123123L, new Instant(43234234L), "extra1");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,11 @@
import org.joda.time.Instant;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Unit tests for {@link SqlQuery1}. */
@RunWith(JUnit4.class)
public class SqlQuery1Test {

private static final Bid BID1_USD = new Bid(5L, 3L, 100L, new Instant(43234234L), "extra1");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,11 @@
import org.joda.time.Instant;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Unit tests for {@link SqlQuery2}. */
@RunWith(JUnit4.class)
public class SqlQuery2Test {

private static final List<Bid> BIDS =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,11 @@
import org.joda.time.Instant;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Unit tests for {@link SqlQuery3}. */
@RunWith(JUnit4.class)
public class SqlQuery3Test {

private static final List<Person> PEOPLE =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,11 @@
import org.joda.time.Instant;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Unit tests for {@link SqlQuery5}. */
@RunWith(JUnit4.class)
public class SqlQuery5Test {

private static final NexmarkConfiguration config = new NexmarkConfiguration();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,11 @@
import org.joda.time.Instant;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Unit tests for {@link SqlQuery7}. */
@RunWith(JUnit4.class)
public class SqlQuery7Test {

private static final NexmarkConfiguration config = new NexmarkConfiguration();
Expand Down