diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md
index cf7f0ab6e15d7..a71c774f328ee 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1928,12 +1928,134 @@ Additional details on supported joins:
- As of Spark 2.4, you can use joins only when the query is in Append output mode. Other output modes are not yet supported.
-- As of Spark 2.4, you cannot use other non-map-like operations before joins. Here are a few examples of
- what cannot be used.
+- You cannot use mapGroupsWithState and flatMapGroupsWithState before and after joins.
- - Cannot use streaming aggregations before joins.
+In append output mode, you can construct a query having non-map-like operations e.g. aggregation, deduplication, stream-stream join before/after join.
- - Cannot use mapGroupsWithState and flatMapGroupsWithState in Update mode before joins.
+For example, here's an example of time window aggregation in both streams followed by stream-stream join with event time window:
+
+
+
+
+{% highlight scala %}
+
+val clicksWindow = clicksWithWatermark
+ .groupBy(window("clickTime", "1 hour"))
+ .count()
+
+val impressionsWindow = impressionsWithWatermark
+ .groupBy(window("impressionTime", "1 hour"))
+ .count()
+
+clicksWindow.join(impressionsWindow, "window", "inner")
+
+{% endhighlight %}
+
+
+
+
+{% highlight java %}
+
+Dataset clicksWindow = clicksWithWatermark
+ .groupBy(functions.window(clicksWithWatermark.col("clickTime"), "1 hour"))
+ .count();
+
+Dataset impressionsWindow = impressionsWithWatermark
+ .groupBy(functions.window(impressionsWithWatermark.col("impressionTime"), "1 hour"))
+ .count();
+
+clicksWindow.join(impressionsWindow, "window", "inner");
+
+{% endhighlight %}
+
+
+
+
+
+{% highlight python %}
+clicksWindow = clicksWithWatermark.groupBy(
+ clicksWithWatermark.clickAdId,
+ window(clicksWithWatermark.clickTime, "1 hour")
+).count()
+
+impressionsWindow = impressionsWithWatermark.groupBy(
+ impressionsWithWatermark.impressionAdId,
+ window(impressionsWithWatermark.impressionTime, "1 hour")
+).count()
+
+clicksWindow.join(impressionsWindow, "window", "inner")
+
+{% endhighlight %}
+
+
+
+
+Here's another example of stream-stream join with time range join condition followed by time window aggregation:
+
+
+
+
+{% highlight scala %}
+
+val joined = impressionsWithWatermark.join(
+ clicksWithWatermark,
+ expr("""
+ clickAdId = impressionAdId AND
+ clickTime >= impressionTime AND
+ clickTime <= impressionTime + interval 1 hour
+ """),
+ joinType = "leftOuter" // can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi"
+)
+
+joined
+ .groupBy($"clickAdId", window($"clickTime", "1 hour"))
+ .count()
+
+{% endhighlight %}
+
+
+
+
+{% highlight java %}
+Dataset joined = impressionsWithWatermark.join(
+ clicksWithWatermark,
+ expr(
+ "clickAdId = impressionAdId AND " +
+ "clickTime >= impressionTime AND " +
+ "clickTime <= impressionTime + interval 1 hour "),
+ "leftOuter" // can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi"
+);
+
+joined
+ .groupBy(joined.col("clickAdId"), functions.window(joined.col("clickTime"), "1 hour"))
+ .count();
+
+{% endhighlight %}
+
+
+
+
+
+{% highlight python %}
+joined = impressionsWithWatermark.join(
+ clicksWithWatermark,
+ expr("""
+ clickAdId = impressionAdId AND
+ clickTime >= impressionTime AND
+ clickTime <= impressionTime + interval 1 hour
+ """),
+ "leftOuter" # can be "inner", "leftOuter", "rightOuter", "fullOuter", "leftSemi"
+)
+
+joined.groupBy(
+ joined.clickAdId,
+ window(joined.clickTime, "1 hour")
+).count()
+
+{% endhighlight %}
+
+
+
### Streaming Deduplication
You can deduplicate records in data streams using a unique identifier in the events. This is exactly same as deduplication on static using a unique identifier column. The query will store the necessary amount of data from previous records such that it can filter duplicate records. Similar to aggregations, you can use deduplication with or without watermarking.