mirror of https://github.com/apache/druid.git
MSQ window functions: Minor cleanup for empty over clause related flows + Exhaustive tests (#16754)
* MSQ window functions: Revamp logic to create separate window stages when empty over() clause is present * Fix tests * Revert changes of creating separate stages for empty over clause * Address review comments
This commit is contained in:
parent
8b8ca0d7fc
commit
c45d4fdbca
|
@ -84,7 +84,6 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
private Supplier<ResultRow> rowSupplierFromFrameCursor;
|
||||
private ResultRow outputRow = null;
|
||||
private FrameWriter frameWriter = null;
|
||||
private final boolean isOverEmpty;
|
||||
|
||||
public WindowOperatorQueryFrameProcessor(
|
||||
WindowOperatorQuery query,
|
||||
|
@ -95,7 +94,6 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
ObjectMapper jsonMapper,
|
||||
final List<OperatorFactory> operatorFactoryList,
|
||||
final RowSignature rowSignature,
|
||||
final boolean isOverEmpty,
|
||||
final int maxRowsMaterializedInWindow,
|
||||
final List<String> partitionColumnNames
|
||||
)
|
||||
|
@ -110,7 +108,6 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
this.frameRowsAndCols = new ArrayList<>();
|
||||
this.resultRowAndCols = new ArrayList<>();
|
||||
this.objectsOfASingleRac = new ArrayList<>();
|
||||
this.isOverEmpty = isOverEmpty;
|
||||
this.maxRowsMaterialized = maxRowsMaterializedInWindow;
|
||||
this.partitionColumnNames = partitionColumnNames;
|
||||
}
|
||||
|
@ -162,7 +159,7 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
*
|
||||
*
|
||||
* The flow would look like:
|
||||
* 1. Validate if the operator has an empty OVER clause
|
||||
* 1. Validate if the operator doesn't have any OVER() clause with PARTITION BY for this stage.
|
||||
* 2. If 1 is true make a giant rows and columns (R&C) using concat as shown above
|
||||
* Let all operators run amok on that R&C
|
||||
* 3. If 1 is false
|
||||
|
@ -187,14 +184,12 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
* We might think to reimplement them in the MSQ way so that we do not have to materialize so much data
|
||||
*/
|
||||
|
||||
// Phase 1 of the execution
|
||||
// eagerly validate presence of empty OVER() clause
|
||||
if (isOverEmpty) {
|
||||
// if OVER() found
|
||||
// have to bring all data to a single executor for processing
|
||||
// convert each frame to rac
|
||||
// concat all the racs to make a giant rac
|
||||
// let all operators run on the giant rac when channel is finished
|
||||
if (partitionColumnNames.isEmpty()) {
|
||||
// If we do not have any OVER() clause with PARTITION BY for this stage.
|
||||
// Bring all data to a single executor for processing.
|
||||
// Convert each frame to RAC.
|
||||
// Concatenate all the racs to make a giant RAC.
|
||||
// Let all operators run on the giant RAC until channel is finished.
|
||||
if (inputChannel.canRead()) {
|
||||
final Frame frame = inputChannel.read();
|
||||
convertRowFrameToRowsAndColumns(frame);
|
||||
|
@ -484,7 +479,7 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
|
||||
/**
|
||||
* Compare two rows based on the columns in partitionColumnNames.
|
||||
* If the partitionColumnNames is empty or null, compare entire row.
|
||||
* If the partitionColumnNames is empty, the method will end up returning true.
|
||||
* <p>
|
||||
* For example, say:
|
||||
* <ul>
|
||||
|
@ -501,9 +496,6 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
*/
|
||||
private boolean comparePartitionKeys(ResultRow row1, ResultRow row2, List<String> partitionColumnNames)
|
||||
{
|
||||
if (partitionColumnNames == null || partitionColumnNames.isEmpty()) {
|
||||
return row1.equals(row2);
|
||||
} else {
|
||||
int match = 0;
|
||||
for (String columnName : partitionColumnNames) {
|
||||
int i = frameReader.signature().indexOf(columnName);
|
||||
|
@ -514,4 +506,3 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
|
|||
return match == partitionColumnNames.size();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Iterables;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectSortedMap;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.frame.processor.FrameProcessor;
|
||||
import org.apache.druid.frame.processor.OutputChannel;
|
||||
import org.apache.druid.frame.processor.OutputChannelFactory;
|
||||
|
@ -59,7 +60,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
private final WindowOperatorQuery query;
|
||||
private final List<OperatorFactory> operatorList;
|
||||
private final RowSignature stageRowSignature;
|
||||
private final boolean isEmptyOver;
|
||||
private final int maxRowsMaterializedInWindow;
|
||||
private final List<String> partitionColumnNames;
|
||||
|
||||
|
@ -68,7 +68,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
@JsonProperty("query") WindowOperatorQuery query,
|
||||
@JsonProperty("operatorList") List<OperatorFactory> operatorFactoryList,
|
||||
@JsonProperty("stageRowSignature") RowSignature stageRowSignature,
|
||||
@JsonProperty("emptyOver") boolean emptyOver,
|
||||
@JsonProperty("maxRowsMaterializedInWindow") int maxRowsMaterializedInWindow,
|
||||
@JsonProperty("partitionColumnNames") List<String> partitionColumnNames
|
||||
)
|
||||
|
@ -76,8 +75,11 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
this.query = Preconditions.checkNotNull(query, "query");
|
||||
this.operatorList = Preconditions.checkNotNull(operatorFactoryList, "bad operator");
|
||||
this.stageRowSignature = Preconditions.checkNotNull(stageRowSignature, "stageSignature");
|
||||
this.isEmptyOver = emptyOver;
|
||||
this.maxRowsMaterializedInWindow = maxRowsMaterializedInWindow;
|
||||
|
||||
if (partitionColumnNames == null) {
|
||||
throw DruidException.defensive("List of partition column names encountered as null.");
|
||||
}
|
||||
this.partitionColumnNames = partitionColumnNames;
|
||||
}
|
||||
|
||||
|
@ -105,12 +107,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
return stageRowSignature;
|
||||
}
|
||||
|
||||
@JsonProperty("emptyOver")
|
||||
public boolean isEmptyOverFound()
|
||||
{
|
||||
return isEmptyOver;
|
||||
}
|
||||
|
||||
@JsonProperty("maxRowsMaterializedInWindow")
|
||||
public int getMaxRowsMaterializedInWindow()
|
||||
{
|
||||
|
@ -166,7 +162,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
frameContext.jsonMapper(),
|
||||
operatorList,
|
||||
stageRowSignature,
|
||||
isEmptyOver,
|
||||
maxRowsMaterializedInWindow,
|
||||
partitionColumnNames
|
||||
);
|
||||
|
@ -190,8 +185,7 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
return false;
|
||||
}
|
||||
WindowOperatorQueryFrameProcessorFactory that = (WindowOperatorQueryFrameProcessorFactory) o;
|
||||
return isEmptyOver == that.isEmptyOver
|
||||
&& maxRowsMaterializedInWindow == that.maxRowsMaterializedInWindow
|
||||
return maxRowsMaterializedInWindow == that.maxRowsMaterializedInWindow
|
||||
&& Objects.equals(query, that.query)
|
||||
&& Objects.equals(operatorList, that.operatorList)
|
||||
&& Objects.equals(partitionColumnNames, that.partitionColumnNames)
|
||||
|
@ -201,6 +195,6 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
|
|||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(query, operatorList, partitionColumnNames, stageRowSignature, isEmptyOver, maxRowsMaterializedInWindow);
|
||||
return Objects.hash(query, operatorList, partitionColumnNames, stageRowSignature, maxRowsMaterializedInWindow);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -124,8 +125,11 @@ public class WindowOperatorQueryKit implements QueryKit<WindowOperatorQuery>
|
|||
}
|
||||
|
||||
if (isEmptyOverPresent) {
|
||||
// empty over clause found
|
||||
// moving everything to a single partition
|
||||
// Move everything to a single partition since we have to load all the data on a single worker anyway to compute empty over() clause.
|
||||
log.info(
|
||||
"Empty over clause is present in the query. Creating a single stage with all operator factories [%s].",
|
||||
queryToRun.getOperators()
|
||||
);
|
||||
queryDefBuilder.add(
|
||||
StageDefinition.builder(firstStageNumber)
|
||||
.inputs(new StageInputSpec(firstStageNumber - 1))
|
||||
|
@ -136,9 +140,8 @@ public class WindowOperatorQueryKit implements QueryKit<WindowOperatorQuery>
|
|||
queryToRun,
|
||||
queryToRun.getOperators(),
|
||||
rowSignature,
|
||||
true,
|
||||
maxRowsMaterialized,
|
||||
new ArrayList<>()
|
||||
Collections.emptyList()
|
||||
))
|
||||
);
|
||||
} else {
|
||||
|
@ -237,7 +240,6 @@ public class WindowOperatorQueryKit implements QueryKit<WindowOperatorQuery>
|
|||
queryToRun,
|
||||
operatorList.get(i),
|
||||
stageRowSignature,
|
||||
false,
|
||||
maxRowsMaterialized,
|
||||
partitionColumnNames
|
||||
))
|
||||
|
@ -257,20 +259,34 @@ public class WindowOperatorQueryKit implements QueryKit<WindowOperatorQuery>
|
|||
{
|
||||
List<List<OperatorFactory>> operatorList = new ArrayList<>();
|
||||
final List<OperatorFactory> operators = originalQuery.getOperators();
|
||||
List<OperatorFactory> operatorFactoryList = new ArrayList<>();
|
||||
for (OperatorFactory of : operators) {
|
||||
operatorFactoryList.add(of);
|
||||
List<OperatorFactory> currentStage = new ArrayList<>();
|
||||
|
||||
for (int i = 0; i < operators.size(); i++) {
|
||||
OperatorFactory of = operators.get(i);
|
||||
currentStage.add(of);
|
||||
|
||||
if (of instanceof WindowOperatorFactory) {
|
||||
operatorList.add(operatorFactoryList);
|
||||
operatorFactoryList = new ArrayList<>();
|
||||
} else if (of instanceof NaivePartitioningOperatorFactory) {
|
||||
if (((NaivePartitioningOperatorFactory) of).getPartitionColumns().isEmpty()) {
|
||||
operatorList.clear();
|
||||
operatorList.add(originalQuery.getOperators());
|
||||
return operatorList;
|
||||
// Process consecutive window operators
|
||||
while (i + 1 < operators.size() && operators.get(i + 1) instanceof WindowOperatorFactory) {
|
||||
i++;
|
||||
currentStage.add(operators.get(i));
|
||||
}
|
||||
|
||||
// Finalize the current stage
|
||||
operatorList.add(new ArrayList<>(currentStage));
|
||||
currentStage.clear();
|
||||
}
|
||||
}
|
||||
|
||||
// There shouldn't be any operators left in currentStage. The last operator should always be WindowOperatorFactory.
|
||||
if (!currentStage.isEmpty()) {
|
||||
throw new ISE(
|
||||
"Found unexpected operators [%s] present in the list of operators [%s].",
|
||||
currentStage,
|
||||
operators
|
||||
);
|
||||
}
|
||||
|
||||
return operatorList;
|
||||
}
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@ public class WindowOperatorQueryFrameProcessorFactoryTest
|
|||
public void testEqualsAndHashcode()
|
||||
{
|
||||
EqualsVerifier.forClass(WindowOperatorQueryFrameProcessorFactory.class)
|
||||
.withNonnullFields("query", "operatorList", "stageRowSignature", "isEmptyOver", "maxRowsMaterializedInWindow", "partitionColumnNames")
|
||||
.withNonnullFields("query", "operatorList", "stageRowSignature", "maxRowsMaterializedInWindow", "partitionColumnNames")
|
||||
.usingGetClass()
|
||||
.verify();
|
||||
}
|
||||
|
|
|
@ -1211,6 +1211,11 @@ public class BaseCalciteQueryTest extends CalciteTestBase
|
|||
skipVectorize = true;
|
||||
}
|
||||
|
||||
protected void sqlNativeIncompatible()
|
||||
{
|
||||
assumeTrue(testBuilder().config.isRunningMSQ(), "test case is not SQL native compatible");
|
||||
}
|
||||
|
||||
protected void msqIncompatible()
|
||||
{
|
||||
assumeFalse(testBuilder().config.isRunningMSQ(), "test case is not MSQ compatible");
|
||||
|
|
|
@ -7607,4 +7607,147 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
|
|||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/empty_over_clause/single_empty_over_1")
|
||||
@Test
|
||||
public void test_empty_over_single_empty_over_1()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/empty_over_clause/single_empty_over_2")
|
||||
@Test
|
||||
public void test_empty_over_single_empty_over_2()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/empty_over_clause/multiple_empty_over_1")
|
||||
@Test
|
||||
public void test_empty_over_multiple_empty_over_1()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_sorting/single_over_1")
|
||||
@Test
|
||||
public void test_over_clause_with_only_sorting_single_over_1()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_sorting/single_over_2")
|
||||
@Test
|
||||
public void test_over_clause_with_only_sorting_single_over_2()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_sorting/single_over_multiple_sort_columns")
|
||||
@Test
|
||||
public void test_over_clause_with_only_sorting_single_over_multiple_sort_columns()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_sorting/multiple_over_same_sort_column")
|
||||
@Test
|
||||
public void test_over_clause_with_only_sorting_multiple_over_same_sort_column()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_sorting/multiple_over_different_sort_column")
|
||||
@Test
|
||||
public void test_over_clause_with_only_sorting_multiple_over_different_sort_column()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_1")
|
||||
@Test
|
||||
public void test_over_clause_with_only_sorting_multiple_over_multiple_sort_columns_1()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_sorting/multiple_over_multiple_sort_columns_2")
|
||||
@Test
|
||||
public void test_over_clause_with_only_sorting_multiple_over_multiple_sort_columns_2()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_partitioning/single_over_1")
|
||||
@Test
|
||||
public void test_over_clause_with_only_partitioning_single_over_1()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_partitioning/single_over_2")
|
||||
@Test
|
||||
public void test_over_clause_with_only_partitioning_single_over_2()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_partitioning/single_over_multiple_partition_columns")
|
||||
@Test
|
||||
public void test_over_clause_with_only_partitioning_single_over_multiple_partition_columns()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_partitioning/multiple_over_same_partition_column")
|
||||
@Test
|
||||
public void test_over_clause_with_only_partitioning_multiple_over_same_partition_column()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_partitioning/multiple_over_different_partition_column")
|
||||
@Test
|
||||
public void test_over_clause_with_only_partitioning_multiple_over_different_partition_column()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_1")
|
||||
@Test
|
||||
public void test_over_clause_with_only_partitioning_multiple_over_multiple_partition_columns_1()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/over_clause_only_partitioning/multiple_over_multiple_partition_columns_2")
|
||||
@Test
|
||||
public void test_over_clause_with_only_partitioning_multiple_over_multiple_partition_columns_2()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
// This test gives the following error on sql-native engine:
|
||||
// Column[w0] of type[class org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn] cannot be sorted.
|
||||
@DrillTest("druid_queries/empty_and_non_empty_over/wikipedia_query_1")
|
||||
@Test
|
||||
public void test_empty_and_non_empty_over_wikipedia_query_1()
|
||||
{
|
||||
sqlNativeIncompatible();
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/empty_and_non_empty_over/wikipedia_query_2")
|
||||
@Test
|
||||
public void test_empty_and_non_empty_over_wikipedia_query_2()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
|
||||
@DrillTest("druid_queries/empty_and_non_empty_over/wikipedia_query_3")
|
||||
@Test
|
||||
public void test_empty_and_non_empty_over_wikipedia_query_3()
|
||||
{
|
||||
windowQueryTest();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,14 @@
|
|||
Austria null #de.wikipedia 0 1 1 1
|
||||
Austria null #de.wikipedia 14 2 2 2
|
||||
Austria null #de.wikipedia 94 3 3 3
|
||||
Austria null #de.wikipedia 272 4 4 4
|
||||
Austria null #de.wikipedia 4685 5 5 5
|
||||
Austria null #de.wikipedia 6979 6 6 6
|
||||
Guatemala null #es.wikipedia 0 12 1 1
|
||||
Guatemala El Salvador #es.wikipedia 1 13 1 1
|
||||
Guatemala Guatemala City #es.wikipedia 173 14 1 1
|
||||
Austria Horsching #de.wikipedia 0 7 1 1
|
||||
Austria Vienna #de.wikipedia 0 8 1 1
|
||||
Austria Vienna #de.wikipedia 72 9 2 2
|
||||
Austria Vienna #es.wikipedia 0 10 3 1
|
||||
Austria Vienna #tr.wikipedia 93 11 4 1
|
|
@ -0,0 +1,8 @@
|
|||
select
|
||||
countryName, cityName, channel, added,
|
||||
row_number() over () as c1,
|
||||
row_number() over (PARTITION BY countryName, cityName) as c2,
|
||||
row_number() over (PARTITION BY cityName, channel ORDER BY channel) as c3
|
||||
from wikipedia
|
||||
where countryName in ('Guatemala', 'Austria')
|
||||
group by countryName, cityName, channel, added
|
|
@ -0,0 +1,14 @@
|
|||
Austria null #de.wikipedia 1 null null
|
||||
Guatemala null #es.wikipedia 10 null null
|
||||
Austria null #de.wikipedia 2 null null
|
||||
Austria null #de.wikipedia 3 null null
|
||||
Austria null #de.wikipedia 4 null null
|
||||
Austria null #de.wikipedia 5 null null
|
||||
Austria null #de.wikipedia 6 null null
|
||||
Guatemala El Salvador #es.wikipedia 11 null null
|
||||
Guatemala Guatemala City #es.wikipedia 12 null null
|
||||
Austria Horsching #de.wikipedia 7 null null
|
||||
Austria Vienna #de.wikipedia 8 null Vienna
|
||||
Austria Vienna #es.wikipedia 13 Vienna null
|
||||
Austria Vienna #de.wikipedia 9 Vienna null
|
||||
Austria Vienna #tr.wikipedia 14 Vienna null
|
|
@ -0,0 +1,8 @@
|
|||
select
|
||||
countryName, cityName, channel,
|
||||
row_number() over () as c1,
|
||||
lag(cityName) over (PARTITION BY countryName, cityName) as c2,
|
||||
lead(cityName) over (PARTITION BY cityName, added) as c3
|
||||
from wikipedia
|
||||
where countryName in ('Guatemala', 'Austria')
|
||||
group by countryName, cityName, channel, added
|
|
@ -0,0 +1,16 @@
|
|||
Republic of Korea Seoul 0 0 16 4 2 null 1 2 0 0 1 2 13 null 12 3 1 2 13
|
||||
Republic of Korea Seoul 2 4 16 5 3 0 2 3 1 1 2 3 13 Austria 11 3 2 2 12
|
||||
Republic of Korea Seoul 4 4 16 6 4 2 3 4 2 2 2 3 12 Republic of Korea 10 3 3 2 11
|
||||
Republic of Korea Seoul 5 5 16 7 5 4 4 5 3 3 2 3 11 Republic of Korea 9 3 4 2 10
|
||||
Republic of Korea Seoul 6 6 16 8 6 5 5 6 4 3 2 3 10 Republic of Korea 8 3 5 2 9
|
||||
Republic of Korea Seoul 12 12 16 9 7 6 6 7 5 3 2 3 9 Republic of Korea 7 3 6 2 8
|
||||
Republic of Korea Seoul 15 15 16 10 8 12 7 8 6 3 2 3 8 Republic of Korea 6 3 7 2 7
|
||||
Republic of Korea Seoul 19 19 16 11 9 15 8 9 7 3 2 3 7 Republic of Korea 5 3 8 2 6
|
||||
Republic of Korea Seoul 22 22 16 12 10 19 9 10 8 3 2 3 6 Republic of Korea 4 3 9 2 5
|
||||
Republic of Korea Seoul 24 24 16 13 11 22 10 11 9 3 2 3 5 Republic of Korea 3 3 10 2 4
|
||||
Republic of Korea Seoul 243 243 16 14 14 24 11 12 10 3 2 3 4 Austria 2 2 11 2 3
|
||||
Republic of Korea Seoul 663 663 16 15 15 243 12 13 11 3 2 3 3 Republic of Korea 1 1 12 2 2
|
||||
Republic of Korea Seoul 1036 1036 16 16 16 663 13 13 12 3 2 2 2 Republic of Korea 0 0 13 1 1
|
||||
Austria Vienna 0 0 16 1 1 1036 1 2 0 0 1 2 3 Republic of Korea 2 2 1 2 3
|
||||
Austria Vienna 72 72 16 2 12 0 2 3 1 1 2 3 3 Republic of Korea 1 1 2 2 2
|
||||
Austria Vienna 93 93 16 3 13 72 3 3 2 2 2 2 2 Austria 0 0 3 1 1
|
|
@ -0,0 +1,23 @@
|
|||
select
|
||||
countryName, cityName, added,
|
||||
sum(added),
|
||||
count(added) over () e1,
|
||||
ROW_NUMBER() over () e2,
|
||||
ROW_NUMBER() over (order by added) e3,
|
||||
lag(added) over (order by cityName, countryName) e4,
|
||||
count(*) over (partition by cityName order by countryName rows between unbounded preceding and current row) c2,
|
||||
count(*) over (partition by cityName order by countryName rows between unbounded preceding and 1 following) c3,
|
||||
count(*) over (partition by cityName order by countryName rows between unbounded preceding and 1 preceding) c4,
|
||||
count(*) over (partition by cityName order by countryName rows between 3 preceding and 1 preceding) c5,
|
||||
count(*) over (partition by cityName order by countryName rows between 1 preceding and current row) c5,
|
||||
count(*) over (partition by cityName order by countryName rows between 1 preceding and 1 FOLLOWING) c7,
|
||||
count(*) over (partition by cityName order by countryName rows between 1 preceding and unbounded FOLLOWING) c5,
|
||||
lag(countryName) over (order by added) e5,
|
||||
count(*) over (partition by cityName order by countryName rows between 1 FOLLOWING and unbounded FOLLOWING) c5,
|
||||
count(*) over (partition by cityName order by countryName rows between 1 FOLLOWING and 3 FOLLOWING) c10,
|
||||
ROW_NUMBER() over (partition by cityName order by added) e6,
|
||||
count(*) over (partition by cityName order by countryName rows between current row and 1 following) c11,
|
||||
count(*) over (partition by cityName order by countryName rows between current row and unbounded following) c12
|
||||
from wikipedia
|
||||
where cityName in ('Vienna', 'Seoul')
|
||||
group by countryName, cityName, added
|
|
@ -0,0 +1,13 @@
|
|||
Austria 1 null
|
||||
Austria 2 Austria
|
||||
Austria 3 Austria
|
||||
Republic of Korea 4 Austria
|
||||
Austria 5 Republic of Korea
|
||||
Republic of Korea 6 Austria
|
||||
Republic of Korea 7 Republic of Korea
|
||||
Republic of Korea 8 Republic of Korea
|
||||
Republic of Korea 9 Republic of Korea
|
||||
Republic of Korea 10 Republic of Korea
|
||||
Republic of Korea 11 Republic of Korea
|
||||
Republic of Korea 12 Republic of Korea
|
||||
Austria 13 Republic of Korea
|
|
@ -0,0 +1,7 @@
|
|||
select
|
||||
countryName,
|
||||
row_number() over () as c1,
|
||||
lag(countryName) over () as c2
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria 1
|
||||
Austria 2
|
||||
Austria 3
|
||||
Republic of Korea 4
|
||||
Austria 5
|
||||
Republic of Korea 6
|
||||
Republic of Korea 7
|
||||
Republic of Korea 8
|
||||
Republic of Korea 9
|
||||
Republic of Korea 10
|
||||
Republic of Korea 11
|
||||
Republic of Korea 12
|
||||
Austria 13
|
|
@ -0,0 +1,4 @@
|
|||
select countryName, row_number() over () as c1
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria null
|
||||
Austria Austria
|
||||
Austria Austria
|
||||
Republic of Korea Austria
|
||||
Austria Republic of Korea
|
||||
Republic of Korea Austria
|
||||
Republic of Korea Republic of Korea
|
||||
Republic of Korea Republic of Korea
|
||||
Republic of Korea Republic of Korea
|
||||
Republic of Korea Republic of Korea
|
||||
Republic of Korea Republic of Korea
|
||||
Republic of Korea Republic of Korea
|
||||
Austria Republic of Korea
|
|
@ -0,0 +1,4 @@
|
|||
select countryName, lag(countryName) over () as c1
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria null #de.wikipedia 1 null
|
||||
Austria Horsching #de.wikipedia 2 null
|
||||
Austria Vienna #de.wikipedia 3 Horsching
|
||||
Austria Vienna #es.wikipedia 1 Vienna
|
||||
Austria Vienna #tr.wikipedia 1 Vienna
|
||||
Republic of Korea null #en.wikipedia 1 null
|
||||
Republic of Korea null #ja.wikipedia 1 null
|
||||
Republic of Korea null #ko.wikipedia 1 null
|
||||
Republic of Korea Jeonju #ko.wikipedia 2 null
|
||||
Republic of Korea Seongnam-si #ko.wikipedia 3 Jeonju
|
||||
Republic of Korea Seoul #ko.wikipedia 4 Seongnam-si
|
||||
Republic of Korea Suwon-si #ko.wikipedia 5 Seoul
|
||||
Republic of Korea Yongsan-dong #ko.wikipedia 6 Suwon-si
|
|
@ -0,0 +1,7 @@
|
|||
select
|
||||
countryName, cityName, channel,
|
||||
row_number() over (PARTITION BY channel) as c1,
|
||||
lag(cityName) over (PARTITION BY countryName) as c2
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,9 @@
|
|||
Austria null 1 null
|
||||
Austria Horsching 1 null
|
||||
Austria Vienna 1 null
|
||||
Republic of Korea null 1 null
|
||||
Republic of Korea Jeonju 1 null
|
||||
Republic of Korea Seongnam-si 1 null
|
||||
Republic of Korea Seoul 1 null
|
||||
Republic of Korea Suwon-si 1 null
|
||||
Republic of Korea Yongsan-dong 1 null
|
|
@ -0,0 +1,7 @@
|
|||
select
|
||||
countryName, cityName,
|
||||
row_number() over (PARTITION BY countryName, cityName) as c1,
|
||||
lag(cityName) over (PARTITION BY cityName, countryName) as c2
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName
|
|
@ -0,0 +1,74 @@
|
|||
Austria null #de.wikipedia 0 1 null
|
||||
Austria Horsching #de.wikipedia 0 1 null
|
||||
Austria Vienna #de.wikipedia 0 1 Horsching
|
||||
Austria null #de.wikipedia 14 2 null
|
||||
Austria Vienna #de.wikipedia 72 2 null
|
||||
Austria null #de.wikipedia 94 3 null
|
||||
Austria null #de.wikipedia 272 4 null
|
||||
Austria null #de.wikipedia 4685 5 null
|
||||
Austria null #de.wikipedia 6979 6 null
|
||||
Republic of Korea null #en.wikipedia 0 1 null
|
||||
Republic of Korea null #en.wikipedia 5 2 null
|
||||
Republic of Korea null #en.wikipedia 75 3 null
|
||||
Austria Vienna #es.wikipedia 0 3 null
|
||||
Republic of Korea null #ja.wikipedia 0 4 null
|
||||
Republic of Korea null #ja.wikipedia 43 5 null
|
||||
Republic of Korea null #ja.wikipedia 46 6 null
|
||||
Republic of Korea null #ja.wikipedia 356 7 null
|
||||
Republic of Korea null #ko.wikipedia 0 8 null
|
||||
Republic of Korea Seoul #ko.wikipedia 0 1 null
|
||||
Republic of Korea Yongsan-dong #ko.wikipedia 0 1 Seoul
|
||||
Republic of Korea null #ko.wikipedia 1 9 null
|
||||
Republic of Korea Jeonju #ko.wikipedia 1 1 null
|
||||
Republic of Korea null #ko.wikipedia 2 10 null
|
||||
Republic of Korea Seoul #ko.wikipedia 2 2 null
|
||||
Republic of Korea null #ko.wikipedia 3 11 null
|
||||
Republic of Korea null #ko.wikipedia 4 12 null
|
||||
Republic of Korea Seoul #ko.wikipedia 4 3 null
|
||||
Republic of Korea Seoul #ko.wikipedia 5 4 null
|
||||
Republic of Korea Seoul #ko.wikipedia 6 5 null
|
||||
Republic of Korea null #ko.wikipedia 8 13 null
|
||||
Republic of Korea null #ko.wikipedia 11 14 null
|
||||
Republic of Korea Seoul #ko.wikipedia 12 6 null
|
||||
Republic of Korea null #ko.wikipedia 13 15 null
|
||||
Republic of Korea null #ko.wikipedia 14 16 null
|
||||
Republic of Korea Seoul #ko.wikipedia 15 7 null
|
||||
Republic of Korea null #ko.wikipedia 19 17 null
|
||||
Republic of Korea Seoul #ko.wikipedia 19 8 null
|
||||
Republic of Korea null #ko.wikipedia 22 18 null
|
||||
Republic of Korea Seoul #ko.wikipedia 22 9 null
|
||||
Republic of Korea null #ko.wikipedia 23 19 null
|
||||
Republic of Korea null #ko.wikipedia 24 20 null
|
||||
Republic of Korea Seoul #ko.wikipedia 24 10 null
|
||||
Republic of Korea null #ko.wikipedia 26 21 null
|
||||
Republic of Korea null #ko.wikipedia 29 22 null
|
||||
Republic of Korea null #ko.wikipedia 30 23 null
|
||||
Republic of Korea null #ko.wikipedia 34 24 null
|
||||
Republic of Korea Suwon-si #ko.wikipedia 40 1 null
|
||||
Republic of Korea null #ko.wikipedia 41 25 null
|
||||
Republic of Korea null #ko.wikipedia 42 26 null
|
||||
Republic of Korea null #ko.wikipedia 51 27 null
|
||||
Republic of Korea null #ko.wikipedia 52 28 null
|
||||
Republic of Korea null #ko.wikipedia 56 29 null
|
||||
Republic of Korea null #ko.wikipedia 59 30 null
|
||||
Republic of Korea null #ko.wikipedia 62 31 null
|
||||
Republic of Korea null #ko.wikipedia 65 32 null
|
||||
Republic of Korea null #ko.wikipedia 73 33 null
|
||||
Republic of Korea null #ko.wikipedia 159 34 null
|
||||
Republic of Korea Seoul #ko.wikipedia 243 11 null
|
||||
Republic of Korea Seongnam-si #ko.wikipedia 254 1 null
|
||||
Republic of Korea null #ko.wikipedia 357 35 null
|
||||
Republic of Korea null #ko.wikipedia 390 36 null
|
||||
Republic of Korea null #ko.wikipedia 400 37 null
|
||||
Republic of Korea null #ko.wikipedia 495 38 null
|
||||
Republic of Korea null #ko.wikipedia 524 39 null
|
||||
Republic of Korea null #ko.wikipedia 566 40 null
|
||||
Republic of Korea Seoul #ko.wikipedia 663 12 null
|
||||
Republic of Korea null #ko.wikipedia 757 41 null
|
||||
Republic of Korea null #ko.wikipedia 827 42 null
|
||||
Republic of Korea null #ko.wikipedia 832 43 null
|
||||
Republic of Korea null #ko.wikipedia 1006 44 null
|
||||
Republic of Korea Seoul #ko.wikipedia 1036 13 null
|
||||
Republic of Korea null #ko.wikipedia 2539 45 null
|
||||
Republic of Korea Suwon-si #ko.wikipedia 3234 2 null
|
||||
Austria Vienna #tr.wikipedia 93 4 null
|
|
@ -0,0 +1,7 @@
|
|||
select
|
||||
countryName, cityName, channel, added,
|
||||
row_number() over (PARTITION BY countryName, cityName) as c1,
|
||||
lag(cityName) over (PARTITION BY channel, added) as c2
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel, added
|
|
@ -0,0 +1,13 @@
|
|||
Austria null #de.wikipedia 1 null
|
||||
Austria Horsching #de.wikipedia 2 null
|
||||
Austria Vienna #de.wikipedia 3 Horsching
|
||||
Republic of Korea null #en.wikipedia 1 null
|
||||
Austria Vienna #es.wikipedia 1 null
|
||||
Republic of Korea null #ja.wikipedia 1 null
|
||||
Republic of Korea null #ko.wikipedia 1 null
|
||||
Republic of Korea Jeonju #ko.wikipedia 2 null
|
||||
Republic of Korea Seongnam-si #ko.wikipedia 3 Jeonju
|
||||
Republic of Korea Seoul #ko.wikipedia 4 Seongnam-si
|
||||
Republic of Korea Suwon-si #ko.wikipedia 5 Seoul
|
||||
Republic of Korea Yongsan-dong #ko.wikipedia 6 Suwon-si
|
||||
Austria Vienna #tr.wikipedia 1 null
|
|
@ -0,0 +1,7 @@
|
|||
select
|
||||
countryName, cityName, channel,
|
||||
row_number() over (PARTITION BY channel) as c1,
|
||||
lag(cityName) over (PARTITION BY channel) as c2
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria null 1
|
||||
Republic of Korea null 2
|
||||
Republic of Korea null 3
|
||||
Republic of Korea null 4
|
||||
Austria Horsching 1
|
||||
Republic of Korea Jeonju 1
|
||||
Republic of Korea Seongnam-si 1
|
||||
Republic of Korea Seoul 1
|
||||
Republic of Korea Suwon-si 1
|
||||
Austria Vienna 1
|
||||
Austria Vienna 2
|
||||
Austria Vienna 3
|
||||
Republic of Korea Yongsan-dong 1
|
|
@ -0,0 +1,6 @@
|
|||
select
|
||||
countryName, cityName,
|
||||
row_number() over (PARTITION BY cityName) as c1
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria null null
|
||||
Republic of Korea null null
|
||||
Republic of Korea null null
|
||||
Republic of Korea null null
|
||||
Austria Horsching null
|
||||
Republic of Korea Jeonju null
|
||||
Republic of Korea Seongnam-si null
|
||||
Republic of Korea Seoul null
|
||||
Republic of Korea Suwon-si null
|
||||
Austria Vienna null
|
||||
Austria Vienna Vienna
|
||||
Austria Vienna Vienna
|
||||
Republic of Korea Yongsan-dong null
|
|
@ -0,0 +1,6 @@
|
|||
select
|
||||
countryName, cityName,
|
||||
lag(cityName) over (PARTITION BY cityName) as c1
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria null #de.wikipedia null
|
||||
Austria Horsching #de.wikipedia null
|
||||
Austria Vienna #de.wikipedia Horsching
|
||||
Austria Vienna #es.wikipedia null
|
||||
Austria Vienna #tr.wikipedia null
|
||||
Republic of Korea null #en.wikipedia null
|
||||
Republic of Korea null #ja.wikipedia null
|
||||
Republic of Korea null #ko.wikipedia null
|
||||
Republic of Korea Jeonju #ko.wikipedia null
|
||||
Republic of Korea Seongnam-si #ko.wikipedia Jeonju
|
||||
Republic of Korea Seoul #ko.wikipedia Seongnam-si
|
||||
Republic of Korea Suwon-si #ko.wikipedia Seoul
|
||||
Republic of Korea Yongsan-dong #ko.wikipedia Suwon-si
|
|
@ -0,0 +1,6 @@
|
|||
select
|
||||
countryName, cityName, channel,
|
||||
lag(cityName) over (PARTITION BY channel, countryName) as c1
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria null #de.wikipedia 1 null
|
||||
Austria Horsching #de.wikipedia 2 null
|
||||
Austria Vienna #de.wikipedia 3 Horsching
|
||||
Austria Vienna #es.wikipedia 5 Vienna
|
||||
Austria Vienna #tr.wikipedia 13 Vienna
|
||||
Republic of Korea null #en.wikipedia 4 Vienna
|
||||
Republic of Korea null #ja.wikipedia 6 null
|
||||
Republic of Korea null #ko.wikipedia 7 null
|
||||
Republic of Korea Jeonju #ko.wikipedia 8 null
|
||||
Republic of Korea Seongnam-si #ko.wikipedia 9 Jeonju
|
||||
Republic of Korea Seoul #ko.wikipedia 10 Seongnam-si
|
||||
Republic of Korea Suwon-si #ko.wikipedia 11 Seoul
|
||||
Republic of Korea Yongsan-dong #ko.wikipedia 12 Suwon-si
|
|
@ -0,0 +1,7 @@
|
|||
select
|
||||
countryName, cityName, channel,
|
||||
row_number() over (order by channel) as c1,
|
||||
lag(cityName) over (order by countryName) as c2
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,9 @@
|
|||
Austria null 1 null
|
||||
Republic of Korea null 4 null
|
||||
Austria Horsching 2 null
|
||||
Republic of Korea Jeonju 5 Horsching
|
||||
Republic of Korea Seongnam-si 6 Jeonju
|
||||
Republic of Korea Seoul 7 Seongnam-si
|
||||
Republic of Korea Suwon-si 8 Seoul
|
||||
Austria Vienna 3 Suwon-si
|
||||
Republic of Korea Yongsan-dong 9 Vienna
|
|
@ -0,0 +1,7 @@
|
|||
select
|
||||
countryName, cityName,
|
||||
row_number() over (order by countryName, cityName) as c1,
|
||||
lag(cityName) over (order by cityName, countryName) as c2
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName
|
|
@ -0,0 +1,74 @@
|
|||
Austria null #de.wikipedia 0 1 null
|
||||
Austria Horsching #de.wikipedia 0 7 null
|
||||
Austria Vienna #de.wikipedia 0 8 Horsching
|
||||
Austria null #de.wikipedia 14 2 Vienna
|
||||
Austria Vienna #de.wikipedia 72 9 null
|
||||
Austria null #de.wikipedia 94 3 Vienna
|
||||
Austria null #de.wikipedia 272 4 null
|
||||
Austria null #de.wikipedia 4685 5 null
|
||||
Austria null #de.wikipedia 6979 6 null
|
||||
Republic of Korea null #en.wikipedia 0 12 null
|
||||
Republic of Korea null #en.wikipedia 5 13 null
|
||||
Republic of Korea null #en.wikipedia 75 14 null
|
||||
Austria Vienna #es.wikipedia 0 10 null
|
||||
Republic of Korea null #ja.wikipedia 0 15 Vienna
|
||||
Republic of Korea null #ja.wikipedia 43 16 null
|
||||
Republic of Korea null #ja.wikipedia 46 17 null
|
||||
Republic of Korea null #ja.wikipedia 356 18 null
|
||||
Republic of Korea null #ko.wikipedia 0 19 null
|
||||
Republic of Korea Seoul #ko.wikipedia 0 59 null
|
||||
Republic of Korea Yongsan-dong #ko.wikipedia 0 74 Seoul
|
||||
Republic of Korea null #ko.wikipedia 1 20 Yongsan-dong
|
||||
Republic of Korea Jeonju #ko.wikipedia 1 57 null
|
||||
Republic of Korea null #ko.wikipedia 2 21 Jeonju
|
||||
Republic of Korea Seoul #ko.wikipedia 2 60 null
|
||||
Republic of Korea null #ko.wikipedia 3 22 Seoul
|
||||
Republic of Korea null #ko.wikipedia 4 23 null
|
||||
Republic of Korea Seoul #ko.wikipedia 4 61 null
|
||||
Republic of Korea Seoul #ko.wikipedia 5 62 Seoul
|
||||
Republic of Korea Seoul #ko.wikipedia 6 63 Seoul
|
||||
Republic of Korea null #ko.wikipedia 8 24 Seoul
|
||||
Republic of Korea null #ko.wikipedia 11 25 null
|
||||
Republic of Korea Seoul #ko.wikipedia 12 64 null
|
||||
Republic of Korea null #ko.wikipedia 13 26 Seoul
|
||||
Republic of Korea null #ko.wikipedia 14 27 null
|
||||
Republic of Korea Seoul #ko.wikipedia 15 65 null
|
||||
Republic of Korea null #ko.wikipedia 19 28 Seoul
|
||||
Republic of Korea Seoul #ko.wikipedia 19 66 null
|
||||
Republic of Korea null #ko.wikipedia 22 29 Seoul
|
||||
Republic of Korea Seoul #ko.wikipedia 22 67 null
|
||||
Republic of Korea null #ko.wikipedia 23 30 Seoul
|
||||
Republic of Korea null #ko.wikipedia 24 31 null
|
||||
Republic of Korea Seoul #ko.wikipedia 24 68 null
|
||||
Republic of Korea null #ko.wikipedia 26 32 Seoul
|
||||
Republic of Korea null #ko.wikipedia 29 33 null
|
||||
Republic of Korea null #ko.wikipedia 30 34 null
|
||||
Republic of Korea null #ko.wikipedia 34 35 null
|
||||
Republic of Korea Suwon-si #ko.wikipedia 40 72 null
|
||||
Republic of Korea null #ko.wikipedia 41 36 Suwon-si
|
||||
Republic of Korea null #ko.wikipedia 42 37 null
|
||||
Republic of Korea null #ko.wikipedia 51 38 null
|
||||
Republic of Korea null #ko.wikipedia 52 39 null
|
||||
Republic of Korea null #ko.wikipedia 56 40 null
|
||||
Republic of Korea null #ko.wikipedia 59 41 null
|
||||
Republic of Korea null #ko.wikipedia 62 42 null
|
||||
Republic of Korea null #ko.wikipedia 65 43 null
|
||||
Republic of Korea null #ko.wikipedia 73 44 null
|
||||
Republic of Korea null #ko.wikipedia 159 45 null
|
||||
Republic of Korea Seoul #ko.wikipedia 243 69 null
|
||||
Republic of Korea Seongnam-si #ko.wikipedia 254 58 Seoul
|
||||
Republic of Korea null #ko.wikipedia 357 46 Seongnam-si
|
||||
Republic of Korea null #ko.wikipedia 390 47 null
|
||||
Republic of Korea null #ko.wikipedia 400 48 null
|
||||
Republic of Korea null #ko.wikipedia 495 49 null
|
||||
Republic of Korea null #ko.wikipedia 524 50 null
|
||||
Republic of Korea null #ko.wikipedia 566 51 null
|
||||
Republic of Korea Seoul #ko.wikipedia 663 70 null
|
||||
Republic of Korea null #ko.wikipedia 757 52 Seoul
|
||||
Republic of Korea null #ko.wikipedia 827 53 null
|
||||
Republic of Korea null #ko.wikipedia 832 54 null
|
||||
Republic of Korea null #ko.wikipedia 1006 55 null
|
||||
Republic of Korea Seoul #ko.wikipedia 1036 71 null
|
||||
Republic of Korea null #ko.wikipedia 2539 56 Seoul
|
||||
Republic of Korea Suwon-si #ko.wikipedia 3234 73 null
|
||||
Austria Vienna #tr.wikipedia 93 11 Suwon-si
|
|
@ -0,0 +1,7 @@
|
|||
select
|
||||
countryName, cityName, channel, added,
|
||||
row_number() over (order by countryName, cityName) as c1,
|
||||
lag(cityName) over (order by channel, added) as c2
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel, added
|
|
@ -0,0 +1,13 @@
|
|||
Austria null #de.wikipedia 1 null
|
||||
Austria Horsching #de.wikipedia 2 null
|
||||
Austria Vienna #de.wikipedia 3 Horsching
|
||||
Republic of Korea null #en.wikipedia 4 Vienna
|
||||
Austria Vienna #es.wikipedia 5 null
|
||||
Republic of Korea null #ja.wikipedia 6 Vienna
|
||||
Republic of Korea null #ko.wikipedia 7 null
|
||||
Republic of Korea Jeonju #ko.wikipedia 8 null
|
||||
Republic of Korea Seongnam-si #ko.wikipedia 9 Jeonju
|
||||
Republic of Korea Seoul #ko.wikipedia 10 Seongnam-si
|
||||
Republic of Korea Suwon-si #ko.wikipedia 11 Seoul
|
||||
Republic of Korea Yongsan-dong #ko.wikipedia 12 Suwon-si
|
||||
Austria Vienna #tr.wikipedia 13 Yongsan-dong
|
|
@ -0,0 +1,7 @@
|
|||
select
|
||||
countryName, cityName, channel,
|
||||
row_number() over (order by channel) as c1,
|
||||
lag(cityName) over (order by channel) as c2
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria null 1
|
||||
Republic of Korea null 2
|
||||
Republic of Korea null 3
|
||||
Republic of Korea null 4
|
||||
Austria Horsching 5
|
||||
Republic of Korea Jeonju 6
|
||||
Republic of Korea Seongnam-si 7
|
||||
Republic of Korea Seoul 8
|
||||
Republic of Korea Suwon-si 9
|
||||
Austria Vienna 10
|
||||
Austria Vienna 11
|
||||
Austria Vienna 12
|
||||
Republic of Korea Yongsan-dong 13
|
|
@ -0,0 +1,6 @@
|
|||
select
|
||||
countryName, cityName,
|
||||
row_number() over (order by cityName) as c1
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria null null
|
||||
Republic of Korea null null
|
||||
Republic of Korea null null
|
||||
Republic of Korea null null
|
||||
Austria Horsching null
|
||||
Republic of Korea Jeonju Horsching
|
||||
Republic of Korea Seongnam-si Jeonju
|
||||
Republic of Korea Seoul Seongnam-si
|
||||
Republic of Korea Suwon-si Seoul
|
||||
Austria Vienna Suwon-si
|
||||
Austria Vienna Vienna
|
||||
Austria Vienna Vienna
|
||||
Republic of Korea Yongsan-dong Vienna
|
|
@ -0,0 +1,6 @@
|
|||
select
|
||||
countryName, cityName,
|
||||
lag(cityName) over (order by cityName) as c1
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
|
@ -0,0 +1,13 @@
|
|||
Austria null #de.wikipedia null
|
||||
Austria Horsching #de.wikipedia null
|
||||
Austria Vienna #de.wikipedia Horsching
|
||||
Republic of Korea null #en.wikipedia Vienna
|
||||
Austria Vienna #es.wikipedia null
|
||||
Republic of Korea null #ja.wikipedia Vienna
|
||||
Republic of Korea null #ko.wikipedia null
|
||||
Republic of Korea Jeonju #ko.wikipedia null
|
||||
Republic of Korea Seongnam-si #ko.wikipedia Jeonju
|
||||
Republic of Korea Seoul #ko.wikipedia Seongnam-si
|
||||
Republic of Korea Suwon-si #ko.wikipedia Seoul
|
||||
Republic of Korea Yongsan-dong #ko.wikipedia Suwon-si
|
||||
Austria Vienna #tr.wikipedia Yongsan-dong
|
|
@ -0,0 +1,6 @@
|
|||
select
|
||||
countryName, cityName, channel,
|
||||
lag(cityName) over (order by channel, countryName) as c1
|
||||
from wikipedia
|
||||
where countryName in ('Austria', 'Republic of Korea')
|
||||
group by countryName, cityName, channel
|
Loading…
Reference in New Issue